From 89d2a918f018b3fb16e2693ce92e6574233daf07 Mon Sep 17 00:00:00 2001 From: zhaoxinyu Date: Wed, 24 Nov 2021 18:59:32 +0800 Subject: [PATCH 01/48] fix the txn_batch_size metric inaccuracy bug when the sink target is MQ --- cdc/sink/codec/interface.go | 30 ++++++++++++++++-------------- cdc/sink/codec/json.go | 1 + cdc/sink/mq.go | 5 +++-- 3 files changed, 20 insertions(+), 16 deletions(-) diff --git a/cdc/sink/codec/interface.go b/cdc/sink/codec/interface.go index 45d56ff1d9a..ec8ec2a94d5 100644 --- a/cdc/sink/codec/interface.go +++ b/cdc/sink/codec/interface.go @@ -57,13 +57,14 @@ type EventBatchEncoder interface { // MQMessage represents an MQ message to the mqSink type MQMessage struct { - Key []byte - Value []byte - Ts uint64 // reserved for possible output sorting - Schema *string // schema - Table *string // table - Type model.MqMessageType // type - Protocol Protocol // protocol + Key []byte + Value []byte + Ts uint64 // reserved for possible output sorting + Schema *string // schema + Table *string // table + Type model.MqMessageType // type + Protocol Protocol // protocol + RowsCount int // rows in one MQ Message } // maximumRecordOverhead is used to calculate ProducerMessage's byteSize by sarama kafka client. @@ -95,13 +96,14 @@ func newResolvedMQMessage(proto Protocol, key, value []byte, ts uint64) *MQMessa // It copies the input byte slices to avoid any surprises in asynchronous MQ writes. func NewMQMessage(proto Protocol, key []byte, value []byte, ts uint64, ty model.MqMessageType, schema, table *string) *MQMessage { ret := &MQMessage{ - Key: nil, - Value: nil, - Ts: ts, - Schema: schema, - Table: table, - Type: ty, - Protocol: proto, + Key: nil, + Value: nil, + Ts: ts, + Schema: schema, + Table: table, + Type: ty, + Protocol: proto, + RowsCount: 0, } if key != nil { diff --git a/cdc/sink/codec/json.go b/cdc/sink/codec/json.go index 723ef7853dd..90f1e1dd0cc 100644 --- a/cdc/sink/codec/json.go +++ b/cdc/sink/codec/json.go @@ -454,6 +454,7 @@ func (d *JSONEventBatchEncoder) AppendRowChangedEvent(e *model.RowChangedEvent) message.Ts = e.CommitTs message.Schema = &e.Table.Schema message.Table = &e.Table.Table + message.RowsCount++ if message.Length() > d.maxMessageSize { // `len(d.messageBuf) == 1` is implied diff --git a/cdc/sink/mq.go b/cdc/sink/mq.go index 5a2bf4edbac..3bce770ac2b 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -279,8 +279,8 @@ func (k *mqSink) runWorker(ctx context.Context, partition int32) error { flushToProducer := func(op codec.EncoderResult) error { return k.statistics.RecordBatchExecution(func() (int, error) { messages := encoder.Build() - thisBatchSize := len(messages) - if thisBatchSize == 0 { + thisBatchSize := 0 + if len(messages) == 0 { return 0, nil } @@ -289,6 +289,7 @@ func (k *mqSink) runWorker(ctx context.Context, partition int32) error { if err != nil { return 0, err } + thisBatchSize += msg.RowsCount } if op == codec.EncoderNeedSyncWrite { From 41b5f65dc0a2ad50ac3f98f9b8898234b8290159 Mon Sep 17 00:00:00 2001 From: zhaoxinyu Date: Wed, 1 Dec 2021 18:16:22 +0800 Subject: [PATCH 02/48] address comments --- cdc/sink/codec/craft.go | 5 ++++- cdc/sink/codec/craft/model.go | 5 +++++ cdc/sink/codec/interface.go | 17 +++++++++++++++-- cdc/sink/codec/json.go | 2 +- cdc/sink/mq.go | 2 +- 5 files changed, 26 insertions(+), 5 deletions(-) diff --git a/cdc/sink/codec/craft.go b/cdc/sink/codec/craft.go index 2766f12950b..bee38072c7e 100644 --- a/cdc/sink/codec/craft.go +++ b/cdc/sink/codec/craft.go @@ -46,7 +46,10 @@ func (e *CraftEventBatchEncoder) flush() { ts := headers.GetTs(0) schema := headers.GetSchema(0) table := headers.GetTable(0) - e.messageBuf = append(e.messageBuf, NewMQMessage(ProtocolCraft, nil, e.rowChangedBuffer.Encode(), ts, model.MqMessageTypeRow, &schema, &table)) + rowsCnt := e.rowChangedBuffer.RowsCount() + mqMessage := NewMQMessage(ProtocolCraft, nil, e.rowChangedBuffer.Encode(), ts, model.MqMessageTypeRow, &schema, &table) + mqMessage.SetRowsCount(rowsCnt) + e.messageBuf = append(e.messageBuf, mqMessage) } // AppendRowChangedEvent implements the EventBatchEncoder interface diff --git a/cdc/sink/codec/craft/model.go b/cdc/sink/codec/craft/model.go index a6b51aaa418..f467d0f980d 100644 --- a/cdc/sink/codec/craft/model.go +++ b/cdc/sink/codec/craft/model.go @@ -498,6 +498,11 @@ func (b *RowChangedEventBuffer) Size() int { return b.estimatedSize } +// Number of rows batched in this buffer +func (b *RowChangedEventBuffer) RowsCount() int { + return b.eventsCount +} + // GetHeaders returns headers of buffer func (b *RowChangedEventBuffer) GetHeaders() *Headers { return b.headers diff --git a/cdc/sink/codec/interface.go b/cdc/sink/codec/interface.go index ec8ec2a94d5..6b4832ef310 100644 --- a/cdc/sink/codec/interface.go +++ b/cdc/sink/codec/interface.go @@ -64,7 +64,7 @@ type MQMessage struct { Table *string // table Type model.MqMessageType // type Protocol Protocol // protocol - RowsCount int // rows in one MQ Message + rowsCount int // rows in one MQ Message } // maximumRecordOverhead is used to calculate ProducerMessage's byteSize by sarama kafka client. @@ -84,6 +84,19 @@ func (m *MQMessage) PhysicalTime() time.Time { return oracle.GetTimeFromTS(m.Ts) } +// return number of rows batched in one MQMessage +func (m *MQMessage) GetRowsCount() int { + return m.rowsCount +} + +func (m *MQMessage) SetRowsCount(cnt int) { + m.rowsCount = cnt +} + +func (m *MQMessage) IncRowsCount() { + m.rowsCount++ +} + func newDDLMQMessage(proto Protocol, key, value []byte, event *model.DDLEvent) *MQMessage { return NewMQMessage(proto, key, value, event.CommitTs, model.MqMessageTypeDDL, &event.TableInfo.Schema, &event.TableInfo.Table) } @@ -103,7 +116,7 @@ func NewMQMessage(proto Protocol, key []byte, value []byte, ts uint64, ty model. Table: table, Type: ty, Protocol: proto, - RowsCount: 0, + rowsCount: 0, } if key != nil { diff --git a/cdc/sink/codec/json.go b/cdc/sink/codec/json.go index 90f1e1dd0cc..4c3c44b65b6 100644 --- a/cdc/sink/codec/json.go +++ b/cdc/sink/codec/json.go @@ -454,7 +454,7 @@ func (d *JSONEventBatchEncoder) AppendRowChangedEvent(e *model.RowChangedEvent) message.Ts = e.CommitTs message.Schema = &e.Table.Schema message.Table = &e.Table.Table - message.RowsCount++ + message.IncRowsCount() if message.Length() > d.maxMessageSize { // `len(d.messageBuf) == 1` is implied diff --git a/cdc/sink/mq.go b/cdc/sink/mq.go index 3bce770ac2b..c1677232274 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -289,7 +289,7 @@ func (k *mqSink) runWorker(ctx context.Context, partition int32) error { if err != nil { return 0, err } - thisBatchSize += msg.RowsCount + thisBatchSize += msg.GetRowsCount() } if op == codec.EncoderNeedSyncWrite { From 381aa83957c146f1186c9fc8ac3f1b778cbf370c Mon Sep 17 00:00:00 2001 From: zhaoxinyu Date: Wed, 1 Dec 2021 18:33:50 +0800 Subject: [PATCH 03/48] add comments for exported functions --- cdc/sink/codec/interface.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/cdc/sink/codec/interface.go b/cdc/sink/codec/interface.go index 6b4832ef310..7a555de7204 100644 --- a/cdc/sink/codec/interface.go +++ b/cdc/sink/codec/interface.go @@ -84,15 +84,17 @@ func (m *MQMessage) PhysicalTime() time.Time { return oracle.GetTimeFromTS(m.Ts) } -// return number of rows batched in one MQMessage +// GetRowsCount returns the number of rows batched in one MQMessage func (m *MQMessage) GetRowsCount() int { return m.rowsCount } +// SetRowsCount set the number of rows func (m *MQMessage) SetRowsCount(cnt int) { m.rowsCount = cnt } +// IncRowsCount increase the number of rows func (m *MQMessage) IncRowsCount() { m.rowsCount++ } From cf6f73fbc504ee2c91a49d2f25f19805c2c81074 Mon Sep 17 00:00:00 2001 From: zhaoxinyu Date: Thu, 9 Dec 2021 10:26:52 +0800 Subject: [PATCH 04/48] fix the compiling problem --- cdc/sink/codec/craft.go | 2 +- cdc/sink/codec/interface.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/cdc/sink/codec/craft.go b/cdc/sink/codec/craft.go index bee38072c7e..ebf5a4b5960 100644 --- a/cdc/sink/codec/craft.go +++ b/cdc/sink/codec/craft.go @@ -47,7 +47,7 @@ func (e *CraftEventBatchEncoder) flush() { schema := headers.GetSchema(0) table := headers.GetTable(0) rowsCnt := e.rowChangedBuffer.RowsCount() - mqMessage := NewMQMessage(ProtocolCraft, nil, e.rowChangedBuffer.Encode(), ts, model.MqMessageTypeRow, &schema, &table) + mqMessage := NewMQMessage(config.ProtocolCraft, nil, e.rowChangedBuffer.Encode(), ts, model.MqMessageTypeRow, &schema, &table) mqMessage.SetRowsCount(rowsCnt) e.messageBuf = append(e.messageBuf, mqMessage) } diff --git a/cdc/sink/codec/interface.go b/cdc/sink/codec/interface.go index 7a555de7204..2bfcecd1dc3 100644 --- a/cdc/sink/codec/interface.go +++ b/cdc/sink/codec/interface.go @@ -63,7 +63,7 @@ type MQMessage struct { Schema *string // schema Table *string // table Type model.MqMessageType // type - Protocol Protocol // protocol + Protocol config.Protocol // protocol rowsCount int // rows in one MQ Message } From 339688e830636c2c5f1929c250ed5a562c5f8841 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Fri, 10 Dec 2021 23:05:57 +0800 Subject: [PATCH 05/48] workerpool: limit the rate to output deadlock warning (#3775) (#3795) --- pkg/workerpool/pool_impl.go | 20 +++++++++++++++++--- pkg/workerpool/pool_test.go | 33 +++++++++++++++++++++++++++++++++ 2 files changed, 50 insertions(+), 3 deletions(-) diff --git a/pkg/workerpool/pool_impl.go b/pkg/workerpool/pool_impl.go index e3f754450b1..5927c12069d 100644 --- a/pkg/workerpool/pool_impl.go +++ b/pkg/workerpool/pool_impl.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/ticdc/pkg/notify" "go.uber.org/zap" "golang.org/x/sync/errgroup" + "golang.org/x/time/rate" ) const ( @@ -298,6 +299,9 @@ type worker struct { isRunning int32 // notifies exits of run() stopNotifier notify.Notifier + + slowSynchronizeThreshold time.Duration + slowSynchronizeLimiter *rate.Limiter } func newWorker() *worker { @@ -305,6 +309,9 @@ func newWorker() *worker { taskCh: make(chan task, 128), handles: make(map[*defaultEventHandle]struct{}), handleCancelCh: make(chan struct{}), // this channel must be unbuffered, i.e. blocking + + slowSynchronizeThreshold: 10 * time.Second, + slowSynchronizeLimiter: rate.NewLimiter(rate.Every(time.Second*5), 1), } } @@ -398,13 +405,20 @@ func (w *worker) synchronize() { break } - if time.Since(startTime) > time.Second*10 { - // likely the workerpool has deadlocked, or there is a bug in the event handlers. - log.Warn("synchronize is taking too long, report a bug", zap.Duration("elapsed", time.Since(startTime))) + if time.Since(startTime) > w.slowSynchronizeThreshold && + w.slowSynchronizeLimiter.Allow() { + // likely the workerpool has deadlocked, or there is a bug + // in the event handlers. + logWarn("synchronize is taking too long, report a bug", + zap.Duration("elapsed", time.Since(startTime)), + zap.Stack("stacktrace")) } } } +// A delegate to log.Warn. It exists only for testing. +var logWarn = log.Warn + func (w *worker) addHandle(handle *defaultEventHandle) { w.handleRWLock.Lock() defer w.handleRWLock.Unlock() diff --git a/pkg/workerpool/pool_test.go b/pkg/workerpool/pool_test.go index 76cf1701ec5..3aad1b0b1f7 100644 --- a/pkg/workerpool/pool_test.go +++ b/pkg/workerpool/pool_test.go @@ -27,6 +27,7 @@ import ( "github.com/stretchr/testify/require" "go.uber.org/zap" "golang.org/x/sync/errgroup" + "golang.org/x/time/rate" ) func TestTaskError(t *testing.T) { @@ -507,6 +508,38 @@ func TestGracefulUnregisterTimeout(t *testing.T) { require.Truef(t, cerror.ErrWorkerPoolGracefulUnregisterTimedOut.Equal(err), "%s", err.Error()) } +func TestSynchronizeLog(t *testing.T) { + w := newWorker() + w.isRunning = 1 + // Always report "synchronize is taking too long". + w.slowSynchronizeThreshold = time.Duration(0) + w.slowSynchronizeLimiter = rate.NewLimiter(rate.Every(100*time.Minute), 1) + + counter := int32(0) + logWarn = func(msg string, fields ...zap.Field) { + atomic.AddInt32(&counter, 1) + } + defer func() { logWarn = log.Warn }() + + doneCh := make(chan struct{}) + go func() { + w.synchronize() + close(doneCh) + }() + + time.Sleep(300 * time.Millisecond) + w.stopNotifier.Notify() + time.Sleep(300 * time.Millisecond) + w.stopNotifier.Notify() + + // Close worker. + atomic.StoreInt32(&w.isRunning, 0) + w.stopNotifier.Close() + <-doneCh + + require.EqualValues(t, 1, atomic.LoadInt32(&counter)) +} + // Benchmark workerpool with ping-pong workflow. // go test -benchmem -run='^$' -bench '^(BenchmarkWorkerpool)$' github.com/pingcap/ticdc/pkg/workerpool func BenchmarkWorkerpool(b *testing.B) { From ef2636afddb3a6b00d40c4dd42cecbef8d9e3784 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Tue, 14 Dec 2021 13:06:34 +0800 Subject: [PATCH 06/48] tests(ticdc): set up the sync diff output directory correctly (#3725) (#3741) --- tests/integration_tests/_utils/check_sync_diff | 5 ++--- tests/integration_tests/autorandom/conf/diff_config.toml | 2 +- .../integration_tests/batch_add_table/conf/diff_config.toml | 2 +- .../capture_session_done_during_task/conf/diff_config.toml | 2 +- .../conf/diff_config.toml | 2 +- .../changefeed_auto_stop/conf/diff_config.toml | 2 +- .../integration_tests/changefeed_error/conf/diff_config.toml | 2 +- .../changefeed_finish/conf/diff_config.toml | 2 +- .../changefeed_pause_resume/conf/diff_config.toml | 2 +- .../changefeed_reconstruct/conf/diff_config.toml | 2 +- .../integration_tests/clustered_index/conf/diff_config.toml | 2 +- tests/integration_tests/common_1/conf/diff_config.toml | 2 +- .../consistent_replicate_nfs/conf/diff_config.toml | 2 +- .../consistent_replicate_s3/conf/diff_config.toml | 2 +- tests/integration_tests/cyclic_ab/conf/diff_config.toml | 2 +- .../cyclic_abc/conf/diff_config_down_tls.toml | 2 +- .../cyclic_abc/conf/diff_config_up_down.toml | 2 +- tests/integration_tests/ddl_attributes/conf/diff_config.toml | 2 +- tests/integration_tests/ddl_reentrant/conf/diff_config.toml | 2 +- tests/integration_tests/ddl_sequence/conf/diff_config.toml | 2 +- .../integration_tests/drop_many_tables/conf/diff_config.toml | 2 +- tests/integration_tests/gc_safepoint/conf/diff_config.toml | 2 +- .../integration_tests/generate_column/conf/diff_config.toml | 2 +- tests/integration_tests/kafka_messages/conf/diff_config.toml | 2 +- .../kafka_sink_error_resume/conf/diff_config.toml | 2 +- .../kill_owner_with_ddl/conf/diff_config.toml | 2 +- .../kv_client_stream_reconnect/conf/diff_config.toml | 2 +- tests/integration_tests/many_pk_or_uk/diff_config.toml | 2 +- tests/integration_tests/move_table/conf/diff_config.toml | 2 +- tests/integration_tests/multi_capture/conf/diff_config.toml | 2 +- tests/integration_tests/multi_source/diff_config.toml | 2 +- .../new_ci_collation_with_old_value/conf/diff_config.toml | 2 +- .../new_ci_collation_without_old_value/conf/diff_config.toml | 2 +- .../owner_remove_table_error/conf/diff_config.toml | 2 +- .../integration_tests/partition_table/conf/diff_config.toml | 2 +- .../processor_err_chan/conf/diff_config.toml | 2 +- tests/integration_tests/processor_panic/diff_config.toml | 2 +- .../processor_resolved_ts_fallback/conf/diff_config.toml | 2 +- .../processor_stop_delay/conf/diff_config.toml | 2 +- tests/integration_tests/region_merge/conf/diff_config.toml | 2 +- tests/integration_tests/resolve_lock/diff_config.toml | 2 +- tests/integration_tests/row_format/conf/diff_config.toml | 2 +- tests/integration_tests/sink_hang/conf/diff_config.toml | 2 +- tests/integration_tests/sink_retry/conf/diff_config.toml | 2 +- tests/integration_tests/split_region/conf/diff_config.toml | 2 +- .../integration_tests/syncpoint/conf/diff_config_final.toml | 4 ++-- .../integration_tests/syncpoint/conf/diff_config_part1.toml | 3 +-- tests/integration_tests/tiflash/conf/diff_config.toml | 2 +- tests/integration_tests/unified_sorter/conf/diff_config.toml | 2 +- 49 files changed, 51 insertions(+), 53 deletions(-) diff --git a/tests/integration_tests/_utils/check_sync_diff b/tests/integration_tests/_utils/check_sync_diff index 42a8c5b5f6c..7f1b72e3f7c 100755 --- a/tests/integration_tests/_utils/check_sync_diff +++ b/tests/integration_tests/_utils/check_sync_diff @@ -14,14 +14,13 @@ binary=sync_diff_inspector PWD=$(pwd) LOG=$workdir/sync_diff_inspector.log -OUTPUT_DIR=/tmp/ticdc_dm_test/output/ mkdir -p /tmp/ticdc_dm_test cd $workdir i=0 while [ $i -lt $check_time ]; do - rm -rf $OUTPUT_DIR + rm -rf $workdir/sync_diff/ $binary --config=$conf >>$LOG 2>&1 ret=$? if [ "$ret" == 0 ]; then @@ -38,7 +37,7 @@ if [ $i -ge $check_time ]; then # show \n and other blanks cat $LOG printf "\n" - cat $OUTPUT_DIR/sync_diff.log + cat $workdir/sync_diff/output/sync_diff.log printf "\n" exit 1 fi diff --git a/tests/integration_tests/autorandom/conf/diff_config.toml b/tests/integration_tests/autorandom/conf/diff_config.toml index 6b27666eaf6..0c7c54811ee 100644 --- a/tests/integration_tests/autorandom/conf/diff_config.toml +++ b/tests/integration_tests/autorandom/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/autorandom/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/batch_add_table/conf/diff_config.toml b/tests/integration_tests/batch_add_table/conf/diff_config.toml index 48fc5dabeae..fb2e5b0d977 100644 --- a/tests/integration_tests/batch_add_table/conf/diff_config.toml +++ b/tests/integration_tests/batch_add_table/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/batch_add_table/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/capture_session_done_during_task/conf/diff_config.toml b/tests/integration_tests/capture_session_done_during_task/conf/diff_config.toml index d5bbea9452c..190cee78e06 100644 --- a/tests/integration_tests/capture_session_done_during_task/conf/diff_config.toml +++ b/tests/integration_tests/capture_session_done_during_task/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/capture_session_done_during_task/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/capture_suicide_while_balance_table/conf/diff_config.toml b/tests/integration_tests/capture_suicide_while_balance_table/conf/diff_config.toml index 7a52b92d91c..381543403c4 100644 --- a/tests/integration_tests/capture_suicide_while_balance_table/conf/diff_config.toml +++ b/tests/integration_tests/capture_suicide_while_balance_table/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/capture_suicide_while_balance_table/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/changefeed_auto_stop/conf/diff_config.toml b/tests/integration_tests/changefeed_auto_stop/conf/diff_config.toml index 5946907b719..a50bddac095 100644 --- a/tests/integration_tests/changefeed_auto_stop/conf/diff_config.toml +++ b/tests/integration_tests/changefeed_auto_stop/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/changefeed_auto_stop/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/changefeed_error/conf/diff_config.toml b/tests/integration_tests/changefeed_error/conf/diff_config.toml index 54e11c9d242..394f3b00b70 100644 --- a/tests/integration_tests/changefeed_error/conf/diff_config.toml +++ b/tests/integration_tests/changefeed_error/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/changefeed_error/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/changefeed_finish/conf/diff_config.toml b/tests/integration_tests/changefeed_finish/conf/diff_config.toml index b191d2973d8..4d551f7d5a5 100644 --- a/tests/integration_tests/changefeed_finish/conf/diff_config.toml +++ b/tests/integration_tests/changefeed_finish/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/changefeed_finish/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/changefeed_pause_resume/conf/diff_config.toml b/tests/integration_tests/changefeed_pause_resume/conf/diff_config.toml index b8eeb0b3a3d..1bb531f7004 100644 --- a/tests/integration_tests/changefeed_pause_resume/conf/diff_config.toml +++ b/tests/integration_tests/changefeed_pause_resume/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/changefeed_pause_resume/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/changefeed_reconstruct/conf/diff_config.toml b/tests/integration_tests/changefeed_reconstruct/conf/diff_config.toml index cf34a8c82fe..1181abbc6c2 100644 --- a/tests/integration_tests/changefeed_reconstruct/conf/diff_config.toml +++ b/tests/integration_tests/changefeed_reconstruct/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/changefeed_reconstruct/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/clustered_index/conf/diff_config.toml b/tests/integration_tests/clustered_index/conf/diff_config.toml index 06604de2bfb..cc84ddc3606 100644 --- a/tests/integration_tests/clustered_index/conf/diff_config.toml +++ b/tests/integration_tests/clustered_index/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/clustered_index/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/common_1/conf/diff_config.toml b/tests/integration_tests/common_1/conf/diff_config.toml index 6df50ab03bd..36b4eb0a9fe 100644 --- a/tests/integration_tests/common_1/conf/diff_config.toml +++ b/tests/integration_tests/common_1/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/common_1/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/consistent_replicate_nfs/conf/diff_config.toml b/tests/integration_tests/consistent_replicate_nfs/conf/diff_config.toml index 908fdb26da8..9677f26b0e4 100644 --- a/tests/integration_tests/consistent_replicate_nfs/conf/diff_config.toml +++ b/tests/integration_tests/consistent_replicate_nfs/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/consistent_replicate_nfs/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/consistent_replicate_s3/conf/diff_config.toml b/tests/integration_tests/consistent_replicate_s3/conf/diff_config.toml index ef672ccb24c..610eab9dc1e 100644 --- a/tests/integration_tests/consistent_replicate_s3/conf/diff_config.toml +++ b/tests/integration_tests/consistent_replicate_s3/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/consistent_replicate_s3/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/cyclic_ab/conf/diff_config.toml b/tests/integration_tests/cyclic_ab/conf/diff_config.toml index 26c78959203..122b642fc77 100644 --- a/tests/integration_tests/cyclic_ab/conf/diff_config.toml +++ b/tests/integration_tests/cyclic_ab/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/cyclic_ab/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/cyclic_abc/conf/diff_config_down_tls.toml b/tests/integration_tests/cyclic_abc/conf/diff_config_down_tls.toml index a38a536fe50..1973a48a3ae 100644 --- a/tests/integration_tests/cyclic_abc/conf/diff_config_down_tls.toml +++ b/tests/integration_tests/cyclic_abc/conf/diff_config_down_tls.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/cyclic_abc/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/cyclic_abc/conf/diff_config_up_down.toml b/tests/integration_tests/cyclic_abc/conf/diff_config_up_down.toml index 26c78959203..bb38ca4729c 100644 --- a/tests/integration_tests/cyclic_abc/conf/diff_config_up_down.toml +++ b/tests/integration_tests/cyclic_abc/conf/diff_config_up_down.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/cyclic_abc/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/ddl_attributes/conf/diff_config.toml b/tests/integration_tests/ddl_attributes/conf/diff_config.toml index 5994a8fad64..81f746e9a9c 100644 --- a/tests/integration_tests/ddl_attributes/conf/diff_config.toml +++ b/tests/integration_tests/ddl_attributes/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/ddl_attributes/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/ddl_reentrant/conf/diff_config.toml b/tests/integration_tests/ddl_reentrant/conf/diff_config.toml index 32c241bd5d1..afc30f86a57 100644 --- a/tests/integration_tests/ddl_reentrant/conf/diff_config.toml +++ b/tests/integration_tests/ddl_reentrant/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/ddl_reentrant/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/ddl_sequence/conf/diff_config.toml b/tests/integration_tests/ddl_sequence/conf/diff_config.toml index c06d75a429a..6ff0b3b6c7d 100644 --- a/tests/integration_tests/ddl_sequence/conf/diff_config.toml +++ b/tests/integration_tests/ddl_sequence/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/ddl_sequence/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/drop_many_tables/conf/diff_config.toml b/tests/integration_tests/drop_many_tables/conf/diff_config.toml index e85c5294595..203c0607219 100644 --- a/tests/integration_tests/drop_many_tables/conf/diff_config.toml +++ b/tests/integration_tests/drop_many_tables/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/drop_many_tables/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/gc_safepoint/conf/diff_config.toml b/tests/integration_tests/gc_safepoint/conf/diff_config.toml index 5d290c6ba58..f4e7a89971b 100644 --- a/tests/integration_tests/gc_safepoint/conf/diff_config.toml +++ b/tests/integration_tests/gc_safepoint/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/gc_safepoint/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/generate_column/conf/diff_config.toml b/tests/integration_tests/generate_column/conf/diff_config.toml index d3c05685f52..c3705f732b2 100644 --- a/tests/integration_tests/generate_column/conf/diff_config.toml +++ b/tests/integration_tests/generate_column/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/generate_column/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/kafka_messages/conf/diff_config.toml b/tests/integration_tests/kafka_messages/conf/diff_config.toml index 527f0835f00..f4a6d29c149 100644 --- a/tests/integration_tests/kafka_messages/conf/diff_config.toml +++ b/tests/integration_tests/kafka_messages/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/kafka_message/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/kafka_sink_error_resume/conf/diff_config.toml b/tests/integration_tests/kafka_sink_error_resume/conf/diff_config.toml index 15774d89159..db689a10112 100644 --- a/tests/integration_tests/kafka_sink_error_resume/conf/diff_config.toml +++ b/tests/integration_tests/kafka_sink_error_resume/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/kafka_sink_error_resume/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/kill_owner_with_ddl/conf/diff_config.toml b/tests/integration_tests/kill_owner_with_ddl/conf/diff_config.toml index b5e8b1d7914..040865d28be 100644 --- a/tests/integration_tests/kill_owner_with_ddl/conf/diff_config.toml +++ b/tests/integration_tests/kill_owner_with_ddl/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/kill_owner_with_ddl/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/kv_client_stream_reconnect/conf/diff_config.toml b/tests/integration_tests/kv_client_stream_reconnect/conf/diff_config.toml index 8b63ddae343..197ca066b01 100644 --- a/tests/integration_tests/kv_client_stream_reconnect/conf/diff_config.toml +++ b/tests/integration_tests/kv_client_stream_reconnect/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/kv_client_stream_reconnect/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/many_pk_or_uk/diff_config.toml b/tests/integration_tests/many_pk_or_uk/diff_config.toml index 26c78959203..5247e336450 100644 --- a/tests/integration_tests/many_pk_or_uk/diff_config.toml +++ b/tests/integration_tests/many_pk_or_uk/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/many_pk_or_uk/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/move_table/conf/diff_config.toml b/tests/integration_tests/move_table/conf/diff_config.toml index 6d8a575cb8a..c6aeddb77d9 100644 --- a/tests/integration_tests/move_table/conf/diff_config.toml +++ b/tests/integration_tests/move_table/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/move_table/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/multi_capture/conf/diff_config.toml b/tests/integration_tests/multi_capture/conf/diff_config.toml index 89dd57024a3..f6cd453010a 100644 --- a/tests/integration_tests/multi_capture/conf/diff_config.toml +++ b/tests/integration_tests/multi_capture/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/multi_capture/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/multi_source/diff_config.toml b/tests/integration_tests/multi_source/diff_config.toml index 26c78959203..103c77690c9 100644 --- a/tests/integration_tests/multi_source/diff_config.toml +++ b/tests/integration_tests/multi_source/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/multi_source/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/new_ci_collation_with_old_value/conf/diff_config.toml b/tests/integration_tests/new_ci_collation_with_old_value/conf/diff_config.toml index ed282c8a8ad..32b8417e91f 100644 --- a/tests/integration_tests/new_ci_collation_with_old_value/conf/diff_config.toml +++ b/tests/integration_tests/new_ci_collation_with_old_value/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/new_ci_collation_with_old_value/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/new_ci_collation_without_old_value/conf/diff_config.toml b/tests/integration_tests/new_ci_collation_without_old_value/conf/diff_config.toml index 19607ddf9ad..412de80d2a2 100644 --- a/tests/integration_tests/new_ci_collation_without_old_value/conf/diff_config.toml +++ b/tests/integration_tests/new_ci_collation_without_old_value/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/new_ci_collation_without_old_value/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/owner_remove_table_error/conf/diff_config.toml b/tests/integration_tests/owner_remove_table_error/conf/diff_config.toml index ea5a5674744..750741652fa 100644 --- a/tests/integration_tests/owner_remove_table_error/conf/diff_config.toml +++ b/tests/integration_tests/owner_remove_table_error/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/owner_remove_table_error/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/partition_table/conf/diff_config.toml b/tests/integration_tests/partition_table/conf/diff_config.toml index 8a03301f160..24e98abed31 100644 --- a/tests/integration_tests/partition_table/conf/diff_config.toml +++ b/tests/integration_tests/partition_table/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/partition_table/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/processor_err_chan/conf/diff_config.toml b/tests/integration_tests/processor_err_chan/conf/diff_config.toml index 0ba4707fd81..6bfc3058a1a 100644 --- a/tests/integration_tests/processor_err_chan/conf/diff_config.toml +++ b/tests/integration_tests/processor_err_chan/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/processor_err_chan/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/processor_panic/diff_config.toml b/tests/integration_tests/processor_panic/diff_config.toml index 26c78959203..9485bb2c819 100644 --- a/tests/integration_tests/processor_panic/diff_config.toml +++ b/tests/integration_tests/processor_panic/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/processor_panic/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/processor_resolved_ts_fallback/conf/diff_config.toml b/tests/integration_tests/processor_resolved_ts_fallback/conf/diff_config.toml index 9dca856dd84..1796c36ecb3 100644 --- a/tests/integration_tests/processor_resolved_ts_fallback/conf/diff_config.toml +++ b/tests/integration_tests/processor_resolved_ts_fallback/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/processor_resolved_ts_fallback/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/processor_stop_delay/conf/diff_config.toml b/tests/integration_tests/processor_stop_delay/conf/diff_config.toml index cb55b724d49..83d857c924c 100644 --- a/tests/integration_tests/processor_stop_delay/conf/diff_config.toml +++ b/tests/integration_tests/processor_stop_delay/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/processor_stop_delay/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/region_merge/conf/diff_config.toml b/tests/integration_tests/region_merge/conf/diff_config.toml index 8dbd018c17d..0a2b43d5a2f 100644 --- a/tests/integration_tests/region_merge/conf/diff_config.toml +++ b/tests/integration_tests/region_merge/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/region_merge/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/resolve_lock/diff_config.toml b/tests/integration_tests/resolve_lock/diff_config.toml index 26c78959203..1af712185a9 100644 --- a/tests/integration_tests/resolve_lock/diff_config.toml +++ b/tests/integration_tests/resolve_lock/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/resolve_lock/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/row_format/conf/diff_config.toml b/tests/integration_tests/row_format/conf/diff_config.toml index d5f470cfec6..28572faf4f5 100644 --- a/tests/integration_tests/row_format/conf/diff_config.toml +++ b/tests/integration_tests/row_format/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/row_format/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/sink_hang/conf/diff_config.toml b/tests/integration_tests/sink_hang/conf/diff_config.toml index 894edbf2bcc..0d1605c60ae 100644 --- a/tests/integration_tests/sink_hang/conf/diff_config.toml +++ b/tests/integration_tests/sink_hang/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/sink_hang/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/sink_retry/conf/diff_config.toml b/tests/integration_tests/sink_retry/conf/diff_config.toml index 9d4155398ce..2d31480fbe4 100644 --- a/tests/integration_tests/sink_retry/conf/diff_config.toml +++ b/tests/integration_tests/sink_retry/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/sink_retry/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/split_region/conf/diff_config.toml b/tests/integration_tests/split_region/conf/diff_config.toml index 8433fad74e2..8e624a5c525 100644 --- a/tests/integration_tests/split_region/conf/diff_config.toml +++ b/tests/integration_tests/split_region/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/split_region/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/syncpoint/conf/diff_config_final.toml b/tests/integration_tests/syncpoint/conf/diff_config_final.toml index 635eced0c7c..5e960cbac03 100644 --- a/tests/integration_tests/syncpoint/conf/diff_config_final.toml +++ b/tests/integration_tests/syncpoint/conf/diff_config_final.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/syncpoint/sync_diff/output" source-instances = ["mysql1"] @@ -26,4 +26,4 @@ check-struct-only = false host = "127.0.0.1" port = 3306 user = "root" - password = "" \ No newline at end of file + password = "" diff --git a/tests/integration_tests/syncpoint/conf/diff_config_part1.toml b/tests/integration_tests/syncpoint/conf/diff_config_part1.toml index 578f58ccfdc..40c3af5f1ec 100644 --- a/tests/integration_tests/syncpoint/conf/diff_config_part1.toml +++ b/tests/integration_tests/syncpoint/conf/diff_config_part1.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/syncpoint/sync_diff/output" source-instances = ["mysql1"] @@ -23,4 +23,3 @@ check-struct-only = false port = 4000 user = "root" password = "" - \ No newline at end of file diff --git a/tests/integration_tests/tiflash/conf/diff_config.toml b/tests/integration_tests/tiflash/conf/diff_config.toml index 9794baad8ea..4740f1d014d 100644 --- a/tests/integration_tests/tiflash/conf/diff_config.toml +++ b/tests/integration_tests/tiflash/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/tiflash/sync_diff/output" source-instances = ["mysql1"] diff --git a/tests/integration_tests/unified_sorter/conf/diff_config.toml b/tests/integration_tests/unified_sorter/conf/diff_config.toml index 99a31dc902b..54b2eb79fdf 100644 --- a/tests/integration_tests/unified_sorter/conf/diff_config.toml +++ b/tests/integration_tests/unified_sorter/conf/diff_config.toml @@ -7,7 +7,7 @@ export-fix-sql = true check-struct-only = false [task] - output-dir = "/tmp/ticdc_dm_test/output" + output-dir = "/tmp/tidb_cdc_test/unified_sort/sync_diff/output" source-instances = ["mysql1"] From a4330bd6e39d80ce417e25afe9fa678736d86ab3 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Tue, 14 Dec 2021 14:22:35 +0800 Subject: [PATCH 07/48] relay(dm): use binlog name comparison (#3710) (#3712) --- dm/relay/relay.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dm/relay/relay.go b/dm/relay/relay.go index d76e48bef12..0a9bfa5da32 100644 --- a/dm/relay/relay.go +++ b/dm/relay/relay.go @@ -240,7 +240,7 @@ func (r *Relay) process(ctx context.Context) error { } } else { _, metaPos := r.meta.Pos() - if neededBinlogName > metaPos.Name { + if mysql.CompareBinlogFileName(neededBinlogName, metaPos.Name) > 0 { isRelayMetaOutdated = true } } From 95d038f28b3c8c6ed715714565931e17b9f5fc73 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Fri, 17 Dec 2021 11:32:35 +0800 Subject: [PATCH 08/48] dm/load: fix concurrent call Loader.Status (#3459) (#3468) --- dm/loader/loader.go | 10 +++---- dm/loader/status.go | 14 +++++++--- dm/loader/status_test.go | 56 ++++++++++++++++++++++++++++++++++++++++ 3 files changed, 71 insertions(+), 9 deletions(-) create mode 100644 dm/loader/status_test.go diff --git a/dm/loader/loader.go b/dm/loader/loader.go index 3046e038398..2b80fba6ac7 100644 --- a/dm/loader/loader.go +++ b/dm/loader/loader.go @@ -441,8 +441,8 @@ type Loader struct { // to calculate remainingTimeGauge metric, map will be init in `l.prepare.prepareDataFiles` dbTableDataTotalSize map[string]map[string]*atomic.Int64 dbTableDataFinishedSize map[string]map[string]*atomic.Int64 - dbTableDataLastFinishedSize map[string]map[string]int64 - dbTableDataLastUpdatedTime time.Time + dbTableDataLastFinishedSize map[string]map[string]*atomic.Int64 + dbTableDataLastUpdatedTime atomic.Time metaBinlog atomic.String metaBinlogGTID atomic.String @@ -1053,12 +1053,12 @@ func (l *Loader) prepareDataFiles(files map[string]struct{}) error { if _, ok := l.dbTableDataTotalSize[db]; !ok { l.dbTableDataTotalSize[db] = make(map[string]*atomic.Int64) l.dbTableDataFinishedSize[db] = make(map[string]*atomic.Int64) - l.dbTableDataLastFinishedSize[db] = make(map[string]int64) + l.dbTableDataLastFinishedSize[db] = make(map[string]*atomic.Int64) } if _, ok := l.dbTableDataTotalSize[db][table]; !ok { l.dbTableDataTotalSize[db][table] = atomic.NewInt64(0) l.dbTableDataFinishedSize[db][table] = atomic.NewInt64(0) - l.dbTableDataLastFinishedSize[db][table] = 0 + l.dbTableDataLastFinishedSize[db][table] = atomic.NewInt64(0) } l.dbTableDataTotalSize[db][table].Add(size) @@ -1083,7 +1083,7 @@ func (l *Loader) prepare() error { l.finishedDataSize.Store(0) // reset before load from checkpoint l.dbTableDataTotalSize = make(map[string]map[string]*atomic.Int64) l.dbTableDataFinishedSize = make(map[string]map[string]*atomic.Int64) - l.dbTableDataLastFinishedSize = make(map[string]map[string]int64) + l.dbTableDataLastFinishedSize = make(map[string]map[string]*atomic.Int64) // check if mydumper dir data exists. if !utils.IsDirExists(l.cfg.Dir) { diff --git a/dm/loader/status.go b/dm/loader/status.go index fe525c6022d..33a619ea30f 100644 --- a/dm/loader/status.go +++ b/dm/loader/status.go @@ -44,19 +44,25 @@ func (l *Loader) printStatus() { totalSize := l.totalDataSize.Load() totalFileCount := l.totalFileCount.Load() - interval := time.Since(l.dbTableDataLastUpdatedTime) + interval := time.Since(l.dbTableDataLastUpdatedTime.Load()) + intervalSecond := interval.Seconds() + if intervalSecond == 0 { + return + } + for db, tables := range l.dbTableDataFinishedSize { for table, size := range tables { curFinished := size.Load() - speed := float64(curFinished-l.dbTableDataLastFinishedSize[db][table]) / interval.Seconds() - l.dbTableDataLastFinishedSize[db][table] = curFinished + lastFinished := l.dbTableDataFinishedSize[db][table].Load() + speed := float64(curFinished-lastFinished) / intervalSecond + l.dbTableDataLastFinishedSize[db][table].Store(curFinished) if speed > 0 { remainingSeconds := float64(l.dbTableDataTotalSize[db][table].Load()-curFinished) / speed remainingTimeGauge.WithLabelValues(l.cfg.Name, l.cfg.WorkerName, l.cfg.SourceID, db, table).Set(remainingSeconds) } } } - l.dbTableDataLastUpdatedTime = time.Now() + l.dbTableDataLastUpdatedTime.Store(time.Now()) l.logger.Info("progress status of load", zap.Int64("finished_bytes", finishedSize), diff --git a/dm/loader/status_test.go b/dm/loader/status_test.go new file mode 100644 index 00000000000..f756258dcc4 --- /dev/null +++ b/dm/loader/status_test.go @@ -0,0 +1,56 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package loader + +import ( + "sync" + + . "github.com/pingcap/check" + "go.uber.org/atomic" + + "github.com/pingcap/ticdc/dm/dm/config" + "github.com/pingcap/ticdc/dm/pkg/log" +) + +func (*testLoaderSuite) TestConcurrentStatus(c *C) { + l := &Loader{} + l.cfg = &config.SubTaskConfig{} + l.logger = log.L() + l.finishedDataSize.Store(100) + l.totalDataSize.Store(200) + l.totalFileCount.Store(10) + l.dbTableDataFinishedSize = map[string]map[string]*atomic.Int64{ + "db1": { + "table1": atomic.NewInt64(10), + "table2": atomic.NewInt64(20), + }, + } + l.dbTableDataLastFinishedSize = map[string]map[string]*atomic.Int64{ + "db1": { + "table1": atomic.NewInt64(0), + "table2": atomic.NewInt64(0), + }, + } + + // test won't race or panic + wg := sync.WaitGroup{} + wg.Add(20) + for i := 0; i < 20; i++ { + go func() { + l.Status(nil) + wg.Done() + }() + } + wg.Wait() +} From 4bb99d14a464f52b12e8202c6c93d7cb0394458e Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Fri, 17 Dec 2021 12:12:35 +0800 Subject: [PATCH 09/48] cdc/sorter: make unified sorter cgroup aware (#3436) (#3439) --- cdc/sorter/unified/backend_pool.go | 21 +++++++++++---------- cdc/sorter/unified/backend_pool_test.go | 11 +++++++---- go.mod | 1 - go.sum | 3 --- 4 files changed, 18 insertions(+), 18 deletions(-) diff --git a/cdc/sorter/unified/backend_pool.go b/cdc/sorter/unified/backend_pool.go index 3f5add5f70b..60375a12600 100644 --- a/cdc/sorter/unified/backend_pool.go +++ b/cdc/sorter/unified/backend_pool.go @@ -24,7 +24,6 @@ import ( "time" "unsafe" - "github.com/mackerelio/go-osstat/memory" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" @@ -33,6 +32,7 @@ import ( cerrors "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/filelock" "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/tidb/util/memory" "go.uber.org/zap" ) @@ -97,6 +97,13 @@ func newBackEndPool(dir string, captureAddr string) (*backEndPool, error) { metricSorterOnDiskDataSizeGauge := sorterOnDiskDataSizeGauge.WithLabelValues(captureAddr) metricSorterOpenFileCountGauge := sorterOpenFileCountGauge.WithLabelValues(captureAddr) + // TODO: The underlaying implementation only recognizes cgroups set by + // containers, we need to support cgroups set by systemd or manually. + // See https://github.com/pingcap/tidb/issues/22132 + totalMemory, err := memory.MemTotal() + if err != nil { + log.Panic("read memory stat failed", zap.Error(err)) + } for { select { case <-ret.cancelCh: @@ -110,14 +117,8 @@ func newBackEndPool(dir string, captureAddr string) (*backEndPool, error) { metricSorterOpenFileCountGauge.Set(float64(atomic.LoadInt64(&openFDCount))) // update memPressure - m, err := memory.Get() - - failpoint.Inject("getMemoryPressureFails", func() { - m = nil - err = errors.New("injected get memory pressure failure") - }) - - if err != nil { + usedMemory, err := memory.MemUsed() + if err != nil || totalMemory == 0 { failpoint.Inject("sorterDebug", func() { log.Panic("unified sorter: getting system memory usage failed", zap.Error(err)) }) @@ -128,7 +129,7 @@ func newBackEndPool(dir string, captureAddr string) (*backEndPool, error) { // encountered, we can fail gracefully. atomic.StoreInt32(&ret.memPressure, 100) } else { - memPressure := m.Used * 100 / m.Total + memPressure := usedMemory * 100 / totalMemory atomic.StoreInt32(&ret.memPressure, int32(memPressure)) } diff --git a/cdc/sorter/unified/backend_pool_test.go b/cdc/sorter/unified/backend_pool_test.go index 3998eb7c456..e616a23f4a2 100644 --- a/cdc/sorter/unified/backend_pool_test.go +++ b/cdc/sorter/unified/backend_pool_test.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/ticdc/pkg/config" "github.com/pingcap/ticdc/pkg/filelock" "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tidb/util/memory" ) type backendPoolSuite struct{} @@ -325,9 +326,11 @@ func (s *backendPoolSuite) TestCleanUpStaleLockNoPermission(c *check.C) { func (s *backendPoolSuite) TestGetMemoryPressureFailure(c *check.C) { defer testleak.AfterTest(c)() - err := failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/getMemoryPressureFails", "return(true)") - c.Assert(err, check.IsNil) - defer failpoint.Disable("github.com/pingcap/ticdc/cdc/sorter/unified/getMemoryPressureFails") //nolint:errcheck + origin := memory.MemTotal + defer func() { + memory.MemTotal = origin + }() + memory.MemTotal = func() (uint64, error) { return 0, nil } dir := c.MkDir() backEndPool, err := newBackEndPool(dir, "") @@ -336,7 +339,7 @@ func (s *backendPoolSuite) TestGetMemoryPressureFailure(c *check.C) { defer backEndPool.terminate() after := time.After(time.Second * 20) - tick := time.Tick(time.Second * 1) + tick := time.Tick(time.Millisecond * 100) for { select { case <-after: diff --git a/go.mod b/go.mod index 3fb7ca4a3f8..90b9c296732 100644 --- a/go.mod +++ b/go.mod @@ -43,7 +43,6 @@ require ( github.com/labstack/echo/v4 v4.6.1 github.com/lib/pq v1.3.0 // indirect github.com/linkedin/goavro/v2 v2.9.8 - github.com/mackerelio/go-osstat v0.1.0 github.com/mattn/go-colorable v0.1.11 // indirect github.com/mattn/go-shellwords v1.0.12 github.com/mattn/go-sqlite3 v2.0.2+incompatible // indirect diff --git a/go.sum b/go.sum index e083bb4e4d8..e333ecd701f 100644 --- a/go.sum +++ b/go.sum @@ -620,8 +620,6 @@ github.com/lib/pq v1.3.0 h1:/qkRGz8zljWiDcFvgpwUpwIAPu3r07TDvs3Rws+o/pU= github.com/lib/pq v1.3.0/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= github.com/linkedin/goavro/v2 v2.9.8 h1:jN50elxBsGBDGVDEKqUlDuU1cFwJ11K/yrJCBMe/7Wg= github.com/linkedin/goavro/v2 v2.9.8/go.mod h1:UgQUb2N/pmueQYH9bfqFioWxzYCZXSfF8Jw03O5sjqA= -github.com/mackerelio/go-osstat v0.1.0 h1:e57QHeHob8kKJ5FhcXGdzx5O6Ktuc5RHMDIkeqhgkFA= -github.com/mackerelio/go-osstat v0.1.0/go.mod h1:1K3NeYLhMHPvzUu+ePYXtoB58wkaRpxZsGClZBJyIFw= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/mailru/easyjson v0.0.0-20180823135443-60711f1a8329/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.0.0-20190614124828-94de47d64c63/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= @@ -1227,7 +1225,6 @@ golang.org/x/sys v0.0.0-20181228144115-9a3f9b0469bb/go.mod h1:STP8DvDyc/dI5b8T5h golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190410235845-0ad05ae3009d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190422165155-953cdadca894/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= From 70d2a98a349a74875f915aa2bd7b0a0e87f7d686 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Fri, 17 Dec 2021 13:02:36 +0800 Subject: [PATCH 10/48] tz (ticdc): fix timezone error (#3887) (#3906) --- pkg/util/tz.go | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/pkg/util/tz.go b/pkg/util/tz.go index 632e9810860..b1e18043542 100644 --- a/pkg/util/tz.go +++ b/pkg/util/tz.go @@ -14,12 +14,12 @@ package util import ( - "os" "path/filepath" "strings" "time" cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/tidb/util/timeutil" ) // GetTimezone returns the timezone specified by the name @@ -57,9 +57,6 @@ func GetLocalTimezone() (*time.Location, error) { if time.Local.String() != "Local" { return time.Local, nil } - str, err := os.Readlink("/etc/localtime") - if err != nil { - return nil, cerror.WrapError(cerror.ErrLoadTimezone, err) - } + str := timeutil.InferSystemTZ() return getTimezoneFromZonefile(str) } From cc71f8556d6df48313e0e484ac403b6aa70a9967 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Fri, 17 Dec 2021 19:20:37 +0800 Subject: [PATCH 11/48] pkg,cdc: do not use log package (#3902) (#3940) --- .golangci.yml | 12 ++++++++++++ cdc/processor/pipeline/mounter.go | 2 +- cdc/puller/frontier/list.go | 3 ++- cdc/sink/codec/canal.go | 2 +- cdc/sink/common/flow_control.go | 2 +- integration/tests/case_date_time.go | 2 +- pkg/context/context.go | 2 +- pkg/context/context_test.go | 2 +- pkg/etcd/etcdkey.go | 2 +- pkg/orchestrator/util/key_utils.go | 3 ++- 10 files changed, 23 insertions(+), 9 deletions(-) diff --git a/.golangci.yml b/.golangci.yml index f33f66e680c..861c668b3bc 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -3,6 +3,7 @@ linters: - unconvert - unparam - revive + - depguard linters-settings: revive: @@ -28,3 +29,14 @@ linters-settings: - name: superfluous-else - name: modifies-parameter - name: unreachable-code + + depguard: + list-type: blacklist + include-go-root: false + packages: + - log + - github.com/juju/errors + packages-with-error-message: + # specify an error message to output when a blacklisted package is used + - log: "logging is allowed only by pingcap/log" + - github.com/juju/errors: "error handling is allowed only by pingcap/errors" diff --git a/cdc/processor/pipeline/mounter.go b/cdc/processor/pipeline/mounter.go index 1bfc94b43c4..2e156b9cdbd 100644 --- a/cdc/processor/pipeline/mounter.go +++ b/cdc/processor/pipeline/mounter.go @@ -15,12 +15,12 @@ package pipeline import ( "context" - "log" "sync" "time" "github.com/edwingeng/deque" "github.com/pingcap/failpoint" + "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/pkg/notify" "github.com/pingcap/ticdc/pkg/pipeline" diff --git a/cdc/puller/frontier/list.go b/cdc/puller/frontier/list.go index 2a14aeafff8..249127d6bfb 100644 --- a/cdc/puller/frontier/list.go +++ b/cdc/puller/frontier/list.go @@ -16,11 +16,12 @@ package frontier import ( "bytes" "fmt" - "log" "math" "strings" _ "unsafe" // required by go:linkname + + "github.com/pingcap/log" ) const ( diff --git a/cdc/sink/codec/canal.go b/cdc/sink/codec/canal.go index cde3bba4f79..91427c9bb8d 100644 --- a/cdc/sink/codec/canal.go +++ b/cdc/sink/codec/canal.go @@ -16,12 +16,12 @@ package codec import ( "context" "fmt" - "log" "strconv" "strings" "github.com/golang/protobuf/proto" // nolint:staticcheck "github.com/pingcap/errors" + "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/model" cerror "github.com/pingcap/ticdc/pkg/errors" canal "github.com/pingcap/ticdc/proto/canal" diff --git a/cdc/sink/common/flow_control.go b/cdc/sink/common/flow_control.go index 47ad19fc6b7..a008d58b25a 100644 --- a/cdc/sink/common/flow_control.go +++ b/cdc/sink/common/flow_control.go @@ -14,12 +14,12 @@ package common import ( - "log" "sync" "sync/atomic" "github.com/edwingeng/deque" "github.com/pingcap/errors" + "github.com/pingcap/log" cerrors "github.com/pingcap/ticdc/pkg/errors" "go.uber.org/zap" ) diff --git a/integration/tests/case_date_time.go b/integration/tests/case_date_time.go index 8d5f622a398..ac060b94036 100644 --- a/integration/tests/case_date_time.go +++ b/integration/tests/case_date_time.go @@ -15,9 +15,9 @@ package tests import ( "errors" - "log" "time" + "github.com/pingcap/log" "github.com/pingcap/ticdc/integration/framework" "github.com/pingcap/ticdc/integration/framework/avro" "github.com/pingcap/ticdc/integration/framework/canal" diff --git a/pkg/context/context.go b/pkg/context/context.go index 6cf689fa2d8..1930d3f5db4 100644 --- a/pkg/context/context.go +++ b/pkg/context/context.go @@ -15,9 +15,9 @@ package context import ( "context" - "log" "time" + "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/kv" "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/pkg/config" diff --git a/pkg/context/context_test.go b/pkg/context/context_test.go index f274e4fad05..184459e5b18 100644 --- a/pkg/context/context_test.go +++ b/pkg/context/context_test.go @@ -129,7 +129,7 @@ func TestThrowPanic(t *testing.T) { t.Parallel() defer func() { panicMsg := recover() - require.Equal(t, panicMsg, "an error has escaped, please report a bug{error 26 0 mock error}") + require.Equal(t, panicMsg, "an error has escaped, please report a bug") }() stdCtx := context.Background() ctx := NewContext(stdCtx, &GlobalVars{}) diff --git a/pkg/etcd/etcdkey.go b/pkg/etcd/etcdkey.go index bf47a210a30..8b91dc36921 100644 --- a/pkg/etcd/etcdkey.go +++ b/pkg/etcd/etcdkey.go @@ -14,9 +14,9 @@ package etcd import ( - "log" "strings" + "github.com/pingcap/log" cerror "github.com/pingcap/ticdc/pkg/errors" ) diff --git a/pkg/orchestrator/util/key_utils.go b/pkg/orchestrator/util/key_utils.go index e9b68ca491c..008c8ae9705 100644 --- a/pkg/orchestrator/util/key_utils.go +++ b/pkg/orchestrator/util/key_utils.go @@ -14,8 +14,9 @@ package util import ( - "log" "strings" + + "github.com/pingcap/log" ) // EtcdKey represents a complete key in Etcd. From b58d176e10eec2b471ddb18b125835b370b7ee6e Mon Sep 17 00:00:00 2001 From: amyangfei Date: Mon, 20 Dec 2021 11:33:46 +0800 Subject: [PATCH 12/48] *: rename repo from pingcap/ticdc to pingcap/tiflow (#3959) --- Dockerfile | 4 +- Dockerfile.development | 4 +- Makefile | 30 +++---- README.md | 10 +-- cdc/capture/capture.go | 22 ++--- cdc/capture/http_handler.go | 14 ++-- cdc/capture/http_validator.go | 20 ++--- cdc/capture/http_validator_test.go | 4 +- cdc/capture/main_test.go | 2 +- cdc/entry/codec.go | 4 +- cdc/entry/main_test.go | 2 +- cdc/entry/mounter.go | 6 +- cdc/entry/mounter_test.go | 4 +- cdc/entry/schema_storage.go | 8 +- cdc/entry/schema_storage_test.go | 4 +- cdc/http_handler.go | 8 +- cdc/http_router.go | 4 +- cdc/http_router_test.go | 2 +- cdc/http_status.go | 12 +-- cdc/http_status_test.go | 6 +- cdc/kv/client.go | 16 ++-- cdc/kv/client_bench_test.go | 10 +-- cdc/kv/client_mock_test.go | 2 +- cdc/kv/client_test.go | 84 +++++++++---------- cdc/kv/grpc_pool_impl.go | 4 +- cdc/kv/grpc_pool_impl_test.go | 4 +- cdc/kv/matcher_test.go | 2 +- cdc/kv/region_worker.go | 12 +-- cdc/kv/region_worker_test.go | 4 +- cdc/kv/resolvedts_heap_test.go | 2 +- cdc/kv/store_op.go | 8 +- cdc/kv/testing.go | 8 +- cdc/kv/token_region.go | 2 +- cdc/kv/token_region_test.go | 2 +- cdc/metrics.go | 22 ++--- cdc/model/capture.go | 2 +- cdc/model/changefeed.go | 6 +- cdc/model/changefeed_test.go | 4 +- cdc/model/http_model.go | 4 +- cdc/model/http_model_test.go | 2 +- cdc/model/kv.go | 2 +- cdc/model/kv_test.go | 2 +- cdc/model/main_test.go | 2 +- cdc/model/owner.go | 2 +- cdc/model/sink.go | 4 +- cdc/model/string.go | 2 +- cdc/owner/async_sink.go | 10 +-- cdc/owner/async_sink_test.go | 14 ++-- cdc/owner/barrier.go | 2 +- cdc/owner/barrier_test.go | 4 +- cdc/owner/changefeed.go | 14 ++-- cdc/owner/changefeed_test.go | 18 ++-- cdc/owner/ddl_puller.go | 16 ++-- cdc/owner/ddl_puller_test.go | 8 +- cdc/owner/feed_state_manager.go | 6 +- cdc/owner/feed_state_manager_test.go | 10 +-- cdc/owner/owner.go | 12 +-- cdc/owner/owner_test.go | 16 ++-- cdc/owner/scheduler.go | 6 +- cdc/owner/scheduler_test.go | 6 +- cdc/owner/schema.go | 12 +-- cdc/owner/schema_test.go | 8 +- cdc/owner/status_provider.go | 4 +- cdc/processor/manager.go | 8 +- cdc/processor/manager_test.go | 14 ++-- cdc/processor/pipeline/cyclic_mark.go | 8 +- cdc/processor/pipeline/cyclic_mark_test.go | 12 +-- cdc/processor/pipeline/mounter.go | 6 +- cdc/processor/pipeline/mounter_test.go | 10 +-- cdc/processor/pipeline/puller.go | 12 +-- cdc/processor/pipeline/sink.go | 8 +- cdc/processor/pipeline/sink_test.go | 12 +-- cdc/processor/pipeline/sorter.go | 14 ++-- cdc/processor/pipeline/sorter_test.go | 16 ++-- cdc/processor/pipeline/table.go | 16 ++-- cdc/processor/processor.go | 32 +++---- cdc/processor/processor_test.go | 18 ++-- cdc/puller/frontier/frontier.go | 2 +- cdc/puller/frontier/frontier_bench_test.go | 2 +- cdc/puller/frontier/frontier_test.go | 4 +- cdc/puller/frontier/heap_test.go | 2 +- cdc/puller/frontier/list_test.go | 2 +- cdc/puller/mock_puller.go | 4 +- cdc/puller/mock_puller_test.go | 6 +- cdc/puller/puller.go | 14 ++-- cdc/puller/puller_test.go | 16 ++-- cdc/redo/applier.go | 4 +- cdc/redo/common/util.go | 2 +- cdc/redo/convert.go | 2 +- cdc/redo/convert_test.go | 2 +- cdc/redo/manager.go | 10 +-- cdc/redo/manager_test.go | 4 +- cdc/redo/reader/blackhole_reader.go | 2 +- cdc/redo/reader/file.go | 8 +- cdc/redo/reader/file_test.go | 8 +- cdc/redo/reader/mock_RedoLogReader.go | 2 +- cdc/redo/reader/mock_fileReader.go | 2 +- cdc/redo/reader/reader.go | 6 +- cdc/redo/reader/reader_test.go | 6 +- cdc/redo/writer/blackhole_writer.go | 2 +- cdc/redo/writer/file.go | 4 +- cdc/redo/writer/file_test.go | 4 +- cdc/redo/writer/mock_RedoLogWriter.go | 2 +- cdc/redo/writer/writer.go | 6 +- cdc/redo/writer/writer_test.go | 6 +- cdc/server.go | 18 ++-- cdc/server_test.go | 10 +-- cdc/sink/black_hole.go | 2 +- cdc/sink/buffer_sink.go | 4 +- cdc/sink/causality.go | 2 +- cdc/sink/causality_test.go | 4 +- cdc/sink/cdclog/file.go | 8 +- cdc/sink/cdclog/s3.go | 8 +- cdc/sink/cdclog/utils.go | 6 +- cdc/sink/codec/avro.go | 8 +- cdc/sink/codec/avro_test.go | 10 +-- cdc/sink/codec/canal.go | 6 +- cdc/sink/codec/canal_flat.go | 6 +- cdc/sink/codec/canal_flat_test.go | 4 +- cdc/sink/codec/canal_test.go | 6 +- cdc/sink/codec/codec_test.go | 8 +- cdc/sink/codec/craft.go | 6 +- cdc/sink/codec/craft/codec_test.go | 2 +- cdc/sink/codec/craft/decoder.go | 4 +- cdc/sink/codec/craft/encoder.go | 2 +- cdc/sink/codec/craft/model.go | 4 +- cdc/sink/codec/craft_test.go | 4 +- cdc/sink/codec/interface.go | 4 +- cdc/sink/codec/interface_test.go | 4 +- cdc/sink/codec/json.go | 4 +- cdc/sink/codec/json_test.go | 4 +- cdc/sink/codec/maxwell.go | 4 +- cdc/sink/codec/maxwell_test.go | 4 +- cdc/sink/codec/schema_registry.go | 8 +- cdc/sink/codec/schema_registry_test.go | 6 +- cdc/sink/common/common.go | 4 +- cdc/sink/common/common_test.go | 4 +- cdc/sink/common/flow_control.go | 2 +- cdc/sink/common/flow_control_test.go | 2 +- cdc/sink/dispatcher/default.go | 2 +- cdc/sink/dispatcher/default_test.go | 4 +- cdc/sink/dispatcher/dispatcher.go | 6 +- cdc/sink/dispatcher/index_value.go | 4 +- cdc/sink/dispatcher/index_value_test.go | 4 +- cdc/sink/dispatcher/switcher_test.go | 6 +- cdc/sink/dispatcher/table.go | 4 +- cdc/sink/dispatcher/table_test.go | 4 +- cdc/sink/dispatcher/ts.go | 2 +- cdc/sink/dispatcher/ts_test.go | 4 +- cdc/sink/manager.go | 4 +- cdc/sink/manager_test.go | 8 +- cdc/sink/mq.go | 22 ++--- cdc/sink/mq_test.go | 14 ++-- cdc/sink/mysql.go | 26 +++--- cdc/sink/mysql_test.go | 18 ++-- cdc/sink/producer/kafka/kafka.go | 14 ++-- cdc/sink/producer/kafka/kafka_test.go | 12 +-- cdc/sink/producer/mq_producer.go | 2 +- cdc/sink/producer/pulsar/producer.go | 4 +- cdc/sink/simple_mysql_tester.go | 10 +-- cdc/sink/sink.go | 10 +-- cdc/sink/sink_test.go | 4 +- cdc/sink/statistics.go | 2 +- cdc/sink/syncpointStore.go | 4 +- cdc/sink/table_sink.go | 4 +- cdc/sink/txns_heap.go | 2 +- cdc/sink/txns_heap_test.go | 4 +- cdc/sorter/encoding/key.go | 2 +- cdc/sorter/encoding/key_test.go | 2 +- cdc/sorter/encoding/value.go | 2 +- cdc/sorter/memory/entry_sorter.go | 6 +- cdc/sorter/memory/entry_sorter_test.go | 4 +- cdc/sorter/sorter.go | 2 +- cdc/sorter/unified/backend.go | 2 +- cdc/sorter/unified/backend_pool.go | 10 +-- cdc/sorter/unified/backend_pool_test.go | 20 ++--- cdc/sorter/unified/file_backend.go | 6 +- cdc/sorter/unified/file_backend_test.go | 8 +- cdc/sorter/unified/heap.go | 2 +- cdc/sorter/unified/heap_sorter.go | 10 +-- cdc/sorter/unified/memory_backend.go | 2 +- cdc/sorter/unified/memory_backend_test.go | 4 +- cdc/sorter/unified/merger.go | 8 +- cdc/sorter/unified/merger_test.go | 28 +++---- cdc/sorter/unified/sorter_test.go | 40 ++++----- cdc/sorter/unified/unified_sorter.go | 8 +- chaos/Dockerfile.chaos | 4 +- cmd/cdc/main.go | 2 +- cmd/kafka-consumer/main.go | 18 ++-- demo/java/README.md | 4 +- dm/_utils/terror_gen/checker_template.go | 2 +- dm/chaos/cases/cases.go | 6 +- dm/chaos/cases/config.go | 2 +- dm/chaos/cases/db.go | 10 +-- dm/chaos/cases/diff.go | 2 +- dm/chaos/cases/generator.go | 2 +- dm/chaos/cases/instance.go | 4 +- dm/chaos/cases/main.go | 4 +- dm/chaos/cases/member.go | 4 +- dm/chaos/cases/source.go | 4 +- dm/chaos/cases/task.go | 10 +-- dm/checker/check_test.go | 6 +- dm/checker/checker.go | 18 ++-- dm/checker/cmd.go | 6 +- dm/cmd/dm-ctl/main.go | 10 +-- dm/cmd/dm-master/main.go | 10 +-- dm/cmd/dm-master/main_test.go | 2 +- dm/cmd/dm-portal/main.go | 6 +- dm/cmd/dm-syncer/config.go | 6 +- dm/cmd/dm-syncer/main.go | 14 ++-- dm/cmd/dm-worker/main.go | 10 +-- dm/cmd/dm-worker/main_test.go | 4 +- .../binlog-event-blackhole/fetcher.go | 2 +- dm/debug-tools/binlog-event-blackhole/main.go | 2 +- dm/dm/common/common.go | 2 +- dm/dm/config/checking_item.go | 2 +- dm/dm/config/source_config.go | 10 +-- dm/dm/config/source_config_test.go | 2 +- dm/dm/config/subtask.go | 8 +- dm/dm/config/task.go | 6 +- dm/dm/config/task_converters.go | 6 +- dm/dm/config/task_converters_test.go | 4 +- dm/dm/config/task_test.go | 2 +- dm/dm/ctl/common/config.go | 4 +- dm/dm/ctl/common/operate_relay.go | 2 +- dm/dm/ctl/common/operate_task.go | 2 +- dm/dm/ctl/common/util.go | 12 +-- dm/dm/ctl/ctl.go | 8 +- dm/dm/ctl/master/binlog.go | 4 +- dm/dm/ctl/master/check_task.go | 6 +- dm/dm/ctl/master/config.go | 10 +-- dm/dm/ctl/master/get_config.go | 4 +- dm/dm/ctl/master/handle_error.go | 6 +- dm/dm/ctl/master/list_member.go | 4 +- dm/dm/ctl/master/offline_member.go | 4 +- dm/dm/ctl/master/operate_leader.go | 4 +- dm/dm/ctl/master/operate_schema.go | 4 +- dm/dm/ctl/master/operate_source.go | 10 +-- dm/dm/ctl/master/operate_task.go | 4 +- dm/dm/ctl/master/pause_relay.go | 4 +- dm/dm/ctl/master/pause_task.go | 2 +- dm/dm/ctl/master/purge_relay.go | 6 +- dm/dm/ctl/master/query_status.go | 4 +- dm/dm/ctl/master/query_status_test.go | 2 +- dm/dm/ctl/master/resume_relay.go | 4 +- dm/dm/ctl/master/resume_task.go | 2 +- dm/dm/ctl/master/show_ddl_locks.go | 4 +- dm/dm/ctl/master/source_table_schema.go | 4 +- dm/dm/ctl/master/start_stop_relay.go | 4 +- dm/dm/ctl/master/start_task.go | 12 +-- dm/dm/ctl/master/stop_task.go | 2 +- dm/dm/ctl/master/transfer_source.go | 4 +- dm/dm/ctl/master/unlock_ddl_lock.go | 4 +- dm/dm/master/agent_pool.go | 2 +- dm/dm/master/bootstrap.go | 22 ++--- dm/dm/master/bootstrap_test.go | 12 +-- dm/dm/master/config.go | 8 +- dm/dm/master/config_test.go | 4 +- dm/dm/master/election.go | 6 +- dm/dm/master/election_test.go | 10 +-- dm/dm/master/etcd.go | 6 +- dm/dm/master/etcd_test.go | 6 +- dm/dm/master/http_handler.go | 8 +- dm/dm/master/metrics/metrics.go | 2 +- dm/dm/master/openapi.go | 22 ++--- dm/dm/master/openapi_test.go | 26 +++--- dm/dm/master/scheduler/scheduler.go | 16 ++-- dm/dm/master/scheduler/scheduler_test.go | 22 ++--- dm/dm/master/scheduler/worker.go | 12 +-- dm/dm/master/scheduler/worker_test.go | 10 +-- dm/dm/master/server.go | 36 ++++---- dm/dm/master/server_test.go | 44 +++++----- dm/dm/master/shardddl/info.go | 2 +- dm/dm/master/shardddl/info_test.go | 2 +- dm/dm/master/shardddl/optimist.go | 18 ++-- dm/dm/master/shardddl/optimist_test.go | 10 +-- dm/dm/master/shardddl/pessimist.go | 14 ++-- dm/dm/master/shardddl/pessimist_test.go | 14 ++-- dm/dm/master/workerrpc/interface.go | 2 +- dm/dm/master/workerrpc/rawgrpc.go | 6 +- dm/dm/master/workerrpc/workerrpc_test.go | 8 +- dm/dm/pbmock/dmmaster.go | 4 +- dm/dm/pbmock/dmworker.go | 4 +- dm/dm/portal/api.go | 6 +- dm/dm/portal/api_test.go | 2 +- dm/dm/portal/config.go | 6 +- dm/dm/unit/unit.go | 8 +- dm/dm/unit/unit_test.go | 4 +- dm/dm/worker/config.go | 8 +- dm/dm/worker/config_test.go | 2 +- dm/dm/worker/join.go | 10 +-- dm/dm/worker/metrics.go | 16 ++-- dm/dm/worker/relay.go | 16 ++-- dm/dm/worker/relay_test.go | 18 ++-- dm/dm/worker/server.go | 24 +++--- dm/dm/worker/server_test.go | 28 +++---- dm/dm/worker/source_worker.go | 22 ++--- dm/dm/worker/source_worker_test.go | 46 +++++----- dm/dm/worker/status.go | 6 +- dm/dm/worker/subtask.go | 26 +++--- dm/dm/worker/subtask_test.go | 18 ++-- dm/dm/worker/task_checker.go | 12 +-- dm/dm/worker/task_checker_test.go | 10 +-- dm/dm/worker/v1meta.go | 6 +- dm/dumpling/dumpling.go | 18 ++-- dm/dumpling/dumpling_test.go | 26 +++--- dm/dumpling/metrics.go | 2 +- dm/dumpling/util.go | 4 +- dm/dumpling/util_test.go | 6 +- dm/loader/checkpoint.go | 12 +-- dm/loader/checkpoint_test.go | 8 +- dm/loader/convert_data.go | 8 +- dm/loader/convert_data_test.go | 2 +- dm/loader/db.go | 14 ++-- dm/loader/lightning.go | 16 ++-- dm/loader/loader.go | 18 ++-- dm/loader/metrics.go | 2 +- dm/loader/status.go | 4 +- dm/loader/status_test.go | 4 +- dm/loader/util.go | 12 +-- dm/openapi/fixtures/task.go | 2 +- dm/openapi/task.go | 2 +- dm/openapi/task_test.go | 2 +- dm/pkg/backoff/backoff.go | 2 +- dm/pkg/backoff/backoff_test.go | 2 +- dm/pkg/binlog/event/common.go | 4 +- dm/pkg/binlog/event/common_test.go | 2 +- dm/pkg/binlog/event/ddl.go | 4 +- dm/pkg/binlog/event/ddl_test.go | 2 +- dm/pkg/binlog/event/dml.go | 4 +- dm/pkg/binlog/event/dml_test.go | 2 +- dm/pkg/binlog/event/event.go | 4 +- dm/pkg/binlog/event/event_test.go | 2 +- dm/pkg/binlog/event/generator.go | 4 +- dm/pkg/binlog/event/generator_test.go | 2 +- dm/pkg/binlog/event/helper.go | 4 +- dm/pkg/binlog/event/helper_test.go | 4 +- dm/pkg/binlog/event/sid_mysql.go | 2 +- dm/pkg/binlog/event/util.go | 2 +- dm/pkg/binlog/event/util_test.go | 2 +- dm/pkg/binlog/filename.go | 2 +- dm/pkg/binlog/position.go | 8 +- dm/pkg/binlog/position_test.go | 2 +- dm/pkg/binlog/reader/file.go | 8 +- dm/pkg/binlog/reader/file_test.go | 8 +- dm/pkg/binlog/reader/mock.go | 2 +- dm/pkg/binlog/reader/reader.go | 2 +- dm/pkg/binlog/reader/tcp.go | 10 +-- dm/pkg/binlog/reader/tcp_test.go | 24 +++--- dm/pkg/binlog/reader/util.go | 12 +-- dm/pkg/binlog/status.go | 2 +- dm/pkg/binlog/writer/file.go | 6 +- dm/pkg/binlog/writer/file_test.go | 4 +- dm/pkg/conn/baseconn.go | 12 +-- dm/pkg/conn/baseconn_test.go | 8 +- dm/pkg/conn/basedb.go | 8 +- dm/pkg/conn/basedb_test.go | 8 +- dm/pkg/conn/mockdb.go | 2 +- dm/pkg/conn/utils.go | 2 +- dm/pkg/conn/utils_test.go | 2 +- dm/pkg/context/context.go | 2 +- dm/pkg/dumpling/utils.go | 6 +- dm/pkg/dumpling/utils_test.go | 4 +- dm/pkg/election/election.go | 4 +- dm/pkg/election/election_test.go | 14 ++-- dm/pkg/encrypt/encrypt.go | 2 +- dm/pkg/etcdutil/etcdutil.go | 6 +- dm/pkg/etcdutil/etcdutil_test.go | 8 +- dm/pkg/func-rollback/rollback.go | 2 +- dm/pkg/gtid/gtid.go | 2 +- dm/pkg/gtid/gtid_test.go | 2 +- dm/pkg/ha/bound.go | 10 +-- dm/pkg/ha/bound_test.go | 2 +- dm/pkg/ha/keepalive.go | 6 +- dm/pkg/ha/keepalive_test.go | 2 +- dm/pkg/ha/load_task.go | 4 +- dm/pkg/ha/ops.go | 4 +- dm/pkg/ha/ops_test.go | 4 +- dm/pkg/ha/relay.go | 10 +-- dm/pkg/ha/relay_test.go | 2 +- dm/pkg/ha/source.go | 8 +- dm/pkg/ha/source_test.go | 2 +- dm/pkg/ha/stage.go | 12 +-- dm/pkg/ha/stage_test.go | 4 +- dm/pkg/ha/subtask.go | 8 +- dm/pkg/ha/subtask_test.go | 2 +- dm/pkg/ha/worker.go | 4 +- dm/pkg/log/log.go | 4 +- dm/pkg/parser/common.go | 6 +- dm/pkg/parser/common_test.go | 4 +- dm/pkg/retry/errors.go | 2 +- dm/pkg/retry/strategy.go | 2 +- dm/pkg/retry/strategy_test.go | 4 +- dm/pkg/schema/tracker.go | 8 +- dm/pkg/schema/tracker_test.go | 4 +- dm/pkg/shardddl/optimism/column.go | 4 +- dm/pkg/shardddl/optimism/info.go | 6 +- dm/pkg/shardddl/optimism/info_test.go | 4 +- dm/pkg/shardddl/optimism/keeper.go | 4 +- dm/pkg/shardddl/optimism/keeper_test.go | 2 +- dm/pkg/shardddl/optimism/lock.go | 6 +- dm/pkg/shardddl/optimism/lock_test.go | 6 +- dm/pkg/shardddl/optimism/operation.go | 4 +- dm/pkg/shardddl/optimism/ops.go | 6 +- dm/pkg/shardddl/optimism/schema.go | 4 +- dm/pkg/shardddl/optimism/table.go | 4 +- dm/pkg/shardddl/pessimism/info.go | 6 +- dm/pkg/shardddl/pessimism/info_test.go | 4 +- dm/pkg/shardddl/pessimism/keeper.go | 4 +- dm/pkg/shardddl/pessimism/lock.go | 6 +- dm/pkg/shardddl/pessimism/lock_test.go | 2 +- dm/pkg/shardddl/pessimism/operation.go | 4 +- dm/pkg/shardddl/pessimism/ops.go | 4 +- dm/pkg/streamer/file.go | 8 +- dm/pkg/streamer/file_test.go | 4 +- dm/pkg/streamer/hub.go | 6 +- dm/pkg/streamer/reader.go | 16 ++-- dm/pkg/streamer/reader_test.go | 14 ++-- dm/pkg/streamer/streamer.go | 10 +-- dm/pkg/streamer/streamer_test.go | 12 +-- dm/pkg/streamer/util.go | 4 +- dm/pkg/upgrade/upgrade.go | 16 ++-- dm/pkg/upgrade/upgrade_test.go | 2 +- dm/pkg/upgrade/version.go | 8 +- dm/pkg/utils/common.go | 4 +- dm/pkg/utils/db.go | 6 +- dm/pkg/utils/db_test.go | 2 +- dm/pkg/utils/encrypt.go | 4 +- dm/pkg/utils/encrypt_test.go | 2 +- dm/pkg/utils/file.go | 4 +- dm/pkg/utils/file_test.go | 2 +- dm/pkg/utils/printer.go | 2 +- dm/pkg/utils/printer_test.go | 2 +- dm/pkg/utils/relay.go | 2 +- dm/pkg/utils/storage_unix.go | 2 +- dm/pkg/utils/storage_windows.go | 2 +- dm/pkg/utils/time.go | 2 +- dm/pkg/utils/util.go | 6 +- dm/pkg/v1dbschema/schema.go | 20 ++--- dm/pkg/v1dbschema/schema_test.go | 16 ++-- dm/pkg/v1workermeta/api.go | 8 +- dm/pkg/v1workermeta/api_test.go | 6 +- dm/pkg/v1workermeta/db.go | 2 +- dm/pkg/v1workermeta/meta.go | 4 +- dm/pkg/v1workermeta/meta_test.go | 6 +- dm/relay/common/util.go | 4 +- dm/relay/config.go | 6 +- dm/relay/meta.go | 8 +- dm/relay/meta_test.go | 2 +- dm/relay/metrics.go | 8 +- dm/relay/purger/file.go | 8 +- dm/relay/purger/file_test.go | 6 +- dm/relay/purger/purger.go | 12 +-- dm/relay/purger/purger_test.go | 8 +- dm/relay/purger/strategy.go | 2 +- dm/relay/purger/strategy_filename.go | 8 +- dm/relay/purger/strategy_inactive.go | 6 +- dm/relay/purger/strategy_space.go | 8 +- dm/relay/purger/strategy_time.go | 6 +- dm/relay/reader/reader.go | 10 +-- dm/relay/reader/reader_test.go | 2 +- dm/relay/relay.go | 32 +++---- dm/relay/relay_test.go | 32 +++---- dm/relay/retry/reader.go | 6 +- dm/relay/transformer/transformer.go | 2 +- dm/relay/transformer/transformer_test.go | 4 +- dm/relay/util.go | 2 +- dm/relay/util_test.go | 4 +- dm/relay/writer/file.go | 14 ++-- dm/relay/writer/file_test.go | 8 +- dm/relay/writer/file_util.go | 10 +-- dm/relay/writer/file_util_test.go | 4 +- dm/relay/writer/writer.go | 2 +- dm/syncer/causality.go | 4 +- dm/syncer/causality_test.go | 10 +-- dm/syncer/checkpoint.go | 24 +++--- dm/syncer/checkpoint_test.go | 18 ++-- dm/syncer/compactor.go | 4 +- dm/syncer/compactor_test.go | 14 ++-- dm/syncer/dbconn/db.go | 16 ++-- dm/syncer/dbconn/upstream_db.go | 14 ++-- dm/syncer/dbconn/upstream_db_test.go | 4 +- dm/syncer/ddl.go | 10 +-- dm/syncer/ddl_test.go | 14 ++-- dm/syncer/dml.go | 4 +- dm/syncer/dml_worker.go | 12 +-- dm/syncer/err-operator/operator.go | 8 +- dm/syncer/err-operator/operator_test.go | 8 +- dm/syncer/error.go | 6 +- dm/syncer/error_test.go | 6 +- dm/syncer/expr_filter_group.go | 6 +- dm/syncer/expr_filter_group_test.go | 8 +- dm/syncer/filter.go | 6 +- dm/syncer/filter_test.go | 12 +-- dm/syncer/handle_error.go | 8 +- dm/syncer/handle_error_test.go | 6 +- dm/syncer/job.go | 2 +- dm/syncer/job_test.go | 2 +- dm/syncer/metrics/metrics.go | 2 +- dm/syncer/mode.go | 4 +- dm/syncer/online-ddl-tools/online_ddl.go | 16 ++-- dm/syncer/optimist.go | 6 +- dm/syncer/relay.go | 10 +-- dm/syncer/safe-mode/mode.go | 6 +- dm/syncer/safe-mode/mode_test.go | 2 +- dm/syncer/schema.go | 12 +-- dm/syncer/shardddl/optimist.go | 4 +- dm/syncer/shardddl/optimist_test.go | 6 +- dm/syncer/shardddl/pessimist.go | 6 +- dm/syncer/shardddl/pessimist_test.go | 8 +- dm/syncer/sharding-meta/shardmeta.go | 10 +-- dm/syncer/sharding-meta/shardmeta_test.go | 2 +- dm/syncer/sharding_group.go | 20 ++--- dm/syncer/sharding_group_test.go | 20 ++--- dm/syncer/status.go | 10 +-- dm/syncer/status_test.go | 12 +-- dm/syncer/streamer_controller.go | 18 ++-- dm/syncer/syncer.go | 50 +++++------ dm/syncer/syncer_test.go | 34 ++++---- dm/syncer/test_injector.go | 4 +- dm/syncer/util.go | 6 +- .../_dmctl_tools/check_exit_safe_binlog.go | 8 +- .../_dmctl_tools/check_master_http_apis.go | 2 +- dm/tests/_dmctl_tools/check_master_online.go | 4 +- .../_dmctl_tools/check_master_online_http.go | 2 +- dm/tests/_dmctl_tools/check_worker_online.go | 4 +- dm/tests/_utils/ha_cases_lib.sh | 4 +- dm/tests/adjust_gtid/run.sh | 2 +- dm/tests/all_mode/run.sh | 26 +++--- dm/tests/case_sensitive/run.sh | 4 +- dm/tests/checkpoint_transaction/run.sh | 2 +- dm/tests/dmctl_basic/run.sh | 2 +- dm/tests/drop_column_with_index/run.sh | 4 +- dm/tests/duplicate_event/run.sh | 4 +- dm/tests/fake_rotate_event/run.sh | 2 +- dm/tests/full_mode/run.sh | 4 +- dm/tests/generate-mock.sh | 2 +- dm/tests/ha_cases_1/run.sh | 4 +- dm/tests/import_goroutine_leak/run.sh | 36 ++++---- dm/tests/incremental_mode/run.sh | 8 +- dm/tests/initial_unit/run.sh | 4 +- dm/tests/lightning_mode/run.sh | 4 +- dm/tests/load_interrupt/run.sh | 6 +- dm/tests/load_task/run.sh | 10 +-- dm/tests/metrics/run.sh | 12 +-- dm/tests/new_relay/run.sh | 4 +- dm/tests/online_ddl/run.sh | 4 +- dm/tests/only_dml/run.sh | 2 +- dm/tests/print_status/run.sh | 6 +- dm/tests/relay_interrupt/run.sh | 6 +- dm/tests/retry_cancel/run.sh.todo | 6 +- dm/tests/safe_mode/run.sh | 10 +-- dm/tests/sequence_safe_mode/run.sh | 8 +- dm/tests/sequence_sharding_optimistic/run.sh | 2 +- dm/tests/shardddl1/run.sh | 4 +- dm/tests/shardddl2/run.sh | 4 +- dm/tests/sharding/run.sh | 2 +- dm/tests/sharding2/run.sh | 4 +- dm/tests/start_task/run.sh | 4 +- dm/tests/utils/dmctl.go | 2 +- dm/tests/utils/log.go | 2 +- ...5-ticdc-high-availability-new-design-cn.md | 6 +- ...021-10-13-ticdc-mq-sink-column-selector.md | 2 +- docs/design/README.md | 4 +- docs/design/TEMPLATE.md | 4 +- docs/design/package.json | 6 +- go.mod | 2 +- .../framework/avro/kafka_docker_env.go | 2 +- .../framework/avro/kafka_docker_env_test.go | 2 +- .../framework/avro/kafka_single_table.go | 2 +- .../framework/avro/kafka_single_table_test.go | 2 +- .../framework/canal/kafka_docker_env.go | 2 +- .../framework/canal/kafka_docker_env_test.go | 2 +- .../framework/canal/kafka_single_table.go | 2 +- .../canal/kafka_single_table_test.go | 2 +- integration/framework/docker_compose_op.go | 4 +- integration/framework/docker_env.go | 4 +- integration/framework/mysql/docker_env.go | 2 +- .../framework/mysql/docker_env_test.go | 2 +- integration/framework/mysql/single_table.go | 2 +- .../framework/mysql/single_table_test.go | 2 +- integration/framework/sql_helper.go | 2 +- integration/integration.go | 12 +-- integration/tests/case_alter.go | 2 +- integration/tests/case_composite_pkey.go | 2 +- integration/tests/case_date_time.go | 8 +- integration/tests/case_delete.go | 2 +- integration/tests/case_many_types.go | 8 +- integration/tests/case_simple.go | 2 +- integration/tests/case_unsigned.go | 2 +- kafka_consumer/Dockerfile | 4 +- pkg/actor/actor.go | 4 +- pkg/actor/actor_test.go | 4 +- pkg/actor/message/message.go | 2 +- pkg/actor/message/message_test.go | 2 +- pkg/actor/system.go | 4 +- pkg/actor/system_test.go | 8 +- pkg/applier/redo.go | 14 ++-- pkg/applier/redo_test.go | 8 +- pkg/cmd/cli/cli.go | 6 +- pkg/cmd/cli/cli_capture.go | 6 +- pkg/cmd/cli/cli_capture_list.go | 10 +-- pkg/cmd/cli/cli_changefeed.go | 6 +- pkg/cmd/cli/cli_changefeed_create.go | 28 +++---- pkg/cmd/cli/cli_changefeed_create_test.go | 4 +- pkg/cmd/cli/cli_changefeed_cyclic.go | 2 +- ...cli_changefeed_cyclic_create_marktables.go | 10 +-- pkg/cmd/cli/cli_changefeed_helper.go | 20 ++--- pkg/cmd/cli/cli_changefeed_helper_test.go | 2 +- pkg/cmd/cli/cli_changefeed_list.go | 12 +-- pkg/cmd/cli/cli_changefeed_pause.go | 10 +-- pkg/cmd/cli/cli_changefeed_query.go | 14 ++-- pkg/cmd/cli/cli_changefeed_remove.go | 10 +-- pkg/cmd/cli/cli_changefeed_resume.go | 12 +-- pkg/cmd/cli/cli_changefeed_statistics.go | 8 +- pkg/cmd/cli/cli_changefeed_update.go | 12 +-- pkg/cmd/cli/cli_changefeed_update_test.go | 4 +- pkg/cmd/cli/cli_processor.go | 6 +- pkg/cmd/cli/cli_processor_list.go | 8 +- pkg/cmd/cli/cli_processor_query.go | 12 +-- pkg/cmd/cli/cli_test.go | 2 +- pkg/cmd/cli/cli_tso.go | 2 +- pkg/cmd/cli/cli_tso_query.go | 4 +- pkg/cmd/cli/cli_unsafe.go | 2 +- .../cli_unsafe_delete_service_gc_safepoint.go | 6 +- pkg/cmd/cli/cli_unsafe_reset.go | 8 +- pkg/cmd/cli/cli_unsafe_show_metadata.go | 6 +- pkg/cmd/cmd.go | 8 +- pkg/cmd/factory/factory.go | 6 +- pkg/cmd/factory/factory_impl.go | 12 +-- pkg/cmd/redo/apply.go | 4 +- pkg/cmd/redo/meta.go | 4 +- pkg/cmd/redo/redo.go | 4 +- pkg/cmd/server/server.go | 20 ++--- pkg/cmd/server/server_test.go | 4 +- pkg/cmd/util/helper.go | 8 +- pkg/cmd/util/helper_test.go | 4 +- pkg/cmd/version/version.go | 2 +- pkg/config/config.go | 6 +- pkg/config/main_test.go | 2 +- pkg/config/sorter.go | 2 +- pkg/context/context.go | 12 +-- pkg/context/context_test.go | 4 +- pkg/context/main_test.go | 2 +- pkg/cyclic/filter.go | 4 +- pkg/cyclic/filter_test.go | 4 +- pkg/cyclic/main_test.go | 2 +- pkg/cyclic/mark/main_test.go | 2 +- pkg/cyclic/mark/mark.go | 6 +- pkg/cyclic/replication.go | 8 +- pkg/cyclic/replication_test.go | 6 +- pkg/errors/main_test.go | 2 +- pkg/etcd/client.go | 4 +- pkg/etcd/client_test.go | 2 +- pkg/etcd/etcd.go | 6 +- pkg/etcd/etcd_test.go | 8 +- pkg/etcd/etcdkey.go | 2 +- pkg/etcd/etcdkey_test.go | 2 +- pkg/filelock/filelock.go | 2 +- pkg/filelock/filelock_test.go | 2 +- pkg/filter/filter.go | 6 +- pkg/filter/filter_test.go | 2 +- pkg/filter/main_test.go | 2 +- pkg/flags/main_test.go | 2 +- pkg/flags/urls.go | 2 +- pkg/httputil/httputil.go | 2 +- pkg/httputil/httputil_test.go | 2 +- pkg/httputil/main_test.go | 2 +- pkg/leakutil/leak_helper_test.go | 2 +- pkg/logutil/log.go | 4 +- pkg/logutil/log_test.go | 2 +- pkg/notify/main_test.go | 2 +- pkg/notify/notify.go | 2 +- pkg/notify/notify_test.go | 2 +- pkg/orchestrator/etcd_worker.go | 6 +- pkg/orchestrator/etcd_worker_bank_test.go | 6 +- pkg/orchestrator/etcd_worker_test.go | 8 +- pkg/orchestrator/interfaces.go | 2 +- pkg/orchestrator/reactor_state.go | 8 +- pkg/orchestrator/reactor_state_test.go | 8 +- pkg/orchestrator/reactor_state_tester.go | 4 +- pkg/orchestrator/util/key_utils_test.go | 2 +- pkg/pdtime/acquirer.go | 2 +- pkg/pdtime/main_test.go | 2 +- pkg/pipeline/context.go | 2 +- pkg/pipeline/main_test.go | 2 +- pkg/pipeline/message.go | 2 +- pkg/pipeline/pipeline.go | 4 +- pkg/pipeline/pipeline_test.go | 8 +- pkg/pipeline/runner.go | 2 +- pkg/pipeline/test.go | 2 +- pkg/quotes/main_test.go | 2 +- pkg/regionspan/span.go | 2 +- pkg/retry/main_test.go | 2 +- pkg/retry/retry_with_opt.go | 2 +- pkg/scheduler/interface.go | 2 +- pkg/scheduler/main_test.go | 2 +- pkg/scheduler/table_number.go | 2 +- pkg/scheduler/table_number_test.go | 2 +- pkg/scheduler/workload.go | 2 +- pkg/scheduler/workload_test.go | 2 +- pkg/security/credential.go | 2 +- pkg/tcpserver/main_test.go | 2 +- pkg/tcpserver/tcp_server.go | 4 +- pkg/tcpserver/tcp_server_test.go | 4 +- pkg/txnutil/gc/gc_manager.go | 8 +- pkg/txnutil/gc/gc_manager_test.go | 8 +- pkg/txnutil/gc/gc_service.go | 4 +- pkg/txnutil/gc/gc_service_test.go | 2 +- pkg/types/main_test.go | 2 +- pkg/types/urls.go | 2 +- pkg/util/bitflag_test.go | 2 +- pkg/util/ctx_test.go | 2 +- pkg/util/fileutil.go | 4 +- pkg/util/fileutil_test.go | 8 +- pkg/util/overlap_merge_test.go | 2 +- pkg/util/test_helper_test.go | 2 +- pkg/util/testleak/leaktest.go | 8 +- pkg/util/tz.go | 2 +- pkg/util/tz_test.go | 2 +- pkg/version/check.go | 8 +- pkg/version/check_test.go | 2 +- pkg/version/main_test.go | 2 +- pkg/version/version.go | 2 +- pkg/workerpool/async_pool_impl.go | 4 +- pkg/workerpool/main_test.go | 2 +- pkg/workerpool/pool_impl.go | 4 +- pkg/workerpool/pool_test.go | 12 +-- .../cdc_state_checker/cdc_monitor.go | 6 +- testing_utils/cdc_state_checker/main.go | 2 +- testing_utils/cdc_state_checker/reactor.go | 4 +- testing_utils/cdc_state_checker/state.go | 8 +- .../many_sorters_test/many_sorters.go | 12 +-- .../sorter_stress_test/sorter_stress.go | 10 +-- .../_utils/run_cdc_cli_tso_query | 2 +- tests/integration_tests/availability/owner.sh | 6 +- tests/integration_tests/bank/case.go | 4 +- .../capture_session_done_during_task/run.sh | 2 +- .../run.sh | 2 +- tests/integration_tests/cdc/cdc.go | 4 +- .../changefeed_auto_stop/run.sh | 2 +- .../integration_tests/changefeed_error/run.sh | 8 +- .../changefeed_fast_fail/run.sh | 2 +- .../consistent_replicate_nfs/run.sh | 2 +- .../consistent_replicate_s3/run.sh | 2 +- tests/integration_tests/dailytest/db.go | 2 +- tests/integration_tests/ddl_puller_lag/run.sh | 2 +- .../force_replicate_table/run.sh | 2 +- tests/integration_tests/gc_safepoint/run.sh | 2 +- .../kafka_sink_error_resume/run.sh | 2 +- .../kill_owner_with_ddl/run.sh | 2 +- .../kv_client_stream_reconnect/run.sh | 2 +- tests/integration_tests/many_pk_or_uk/main.go | 2 +- tests/integration_tests/move_table/main.go | 6 +- tests/integration_tests/multi_source/main.go | 2 +- .../owner_remove_table_error/run.sh | 2 +- .../processor_err_chan/run.sh | 2 +- .../integration_tests/processor_panic/main.go | 4 +- .../integration_tests/processor_panic/run.sh | 2 +- .../processor_resolved_ts_fallback/run.sh | 2 +- .../processor_stop_delay/run.sh | 2 +- tests/integration_tests/resolve_lock/main.go | 2 +- tests/integration_tests/sink_hang/run.sh | 2 +- tests/integration_tests/sink_retry/run.sh | 2 +- tools/check/check-errdoc.sh | 2 +- 765 files changed, 2539 insertions(+), 2539 deletions(-) diff --git a/Dockerfile b/Dockerfile index c420ef81bf4..f8917bcb66d 100644 --- a/Dockerfile +++ b/Dockerfile @@ -1,12 +1,12 @@ FROM golang:1.16-alpine as builder RUN apk add --no-cache git make bash -WORKDIR /go/src/github.com/pingcap/ticdc +WORKDIR /go/src/github.com/pingcap/tiflow COPY . . ENV CDC_ENABLE_VENDOR=0 RUN make FROM alpine:3.12 RUN apk add --no-cache tzdata bash curl socat -COPY --from=builder /go/src/github.com/pingcap/ticdc/bin/cdc /cdc +COPY --from=builder /go/src/github.com/pingcap/tiflow/bin/cdc /cdc EXPOSE 8300 CMD [ "/cdc" ] diff --git a/Dockerfile.development b/Dockerfile.development index d91a4ab4b0c..1d01bf9c8d1 100644 --- a/Dockerfile.development +++ b/Dockerfile.development @@ -1,6 +1,6 @@ FROM golang:1.16-alpine3.12 as builder RUN apk add --no-cache git make bash -WORKDIR /go/src/github.com/pingcap/ticdc +WORKDIR /go/src/github.com/pingcap/tiflow COPY . . ENV CDC_ENABLE_VENDOR=1 RUN go mod vendor @@ -10,6 +10,6 @@ RUN make failpoint-disable FROM alpine:3.12 RUN apk add --no-cache tzdata bash curl socat -COPY --from=builder /go/src/github.com/pingcap/ticdc/bin/cdc /cdc +COPY --from=builder /go/src/github.com/pingcap/tiflow/bin/cdc /cdc EXPOSE 8300 CMD [ "/cdc" ] diff --git a/Makefile b/Makefile index 0ead040af7c..31d3798b6a0 100644 --- a/Makefile +++ b/Makefile @@ -3,7 +3,7 @@ integration_test_build integration_test integration_test_mysql integration_test_kafka bank \ dm dm-master dm-worker dmctl dm-portal dm-syncer dm_coverage -PROJECT=ticdc +PROJECT=tiflow P=3 FAIL_ON_STDOUT := awk '{ print } END { if (NR > 0) { exit 1 } }' @@ -30,11 +30,11 @@ GOTESTNORACE := CGO_ENABLED=1 $(GO) test -p $(P) ARCH := "$(shell uname -s)" LINUX := "Linux" MAC := "Darwin" -CDC_PKG := github.com/pingcap/ticdc -DM_PKG := github.com/pingcap/ticdc/dm +CDC_PKG := github.com/pingcap/tiflow +DM_PKG := github.com/pingcap/tiflow/dm PACKAGE_LIST := go list ./... | grep -vE 'vendor|proto|ticdc\/tests|integration|testing_utils|pb|pbmock' -PACKAGE_LIST_WITHOUT_DM := $(PACKAGE_LIST) | grep -vE 'github.com/pingcap/ticdc/dm' -DM_PACKAGE_LIST := go list github.com/pingcap/ticdc/dm/... | grep -vE 'pb|pbmock|dm/cmd' +PACKAGE_LIST_WITHOUT_DM := $(PACKAGE_LIST) | grep -vE 'github.com/pingcap/tiflow/dm' +DM_PACKAGE_LIST := go list github.com/pingcap/tiflow/dm/... | grep -vE 'pb|pbmock|dm/cmd' PACKAGES := $$($(PACKAGE_LIST)) PACKAGES_WITHOUT_DM := $$($(PACKAGE_LIST_WITHOUT_DM)) DM_PACKAGES := $$($(DM_PACKAGE_LIST)) @@ -141,8 +141,8 @@ check_third_party_binary: integration_test_build: check_failpoint_ctl $(FAILPOINT_ENABLE) $(GOTEST) -ldflags '$(LDFLAGS)' -c -cover -covermode=atomic \ - -coverpkg=github.com/pingcap/ticdc/... \ - -o bin/cdc.test github.com/pingcap/ticdc/cmd/cdc \ + -coverpkg=github.com/pingcap/tiflow/... \ + -o bin/cdc.test github.com/pingcap/tiflow/cmd/cdc \ || { $(FAILPOINT_DISABLE); exit 1; } $(GOBUILD) -ldflags '$(LDFLAGS)' -o bin/cdc ./cmd/cdc/main.go \ || { $(FAILPOINT_DISABLE); exit 1; } @@ -273,20 +273,20 @@ dm_unit_test: check_failpoint_ctl dm_integration_test_build: check_failpoint_ctl $(FAILPOINT_ENABLE) $(GOTEST) -ldflags '$(LDFLAGS)' -c -cover -covermode=atomic \ - -coverpkg=github.com/pingcap/ticdc/dm/... \ - -o bin/dm-worker.test github.com/pingcap/ticdc/dm/cmd/dm-worker \ + -coverpkg=github.com/pingcap/tiflow/dm/... \ + -o bin/dm-worker.test github.com/pingcap/tiflow/dm/cmd/dm-worker \ || { $(FAILPOINT_DISABLE); exit 1; } $(GOTEST) -ldflags '$(LDFLAGS)' -c -cover -covermode=atomic \ - -coverpkg=github.com/pingcap/ticdc/dm/... \ - -o bin/dm-master.test github.com/pingcap/ticdc/dm/cmd/dm-master \ + -coverpkg=github.com/pingcap/tiflow/dm/... \ + -o bin/dm-master.test github.com/pingcap/tiflow/dm/cmd/dm-master \ || { $(FAILPOINT_DISABLE); exit 1; } $(GOTESTNORACE) -ldflags '$(LDFLAGS)' -c -cover -covermode=count \ - -coverpkg=github.com/pingcap/ticdc/dm/... \ - -o bin/dmctl.test github.com/pingcap/ticdc/dm/cmd/dm-ctl \ + -coverpkg=github.com/pingcap/tiflow/dm/... \ + -o bin/dmctl.test github.com/pingcap/tiflow/dm/cmd/dm-ctl \ || { $(FAILPOINT_DISABLE); exit 1; } $(GOTEST) -ldflags '$(LDFLAGS)' -c -cover -covermode=atomic \ - -coverpkg=github.com/pingcap/ticdc/dm/... \ - -o bin/dm-syncer.test github.com/pingcap/ticdc/dm/cmd/dm-syncer \ + -coverpkg=github.com/pingcap/tiflow/dm/... \ + -o bin/dm-syncer.test github.com/pingcap/tiflow/dm/cmd/dm-syncer \ || { $(FAILPOINT_DISABLE); exit 1; } $(FAILPOINT_DISABLE) ./dm/tests/prepare_tools.sh diff --git a/README.md b/README.md index 9ed4a02657e..6c6a9f72eee 100644 --- a/README.md +++ b/README.md @@ -1,10 +1,10 @@ # TiCDC -[![Build Status](https://github.com/pingcap/ticdc/actions/workflows/check_and_build.yaml/badge.svg?branch=master)](https://github.com/pingcap/ticdc/actions/workflows/check_and_build.yaml?query=event%3Apush+branch%3Amaster) -[![codecov](https://codecov.io/gh/pingcap/ticdc/branch/master/graph/badge.svg)](https://codecov.io/gh/pingcap/ticdc) -[![Coverage Status](https://coveralls.io/repos/github/pingcap/ticdc/badge.svg)](https://coveralls.io/github/pingcap/ticdc) -[![LICENSE](https://img.shields.io/github/license/pingcap/ticdc.svg)](https://github.com/pingcap/ticdc/blob/master/LICENSE) -[![Go Report Card](https://goreportcard.com/badge/github.com/pingcap/ticdc)](https://goreportcard.com/report/github.com/pingcap/ticdc) +[![Build Status](https://github.com/pingcap/tiflow/actions/workflows/check_and_build.yaml/badge.svg?branch=master)](https://github.com/pingcap/tiflow/actions/workflows/check_and_build.yaml?query=event%3Apush+branch%3Amaster) +[![codecov](https://codecov.io/gh/pingcap/tiflow/branch/master/graph/badge.svg)](https://codecov.io/gh/pingcap/tiflow) +[![Coverage Status](https://coveralls.io/repos/github/pingcap/tiflow/badge.svg)](https://coveralls.io/github/pingcap/tiflow) +[![LICENSE](https://img.shields.io/github/license/pingcap/tiflow.svg)](https://github.com/pingcap/tiflow/blob/master/LICENSE) +[![Go Report Card](https://goreportcard.com/badge/github.com/pingcap/tiflow)](https://goreportcard.com/report/github.com/pingcap/tiflow) **TiCDC** is [TiDB](https://docs.pingcap.com/tidb/stable)'s change data capture framework. It supports replicating change data to various downstreams, including MySQL protocol-compatible databases, message queues via the open CDC protocol and other systems such as local file storage. diff --git a/cdc/capture/capture.go b/cdc/capture/capture.go index f35b5eea01e..de7b8ba3982 100644 --- a/cdc/capture/capture.go +++ b/cdc/capture/capture.go @@ -24,18 +24,18 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/kv" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/owner" - "github.com/pingcap/ticdc/cdc/processor" - "github.com/pingcap/ticdc/pkg/config" - cdcContext "github.com/pingcap/ticdc/pkg/context" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/orchestrator" - "github.com/pingcap/ticdc/pkg/pdtime" - "github.com/pingcap/ticdc/pkg/version" tidbkv "github.com/pingcap/tidb/kv" + "github.com/pingcap/tiflow/cdc/kv" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/owner" + "github.com/pingcap/tiflow/cdc/processor" + "github.com/pingcap/tiflow/pkg/config" + cdcContext "github.com/pingcap/tiflow/pkg/context" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/orchestrator" + "github.com/pingcap/tiflow/pkg/pdtime" + "github.com/pingcap/tiflow/pkg/version" pd "github.com/tikv/pd/client" "go.etcd.io/etcd/clientv3/concurrency" "go.etcd.io/etcd/mvcc" diff --git a/cdc/capture/http_handler.go b/cdc/capture/http_handler.go index ae937f15f0a..1b340c7477e 100644 --- a/cdc/capture/http_handler.go +++ b/cdc/capture/http_handler.go @@ -20,14 +20,14 @@ import ( "github.com/gin-gonic/gin" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/owner" - "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/logutil" - "github.com/pingcap/ticdc/pkg/retry" - "github.com/pingcap/ticdc/pkg/version" "github.com/pingcap/tidb/br/pkg/httputil" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/owner" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/logutil" + "github.com/pingcap/tiflow/pkg/retry" + "github.com/pingcap/tiflow/pkg/version" "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" ) diff --git a/cdc/capture/http_validator.go b/cdc/capture/http_validator.go index 442c5ca3497..0281a23d63f 100644 --- a/cdc/capture/http_validator.go +++ b/cdc/capture/http_validator.go @@ -19,17 +19,17 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/entry" - "github.com/pingcap/ticdc/cdc/kv" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/sink" - "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/filter" - "github.com/pingcap/ticdc/pkg/txnutil/gc" - "github.com/pingcap/ticdc/pkg/util" - "github.com/pingcap/ticdc/pkg/version" tidbkv "github.com/pingcap/tidb/kv" + "github.com/pingcap/tiflow/cdc/entry" + "github.com/pingcap/tiflow/cdc/kv" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sink" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/filter" + "github.com/pingcap/tiflow/pkg/txnutil/gc" + "github.com/pingcap/tiflow/pkg/util" + "github.com/pingcap/tiflow/pkg/version" "github.com/r3labs/diff" "github.com/tikv/client-go/v2/oracle" ) diff --git a/cdc/capture/http_validator_test.go b/cdc/capture/http_validator_test.go index 8758fef2366..fe855009bf5 100644 --- a/cdc/capture/http_validator_test.go +++ b/cdc/capture/http_validator_test.go @@ -17,8 +17,8 @@ import ( "context" "testing" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" "github.com/stretchr/testify/require" ) diff --git a/cdc/capture/main_test.go b/cdc/capture/main_test.go index b29c295421f..8984319767d 100644 --- a/cdc/capture/main_test.go +++ b/cdc/capture/main_test.go @@ -16,7 +16,7 @@ package capture import ( "testing" - "github.com/pingcap/ticdc/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/leakutil" ) func TestMain(m *testing.M) { diff --git a/cdc/entry/codec.go b/cdc/entry/codec.go index bfa4bbbd11e..40b03c0b30c 100644 --- a/cdc/entry/codec.go +++ b/cdc/entry/codec.go @@ -18,14 +18,14 @@ import ( "time" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/model" - cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/rowcodec" + "github.com/pingcap/tiflow/cdc/model" + cerror "github.com/pingcap/tiflow/pkg/errors" ) var ( diff --git a/cdc/entry/main_test.go b/cdc/entry/main_test.go index fce350517c1..74b87377b2f 100644 --- a/cdc/entry/main_test.go +++ b/cdc/entry/main_test.go @@ -16,7 +16,7 @@ package entry import ( "testing" - "github.com/pingcap/ticdc/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/leakutil" ) func TestMain(m *testing.M) { diff --git a/cdc/entry/mounter.go b/cdc/entry/mounter.go index 15845e117a2..520a0f20233 100644 --- a/cdc/entry/mounter.go +++ b/cdc/entry/mounter.go @@ -24,15 +24,15 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/util" "github.com/pingcap/tidb/kv" timodel "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" + "github.com/pingcap/tiflow/cdc/model" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" "golang.org/x/sync/errgroup" ) diff --git a/cdc/entry/mounter_test.go b/cdc/entry/mounter_test.go index 0e326ce329f..686025ac3b0 100644 --- a/cdc/entry/mounter_test.go +++ b/cdc/entry/mounter_test.go @@ -20,14 +20,14 @@ import ( "time" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/regionspan" ticonfig "github.com/pingcap/tidb/config" tidbkv "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/testkit" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/regionspan" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" diff --git a/cdc/entry/schema_storage.go b/cdc/entry/schema_storage.go index 2e9ce1b60a5..f38dd358b76 100644 --- a/cdc/entry/schema_storage.go +++ b/cdc/entry/schema_storage.go @@ -22,12 +22,12 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/filter" - "github.com/pingcap/ticdc/pkg/retry" timeta "github.com/pingcap/tidb/meta" timodel "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tiflow/cdc/model" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/filter" + "github.com/pingcap/tiflow/pkg/retry" "go.uber.org/zap" "go.uber.org/zap/zapcore" ) diff --git a/cdc/entry/schema_storage_test.go b/cdc/entry/schema_storage_test.go index 34c72b9e3a9..9ee426af65a 100644 --- a/cdc/entry/schema_storage_test.go +++ b/cdc/entry/schema_storage_test.go @@ -20,8 +20,6 @@ import ( "testing" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/kv" - "github.com/pingcap/ticdc/cdc/model" ticonfig "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" tidbkv "github.com/pingcap/tidb/kv" @@ -33,6 +31,8 @@ import ( "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/testkit" "github.com/pingcap/tidb/types" + "github.com/pingcap/tiflow/cdc/kv" + "github.com/pingcap/tiflow/cdc/model" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" ) diff --git a/cdc/http_handler.go b/cdc/http_handler.go index a4c77d122ee..f057b0c1088 100644 --- a/cdc/http_handler.go +++ b/cdc/http_handler.go @@ -23,10 +23,10 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/owner" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/logutil" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/owner" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/logutil" "github.com/tikv/client-go/v2/oracle" "go.etcd.io/etcd/clientv3/concurrency" "go.uber.org/zap" diff --git a/cdc/http_router.go b/cdc/http_router.go index d6b74cb7aac..51019609629 100644 --- a/cdc/http_router.go +++ b/cdc/http_router.go @@ -21,12 +21,12 @@ import ( "time" "github.com/gin-gonic/gin" - "github.com/pingcap/ticdc/cdc/capture" + "github.com/pingcap/tiflow/cdc/capture" swaggerFiles "github.com/swaggo/files" ginSwagger "github.com/swaggo/gin-swagger" // use for OpenAPI online docs - _ "github.com/pingcap/ticdc/docs/api" + _ "github.com/pingcap/tiflow/docs/api" ) // newRouter create a router for OpenAPI diff --git a/cdc/http_router_test.go b/cdc/http_router_test.go index 4821a72b6d2..47ebb4245b9 100644 --- a/cdc/http_router_test.go +++ b/cdc/http_router_test.go @@ -19,7 +19,7 @@ import ( "net/http/httptest" "testing" - "github.com/pingcap/ticdc/cdc/capture" + "github.com/pingcap/tiflow/cdc/capture" "github.com/stretchr/testify/require" ) diff --git a/cdc/http_status.go b/cdc/http_status.go index 5aff78126a0..7b534b502ec 100644 --- a/cdc/http_status.go +++ b/cdc/http_status.go @@ -26,12 +26,12 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/capture" - "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/util" - "github.com/pingcap/ticdc/pkg/version" + "github.com/pingcap/tiflow/cdc/capture" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/util" + "github.com/pingcap/tiflow/pkg/version" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promhttp" "go.etcd.io/etcd/clientv3" diff --git a/cdc/http_status_test.go b/cdc/http_status_test.go index 71cbb66a188..3e758b62351 100644 --- a/cdc/http_status_test.go +++ b/cdc/http_status_test.go @@ -23,8 +23,8 @@ import ( "github.com/pingcap/check" "github.com/pingcap/failpoint" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/util/testleak" "go.etcd.io/etcd/clientv3/concurrency" ) @@ -127,7 +127,7 @@ func testRequestNonOwnerFailed(c *check.C, uri string) { } func testHandleFailpoint(c *check.C) { - fp := "github.com/pingcap/ticdc/cdc/TestHandleFailpoint" + fp := "github.com/pingcap/tiflow/cdc/TestHandleFailpoint" uri := fmt.Sprintf("http://%s/debug/fail/%s", advertiseAddr4Test, fp) body := bytes.NewReader([]byte("return(true)")) req, err := http.NewRequest("PUT", uri, body) diff --git a/cdc/kv/client.go b/cdc/kv/client.go index 0a726bd1f86..b78601f3719 100644 --- a/cdc/kv/client.go +++ b/cdc/kv/client.go @@ -29,14 +29,14 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/regionspan" - "github.com/pingcap/ticdc/pkg/retry" - "github.com/pingcap/ticdc/pkg/txnutil" - "github.com/pingcap/ticdc/pkg/util" - "github.com/pingcap/ticdc/pkg/version" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/regionspan" + "github.com/pingcap/tiflow/pkg/retry" + "github.com/pingcap/tiflow/pkg/txnutil" + "github.com/pingcap/tiflow/pkg/util" + "github.com/pingcap/tiflow/pkg/version" "github.com/prometheus/client_golang/prometheus" tidbkv "github.com/tikv/client-go/v2/kv" "github.com/tikv/client-go/v2/tikv" diff --git a/cdc/kv/client_bench_test.go b/cdc/kv/client_bench_test.go index 9f0554c88e6..9375524d265 100644 --- a/cdc/kv/client_bench_test.go +++ b/cdc/kv/client_bench_test.go @@ -24,12 +24,12 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/cdcpb" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/regionspan" - "github.com/pingcap/ticdc/pkg/retry" - "github.com/pingcap/ticdc/pkg/security" - "github.com/pingcap/ticdc/pkg/txnutil" "github.com/pingcap/tidb/store/mockstore/mockcopr" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/regionspan" + "github.com/pingcap/tiflow/pkg/retry" + "github.com/pingcap/tiflow/pkg/security" + "github.com/pingcap/tiflow/pkg/txnutil" "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/testutils" "github.com/tikv/client-go/v2/tikv" diff --git a/cdc/kv/client_mock_test.go b/cdc/kv/client_mock_test.go index d98961c2de7..e077006b746 100644 --- a/cdc/kv/client_mock_test.go +++ b/cdc/kv/client_mock_test.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/kvproto/pkg/cdcpb" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/ticdc/pkg/version" + "github.com/pingcap/tiflow/pkg/version" pd "github.com/tikv/pd/client" ) diff --git a/cdc/kv/client_test.go b/cdc/kv/client_test.go index 5917dbfb17c..067f3682617 100644 --- a/cdc/kv/client_test.go +++ b/cdc/kv/client_test.go @@ -32,18 +32,18 @@ import ( "github.com/pingcap/kvproto/pkg/errorpb" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/regionspan" - "github.com/pingcap/ticdc/pkg/retry" - "github.com/pingcap/ticdc/pkg/security" - "github.com/pingcap/ticdc/pkg/txnutil" - "github.com/pingcap/ticdc/pkg/util" - "github.com/pingcap/ticdc/pkg/util/testleak" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/mockstore/mockcopr" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/regionspan" + "github.com/pingcap/tiflow/pkg/retry" + "github.com/pingcap/tiflow/pkg/security" + "github.com/pingcap/tiflow/pkg/txnutil" + "github.com/pingcap/tiflow/pkg/util" + "github.com/pingcap/tiflow/pkg/util/testleak" "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/testutils" "github.com/tikv/client-go/v2/tikv" @@ -1388,10 +1388,10 @@ func (s *clientSuite) testStreamRecvWithError(c *check.C, failpointStr string) { cluster.AddStore(1, addr1) cluster.Bootstrap(regionID, []uint64{1}, []uint64{4}, 4) - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientStreamRecvError", failpointStr) + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientStreamRecvError", failpointStr) c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientStreamRecvError") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientStreamRecvError") }() baseAllocatedID := currentRequestID() lockresolver := txnutil.NewLockerResolver(kvStorage) @@ -1576,10 +1576,10 @@ func (s *clientSuite) TestStreamRecvWithErrorAndResolvedGoBack(c *check.C) { }, retry.WithBackoffBaseDelay(200), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(10)) c.Assert(err, check.IsNil) - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientStreamRecvError", "1*return(\"\")") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientStreamRecvError", "1*return(\"\")") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientStreamRecvError") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientStreamRecvError") }() ch1 <- resolved @@ -1730,10 +1730,10 @@ func (s *clientSuite) TestIncompatibleTiKV(c *check.C) { cluster.AddStore(1, addr1) cluster.Bootstrap(regionID, []uint64{1}, []uint64{4}, 4) - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientDelayWhenIncompatible", "return(true)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientDelayWhenIncompatible", "return(true)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientDelayWhenIncompatible") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientDelayWhenIncompatible") }() lockresolver := txnutil.NewLockerResolver(kvStorage) isPullInit := &mockPullerInit{} @@ -1990,10 +1990,10 @@ func (s *clientSuite) TestResolveLock(c *check.C) { cluster.AddStore(1, addr1) cluster.Bootstrap(regionID, []uint64{1}, []uint64{4}, 4) - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientResolveLockInterval", "return(3)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientResolveLockInterval", "return(3)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientResolveLockInterval") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientResolveLockInterval") }() baseAllocatedID := currentRequestID() lockresolver := txnutil.NewLockerResolver(kvStorage) @@ -2089,10 +2089,10 @@ func (s *clientSuite) testEventCommitTsFallback(c *check.C, events []*cdcpb.Chan logPanic = log.Panic }() - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientErrUnreachable", "return(true)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientErrUnreachable", "return(true)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientErrUnreachable") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientErrUnreachable") }() baseAllocatedID := currentRequestID() lockresolver := txnutil.NewLockerResolver(kvStorage) @@ -2237,10 +2237,10 @@ func (s *clientSuite) testEventAfterFeedStop(c *check.C) { // add 2s delay to simulate event feed processor has been marked stopped, but // before event feed processor is reconstruct, some duplicated events are // sent to event feed processor. - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientSingleFeedProcessDelay", "1*sleep(2000)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientSingleFeedProcessDelay", "1*sleep(2000)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientSingleFeedProcessDelay") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientSingleFeedProcessDelay") }() baseAllocatedID := currentRequestID() lockresolver := txnutil.NewLockerResolver(kvStorage) @@ -2738,13 +2738,13 @@ func (s *clientSuite) TestFailRegionReentrant(c *check.C) { cluster.AddStore(1, addr1) cluster.Bootstrap(regionID, []uint64{1}, []uint64{4}, 4) - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientRegionReentrantError", "1*return(\"ok\")->1*return(\"error\")") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientRegionReentrantError", "1*return(\"ok\")->1*return(\"error\")") c.Assert(err, check.IsNil) - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientRegionReentrantErrorDelay", "sleep(500)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientRegionReentrantErrorDelay", "sleep(500)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientRegionReentrantError") - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientRegionReentrantErrorDelay") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientRegionReentrantError") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientRegionReentrantErrorDelay") }() baseAllocatedID := currentRequestID() lockresolver := txnutil.NewLockerResolver(kvStorage.(tikv.Storage)) @@ -2820,13 +2820,13 @@ func (s *clientSuite) TestClientV1UnlockRangeReentrant(c *check.C) { cluster.Bootstrap(regionID3, []uint64{1}, []uint64{4}, 4) cluster.SplitRaw(regionID3, regionID4, []byte("b"), []uint64{5}, 5) - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientStreamRecvError", "1*return(\"injected stream recv error\")") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientStreamRecvError", "1*return(\"injected stream recv error\")") c.Assert(err, check.IsNil) - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientPendingRegionDelay", "1*sleep(0)->1*sleep(2000)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientPendingRegionDelay", "1*sleep(0)->1*sleep(2000)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientStreamRecvError") - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientPendingRegionDelay") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientStreamRecvError") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientPendingRegionDelay") }() lockresolver := txnutil.NewLockerResolver(kvStorage) isPullInit := &mockPullerInit{} @@ -2883,16 +2883,16 @@ func (s *clientSuite) testClientErrNoPendingRegion(c *check.C) { cluster.Bootstrap(regionID3, []uint64{1}, []uint64{4}, 4) cluster.SplitRaw(regionID3, regionID4, []byte("b"), []uint64{5}, 5) - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientStreamRecvError", "1*return(\"injected error\")") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientStreamRecvError", "1*return(\"injected error\")") c.Assert(err, check.IsNil) - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientPendingRegionDelay", "1*sleep(0)->2*sleep(1000)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientPendingRegionDelay", "1*sleep(0)->2*sleep(1000)") c.Assert(err, check.IsNil) - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientStreamCloseDelay", "sleep(2000)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientStreamCloseDelay", "sleep(2000)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientStreamRecvError") - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientPendingRegionDelay") - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientStreamCloseDelay") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientStreamRecvError") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientPendingRegionDelay") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientStreamCloseDelay") }() lockresolver := txnutil.NewLockerResolver(kvStorage) isPullInit := &mockPullerInit{} @@ -2961,12 +2961,12 @@ func (s *clientSuite) testKVClientForceReconnect(c *check.C) { cluster.AddStore(1, addr1) cluster.Bootstrap(regionID3, []uint64{1}, []uint64{4}, 4) - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientResolveLockInterval", "return(1)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientResolveLockInterval", "return(1)") c.Assert(err, check.IsNil) originalReconnectInterval := reconnectInterval reconnectInterval = 3 * time.Second defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientResolveLockInterval") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientResolveLockInterval") reconnectInterval = originalReconnectInterval }() @@ -3114,10 +3114,10 @@ func (s *clientSuite) TestConcurrentProcessRangeRequest(c *check.C) { cluster.AddStore(storeID, addr1) cluster.Bootstrap(regionID, []uint64{storeID}, []uint64{peerID}, peerID) - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientMockRangeLock", "1*return(20)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientMockRangeLock", "1*return(20)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientMockRangeLock") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientMockRangeLock") }() lockresolver := txnutil.NewLockerResolver(kvStorage) isPullInit := &mockPullerInit{} @@ -3226,10 +3226,10 @@ func (s *clientSuite) TestEvTimeUpdate(c *check.C) { originalReconnectInterval := reconnectInterval reconnectInterval = 1500 * time.Millisecond - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/kv/kvClientCheckUnInitRegionInterval", "return(2)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/kv/kvClientCheckUnInitRegionInterval", "return(2)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/kv/kvClientCheckUnInitRegionInterval") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/kv/kvClientCheckUnInitRegionInterval") reconnectInterval = originalReconnectInterval }() diff --git a/cdc/kv/grpc_pool_impl.go b/cdc/kv/grpc_pool_impl.go index c232254b8a8..7ed90c4d07b 100644 --- a/cdc/kv/grpc_pool_impl.go +++ b/cdc/kv/grpc_pool_impl.go @@ -19,8 +19,8 @@ import ( "time" "github.com/pingcap/log" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/security" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/security" "go.uber.org/zap" "google.golang.org/grpc" gbackoff "google.golang.org/grpc/backoff" diff --git a/cdc/kv/grpc_pool_impl_test.go b/cdc/kv/grpc_pool_impl_test.go index a143f672b68..70e8b7413bb 100644 --- a/cdc/kv/grpc_pool_impl_test.go +++ b/cdc/kv/grpc_pool_impl_test.go @@ -17,8 +17,8 @@ import ( "context" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/security" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/security" + "github.com/pingcap/tiflow/pkg/util/testleak" ) // Use clientSuite for some special reasons, the embed etcd uses zap as the only candidate diff --git a/cdc/kv/matcher_test.go b/cdc/kv/matcher_test.go index b911a74e77f..fd9f745e7aa 100644 --- a/cdc/kv/matcher_test.go +++ b/cdc/kv/matcher_test.go @@ -16,7 +16,7 @@ package kv import ( "github.com/pingcap/check" "github.com/pingcap/kvproto/pkg/cdcpb" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type matcherSuite struct{} diff --git a/cdc/kv/region_worker.go b/cdc/kv/region_worker.go index b92d06dff07..dead08c2626 100644 --- a/cdc/kv/region_worker.go +++ b/cdc/kv/region_worker.go @@ -25,12 +25,12 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/cdcpb" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/regionspan" - "github.com/pingcap/ticdc/pkg/util" - "github.com/pingcap/ticdc/pkg/workerpool" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/regionspan" + "github.com/pingcap/tiflow/pkg/util" + "github.com/pingcap/tiflow/pkg/workerpool" "github.com/prometheus/client_golang/prometheus" "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" diff --git a/cdc/kv/region_worker_test.go b/cdc/kv/region_worker_test.go index 68b983d5668..13eac0d4ca7 100644 --- a/cdc/kv/region_worker_test.go +++ b/cdc/kv/region_worker_test.go @@ -19,8 +19,8 @@ import ( "sync" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type regionWorkerSuite struct{} diff --git a/cdc/kv/resolvedts_heap_test.go b/cdc/kv/resolvedts_heap_test.go index 2a79e609c94..4fe92e9b5f1 100644 --- a/cdc/kv/resolvedts_heap_test.go +++ b/cdc/kv/resolvedts_heap_test.go @@ -17,7 +17,7 @@ import ( "time" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type rtsHeapSuite struct{} diff --git a/cdc/kv/store_op.go b/cdc/kv/store_op.go index 6047db69de9..a40e409001c 100644 --- a/cdc/kv/store_op.go +++ b/cdc/kv/store_op.go @@ -20,16 +20,16 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/flags" - "github.com/pingcap/ticdc/pkg/security" tidbconfig "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/kv" tidbkv "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/store" "github.com/pingcap/tidb/store/driver" + "github.com/pingcap/tiflow/cdc/model" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/flags" + "github.com/pingcap/tiflow/pkg/security" "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/tikv" "go.uber.org/zap" diff --git a/cdc/kv/testing.go b/cdc/kv/testing.go index b28fe933563..72795240b83 100644 --- a/cdc/kv/testing.go +++ b/cdc/kv/testing.go @@ -22,13 +22,13 @@ import ( "time" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/regionspan" - "github.com/pingcap/ticdc/pkg/security" - "github.com/pingcap/ticdc/pkg/txnutil" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store" "github.com/pingcap/tidb/store/driver" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/regionspan" + "github.com/pingcap/tiflow/pkg/security" + "github.com/pingcap/tiflow/pkg/txnutil" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/tikv" pd "github.com/tikv/pd/client" diff --git a/cdc/kv/token_region.go b/cdc/kv/token_region.go index 4275b11bf81..61b3c9d10e1 100644 --- a/cdc/kv/token_region.go +++ b/cdc/kv/token_region.go @@ -19,7 +19,7 @@ import ( "time" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/tiflow/pkg/util" "github.com/prometheus/client_golang/prometheus" ) diff --git a/cdc/kv/token_region_test.go b/cdc/kv/token_region_test.go index 76efd053860..a5ad7d5279e 100644 --- a/cdc/kv/token_region_test.go +++ b/cdc/kv/token_region_test.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/check" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" "github.com/tikv/client-go/v2/tikv" "golang.org/x/sync/errgroup" ) diff --git a/cdc/metrics.go b/cdc/metrics.go index ae7cfac34e7..f27d4e1de66 100644 --- a/cdc/metrics.go +++ b/cdc/metrics.go @@ -14,17 +14,17 @@ package cdc import ( - "github.com/pingcap/ticdc/cdc/entry" - "github.com/pingcap/ticdc/cdc/kv" - "github.com/pingcap/ticdc/cdc/owner" - "github.com/pingcap/ticdc/cdc/processor" - tablepipeline "github.com/pingcap/ticdc/cdc/processor/pipeline" - "github.com/pingcap/ticdc/cdc/puller" - "github.com/pingcap/ticdc/cdc/sink" - "github.com/pingcap/ticdc/cdc/sorter/memory" - "github.com/pingcap/ticdc/cdc/sorter/unified" - "github.com/pingcap/ticdc/pkg/actor" - "github.com/pingcap/ticdc/pkg/etcd" + "github.com/pingcap/tiflow/cdc/entry" + "github.com/pingcap/tiflow/cdc/kv" + "github.com/pingcap/tiflow/cdc/owner" + "github.com/pingcap/tiflow/cdc/processor" + tablepipeline "github.com/pingcap/tiflow/cdc/processor/pipeline" + "github.com/pingcap/tiflow/cdc/puller" + "github.com/pingcap/tiflow/cdc/sink" + "github.com/pingcap/tiflow/cdc/sorter/memory" + "github.com/pingcap/tiflow/cdc/sorter/unified" + "github.com/pingcap/tiflow/pkg/actor" + "github.com/pingcap/tiflow/pkg/etcd" "github.com/prometheus/client_golang/prometheus" ) diff --git a/cdc/model/capture.go b/cdc/model/capture.go index f6594786211..aa2aa76331b 100644 --- a/cdc/model/capture.go +++ b/cdc/model/capture.go @@ -17,7 +17,7 @@ import ( "encoding/json" "github.com/pingcap/errors" - cerror "github.com/pingcap/ticdc/pkg/errors" + cerror "github.com/pingcap/tiflow/pkg/errors" ) // CaptureInfo store in etcd. diff --git a/cdc/model/changefeed.go b/cdc/model/changefeed.go index 133b6297427..d1806071bc9 100644 --- a/cdc/model/changefeed.go +++ b/cdc/model/changefeed.go @@ -22,9 +22,9 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/cyclic/mark" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/cyclic/mark" + cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" ) diff --git a/cdc/model/changefeed_test.go b/cdc/model/changefeed_test.go index b0e5befe4ea..1a2d9103f20 100644 --- a/cdc/model/changefeed_test.go +++ b/cdc/model/changefeed_test.go @@ -19,10 +19,10 @@ import ( "testing" "time" - "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" filter "github.com/pingcap/tidb-tools/pkg/table-filter" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" ) diff --git a/cdc/model/http_model.go b/cdc/model/http_model.go index b6f80f179fb..25d9e6d05c6 100644 --- a/cdc/model/http_model.go +++ b/cdc/model/http_model.go @@ -18,8 +18,8 @@ import ( "fmt" "time" - "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" ) // JSONTime used to wrap time into json format diff --git a/cdc/model/http_model_test.go b/cdc/model/http_model_test.go index 72dd7d0dc3d..5e46d787a28 100644 --- a/cdc/model/http_model_test.go +++ b/cdc/model/http_model_test.go @@ -17,7 +17,7 @@ import ( "encoding/json" "testing" - cerror "github.com/pingcap/ticdc/pkg/errors" + cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/stretchr/testify/require" ) diff --git a/cdc/model/kv.go b/cdc/model/kv.go index 1e249248970..4af1c9569b3 100644 --- a/cdc/model/kv.go +++ b/cdc/model/kv.go @@ -18,7 +18,7 @@ package model import ( "fmt" - "github.com/pingcap/ticdc/pkg/regionspan" + "github.com/pingcap/tiflow/pkg/regionspan" ) // OpType for the kv, delete or put diff --git a/cdc/model/kv_test.go b/cdc/model/kv_test.go index c1465240d1a..7f5e25113b7 100644 --- a/cdc/model/kv_test.go +++ b/cdc/model/kv_test.go @@ -16,7 +16,7 @@ package model import ( "testing" - "github.com/pingcap/ticdc/pkg/regionspan" + "github.com/pingcap/tiflow/pkg/regionspan" "github.com/stretchr/testify/require" ) diff --git a/cdc/model/main_test.go b/cdc/model/main_test.go index 18154f1c399..e7ec3924ffe 100644 --- a/cdc/model/main_test.go +++ b/cdc/model/main_test.go @@ -16,7 +16,7 @@ package model import ( "testing" - "github.com/pingcap/ticdc/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/leakutil" ) func TestMain(m *testing.M) { diff --git a/cdc/model/owner.go b/cdc/model/owner.go index 2ca915e44dd..f4726d8edd4 100644 --- a/cdc/model/owner.go +++ b/cdc/model/owner.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - cerror "github.com/pingcap/ticdc/pkg/errors" + cerror "github.com/pingcap/tiflow/pkg/errors" "go.uber.org/zap" ) diff --git a/cdc/model/sink.go b/cdc/model/sink.go index e05447d3fd5..b7cb4d18c54 100644 --- a/cdc/model/sink.go +++ b/cdc/model/sink.go @@ -19,9 +19,9 @@ import ( "sync" "github.com/pingcap/log" - "github.com/pingcap/ticdc/pkg/quotes" - "github.com/pingcap/ticdc/pkg/util" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tiflow/pkg/quotes" + "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" ) diff --git a/cdc/model/string.go b/cdc/model/string.go index eda57d46756..0b2ed1908ff 100644 --- a/cdc/model/string.go +++ b/cdc/model/string.go @@ -16,7 +16,7 @@ package model import ( "strings" - cerror "github.com/pingcap/ticdc/pkg/errors" + cerror "github.com/pingcap/tiflow/pkg/errors" ) // HolderString returns a string of place holders separated by comma diff --git a/cdc/owner/async_sink.go b/cdc/owner/async_sink.go index e51af798c6c..632883e5dfd 100644 --- a/cdc/owner/async_sink.go +++ b/cdc/owner/async_sink.go @@ -22,11 +22,11 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/sink" - cdcContext "github.com/pingcap/ticdc/pkg/context" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/filter" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sink" + cdcContext "github.com/pingcap/tiflow/pkg/context" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/filter" "go.uber.org/zap" ) diff --git a/cdc/owner/async_sink_test.go b/cdc/owner/async_sink_test.go index 3c7fc11a11e..981c1905e6b 100644 --- a/cdc/owner/async_sink_test.go +++ b/cdc/owner/async_sink_test.go @@ -21,13 +21,13 @@ import ( "github.com/pingcap/check" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/sink" - "github.com/pingcap/ticdc/pkg/config" - cdcContext "github.com/pingcap/ticdc/pkg/context" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/retry" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sink" + "github.com/pingcap/tiflow/pkg/config" + cdcContext "github.com/pingcap/tiflow/pkg/context" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/retry" + "github.com/pingcap/tiflow/pkg/util/testleak" ) var _ = check.Suite(&asyncSinkSuite{}) diff --git a/cdc/owner/barrier.go b/cdc/owner/barrier.go index 6eed7aba493..abe951090d8 100644 --- a/cdc/owner/barrier.go +++ b/cdc/owner/barrier.go @@ -17,7 +17,7 @@ import ( "math" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/tiflow/cdc/model" ) type barrierType int diff --git a/cdc/owner/barrier_test.go b/cdc/owner/barrier_test.go index d8a6b851cf0..5b09cd030ba 100644 --- a/cdc/owner/barrier_test.go +++ b/cdc/owner/barrier_test.go @@ -19,8 +19,8 @@ import ( "testing" "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/util/testleak" ) func Test(t *testing.T) { check.TestingT(t) } diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go index 3bb3dfbb826..2ee838cb178 100644 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -20,15 +20,15 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/redo" - cdcContext "github.com/pingcap/ticdc/pkg/context" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/orchestrator" - "github.com/pingcap/ticdc/pkg/txnutil/gc" - "github.com/pingcap/ticdc/pkg/util" timodel "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/sessionctx/binloginfo" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/redo" + cdcContext "github.com/pingcap/tiflow/pkg/context" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/orchestrator" + "github.com/pingcap/tiflow/pkg/txnutil/gc" + "github.com/pingcap/tiflow/pkg/util" "github.com/prometheus/client_golang/prometheus" "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" diff --git a/cdc/owner/changefeed_test.go b/cdc/owner/changefeed_test.go index e3a630a4340..2657ce5fa8f 100644 --- a/cdc/owner/changefeed_test.go +++ b/cdc/owner/changefeed_test.go @@ -20,16 +20,16 @@ import ( "github.com/pingcap/check" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/entry" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" - cdcContext "github.com/pingcap/ticdc/pkg/context" - "github.com/pingcap/ticdc/pkg/orchestrator" - "github.com/pingcap/ticdc/pkg/pdtime" - "github.com/pingcap/ticdc/pkg/txnutil/gc" - "github.com/pingcap/ticdc/pkg/util/testleak" - "github.com/pingcap/ticdc/pkg/version" timodel "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tiflow/cdc/entry" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + cdcContext "github.com/pingcap/tiflow/pkg/context" + "github.com/pingcap/tiflow/pkg/orchestrator" + "github.com/pingcap/tiflow/pkg/pdtime" + "github.com/pingcap/tiflow/pkg/txnutil/gc" + "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/version" "github.com/tikv/client-go/v2/oracle" ) diff --git a/cdc/owner/ddl_puller.go b/cdc/owner/ddl_puller.go index 95c3e9b13ae..ad5452eb00e 100644 --- a/cdc/owner/ddl_puller.go +++ b/cdc/owner/ddl_puller.go @@ -19,15 +19,15 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/entry" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/puller" - "github.com/pingcap/ticdc/cdc/sorter/memory" - cdcContext "github.com/pingcap/ticdc/pkg/context" - "github.com/pingcap/ticdc/pkg/filter" - "github.com/pingcap/ticdc/pkg/regionspan" - "github.com/pingcap/ticdc/pkg/util" timodel "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tiflow/cdc/entry" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/puller" + "github.com/pingcap/tiflow/cdc/sorter/memory" + cdcContext "github.com/pingcap/tiflow/pkg/context" + "github.com/pingcap/tiflow/pkg/filter" + "github.com/pingcap/tiflow/pkg/regionspan" + "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" "golang.org/x/sync/errgroup" ) diff --git a/cdc/owner/ddl_puller_test.go b/cdc/owner/ddl_puller_test.go index 4e9cc5e6396..3ed7b4a45cd 100644 --- a/cdc/owner/ddl_puller_test.go +++ b/cdc/owner/ddl_puller_test.go @@ -21,12 +21,12 @@ import ( "github.com/pingcap/check" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/model" - cdcContext "github.com/pingcap/ticdc/pkg/context" - "github.com/pingcap/ticdc/pkg/retry" - "github.com/pingcap/ticdc/pkg/util/testleak" timodel "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tiflow/cdc/model" + cdcContext "github.com/pingcap/tiflow/pkg/context" + "github.com/pingcap/tiflow/pkg/retry" + "github.com/pingcap/tiflow/pkg/util/testleak" ) var _ = check.Suite(&ddlPullerSuite{}) diff --git a/cdc/owner/feed_state_manager.go b/cdc/owner/feed_state_manager.go index 909cc9a2f80..256ad0e07eb 100644 --- a/cdc/owner/feed_state_manager.go +++ b/cdc/owner/feed_state_manager.go @@ -18,9 +18,9 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/orchestrator" + "github.com/pingcap/tiflow/cdc/model" + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/orchestrator" "go.uber.org/zap" ) diff --git a/cdc/owner/feed_state_manager_test.go b/cdc/owner/feed_state_manager_test.go index 66d8c951a46..6bb962938e7 100644 --- a/cdc/owner/feed_state_manager_test.go +++ b/cdc/owner/feed_state_manager_test.go @@ -15,11 +15,11 @@ package owner import ( "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" - cdcContext "github.com/pingcap/ticdc/pkg/context" - "github.com/pingcap/ticdc/pkg/orchestrator" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + cdcContext "github.com/pingcap/tiflow/pkg/context" + "github.com/pingcap/tiflow/pkg/orchestrator" + "github.com/pingcap/tiflow/pkg/util/testleak" ) var _ = check.Suite(&feedStateManagerSuite{}) diff --git a/cdc/owner/owner.go b/cdc/owner/owner.go index 358bfee893c..c830f5081f7 100644 --- a/cdc/owner/owner.go +++ b/cdc/owner/owner.go @@ -25,12 +25,12 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - cdcContext "github.com/pingcap/ticdc/pkg/context" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/orchestrator" - "github.com/pingcap/ticdc/pkg/txnutil/gc" - "github.com/pingcap/ticdc/pkg/version" + "github.com/pingcap/tiflow/cdc/model" + cdcContext "github.com/pingcap/tiflow/pkg/context" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/orchestrator" + "github.com/pingcap/tiflow/pkg/txnutil/gc" + "github.com/pingcap/tiflow/pkg/version" pd "github.com/tikv/pd/client" "go.uber.org/zap" ) diff --git a/cdc/owner/owner_test.go b/cdc/owner/owner_test.go index 25ad60ae026..21fdaa43937 100644 --- a/cdc/owner/owner_test.go +++ b/cdc/owner/owner_test.go @@ -21,14 +21,14 @@ import ( "time" "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" - cdcContext "github.com/pingcap/ticdc/pkg/context" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/orchestrator" - "github.com/pingcap/ticdc/pkg/txnutil/gc" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + cdcContext "github.com/pingcap/tiflow/pkg/context" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/orchestrator" + "github.com/pingcap/tiflow/pkg/txnutil/gc" + "github.com/pingcap/tiflow/pkg/util/testleak" "github.com/tikv/client-go/v2/oracle" ) diff --git a/cdc/owner/scheduler.go b/cdc/owner/scheduler.go index 9b674d0b743..2bf474790ac 100644 --- a/cdc/owner/scheduler.go +++ b/cdc/owner/scheduler.go @@ -19,9 +19,9 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/orchestrator" + "github.com/pingcap/tiflow/cdc/model" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/orchestrator" "go.uber.org/zap" ) diff --git a/cdc/owner/scheduler_test.go b/cdc/owner/scheduler_test.go index f2dbb5ed545..320100237a3 100644 --- a/cdc/owner/scheduler_test.go +++ b/cdc/owner/scheduler_test.go @@ -18,9 +18,9 @@ import ( "math/rand" "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/orchestrator" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/orchestrator" + "github.com/pingcap/tiflow/pkg/util/testleak" ) var _ = check.Suite(&schedulerSuite{}) diff --git a/cdc/owner/schema.go b/cdc/owner/schema.go index d24584bda61..0b1af16beb3 100644 --- a/cdc/owner/schema.go +++ b/cdc/owner/schema.go @@ -16,15 +16,15 @@ package owner import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/entry" - "github.com/pingcap/ticdc/cdc/kv" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/cyclic/mark" - "github.com/pingcap/ticdc/pkg/filter" tidbkv "github.com/pingcap/tidb/kv" timeta "github.com/pingcap/tidb/meta" timodel "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tiflow/cdc/entry" + "github.com/pingcap/tiflow/cdc/kv" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/cyclic/mark" + "github.com/pingcap/tiflow/pkg/filter" "go.uber.org/zap" ) diff --git a/cdc/owner/schema_test.go b/cdc/owner/schema_test.go index 06e3d7e6ba3..9ed7e8ec14f 100644 --- a/cdc/owner/schema_test.go +++ b/cdc/owner/schema_test.go @@ -17,12 +17,12 @@ import ( "sort" "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/entry" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/util/testleak" timodel "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/cdc/entry" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/util/testleak" "github.com/tikv/client-go/v2/oracle" ) diff --git a/cdc/owner/status_provider.go b/cdc/owner/status_provider.go index 69dec072d21..c13f23e9967 100644 --- a/cdc/owner/status_provider.go +++ b/cdc/owner/status_provider.go @@ -16,10 +16,10 @@ package owner import ( "context" - cerror "github.com/pingcap/ticdc/pkg/errors" + cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/tiflow/cdc/model" ) // StatusProvider provide some func to get meta-information from owner diff --git a/cdc/processor/manager.go b/cdc/processor/manager.go index 968a65ebca4..a84b91a3fbf 100644 --- a/cdc/processor/manager.go +++ b/cdc/processor/manager.go @@ -22,10 +22,10 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - cdcContext "github.com/pingcap/ticdc/pkg/context" - cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/orchestrator" + "github.com/pingcap/tiflow/cdc/model" + cdcContext "github.com/pingcap/tiflow/pkg/context" + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/orchestrator" "go.uber.org/zap" ) diff --git a/cdc/processor/manager_test.go b/cdc/processor/manager_test.go index 94b969d2222..5aa066d0944 100644 --- a/cdc/processor/manager_test.go +++ b/cdc/processor/manager_test.go @@ -21,13 +21,13 @@ import ( "github.com/pingcap/check" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/model" - tablepipeline "github.com/pingcap/ticdc/cdc/processor/pipeline" - "github.com/pingcap/ticdc/pkg/config" - cdcContext "github.com/pingcap/ticdc/pkg/context" - cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/orchestrator" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + tablepipeline "github.com/pingcap/tiflow/cdc/processor/pipeline" + "github.com/pingcap/tiflow/pkg/config" + cdcContext "github.com/pingcap/tiflow/pkg/context" + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/orchestrator" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type managerSuite struct { diff --git a/cdc/processor/pipeline/cyclic_mark.go b/cdc/processor/pipeline/cyclic_mark.go index da0ac0562d8..2350f6bd3b2 100644 --- a/cdc/processor/pipeline/cyclic_mark.go +++ b/cdc/processor/pipeline/cyclic_mark.go @@ -16,10 +16,10 @@ package pipeline import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/entry" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/cyclic/mark" - "github.com/pingcap/ticdc/pkg/pipeline" + "github.com/pingcap/tiflow/cdc/entry" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/cyclic/mark" + "github.com/pingcap/tiflow/pkg/pipeline" "go.uber.org/zap" ) diff --git a/cdc/processor/pipeline/cyclic_mark_test.go b/cdc/processor/pipeline/cyclic_mark_test.go index 38c03c42616..0b83fb5cefc 100644 --- a/cdc/processor/pipeline/cyclic_mark_test.go +++ b/cdc/processor/pipeline/cyclic_mark_test.go @@ -20,13 +20,13 @@ import ( "github.com/google/go-cmp/cmp" "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" - cdcContext "github.com/pingcap/ticdc/pkg/context" - "github.com/pingcap/ticdc/pkg/cyclic/mark" - "github.com/pingcap/ticdc/pkg/pipeline" - "github.com/pingcap/ticdc/pkg/util/testleak" "github.com/pingcap/tidb/tablecodec" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + cdcContext "github.com/pingcap/tiflow/pkg/context" + "github.com/pingcap/tiflow/pkg/cyclic/mark" + "github.com/pingcap/tiflow/pkg/pipeline" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type markSuite struct{} diff --git a/cdc/processor/pipeline/mounter.go b/cdc/processor/pipeline/mounter.go index 2e156b9cdbd..c8034ae89ae 100644 --- a/cdc/processor/pipeline/mounter.go +++ b/cdc/processor/pipeline/mounter.go @@ -21,9 +21,9 @@ import ( "github.com/edwingeng/deque" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/notify" - "github.com/pingcap/ticdc/pkg/pipeline" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/notify" + "github.com/pingcap/tiflow/pkg/pipeline" "go.uber.org/zap" "golang.org/x/sync/errgroup" "golang.org/x/time/rate" diff --git a/cdc/processor/pipeline/mounter_test.go b/cdc/processor/pipeline/mounter_test.go index dbb19ccee56..c2d2609ed15 100644 --- a/cdc/processor/pipeline/mounter_test.go +++ b/cdc/processor/pipeline/mounter_test.go @@ -22,11 +22,11 @@ import ( "github.com/pingcap/check" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - cdcContext "github.com/pingcap/ticdc/pkg/context" - "github.com/pingcap/ticdc/pkg/pipeline" - "github.com/pingcap/ticdc/pkg/retry" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + cdcContext "github.com/pingcap/tiflow/pkg/context" + "github.com/pingcap/tiflow/pkg/pipeline" + "github.com/pingcap/tiflow/pkg/retry" + "github.com/pingcap/tiflow/pkg/util/testleak" "go.uber.org/zap" ) diff --git a/cdc/processor/pipeline/puller.go b/cdc/processor/pipeline/puller.go index 1a8d3492406..02ee12707e1 100644 --- a/cdc/processor/pipeline/puller.go +++ b/cdc/processor/pipeline/puller.go @@ -17,12 +17,12 @@ import ( "context" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/puller" - cdcContext "github.com/pingcap/ticdc/pkg/context" - "github.com/pingcap/ticdc/pkg/pipeline" - "github.com/pingcap/ticdc/pkg/regionspan" - "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/puller" + cdcContext "github.com/pingcap/tiflow/pkg/context" + "github.com/pingcap/tiflow/pkg/pipeline" + "github.com/pingcap/tiflow/pkg/regionspan" + "github.com/pingcap/tiflow/pkg/util" "github.com/tikv/client-go/v2/oracle" "golang.org/x/sync/errgroup" ) diff --git a/cdc/processor/pipeline/sink.go b/cdc/processor/pipeline/sink.go index 651a250a340..b436df2a050 100644 --- a/cdc/processor/pipeline/sink.go +++ b/cdc/processor/pipeline/sink.go @@ -20,10 +20,10 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/sink" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/pipeline" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sink" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/pipeline" "go.uber.org/zap" ) diff --git a/cdc/processor/pipeline/sink_test.go b/cdc/processor/pipeline/sink_test.go index 12ba3bb1fa5..2e6b25cce0e 100644 --- a/cdc/processor/pipeline/sink_test.go +++ b/cdc/processor/pipeline/sink_test.go @@ -20,12 +20,12 @@ import ( "time" "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" - cdcContext "github.com/pingcap/ticdc/pkg/context" - cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/pipeline" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + cdcContext "github.com/pingcap/tiflow/pkg/context" + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/pipeline" + "github.com/pingcap/tiflow/pkg/util/testleak" "github.com/tikv/client-go/v2/oracle" ) diff --git a/cdc/processor/pipeline/sorter.go b/cdc/processor/pipeline/sorter.go index 755981c3d64..cfd7e60a226 100644 --- a/cdc/processor/pipeline/sorter.go +++ b/cdc/processor/pipeline/sorter.go @@ -21,13 +21,13 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/entry" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/sorter" - "github.com/pingcap/ticdc/cdc/sorter/memory" - "github.com/pingcap/ticdc/cdc/sorter/unified" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/pipeline" + "github.com/pingcap/tiflow/cdc/entry" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sorter" + "github.com/pingcap/tiflow/cdc/sorter/memory" + "github.com/pingcap/tiflow/cdc/sorter/unified" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/pipeline" "go.uber.org/zap" "golang.org/x/sync/errgroup" ) diff --git a/cdc/processor/pipeline/sorter_test.go b/cdc/processor/pipeline/sorter_test.go index 35ebba8a165..6959951ff95 100644 --- a/cdc/processor/pipeline/sorter_test.go +++ b/cdc/processor/pipeline/sorter_test.go @@ -18,13 +18,13 @@ import ( "strings" "testing" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/sorter/memory" - "github.com/pingcap/ticdc/cdc/sorter/unified" - "github.com/pingcap/ticdc/pkg/config" - cdcContext "github.com/pingcap/ticdc/pkg/context" - "github.com/pingcap/ticdc/pkg/leakutil" - "github.com/pingcap/ticdc/pkg/pipeline" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sorter/memory" + "github.com/pingcap/tiflow/cdc/sorter/unified" + "github.com/pingcap/tiflow/pkg/config" + cdcContext "github.com/pingcap/tiflow/pkg/context" + "github.com/pingcap/tiflow/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/pipeline" "github.com/stretchr/testify/require" "go.uber.org/goleak" ) @@ -32,7 +32,7 @@ import ( func TestMain(m *testing.M) { leakutil.SetUpLeakTest( m, - goleak.IgnoreTopFunction("github.com/pingcap/ticdc/cdc/sorter/unified.newBackEndPool.func1"), + goleak.IgnoreTopFunction("github.com/pingcap/tiflow/cdc/sorter/unified.newBackEndPool.func1"), ) } diff --git a/cdc/processor/pipeline/table.go b/cdc/processor/pipeline/table.go index 7ee785b90dc..eb6b05e8ef5 100644 --- a/cdc/processor/pipeline/table.go +++ b/cdc/processor/pipeline/table.go @@ -18,14 +18,14 @@ import ( "time" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/entry" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/sink" - "github.com/pingcap/ticdc/cdc/sink/common" - serverConfig "github.com/pingcap/ticdc/pkg/config" - cdcContext "github.com/pingcap/ticdc/pkg/context" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/pipeline" + "github.com/pingcap/tiflow/cdc/entry" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sink" + "github.com/pingcap/tiflow/cdc/sink/common" + serverConfig "github.com/pingcap/tiflow/pkg/config" + cdcContext "github.com/pingcap/tiflow/pkg/context" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/pipeline" "go.uber.org/zap" ) diff --git a/cdc/processor/processor.go b/cdc/processor/processor.go index 36787e3d70b..b9ae2843c84 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -25,22 +25,22 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/entry" - "github.com/pingcap/ticdc/cdc/kv" - "github.com/pingcap/ticdc/cdc/model" - tablepipeline "github.com/pingcap/ticdc/cdc/processor/pipeline" - "github.com/pingcap/ticdc/cdc/puller" - "github.com/pingcap/ticdc/cdc/redo" - "github.com/pingcap/ticdc/cdc/sink" - "github.com/pingcap/ticdc/cdc/sorter/memory" - cdcContext "github.com/pingcap/ticdc/pkg/context" - "github.com/pingcap/ticdc/pkg/cyclic/mark" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/filter" - "github.com/pingcap/ticdc/pkg/orchestrator" - "github.com/pingcap/ticdc/pkg/regionspan" - "github.com/pingcap/ticdc/pkg/retry" - "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/tiflow/cdc/entry" + "github.com/pingcap/tiflow/cdc/kv" + "github.com/pingcap/tiflow/cdc/model" + tablepipeline "github.com/pingcap/tiflow/cdc/processor/pipeline" + "github.com/pingcap/tiflow/cdc/puller" + "github.com/pingcap/tiflow/cdc/redo" + "github.com/pingcap/tiflow/cdc/sink" + "github.com/pingcap/tiflow/cdc/sorter/memory" + cdcContext "github.com/pingcap/tiflow/pkg/context" + "github.com/pingcap/tiflow/pkg/cyclic/mark" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/filter" + "github.com/pingcap/tiflow/pkg/orchestrator" + "github.com/pingcap/tiflow/pkg/regionspan" + "github.com/pingcap/tiflow/pkg/retry" + "github.com/pingcap/tiflow/pkg/util" "github.com/prometheus/client_golang/prometheus" "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" diff --git a/cdc/processor/processor_test.go b/cdc/processor/processor_test.go index b6156afbc5b..d95a737ed70 100644 --- a/cdc/processor/processor_test.go +++ b/cdc/processor/processor_test.go @@ -24,15 +24,15 @@ import ( "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/entry" - "github.com/pingcap/ticdc/cdc/model" - tablepipeline "github.com/pingcap/ticdc/cdc/processor/pipeline" - "github.com/pingcap/ticdc/cdc/redo" - cdcContext "github.com/pingcap/ticdc/pkg/context" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/orchestrator" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/entry" + "github.com/pingcap/tiflow/cdc/model" + tablepipeline "github.com/pingcap/tiflow/cdc/processor/pipeline" + "github.com/pingcap/tiflow/cdc/redo" + cdcContext "github.com/pingcap/tiflow/pkg/context" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/orchestrator" + "github.com/pingcap/tiflow/pkg/util/testleak" ) func Test(t *testing.T) { check.TestingT(t) } diff --git a/cdc/puller/frontier/frontier.go b/cdc/puller/frontier/frontier.go index 8cb4e160620..6372b21dc8f 100644 --- a/cdc/puller/frontier/frontier.go +++ b/cdc/puller/frontier/frontier.go @@ -19,7 +19,7 @@ import ( "math" "strings" - "github.com/pingcap/ticdc/pkg/regionspan" + "github.com/pingcap/tiflow/pkg/regionspan" ) // Frontier checks resolved event of spans and moves the global resolved ts ahead diff --git a/cdc/puller/frontier/frontier_bench_test.go b/cdc/puller/frontier/frontier_bench_test.go index b966740ceef..158122129b1 100644 --- a/cdc/puller/frontier/frontier_bench_test.go +++ b/cdc/puller/frontier/frontier_bench_test.go @@ -17,7 +17,7 @@ import ( "fmt" "testing" - "github.com/pingcap/ticdc/pkg/regionspan" + "github.com/pingcap/tiflow/pkg/regionspan" ) func toCMPBytes(i int) []byte { diff --git a/cdc/puller/frontier/frontier_test.go b/cdc/puller/frontier/frontier_test.go index 0533fc133ef..3bcde1d8abd 100644 --- a/cdc/puller/frontier/frontier_test.go +++ b/cdc/puller/frontier/frontier_test.go @@ -20,8 +20,8 @@ import ( "testing" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/regionspan" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/regionspan" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type spanFrontierSuite struct{} diff --git a/cdc/puller/frontier/heap_test.go b/cdc/puller/frontier/heap_test.go index 5bc9686106e..ccfc9befc17 100644 --- a/cdc/puller/frontier/heap_test.go +++ b/cdc/puller/frontier/heap_test.go @@ -19,7 +19,7 @@ import ( "time" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type tsHeapSuite struct{} diff --git a/cdc/puller/frontier/list_test.go b/cdc/puller/frontier/list_test.go index 7acf366f77e..92c27a87dbc 100644 --- a/cdc/puller/frontier/list_test.go +++ b/cdc/puller/frontier/list_test.go @@ -18,7 +18,7 @@ import ( "math/rand" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type spanListSuite struct{} diff --git a/cdc/puller/mock_puller.go b/cdc/puller/mock_puller.go index d9be65da364..953585454f7 100644 --- a/cdc/puller/mock_puller.go +++ b/cdc/puller/mock_puller.go @@ -21,14 +21,14 @@ import ( "github.com/pingcap/check" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/regionspan" "github.com/pingcap/tidb/domain" tidbkv "github.com/pingcap/tidb/kv" timodel "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/regionspan" "github.com/tikv/client-go/v2/testutils" "go.uber.org/zap" ) diff --git a/cdc/puller/mock_puller_test.go b/cdc/puller/mock_puller_test.go index 1c80954cd3c..e41ddeafbab 100644 --- a/cdc/puller/mock_puller_test.go +++ b/cdc/puller/mock_puller_test.go @@ -22,9 +22,9 @@ import ( "github.com/pingcap/check" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/entry" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/tiflow/cdc/entry" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/util" "github.com/tikv/client-go/v2/oracle" ) diff --git a/cdc/puller/puller.go b/cdc/puller/puller.go index d015dc4bb09..ea17008fdc5 100644 --- a/cdc/puller/puller.go +++ b/cdc/puller/puller.go @@ -20,13 +20,13 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/kv" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/puller/frontier" - "github.com/pingcap/ticdc/pkg/regionspan" - "github.com/pingcap/ticdc/pkg/txnutil" - "github.com/pingcap/ticdc/pkg/util" tidbkv "github.com/pingcap/tidb/kv" + "github.com/pingcap/tiflow/cdc/kv" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/puller/frontier" + "github.com/pingcap/tiflow/pkg/regionspan" + "github.com/pingcap/tiflow/pkg/txnutil" + "github.com/pingcap/tiflow/pkg/util" "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/tikv" pd "github.com/tikv/pd/client" @@ -159,7 +159,7 @@ func (p *pullerImpl) Run(ctx context.Context) error { lastResolvedTs := p.checkpointTs g.Go(func() error { output := func(raw *model.RawKVEntry) error { - // even after https://github.com/pingcap/ticdc/pull/2038, kv client + // even after https://github.com/pingcap/tiflow/pull/2038, kv client // could still miss region change notification, which leads to resolved // ts update missing in puller, however resolved ts fallback here can // be ignored since no late data is received and the guarantee of diff --git a/cdc/puller/puller_test.go b/cdc/puller/puller_test.go index 4b34057d164..b374b7fe30a 100644 --- a/cdc/puller/puller_test.go +++ b/cdc/puller/puller_test.go @@ -21,16 +21,16 @@ import ( "github.com/pingcap/check" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/kv" - "github.com/pingcap/ticdc/cdc/model" - cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/regionspan" - "github.com/pingcap/ticdc/pkg/retry" - "github.com/pingcap/ticdc/pkg/security" - "github.com/pingcap/ticdc/pkg/txnutil" - "github.com/pingcap/ticdc/pkg/util/testleak" tidbkv "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tiflow/cdc/kv" + "github.com/pingcap/tiflow/cdc/model" + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/regionspan" + "github.com/pingcap/tiflow/pkg/retry" + "github.com/pingcap/tiflow/pkg/security" + "github.com/pingcap/tiflow/pkg/txnutil" + "github.com/pingcap/tiflow/pkg/util/testleak" "github.com/tikv/client-go/v2/tikv" pd "github.com/tikv/pd/client" ) diff --git a/cdc/redo/applier.go b/cdc/redo/applier.go index b81fc02ed8d..e4d9ef520a6 100644 --- a/cdc/redo/applier.go +++ b/cdc/redo/applier.go @@ -16,8 +16,8 @@ package redo import ( "context" - "github.com/pingcap/ticdc/cdc/redo/reader" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/tiflow/cdc/redo/reader" + cerror "github.com/pingcap/tiflow/pkg/errors" ) // NewRedoReader creates a new redo log reader diff --git a/cdc/redo/common/util.go b/cdc/redo/common/util.go index 106fe1b2ba8..5ba7321583a 100644 --- a/cdc/redo/common/util.go +++ b/cdc/redo/common/util.go @@ -22,8 +22,8 @@ import ( "github.com/pingcap/errors" backuppb "github.com/pingcap/kvproto/pkg/brpb" - cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/tidb/br/pkg/storage" + cerror "github.com/pingcap/tiflow/pkg/errors" ) // InitS3storage init a storage used for s3, diff --git a/cdc/redo/convert.go b/cdc/redo/convert.go index 2f62ebee206..4d18ab8ca50 100644 --- a/cdc/redo/convert.go +++ b/cdc/redo/convert.go @@ -16,8 +16,8 @@ package redo import ( "bytes" - "github.com/pingcap/ticdc/cdc/model" pmodel "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tiflow/cdc/model" ) // RowToRedo converts row changed event to redo log row diff --git a/cdc/redo/convert_test.go b/cdc/redo/convert_test.go index 3b1fe695be0..2db1303922f 100644 --- a/cdc/redo/convert_test.go +++ b/cdc/redo/convert_test.go @@ -16,9 +16,9 @@ package redo import ( "testing" - "github.com/pingcap/ticdc/cdc/model" timodel "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/cdc/model" "github.com/stretchr/testify/require" ) diff --git a/cdc/redo/manager.go b/cdc/redo/manager.go index 94553c9db8b..886cd781f0f 100644 --- a/cdc/redo/manager.go +++ b/cdc/redo/manager.go @@ -23,12 +23,12 @@ import ( "time" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/redo/writer" - "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/util" "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/redo/writer" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" ) diff --git a/cdc/redo/manager_test.go b/cdc/redo/manager_test.go index 18a5f9fb48e..72b51731d5d 100644 --- a/cdc/redo/manager_test.go +++ b/cdc/redo/manager_test.go @@ -18,8 +18,8 @@ import ( "testing" "time" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" "github.com/stretchr/testify/require" ) diff --git a/cdc/redo/reader/blackhole_reader.go b/cdc/redo/reader/blackhole_reader.go index 97db8592c6b..724ea6a0a3e 100644 --- a/cdc/redo/reader/blackhole_reader.go +++ b/cdc/redo/reader/blackhole_reader.go @@ -16,7 +16,7 @@ package reader import ( "context" - "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/tiflow/cdc/model" ) // BlackHoleReader is a blockHole storage which implements LogReader interface diff --git a/cdc/redo/reader/file.go b/cdc/redo/reader/file.go index d809feb5b9f..976d4a474a5 100644 --- a/cdc/redo/reader/file.go +++ b/cdc/redo/reader/file.go @@ -29,11 +29,11 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/redo/common" - "github.com/pingcap/ticdc/cdc/redo/writer" - cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/redo/common" + "github.com/pingcap/tiflow/cdc/redo/writer" + cerror "github.com/pingcap/tiflow/pkg/errors" "go.uber.org/multierr" "go.uber.org/zap" "golang.org/x/sync/errgroup" diff --git a/cdc/redo/reader/file_test.go b/cdc/redo/reader/file_test.go index 0de8bc77295..47c55dbf303 100644 --- a/cdc/redo/reader/file_test.go +++ b/cdc/redo/reader/file_test.go @@ -23,10 +23,10 @@ import ( "testing" "time" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/redo/common" - "github.com/pingcap/ticdc/cdc/redo/writer" - "github.com/pingcap/ticdc/pkg/leakutil" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/redo/common" + "github.com/pingcap/tiflow/cdc/redo/writer" + "github.com/pingcap/tiflow/pkg/leakutil" "github.com/stretchr/testify/require" "golang.org/x/net/context" ) diff --git a/cdc/redo/reader/mock_RedoLogReader.go b/cdc/redo/reader/mock_RedoLogReader.go index d949a3194f8..484c735046d 100644 --- a/cdc/redo/reader/mock_RedoLogReader.go +++ b/cdc/redo/reader/mock_RedoLogReader.go @@ -18,7 +18,7 @@ package reader import ( context "context" - model "github.com/pingcap/ticdc/cdc/model" + model "github.com/pingcap/tiflow/cdc/model" mock "github.com/stretchr/testify/mock" ) diff --git a/cdc/redo/reader/mock_fileReader.go b/cdc/redo/reader/mock_fileReader.go index de323d8e270..3339fb7237a 100644 --- a/cdc/redo/reader/mock_fileReader.go +++ b/cdc/redo/reader/mock_fileReader.go @@ -16,7 +16,7 @@ package reader import ( - model "github.com/pingcap/ticdc/cdc/model" + model "github.com/pingcap/tiflow/cdc/model" mock "github.com/stretchr/testify/mock" ) diff --git a/cdc/redo/reader/reader.go b/cdc/redo/reader/reader.go index 339c5184956..e1c88491ddc 100644 --- a/cdc/redo/reader/reader.go +++ b/cdc/redo/reader/reader.go @@ -24,9 +24,9 @@ import ( "sync" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/redo/common" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/redo/common" + cerror "github.com/pingcap/tiflow/pkg/errors" "go.uber.org/multierr" ) diff --git a/cdc/redo/reader/reader_test.go b/cdc/redo/reader/reader_test.go index 2d89bafe5ab..dd963d7cf55 100644 --- a/cdc/redo/reader/reader_test.go +++ b/cdc/redo/reader/reader_test.go @@ -26,11 +26,11 @@ import ( "github.com/golang/mock/gomock" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/redo/common" - "github.com/pingcap/ticdc/cdc/redo/writer" mockstorage "github.com/pingcap/tidb/br/pkg/mock/storage" "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/redo/common" + "github.com/pingcap/tiflow/cdc/redo/writer" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" "go.uber.org/multierr" diff --git a/cdc/redo/writer/blackhole_writer.go b/cdc/redo/writer/blackhole_writer.go index 9accc98086b..6682edef6f3 100644 --- a/cdc/redo/writer/blackhole_writer.go +++ b/cdc/redo/writer/blackhole_writer.go @@ -18,7 +18,7 @@ import ( "sync" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/tiflow/cdc/model" "go.uber.org/zap" ) diff --git a/cdc/redo/writer/file.go b/cdc/redo/writer/file.go index d307f98f45c..86a636af3d1 100644 --- a/cdc/redo/writer/file.go +++ b/cdc/redo/writer/file.go @@ -28,9 +28,9 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/redo/common" - cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tiflow/cdc/redo/common" + cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/uber-go/atomic" pioutil "go.etcd.io/etcd/pkg/ioutil" "go.uber.org/multierr" diff --git a/cdc/redo/writer/file_test.go b/cdc/redo/writer/file_test.go index ad3a7f7bdb7..e71b814d553 100644 --- a/cdc/redo/writer/file_test.go +++ b/cdc/redo/writer/file_test.go @@ -25,9 +25,9 @@ import ( "github.com/golang/mock/gomock" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/redo/common" - "github.com/pingcap/ticdc/pkg/leakutil" mockstorage "github.com/pingcap/tidb/br/pkg/mock/storage" + "github.com/pingcap/tiflow/cdc/redo/common" + "github.com/pingcap/tiflow/pkg/leakutil" "github.com/stretchr/testify/require" "github.com/uber-go/atomic" ) diff --git a/cdc/redo/writer/mock_RedoLogWriter.go b/cdc/redo/writer/mock_RedoLogWriter.go index 3b7416888fa..361aa6f61ab 100644 --- a/cdc/redo/writer/mock_RedoLogWriter.go +++ b/cdc/redo/writer/mock_RedoLogWriter.go @@ -18,7 +18,7 @@ package writer import ( context "context" - model "github.com/pingcap/ticdc/cdc/model" + model "github.com/pingcap/tiflow/cdc/model" mock "github.com/stretchr/testify/mock" ) diff --git a/cdc/redo/writer/writer.go b/cdc/redo/writer/writer.go index 37b019c0742..fceb3de233b 100644 --- a/cdc/redo/writer/writer.go +++ b/cdc/redo/writer/writer.go @@ -28,10 +28,10 @@ import ( "github.com/aws/aws-sdk-go/service/s3" "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/redo/common" - cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/redo/common" + cerror "github.com/pingcap/tiflow/pkg/errors" "go.uber.org/multierr" "go.uber.org/zap" "golang.org/x/sync/errgroup" diff --git a/cdc/redo/writer/writer_test.go b/cdc/redo/writer/writer_test.go index bd7432ca49a..618281b5537 100644 --- a/cdc/redo/writer/writer_test.go +++ b/cdc/redo/writer/writer_test.go @@ -28,10 +28,10 @@ import ( "github.com/aws/aws-sdk-go/service/s3" "github.com/golang/mock/gomock" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/redo/common" - cerror "github.com/pingcap/ticdc/pkg/errors" mockstorage "github.com/pingcap/tidb/br/pkg/mock/storage" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/redo/common" + cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" "go.uber.org/multierr" diff --git a/cdc/server.go b/cdc/server.go index 06ace4658ff..9decacdbd77 100644 --- a/cdc/server.go +++ b/cdc/server.go @@ -24,16 +24,16 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/capture" - "github.com/pingcap/ticdc/cdc/kv" - "github.com/pingcap/ticdc/cdc/sorter/unified" - "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/httputil" - "github.com/pingcap/ticdc/pkg/util" - "github.com/pingcap/ticdc/pkg/version" tidbkv "github.com/pingcap/tidb/kv" + "github.com/pingcap/tiflow/cdc/capture" + "github.com/pingcap/tiflow/cdc/kv" + "github.com/pingcap/tiflow/cdc/sorter/unified" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/httputil" + "github.com/pingcap/tiflow/pkg/util" + "github.com/pingcap/tiflow/pkg/version" "github.com/prometheus/client_golang/prometheus" pd "github.com/tikv/pd/client" "go.etcd.io/etcd/clientv3" diff --git a/cdc/server_test.go b/cdc/server_test.go index 0ff6241080d..5fdbe3c785f 100644 --- a/cdc/server_test.go +++ b/cdc/server_test.go @@ -20,11 +20,11 @@ import ( "time" "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/util" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/util" + "github.com/pingcap/tiflow/pkg/util/testleak" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/embed" "golang.org/x/sync/errgroup" diff --git a/cdc/sink/black_hole.go b/cdc/sink/black_hole.go index 3eca14a0119..42e623422f7 100644 --- a/cdc/sink/black_hole.go +++ b/cdc/sink/black_hole.go @@ -18,7 +18,7 @@ import ( "sync/atomic" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/tiflow/cdc/model" "go.uber.org/zap" ) diff --git a/cdc/sink/buffer_sink.go b/cdc/sink/buffer_sink.go index d8d70456ca7..73cc3be0b9d 100644 --- a/cdc/sink/buffer_sink.go +++ b/cdc/sink/buffer_sink.go @@ -22,8 +22,8 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" ) diff --git a/cdc/sink/causality.go b/cdc/sink/causality.go index 0a9377c36e2..0d7c0389735 100644 --- a/cdc/sink/causality.go +++ b/cdc/sink/causality.go @@ -19,7 +19,7 @@ import ( "github.com/pingcap/log" "go.uber.org/zap" - "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/tiflow/cdc/model" ) // causality provides a simple mechanism to improve the concurrency of SQLs execution under the premise of ensuring correctness. diff --git a/cdc/sink/causality_test.go b/cdc/sink/causality_test.go index e7a14ecc008..06dbdd1e5da 100644 --- a/cdc/sink/causality_test.go +++ b/cdc/sink/causality_test.go @@ -18,9 +18,9 @@ import ( "sort" "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/util/testleak" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type testCausalitySuite struct{} diff --git a/cdc/sink/cdclog/file.go b/cdc/sink/cdclog/file.go index c6794b3c6e2..ee57b4e3450 100644 --- a/cdc/sink/cdclog/file.go +++ b/cdc/sink/cdclog/file.go @@ -21,11 +21,11 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/sink/codec" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/quotes" parsemodel "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sink/codec" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/quotes" "github.com/uber-go/atomic" "go.uber.org/zap" ) diff --git a/cdc/sink/cdclog/s3.go b/cdc/sink/cdclog/s3.go index f76d6c23946..2b6da88cb1f 100644 --- a/cdc/sink/cdclog/s3.go +++ b/cdc/sink/cdclog/s3.go @@ -22,12 +22,12 @@ import ( "github.com/pingcap/errors" backup "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/sink/codec" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/quotes" "github.com/pingcap/tidb/br/pkg/storage" parsemodel "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sink/codec" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/quotes" "github.com/uber-go/atomic" "go.uber.org/zap" ) diff --git a/cdc/sink/cdclog/utils.go b/cdc/sink/cdclog/utils.go index f2b351ca836..4c328cd4c6f 100644 --- a/cdc/sink/cdclog/utils.go +++ b/cdc/sink/cdclog/utils.go @@ -21,10 +21,10 @@ import ( "time" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/sink/codec" - "github.com/pingcap/ticdc/pkg/quotes" "github.com/pingcap/tidb/br/pkg/storage" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sink/codec" + "github.com/pingcap/tiflow/pkg/quotes" "github.com/uber-go/atomic" "go.uber.org/zap" "golang.org/x/sync/errgroup" diff --git a/cdc/sink/codec/avro.go b/cdc/sink/codec/avro.go index eed118c3ce8..cb6ed9e9952 100644 --- a/cdc/sink/codec/avro.go +++ b/cdc/sink/codec/avro.go @@ -26,12 +26,12 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/security" - "github.com/pingcap/ticdc/pkg/util" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/types" + "github.com/pingcap/tiflow/cdc/model" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/security" + "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" ) diff --git a/cdc/sink/codec/avro_test.go b/cdc/sink/codec/avro_test.go index 4b0290fa726..0d243110359 100644 --- a/cdc/sink/codec/avro_test.go +++ b/cdc/sink/codec/avro_test.go @@ -21,14 +21,14 @@ import ( "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/puller" - "github.com/pingcap/ticdc/pkg/regionspan" - "github.com/pingcap/ticdc/pkg/security" - "github.com/pingcap/ticdc/pkg/util/testleak" model2 "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/types" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/puller" + "github.com/pingcap/tiflow/pkg/regionspan" + "github.com/pingcap/tiflow/pkg/security" + "github.com/pingcap/tiflow/pkg/util/testleak" "go.uber.org/zap" ) diff --git a/cdc/sink/codec/canal.go b/cdc/sink/codec/canal.go index 91427c9bb8d..089103d2f8d 100644 --- a/cdc/sink/codec/canal.go +++ b/cdc/sink/codec/canal.go @@ -22,12 +22,12 @@ import ( "github.com/golang/protobuf/proto" // nolint:staticcheck "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - cerror "github.com/pingcap/ticdc/pkg/errors" - canal "github.com/pingcap/ticdc/proto/canal" mm "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" parser_types "github.com/pingcap/tidb/parser/types" + "github.com/pingcap/tiflow/cdc/model" + cerror "github.com/pingcap/tiflow/pkg/errors" + canal "github.com/pingcap/tiflow/proto/canal" "go.uber.org/zap" "golang.org/x/text/encoding" "golang.org/x/text/encoding/charmap" diff --git a/cdc/sink/codec/canal_flat.go b/cdc/sink/codec/canal_flat.go index 5929488db07..93579556ecd 100644 --- a/cdc/sink/codec/canal_flat.go +++ b/cdc/sink/codec/canal_flat.go @@ -19,9 +19,9 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - cerrors "github.com/pingcap/ticdc/pkg/errors" - canal "github.com/pingcap/ticdc/proto/canal" + "github.com/pingcap/tiflow/cdc/model" + cerrors "github.com/pingcap/tiflow/pkg/errors" + canal "github.com/pingcap/tiflow/proto/canal" "go.uber.org/zap" ) diff --git a/cdc/sink/codec/canal_flat_test.go b/cdc/sink/codec/canal_flat_test.go index 14df059e673..0d6cbb7531b 100644 --- a/cdc/sink/codec/canal_flat_test.go +++ b/cdc/sink/codec/canal_flat_test.go @@ -17,10 +17,10 @@ import ( "encoding/json" "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/util/testleak" mm "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/util/testleak" "golang.org/x/text/encoding/charmap" ) diff --git a/cdc/sink/codec/canal_test.go b/cdc/sink/codec/canal_test.go index 770861c568a..bf8f0fa113d 100644 --- a/cdc/sink/codec/canal_test.go +++ b/cdc/sink/codec/canal_test.go @@ -20,9 +20,9 @@ import ( "github.com/pingcap/tidb/parser/mysql" "golang.org/x/text/encoding/charmap" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/util/testleak" - canal "github.com/pingcap/ticdc/proto/canal" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/util/testleak" + canal "github.com/pingcap/tiflow/proto/canal" ) type canalBatchSuite struct { diff --git a/cdc/sink/codec/codec_test.go b/cdc/sink/codec/codec_test.go index 75b55d31846..57744272290 100644 --- a/cdc/sink/codec/codec_test.go +++ b/cdc/sink/codec/codec_test.go @@ -19,11 +19,11 @@ import ( "testing" "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/sink/codec/craft" - "github.com/pingcap/ticdc/pkg/util/testleak" - "github.com/pingcap/ticdc/proto/benchmark" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sink/codec/craft" + "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/pingcap/tiflow/proto/benchmark" ) var ( diff --git a/cdc/sink/codec/craft.go b/cdc/sink/codec/craft.go index ebf5a4b5960..8cae661b434 100644 --- a/cdc/sink/codec/craft.go +++ b/cdc/sink/codec/craft.go @@ -19,9 +19,9 @@ import ( "strconv" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/sink/codec/craft" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sink/codec/craft" + cerror "github.com/pingcap/tiflow/pkg/errors" ) // CraftEventBatchEncoder encodes the events into the byte of a batch into craft binary format. diff --git a/cdc/sink/codec/craft/codec_test.go b/cdc/sink/codec/craft/codec_test.go index 38221a136bc..7da187c3f2a 100644 --- a/cdc/sink/codec/craft/codec_test.go +++ b/cdc/sink/codec/craft/codec_test.go @@ -19,7 +19,7 @@ import ( "time" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" ) func init() { diff --git a/cdc/sink/codec/craft/decoder.go b/cdc/sink/codec/craft/decoder.go index 66a947bf7e0..6295d7586d6 100644 --- a/cdc/sink/codec/craft/decoder.go +++ b/cdc/sink/codec/craft/decoder.go @@ -19,10 +19,10 @@ import ( "unsafe" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/model" - cerror "github.com/pingcap/ticdc/pkg/errors" pmodel "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/cdc/model" + cerror "github.com/pingcap/tiflow/pkg/errors" ) /// create string from byte slice without copying diff --git a/cdc/sink/codec/craft/encoder.go b/cdc/sink/codec/craft/encoder.go index 267fb262560..69938f99a27 100644 --- a/cdc/sink/codec/craft/encoder.go +++ b/cdc/sink/codec/craft/encoder.go @@ -18,8 +18,8 @@ import ( "math" "unsafe" - "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/cdc/model" ) // create byte slice from string without copying diff --git a/cdc/sink/codec/craft/model.go b/cdc/sink/codec/craft/model.go index f467d0f980d..61ea83098f1 100644 --- a/cdc/sink/codec/craft/model.go +++ b/cdc/sink/codec/craft/model.go @@ -15,8 +15,8 @@ package craft import ( "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/model" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/tiflow/cdc/model" + cerror "github.com/pingcap/tiflow/pkg/errors" ) const ( diff --git a/cdc/sink/codec/craft_test.go b/cdc/sink/codec/craft_test.go index 91e7410f1e6..e981e775f97 100644 --- a/cdc/sink/codec/craft_test.go +++ b/cdc/sink/codec/craft_test.go @@ -18,9 +18,9 @@ import ( "strconv" "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/util/testleak" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type craftBatchSuite struct { diff --git a/cdc/sink/codec/interface.go b/cdc/sink/codec/interface.go index 2bfcecd1dc3..ca8548dae7f 100644 --- a/cdc/sink/codec/interface.go +++ b/cdc/sink/codec/interface.go @@ -20,8 +20,8 @@ import ( "time" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/security" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/security" "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" ) diff --git a/cdc/sink/codec/interface_test.go b/cdc/sink/codec/interface_test.go index 6c28b52fb78..f82b4fb49bc 100644 --- a/cdc/sink/codec/interface_test.go +++ b/cdc/sink/codec/interface_test.go @@ -15,11 +15,11 @@ package codec import ( "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/util/testleak" timodel "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/types" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type codecInterfaceSuite struct{} diff --git a/cdc/sink/codec/json.go b/cdc/sink/codec/json.go index 4c3c44b65b6..c7bb5dbc7ba 100644 --- a/cdc/sink/codec/json.go +++ b/cdc/sink/codec/json.go @@ -25,12 +25,12 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - cerror "github.com/pingcap/ticdc/pkg/errors" timodel "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + cerror "github.com/pingcap/tiflow/pkg/errors" "go.uber.org/zap" - "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/tiflow/cdc/model" ) const ( diff --git a/cdc/sink/codec/json_test.go b/cdc/sink/codec/json_test.go index 461524de463..7f02c564904 100644 --- a/cdc/sink/codec/json_test.go +++ b/cdc/sink/codec/json_test.go @@ -20,9 +20,9 @@ import ( "testing" "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/util/testleak" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/util/testleak" ) func Test(t *testing.T) { check.TestingT(t) } diff --git a/cdc/sink/codec/maxwell.go b/cdc/sink/codec/maxwell.go index 9be0f7ba305..d6e54c72448 100644 --- a/cdc/sink/codec/maxwell.go +++ b/cdc/sink/codec/maxwell.go @@ -20,10 +20,10 @@ import ( "encoding/json" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/model" - cerror "github.com/pingcap/ticdc/pkg/errors" model2 "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/cdc/model" + cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/tikv/pd/pkg/tsoutil" ) diff --git a/cdc/sink/codec/maxwell_test.go b/cdc/sink/codec/maxwell_test.go index f624efc453d..5e6fea4a58e 100644 --- a/cdc/sink/codec/maxwell_test.go +++ b/cdc/sink/codec/maxwell_test.go @@ -15,8 +15,8 @@ package codec import ( "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type maxwellbatchSuite struct { diff --git a/cdc/sink/codec/schema_registry.go b/cdc/sink/codec/schema_registry.go index 24b166cd21f..bca5cf6b57d 100644 --- a/cdc/sink/codec/schema_registry.go +++ b/cdc/sink/codec/schema_registry.go @@ -29,10 +29,10 @@ import ( "github.com/linkedin/goavro/v2" "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/httputil" - "github.com/pingcap/ticdc/pkg/security" + "github.com/pingcap/tiflow/cdc/model" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/httputil" + "github.com/pingcap/tiflow/pkg/security" "go.uber.org/zap" ) diff --git a/cdc/sink/codec/schema_registry_test.go b/cdc/sink/codec/schema_registry_test.go index b6bc5795b5b..57759e07b66 100644 --- a/cdc/sink/codec/schema_registry_test.go +++ b/cdc/sink/codec/schema_registry_test.go @@ -25,9 +25,9 @@ import ( "github.com/jarcoal/httpmock" "github.com/linkedin/goavro/v2" "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/security" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/security" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type AvroSchemaRegistrySuite struct{} diff --git a/cdc/sink/common/common.go b/cdc/sink/common/common.go index 844e2f0bedd..cf018a7c8b5 100644 --- a/cdc/sink/common/common.go +++ b/cdc/sink/common/common.go @@ -19,8 +19,8 @@ import ( "sync/atomic" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/filter" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/filter" "go.uber.org/zap" ) diff --git a/cdc/sink/common/common_test.go b/cdc/sink/common/common_test.go index 28a87086337..4cadba85e56 100644 --- a/cdc/sink/common/common_test.go +++ b/cdc/sink/common/common_test.go @@ -19,8 +19,8 @@ import ( "github.com/google/go-cmp/cmp" "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type SinkCommonSuite struct{} diff --git a/cdc/sink/common/flow_control.go b/cdc/sink/common/flow_control.go index a008d58b25a..9a13cc38389 100644 --- a/cdc/sink/common/flow_control.go +++ b/cdc/sink/common/flow_control.go @@ -20,7 +20,7 @@ import ( "github.com/edwingeng/deque" "github.com/pingcap/errors" "github.com/pingcap/log" - cerrors "github.com/pingcap/ticdc/pkg/errors" + cerrors "github.com/pingcap/tiflow/pkg/errors" "go.uber.org/zap" ) diff --git a/cdc/sink/common/flow_control_test.go b/cdc/sink/common/flow_control_test.go index 1b002f45000..d7c9dbefe26 100644 --- a/cdc/sink/common/flow_control_test.go +++ b/cdc/sink/common/flow_control_test.go @@ -22,7 +22,7 @@ import ( "time" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" "golang.org/x/sync/errgroup" ) diff --git a/cdc/sink/dispatcher/default.go b/cdc/sink/dispatcher/default.go index 741f0cad3f2..31d75eb0abd 100644 --- a/cdc/sink/dispatcher/default.go +++ b/cdc/sink/dispatcher/default.go @@ -14,7 +14,7 @@ package dispatcher import ( - "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/tiflow/cdc/model" ) type defaultDispatcher struct { diff --git a/cdc/sink/dispatcher/default_test.go b/cdc/sink/dispatcher/default_test.go index eb2d13540e7..93fe3952af1 100644 --- a/cdc/sink/dispatcher/default_test.go +++ b/cdc/sink/dispatcher/default_test.go @@ -15,8 +15,8 @@ package dispatcher import ( "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type DefaultDispatcherSuite struct{} diff --git a/cdc/sink/dispatcher/dispatcher.go b/cdc/sink/dispatcher/dispatcher.go index 7d6f86873c3..6b4818295d1 100644 --- a/cdc/sink/dispatcher/dispatcher.go +++ b/cdc/sink/dispatcher/dispatcher.go @@ -17,10 +17,10 @@ import ( "strings" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" filter "github.com/pingcap/tidb-tools/pkg/table-filter" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" "go.uber.org/zap" ) diff --git a/cdc/sink/dispatcher/index_value.go b/cdc/sink/dispatcher/index_value.go index e9824fc2808..79913bc636c 100644 --- a/cdc/sink/dispatcher/index_value.go +++ b/cdc/sink/dispatcher/index_value.go @@ -14,8 +14,8 @@ package dispatcher import ( - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/hash" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/hash" ) type indexValueDispatcher struct { diff --git a/cdc/sink/dispatcher/index_value_test.go b/cdc/sink/dispatcher/index_value_test.go index cbbffbaca90..5f0890885c2 100644 --- a/cdc/sink/dispatcher/index_value_test.go +++ b/cdc/sink/dispatcher/index_value_test.go @@ -15,8 +15,8 @@ package dispatcher import ( "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type IndexValueDispatcherSuite struct{} diff --git a/cdc/sink/dispatcher/switcher_test.go b/cdc/sink/dispatcher/switcher_test.go index e6fc8d7c984..6f3caf20143 100644 --- a/cdc/sink/dispatcher/switcher_test.go +++ b/cdc/sink/dispatcher/switcher_test.go @@ -15,9 +15,9 @@ package dispatcher import ( "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type SwitcherSuite struct{} diff --git a/cdc/sink/dispatcher/table.go b/cdc/sink/dispatcher/table.go index 28d4c04d464..98617f7ce22 100644 --- a/cdc/sink/dispatcher/table.go +++ b/cdc/sink/dispatcher/table.go @@ -14,8 +14,8 @@ package dispatcher import ( - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/hash" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/hash" ) type tableDispatcher struct { diff --git a/cdc/sink/dispatcher/table_test.go b/cdc/sink/dispatcher/table_test.go index 5aeb9d80936..9b2f17c4108 100644 --- a/cdc/sink/dispatcher/table_test.go +++ b/cdc/sink/dispatcher/table_test.go @@ -15,8 +15,8 @@ package dispatcher import ( "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type TableDispatcherSuite struct{} diff --git a/cdc/sink/dispatcher/ts.go b/cdc/sink/dispatcher/ts.go index 06d1fb1634a..e061734d210 100644 --- a/cdc/sink/dispatcher/ts.go +++ b/cdc/sink/dispatcher/ts.go @@ -13,7 +13,7 @@ package dispatcher -import "github.com/pingcap/ticdc/cdc/model" +import "github.com/pingcap/tiflow/cdc/model" type tsDispatcher struct { partitionNum int32 diff --git a/cdc/sink/dispatcher/ts_test.go b/cdc/sink/dispatcher/ts_test.go index 31099f6067a..f4eac2471bb 100644 --- a/cdc/sink/dispatcher/ts_test.go +++ b/cdc/sink/dispatcher/ts_test.go @@ -17,8 +17,8 @@ import ( "testing" "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/util/testleak" ) func Test(t *testing.T) { check.TestingT(t) } diff --git a/cdc/sink/manager.go b/cdc/sink/manager.go index 8df65c0d9b0..dd09ccd412c 100644 --- a/cdc/sink/manager.go +++ b/cdc/sink/manager.go @@ -22,8 +22,8 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - redo "github.com/pingcap/ticdc/cdc/redo" + "github.com/pingcap/tiflow/cdc/model" + redo "github.com/pingcap/tiflow/cdc/redo" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" ) diff --git a/cdc/sink/manager_test.go b/cdc/sink/manager_test.go index 29ea4bd83d2..912100b9eb6 100644 --- a/cdc/sink/manager_test.go +++ b/cdc/sink/manager_test.go @@ -24,9 +24,9 @@ import ( "github.com/pingcap/check" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/redo" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/redo" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type managerSuite struct{} @@ -251,7 +251,7 @@ func (s *managerSuite) TestManagerDestroyTableSink(c *check.C) { } // Run the benchmark -// go test -benchmem -run='^$' -bench '^(BenchmarkManagerFlushing)$' github.com/pingcap/ticdc/cdc/sink +// go test -benchmem -run='^$' -bench '^(BenchmarkManagerFlushing)$' github.com/pingcap/tiflow/cdc/sink func BenchmarkManagerFlushing(b *testing.B) { ctx, cancel := context.WithCancel(context.Background()) errCh := make(chan error, 16) diff --git a/cdc/sink/mq.go b/cdc/sink/mq.go index c1677232274..0da86336b62 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -22,17 +22,17 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/sink/codec" - "github.com/pingcap/ticdc/cdc/sink/dispatcher" - "github.com/pingcap/ticdc/cdc/sink/producer" - "github.com/pingcap/ticdc/cdc/sink/producer/kafka" - "github.com/pingcap/ticdc/cdc/sink/producer/pulsar" - "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/filter" - "github.com/pingcap/ticdc/pkg/notify" - "github.com/pingcap/ticdc/pkg/security" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sink/codec" + "github.com/pingcap/tiflow/cdc/sink/dispatcher" + "github.com/pingcap/tiflow/cdc/sink/producer" + "github.com/pingcap/tiflow/cdc/sink/producer/kafka" + "github.com/pingcap/tiflow/cdc/sink/producer/pulsar" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/filter" + "github.com/pingcap/tiflow/pkg/notify" + "github.com/pingcap/tiflow/pkg/security" "go.uber.org/zap" "golang.org/x/sync/errgroup" ) diff --git a/cdc/sink/mq_test.go b/cdc/sink/mq_test.go index b3f80867a59..c94152b7e05 100644 --- a/cdc/sink/mq_test.go +++ b/cdc/sink/mq_test.go @@ -19,16 +19,16 @@ import ( "net/url" "github.com/pingcap/failpoint" - "github.com/pingcap/ticdc/cdc/sink/codec" + "github.com/pingcap/tiflow/cdc/sink/codec" "github.com/Shopify/sarama" "github.com/pingcap/check" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/filter" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/filter" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type mqSinkSuite struct{} @@ -199,7 +199,7 @@ func (s mqSinkSuite) TestPulsarSinkEncoderConfig(c *check.C) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - err := failpoint.Enable("github.com/pingcap/ticdc/cdc/sink/producer/pulsar/MockPulsar", "return(true)") + err := failpoint.Enable("github.com/pingcap/tiflow/cdc/sink/producer/pulsar/MockPulsar", "return(true)") c.Assert(err, check.IsNil) uri := "pulsar://127.0.0.1:1234/kafka-test?" + diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index b74efd48736..d0704ba8a92 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -29,23 +29,23 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/sink/common" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/cyclic" - "github.com/pingcap/ticdc/pkg/cyclic/mark" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/filter" - tifilter "github.com/pingcap/ticdc/pkg/filter" - "github.com/pingcap/ticdc/pkg/notify" - "github.com/pingcap/ticdc/pkg/quotes" - "github.com/pingcap/ticdc/pkg/retry" - "github.com/pingcap/ticdc/pkg/security" - "github.com/pingcap/ticdc/pkg/util" tddl "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/infoschema" timodel "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sink/common" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/cyclic" + "github.com/pingcap/tiflow/pkg/cyclic/mark" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/filter" + tifilter "github.com/pingcap/tiflow/pkg/filter" + "github.com/pingcap/tiflow/pkg/notify" + "github.com/pingcap/tiflow/pkg/quotes" + "github.com/pingcap/tiflow/pkg/retry" + "github.com/pingcap/tiflow/pkg/security" + "github.com/pingcap/tiflow/pkg/util" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" ) diff --git a/cdc/sink/mysql_test.go b/cdc/sink/mysql_test.go index 8a9ad5b1a31..bc242861eba 100644 --- a/cdc/sink/mysql_test.go +++ b/cdc/sink/mysql_test.go @@ -31,18 +31,18 @@ import ( dmysql "github.com/go-sql-driver/mysql" "github.com/pingcap/check" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/sink/common" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/cyclic/mark" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/filter" - "github.com/pingcap/ticdc/pkg/notify" - "github.com/pingcap/ticdc/pkg/retry" - "github.com/pingcap/ticdc/pkg/util/testleak" "github.com/pingcap/tidb/infoschema" timodel "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sink/common" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/cyclic/mark" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/filter" + "github.com/pingcap/tiflow/pkg/notify" + "github.com/pingcap/tiflow/pkg/retry" + "github.com/pingcap/tiflow/pkg/util/testleak" "golang.org/x/sync/errgroup" ) diff --git a/cdc/sink/producer/kafka/kafka.go b/cdc/sink/producer/kafka/kafka.go index e718f543dcc..9b9fc650992 100644 --- a/cdc/sink/producer/kafka/kafka.go +++ b/cdc/sink/producer/kafka/kafka.go @@ -28,12 +28,12 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/sink/codec" - "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/notify" - "github.com/pingcap/ticdc/pkg/security" - "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/tiflow/cdc/sink/codec" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/notify" + "github.com/pingcap/tiflow/pkg/security" + "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" ) @@ -550,7 +550,7 @@ func newSaramaConfig(ctx context.Context, c *Config) (*sarama.Config, error) { // Metadata.Retry.Backoff * Metadata.Retry.Max` // to fail. // See: https://github.com/Shopify/sarama/issues/765 - // and https://github.com/pingcap/ticdc/issues/3352. + // and https://github.com/pingcap/tiflow/issues/3352. config.Metadata.Timeout = 1 * time.Minute config.Producer.Partitioner = sarama.NewManualPartitioner diff --git a/cdc/sink/producer/kafka/kafka_test.go b/cdc/sink/producer/kafka/kafka_test.go index b4f72df36bd..2206b96d582 100644 --- a/cdc/sink/producer/kafka/kafka_test.go +++ b/cdc/sink/producer/kafka/kafka_test.go @@ -24,12 +24,12 @@ import ( "github.com/Shopify/sarama" "github.com/pingcap/check" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/sink/codec" - "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/security" - "github.com/pingcap/ticdc/pkg/util" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/sink/codec" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/security" + "github.com/pingcap/tiflow/pkg/util" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type kafkaSuite struct{} diff --git a/cdc/sink/producer/mq_producer.go b/cdc/sink/producer/mq_producer.go index 380065b679b..7e9664f61ed 100644 --- a/cdc/sink/producer/mq_producer.go +++ b/cdc/sink/producer/mq_producer.go @@ -16,7 +16,7 @@ package producer import ( "context" - "github.com/pingcap/ticdc/cdc/sink/codec" + "github.com/pingcap/tiflow/cdc/sink/codec" ) // Producer is an interface of mq producer diff --git a/cdc/sink/producer/pulsar/producer.go b/cdc/sink/producer/pulsar/producer.go index 26cb6c3485a..bf9664d2dce 100644 --- a/cdc/sink/producer/pulsar/producer.go +++ b/cdc/sink/producer/pulsar/producer.go @@ -21,8 +21,8 @@ import ( "github.com/apache/pulsar-client-go/pulsar" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/sink/codec" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/tiflow/cdc/sink/codec" + cerror "github.com/pingcap/tiflow/pkg/errors" "go.uber.org/zap" ) diff --git a/cdc/sink/simple_mysql_tester.go b/cdc/sink/simple_mysql_tester.go index 3b3d7edc885..2bb46c14bb9 100644 --- a/cdc/sink/simple_mysql_tester.go +++ b/cdc/sink/simple_mysql_tester.go @@ -25,11 +25,11 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/filter" - "github.com/pingcap/ticdc/pkg/quotes" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/filter" + "github.com/pingcap/tiflow/pkg/quotes" "go.uber.org/zap" ) diff --git a/cdc/sink/sink.go b/cdc/sink/sink.go index c5d7cd18ca0..155012bc28a 100644 --- a/cdc/sink/sink.go +++ b/cdc/sink/sink.go @@ -18,11 +18,11 @@ import ( "net/url" "strings" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/sink/cdclog" - "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/filter" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sink/cdclog" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/filter" ) // Sink options keys diff --git a/cdc/sink/sink_test.go b/cdc/sink/sink_test.go index 557239a9861..ae68c9f3984 100644 --- a/cdc/sink/sink_test.go +++ b/cdc/sink/sink_test.go @@ -17,8 +17,8 @@ import ( "context" "testing" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/util/testleak" "github.com/stretchr/testify/require" ) diff --git a/cdc/sink/statistics.go b/cdc/sink/statistics.go index b09a8c988c1..26c1bad7fb4 100644 --- a/cdc/sink/statistics.go +++ b/cdc/sink/statistics.go @@ -19,7 +19,7 @@ import ( "time" "github.com/pingcap/log" - "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/tiflow/pkg/util" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" ) diff --git a/cdc/sink/syncpointStore.go b/cdc/sink/syncpointStore.go index d20cb5e5960..230993ffbae 100644 --- a/cdc/sink/syncpointStore.go +++ b/cdc/sink/syncpointStore.go @@ -18,8 +18,8 @@ import ( "net/url" "strings" - "github.com/pingcap/ticdc/cdc/model" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/tiflow/cdc/model" + cerror "github.com/pingcap/tiflow/pkg/errors" ) // SyncpointStore is an abstraction for anything that a changefeed may emit into. diff --git a/cdc/sink/table_sink.go b/cdc/sink/table_sink.go index 1e9679c7bc4..83a1acb8c86 100644 --- a/cdc/sink/table_sink.go +++ b/cdc/sink/table_sink.go @@ -20,8 +20,8 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/redo" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/redo" "go.uber.org/zap" ) diff --git a/cdc/sink/txns_heap.go b/cdc/sink/txns_heap.go index a514a05dd38..16dd48e53bb 100644 --- a/cdc/sink/txns_heap.go +++ b/cdc/sink/txns_heap.go @@ -16,7 +16,7 @@ package sink import ( "container/heap" - "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/tiflow/cdc/model" ) type innerTxnsHeap []innerHeapEntry diff --git a/cdc/sink/txns_heap_test.go b/cdc/sink/txns_heap_test.go index b488260419c..7084dc8ea47 100644 --- a/cdc/sink/txns_heap_test.go +++ b/cdc/sink/txns_heap_test.go @@ -15,8 +15,8 @@ package sink import ( "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type TxnsHeapSuite struct{} diff --git a/cdc/sorter/encoding/key.go b/cdc/sorter/encoding/key.go index 53a95032313..a3adf9ded5b 100644 --- a/cdc/sorter/encoding/key.go +++ b/cdc/sorter/encoding/key.go @@ -17,7 +17,7 @@ import ( "encoding/binary" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/tiflow/cdc/model" "go.uber.org/zap" ) diff --git a/cdc/sorter/encoding/key_test.go b/cdc/sorter/encoding/key_test.go index d52d2e04aa7..f4020e8f1ac 100644 --- a/cdc/sorter/encoding/key_test.go +++ b/cdc/sorter/encoding/key_test.go @@ -17,7 +17,7 @@ import ( "bytes" "testing" - "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/tiflow/cdc/model" "github.com/stretchr/testify/require" ) diff --git a/cdc/sorter/encoding/value.go b/cdc/sorter/encoding/value.go index ec976dd2eaf..f29e1c95525 100644 --- a/cdc/sorter/encoding/value.go +++ b/cdc/sorter/encoding/value.go @@ -15,7 +15,7 @@ package encoding import ( "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/tiflow/cdc/model" ) // SerializerDeserializer is the interface encodes and decodes model.PolymorphicEvent. diff --git a/cdc/sorter/memory/entry_sorter.go b/cdc/sorter/memory/entry_sorter.go index f8132088dcf..916fad5dc32 100644 --- a/cdc/sorter/memory/entry_sorter.go +++ b/cdc/sorter/memory/entry_sorter.go @@ -22,9 +22,9 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/notify" - "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/notify" + "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" "golang.org/x/sync/errgroup" ) diff --git a/cdc/sorter/memory/entry_sorter_test.go b/cdc/sorter/memory/entry_sorter_test.go index 55e3229bc7a..f8c608f5def 100644 --- a/cdc/sorter/memory/entry_sorter_test.go +++ b/cdc/sorter/memory/entry_sorter_test.go @@ -21,8 +21,8 @@ import ( "github.com/pingcap/check" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type mockEntrySorterSuite struct{} diff --git a/cdc/sorter/sorter.go b/cdc/sorter/sorter.go index 9145d5534fc..512f47b776c 100644 --- a/cdc/sorter/sorter.go +++ b/cdc/sorter/sorter.go @@ -16,7 +16,7 @@ package sorter import ( "context" - "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/tiflow/cdc/model" ) // EventSorter accepts unsorted PolymorphicEvents, sort them in background and returns diff --git a/cdc/sorter/unified/backend.go b/cdc/sorter/unified/backend.go index 576747c2083..6f546bfcfab 100644 --- a/cdc/sorter/unified/backend.go +++ b/cdc/sorter/unified/backend.go @@ -13,7 +13,7 @@ package unified -import "github.com/pingcap/ticdc/cdc/model" +import "github.com/pingcap/tiflow/cdc/model" type backEnd interface { reader() (backEndReader, error) diff --git a/cdc/sorter/unified/backend_pool.go b/cdc/sorter/unified/backend_pool.go index 60375a12600..72b015d187a 100644 --- a/cdc/sorter/unified/backend_pool.go +++ b/cdc/sorter/unified/backend_pool.go @@ -27,12 +27,12 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - sorterencoding "github.com/pingcap/ticdc/cdc/sorter/encoding" - "github.com/pingcap/ticdc/pkg/config" - cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/filelock" - "github.com/pingcap/ticdc/pkg/util" "github.com/pingcap/tidb/util/memory" + sorterencoding "github.com/pingcap/tiflow/cdc/sorter/encoding" + "github.com/pingcap/tiflow/pkg/config" + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/filelock" + "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" ) diff --git a/cdc/sorter/unified/backend_pool_test.go b/cdc/sorter/unified/backend_pool_test.go index e616a23f4a2..0db78c71670 100644 --- a/cdc/sorter/unified/backend_pool_test.go +++ b/cdc/sorter/unified/backend_pool_test.go @@ -23,10 +23,10 @@ import ( "github.com/pingcap/check" "github.com/pingcap/failpoint" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/filelock" - "github.com/pingcap/ticdc/pkg/util/testleak" "github.com/pingcap/tidb/util/memory" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/filelock" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type backendPoolSuite struct{} @@ -51,7 +51,7 @@ func (s *backendPoolSuite) TestBasicFunction(c *check.C) { conf.Sorter.MaxMemoryConsumption = 16 * 1024 * 1024 * 1024 // 16G config.StoreGlobalServerConfig(conf) - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/memoryPressureInjectPoint", "return(100)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/memoryPressureInjectPoint", "return(100)") c.Assert(err, check.IsNil) ctx, cancel := context.WithTimeout(context.Background(), time.Second*20) @@ -68,9 +68,9 @@ func (s *backendPoolSuite) TestBasicFunction(c *check.C) { fileName := backEnd.(*fileBackEnd).fileName c.Assert(fileName, check.Not(check.Equals), "") - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/memoryPressureInjectPoint", "return(0)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/memoryPressureInjectPoint", "return(0)") c.Assert(err, check.IsNil) - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/memoryUsageInjectPoint", "return(34359738368)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/memoryUsageInjectPoint", "return(34359738368)") c.Assert(err, check.IsNil) backEnd1, err := backEndPool.alloc(ctx) @@ -80,9 +80,9 @@ func (s *backendPoolSuite) TestBasicFunction(c *check.C) { c.Assert(fileName1, check.Not(check.Equals), "") c.Assert(fileName1, check.Not(check.Equals), fileName) - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/memoryPressureInjectPoint", "return(0)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/memoryPressureInjectPoint", "return(0)") c.Assert(err, check.IsNil) - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/memoryUsageInjectPoint", "return(0)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/memoryUsageInjectPoint", "return(0)") c.Assert(err, check.IsNil) backEnd2, err := backEndPool.alloc(ctx) @@ -161,9 +161,9 @@ func (s *backendPoolSuite) TestCleanUpSelf(c *check.C) { conf.Sorter.MaxMemoryConsumption = 16 * 1024 * 1024 * 1024 // 16G config.StoreGlobalServerConfig(conf) - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/memoryPressureInjectPoint", "return(100)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/memoryPressureInjectPoint", "return(100)") c.Assert(err, check.IsNil) - defer failpoint.Disable("github.com/pingcap/ticdc/cdc/sorter/unified/memoryPressureInjectPoint") //nolint:errcheck + defer failpoint.Disable("github.com/pingcap/tiflow/cdc/sorter/unified/memoryPressureInjectPoint") //nolint:errcheck backEndPool, err := newBackEndPool(sorterDir, "") c.Assert(err, check.IsNil) diff --git a/cdc/sorter/unified/file_backend.go b/cdc/sorter/unified/file_backend.go index 24ed9959163..67939cbeb10 100644 --- a/cdc/sorter/unified/file_backend.go +++ b/cdc/sorter/unified/file_backend.go @@ -23,9 +23,9 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/sorter/encoding" - cerrors "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sorter/encoding" + cerrors "github.com/pingcap/tiflow/pkg/errors" "go.uber.org/zap" ) diff --git a/cdc/sorter/unified/file_backend_test.go b/cdc/sorter/unified/file_backend_test.go index 3ec9fce76da..b755dbe8871 100644 --- a/cdc/sorter/unified/file_backend_test.go +++ b/cdc/sorter/unified/file_backend_test.go @@ -18,10 +18,10 @@ import ( "os" "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/sorter/encoding" - cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sorter/encoding" + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type fileBackendSuite struct{} diff --git a/cdc/sorter/unified/heap.go b/cdc/sorter/unified/heap.go index 6d75fab55fc..a8e1baec64f 100644 --- a/cdc/sorter/unified/heap.go +++ b/cdc/sorter/unified/heap.go @@ -13,7 +13,7 @@ package unified -import "github.com/pingcap/ticdc/cdc/model" +import "github.com/pingcap/tiflow/cdc/model" type sortItem struct { entry *model.PolymorphicEvent diff --git a/cdc/sorter/unified/heap_sorter.go b/cdc/sorter/unified/heap_sorter.go index a03f88d78ce..5fe35a6c5ab 100644 --- a/cdc/sorter/unified/heap_sorter.go +++ b/cdc/sorter/unified/heap_sorter.go @@ -23,11 +23,11 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" - cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/util" - "github.com/pingcap/ticdc/pkg/workerpool" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/util" + "github.com/pingcap/tiflow/pkg/workerpool" "go.uber.org/zap" ) diff --git a/cdc/sorter/unified/memory_backend.go b/cdc/sorter/unified/memory_backend.go index 27ca7f511b6..67d1ce7f3e6 100644 --- a/cdc/sorter/unified/memory_backend.go +++ b/cdc/sorter/unified/memory_backend.go @@ -18,7 +18,7 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/tiflow/cdc/model" "go.uber.org/zap" ) diff --git a/cdc/sorter/unified/memory_backend_test.go b/cdc/sorter/unified/memory_backend_test.go index d95346fa6d5..e44117665aa 100644 --- a/cdc/sorter/unified/memory_backend_test.go +++ b/cdc/sorter/unified/memory_backend_test.go @@ -19,8 +19,8 @@ import ( "time" "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type memoryBackendSuite struct{} diff --git a/cdc/sorter/unified/merger.go b/cdc/sorter/unified/merger.go index 97ff7baa9ac..8be03a4db6a 100644 --- a/cdc/sorter/unified/merger.go +++ b/cdc/sorter/unified/merger.go @@ -25,10 +25,10 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/notify" - "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/tiflow/cdc/model" + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/notify" + "github.com/pingcap/tiflow/pkg/util" "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" "golang.org/x/sync/errgroup" diff --git a/cdc/sorter/unified/merger_test.go b/cdc/sorter/unified/merger_test.go index fa8813aeaec..095787189dc 100644 --- a/cdc/sorter/unified/merger_test.go +++ b/cdc/sorter/unified/merger_test.go @@ -21,8 +21,8 @@ import ( "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/util/testleak" "go.uber.org/zap" "go.uber.org/zap/zapcore" "golang.org/x/sync/errgroup" @@ -93,7 +93,7 @@ func (b *mockFlushTaskBuilder) build() *flushTask { // It tests the most basic scenario. func (s *sorterSuite) TestMergerSingleHeap(c *check.C) { defer testleak.AfterTest(c)() - err := failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/sorterDebug", "return(true)") + err := failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterDebug", "return(true)") if err != nil { log.Panic("Could not enable failpoint", zap.Error(err)) } @@ -164,7 +164,7 @@ func (s *sorterSuite) TestMergerSingleHeap(c *check.C) { // TestMergerSingleHeapRetire simulates a situation where the resolved event is not the last event in a flushTask func (s *sorterSuite) TestMergerSingleHeapRetire(c *check.C) { defer testleak.AfterTest(c)() - err := failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/sorterDebug", "return(true)") + err := failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterDebug", "return(true)") if err != nil { log.Panic("Could not enable failpoint", zap.Error(err)) } @@ -237,14 +237,14 @@ func (s *sorterSuite) TestMergerSingleHeapRetire(c *check.C) { // Expects intermediate resolved events to be generated, so that the sink would not get stuck in a real life situation. func (s *sorterSuite) TestMergerSortDelay(c *check.C) { defer testleak.AfterTest(c)() - err := failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/sorterDebug", "return(true)") + err := failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterDebug", "return(true)") c.Assert(err, check.IsNil) // enable the failpoint to simulate delays - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/sorterMergeDelay", "sleep(5)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterMergeDelay", "sleep(5)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/sorter/unified/sorterMergeDelay") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterMergeDelay") }() log.SetLevel(zapcore.DebugLevel) @@ -317,14 +317,14 @@ func (s *sorterSuite) TestMergerSortDelay(c *check.C) { // Expects proper clean-up of the data. func (s *sorterSuite) TestMergerCancel(c *check.C) { defer testleak.AfterTest(c)() - err := failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/sorterDebug", "return(true)") + err := failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterDebug", "return(true)") c.Assert(err, check.IsNil) // enable the failpoint to simulate delays - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/sorterMergeDelay", "sleep(10)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterMergeDelay", "sleep(10)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/sorter/unified/sorterMergeDelay") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterMergeDelay") }() log.SetLevel(zapcore.DebugLevel) @@ -380,7 +380,7 @@ func (s *sorterSuite) TestMergerCancel(c *check.C) { // Expects proper clean-up of the data. func (s *sorterSuite) TestMergerCancelWithUnfinishedFlushTasks(c *check.C) { defer testleak.AfterTest(c)() - err := failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/sorterDebug", "return(true)") + err := failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterDebug", "return(true)") c.Assert(err, check.IsNil) log.SetLevel(zapcore.DebugLevel) @@ -436,7 +436,7 @@ func (s *sorterSuite) TestMergerCancelWithUnfinishedFlushTasks(c *check.C) { // There is expected to be NO fatal error. func (s *sorterSuite) TestMergerCloseChannel(c *check.C) { defer testleak.AfterTest(c)() - err := failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/sorterDebug", "return(true)") + err := failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterDebug", "return(true)") c.Assert(err, check.IsNil) log.SetLevel(zapcore.DebugLevel) @@ -482,9 +482,9 @@ func (s *sorterSuite) TestMergerCloseChannel(c *check.C) { // a significant period of time. func (s *sorterSuite) TestMergerOutputBlocked(c *check.C) { defer testleak.AfterTest(c)() - err := failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/sorterDebug", "return(true)") + err := failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterDebug", "return(true)") c.Assert(err, check.IsNil) - defer failpoint.Disable("github.com/pingcap/ticdc/cdc/sorter/unified/sorterDebug") //nolint:errcheck + defer failpoint.Disable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterDebug") //nolint:errcheck ctx, cancel := context.WithTimeout(context.TODO(), time.Second*25) defer cancel() diff --git a/cdc/sorter/unified/sorter_test.go b/cdc/sorter/unified/sorter_test.go index 3ef281dced9..af395b41d13 100644 --- a/cdc/sorter/unified/sorter_test.go +++ b/cdc/sorter/unified/sorter_test.go @@ -26,10 +26,10 @@ import ( "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/sorter" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sorter" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/util/testleak" "go.uber.org/zap" "go.uber.org/zap/zapcore" "golang.org/x/sync/errgroup" @@ -128,14 +128,14 @@ func (s *sorterSuite) TestSorterCancel(c *check.C) { } func testSorter(ctx context.Context, c *check.C, sorter sorter.EventSorter, count int) error { - err := failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/sorterDebug", "return(true)") + err := failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterDebug", "return(true)") if err != nil { log.Panic("Could not enable failpoint", zap.Error(err)) } - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/pkg/util/InjectCheckDataDirSatisfied", ""), check.IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/pkg/util/InjectCheckDataDirSatisfied", ""), check.IsNil) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/ticdc/pkg/util/InjectCheckDataDirSatisfied"), check.IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tiflow/pkg/util/InjectCheckDataDirSatisfied"), check.IsNil) }() ctx, cancel := context.WithCancel(ctx) @@ -309,17 +309,17 @@ func (s *sorterSuite) TestSorterCancelRestart(c *check.C) { c.Assert(err, check.IsNil) // enable the failpoint to simulate delays - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/asyncFlushStartDelay", "sleep(100)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/asyncFlushStartDelay", "sleep(100)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/sorter/unified/asyncFlushStartDelay") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sorter/unified/asyncFlushStartDelay") }() // enable the failpoint to simulate delays - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/asyncFlushInProcessDelay", "1%sleep(1)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/asyncFlushInProcessDelay", "1%sleep(1)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/sorter/unified/asyncFlushInProcessDelay") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sorter/unified/asyncFlushInProcessDelay") }() for i := 0; i < 5; i++ { @@ -358,10 +358,10 @@ func (s *sorterSuite) TestSorterIOError(c *check.C) { defer cancel() // enable the failpoint to simulate backEnd allocation error (usually would happen when creating a file) - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/InjectErrorBackEndAlloc", "return(true)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/InjectErrorBackEndAlloc", "return(true)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/sorter/unified/InjectErrorBackEndAlloc") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sorter/unified/InjectErrorBackEndAlloc") }() finishedCh := make(chan struct{}) @@ -379,12 +379,12 @@ func (s *sorterSuite) TestSorterIOError(c *check.C) { } UnifiedSorterCleanUp() - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/sorter/unified/InjectErrorBackEndAlloc") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sorter/unified/InjectErrorBackEndAlloc") // enable the failpoint to simulate backEnd write error (usually would happen when writing to a file) - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/InjectErrorBackEndWrite", "return(true)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/InjectErrorBackEndWrite", "return(true)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/sorter/unified/InjectErrorBackEndWrite") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sorter/unified/InjectErrorBackEndWrite") }() // recreate the sorter @@ -435,16 +435,16 @@ func (s *sorterSuite) TestSorterErrorReportCorrect(c *check.C) { defer cancel() // enable the failpoint to simulate backEnd allocation error (usually would happen when creating a file) - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/InjectHeapSorterExitDelay", "sleep(2000)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/InjectHeapSorterExitDelay", "sleep(2000)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/sorter/unified/InjectHeapSorterExitDelay") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sorter/unified/InjectHeapSorterExitDelay") }() - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/InjectErrorBackEndAlloc", "return(true)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/InjectErrorBackEndAlloc", "return(true)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/sorter/unified/InjectErrorBackEndAlloc") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sorter/unified/InjectErrorBackEndAlloc") }() finishedCh := make(chan struct{}) diff --git a/cdc/sorter/unified/unified_sorter.go b/cdc/sorter/unified/unified_sorter.go index 200d7577cec..1cd502f7186 100644 --- a/cdc/sorter/unified/unified_sorter.go +++ b/cdc/sorter/unified/unified_sorter.go @@ -21,10 +21,10 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/util" "golang.org/x/sync/errgroup" ) diff --git a/chaos/Dockerfile.chaos b/chaos/Dockerfile.chaos index d1983f125f7..e8e22ad76a8 100644 --- a/chaos/Dockerfile.chaos +++ b/chaos/Dockerfile.chaos @@ -1,10 +1,10 @@ FROM golang:1.16-alpine as builder RUN apk add --no-cache git make bash -WORKDIR /go/src/github.com/pingcap/ticdc +WORKDIR /go/src/github.com/pingcap/tiflow COPY . . RUN make bank FROM alpine:3.12 RUN apk add --no-cache tzdata bash curl socat -COPY --from=builder /go/src/github.com/pingcap/ticdc/bin/bank /bank +COPY --from=builder /go/src/github.com/pingcap/tiflow/bin/bank /bank CMD [ "/bank" ] diff --git a/cmd/cdc/main.go b/cmd/cdc/main.go index 16d7a0e685d..9097cb6b245 100644 --- a/cmd/cdc/main.go +++ b/cmd/cdc/main.go @@ -14,8 +14,8 @@ package main import ( - "github.com/pingcap/ticdc/pkg/cmd" _ "github.com/pingcap/tidb/types/parser_driver" + "github.com/pingcap/tiflow/pkg/cmd" ) func main() { diff --git a/cmd/kafka-consumer/main.go b/cmd/kafka-consumer/main.go index 566063d6406..6d1d3ff324e 100644 --- a/cmd/kafka-consumer/main.go +++ b/cmd/kafka-consumer/main.go @@ -32,15 +32,15 @@ import ( "github.com/google/uuid" "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/sink" - "github.com/pingcap/ticdc/cdc/sink/codec" - "github.com/pingcap/ticdc/pkg/config" - cdcfilter "github.com/pingcap/ticdc/pkg/filter" - "github.com/pingcap/ticdc/pkg/logutil" - "github.com/pingcap/ticdc/pkg/quotes" - "github.com/pingcap/ticdc/pkg/security" - "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sink" + "github.com/pingcap/tiflow/cdc/sink/codec" + "github.com/pingcap/tiflow/pkg/config" + cdcfilter "github.com/pingcap/tiflow/pkg/filter" + "github.com/pingcap/tiflow/pkg/logutil" + "github.com/pingcap/tiflow/pkg/quotes" + "github.com/pingcap/tiflow/pkg/security" + "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" ) diff --git a/demo/java/README.md b/demo/java/README.md index cd545586d8f..db3761851c2 100644 --- a/demo/java/README.md +++ b/demo/java/README.md @@ -26,7 +26,7 @@ for (KafkaMessage kafkaMessage : kafkaMessages) { } ``` -[See com.pingcap.ticdc.cdc.TicdcEventDecoderTest.](src/test/java/com/pingcap/ticdc/cdc/TicdcEventDecoderTest.java). +[See com.pingcap.ticdc.cdc.TicdcEventDecoderTest.](src/test/java/com/pingcap/tiflow/cdc/TicdcEventDecoderTest.java). # How to install Prerequisites for building: @@ -36,7 +36,7 @@ Prerequisites for building: * Java 8 ``` -git clone git@github.com:pingcap/ticdc.git +git clone git@github.com:pingcap/tiflow.git cd ticdc/demo/java mvn install ``` diff --git a/dm/_utils/terror_gen/checker_template.go b/dm/_utils/terror_gen/checker_template.go index 84f00325bc8..668112c87cb 100644 --- a/dm/_utils/terror_gen/checker_template.go +++ b/dm/_utils/terror_gen/checker_template.go @@ -25,7 +25,7 @@ import ( "github.com/BurntSushi/toml" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) const ( diff --git a/dm/chaos/cases/cases.go b/dm/chaos/cases/cases.go index 7a07890f4c9..ea15972deae 100644 --- a/dm/chaos/cases/cases.go +++ b/dm/chaos/cases/cases.go @@ -19,9 +19,9 @@ import ( "golang.org/x/sync/errgroup" - config2 "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/utils" + config2 "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/utils" ) var ( diff --git a/dm/chaos/cases/config.go b/dm/chaos/cases/config.go index d7c19a3a4cf..fde1216e088 100644 --- a/dm/chaos/cases/config.go +++ b/dm/chaos/cases/config.go @@ -17,7 +17,7 @@ import ( "flag" "time" - config2 "github.com/pingcap/ticdc/dm/dm/config" + config2 "github.com/pingcap/tiflow/dm/dm/config" ) // config is used to run chaos tests. diff --git a/dm/chaos/cases/db.go b/dm/chaos/cases/db.go index e0cef8939b9..dcd6ee82547 100644 --- a/dm/chaos/cases/db.go +++ b/dm/chaos/cases/db.go @@ -25,11 +25,11 @@ import ( "github.com/pingcap/tidb/errno" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/conn" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/retry" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/retry" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // dbConn holds a connection to a database and supports to reset the connection. diff --git a/dm/chaos/cases/diff.go b/dm/chaos/cases/diff.go index 218cc6dc7ef..e6b9d883bd2 100644 --- a/dm/chaos/cases/diff.go +++ b/dm/chaos/cases/diff.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/tidb-tools/pkg/diff" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/log" ) // diffDataLoop checks whether target has the same data with source via `sync-diff-inspector` multiple times. diff --git a/dm/chaos/cases/generator.go b/dm/chaos/cases/generator.go index a500bd8ccf0..65afba47cf7 100644 --- a/dm/chaos/cases/generator.go +++ b/dm/chaos/cases/generator.go @@ -18,7 +18,7 @@ import ( "fmt" "math/rand" - config2 "github.com/pingcap/ticdc/dm/dm/config" + config2 "github.com/pingcap/tiflow/dm/dm/config" ) const ( diff --git a/dm/chaos/cases/instance.go b/dm/chaos/cases/instance.go index 12142190aec..83fd99fea54 100644 --- a/dm/chaos/cases/instance.go +++ b/dm/chaos/cases/instance.go @@ -16,8 +16,8 @@ package main import ( "context" - config2 "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/conn" + config2 "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/conn" ) // set lesser sql_mode to tolerate some SQLs generated by go-sqlsmith. diff --git a/dm/chaos/cases/main.go b/dm/chaos/cases/main.go index 92d23cde904..442943f742f 100644 --- a/dm/chaos/cases/main.go +++ b/dm/chaos/cases/main.go @@ -28,8 +28,8 @@ import ( "go.uber.org/zap" "google.golang.org/grpc" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/log" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/log" ) // main starts to run the test case logic after MySQL, TiDB and DM have been set up. diff --git a/dm/chaos/cases/member.go b/dm/chaos/cases/member.go index 1bf97b70cda..04b8c268bed 100644 --- a/dm/chaos/cases/member.go +++ b/dm/chaos/cases/member.go @@ -18,8 +18,8 @@ import ( "fmt" "time" - "github.com/pingcap/ticdc/dm/dm/master/scheduler" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/master/scheduler" + "github.com/pingcap/tiflow/dm/dm/pb" ) const ( diff --git a/dm/chaos/cases/source.go b/dm/chaos/cases/source.go index e149ad9b4a8..c5c1b53179a 100644 --- a/dm/chaos/cases/source.go +++ b/dm/chaos/cases/source.go @@ -21,8 +21,8 @@ import ( "strings" "time" - config2 "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" + config2 "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" ) // createSources does `operate-source create` operation for two sources. diff --git a/dm/chaos/cases/task.go b/dm/chaos/cases/task.go index ab69874fa65..ef00a021c54 100644 --- a/dm/chaos/cases/task.go +++ b/dm/chaos/cases/task.go @@ -28,11 +28,11 @@ import ( "go.uber.org/zap" "golang.org/x/sync/errgroup" - config2 "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/conn" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/utils" + config2 "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/utils" ) const ( diff --git a/dm/checker/check_test.go b/dm/checker/check_test.go index 44f27f6777f..527846b8281 100644 --- a/dm/checker/check_test.go +++ b/dm/checker/check_test.go @@ -21,9 +21,9 @@ import ( "github.com/DATA-DOG/go-sqlmock" router "github.com/pingcap/tidb-tools/pkg/table-router" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/pkg/conn" tc "github.com/pingcap/check" ) diff --git a/dm/checker/checker.go b/dm/checker/checker.go index 03242f767c8..6ffb82472d3 100644 --- a/dm/checker/checker.go +++ b/dm/checker/checker.go @@ -24,15 +24,15 @@ import ( "sync" "time" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/dm/unit" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/conn" - fr "github.com/pingcap/ticdc/dm/pkg/func-rollback" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/unit" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/conn" + fr "github.com/pingcap/tiflow/dm/pkg/func-rollback" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" _ "github.com/go-sql-driver/mysql" // for mysql "github.com/pingcap/tidb-tools/pkg/check" diff --git a/dm/checker/cmd.go b/dm/checker/cmd.go index 0a95a605cb1..66064747944 100644 --- a/dm/checker/cmd.go +++ b/dm/checker/cmd.go @@ -16,9 +16,9 @@ package checker import ( "context" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/terror" ) var ( diff --git a/dm/cmd/dm-ctl/main.go b/dm/cmd/dm-ctl/main.go index d73487f7a98..65a503ecee3 100644 --- a/dm/cmd/dm-ctl/main.go +++ b/dm/cmd/dm-ctl/main.go @@ -19,11 +19,11 @@ import ( "os/signal" "syscall" - "github.com/pingcap/ticdc/dm/dm/ctl" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/ctl" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) func main() { diff --git a/dm/cmd/dm-master/main.go b/dm/cmd/dm-master/main.go index ea30b86e6f2..890492ef3b1 100644 --- a/dm/cmd/dm-master/main.go +++ b/dm/cmd/dm-master/main.go @@ -25,11 +25,11 @@ import ( "github.com/pingcap/errors" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/master" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/master" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) func main() { diff --git a/dm/cmd/dm-master/main_test.go b/dm/cmd/dm-master/main_test.go index 77e439ef0de..bfba20b094d 100644 --- a/dm/cmd/dm-master/main_test.go +++ b/dm/cmd/dm-master/main_test.go @@ -23,7 +23,7 @@ import ( "testing" "time" - _ "github.com/pingcap/ticdc/dm/pkg/printinit" + _ "github.com/pingcap/tiflow/dm/pkg/printinit" ) func TestRunMain(_ *testing.T) { diff --git a/dm/cmd/dm-portal/main.go b/dm/cmd/dm-portal/main.go index 1d6295cae4f..6bece1baf02 100644 --- a/dm/cmd/dm-portal/main.go +++ b/dm/cmd/dm-portal/main.go @@ -21,9 +21,9 @@ import ( "github.com/rakyll/statik/fs" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/portal" - _ "github.com/pingcap/ticdc/dm/dm/portal/statik" - "github.com/pingcap/ticdc/dm/pkg/log" + "github.com/pingcap/tiflow/dm/dm/portal" + _ "github.com/pingcap/tiflow/dm/dm/portal/statik" + "github.com/pingcap/tiflow/dm/pkg/log" ) func main() { diff --git a/dm/cmd/dm-syncer/config.go b/dm/cmd/dm-syncer/config.go index 14055016432..c1d2173974e 100644 --- a/dm/cmd/dm-syncer/config.go +++ b/dm/cmd/dm-syncer/config.go @@ -25,9 +25,9 @@ import ( "github.com/pingcap/tidb-tools/pkg/filter" router "github.com/pingcap/tidb-tools/pkg/table-router" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // commonConfig collects common item for both new config and old config. diff --git a/dm/cmd/dm-syncer/main.go b/dm/cmd/dm-syncer/main.go index 86c27d51387..0f89de19696 100644 --- a/dm/cmd/dm-syncer/main.go +++ b/dm/cmd/dm-syncer/main.go @@ -22,13 +22,13 @@ import ( "strings" "syscall" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" - "github.com/pingcap/ticdc/dm/syncer" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/syncer" "github.com/pingcap/errors" globalLog "github.com/pingcap/log" diff --git a/dm/cmd/dm-worker/main.go b/dm/cmd/dm-worker/main.go index 410f14553bf..66e022a5ef7 100644 --- a/dm/cmd/dm-worker/main.go +++ b/dm/cmd/dm-worker/main.go @@ -27,11 +27,11 @@ import ( lightningLog "github.com/pingcap/tidb/br/pkg/lightning/log" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/worker" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/worker" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) func main() { diff --git a/dm/cmd/dm-worker/main_test.go b/dm/cmd/dm-worker/main_test.go index 76d7be520c2..81aa1f031a4 100644 --- a/dm/cmd/dm-worker/main_test.go +++ b/dm/cmd/dm-worker/main_test.go @@ -22,8 +22,8 @@ import ( "testing" "time" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/utils" "go.uber.org/zap" ) diff --git a/dm/debug-tools/binlog-event-blackhole/fetcher.go b/dm/debug-tools/binlog-event-blackhole/fetcher.go index b5028b516a1..081e0a7e313 100644 --- a/dm/debug-tools/binlog-event-blackhole/fetcher.go +++ b/dm/debug-tools/binlog-event-blackhole/fetcher.go @@ -22,7 +22,7 @@ import ( "go.uber.org/atomic" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/log" ) // registerSlave register a slave connection on the master. diff --git a/dm/debug-tools/binlog-event-blackhole/main.go b/dm/debug-tools/binlog-event-blackhole/main.go index 0f8d9b8bb97..13bdf299aae 100644 --- a/dm/debug-tools/binlog-event-blackhole/main.go +++ b/dm/debug-tools/binlog-event-blackhole/main.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/errors" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/log" ) func main() { diff --git a/dm/dm/common/common.go b/dm/dm/common/common.go index f55d5aee8e8..006797a6f1b 100644 --- a/dm/dm/common/common.go +++ b/dm/dm/common/common.go @@ -19,7 +19,7 @@ import ( "path" "strings" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) var ( diff --git a/dm/dm/config/checking_item.go b/dm/dm/config/checking_item.go index 9780b34362c..8723753eed9 100644 --- a/dm/dm/config/checking_item.go +++ b/dm/dm/config/checking_item.go @@ -17,7 +17,7 @@ import ( "bytes" "fmt" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // DM definition checking items diff --git a/dm/dm/config/source_config.go b/dm/dm/config/source_config.go index b200577cece..448ea3c3898 100644 --- a/dm/dm/config/source_config.go +++ b/dm/dm/config/source_config.go @@ -32,11 +32,11 @@ import ( bf "github.com/pingcap/tidb-tools/pkg/binlog-filter" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) const ( diff --git a/dm/dm/config/source_config_test.go b/dm/dm/config/source_config_test.go index 9f6bfd3f5b9..0ef3d901ebc 100644 --- a/dm/dm/config/source_config_test.go +++ b/dm/dm/config/source_config_test.go @@ -28,7 +28,7 @@ import ( . "github.com/pingcap/check" bf "github.com/pingcap/tidb-tools/pkg/binlog-filter" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // do not forget to update this path if the file removed/renamed. diff --git a/dm/dm/config/subtask.go b/dm/dm/config/subtask.go index bdc66bc5fa2..9a59285cb1a 100644 --- a/dm/dm/config/subtask.go +++ b/dm/dm/config/subtask.go @@ -29,10 +29,10 @@ import ( lcfg "github.com/pingcap/tidb/br/pkg/lightning/config" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/dumpling" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/dumpling" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // task modes. diff --git a/dm/dm/config/task.go b/dm/dm/config/task.go index 48dd784b746..091f93722c3 100644 --- a/dm/dm/config/task.go +++ b/dm/dm/config/task.go @@ -31,9 +31,9 @@ import ( "go.uber.org/zap" "gopkg.in/yaml.v2" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // Online DDL Scheme. diff --git a/dm/dm/config/task_converters.go b/dm/dm/config/task_converters.go index 92d5a91eb1c..dba66c811cd 100644 --- a/dm/dm/config/task_converters.go +++ b/dm/dm/config/task_converters.go @@ -22,9 +22,9 @@ import ( "github.com/pingcap/tidb-tools/pkg/filter" router "github.com/pingcap/tidb-tools/pkg/table-router" - "github.com/pingcap/ticdc/dm/openapi" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/openapi" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // TaskConfigToSubTaskConfigs generates sub task configs by TaskConfig. diff --git a/dm/dm/config/task_converters_test.go b/dm/dm/config/task_converters_test.go index 149c5d727e3..1b6eac3527d 100644 --- a/dm/dm/config/task_converters_test.go +++ b/dm/dm/config/task_converters_test.go @@ -19,8 +19,8 @@ import ( bf "github.com/pingcap/tidb-tools/pkg/binlog-filter" "github.com/pingcap/tidb-tools/pkg/filter" - "github.com/pingcap/ticdc/dm/openapi" - "github.com/pingcap/ticdc/dm/openapi/fixtures" + "github.com/pingcap/tiflow/dm/openapi" + "github.com/pingcap/tiflow/dm/openapi/fixtures" ) func (t *testConfig) TestTaskGetTargetDBCfg(c *check.C) { diff --git a/dm/dm/config/task_test.go b/dm/dm/config/task_test.go index 29b8e0b320d..8ace25f0278 100644 --- a/dm/dm/config/task_test.go +++ b/dm/dm/config/task_test.go @@ -25,7 +25,7 @@ import ( "github.com/pingcap/tidb-tools/pkg/filter" router "github.com/pingcap/tidb-tools/pkg/table-router" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" "github.com/coreos/go-semver/semver" ) diff --git a/dm/dm/ctl/common/config.go b/dm/dm/ctl/common/config.go index 1a4bcd78614..50c49044d1a 100644 --- a/dm/dm/ctl/common/config.go +++ b/dm/dm/ctl/common/config.go @@ -21,8 +21,8 @@ import ( "strings" "time" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/utils" "github.com/BurntSushi/toml" "github.com/pingcap/errors" diff --git a/dm/dm/ctl/common/operate_relay.go b/dm/dm/ctl/common/operate_relay.go index 5a0d88b4566..8e7c24c154b 100644 --- a/dm/dm/ctl/common/operate_relay.go +++ b/dm/dm/ctl/common/operate_relay.go @@ -16,7 +16,7 @@ package common import ( "context" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/pb" ) // OperateRelay does operation on relay unit. diff --git a/dm/dm/ctl/common/operate_task.go b/dm/dm/ctl/common/operate_task.go index 449cfcd65b1..dbfc4b29665 100644 --- a/dm/dm/ctl/common/operate_task.go +++ b/dm/dm/ctl/common/operate_task.go @@ -16,7 +16,7 @@ package common import ( "context" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/pb" ) // OperateTask does operation on task. diff --git a/dm/dm/ctl/common/util.go b/dm/dm/ctl/common/util.go index 03280895792..e6d9be81a55 100644 --- a/dm/dm/ctl/common/util.go +++ b/dm/dm/ctl/common/util.go @@ -38,12 +38,12 @@ import ( "google.golang.org/grpc/codes" "google.golang.org/grpc/status" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/log" - parserpkg "github.com/pingcap/ticdc/dm/pkg/parser" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/log" + parserpkg "github.com/pingcap/tiflow/dm/pkg/parser" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) var ( diff --git a/dm/dm/ctl/ctl.go b/dm/dm/ctl/ctl.go index 1abf5a22d22..0dcc2f7da74 100644 --- a/dm/dm/ctl/ctl.go +++ b/dm/dm/ctl/ctl.go @@ -19,10 +19,10 @@ import ( "os" "strings" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/ctl/master" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/ctl/master" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/utils" "github.com/chzyer/readline" "github.com/pingcap/errors" diff --git a/dm/dm/ctl/master/binlog.go b/dm/dm/ctl/master/binlog.go index f76f3d2f363..c02dc8d0969 100644 --- a/dm/dm/ctl/master/binlog.go +++ b/dm/dm/ctl/master/binlog.go @@ -14,8 +14,8 @@ package master import ( - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/pb" "github.com/pingcap/errors" "github.com/spf13/cobra" diff --git a/dm/dm/ctl/master/check_task.go b/dm/dm/ctl/master/check_task.go index 75937c771b1..5cd0b5c2ae9 100644 --- a/dm/dm/ctl/master/check_task.go +++ b/dm/dm/ctl/master/check_task.go @@ -20,9 +20,9 @@ import ( "github.com/spf13/cobra" - "github.com/pingcap/ticdc/dm/checker" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/checker" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/pb" ) // NewCheckTaskCmd creates a CheckTask command. diff --git a/dm/dm/ctl/master/config.go b/dm/dm/ctl/master/config.go index 87691eabb3f..c80506662fe 100644 --- a/dm/dm/ctl/master/config.go +++ b/dm/dm/ctl/master/config.go @@ -25,11 +25,11 @@ import ( "github.com/spf13/cobra" "go.etcd.io/etcd/clientv3" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/ha" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/ha" + "github.com/pingcap/tiflow/dm/pkg/utils" ) var ( diff --git a/dm/dm/ctl/master/get_config.go b/dm/dm/ctl/master/get_config.go index 3301ecf4d95..bed923632a1 100644 --- a/dm/dm/ctl/master/get_config.go +++ b/dm/dm/ctl/master/get_config.go @@ -21,8 +21,8 @@ import ( "github.com/spf13/cobra" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/pb" ) const cmdGetTaskConfig = "get-task-config" diff --git a/dm/dm/ctl/master/handle_error.go b/dm/dm/ctl/master/handle_error.go index 68c752f1468..b9a296584fb 100644 --- a/dm/dm/ctl/master/handle_error.go +++ b/dm/dm/ctl/master/handle_error.go @@ -20,9 +20,9 @@ import ( "github.com/spf13/cobra" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/binlog" ) // NewHandleErrorCmd creates a HandleError command. diff --git a/dm/dm/ctl/master/list_member.go b/dm/dm/ctl/master/list_member.go index 6cb18ec4d75..ffec67d7183 100644 --- a/dm/dm/ctl/master/list_member.go +++ b/dm/dm/ctl/master/list_member.go @@ -20,8 +20,8 @@ import ( "github.com/spf13/cobra" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/pb" ) var listMemberFlags = ListMemberFlags{} diff --git a/dm/dm/ctl/master/offline_member.go b/dm/dm/ctl/master/offline_member.go index 334a8410b8b..972e002f597 100644 --- a/dm/dm/ctl/master/offline_member.go +++ b/dm/dm/ctl/master/offline_member.go @@ -17,8 +17,8 @@ import ( "context" "os" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/pb" "github.com/pingcap/errors" "github.com/spf13/cobra" diff --git a/dm/dm/ctl/master/operate_leader.go b/dm/dm/ctl/master/operate_leader.go index 628322870f2..77c9c7a2591 100644 --- a/dm/dm/ctl/master/operate_leader.go +++ b/dm/dm/ctl/master/operate_leader.go @@ -20,8 +20,8 @@ import ( "github.com/spf13/cobra" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/pb" ) // NewOperateLeaderCmd creates a OperateLeader command. diff --git a/dm/dm/ctl/master/operate_schema.go b/dm/dm/ctl/master/operate_schema.go index cf85a1ff109..b57b7c035dc 100644 --- a/dm/dm/ctl/master/operate_schema.go +++ b/dm/dm/ctl/master/operate_schema.go @@ -20,8 +20,8 @@ import ( "github.com/spf13/cobra" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/pb" ) // NewOperateSchemaCmd creates a OperateSchema command. diff --git a/dm/dm/ctl/master/operate_source.go b/dm/dm/ctl/master/operate_source.go index 27b24f9937b..c50e405217d 100644 --- a/dm/dm/ctl/master/operate_source.go +++ b/dm/dm/ctl/master/operate_source.go @@ -22,11 +22,11 @@ import ( "github.com/spf13/cobra" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // NewOperateSourceCmd creates a OperateSource command. diff --git a/dm/dm/ctl/master/operate_task.go b/dm/dm/ctl/master/operate_task.go index 784fabd8c73..fa460e50f55 100644 --- a/dm/dm/ctl/master/operate_task.go +++ b/dm/dm/ctl/master/operate_task.go @@ -23,8 +23,8 @@ import ( "github.com/spf13/cobra" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/pb" ) const ( diff --git a/dm/dm/ctl/master/pause_relay.go b/dm/dm/ctl/master/pause_relay.go index afcaa8c2c23..8ee9cd5fb98 100644 --- a/dm/dm/ctl/master/pause_relay.go +++ b/dm/dm/ctl/master/pause_relay.go @@ -20,8 +20,8 @@ import ( "github.com/spf13/cobra" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/pb" ) // NewPauseRelayCmd creates a PauseRelay command. diff --git a/dm/dm/ctl/master/pause_task.go b/dm/dm/ctl/master/pause_task.go index 2ec08c58066..4961a7c57d9 100644 --- a/dm/dm/ctl/master/pause_task.go +++ b/dm/dm/ctl/master/pause_task.go @@ -16,7 +16,7 @@ package master import ( "github.com/spf13/cobra" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/pb" ) // NewPauseTaskCmd creates a PauseTask command. diff --git a/dm/dm/ctl/master/purge_relay.go b/dm/dm/ctl/master/purge_relay.go index 433b4c95d14..109ccfa767b 100644 --- a/dm/dm/ctl/master/purge_relay.go +++ b/dm/dm/ctl/master/purge_relay.go @@ -22,9 +22,9 @@ import ( "github.com/pingcap/errors" "github.com/spf13/cobra" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // NewPurgeRelayCmd creates a PurgeRelay command diff --git a/dm/dm/ctl/master/query_status.go b/dm/dm/ctl/master/query_status.go index 726958a5054..f5d11d333cb 100644 --- a/dm/dm/ctl/master/query_status.go +++ b/dm/dm/ctl/master/query_status.go @@ -21,8 +21,8 @@ import ( "github.com/spf13/cobra" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/pb" ) const stageError = "Error" diff --git a/dm/dm/ctl/master/query_status_test.go b/dm/dm/ctl/master/query_status_test.go index 2956ad4f4cc..7fc47dcb519 100644 --- a/dm/dm/ctl/master/query_status_test.go +++ b/dm/dm/ctl/master/query_status_test.go @@ -17,7 +17,7 @@ import ( "sort" "testing" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/pb" "github.com/pingcap/check" ) diff --git a/dm/dm/ctl/master/resume_relay.go b/dm/dm/ctl/master/resume_relay.go index 921b2f1577f..6b2d0101f60 100644 --- a/dm/dm/ctl/master/resume_relay.go +++ b/dm/dm/ctl/master/resume_relay.go @@ -20,8 +20,8 @@ import ( "github.com/spf13/cobra" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/pb" ) // NewResumeRelayCmd creates a ResumeRelay command. diff --git a/dm/dm/ctl/master/resume_task.go b/dm/dm/ctl/master/resume_task.go index 4715daa36c2..85fc516b743 100644 --- a/dm/dm/ctl/master/resume_task.go +++ b/dm/dm/ctl/master/resume_task.go @@ -16,7 +16,7 @@ package master import ( "github.com/spf13/cobra" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/pb" ) // NewResumeTaskCmd creates a ResumeTask command. diff --git a/dm/dm/ctl/master/show_ddl_locks.go b/dm/dm/ctl/master/show_ddl_locks.go index 9e42cc3e3d7..ebca2044130 100644 --- a/dm/dm/ctl/master/show_ddl_locks.go +++ b/dm/dm/ctl/master/show_ddl_locks.go @@ -18,8 +18,8 @@ import ( "errors" "os" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/pb" "github.com/spf13/cobra" ) diff --git a/dm/dm/ctl/master/source_table_schema.go b/dm/dm/ctl/master/source_table_schema.go index c0700e915ad..91c5acd390d 100644 --- a/dm/dm/ctl/master/source_table_schema.go +++ b/dm/dm/ctl/master/source_table_schema.go @@ -14,8 +14,8 @@ package master import ( - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/pb" "github.com/spf13/cobra" ) diff --git a/dm/dm/ctl/master/start_stop_relay.go b/dm/dm/ctl/master/start_stop_relay.go index 8cd101fce8b..749af913aeb 100644 --- a/dm/dm/ctl/master/start_stop_relay.go +++ b/dm/dm/ctl/master/start_stop_relay.go @@ -18,8 +18,8 @@ import ( "errors" "os" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/pb" "github.com/spf13/cobra" ) diff --git a/dm/dm/ctl/master/start_task.go b/dm/dm/ctl/master/start_task.go index b8fc94c999f..21e286a41a5 100644 --- a/dm/dm/ctl/master/start_task.go +++ b/dm/dm/ctl/master/start_task.go @@ -21,12 +21,12 @@ import ( "github.com/spf13/cobra" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/checker" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/checker" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // NewStartTaskCmd creates a StartTask command. diff --git a/dm/dm/ctl/master/stop_task.go b/dm/dm/ctl/master/stop_task.go index 90b2390b8e4..1edd46b93af 100644 --- a/dm/dm/ctl/master/stop_task.go +++ b/dm/dm/ctl/master/stop_task.go @@ -16,7 +16,7 @@ package master import ( "github.com/spf13/cobra" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/pb" ) // NewStopTaskCmd creates a StopTask command. diff --git a/dm/dm/ctl/master/transfer_source.go b/dm/dm/ctl/master/transfer_source.go index 252710e5c36..a7947b883b3 100644 --- a/dm/dm/ctl/master/transfer_source.go +++ b/dm/dm/ctl/master/transfer_source.go @@ -18,8 +18,8 @@ import ( "errors" "os" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/pb" "github.com/spf13/cobra" ) diff --git a/dm/dm/ctl/master/unlock_ddl_lock.go b/dm/dm/ctl/master/unlock_ddl_lock.go index 48fcb1ebead..d551b7b61fa 100644 --- a/dm/dm/ctl/master/unlock_ddl_lock.go +++ b/dm/dm/ctl/master/unlock_ddl_lock.go @@ -19,8 +19,8 @@ import ( "fmt" "os" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/pb" "github.com/spf13/cobra" ) diff --git a/dm/dm/master/agent_pool.go b/dm/dm/master/agent_pool.go index a9d2c628471..4f4eee68196 100644 --- a/dm/dm/master/agent_pool.go +++ b/dm/dm/master/agent_pool.go @@ -17,7 +17,7 @@ import ( "context" "math" - "github.com/pingcap/ticdc/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/log" "go.uber.org/zap" "golang.org/x/time/rate" diff --git a/dm/dm/master/bootstrap.go b/dm/dm/master/bootstrap.go index 1b3609bc0f0..9c090843387 100644 --- a/dm/dm/master/bootstrap.go +++ b/dm/dm/master/bootstrap.go @@ -25,17 +25,17 @@ import ( "github.com/pingcap/errors" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/master/scheduler" - "github.com/pingcap/ticdc/dm/dm/master/workerrpc" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/conn" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/upgrade" - "github.com/pingcap/ticdc/dm/pkg/v1dbschema" - "github.com/pingcap/ticdc/dm/pkg/v1workermeta" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/master/scheduler" + "github.com/pingcap/tiflow/dm/dm/master/workerrpc" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/upgrade" + "github.com/pingcap/tiflow/dm/pkg/v1dbschema" + "github.com/pingcap/tiflow/dm/pkg/v1workermeta" ) var ( diff --git a/dm/dm/master/bootstrap_test.go b/dm/dm/master/bootstrap_test.go index e6df89f539a..d4daaec6bd5 100644 --- a/dm/dm/master/bootstrap_test.go +++ b/dm/dm/master/bootstrap_test.go @@ -25,12 +25,12 @@ import ( . "github.com/pingcap/check" filter "github.com/pingcap/tidb-tools/pkg/binlog-filter" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/dm/pbmock" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/pbmock" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) const ( diff --git a/dm/dm/master/config.go b/dm/dm/master/config.go index b35ec573645..e595906a986 100644 --- a/dm/dm/master/config.go +++ b/dm/dm/master/config.go @@ -29,10 +29,10 @@ import ( "go.etcd.io/etcd/embed" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) const ( diff --git a/dm/dm/master/config_test.go b/dm/dm/master/config_test.go index 051fb17a947..a1486644dd7 100644 --- a/dm/dm/master/config_test.go +++ b/dm/dm/master/config_test.go @@ -25,8 +25,8 @@ import ( "github.com/pingcap/check" "go.etcd.io/etcd/embed" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) var ( diff --git a/dm/dm/master/election.go b/dm/dm/master/election.go index 0532fd6b2e6..d6bd3989c42 100644 --- a/dm/dm/master/election.go +++ b/dm/dm/master/election.go @@ -23,9 +23,9 @@ import ( "go.uber.org/zap" "google.golang.org/grpc" - "github.com/pingcap/ticdc/dm/dm/master/metrics" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/log" + "github.com/pingcap/tiflow/dm/dm/master/metrics" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/log" ) const ( diff --git a/dm/dm/master/election_test.go b/dm/dm/master/election_test.go index 754f3de4ac9..11a6c62aa8e 100644 --- a/dm/dm/master/election_test.go +++ b/dm/dm/master/election_test.go @@ -23,8 +23,8 @@ import ( "github.com/pingcap/failpoint" "github.com/tikv/pd/pkg/tempurl" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/pkg/utils" ) var _ = check.Suite(&testElectionSuite{}) @@ -78,9 +78,9 @@ func (t *testElectionSuite) TestFailToStartLeader(c *check.C) { c.Assert(leaderID, check.Equals, cfg1.Name) // fail to start scheduler/pessimism/optimism - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/dm/master/FailToStartLeader", `return("dm-master-2")`), check.IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/dm/master/FailToStartLeader", `return("dm-master-2")`), check.IsNil) //nolint:errcheck - defer failpoint.Disable("github.com/pingcap/ticdc/dm/dm/master/FailToStartLeader") + defer failpoint.Disable("github.com/pingcap/tiflow/dm/dm/master/FailToStartLeader") s1.election.Resign() time.Sleep(1 * time.Second) @@ -91,7 +91,7 @@ func (t *testElectionSuite) TestFailToStartLeader(c *check.C) { c.Assert(leaderID, check.Equals, cfg1.Name) //nolint:errcheck - failpoint.Disable("github.com/pingcap/ticdc/dm/dm/master/FailToStartLeader") + failpoint.Disable("github.com/pingcap/tiflow/dm/dm/master/FailToStartLeader") s1.election.Resign() time.Sleep(1 * time.Second) diff --git a/dm/dm/master/etcd.go b/dm/dm/master/etcd.go index 5447e1ffcca..bd7762cc4b7 100644 --- a/dm/dm/master/etcd.go +++ b/dm/dm/master/etcd.go @@ -26,9 +26,9 @@ import ( "go.uber.org/zap" "google.golang.org/grpc" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) const ( diff --git a/dm/dm/master/etcd_test.go b/dm/dm/master/etcd_test.go index d16bfd1102a..e3a8a8b1572 100644 --- a/dm/dm/master/etcd_test.go +++ b/dm/dm/master/etcd_test.go @@ -27,9 +27,9 @@ import ( "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/embed" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) var _ = check.Suite(&testEtcdSuite{}) diff --git a/dm/dm/master/http_handler.go b/dm/dm/master/http_handler.go index 33054c8f83f..e2bca8ff576 100644 --- a/dm/dm/master/http_handler.go +++ b/dm/dm/master/http_handler.go @@ -22,10 +22,10 @@ import ( "github.com/grpc-ecosystem/grpc-gateway/runtime" "google.golang.org/grpc" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // statusHandler handles process status. diff --git a/dm/dm/master/metrics/metrics.go b/dm/dm/master/metrics/metrics.go index 95f54524875..486ac22af1e 100644 --- a/dm/dm/master/metrics/metrics.go +++ b/dm/dm/master/metrics/metrics.go @@ -20,7 +20,7 @@ import ( cpu "github.com/pingcap/tidb-tools/pkg/utils" "github.com/prometheus/client_golang/prometheus" - "github.com/pingcap/ticdc/dm/pkg/metricsproxy" + "github.com/pingcap/tiflow/dm/pkg/metricsproxy" ) // used for ddlPendingCounter, no "Resolved" lock because they will be diff --git a/dm/dm/master/openapi.go b/dm/dm/master/openapi.go index 2c598ca46ef..66ae60fd39b 100644 --- a/dm/dm/master/openapi.go +++ b/dm/dm/master/openapi.go @@ -26,17 +26,17 @@ import ( echomiddleware "github.com/labstack/echo/v4/middleware" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/checker" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/master/scheduler" - "github.com/pingcap/ticdc/dm/dm/master/workerrpc" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/openapi" - "github.com/pingcap/ticdc/dm/pkg/conn" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/checker" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/master/scheduler" + "github.com/pingcap/tiflow/dm/dm/master/workerrpc" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/openapi" + "github.com/pingcap/tiflow/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) const ( diff --git a/dm/dm/master/openapi_test.go b/dm/dm/master/openapi_test.go index ad6b9723bce..de038b3df72 100644 --- a/dm/dm/master/openapi_test.go +++ b/dm/dm/master/openapi_test.go @@ -31,17 +31,17 @@ import ( "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/integration" - "github.com/pingcap/ticdc/dm/checker" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/master/workerrpc" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/dm/pbmock" - "github.com/pingcap/ticdc/dm/openapi" - "github.com/pingcap/ticdc/dm/openapi/fixtures" - "github.com/pingcap/ticdc/dm/pkg/conn" - "github.com/pingcap/ticdc/dm/pkg/ha" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/checker" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/master/workerrpc" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/pbmock" + "github.com/pingcap/tiflow/dm/openapi" + "github.com/pingcap/tiflow/dm/openapi/fixtures" + "github.com/pingcap/tiflow/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/pkg/ha" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) var openAPITestSuite = check.SerialSuites(&openAPISuite{}) @@ -426,7 +426,7 @@ func (t *openAPISuite) TestRelayAPI(c *check.C) { func (t *openAPISuite) TestTaskAPI(c *check.C) { ctx, cancel := context.WithCancel(context.Background()) s := setupServer(ctx, c) - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/dm/master/MockSkipAdjustTargetDB", `return(true)`), check.IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/dm/master/MockSkipAdjustTargetDB", `return(true)`), check.IsNil) checker.CheckSyncConfigFunc = mockCheckSyncConfig ctrl := gomock.NewController(c) defer func() { @@ -536,7 +536,7 @@ func (t *openAPISuite) TestTaskAPI(c *check.C) { c.Assert(result.Code(), check.Equals, http.StatusNoContent) subTaskM = s.scheduler.GetSubTaskCfgsByTask(task.Name) c.Assert(len(subTaskM) == 0, check.IsTrue) - c.Assert(failpoint.Disable("github.com/pingcap/ticdc/dm/dm/master/MockSkipAdjustTargetDB"), check.IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tiflow/dm/dm/master/MockSkipAdjustTargetDB"), check.IsNil) // list tasks result = testutil.NewRequest().Get(taskURL).Go(t.testT, s.echo) diff --git a/dm/dm/master/scheduler/scheduler.go b/dm/dm/master/scheduler/scheduler.go index b11dac80510..00fe67fd70b 100644 --- a/dm/dm/master/scheduler/scheduler.go +++ b/dm/dm/master/scheduler/scheduler.go @@ -24,14 +24,14 @@ import ( "go.etcd.io/etcd/clientv3" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/master/metrics" - "github.com/pingcap/ticdc/dm/dm/master/workerrpc" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" - "github.com/pingcap/ticdc/dm/pkg/ha" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/master/metrics" + "github.com/pingcap/tiflow/dm/dm/master/workerrpc" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/pkg/ha" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // Scheduler schedules tasks for DM-worker instances, including: diff --git a/dm/dm/master/scheduler/scheduler_test.go b/dm/dm/master/scheduler/scheduler_test.go index 66581204c5d..5335a9daf47 100644 --- a/dm/dm/master/scheduler/scheduler_test.go +++ b/dm/dm/master/scheduler/scheduler_test.go @@ -26,13 +26,13 @@ import ( v3rpc "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" "go.etcd.io/etcd/integration" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/master/workerrpc" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/ha" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/master/workerrpc" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/ha" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) const ( @@ -1212,11 +1212,11 @@ func (t *testScheduler) TestTransferSource(c *C) { // now we have (worker1, nil) (worker2, source2) (worker3, source3) (worker4, source1) // test fail halfway won't left old worker unbound - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/dm/master/scheduler/failToReplaceSourceBound", `return()`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/dm/master/scheduler/failToReplaceSourceBound", `return()`), IsNil) c.Assert(s.TransferSource(sourceID1, workerName1), NotNil) c.Assert(s.bounds[sourceID1], DeepEquals, worker4) c.Assert(worker1.Stage(), Equals, WorkerFree) - c.Assert(failpoint.Disable("github.com/pingcap/ticdc/dm/dm/master/scheduler/failToReplaceSourceBound"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tiflow/dm/dm/master/scheduler/failToReplaceSourceBound"), IsNil) // test can't transfer when there's any running task on the source s.expectSubTaskStages.Store("test", map[string]ha.Stage{sourceID1: {Expect: pb.Stage_Running}}) @@ -1380,12 +1380,12 @@ func (t *testScheduler) TestCloseAllWorkers(c *C) { c.Assert(err, IsNil) } - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/dm/master/scheduler/failToRecoverWorkersBounds", "return"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/dm/master/scheduler/failToRecoverWorkersBounds", "return"), IsNil) // Test closed when fail to start c.Assert(s.Start(ctx, etcdTestCli), ErrorMatches, "failToRecoverWorkersBounds") c.Assert(s.workers, HasLen, 3) checkAllWorkersClosed(c, s, true) - c.Assert(failpoint.Disable("github.com/pingcap/ticdc/dm/dm/master/scheduler/failToRecoverWorkersBounds"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tiflow/dm/dm/master/scheduler/failToRecoverWorkersBounds"), IsNil) s.workers = map[string]*Worker{} c.Assert(s.Start(ctx, etcdTestCli), IsNil) diff --git a/dm/dm/master/scheduler/worker.go b/dm/dm/master/scheduler/worker.go index fc100ecc1db..cbac851ef33 100644 --- a/dm/dm/master/scheduler/worker.go +++ b/dm/dm/master/scheduler/worker.go @@ -20,12 +20,12 @@ import ( "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/master/metrics" - "github.com/pingcap/ticdc/dm/dm/master/workerrpc" - "github.com/pingcap/ticdc/dm/pkg/ha" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/master/metrics" + "github.com/pingcap/tiflow/dm/dm/master/workerrpc" + "github.com/pingcap/tiflow/dm/pkg/ha" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // WorkerStage represents the stage of a DM-worker instance. diff --git a/dm/dm/master/scheduler/worker_test.go b/dm/dm/master/scheduler/worker_test.go index 0f113256665..fa571ebbcee 100644 --- a/dm/dm/master/scheduler/worker_test.go +++ b/dm/dm/master/scheduler/worker_test.go @@ -19,11 +19,11 @@ import ( . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/master/workerrpc" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/ha" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/master/workerrpc" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/ha" + "github.com/pingcap/tiflow/dm/pkg/terror" ) type testWorker struct{} diff --git a/dm/dm/master/server.go b/dm/dm/master/server.go index fd9ebdb197b..1f50386f448 100644 --- a/dm/dm/master/server.go +++ b/dm/dm/master/server.go @@ -36,24 +36,24 @@ import ( "go.uber.org/zap" "google.golang.org/grpc" - "github.com/pingcap/ticdc/dm/checker" - dmcommon "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/dm/config" - ctlcommon "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/master/metrics" - "github.com/pingcap/ticdc/dm/dm/master/scheduler" - "github.com/pingcap/ticdc/dm/dm/master/shardddl" - "github.com/pingcap/ticdc/dm/dm/master/workerrpc" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/dm/unit" - "github.com/pingcap/ticdc/dm/pkg/conn" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/cputil" - "github.com/pingcap/ticdc/dm/pkg/election" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/checker" + dmcommon "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/dm/config" + ctlcommon "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/master/metrics" + "github.com/pingcap/tiflow/dm/dm/master/scheduler" + "github.com/pingcap/tiflow/dm/dm/master/shardddl" + "github.com/pingcap/tiflow/dm/dm/master/workerrpc" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/unit" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/cputil" + "github.com/pingcap/tiflow/dm/pkg/election" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) const ( diff --git a/dm/dm/master/server_test.go b/dm/dm/master/server_test.go index 8ee1bfb628f..842f9283221 100644 --- a/dm/dm/master/server_test.go +++ b/dm/dm/master/server_test.go @@ -44,24 +44,24 @@ import ( "go.etcd.io/etcd/integration" "google.golang.org/grpc" - "github.com/pingcap/ticdc/dm/checker" - common2 "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/master/scheduler" - "github.com/pingcap/ticdc/dm/dm/master/shardddl" - "github.com/pingcap/ticdc/dm/dm/master/workerrpc" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/dm/pbmock" - "github.com/pingcap/ticdc/dm/pkg/conn" - "github.com/pingcap/ticdc/dm/pkg/cputil" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" - "github.com/pingcap/ticdc/dm/pkg/ha" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/shardddl/optimism" - "github.com/pingcap/ticdc/dm/pkg/shardddl/pessimism" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/checker" + common2 "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/master/scheduler" + "github.com/pingcap/tiflow/dm/dm/master/shardddl" + "github.com/pingcap/tiflow/dm/dm/master/workerrpc" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/pbmock" + "github.com/pingcap/tiflow/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/pkg/cputil" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/pkg/ha" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/shardddl/optimism" + "github.com/pingcap/tiflow/dm/pkg/shardddl/pessimism" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // use task config from integration test `sharding`. @@ -2074,12 +2074,12 @@ func (t *testMaster) subTaskStageMatch(c *check.C, s *scheduler.Scheduler, task, } func (t *testMaster) TestGRPCLongResponse(c *check.C) { - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/dm/master/LongRPCResponse", `return()`), check.IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/dm/master/LongRPCResponse", `return()`), check.IsNil) //nolint:errcheck - defer failpoint.Disable("github.com/pingcap/ticdc/dm/dm/master/LongRPCResponse") - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/dm/ctl/common/SkipUpdateMasterClient", `return()`), check.IsNil) + defer failpoint.Disable("github.com/pingcap/tiflow/dm/dm/master/LongRPCResponse") + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/dm/ctl/common/SkipUpdateMasterClient", `return()`), check.IsNil) //nolint:errcheck - defer failpoint.Disable("github.com/pingcap/ticdc/dm/dm/ctl/common/SkipUpdateMasterClient") + defer failpoint.Disable("github.com/pingcap/tiflow/dm/dm/ctl/common/SkipUpdateMasterClient") masterAddr := tempurl.Alloc()[len("http://"):] lis, err := net.Listen("tcp", masterAddr) diff --git a/dm/dm/master/shardddl/info.go b/dm/dm/master/shardddl/info.go index b4bbc7c1712..916e206c733 100644 --- a/dm/dm/master/shardddl/info.go +++ b/dm/dm/master/shardddl/info.go @@ -16,7 +16,7 @@ package shardddl import ( "sort" - "github.com/pingcap/ticdc/dm/pkg/shardddl/pessimism" + "github.com/pingcap/tiflow/dm/pkg/shardddl/pessimism" ) // PessimismInfoSlice attaches the methods of Interface to []pessimism.Info, diff --git a/dm/dm/master/shardddl/info_test.go b/dm/dm/master/shardddl/info_test.go index 49ddc0e104a..753a2eaae90 100644 --- a/dm/dm/master/shardddl/info_test.go +++ b/dm/dm/master/shardddl/info_test.go @@ -16,7 +16,7 @@ package shardddl import ( . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/pkg/shardddl/pessimism" + "github.com/pingcap/tiflow/dm/pkg/shardddl/pessimism" ) func (t *testPessimist) TestInfoSlice(c *C) { diff --git a/dm/dm/master/shardddl/optimist.go b/dm/dm/master/shardddl/optimist.go index eda57ea0332..bab8ef7e86d 100644 --- a/dm/dm/master/shardddl/optimist.go +++ b/dm/dm/master/shardddl/optimist.go @@ -26,15 +26,15 @@ import ( "go.etcd.io/etcd/clientv3" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/master/metrics" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/shardddl/optimism" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/master/metrics" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/shardddl/optimism" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // Optimist is used to coordinate the shard DDL migration in optimism mode. diff --git a/dm/dm/master/shardddl/optimist_test.go b/dm/dm/master/shardddl/optimist_test.go index b0ba6739437..873fe8c0794 100644 --- a/dm/dm/master/shardddl/optimist_test.go +++ b/dm/dm/master/shardddl/optimist_test.go @@ -30,11 +30,11 @@ import ( "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/integration" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/shardddl/optimism" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/shardddl/optimism" + "github.com/pingcap/tiflow/dm/pkg/utils" ) type testOptimist struct{} diff --git a/dm/dm/master/shardddl/pessimist.go b/dm/dm/master/shardddl/pessimist.go index e1f44df1e08..61f58a566ed 100644 --- a/dm/dm/master/shardddl/pessimist.go +++ b/dm/dm/master/shardddl/pessimist.go @@ -24,13 +24,13 @@ import ( "github.com/pingcap/failpoint" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/master/metrics" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/shardddl/pessimism" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/master/metrics" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/shardddl/pessimism" + "github.com/pingcap/tiflow/dm/pkg/terror" ) var ( diff --git a/dm/dm/master/shardddl/pessimist_test.go b/dm/dm/master/shardddl/pessimist_test.go index 721e0a11c83..b2daee93b4c 100644 --- a/dm/dm/master/shardddl/pessimist_test.go +++ b/dm/dm/master/shardddl/pessimist_test.go @@ -25,13 +25,13 @@ import ( v3rpc "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" "go.etcd.io/etcd/integration" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/shardddl/pessimism" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/shardddl/pessimism" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) var ( diff --git a/dm/dm/master/workerrpc/interface.go b/dm/dm/master/workerrpc/interface.go index 94939b7504d..c704e4a04c2 100644 --- a/dm/dm/master/workerrpc/interface.go +++ b/dm/dm/master/workerrpc/interface.go @@ -17,7 +17,7 @@ import ( "context" "time" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/pb" ) // CmdType represents the concrete request type in Request or response type in Response. diff --git a/dm/dm/master/workerrpc/rawgrpc.go b/dm/dm/master/workerrpc/rawgrpc.go index b6f7d2f85c4..ace17d24b0a 100644 --- a/dm/dm/master/workerrpc/rawgrpc.go +++ b/dm/dm/master/workerrpc/rawgrpc.go @@ -22,9 +22,9 @@ import ( "google.golang.org/grpc" "google.golang.org/grpc/backoff" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // GRPCClient stores raw grpc connection and worker client. diff --git a/dm/dm/master/workerrpc/workerrpc_test.go b/dm/dm/master/workerrpc/workerrpc_test.go index bba2f148383..ab1d93113bf 100644 --- a/dm/dm/master/workerrpc/workerrpc_test.go +++ b/dm/dm/master/workerrpc/workerrpc_test.go @@ -23,10 +23,10 @@ import ( "github.com/pingcap/errors" "github.com/tikv/pd/pkg/tempurl" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/dm/pbmock" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/pbmock" + "github.com/pingcap/tiflow/dm/pkg/terror" ) var _ = Suite(&testWorkerRPCSuite{}) diff --git a/dm/dm/pbmock/dmmaster.go b/dm/dm/pbmock/dmmaster.go index e594da6a59c..e30c76f198f 100644 --- a/dm/dm/pbmock/dmmaster.go +++ b/dm/dm/pbmock/dmmaster.go @@ -1,5 +1,5 @@ // Code generated by MockGen. DO NOT EDIT. -// Source: github.com/pingcap/ticdc/dm/dm/pb (interfaces: MasterClient,MasterServer) +// Source: github.com/pingcap/tiflow/dm/dm/pb (interfaces: MasterClient,MasterServer) // Package pbmock is a generated GoMock package. package pbmock @@ -9,7 +9,7 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" - pb "github.com/pingcap/ticdc/dm/dm/pb" + pb "github.com/pingcap/tiflow/dm/dm/pb" grpc "google.golang.org/grpc" ) diff --git a/dm/dm/pbmock/dmworker.go b/dm/dm/pbmock/dmworker.go index 4e4b33aa3bd..31355d49404 100644 --- a/dm/dm/pbmock/dmworker.go +++ b/dm/dm/pbmock/dmworker.go @@ -1,5 +1,5 @@ // Code generated by MockGen. DO NOT EDIT. -// Source: github.com/pingcap/ticdc/dm/dm/pb (interfaces: WorkerClient,WorkerServer) +// Source: github.com/pingcap/tiflow/dm/dm/pb (interfaces: WorkerClient,WorkerServer) // Package pbmock is a generated GoMock package. package pbmock @@ -9,7 +9,7 @@ import ( reflect "reflect" gomock "github.com/golang/mock/gomock" - pb "github.com/pingcap/ticdc/dm/dm/pb" + pb "github.com/pingcap/tiflow/dm/dm/pb" grpc "google.golang.org/grpc" ) diff --git a/dm/dm/portal/api.go b/dm/dm/portal/api.go index 1a163dae022..e5eb688456e 100644 --- a/dm/dm/portal/api.go +++ b/dm/dm/portal/api.go @@ -36,9 +36,9 @@ import ( "go.uber.org/zap" "gopkg.in/yaml.v2" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/utils" ) const ( diff --git a/dm/dm/portal/api_test.go b/dm/dm/portal/api_test.go index 010ebe2ec74..c167b801156 100644 --- a/dm/dm/portal/api_test.go +++ b/dm/dm/portal/api_test.go @@ -34,7 +34,7 @@ import ( router "github.com/pingcap/tidb-tools/pkg/table-router" "github.com/pingcap/tidb/br/pkg/mock" - "github.com/pingcap/ticdc/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/config" ) var _ = Suite(&testPortalSuite{}) diff --git a/dm/dm/portal/config.go b/dm/dm/portal/config.go index d3c953bd512..d65541905de 100644 --- a/dm/dm/portal/config.go +++ b/dm/dm/portal/config.go @@ -26,9 +26,9 @@ import ( router "github.com/pingcap/tidb-tools/pkg/table-router" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // Config is the dm-portal's config. diff --git a/dm/dm/unit/unit.go b/dm/dm/unit/unit.go index 3b851d40d64..4451a652e86 100644 --- a/dm/dm/unit/unit.go +++ b/dm/dm/unit/unit.go @@ -18,10 +18,10 @@ import ( "strings" "time" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/terror" ) const ( diff --git a/dm/dm/unit/unit_test.go b/dm/dm/unit/unit_test.go index d1a44418564..b37118886b2 100644 --- a/dm/dm/unit/unit_test.go +++ b/dm/dm/unit/unit_test.go @@ -20,8 +20,8 @@ import ( "github.com/pingcap/check" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/terror" ) func TestSuite(t *testing.T) { diff --git a/dm/dm/worker/config.go b/dm/dm/worker/config.go index 9b738d0fb35..30947e5dcd6 100644 --- a/dm/dm/worker/config.go +++ b/dm/dm/worker/config.go @@ -25,10 +25,10 @@ import ( "github.com/BurntSushi/toml" "github.com/pingcap/failpoint" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // SampleConfigFile is sample config file of dm-worker. diff --git a/dm/dm/worker/config_test.go b/dm/dm/worker/config_test.go index ebbc3beabe5..30ccc607484 100644 --- a/dm/dm/worker/config_test.go +++ b/dm/dm/worker/config_test.go @@ -21,7 +21,7 @@ import ( "github.com/kami-zh/go-capturer" "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) var ( diff --git a/dm/dm/worker/join.go b/dm/dm/worker/join.go index b88881fbcfd..ece56ec1a4d 100644 --- a/dm/dm/worker/join.go +++ b/dm/dm/worker/join.go @@ -23,11 +23,11 @@ import ( "go.uber.org/zap" "google.golang.org/grpc" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/ha" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/ha" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // GetJoinURLs gets the endpoints from the join address. diff --git a/dm/dm/worker/metrics.go b/dm/dm/worker/metrics.go index 3ed9710d695..2a77ffd46dd 100644 --- a/dm/dm/worker/metrics.go +++ b/dm/dm/worker/metrics.go @@ -24,14 +24,14 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promhttp" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/dumpling" - "github.com/pingcap/ticdc/dm/loader" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/metricsproxy" - "github.com/pingcap/ticdc/dm/pkg/utils" - "github.com/pingcap/ticdc/dm/relay" - syncer "github.com/pingcap/ticdc/dm/syncer/metrics" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/dumpling" + "github.com/pingcap/tiflow/dm/loader" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/metricsproxy" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/relay" + syncer "github.com/pingcap/tiflow/dm/syncer/metrics" ) const ( diff --git a/dm/dm/worker/relay.go b/dm/dm/worker/relay.go index a4ff098ecad..6fd3655ba7b 100644 --- a/dm/dm/worker/relay.go +++ b/dm/dm/worker/relay.go @@ -21,14 +21,14 @@ import ( "go.uber.org/atomic" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/streamer" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/relay" - "github.com/pingcap/ticdc/dm/relay/purger" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/streamer" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/relay" + "github.com/pingcap/tiflow/dm/relay/purger" ) // RelayHolder for relay unit. diff --git a/dm/dm/worker/relay_test.go b/dm/dm/worker/relay_test.go index 6c447c40048..9a8b94e7938 100644 --- a/dm/dm/worker/relay_test.go +++ b/dm/dm/worker/relay_test.go @@ -21,15 +21,15 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/dm/unit" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/gtid" - pkgstreamer "github.com/pingcap/ticdc/dm/pkg/streamer" - "github.com/pingcap/ticdc/dm/pkg/utils" - "github.com/pingcap/ticdc/dm/relay" - "github.com/pingcap/ticdc/dm/relay/purger" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/unit" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/gtid" + pkgstreamer "github.com/pingcap/tiflow/dm/pkg/streamer" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/relay" + "github.com/pingcap/tiflow/dm/relay/purger" ) type testRelay struct{} diff --git a/dm/dm/worker/server.go b/dm/dm/worker/server.go index e8227b2d56a..cac9331fdf4 100644 --- a/dm/dm/worker/server.go +++ b/dm/dm/worker/server.go @@ -20,18 +20,18 @@ import ( "sync" "time" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/dm/unit" - "github.com/pingcap/ticdc/dm/pkg/binlog" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" - "github.com/pingcap/ticdc/dm/pkg/ha" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" - "github.com/pingcap/ticdc/dm/syncer" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/unit" + "github.com/pingcap/tiflow/dm/pkg/binlog" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/pkg/ha" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/syncer" "github.com/pingcap/errors" toolutils "github.com/pingcap/tidb-tools/pkg/utils" diff --git a/dm/dm/worker/server_test.go b/dm/dm/worker/server_test.go index 20d620c8e3f..c9458ece493 100644 --- a/dm/dm/worker/server_test.go +++ b/dm/dm/worker/server_test.go @@ -31,16 +31,16 @@ import ( v3rpc "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" "google.golang.org/grpc" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/dm/unit" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/ha" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/streamer" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/unit" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/ha" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/streamer" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // do not forget to update this path if the file removed/renamed. @@ -65,12 +65,12 @@ func (t *testServer) SetUpSuite(c *C) { c.Assert(err, IsNil) getMinLocForSubTaskFunc = getFakeLocForSubTask - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/dm/worker/MockGetSourceCfgFromETCD", `return(true)`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/dm/worker/MockGetSourceCfgFromETCD", `return(true)`), IsNil) } func (t *testServer) TearDownSuite(c *C) { getMinLocForSubTaskFunc = getMinLocForSubTask - c.Assert(failpoint.Disable("github.com/pingcap/ticdc/dm/dm/worker/MockGetSourceCfgFromETCD"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tiflow/dm/dm/worker/MockGetSourceCfgFromETCD"), IsNil) } func createMockETCD(dir string, host string) (*embed.Etcd, error) { @@ -295,7 +295,7 @@ func (t *testServer) TestHandleSourceBoundAfterError(c *C) { }), IsTrue) // enable failpoint - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/pkg/ha/FailToGetSourceCfg", `return(true)`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/pkg/ha/FailToGetSourceCfg", `return(true)`), IsNil) sourceCfg := loadSourceConfigWithoutPassword(c) sourceCfg.EnableRelay = false _, err = ha.PutSourceCfg(etcdCli, sourceCfg) @@ -331,7 +331,7 @@ func (t *testServer) TestHandleSourceBoundAfterError(c *C) { // stop watching and disable failpoint cancel() wg.Wait() - c.Assert(failpoint.Disable("github.com/pingcap/ticdc/dm/pkg/ha/FailToGetSourceCfg"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tiflow/dm/pkg/ha/FailToGetSourceCfg"), IsNil) _, err = ha.PutSourceBound(etcdCli, sourceBound) c.Assert(err, IsNil) diff --git a/dm/dm/worker/source_worker.go b/dm/dm/worker/source_worker.go index 11e626fda7d..9ea3e9385cc 100644 --- a/dm/dm/worker/source_worker.go +++ b/dm/dm/worker/source_worker.go @@ -27,17 +27,17 @@ import ( "go.uber.org/atomic" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/conn" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" - "github.com/pingcap/ticdc/dm/pkg/ha" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/streamer" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" - "github.com/pingcap/ticdc/dm/relay/purger" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/pkg/ha" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/streamer" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/relay/purger" ) // SourceWorker manages a source(upstream) which is mainly related to subtasks and relay. diff --git a/dm/dm/worker/source_worker_test.go b/dm/dm/worker/source_worker_test.go index 0d65f19284f..2b234f7b924 100644 --- a/dm/dm/worker/source_worker_test.go +++ b/dm/dm/worker/source_worker_test.go @@ -26,14 +26,14 @@ import ( "github.com/tikv/pd/pkg/tempurl" "go.etcd.io/etcd/clientv3" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/dm/unit" - "github.com/pingcap/ticdc/dm/pkg/conn" - "github.com/pingcap/ticdc/dm/pkg/ha" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/streamer" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/unit" + "github.com/pingcap/tiflow/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/pkg/ha" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/streamer" + "github.com/pingcap/tiflow/dm/pkg/utils" ) var emptyWorkerStatusInfoJSONLength = 25 @@ -122,12 +122,12 @@ func (t *testServer2) SetUpSuite(c *C) { c.Assert(err, IsNil) getMinLocForSubTaskFunc = getFakeLocForSubTask - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/dm/worker/MockGetSourceCfgFromETCD", `return(true)`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/dm/worker/MockGetSourceCfgFromETCD", `return(true)`), IsNil) } func (t *testServer2) TearDownSuite(c *C) { getMinLocForSubTaskFunc = getMinLocForSubTask - c.Assert(failpoint.Disable("github.com/pingcap/ticdc/dm/dm/worker/MockGetSourceCfgFromETCD"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tiflow/dm/dm/worker/MockGetSourceCfgFromETCD"), IsNil) } func (t *testServer2) TestTaskAutoResume(c *C) { @@ -162,18 +162,18 @@ func (t *testServer2) TestTaskAutoResume(c *C) { NewRelayHolder = NewRealRelayHolder }() - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/dumpling/dumpUnitProcessForever", `return()`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/dumpling/dumpUnitProcessForever", `return()`), IsNil) //nolint:errcheck - defer failpoint.Disable("github.com/pingcap/ticdc/dm/dumpling/dumpUnitProcessForever") - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/dm/worker/mockCreateUnitsDumpOnly", `return(true)`), IsNil) + defer failpoint.Disable("github.com/pingcap/tiflow/dm/dumpling/dumpUnitProcessForever") + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/dm/worker/mockCreateUnitsDumpOnly", `return(true)`), IsNil) //nolint:errcheck - defer failpoint.Disable("github.com/pingcap/ticdc/dm/dm/worker/mockCreateUnitsDumpOnly") - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/loader/ignoreLoadCheckpointErr", `return()`), IsNil) + defer failpoint.Disable("github.com/pingcap/tiflow/dm/dm/worker/mockCreateUnitsDumpOnly") + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/loader/ignoreLoadCheckpointErr", `return()`), IsNil) //nolint:errcheck - defer failpoint.Disable("github.com/pingcap/ticdc/dm/loader/ignoreLoadCheckpointErr") - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/dumpling/dumpUnitProcessWithError", `return("test auto resume inject error")`), IsNil) + defer failpoint.Disable("github.com/pingcap/tiflow/dm/loader/ignoreLoadCheckpointErr") + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/dumpling/dumpUnitProcessWithError", `return("test auto resume inject error")`), IsNil) //nolint:errcheck - defer failpoint.Disable("github.com/pingcap/ticdc/dm/dumpling/dumpUnitProcessWithError") + defer failpoint.Disable("github.com/pingcap/tiflow/dm/dumpling/dumpUnitProcessWithError") s := NewServer(cfg) defer s.Close() @@ -210,7 +210,7 @@ func (t *testServer2) TestTaskAutoResume(c *C) { return false }), IsTrue) //nolint:errcheck - failpoint.Disable("github.com/pingcap/ticdc/dm/dumpling/dumpUnitProcessWithError") + failpoint.Disable("github.com/pingcap/tiflow/dm/dumpling/dumpUnitProcessWithError") rtsc, ok := s.getWorker(true).taskStatusChecker.(*realTaskStatusChecker) c.Assert(ok, IsTrue) @@ -251,7 +251,7 @@ func (t *testWorkerFunctionalities) SetUpSuite(c *C) { return []unit.Unit{mockDumper, mockLoader, mockSync} } getMinLocForSubTaskFunc = getFakeLocForSubTask - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/dm/worker/MockGetSourceCfgFromETCD", `return(true)`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/dm/worker/MockGetSourceCfgFromETCD", `return(true)`), IsNil) } func (t *testWorkerFunctionalities) TearDownSuite(c *C) { @@ -259,7 +259,7 @@ func (t *testWorkerFunctionalities) TearDownSuite(c *C) { NewSubTask = NewRealSubTask createUnits = createRealUnits getMinLocForSubTaskFunc = getMinLocForSubTask - c.Assert(failpoint.Disable("github.com/pingcap/ticdc/dm/dm/worker/MockGetSourceCfgFromETCD"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tiflow/dm/dm/worker/MockGetSourceCfgFromETCD"), IsNil) } func (t *testWorkerFunctionalities) TestWorkerFunctionalities(c *C) { @@ -424,14 +424,14 @@ func (t *testWorkerEtcdCompact) SetUpSuite(c *C) { mockSync := NewMockUnit(pb.UnitType_Sync) return []unit.Unit{mockDumper, mockLoader, mockSync} } - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/dm/worker/MockGetSourceCfgFromETCD", `return(true)`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/dm/worker/MockGetSourceCfgFromETCD", `return(true)`), IsNil) } func (t *testWorkerEtcdCompact) TearDownSuite(c *C) { NewRelayHolder = NewRealRelayHolder NewSubTask = NewRealSubTask createUnits = createRealUnits - c.Assert(failpoint.Disable("github.com/pingcap/ticdc/dm/dm/worker/MockGetSourceCfgFromETCD"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tiflow/dm/dm/worker/MockGetSourceCfgFromETCD"), IsNil) } func (t *testWorkerEtcdCompact) TestWatchSubtaskStageEtcdCompact(c *C) { diff --git a/dm/dm/worker/status.go b/dm/dm/worker/status.go index 4430a159beb..7940fa7a8ed 100644 --- a/dm/dm/worker/status.go +++ b/dm/dm/worker/status.go @@ -20,9 +20,9 @@ import ( "github.com/gogo/protobuf/jsonpb" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/binlog" ) // Status returns the status of the current sub task. diff --git a/dm/dm/worker/subtask.go b/dm/dm/worker/subtask.go index 81ef19296bb..c7a28751e2e 100644 --- a/dm/dm/worker/subtask.go +++ b/dm/dm/worker/subtask.go @@ -25,19 +25,19 @@ import ( "go.uber.org/atomic" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/dm/unit" - "github.com/pingcap/ticdc/dm/dumpling" - "github.com/pingcap/ticdc/dm/loader" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/shardddl/pessimism" - "github.com/pingcap/ticdc/dm/pkg/streamer" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" - "github.com/pingcap/ticdc/dm/syncer" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/unit" + "github.com/pingcap/tiflow/dm/dumpling" + "github.com/pingcap/tiflow/dm/loader" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/shardddl/pessimism" + "github.com/pingcap/tiflow/dm/pkg/streamer" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/syncer" ) const ( diff --git a/dm/dm/worker/subtask_test.go b/dm/dm/worker/subtask_test.go index d53cdaac6b0..88d9e7efd5c 100644 --- a/dm/dm/worker/subtask_test.go +++ b/dm/dm/worker/subtask_test.go @@ -18,15 +18,15 @@ import ( "strings" "time" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/dm/unit" - "github.com/pingcap/ticdc/dm/dumpling" - "github.com/pingcap/ticdc/dm/loader" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/streamer" - "github.com/pingcap/ticdc/dm/pkg/utils" - "github.com/pingcap/ticdc/dm/syncer" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/unit" + "github.com/pingcap/tiflow/dm/dumpling" + "github.com/pingcap/tiflow/dm/loader" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/streamer" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/syncer" . "github.com/pingcap/check" "github.com/pingcap/errors" diff --git a/dm/dm/worker/task_checker.go b/dm/dm/worker/task_checker.go index 3d82b6fafe0..2cacdc8e9a0 100644 --- a/dm/dm/worker/task_checker.go +++ b/dm/dm/worker/task_checker.go @@ -24,12 +24,12 @@ import ( "go.uber.org/atomic" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/backoff" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/retry" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/backoff" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/retry" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // Backoff related constants diff --git a/dm/dm/worker/task_checker_test.go b/dm/dm/worker/task_checker_test.go index b35fbe4b117..1a82bf4070a 100644 --- a/dm/dm/worker/task_checker_test.go +++ b/dm/dm/worker/task_checker_test.go @@ -21,11 +21,11 @@ import ( tmysql "github.com/pingcap/tidb/parser/mysql" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/dm/unit" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/unit" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) var _ = check.Suite(&testTaskCheckerSuite{}) diff --git a/dm/dm/worker/v1meta.go b/dm/dm/worker/v1meta.go index c8613e6fd5d..3b5c08e2e06 100644 --- a/dm/dm/worker/v1meta.go +++ b/dm/dm/worker/v1meta.go @@ -19,9 +19,9 @@ import ( "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/v1workermeta" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/v1workermeta" ) // OperateV1Meta implements WorkerServer.OperateV1Meta. diff --git a/dm/dumpling/dumpling.go b/dm/dumpling/dumpling.go index 05cf5280cc7..cab39480b0a 100644 --- a/dm/dumpling/dumpling.go +++ b/dm/dumpling/dumpling.go @@ -27,15 +27,15 @@ import ( "go.uber.org/atomic" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/dm/unit" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/conn" - dutils "github.com/pingcap/ticdc/dm/pkg/dumpling" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/unit" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/conn" + dutils "github.com/pingcap/tiflow/dm/pkg/dumpling" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // Dumpling dumps full data from a MySQL-compatible database. diff --git a/dm/dumpling/dumpling_test.go b/dm/dumpling/dumpling_test.go index 6427353214d..ce5d658a1d0 100644 --- a/dm/dumpling/dumpling_test.go +++ b/dm/dumpling/dumpling_test.go @@ -22,9 +22,9 @@ import ( "github.com/pingcap/tidb-tools/pkg/filter" "github.com/pingcap/tidb/dumpling/export" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/log" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/log" . "github.com/pingcap/check" ) @@ -73,9 +73,9 @@ func (d *testDumplingSuite) TestDumpling(c *C) { c.Assert(err, IsNil) resultCh := make(chan pb.ProcessResult, 1) - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/dumpling/dumpUnitProcessNoError", `return(true)`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/dumpling/dumpUnitProcessNoError", `return(true)`), IsNil) //nolint:errcheck - defer failpoint.Disable("github.com/pingcap/ticdc/dm/dumpling/dumpUnitProcessNoError") + defer failpoint.Disable("github.com/pingcap/tiflow/dm/dumpling/dumpUnitProcessNoError") dumpling.Process(ctx, resultCh) c.Assert(len(resultCh), Equals, 1) @@ -83,11 +83,11 @@ func (d *testDumplingSuite) TestDumpling(c *C) { c.Assert(result.IsCanceled, IsFalse) c.Assert(len(result.Errors), Equals, 0) //nolint:errcheck - failpoint.Disable("github.com/pingcap/ticdc/dm/dumpling/dumpUnitProcessNoError") + failpoint.Disable("github.com/pingcap/tiflow/dm/dumpling/dumpUnitProcessNoError") - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/dumpling/dumpUnitProcessWithError", `return("unknown error")`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/dumpling/dumpUnitProcessWithError", `return("unknown error")`), IsNil) // nolint:errcheck - defer failpoint.Disable("github.com/pingcap/ticdc/dm/dumpling/dumpUnitProcessWithError") + defer failpoint.Disable("github.com/pingcap/tiflow/dm/dumpling/dumpUnitProcessWithError") // return error dumpling.Process(ctx, resultCh) @@ -97,15 +97,15 @@ func (d *testDumplingSuite) TestDumpling(c *C) { c.Assert(len(result.Errors), Equals, 1) c.Assert(result.Errors[0].Message, Equals, "unknown error") // nolint:errcheck - failpoint.Disable("github.com/pingcap/ticdc/dm/dumpling/dumpUnitProcessWithError") + failpoint.Disable("github.com/pingcap/tiflow/dm/dumpling/dumpUnitProcessWithError") // cancel - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/dumpling/dumpUnitProcessCancel", `return("unknown error")`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/dumpling/dumpUnitProcessCancel", `return("unknown error")`), IsNil) // nolint:errcheck - defer failpoint.Disable("github.com/pingcap/ticdc/dm/dumpling/dumpUnitProcessCancel") - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/dumpling/dumpUnitProcessForever", `return(true)`), IsNil) + defer failpoint.Disable("github.com/pingcap/tiflow/dm/dumpling/dumpUnitProcessCancel") + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/dumpling/dumpUnitProcessForever", `return(true)`), IsNil) //nolint:errcheck - defer failpoint.Disable("github.com/pingcap/ticdc/dm/dumpling/dumpUnitProcessForever") + defer failpoint.Disable("github.com/pingcap/tiflow/dm/dumpling/dumpUnitProcessForever") ctx2, cancel2 := context.WithTimeout(context.Background(), 3*time.Second) defer cancel2() diff --git a/dm/dumpling/metrics.go b/dm/dumpling/metrics.go index be1cf40fc01..154728b98cd 100644 --- a/dm/dumpling/metrics.go +++ b/dm/dumpling/metrics.go @@ -19,7 +19,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/metricsproxy" + "github.com/pingcap/tiflow/dm/pkg/metricsproxy" ) // should alert. diff --git a/dm/dumpling/util.go b/dm/dumpling/util.go index a69ef652bea..a73ac959f51 100644 --- a/dm/dumpling/util.go +++ b/dm/dumpling/util.go @@ -22,8 +22,8 @@ import ( "github.com/pingcap/tidb/dumpling/export" "github.com/spf13/pflag" - dutils "github.com/pingcap/ticdc/dm/pkg/dumpling" - "github.com/pingcap/ticdc/dm/pkg/log" + dutils "github.com/pingcap/tiflow/dm/pkg/dumpling" + "github.com/pingcap/tiflow/dm/pkg/log" ) // ParseArgLikeBash parses list arguments like bash, which helps us to run diff --git a/dm/dumpling/util_test.go b/dm/dumpling/util_test.go index 67e90123c98..22b5927f28c 100644 --- a/dm/dumpling/util_test.go +++ b/dm/dumpling/util_test.go @@ -23,9 +23,9 @@ import ( "github.com/pingcap/tidb-tools/pkg/filter" tfilter "github.com/pingcap/tidb-tools/pkg/table-filter" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/conn" - "github.com/pingcap/ticdc/dm/pkg/log" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/pkg/log" ) func (d *testDumplingSuite) TestParseArgs(c *C) { diff --git a/dm/loader/checkpoint.go b/dm/loader/checkpoint.go index c3f380ce1b7..8d5541bbace 100644 --- a/dm/loader/checkpoint.go +++ b/dm/loader/checkpoint.go @@ -20,12 +20,12 @@ import ( "sync" "time" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/conn" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/cputil" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/cputil" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" "github.com/pingcap/tidb-tools/pkg/dbutil" "go.uber.org/zap" diff --git a/dm/loader/checkpoint_test.go b/dm/loader/checkpoint_test.go index 043bcc47a0c..976d984bcdc 100644 --- a/dm/loader/checkpoint_test.go +++ b/dm/loader/checkpoint_test.go @@ -21,10 +21,10 @@ import ( "github.com/DATA-DOG/go-sqlmock" . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/conn" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/cputil" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/cputil" ) var _ = Suite(&testCheckPointSuite{}) diff --git a/dm/loader/convert_data.go b/dm/loader/convert_data.go index fcbdf368626..f97e841778f 100644 --- a/dm/loader/convert_data.go +++ b/dm/loader/convert_data.go @@ -22,10 +22,10 @@ import ( "strings" "unsafe" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - parserpkg "github.com/pingcap/ticdc/dm/pkg/parser" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + parserpkg "github.com/pingcap/tiflow/dm/pkg/parser" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" "github.com/pingcap/errors" cm "github.com/pingcap/tidb-tools/pkg/column-mapping" diff --git a/dm/loader/convert_data_test.go b/dm/loader/convert_data_test.go index 36e62c76a42..14b6cdb4dc4 100644 --- a/dm/loader/convert_data_test.go +++ b/dm/loader/convert_data_test.go @@ -17,7 +17,7 @@ import ( cm "github.com/pingcap/tidb-tools/pkg/column-mapping" router "github.com/pingcap/tidb-tools/pkg/table-router" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" . "github.com/pingcap/check" ) diff --git a/dm/loader/db.go b/dm/loader/db.go index d588c6a3626..5be41bba1bc 100644 --- a/dm/loader/db.go +++ b/dm/loader/db.go @@ -25,13 +25,13 @@ import ( tmysql "github.com/pingcap/tidb/parser/mysql" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/conn" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/retry" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/retry" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // DBConn represents a live DB connection diff --git a/dm/loader/lightning.go b/dm/loader/lightning.go index e71fd4038b5..f92278f4c41 100644 --- a/dm/loader/lightning.go +++ b/dm/loader/lightning.go @@ -29,14 +29,14 @@ import ( "go.uber.org/atomic" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/dm/unit" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/conn" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/unit" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/utils" ) const ( diff --git a/dm/loader/loader.go b/dm/loader/loader.go index 2b80fba6ac7..ae03d03876c 100644 --- a/dm/loader/loader.go +++ b/dm/loader/loader.go @@ -28,15 +28,15 @@ import ( "go.etcd.io/etcd/clientv3" "golang.org/x/sync/errgroup" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/dm/unit" - "github.com/pingcap/ticdc/dm/pkg/conn" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - fr "github.com/pingcap/ticdc/dm/pkg/func-rollback" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/unit" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + fr "github.com/pingcap/tiflow/dm/pkg/func-rollback" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" "github.com/pingcap/errors" "github.com/pingcap/failpoint" diff --git a/dm/loader/metrics.go b/dm/loader/metrics.go index 66c6a4c1a8d..7e00b854c58 100644 --- a/dm/loader/metrics.go +++ b/dm/loader/metrics.go @@ -16,7 +16,7 @@ package loader import ( "github.com/prometheus/client_golang/prometheus" - "github.com/pingcap/ticdc/dm/pkg/metricsproxy" + "github.com/pingcap/tiflow/dm/pkg/metricsproxy" ) var ( diff --git a/dm/loader/status.go b/dm/loader/status.go index 33a619ea30f..56f8ffcb7d4 100644 --- a/dm/loader/status.go +++ b/dm/loader/status.go @@ -18,8 +18,8 @@ import ( "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/binlog" ) // Status implements Unit.Status. diff --git a/dm/loader/status_test.go b/dm/loader/status_test.go index f756258dcc4..f6476c53b80 100644 --- a/dm/loader/status_test.go +++ b/dm/loader/status_test.go @@ -19,8 +19,8 @@ import ( . "github.com/pingcap/check" "go.uber.org/atomic" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/log" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/log" ) func (*testLoaderSuite) TestConcurrentStatus(c *C) { diff --git a/dm/loader/util.go b/dm/loader/util.go index e1830317843..523daae7b39 100644 --- a/dm/loader/util.go +++ b/dm/loader/util.go @@ -24,12 +24,12 @@ import ( "go.etcd.io/etcd/clientv3" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/dumpling" - "github.com/pingcap/ticdc/dm/pkg/ha" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/dumpling" + "github.com/pingcap/tiflow/dm/pkg/ha" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // SQLReplace works like strings.Replace but only supports one replacement. diff --git a/dm/openapi/fixtures/task.go b/dm/openapi/fixtures/task.go index 83c6ffb44fd..dd749ec6459 100644 --- a/dm/openapi/fixtures/task.go +++ b/dm/openapi/fixtures/task.go @@ -16,7 +16,7 @@ package fixtures import ( "encoding/json" - "github.com/pingcap/ticdc/dm/openapi" + "github.com/pingcap/tiflow/dm/openapi" ) var ( diff --git a/dm/openapi/task.go b/dm/openapi/task.go index 4b1b0963d0f..c24f60bccc7 100644 --- a/dm/openapi/task.go +++ b/dm/openapi/task.go @@ -14,7 +14,7 @@ package openapi import ( - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) var defaultMetaSchema = "dm_meta" diff --git a/dm/openapi/task_test.go b/dm/openapi/task_test.go index c6458742722..e916ece4b1d 100644 --- a/dm/openapi/task_test.go +++ b/dm/openapi/task_test.go @@ -18,7 +18,7 @@ import ( "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) var _ = check.Suite(&taskSuite{}) diff --git a/dm/pkg/backoff/backoff.go b/dm/pkg/backoff/backoff.go index 30dd74b20a8..6f7d01b4e66 100644 --- a/dm/pkg/backoff/backoff.go +++ b/dm/pkg/backoff/backoff.go @@ -18,7 +18,7 @@ import ( "math/rand" "time" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // Backoff is an exponential counter, it starts from `Min` duration, and after diff --git a/dm/pkg/backoff/backoff_test.go b/dm/pkg/backoff/backoff_test.go index fd0e0cf62f0..0e8110f8121 100644 --- a/dm/pkg/backoff/backoff_test.go +++ b/dm/pkg/backoff/backoff_test.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) func TestT(t *testing.T) { diff --git a/dm/pkg/binlog/event/common.go b/dm/pkg/binlog/event/common.go index 5f291f248ec..c9280c89e6c 100644 --- a/dm/pkg/binlog/event/common.go +++ b/dm/pkg/binlog/event/common.go @@ -20,8 +20,8 @@ import ( gmysql "github.com/go-mysql-org/go-mysql/mysql" "github.com/go-mysql-org/go-mysql/replication" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // DDLDMLResult represents a binlog event result for generated DDL/DML. diff --git a/dm/pkg/binlog/event/common_test.go b/dm/pkg/binlog/event/common_test.go index 1945cf67546..ab584675607 100644 --- a/dm/pkg/binlog/event/common_test.go +++ b/dm/pkg/binlog/event/common_test.go @@ -22,7 +22,7 @@ import ( "github.com/go-mysql-org/go-mysql/replication" . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/gtid" ) var _ = Suite(&testCommonSuite{}) diff --git a/dm/pkg/binlog/event/ddl.go b/dm/pkg/binlog/event/ddl.go index 0c331773b8e..7eed3b4ea92 100644 --- a/dm/pkg/binlog/event/ddl.go +++ b/dm/pkg/binlog/event/ddl.go @@ -20,8 +20,8 @@ import ( "github.com/go-mysql-org/go-mysql/replication" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // GenCreateDatabaseEvents generates binlog events for `CREATE DATABASE`. diff --git a/dm/pkg/binlog/event/ddl_test.go b/dm/pkg/binlog/event/ddl_test.go index 41503b4b80d..40c37227037 100644 --- a/dm/pkg/binlog/event/ddl_test.go +++ b/dm/pkg/binlog/event/ddl_test.go @@ -20,7 +20,7 @@ import ( gmysql "github.com/go-mysql-org/go-mysql/mysql" . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/gtid" ) var _ = Suite(&testDDLSuite{}) diff --git a/dm/pkg/binlog/event/dml.go b/dm/pkg/binlog/event/dml.go index b2c2acc14de..71259ce33ec 100644 --- a/dm/pkg/binlog/event/dml.go +++ b/dm/pkg/binlog/event/dml.go @@ -19,8 +19,8 @@ import ( "github.com/go-mysql-org/go-mysql/replication" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // DMLData represents data used to generate events for DML statements. diff --git a/dm/pkg/binlog/event/dml_test.go b/dm/pkg/binlog/event/dml_test.go index ec540ea11a8..a18a0516f3d 100644 --- a/dm/pkg/binlog/event/dml_test.go +++ b/dm/pkg/binlog/event/dml_test.go @@ -20,7 +20,7 @@ import ( "github.com/go-mysql-org/go-mysql/replication" . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/gtid" ) var _ = Suite(&testDMLSuite{}) diff --git a/dm/pkg/binlog/event/event.go b/dm/pkg/binlog/event/event.go index d2178016d49..63e0bf9bfb9 100644 --- a/dm/pkg/binlog/event/event.go +++ b/dm/pkg/binlog/event/event.go @@ -23,8 +23,8 @@ import ( gmysql "github.com/go-mysql-org/go-mysql/mysql" "github.com/go-mysql-org/go-mysql/replication" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // flags used in RowsEvent. diff --git a/dm/pkg/binlog/event/event_test.go b/dm/pkg/binlog/event/event_test.go index b9c9177e6d9..7f849d5ee6c 100644 --- a/dm/pkg/binlog/event/event_test.go +++ b/dm/pkg/binlog/event/event_test.go @@ -27,7 +27,7 @@ import ( "github.com/go-mysql-org/go-mysql/replication" . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/gtid" ) var _ = Suite(&testEventSuite{}) diff --git a/dm/pkg/binlog/event/generator.go b/dm/pkg/binlog/event/generator.go index adcd6bb4a95..431ff3a97d9 100644 --- a/dm/pkg/binlog/event/generator.go +++ b/dm/pkg/binlog/event/generator.go @@ -17,8 +17,8 @@ import ( gmysql "github.com/go-mysql-org/go-mysql/mysql" "github.com/go-mysql-org/go-mysql/replication" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // Generator represents a binlog events generator. diff --git a/dm/pkg/binlog/event/generator_test.go b/dm/pkg/binlog/event/generator_test.go index 973decf180d..bf7ce941279 100644 --- a/dm/pkg/binlog/event/generator_test.go +++ b/dm/pkg/binlog/event/generator_test.go @@ -22,7 +22,7 @@ import ( "github.com/go-mysql-org/go-mysql/replication" . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/gtid" ) var _ = Suite(&testGeneratorSuite{}) diff --git a/dm/pkg/binlog/event/helper.go b/dm/pkg/binlog/event/helper.go index d1294ce82d9..ec36c0adfd7 100644 --- a/dm/pkg/binlog/event/helper.go +++ b/dm/pkg/binlog/event/helper.go @@ -17,8 +17,8 @@ import ( gmysql "github.com/go-mysql-org/go-mysql/mysql" "github.com/go-mysql-org/go-mysql/replication" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // GTIDsFromPreviousGTIDsEvent get GTID set from a PreviousGTIDsEvent. diff --git a/dm/pkg/binlog/event/helper_test.go b/dm/pkg/binlog/event/helper_test.go index 34fbaec9be1..750da640774 100644 --- a/dm/pkg/binlog/event/helper_test.go +++ b/dm/pkg/binlog/event/helper_test.go @@ -20,8 +20,8 @@ import ( "github.com/go-mysql-org/go-mysql/replication" . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/terror" ) var _ = Suite(&testHelperSuite{}) diff --git a/dm/pkg/binlog/event/sid_mysql.go b/dm/pkg/binlog/event/sid_mysql.go index abe99fa28e1..ff2f334eb4e 100644 --- a/dm/pkg/binlog/event/sid_mysql.go +++ b/dm/pkg/binlog/event/sid_mysql.go @@ -21,7 +21,7 @@ import ( "github.com/go-mysql-org/go-mysql/replication" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // SID represents a SERVER_UUID in GTIDEvent/PrevGTIDEvent. diff --git a/dm/pkg/binlog/event/util.go b/dm/pkg/binlog/event/util.go index 9e3a29e3897..2e83fde673d 100644 --- a/dm/pkg/binlog/event/util.go +++ b/dm/pkg/binlog/event/util.go @@ -31,7 +31,7 @@ import ( "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // encodeTableMapColumnMeta generates the column_meta_def according to the column_type_def. diff --git a/dm/pkg/binlog/event/util_test.go b/dm/pkg/binlog/event/util_test.go index 76916a8c5d6..9bb657af40c 100644 --- a/dm/pkg/binlog/event/util_test.go +++ b/dm/pkg/binlog/event/util_test.go @@ -21,7 +21,7 @@ import ( . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) var _ = Suite(&testUtilSuite{}) diff --git a/dm/pkg/binlog/filename.go b/dm/pkg/binlog/filename.go index 6bd6b4a9f24..a6d02d3a483 100644 --- a/dm/pkg/binlog/filename.go +++ b/dm/pkg/binlog/filename.go @@ -18,7 +18,7 @@ import ( "strconv" "strings" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) const ( diff --git a/dm/pkg/binlog/position.go b/dm/pkg/binlog/position.go index 91c8fc6ac9a..7691f3cb73b 100644 --- a/dm/pkg/binlog/position.go +++ b/dm/pkg/binlog/position.go @@ -21,10 +21,10 @@ import ( gmysql "github.com/go-mysql-org/go-mysql/mysql" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) const ( diff --git a/dm/pkg/binlog/position_test.go b/dm/pkg/binlog/position_test.go index d43b2af875c..715451718fb 100644 --- a/dm/pkg/binlog/position_test.go +++ b/dm/pkg/binlog/position_test.go @@ -19,7 +19,7 @@ import ( gmysql "github.com/go-mysql-org/go-mysql/mysql" . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/gtid" ) var _ = Suite(&testPositionSuite{}) diff --git a/dm/pkg/binlog/reader/file.go b/dm/pkg/binlog/reader/file.go index f5f06ae0123..bef16862573 100644 --- a/dm/pkg/binlog/reader/file.go +++ b/dm/pkg/binlog/reader/file.go @@ -28,10 +28,10 @@ import ( "go.uber.org/atomic" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/binlog/common" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/binlog/common" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // FileReader is a binlog event reader which reads binlog events from a file. diff --git a/dm/pkg/binlog/reader/file_test.go b/dm/pkg/binlog/reader/file_test.go index 1b0018f8b64..9685c7ed982 100644 --- a/dm/pkg/binlog/reader/file_test.go +++ b/dm/pkg/binlog/reader/file_test.go @@ -26,10 +26,10 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/dm/pkg/binlog/common" - "github.com/pingcap/ticdc/dm/pkg/binlog/event" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/binlog/common" + "github.com/pingcap/tiflow/dm/pkg/binlog/event" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/terror" ) var _ = Suite(&testFileReaderSuite{}) diff --git a/dm/pkg/binlog/reader/mock.go b/dm/pkg/binlog/reader/mock.go index 7b650d60f52..94ab389abde 100644 --- a/dm/pkg/binlog/reader/mock.go +++ b/dm/pkg/binlog/reader/mock.go @@ -19,7 +19,7 @@ import ( gmysql "github.com/go-mysql-org/go-mysql/mysql" "github.com/go-mysql-org/go-mysql/replication" - "github.com/pingcap/ticdc/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/gtid" ) // MockReader is a binlog event reader which read binlog events from an input channel. diff --git a/dm/pkg/binlog/reader/reader.go b/dm/pkg/binlog/reader/reader.go index 7202aba81d7..0b131425bc7 100644 --- a/dm/pkg/binlog/reader/reader.go +++ b/dm/pkg/binlog/reader/reader.go @@ -19,7 +19,7 @@ import ( gmysql "github.com/go-mysql-org/go-mysql/mysql" "github.com/go-mysql-org/go-mysql/replication" - "github.com/pingcap/ticdc/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/gtid" ) // Reader is a binlog event reader, it may read binlog events from a TCP stream, binlog files or any other in-memory buffer. diff --git a/dm/pkg/binlog/reader/tcp.go b/dm/pkg/binlog/reader/tcp.go index 7a27770f231..be44d0aaafa 100644 --- a/dm/pkg/binlog/reader/tcp.go +++ b/dm/pkg/binlog/reader/tcp.go @@ -29,11 +29,11 @@ import ( "github.com/pingcap/failpoint" - "github.com/pingcap/ticdc/dm/pkg/binlog/common" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/binlog/common" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) var customID int64 diff --git a/dm/pkg/binlog/reader/tcp_test.go b/dm/pkg/binlog/reader/tcp_test.go index 9c94ea93185..9891263c969 100644 --- a/dm/pkg/binlog/reader/tcp_test.go +++ b/dm/pkg/binlog/reader/tcp_test.go @@ -24,8 +24,8 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/failpoint" - "github.com/pingcap/ticdc/dm/pkg/binlog/common" - "github.com/pingcap/ticdc/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/binlog/common" + "github.com/pingcap/tiflow/dm/pkg/gtid" ) var ( @@ -41,19 +41,19 @@ func TestSuite(t *testing.T) { type testTCPReaderSuite struct{} func (t *testTCPReaderSuite) SetUpSuite(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/pkg/binlog/reader/MockTCPReaderStartSyncByPos", "return(true)"), IsNil) - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/pkg/binlog/reader/MockTCPReaderStartSyncByGTID", "return(true)"), IsNil) - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/pkg/binlog/reader/MockTCPReaderClose", "return(true)"), IsNil) - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/pkg/binlog/reader/MockTCPReaderGetEvent", "return(true)"), IsNil) - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/pkg/binlog/reader/MockTCPReaderStatus", "return(true)"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/pkg/binlog/reader/MockTCPReaderStartSyncByPos", "return(true)"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/pkg/binlog/reader/MockTCPReaderStartSyncByGTID", "return(true)"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/pkg/binlog/reader/MockTCPReaderClose", "return(true)"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/pkg/binlog/reader/MockTCPReaderGetEvent", "return(true)"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/pkg/binlog/reader/MockTCPReaderStatus", "return(true)"), IsNil) } func (t *testTCPReaderSuite) TearDownSuite(c *C) { - c.Assert(failpoint.Disable("github.com/pingcap/ticdc/dm/pkg/binlog/reader/MockTCPReaderStartSyncByPos"), IsNil) - c.Assert(failpoint.Disable("github.com/pingcap/ticdc/dm/pkg/binlog/reader/MockTCPReaderStartSyncByGTID"), IsNil) - c.Assert(failpoint.Disable("github.com/pingcap/ticdc/dm/pkg/binlog/reader/MockTCPReaderClose"), IsNil) - c.Assert(failpoint.Disable("github.com/pingcap/ticdc/dm/pkg/binlog/reader/MockTCPReaderGetEvent"), IsNil) - c.Assert(failpoint.Disable("github.com/pingcap/ticdc/dm/pkg/binlog/reader/MockTCPReaderStatus"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tiflow/dm/pkg/binlog/reader/MockTCPReaderStartSyncByPos"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tiflow/dm/pkg/binlog/reader/MockTCPReaderStartSyncByGTID"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tiflow/dm/pkg/binlog/reader/MockTCPReaderClose"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tiflow/dm/pkg/binlog/reader/MockTCPReaderGetEvent"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tiflow/dm/pkg/binlog/reader/MockTCPReaderStatus"), IsNil) } func (t *testTCPReaderSuite) TestSyncPos(c *C) { diff --git a/dm/pkg/binlog/reader/util.go b/dm/pkg/binlog/reader/util.go index 702ba9d7a17..9d417746523 100644 --- a/dm/pkg/binlog/reader/util.go +++ b/dm/pkg/binlog/reader/util.go @@ -24,12 +24,12 @@ import ( "github.com/pingcap/failpoint" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/binlog/event" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" - "github.com/pingcap/ticdc/dm/relay/common" + "github.com/pingcap/tiflow/dm/pkg/binlog/event" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/relay/common" ) // GetGTIDsForPosFromStreamer tries to get GTID sets for the specified binlog position (for the corresponding txn) from a Streamer. diff --git a/dm/pkg/binlog/status.go b/dm/pkg/binlog/status.go index 36c5ee6b0f5..0c00f142c98 100644 --- a/dm/pkg/binlog/status.go +++ b/dm/pkg/binlog/status.go @@ -20,7 +20,7 @@ import ( gmysql "github.com/go-mysql-org/go-mysql/mysql" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // in MySQL, we can set `max_binlog_size` to control the max size of a binlog file. diff --git a/dm/pkg/binlog/writer/file.go b/dm/pkg/binlog/writer/file.go index 8295e89d223..a0834e82859 100644 --- a/dm/pkg/binlog/writer/file.go +++ b/dm/pkg/binlog/writer/file.go @@ -22,9 +22,9 @@ import ( "go.uber.org/atomic" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/binlog/common" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/binlog/common" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // FileWriter is a binlog event writer which writes binlog events to a file. diff --git a/dm/pkg/binlog/writer/file_test.go b/dm/pkg/binlog/writer/file_test.go index 3b8ba49fb23..fbdb9d4035e 100644 --- a/dm/pkg/binlog/writer/file_test.go +++ b/dm/pkg/binlog/writer/file_test.go @@ -23,8 +23,8 @@ import ( . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/pkg/binlog/common" - "github.com/pingcap/ticdc/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/binlog/common" + "github.com/pingcap/tiflow/dm/pkg/log" ) func TestSuite(t *testing.T) { diff --git a/dm/pkg/conn/baseconn.go b/dm/pkg/conn/baseconn.go index 95641069d3e..34f62c877d4 100644 --- a/dm/pkg/conn/baseconn.go +++ b/dm/pkg/conn/baseconn.go @@ -25,12 +25,12 @@ import ( "github.com/pingcap/failpoint" "go.uber.org/zap" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/metricsproxy" - "github.com/pingcap/ticdc/dm/pkg/retry" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/metricsproxy" + "github.com/pingcap/tiflow/dm/pkg/retry" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // BaseConn is the basic connection we use in dm diff --git a/dm/pkg/conn/baseconn_test.go b/dm/pkg/conn/baseconn_test.go index e6b70bf90ce..67948010ef8 100644 --- a/dm/pkg/conn/baseconn_test.go +++ b/dm/pkg/conn/baseconn_test.go @@ -18,10 +18,10 @@ import ( "strings" "testing" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/metricsproxy" - "github.com/pingcap/ticdc/dm/pkg/retry" - "github.com/pingcap/ticdc/dm/pkg/terror" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/metricsproxy" + "github.com/pingcap/tiflow/dm/pkg/retry" + "github.com/pingcap/tiflow/dm/pkg/terror" "github.com/DATA-DOG/go-sqlmock" . "github.com/pingcap/check" diff --git a/dm/pkg/conn/basedb.go b/dm/pkg/conn/basedb.go index de3566afe7b..250adb6aaeb 100644 --- a/dm/pkg/conn/basedb.go +++ b/dm/pkg/conn/basedb.go @@ -26,10 +26,10 @@ import ( "github.com/DATA-DOG/go-sqlmock" "github.com/pingcap/failpoint" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/retry" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/retry" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" "github.com/go-sql-driver/mysql" toolutils "github.com/pingcap/tidb-tools/pkg/utils" diff --git a/dm/pkg/conn/basedb_test.go b/dm/pkg/conn/basedb_test.go index c6053a0bcc3..92860cde982 100644 --- a/dm/pkg/conn/basedb_test.go +++ b/dm/pkg/conn/basedb_test.go @@ -19,8 +19,8 @@ import ( sqlmock "github.com/DATA-DOG/go-sqlmock" - "github.com/pingcap/ticdc/dm/dm/config" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" + "github.com/pingcap/tiflow/dm/dm/config" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" ) var _ = Suite(&testBaseDBSuite{}) @@ -63,9 +63,9 @@ func (t *testBaseDBSuite) TestGetBaseConn(c *C) { } func (t *testBaseDBSuite) TestFailDBPing(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/pkg/conn/failDBPing", "return"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/pkg/conn/failDBPing", "return"), IsNil) //nolint:errcheck - defer failpoint.Disable("github.com/pingcap/ticdc/dm/pkg/conn/failDBPing") + defer failpoint.Disable("github.com/pingcap/tiflow/dm/pkg/conn/failDBPing") cfg := &config.DBConfig{User: "root", Host: "127.0.0.1", Port: 3306} cfg.Adjust() diff --git a/dm/pkg/conn/mockdb.go b/dm/pkg/conn/mockdb.go index 7c1e95af018..912d018462b 100644 --- a/dm/pkg/conn/mockdb.go +++ b/dm/pkg/conn/mockdb.go @@ -19,7 +19,7 @@ import ( "github.com/DATA-DOG/go-sqlmock" check "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/config" ) type mockDBProvider struct { diff --git a/dm/pkg/conn/utils.go b/dm/pkg/conn/utils.go index 277fe573a00..0bc5a5226a3 100644 --- a/dm/pkg/conn/utils.go +++ b/dm/pkg/conn/utils.go @@ -19,7 +19,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb-tools/pkg/dbutil" - "github.com/pingcap/ticdc/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/config" ) // FetchTimeZoneSetting fetch target db global time_zone setting. diff --git a/dm/pkg/conn/utils_test.go b/dm/pkg/conn/utils_test.go index 93c208593f7..b366fb2517d 100644 --- a/dm/pkg/conn/utils_test.go +++ b/dm/pkg/conn/utils_test.go @@ -18,7 +18,7 @@ import ( . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/config" ) var _ = Suite(testUtilSuite{}) diff --git a/dm/pkg/context/context.go b/dm/pkg/context/context.go index 8a673d74071..dfc4c379fd7 100644 --- a/dm/pkg/context/context.go +++ b/dm/pkg/context/context.go @@ -17,7 +17,7 @@ import ( "context" "time" - "github.com/pingcap/ticdc/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/log" ) // Context is used to in dm to record some context field like diff --git a/dm/pkg/dumpling/utils.go b/dm/pkg/dumpling/utils.go index 37bd2300e64..fb8c60dc661 100644 --- a/dm/pkg/dumpling/utils.go +++ b/dm/pkg/dumpling/utils.go @@ -24,9 +24,9 @@ import ( "github.com/docker/go-units" "github.com/go-mysql-org/go-mysql/mysql" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // ParseMetaData parses mydumper's output meta file and returns binlog location. diff --git a/dm/pkg/dumpling/utils_test.go b/dm/pkg/dumpling/utils_test.go index 5563a80c3a8..7dd864519bb 100644 --- a/dm/pkg/dumpling/utils_test.go +++ b/dm/pkg/dumpling/utils_test.go @@ -20,8 +20,8 @@ import ( "github.com/go-mysql-org/go-mysql/mysql" . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/terror" ) var _ = Suite(&testSuite{}) diff --git a/dm/pkg/election/election.go b/dm/pkg/election/election.go index 588c469aa79..f5cef51e0d3 100644 --- a/dm/pkg/election/election.go +++ b/dm/pkg/election/election.go @@ -29,8 +29,8 @@ import ( "go.uber.org/atomic" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) const ( diff --git a/dm/pkg/election/election_test.go b/dm/pkg/election/election_test.go index 428d2cef3c9..bac257439b6 100644 --- a/dm/pkg/election/election_test.go +++ b/dm/pkg/election/election_test.go @@ -27,10 +27,10 @@ import ( "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/embed" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) var _ = SerialSuites(&testElectionSuite{}) @@ -110,9 +110,9 @@ func testElection2After1(t *testElectionSuite, c *C, normalExit bool) { ctx1, cancel1 := context.WithCancel(context.Background()) defer cancel1() if !normalExit { - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/pkg/election/mockCampaignLoopExitedAbnormally", `return()`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/pkg/election/mockCampaignLoopExitedAbnormally", `return()`), IsNil) //nolint:errcheck - defer failpoint.Disable("github.com/pingcap/ticdc/dm/pkg/election/mockCampaignLoopExitedAbnormally") + defer failpoint.Disable("github.com/pingcap/tiflow/dm/pkg/election/mockCampaignLoopExitedAbnormally") } e1, err := NewElection(ctx1, cli, sessionTTL, key, ID1, addr1, t.notifyBlockTime) c.Assert(err, IsNil) @@ -131,7 +131,7 @@ func testElection2After1(t *testElectionSuite, c *C, normalExit bool) { c.Assert(leaderID, Equals, e1.ID()) c.Assert(leaderAddr, Equals, addr1) if !normalExit { - c.Assert(failpoint.Disable("github.com/pingcap/ticdc/dm/pkg/election/mockCampaignLoopExitedAbnormally"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tiflow/dm/pkg/election/mockCampaignLoopExitedAbnormally"), IsNil) } // start e2 diff --git a/dm/pkg/encrypt/encrypt.go b/dm/pkg/encrypt/encrypt.go index 87520e20d59..a7023d874fb 100644 --- a/dm/pkg/encrypt/encrypt.go +++ b/dm/pkg/encrypt/encrypt.go @@ -20,7 +20,7 @@ import ( "crypto/rand" "encoding/hex" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) var ( diff --git a/dm/pkg/etcdutil/etcdutil.go b/dm/pkg/etcdutil/etcdutil.go index 2e6ffd058d6..2907bbe9695 100644 --- a/dm/pkg/etcdutil/etcdutil.go +++ b/dm/pkg/etcdutil/etcdutil.go @@ -26,9 +26,9 @@ import ( v3rpc "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" "go.uber.org/zap" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/retry" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/retry" ) const ( diff --git a/dm/pkg/etcdutil/etcdutil_test.go b/dm/pkg/etcdutil/etcdutil_test.go index bcc9c6c5f12..0d8afe3aa96 100644 --- a/dm/pkg/etcdutil/etcdutil_test.go +++ b/dm/pkg/etcdutil/etcdutil_test.go @@ -29,8 +29,8 @@ import ( "go.etcd.io/etcd/etcdserver/etcdserverpb" "go.etcd.io/etcd/integration" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) var _ = Suite(&testEtcdUtilSuite{}) @@ -239,9 +239,9 @@ func (t *testEtcdUtilSuite) testDoOpsInOneTxnWithRetry(c *C) { c.Assert(resp.Responses, HasLen, 2) // enable failpoint - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/pkg/etcdutil/ErrNoSpace", `3*return()`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/pkg/etcdutil/ErrNoSpace", `3*return()`), IsNil) //nolint:errcheck - defer failpoint.Disable("github.com/pingcap/ticdc/dm/pkg/etcdutil/ErrNoSpace") + defer failpoint.Disable("github.com/pingcap/tiflow/dm/pkg/etcdutil/ErrNoSpace") // put again resp, rev2, err = DoOpsInOneCmpsTxnWithRetry(cli, []clientv3.Cmp{clientv3util.KeyMissing(key1), clientv3util.KeyMissing(key2)}, []clientv3.Op{ diff --git a/dm/pkg/func-rollback/rollback.go b/dm/pkg/func-rollback/rollback.go index 4c7a8e8e321..3f5935eb6e9 100644 --- a/dm/pkg/func-rollback/rollback.go +++ b/dm/pkg/func-rollback/rollback.go @@ -16,7 +16,7 @@ package rollback import ( "sync" - "github.com/pingcap/ticdc/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/log" "go.uber.org/zap" ) diff --git a/dm/pkg/gtid/gtid.go b/dm/pkg/gtid/gtid.go index b125ddec2a8..d1bd972083b 100644 --- a/dm/pkg/gtid/gtid.go +++ b/dm/pkg/gtid/gtid.go @@ -17,7 +17,7 @@ import ( "github.com/go-mysql-org/go-mysql/mysql" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // Set provide gtid operations for syncer. diff --git a/dm/pkg/gtid/gtid_test.go b/dm/pkg/gtid/gtid_test.go index cb7c88a2223..fbe731bdaa8 100644 --- a/dm/pkg/gtid/gtid_test.go +++ b/dm/pkg/gtid/gtid_test.go @@ -20,7 +20,7 @@ import ( "github.com/go-mysql-org/go-mysql/mysql" . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) var _ = Suite(&testGTIDSuite{}) diff --git a/dm/pkg/ha/bound.go b/dm/pkg/ha/bound.go index db209dae42c..1a6da3bd7af 100644 --- a/dm/pkg/ha/bound.go +++ b/dm/pkg/ha/bound.go @@ -23,11 +23,11 @@ import ( "go.etcd.io/etcd/mvcc/mvccpb" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) const ( diff --git a/dm/pkg/ha/bound_test.go b/dm/pkg/ha/bound_test.go index caa17786c49..ec689fa2f4a 100644 --- a/dm/pkg/ha/bound_test.go +++ b/dm/pkg/ha/bound_test.go @@ -17,7 +17,7 @@ import ( "context" "time" - "github.com/pingcap/ticdc/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/config" . "github.com/pingcap/check" ) diff --git a/dm/pkg/ha/keepalive.go b/dm/pkg/ha/keepalive.go index 590c6663319..ada13ba86da 100644 --- a/dm/pkg/ha/keepalive.go +++ b/dm/pkg/ha/keepalive.go @@ -23,9 +23,9 @@ import ( "go.etcd.io/etcd/mvcc/mvccpb" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" - "github.com/pingcap/ticdc/dm/pkg/log" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/pkg/log" ) var ( diff --git a/dm/pkg/ha/keepalive_test.go b/dm/pkg/ha/keepalive_test.go index b170b2600a4..3cbabc701b3 100644 --- a/dm/pkg/ha/keepalive_test.go +++ b/dm/pkg/ha/keepalive_test.go @@ -20,7 +20,7 @@ import ( "sync/atomic" "time" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/utils" . "github.com/pingcap/check" ) diff --git a/dm/pkg/ha/load_task.go b/dm/pkg/ha/load_task.go index 4f90935c38a..4d019600470 100644 --- a/dm/pkg/ha/load_task.go +++ b/dm/pkg/ha/load_task.go @@ -21,8 +21,8 @@ import ( "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/mvcc/mvccpb" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" ) // LoadTask uses to watch load worker events. diff --git a/dm/pkg/ha/ops.go b/dm/pkg/ha/ops.go index 4fc3c2956c5..2cdbbe8844c 100644 --- a/dm/pkg/ha/ops.go +++ b/dm/pkg/ha/ops.go @@ -17,8 +17,8 @@ import ( "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/mvcc/mvccpb" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" ) // PutRelayStageRelayConfigSourceBound puts the following data in one txn. diff --git a/dm/pkg/ha/ops_test.go b/dm/pkg/ha/ops_test.go index b23dd5f227e..d0730879f85 100644 --- a/dm/pkg/ha/ops_test.go +++ b/dm/pkg/ha/ops_test.go @@ -16,8 +16,8 @@ package ha import ( . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" ) func (t *testForEtcd) TestOpsEtcd(c *C) { diff --git a/dm/pkg/ha/relay.go b/dm/pkg/ha/relay.go index 937e1beea18..3b9bd029a60 100644 --- a/dm/pkg/ha/relay.go +++ b/dm/pkg/ha/relay.go @@ -21,11 +21,11 @@ import ( "go.etcd.io/etcd/mvcc/mvccpb" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // RelaySource represents the bound relationship between the DM-worker instance and its upstream relay source. diff --git a/dm/pkg/ha/relay_test.go b/dm/pkg/ha/relay_test.go index 3fdac9186b9..8fdd781b59c 100644 --- a/dm/pkg/ha/relay_test.go +++ b/dm/pkg/ha/relay_test.go @@ -14,7 +14,7 @@ package ha import ( - "github.com/pingcap/ticdc/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/config" . "github.com/pingcap/check" ) diff --git a/dm/pkg/ha/source.go b/dm/pkg/ha/source.go index 7cbdce48663..5224e5d04bb 100644 --- a/dm/pkg/ha/source.go +++ b/dm/pkg/ha/source.go @@ -19,10 +19,10 @@ import ( "github.com/pingcap/failpoint" "go.etcd.io/etcd/clientv3" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // PutSourceCfg puts the config of the upstream source into etcd. diff --git a/dm/pkg/ha/source_test.go b/dm/pkg/ha/source_test.go index dcc2fe96c0b..2dfd3120b21 100644 --- a/dm/pkg/ha/source_test.go +++ b/dm/pkg/ha/source_test.go @@ -25,7 +25,7 @@ import ( "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/integration" - "github.com/pingcap/ticdc/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/config" ) const ( diff --git a/dm/pkg/ha/stage.go b/dm/pkg/ha/stage.go index 1d3451af9dd..781de9667bf 100644 --- a/dm/pkg/ha/stage.go +++ b/dm/pkg/ha/stage.go @@ -21,12 +21,12 @@ import ( "go.etcd.io/etcd/mvcc/mvccpb" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // Stage represents the running stage for a relay or subtask. diff --git a/dm/pkg/ha/stage_test.go b/dm/pkg/ha/stage_test.go index 3b1b9a3d2c1..a92645bf530 100644 --- a/dm/pkg/ha/stage_test.go +++ b/dm/pkg/ha/stage_test.go @@ -19,8 +19,8 @@ import ( . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" ) func (t *testForEtcd) TestStageJSON(c *C) { diff --git a/dm/pkg/ha/subtask.go b/dm/pkg/ha/subtask.go index afae7c11eb4..660a487bd87 100644 --- a/dm/pkg/ha/subtask.go +++ b/dm/pkg/ha/subtask.go @@ -18,10 +18,10 @@ import ( "go.etcd.io/etcd/clientv3" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // GetSubTaskCfg gets the subtask config of the specified source and task name. diff --git a/dm/pkg/ha/subtask_test.go b/dm/pkg/ha/subtask_test.go index 28548b43bf5..eb1e9f567e3 100644 --- a/dm/pkg/ha/subtask_test.go +++ b/dm/pkg/ha/subtask_test.go @@ -18,7 +18,7 @@ import ( . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/config" ) const ( diff --git a/dm/pkg/ha/worker.go b/dm/pkg/ha/worker.go index 13171c349b8..e28f7663f19 100644 --- a/dm/pkg/ha/worker.go +++ b/dm/pkg/ha/worker.go @@ -19,8 +19,8 @@ import ( "go.etcd.io/etcd/clientv3" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" ) // WorkerInfo represents the node information of the DM-worker. diff --git a/dm/pkg/log/log.go b/dm/pkg/log/log.go index ba2482562c9..98920087430 100644 --- a/dm/pkg/log/log.go +++ b/dm/pkg/log/log.go @@ -24,8 +24,8 @@ import ( "go.uber.org/zap" "go.uber.org/zap/zapcore" - "github.com/pingcap/ticdc/dm/pkg/helper" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/helper" + "github.com/pingcap/tiflow/dm/pkg/terror" ) const ( diff --git a/dm/pkg/parser/common.go b/dm/pkg/parser/common.go index 7f32b718915..a8f40a4c9b5 100644 --- a/dm/pkg/parser/common.go +++ b/dm/pkg/parser/common.go @@ -16,9 +16,9 @@ package parser import ( "bytes" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" "github.com/pingcap/tidb-tools/pkg/filter" "github.com/pingcap/tidb/parser" diff --git a/dm/pkg/parser/common_test.go b/dm/pkg/parser/common_test.go index 8450490ae38..ccdaf6737aa 100644 --- a/dm/pkg/parser/common_test.go +++ b/dm/pkg/parser/common_test.go @@ -20,8 +20,8 @@ import ( "github.com/pingcap/tidb-tools/pkg/filter" "github.com/pingcap/tidb/parser" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) var _ = Suite(&testParserSuite{}) diff --git a/dm/pkg/retry/errors.go b/dm/pkg/retry/errors.go index 3bbbed36fb1..849329262c1 100644 --- a/dm/pkg/retry/errors.go +++ b/dm/pkg/retry/errors.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/errors" tmysql "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // some error reference: https://docs.pingcap.com/tidb/stable/tidb-limitations#limitations-on-a-single-table diff --git a/dm/pkg/retry/strategy.go b/dm/pkg/retry/strategy.go index 01838de8bd2..bbaceb65fb6 100644 --- a/dm/pkg/retry/strategy.go +++ b/dm/pkg/retry/strategy.go @@ -16,7 +16,7 @@ package retry import ( "time" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" ) // backoffStrategy represents enum of retry wait interval. diff --git a/dm/pkg/retry/strategy_test.go b/dm/pkg/retry/strategy_test.go index 3ca8f700805..eb087900b7f 100644 --- a/dm/pkg/retry/strategy_test.go +++ b/dm/pkg/retry/strategy_test.go @@ -18,8 +18,8 @@ import ( "testing" "time" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/terror" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/terror" . "github.com/pingcap/check" "github.com/pingcap/tidb-tools/pkg/dbutil" diff --git a/dm/pkg/schema/tracker.go b/dm/pkg/schema/tracker.go index 27dd4f55361..2f7d3624091 100644 --- a/dm/pkg/schema/tracker.go +++ b/dm/pkg/schema/tracker.go @@ -33,10 +33,10 @@ import ( "github.com/pingcap/tidb/store/mockstore" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/conn" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/log" - dmterror "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/log" + dmterror "github.com/pingcap/tiflow/dm/pkg/terror" ) const ( diff --git a/dm/pkg/schema/tracker_test.go b/dm/pkg/schema/tracker_test.go index b8262e30da8..621fdf48780 100644 --- a/dm/pkg/schema/tracker_test.go +++ b/dm/pkg/schema/tracker_test.go @@ -29,8 +29,8 @@ import ( "github.com/pingcap/tidb/parser/model" "go.uber.org/zap/zapcore" - "github.com/pingcap/ticdc/dm/pkg/conn" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/pkg/terror" ) func Test(t *testing.T) { diff --git a/dm/pkg/shardddl/optimism/column.go b/dm/pkg/shardddl/optimism/column.go index b7c869f422f..bddf728b841 100644 --- a/dm/pkg/shardddl/optimism/column.go +++ b/dm/pkg/shardddl/optimism/column.go @@ -18,8 +18,8 @@ import ( "go.etcd.io/etcd/clientv3" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" ) // GetAllDroppedColumns gets the all partially dropped columns. diff --git a/dm/pkg/shardddl/optimism/info.go b/dm/pkg/shardddl/optimism/info.go index ae703290c14..eb0179a7a58 100644 --- a/dm/pkg/shardddl/optimism/info.go +++ b/dm/pkg/shardddl/optimism/info.go @@ -23,9 +23,9 @@ import ( "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/mvcc/mvccpb" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" - "github.com/pingcap/ticdc/dm/pkg/log" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/pkg/log" ) // TODO: much of the code in optimistic mode is very similar to pessimistic mode, we can try to combine them together. diff --git a/dm/pkg/shardddl/optimism/info_test.go b/dm/pkg/shardddl/optimism/info_test.go index a552ac939a0..63df2a422a1 100644 --- a/dm/pkg/shardddl/optimism/info_test.go +++ b/dm/pkg/shardddl/optimism/info_test.go @@ -30,8 +30,8 @@ import ( "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/integration" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" ) var etcdTestCli *clientv3.Client diff --git a/dm/pkg/shardddl/optimism/keeper.go b/dm/pkg/shardddl/optimism/keeper.go index 8fc57437784..b88493e78f1 100644 --- a/dm/pkg/shardddl/optimism/keeper.go +++ b/dm/pkg/shardddl/optimism/keeper.go @@ -20,8 +20,8 @@ import ( "github.com/pingcap/tidb-tools/pkg/schemacmp" "go.etcd.io/etcd/clientv3" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // LockKeeper used to keep and handle DDL lock conveniently. diff --git a/dm/pkg/shardddl/optimism/keeper_test.go b/dm/pkg/shardddl/optimism/keeper_test.go index adbba568ce4..6b686872c66 100644 --- a/dm/pkg/shardddl/optimism/keeper_test.go +++ b/dm/pkg/shardddl/optimism/keeper_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/tidb/util/mock" "go.etcd.io/etcd/integration" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/utils" ) type testKeeper struct{} diff --git a/dm/pkg/shardddl/optimism/lock.go b/dm/pkg/shardddl/optimism/lock.go index 55d65f4cf5b..7b9740ce126 100644 --- a/dm/pkg/shardddl/optimism/lock.go +++ b/dm/pkg/shardddl/optimism/lock.go @@ -23,9 +23,9 @@ import ( "go.etcd.io/etcd/clientv3" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/master/metrics" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/master/metrics" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // DropColumnStage represents whether drop column done for a sharding table. diff --git a/dm/pkg/shardddl/optimism/lock_test.go b/dm/pkg/shardddl/optimism/lock_test.go index 7987e9d35d2..1ed4ebc64a0 100644 --- a/dm/pkg/shardddl/optimism/lock_test.go +++ b/dm/pkg/shardddl/optimism/lock_test.go @@ -24,9 +24,9 @@ import ( "github.com/pingcap/tidb/util/mock" "go.etcd.io/etcd/integration" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) type testLock struct{} diff --git a/dm/pkg/shardddl/optimism/operation.go b/dm/pkg/shardddl/optimism/operation.go index 624c3a9f689..99dcd7dda52 100644 --- a/dm/pkg/shardddl/optimism/operation.go +++ b/dm/pkg/shardddl/optimism/operation.go @@ -21,8 +21,8 @@ import ( "go.etcd.io/etcd/clientv3/clientv3util" "go.etcd.io/etcd/mvcc/mvccpb" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" ) // ConflictStage represents the current shard DDL conflict stage in the optimistic mode. diff --git a/dm/pkg/shardddl/optimism/ops.go b/dm/pkg/shardddl/optimism/ops.go index dc727789445..7cd13bace1d 100644 --- a/dm/pkg/shardddl/optimism/ops.go +++ b/dm/pkg/shardddl/optimism/ops.go @@ -16,9 +16,9 @@ package optimism import ( "go.etcd.io/etcd/clientv3" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // PutSourceTablesInfo puts source tables and a shard DDL info. diff --git a/dm/pkg/shardddl/optimism/schema.go b/dm/pkg/shardddl/optimism/schema.go index 7f4008eef2c..c4f4167e4df 100644 --- a/dm/pkg/shardddl/optimism/schema.go +++ b/dm/pkg/shardddl/optimism/schema.go @@ -20,8 +20,8 @@ import ( "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/clientv3/clientv3util" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" ) // InitSchema represents the initial schema (schema before the lock constructed) of a merged table. diff --git a/dm/pkg/shardddl/optimism/table.go b/dm/pkg/shardddl/optimism/table.go index fd624e000ff..0951720fd77 100644 --- a/dm/pkg/shardddl/optimism/table.go +++ b/dm/pkg/shardddl/optimism/table.go @@ -21,8 +21,8 @@ import ( "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/mvcc/mvccpb" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" ) // SourceTables represents the upstream/sources tables for a data migration **subtask**. diff --git a/dm/pkg/shardddl/pessimism/info.go b/dm/pkg/shardddl/pessimism/info.go index 18bab7f2aec..066ab514500 100644 --- a/dm/pkg/shardddl/pessimism/info.go +++ b/dm/pkg/shardddl/pessimism/info.go @@ -21,9 +21,9 @@ import ( "go.etcd.io/etcd/clientv3/clientv3util" "go.etcd.io/etcd/mvcc/mvccpb" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // Info represents the shard DDL information. diff --git a/dm/pkg/shardddl/pessimism/info_test.go b/dm/pkg/shardddl/pessimism/info_test.go index bf4642d643b..43427197d3e 100644 --- a/dm/pkg/shardddl/pessimism/info_test.go +++ b/dm/pkg/shardddl/pessimism/info_test.go @@ -25,8 +25,8 @@ import ( "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/integration" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/pkg/utils" ) var etcdTestCli *clientv3.Client diff --git a/dm/pkg/shardddl/pessimism/keeper.go b/dm/pkg/shardddl/pessimism/keeper.go index 215c5661de1..3af1e6e88f7 100644 --- a/dm/pkg/shardddl/pessimism/keeper.go +++ b/dm/pkg/shardddl/pessimism/keeper.go @@ -16,8 +16,8 @@ package pessimism import ( "sync" - "github.com/pingcap/ticdc/dm/dm/master/metrics" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/master/metrics" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // LockKeeper used to keep and handle DDL lock conveniently. diff --git a/dm/pkg/shardddl/pessimism/lock.go b/dm/pkg/shardddl/pessimism/lock.go index b1485242ea7..3061e0c9f46 100644 --- a/dm/pkg/shardddl/pessimism/lock.go +++ b/dm/pkg/shardddl/pessimism/lock.go @@ -16,9 +16,9 @@ package pessimism import ( "sync" - "github.com/pingcap/ticdc/dm/dm/master/metrics" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/master/metrics" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // Lock represents the shard DDL lock in memory. diff --git a/dm/pkg/shardddl/pessimism/lock_test.go b/dm/pkg/shardddl/pessimism/lock_test.go index 2abf36f7671..322d0c37527 100644 --- a/dm/pkg/shardddl/pessimism/lock_test.go +++ b/dm/pkg/shardddl/pessimism/lock_test.go @@ -16,7 +16,7 @@ package pessimism import ( . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) type testLock struct{} diff --git a/dm/pkg/shardddl/pessimism/operation.go b/dm/pkg/shardddl/pessimism/operation.go index 9988dfcc163..7a7fc786c7d 100644 --- a/dm/pkg/shardddl/pessimism/operation.go +++ b/dm/pkg/shardddl/pessimism/operation.go @@ -21,8 +21,8 @@ import ( "go.etcd.io/etcd/clientv3/clientv3util" "go.etcd.io/etcd/mvcc/mvccpb" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" ) // Operation represents a shard DDL coordinate operation. diff --git a/dm/pkg/shardddl/pessimism/ops.go b/dm/pkg/shardddl/pessimism/ops.go index 359650ffa01..643afb06df5 100644 --- a/dm/pkg/shardddl/pessimism/ops.go +++ b/dm/pkg/shardddl/pessimism/ops.go @@ -18,8 +18,8 @@ import ( "go.etcd.io/etcd/clientv3" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" ) // TODO(csuzhangxc): assign terror code before merged into the master branch. diff --git a/dm/pkg/streamer/file.go b/dm/pkg/streamer/file.go index 8a426075efd..395e4fde9ef 100644 --- a/dm/pkg/streamer/file.go +++ b/dm/pkg/streamer/file.go @@ -24,10 +24,10 @@ import ( "github.com/BurntSushi/toml" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // FileCmp is a compare condition used when collecting binlog files. diff --git a/dm/pkg/streamer/file_test.go b/dm/pkg/streamer/file_test.go index f02a89043ae..a50df8d143e 100644 --- a/dm/pkg/streamer/file_test.go +++ b/dm/pkg/streamer/file_test.go @@ -26,8 +26,8 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) var _ = Suite(&testFileSuite{}) diff --git a/dm/pkg/streamer/hub.go b/dm/pkg/streamer/hub.go index 0099c61cde9..1886baa5a64 100644 --- a/dm/pkg/streamer/hub.go +++ b/dm/pkg/streamer/hub.go @@ -18,9 +18,9 @@ import ( "strings" "sync" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) var ( diff --git a/dm/pkg/streamer/reader.go b/dm/pkg/streamer/reader.go index f078f9e0d24..48a92f1d5e9 100644 --- a/dm/pkg/streamer/reader.go +++ b/dm/pkg/streamer/reader.go @@ -29,14 +29,14 @@ import ( "github.com/pingcap/errors" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/binlog/event" - "github.com/pingcap/ticdc/dm/pkg/binlog/reader" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/binlog/event" + "github.com/pingcap/tiflow/dm/pkg/binlog/reader" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // Meta represents binlog meta information in relay.meta. diff --git a/dm/pkg/streamer/reader_test.go b/dm/pkg/streamer/reader_test.go index 5e7942ac509..1a70eb09682 100644 --- a/dm/pkg/streamer/reader_test.go +++ b/dm/pkg/streamer/reader_test.go @@ -36,11 +36,11 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" - "github.com/pingcap/ticdc/dm/pkg/binlog/event" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/binlog/event" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) var parseFileTimeout = 10 * time.Second @@ -61,11 +61,11 @@ func (t *testReaderSuite) SetUpSuite(c *C) { t.lastPos = 0 t.lastGTID, err = gtid.ParserGTID(gmysql.MySQLFlavor, "ba8f633f-1f15-11eb-b1c7-0242ac110002:0") c.Assert(err, IsNil) - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/pkg/streamer/SetHeartbeatInterval", "return(10000)"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/pkg/streamer/SetHeartbeatInterval", "return(10000)"), IsNil) } func (t *testReaderSuite) TearDownSuite(c *C) { - c.Assert(failpoint.Disable("github.com/pingcap/ticdc/dm/pkg/streamer/SetHeartbeatInterval"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tiflow/dm/pkg/streamer/SetHeartbeatInterval"), IsNil) } func (t *testReaderSuite) TestParseFileBase(c *C) { diff --git a/dm/pkg/streamer/streamer.go b/dm/pkg/streamer/streamer.go index 1398508300e..6a791eadbf4 100644 --- a/dm/pkg/streamer/streamer.go +++ b/dm/pkg/streamer/streamer.go @@ -17,11 +17,11 @@ import ( "context" "time" - "github.com/pingcap/ticdc/dm/pkg/binlog/common" - "github.com/pingcap/ticdc/dm/pkg/binlog/event" - "github.com/pingcap/ticdc/dm/pkg/binlog/reader" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/binlog/common" + "github.com/pingcap/tiflow/dm/pkg/binlog/event" + "github.com/pingcap/tiflow/dm/pkg/binlog/reader" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" "github.com/go-mysql-org/go-mysql/replication" "github.com/pingcap/failpoint" diff --git a/dm/pkg/streamer/streamer_test.go b/dm/pkg/streamer/streamer_test.go index 279d874fb92..67a89123fcc 100644 --- a/dm/pkg/streamer/streamer_test.go +++ b/dm/pkg/streamer/streamer_test.go @@ -22,8 +22,8 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" - "github.com/pingcap/ticdc/dm/pkg/binlog/event" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/binlog/event" + "github.com/pingcap/tiflow/dm/pkg/terror" ) var _ = Suite(&testStreamerSuite{}) @@ -31,9 +31,9 @@ var _ = Suite(&testStreamerSuite{}) type testStreamerSuite struct{} func (t *testStreamerSuite) TestStreamer(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/pkg/streamer/SetHeartbeatInterval", "return(10000)"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/pkg/streamer/SetHeartbeatInterval", "return(10000)"), IsNil) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/ticdc/dm/pkg/streamer/SetHeartbeatInterval"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tiflow/dm/pkg/streamer/SetHeartbeatInterval"), IsNil) }() ctx, cancel := context.WithTimeout(context.Background(), time.Second) @@ -105,9 +105,9 @@ func (t *testStreamerSuite) TestStreamer(c *C) { } func (t *testStreamerSuite) TestHeartbeat(c *C) { - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/pkg/streamer/SetHeartbeatInterval", "return(1)"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/pkg/streamer/SetHeartbeatInterval", "return(1)"), IsNil) defer func() { - c.Assert(failpoint.Disable("github.com/pingcap/ticdc/dm/pkg/streamer/SetHeartbeatInterval"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tiflow/dm/pkg/streamer/SetHeartbeatInterval"), IsNil) }() ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second) diff --git a/dm/pkg/streamer/util.go b/dm/pkg/streamer/util.go index bdd29073e19..84dd0b5ffb3 100644 --- a/dm/pkg/streamer/util.go +++ b/dm/pkg/streamer/util.go @@ -19,8 +19,8 @@ import ( "github.com/pingcap/errors" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // getNextUUID gets (the nextUUID and its suffix) after the current UUID. diff --git a/dm/pkg/upgrade/upgrade.go b/dm/pkg/upgrade/upgrade.go index b4e9ce336a3..79fe79aa2e8 100644 --- a/dm/pkg/upgrade/upgrade.go +++ b/dm/pkg/upgrade/upgrade.go @@ -22,14 +22,14 @@ import ( "go.etcd.io/etcd/clientv3" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/conn" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/cputil" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/cputil" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // upgrades records all functions used to upgrade from one version to the later version. diff --git a/dm/pkg/upgrade/upgrade_test.go b/dm/pkg/upgrade/upgrade_test.go index 9b62e75df89..0504ccd79b5 100644 --- a/dm/pkg/upgrade/upgrade_test.go +++ b/dm/pkg/upgrade/upgrade_test.go @@ -22,7 +22,7 @@ import ( "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/integration" - "github.com/pingcap/ticdc/dm/dm/common" + "github.com/pingcap/tiflow/dm/dm/common" ) var ( diff --git a/dm/pkg/upgrade/version.go b/dm/pkg/upgrade/version.go index 2d203f4edf6..6d6d6f249a8 100644 --- a/dm/pkg/upgrade/version.go +++ b/dm/pkg/upgrade/version.go @@ -18,10 +18,10 @@ import ( "go.etcd.io/etcd/clientv3" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/pkg/etcdutil" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/pkg/etcdutil" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) const ( diff --git a/dm/pkg/utils/common.go b/dm/pkg/utils/common.go index b7cd414cc85..6f88836fd40 100644 --- a/dm/pkg/utils/common.go +++ b/dm/pkg/utils/common.go @@ -34,8 +34,8 @@ import ( "github.com/shopspring/decimal" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // TrimCtrlChars returns a slice of the string s with all leading diff --git a/dm/pkg/utils/db.go b/dm/pkg/utils/db.go index 456271a0c56..b9519b28a48 100644 --- a/dm/pkg/utils/db.go +++ b/dm/pkg/utils/db.go @@ -34,9 +34,9 @@ import ( tmysql "github.com/pingcap/tidb/parser/mysql" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) const ( diff --git a/dm/pkg/utils/db_test.go b/dm/pkg/utils/db_test.go index 2811c90c8a2..3687f1eaf82 100644 --- a/dm/pkg/utils/db_test.go +++ b/dm/pkg/utils/db_test.go @@ -26,7 +26,7 @@ import ( "github.com/pingcap/errors" tmysql "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/ticdc/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/gtid" ) var _ = Suite(&testDBSuite{}) diff --git a/dm/pkg/utils/encrypt.go b/dm/pkg/utils/encrypt.go index fc409cfcd6b..db7078656ae 100644 --- a/dm/pkg/utils/encrypt.go +++ b/dm/pkg/utils/encrypt.go @@ -16,8 +16,8 @@ package utils import ( "encoding/base64" - "github.com/pingcap/ticdc/dm/pkg/encrypt" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/encrypt" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // Encrypt tries to encrypt plaintext to base64 encoded ciphertext. diff --git a/dm/pkg/utils/encrypt_test.go b/dm/pkg/utils/encrypt_test.go index 0019edd1e91..e179b5bf114 100644 --- a/dm/pkg/utils/encrypt_test.go +++ b/dm/pkg/utils/encrypt_test.go @@ -18,7 +18,7 @@ import ( . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) var _ = Suite(&testEncryptSuite{}) diff --git a/dm/pkg/utils/file.go b/dm/pkg/utils/file.go index c8dba92bd0a..27d08a8b645 100644 --- a/dm/pkg/utils/file.go +++ b/dm/pkg/utils/file.go @@ -22,8 +22,8 @@ import ( "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // IsFileExists checks if file exists. diff --git a/dm/pkg/utils/file_test.go b/dm/pkg/utils/file_test.go index 88047d2c840..bdcb49f5c81 100644 --- a/dm/pkg/utils/file_test.go +++ b/dm/pkg/utils/file_test.go @@ -19,7 +19,7 @@ import ( . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) var _ = Suite(&testFileSuite{}) diff --git a/dm/pkg/utils/printer.go b/dm/pkg/utils/printer.go index e4e4d6603f2..69bafe312b5 100644 --- a/dm/pkg/utils/printer.go +++ b/dm/pkg/utils/printer.go @@ -16,7 +16,7 @@ package utils import ( "fmt" - "github.com/pingcap/ticdc/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/log" "go.uber.org/zap" "go.uber.org/zap/zapcore" diff --git a/dm/pkg/utils/printer_test.go b/dm/pkg/utils/printer_test.go index 8b39a04dd60..509aa650848 100644 --- a/dm/pkg/utils/printer_test.go +++ b/dm/pkg/utils/printer_test.go @@ -17,7 +17,7 @@ import ( capturer "github.com/kami-zh/go-capturer" . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/log" ) var _ = Suite(&testPrinterSuite{}) diff --git a/dm/pkg/utils/relay.go b/dm/pkg/utils/relay.go index b010e443fd4..ca9a9fa01c4 100644 --- a/dm/pkg/utils/relay.go +++ b/dm/pkg/utils/relay.go @@ -24,7 +24,7 @@ import ( "github.com/go-mysql-org/go-mysql/replication" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // not support to config yet. diff --git a/dm/pkg/utils/storage_unix.go b/dm/pkg/utils/storage_unix.go index 7b455ed5d19..4b255765e3f 100644 --- a/dm/pkg/utils/storage_unix.go +++ b/dm/pkg/utils/storage_unix.go @@ -21,7 +21,7 @@ import ( "golang.org/x/sys/unix" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // GetStorageSize gets storage's capacity and available size. diff --git a/dm/pkg/utils/storage_windows.go b/dm/pkg/utils/storage_windows.go index 4fb069a8085..7ea6b4185c2 100644 --- a/dm/pkg/utils/storage_windows.go +++ b/dm/pkg/utils/storage_windows.go @@ -20,7 +20,7 @@ import ( "syscall" "unsafe" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) var ( diff --git a/dm/pkg/utils/time.go b/dm/pkg/utils/time.go index 7991f1e6f37..d333d21a285 100644 --- a/dm/pkg/utils/time.go +++ b/dm/pkg/utils/time.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/tidb-tools/pkg/dbutil" "github.com/pingcap/tidb/types" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // ParseTimeZone parse the time zone location by name or offset diff --git a/dm/pkg/utils/util.go b/dm/pkg/utils/util.go index 619609cee12..aba654f1133 100644 --- a/dm/pkg/utils/util.go +++ b/dm/pkg/utils/util.go @@ -30,9 +30,9 @@ import ( "go.uber.org/zap" "golang.org/x/net/http/httpproxy" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) var ( diff --git a/dm/pkg/v1dbschema/schema.go b/dm/pkg/v1dbschema/schema.go index 6f1e32a9580..999f902f8dc 100644 --- a/dm/pkg/v1dbschema/schema.go +++ b/dm/pkg/v1dbschema/schema.go @@ -26,16 +26,16 @@ import ( "github.com/pingcap/tidb/errno" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/binlog/reader" - "github.com/pingcap/ticdc/dm/pkg/conn" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/cputil" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/binlog/reader" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/cputil" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // UpdateSchema updates the DB schema from v1.0.x to v2.0.x, including: diff --git a/dm/pkg/v1dbschema/schema_test.go b/dm/pkg/v1dbschema/schema_test.go index f4a10e857c5..bdd0522dfb2 100644 --- a/dm/pkg/v1dbschema/schema_test.go +++ b/dm/pkg/v1dbschema/schema_test.go @@ -24,10 +24,10 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/failpoint" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/conn" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/gtid" ) func TestSuite(t *testing.T) { @@ -101,12 +101,12 @@ func (t *testSchema) TestSchemaV106ToV20x(c *C) { endGS, _ = gtid.ParserGTID(gmysql.MySQLFlavor, "ccb992ad-a557-11ea-ba6a-0242ac140002:1-16") ) - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/pkg/v1dbschema/MockGetGTIDsForPos", `return("ccb992ad-a557-11ea-ba6a-0242ac140002:10-16")`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/pkg/v1dbschema/MockGetGTIDsForPos", `return("ccb992ad-a557-11ea-ba6a-0242ac140002:10-16")`), IsNil) //nolint:errcheck - defer failpoint.Disable("github.com/pingcap/ticdc/dm/pkg/v1dbschema/MockGetGTIDsForPos") - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/pkg/utils/GetGTIDPurged", `return("ccb992ad-a557-11ea-ba6a-0242ac140002:1-9")`), IsNil) + defer failpoint.Disable("github.com/pingcap/tiflow/dm/pkg/v1dbschema/MockGetGTIDsForPos") + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/pkg/utils/GetGTIDPurged", `return("ccb992ad-a557-11ea-ba6a-0242ac140002:1-9")`), IsNil) //nolint:errcheck - defer failpoint.Disable("github.com/pingcap/ticdc/dm/pkg/utils/GetGTIDPurged") + defer failpoint.Disable("github.com/pingcap/tiflow/dm/pkg/utils/GetGTIDPurged") // update schema without GTID enabled. // mock updateSyncerCheckpoint diff --git a/dm/pkg/v1workermeta/api.go b/dm/pkg/v1workermeta/api.go index 2aa8dca1f3a..8af0f361651 100644 --- a/dm/pkg/v1workermeta/api.go +++ b/dm/pkg/v1workermeta/api.go @@ -20,10 +20,10 @@ import ( "github.com/BurntSushi/toml" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // `var` rather than `const` for testing. diff --git a/dm/pkg/v1workermeta/api_test.go b/dm/pkg/v1workermeta/api_test.go index 1fe6cfc13af..1d13a81d3ec 100644 --- a/dm/pkg/v1workermeta/api_test.go +++ b/dm/pkg/v1workermeta/api_test.go @@ -21,9 +21,9 @@ import ( . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) type testAPI struct{} diff --git a/dm/pkg/v1workermeta/db.go b/dm/pkg/v1workermeta/db.go index 5308377ae3c..694f93cd376 100644 --- a/dm/pkg/v1workermeta/db.go +++ b/dm/pkg/v1workermeta/db.go @@ -17,7 +17,7 @@ import ( "github.com/syndtr/goleveldb/leveldb" "github.com/syndtr/goleveldb/leveldb/opt" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // kvConfig is the configuration of goleveldb. diff --git a/dm/pkg/v1workermeta/meta.go b/dm/pkg/v1workermeta/meta.go index f7f1fa53f30..9f3f0f986de 100644 --- a/dm/pkg/v1workermeta/meta.go +++ b/dm/pkg/v1workermeta/meta.go @@ -18,8 +18,8 @@ import ( "github.com/syndtr/goleveldb/leveldb" "github.com/syndtr/goleveldb/leveldb/util" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // meta stores metadata of tasks. diff --git a/dm/pkg/v1workermeta/meta_test.go b/dm/pkg/v1workermeta/meta_test.go index 0cfbfafbf1c..2781b4ed3fb 100644 --- a/dm/pkg/v1workermeta/meta_test.go +++ b/dm/pkg/v1workermeta/meta_test.go @@ -20,9 +20,9 @@ import ( . "github.com/pingcap/check" "github.com/syndtr/goleveldb/leveldb" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/terror" ) func TestSuite(t *testing.T) { diff --git a/dm/relay/common/util.go b/dm/relay/common/util.go index d75c6fc1ac1..87a65e5738c 100644 --- a/dm/relay/common/util.go +++ b/dm/relay/common/util.go @@ -17,8 +17,8 @@ import ( "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" - parserpkg "github.com/pingcap/ticdc/dm/pkg/parser" - "github.com/pingcap/ticdc/dm/pkg/utils" + parserpkg "github.com/pingcap/tiflow/dm/pkg/parser" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // CheckIsDDL checks input SQL whether is a valid DDL statement. diff --git a/dm/relay/config.go b/dm/relay/config.go index 5efac22e0ad..965af3f53e9 100644 --- a/dm/relay/config.go +++ b/dm/relay/config.go @@ -16,9 +16,9 @@ package relay import ( "encoding/json" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/relay/retry" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/relay/retry" ) // Config is the configuration for Relay. diff --git a/dm/relay/meta.go b/dm/relay/meta.go index 1b46f0c293b..c48f2c5ae2b 100644 --- a/dm/relay/meta.go +++ b/dm/relay/meta.go @@ -23,10 +23,10 @@ import ( "github.com/BurntSushi/toml" "github.com/go-mysql-org/go-mysql/mysql" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) var ( diff --git a/dm/relay/meta_test.go b/dm/relay/meta_test.go index e31ef1e8196..56c25f69875 100644 --- a/dm/relay/meta_test.go +++ b/dm/relay/meta_test.go @@ -21,7 +21,7 @@ import ( "github.com/go-mysql-org/go-mysql/mysql" . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/gtid" ) var _ = Suite(&testMetaSuite{}) diff --git a/dm/relay/metrics.go b/dm/relay/metrics.go index 9c670c6f4c6..a7d5753e7bb 100644 --- a/dm/relay/metrics.go +++ b/dm/relay/metrics.go @@ -20,10 +20,10 @@ import ( "github.com/pingcap/failpoint" "github.com/prometheus/client_golang/prometheus" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/metricsproxy" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/metricsproxy" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) var ( diff --git a/dm/relay/purger/file.go b/dm/relay/purger/file.go index 5e8c2cdb4d7..1b2e0779fc4 100644 --- a/dm/relay/purger/file.go +++ b/dm/relay/purger/file.go @@ -20,10 +20,10 @@ import ( "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/streamer" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/streamer" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // subRelayFiles represents relay log files in one sub directory. diff --git a/dm/relay/purger/file_test.go b/dm/relay/purger/file_test.go index c926bd47a35..9675bdc750b 100644 --- a/dm/relay/purger/file_test.go +++ b/dm/relay/purger/file_test.go @@ -20,9 +20,9 @@ import ( . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/streamer" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/streamer" + "github.com/pingcap/tiflow/dm/pkg/utils" ) func (t *testPurgerSuite) TestPurgeRelayFilesBeforeFile(c *C) { diff --git a/dm/relay/purger/purger.go b/dm/relay/purger/purger.go index c869748536b..d7a5efb763b 100644 --- a/dm/relay/purger/purger.go +++ b/dm/relay/purger/purger.go @@ -22,12 +22,12 @@ import ( "go.uber.org/atomic" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/streamer" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/streamer" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // RelayOperator represents an operator for relay log files, like writer, reader. diff --git a/dm/relay/purger/purger_test.go b/dm/relay/purger/purger_test.go index c43def46daa..f3c7e0bf2ed 100644 --- a/dm/relay/purger/purger_test.go +++ b/dm/relay/purger/purger_test.go @@ -24,10 +24,10 @@ import ( . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/streamer" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/streamer" + "github.com/pingcap/tiflow/dm/pkg/utils" ) var _ = Suite(&testPurgerSuite{ diff --git a/dm/relay/purger/strategy.go b/dm/relay/purger/strategy.go index 2ee1e5955f8..dad84cdfe8b 100644 --- a/dm/relay/purger/strategy.go +++ b/dm/relay/purger/strategy.go @@ -13,7 +13,7 @@ package purger -import "github.com/pingcap/ticdc/dm/pkg/streamer" +import "github.com/pingcap/tiflow/dm/pkg/streamer" type strategyType uint32 diff --git a/dm/relay/purger/strategy_filename.go b/dm/relay/purger/strategy_filename.go index 6d346b1139d..aa90d5da005 100644 --- a/dm/relay/purger/strategy_filename.go +++ b/dm/relay/purger/strategy_filename.go @@ -20,10 +20,10 @@ import ( "go.uber.org/atomic" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/streamer" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/streamer" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) var fakeTaskName = strategyFilename.String() diff --git a/dm/relay/purger/strategy_inactive.go b/dm/relay/purger/strategy_inactive.go index 0c404a2b29d..dc08a76a52a 100644 --- a/dm/relay/purger/strategy_inactive.go +++ b/dm/relay/purger/strategy_inactive.go @@ -20,9 +20,9 @@ import ( "go.uber.org/atomic" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/streamer" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/streamer" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // inactiveArgs represents args needed by inactiveStrategy. diff --git a/dm/relay/purger/strategy_space.go b/dm/relay/purger/strategy_space.go index ce281b01a35..e97303dc508 100644 --- a/dm/relay/purger/strategy_space.go +++ b/dm/relay/purger/strategy_space.go @@ -20,10 +20,10 @@ import ( "go.uber.org/atomic" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/streamer" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/streamer" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // spaceArgs represents args needed by spaceStrategy. diff --git a/dm/relay/purger/strategy_time.go b/dm/relay/purger/strategy_time.go index b52f143c37d..9ad03febb2e 100644 --- a/dm/relay/purger/strategy_time.go +++ b/dm/relay/purger/strategy_time.go @@ -21,9 +21,9 @@ import ( "go.uber.org/atomic" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/streamer" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/streamer" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // timeArgs represents args needed by timeStrategy. diff --git a/dm/relay/reader/reader.go b/dm/relay/reader/reader.go index e928ccdd823..11fc5c902af 100644 --- a/dm/relay/reader/reader.go +++ b/dm/relay/reader/reader.go @@ -21,11 +21,11 @@ import ( "github.com/go-mysql-org/go-mysql/replication" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/binlog/common" - br "github.com/pingcap/ticdc/dm/pkg/binlog/reader" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/binlog/common" + br "github.com/pingcap/tiflow/dm/pkg/binlog/reader" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // Result represents a read operation result. diff --git a/dm/relay/reader/reader_test.go b/dm/relay/reader/reader_test.go index e45f1d4eb76..617452d7cbf 100644 --- a/dm/relay/reader/reader_test.go +++ b/dm/relay/reader/reader_test.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/check" "github.com/pingcap/errors" - br "github.com/pingcap/ticdc/dm/pkg/binlog/reader" + br "github.com/pingcap/tiflow/dm/pkg/binlog/reader" ) var _ = check.Suite(&testReaderSuite{}) diff --git a/dm/relay/relay.go b/dm/relay/relay.go index 0a9bfa5da32..5b30835ad60 100644 --- a/dm/relay/relay.go +++ b/dm/relay/relay.go @@ -32,22 +32,22 @@ import ( "go.uber.org/atomic" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/dm/unit" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/binlog/common" - binlogReader "github.com/pingcap/ticdc/dm/pkg/binlog/reader" - "github.com/pingcap/ticdc/dm/pkg/conn" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/log" - pkgstreamer "github.com/pingcap/ticdc/dm/pkg/streamer" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" - "github.com/pingcap/ticdc/dm/relay/reader" - "github.com/pingcap/ticdc/dm/relay/retry" - "github.com/pingcap/ticdc/dm/relay/transformer" - "github.com/pingcap/ticdc/dm/relay/writer" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/unit" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/binlog/common" + binlogReader "github.com/pingcap/tiflow/dm/pkg/binlog/reader" + "github.com/pingcap/tiflow/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/log" + pkgstreamer "github.com/pingcap/tiflow/dm/pkg/streamer" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/relay/reader" + "github.com/pingcap/tiflow/dm/relay/retry" + "github.com/pingcap/tiflow/dm/relay/transformer" + "github.com/pingcap/tiflow/dm/relay/writer" ) // used to fill RelayLogInfo. diff --git a/dm/relay/relay_test.go b/dm/relay/relay_test.go index 6e782e58e56..d9362229f68 100644 --- a/dm/relay/relay_test.go +++ b/dm/relay/relay_test.go @@ -32,16 +32,16 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/parser" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/binlog/event" - "github.com/pingcap/ticdc/dm/pkg/conn" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/utils" - "github.com/pingcap/ticdc/dm/relay/reader" - "github.com/pingcap/ticdc/dm/relay/retry" - "github.com/pingcap/ticdc/dm/relay/transformer" - "github.com/pingcap/ticdc/dm/relay/writer" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/binlog/event" + "github.com/pingcap/tiflow/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/relay/reader" + "github.com/pingcap/tiflow/dm/relay/retry" + "github.com/pingcap/tiflow/dm/relay/transformer" + "github.com/pingcap/tiflow/dm/relay/writer" ) var _ = Suite(&testRelaySuite{}) @@ -168,9 +168,9 @@ func (t *testRelaySuite) TestTryRecoverLatestFile(c *C) { relayCfg = newRelayCfg(c, gmysql.MySQLFlavor) r = NewRelay(relayCfg).(*Relay) ) - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/pkg/utils/GetGTIDPurged", `return("406a3f61-690d-11e7-87c5-6c92bf46f384:1-122")`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/pkg/utils/GetGTIDPurged", `return("406a3f61-690d-11e7-87c5-6c92bf46f384:1-122")`), IsNil) //nolint:errcheck - defer failpoint.Disable("github.com/pingcap/ticdc/dm/pkg/utils/GetGTIDPurged") + defer failpoint.Disable("github.com/pingcap/tiflow/dm/pkg/utils/GetGTIDPurged") cfg := getDBConfigForTest() conn.InitMockDB(c) db, err := conn.DefaultDBProvider.Apply(cfg) @@ -295,9 +295,9 @@ func (t *testRelaySuite) TestTryRecoverMeta(c *C) { f.Close() // recover with empty GTIDs. - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/pkg/utils/GetGTIDPurged", `return("")`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/pkg/utils/GetGTIDPurged", `return("")`), IsNil) //nolint:errcheck - defer failpoint.Disable("github.com/pingcap/ticdc/dm/pkg/utils/GetGTIDPurged") + defer failpoint.Disable("github.com/pingcap/tiflow/dm/pkg/utils/GetGTIDPurged") c.Assert(r.tryRecoverLatestFile(context.Background(), parser2), IsNil) _, latestPos := r.meta.Pos() c.Assert(latestPos, DeepEquals, gmysql.Position{Name: filename, Pos: g.LatestPos}) @@ -595,9 +595,9 @@ func (t *testRelaySuite) TestReSetupMeta(c *C) { mockGetRandomServerID(mockDB) // mock AddGSetWithPurged mockDB.ExpectQuery("select @@GLOBAL.gtid_purged").WillReturnRows(sqlmock.NewRows([]string{"@@GLOBAL.gtid_purged"}).AddRow("")) - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/pkg/binlog/reader/MockGetEmptyPreviousGTIDFromGTIDSet", "return()"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/pkg/binlog/reader/MockGetEmptyPreviousGTIDFromGTIDSet", "return()"), IsNil) //nolint:errcheck - defer failpoint.Disable("github.com/pingcap/ticdc/dm/pkg/binlog/reader/MockGetEmptyPreviousGTIDFromGTIDSet") + defer failpoint.Disable("github.com/pingcap/tiflow/dm/pkg/binlog/reader/MockGetEmptyPreviousGTIDFromGTIDSet") c.Assert(r.reSetupMeta(ctx), IsNil) uuid001 := fmt.Sprintf("%s.000001", uuid) t.verifyMetadata(c, r, uuid001, gmysql.Position{Name: r.cfg.BinLogName, Pos: 4}, emptyGTID.String(), []string{uuid001}) diff --git a/dm/relay/retry/reader.go b/dm/relay/retry/reader.go index b155aaf8b14..353d83ec830 100644 --- a/dm/relay/retry/reader.go +++ b/dm/relay/retry/reader.go @@ -17,9 +17,9 @@ import ( "context" "time" - "github.com/pingcap/ticdc/dm/pkg/backoff" - "github.com/pingcap/ticdc/dm/pkg/retry" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/backoff" + "github.com/pingcap/tiflow/dm/pkg/retry" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // ReaderRetryConfig is the configuration used for binlog reader retry backoff. diff --git a/dm/relay/transformer/transformer.go b/dm/relay/transformer/transformer.go index a9aa0d3c090..9ec7636ad50 100644 --- a/dm/relay/transformer/transformer.go +++ b/dm/relay/transformer/transformer.go @@ -18,7 +18,7 @@ import ( "github.com/go-mysql-org/go-mysql/replication" "github.com/pingcap/tidb/parser" - "github.com/pingcap/ticdc/dm/relay/common" + "github.com/pingcap/tiflow/dm/relay/common" ) const ( diff --git a/dm/relay/transformer/transformer_test.go b/dm/relay/transformer/transformer_test.go index cbf4fbfee8b..a6501773ea0 100644 --- a/dm/relay/transformer/transformer_test.go +++ b/dm/relay/transformer/transformer_test.go @@ -22,8 +22,8 @@ import ( "github.com/pingcap/check" "github.com/pingcap/tidb/parser" - "github.com/pingcap/ticdc/dm/pkg/binlog/event" - "github.com/pingcap/ticdc/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/binlog/event" + "github.com/pingcap/tiflow/dm/pkg/gtid" ) var _ = check.Suite(&testTransformerSuite{}) diff --git a/dm/relay/util.go b/dm/relay/util.go index 0f1b5e67906..9bfce99e985 100644 --- a/dm/relay/util.go +++ b/dm/relay/util.go @@ -18,7 +18,7 @@ import ( "database/sql" "strings" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // isNewServer checks whether is connecting to a new server. diff --git a/dm/relay/util_test.go b/dm/relay/util_test.go index cef391a13b1..5353cadf973 100644 --- a/dm/relay/util_test.go +++ b/dm/relay/util_test.go @@ -21,8 +21,8 @@ import ( gmysql "github.com/go-mysql-org/go-mysql/mysql" . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/pkg/conn" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/pkg/utils" ) var _ = Suite(&testUtilSuite{}) diff --git a/dm/relay/writer/file.go b/dm/relay/writer/file.go index fc444245299..94c91d2588f 100644 --- a/dm/relay/writer/file.go +++ b/dm/relay/writer/file.go @@ -27,13 +27,13 @@ import ( "go.uber.org/atomic" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/binlog/common" - "github.com/pingcap/ticdc/dm/pkg/binlog/event" - bw "github.com/pingcap/ticdc/dm/pkg/binlog/writer" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/binlog/common" + "github.com/pingcap/tiflow/dm/pkg/binlog/event" + bw "github.com/pingcap/tiflow/dm/pkg/binlog/writer" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // FileConfig is the configuration used by the FileWriter. diff --git a/dm/relay/writer/file_test.go b/dm/relay/writer/file_test.go index 685672facb8..cb50424cf2d 100644 --- a/dm/relay/writer/file_test.go +++ b/dm/relay/writer/file_test.go @@ -27,10 +27,10 @@ import ( "github.com/pingcap/check" "github.com/pingcap/tidb/parser" - "github.com/pingcap/ticdc/dm/pkg/binlog/common" - "github.com/pingcap/ticdc/dm/pkg/binlog/event" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/binlog/common" + "github.com/pingcap/tiflow/dm/pkg/binlog/event" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/log" ) var _ = check.Suite(&testFileWriterSuite{}) diff --git a/dm/relay/writer/file_util.go b/dm/relay/writer/file_util.go index c8cf5a01d2d..75338da2570 100644 --- a/dm/relay/writer/file_util.go +++ b/dm/relay/writer/file_util.go @@ -26,11 +26,11 @@ import ( "github.com/google/uuid" "github.com/pingcap/tidb/parser" - "github.com/pingcap/ticdc/dm/pkg/binlog/event" - "github.com/pingcap/ticdc/dm/pkg/binlog/reader" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/relay/common" + "github.com/pingcap/tiflow/dm/pkg/binlog/event" + "github.com/pingcap/tiflow/dm/pkg/binlog/reader" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/relay/common" ) // checkBinlogHeaderExist checks if the file has a binlog file header. diff --git a/dm/relay/writer/file_util_test.go b/dm/relay/writer/file_util_test.go index 7b1d158a827..14e8e6e37cd 100644 --- a/dm/relay/writer/file_util_test.go +++ b/dm/relay/writer/file_util_test.go @@ -27,8 +27,8 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/parser" - "github.com/pingcap/ticdc/dm/pkg/binlog/event" - "github.com/pingcap/ticdc/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/binlog/event" + "github.com/pingcap/tiflow/dm/pkg/gtid" ) var _ = check.Suite(&testFileUtilSuite{}) diff --git a/dm/relay/writer/writer.go b/dm/relay/writer/writer.go index c06939f92ae..d38e768dd96 100644 --- a/dm/relay/writer/writer.go +++ b/dm/relay/writer/writer.go @@ -19,7 +19,7 @@ import ( gmysql "github.com/go-mysql-org/go-mysql/mysql" "github.com/go-mysql-org/go-mysql/replication" - "github.com/pingcap/ticdc/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/gtid" ) const ( diff --git a/dm/syncer/causality.go b/dm/syncer/causality.go index c83b89b091f..24f1282ad44 100644 --- a/dm/syncer/causality.go +++ b/dm/syncer/causality.go @@ -18,8 +18,8 @@ import ( "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/syncer/metrics" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/syncer/metrics" ) // causality provides a simple mechanism to improve the concurrency of SQLs execution under the premise of ensuring correctness. diff --git a/dm/syncer/causality_test.go b/dm/syncer/causality_test.go index f90d73825e0..13e87eddb2d 100644 --- a/dm/syncer/causality_test.go +++ b/dm/syncer/causality_test.go @@ -21,11 +21,11 @@ import ( "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/util/mock" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/binlog" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/binlog" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/utils" ) func (s *testSyncerSuite) TestDetectConflict(c *C) { diff --git a/dm/syncer/checkpoint.go b/dm/syncer/checkpoint.go index 560fe80f778..e1160a9aa22 100644 --- a/dm/syncer/checkpoint.go +++ b/dm/syncer/checkpoint.go @@ -24,18 +24,18 @@ import ( "sync" "time" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/conn" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/cputil" - "github.com/pingcap/ticdc/dm/pkg/dumpling" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/schema" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" - "github.com/pingcap/ticdc/dm/syncer/dbconn" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/cputil" + "github.com/pingcap/tiflow/dm/pkg/dumpling" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/schema" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/syncer/dbconn" "github.com/go-mysql-org/go-mysql/mysql" "github.com/pingcap/failpoint" diff --git a/dm/syncer/checkpoint_test.go b/dm/syncer/checkpoint_test.go index 73ff6307321..4564d630bfd 100644 --- a/dm/syncer/checkpoint_test.go +++ b/dm/syncer/checkpoint_test.go @@ -21,15 +21,15 @@ import ( "path/filepath" "strings" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/conn" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/cputil" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/retry" - "github.com/pingcap/ticdc/dm/pkg/schema" - "github.com/pingcap/ticdc/dm/syncer/dbconn" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/cputil" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/retry" + "github.com/pingcap/tiflow/dm/pkg/schema" + "github.com/pingcap/tiflow/dm/syncer/dbconn" "github.com/DATA-DOG/go-sqlmock" "github.com/go-mysql-org/go-mysql/mysql" diff --git a/dm/syncer/compactor.go b/dm/syncer/compactor.go index f9875e63dac..e8b96dbd2a4 100644 --- a/dm/syncer/compactor.go +++ b/dm/syncer/compactor.go @@ -20,8 +20,8 @@ import ( "github.com/pingcap/tidb-tools/pkg/filter" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/syncer/metrics" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/syncer/metrics" ) // compactor compacts multiple statements into one statement. diff --git a/dm/syncer/compactor_test.go b/dm/syncer/compactor_test.go index ade587c4802..99100e13727 100644 --- a/dm/syncer/compactor_test.go +++ b/dm/syncer/compactor_test.go @@ -24,11 +24,11 @@ import ( "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/util/mock" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/binlog" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/binlog" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // mockExecute mock a kv store. @@ -229,9 +229,9 @@ func (s *testSyncerSuite) TestCompactorSafeMode(c *C) { }, } - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/syncer/SkipFlushCompactor", `return()`), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/syncer/SkipFlushCompactor", `return()`), IsNil) //nolint:errcheck - defer failpoint.Disable("github.com/pingcap/ticdc/dm/syncer/SkipFlushCompactor") + defer failpoint.Disable("github.com/pingcap/tiflow/dm/syncer/SkipFlushCompactor") outCh := compactorWrap(inCh, syncer) diff --git a/dm/syncer/dbconn/db.go b/dm/syncer/dbconn/db.go index 1e5ff3bed2b..379ef4a6de2 100644 --- a/dm/syncer/dbconn/db.go +++ b/dm/syncer/dbconn/db.go @@ -22,14 +22,14 @@ import ( "github.com/pingcap/tidb-tools/pkg/dbutil" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/conn" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/retry" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" - "github.com/pingcap/ticdc/dm/syncer/metrics" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/retry" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/syncer/metrics" ) var retryTimeout = 3 * time.Second diff --git a/dm/syncer/dbconn/upstream_db.go b/dm/syncer/dbconn/upstream_db.go index 4303d198775..b8e704f14ed 100644 --- a/dm/syncer/dbconn/upstream_db.go +++ b/dm/syncer/dbconn/upstream_db.go @@ -23,13 +23,13 @@ import ( tmysql "github.com/pingcap/tidb/parser/mysql" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/conn" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // UpStreamConn connect to upstream DB diff --git a/dm/syncer/dbconn/upstream_db_test.go b/dm/syncer/dbconn/upstream_db_test.go index d20e06f4855..9fd16b519cf 100644 --- a/dm/syncer/dbconn/upstream_db_test.go +++ b/dm/syncer/dbconn/upstream_db_test.go @@ -23,8 +23,8 @@ import ( "github.com/google/uuid" . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/utils" ) var _ = Suite(&testDBSuite{}) diff --git a/dm/syncer/ddl.go b/dm/syncer/ddl.go index 7776e8d4304..f8bc7d66530 100644 --- a/dm/syncer/ddl.go +++ b/dm/syncer/ddl.go @@ -23,11 +23,11 @@ import ( "github.com/pingcap/tidb/parser/ast" "go.uber.org/zap" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - parserpkg "github.com/pingcap/ticdc/dm/pkg/parser" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" - "github.com/pingcap/ticdc/dm/syncer/metrics" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + parserpkg "github.com/pingcap/tiflow/dm/pkg/parser" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/syncer/metrics" ) func parseOneStmt(qec *queryEventContext) (stmt ast.StmtNode, err error) { diff --git a/dm/syncer/ddl_test.go b/dm/syncer/ddl_test.go index 0aca3dd8bbd..a07b5eb4b7e 100644 --- a/dm/syncer/ddl_test.go +++ b/dm/syncer/ddl_test.go @@ -30,13 +30,13 @@ import ( "github.com/pingcap/tidb/parser/ast" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/log" - parserpkg "github.com/pingcap/ticdc/dm/pkg/parser" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" - onlineddl "github.com/pingcap/ticdc/dm/syncer/online-ddl-tools" + "github.com/pingcap/tiflow/dm/dm/config" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/log" + parserpkg "github.com/pingcap/tiflow/dm/pkg/parser" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" + onlineddl "github.com/pingcap/tiflow/dm/syncer/online-ddl-tools" ) var _ = Suite(&testDDLSuite{}) diff --git a/dm/syncer/dml.go b/dm/syncer/dml.go index 9b96f5c0f7d..0323407754f 100644 --- a/dm/syncer/dml.go +++ b/dm/syncer/dml.go @@ -27,8 +27,8 @@ import ( "github.com/pingcap/tidb/parser/types" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // this type is used to generate DML SQL, opType is used to mark type in binlog. diff --git a/dm/syncer/dml_worker.go b/dm/syncer/dml_worker.go index c7ff1f3d405..175df8ce0a6 100644 --- a/dm/syncer/dml_worker.go +++ b/dm/syncer/dml_worker.go @@ -22,12 +22,12 @@ import ( "github.com/pingcap/tidb-tools/pkg/filter" "go.uber.org/zap" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" - "github.com/pingcap/ticdc/dm/syncer/dbconn" - "github.com/pingcap/ticdc/dm/syncer/metrics" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/syncer/dbconn" + "github.com/pingcap/tiflow/dm/syncer/metrics" ) // DMLWorker is used to sync dml. diff --git a/dm/syncer/err-operator/operator.go b/dm/syncer/err-operator/operator.go index 6a68572e325..01e25106009 100644 --- a/dm/syncer/err-operator/operator.go +++ b/dm/syncer/err-operator/operator.go @@ -23,10 +23,10 @@ import ( "github.com/google/uuid" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // Operator contains an operation for specified binlog pos diff --git a/dm/syncer/err-operator/operator_test.go b/dm/syncer/err-operator/operator_test.go index c43beeb2c5f..c2abe9f731b 100644 --- a/dm/syncer/err-operator/operator_test.go +++ b/dm/syncer/err-operator/operator_test.go @@ -20,10 +20,10 @@ import ( "github.com/go-mysql-org/go-mysql/replication" . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) var _ = Suite(&testOperatorSuite{}) diff --git a/dm/syncer/error.go b/dm/syncer/error.go index 3c13248f6e3..a1638b932a5 100644 --- a/dm/syncer/error.go +++ b/dm/syncer/error.go @@ -29,9 +29,9 @@ import ( tmysql "github.com/pingcap/tidb/parser/mysql" "go.uber.org/zap" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/syncer/dbconn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/syncer/dbconn" ) func ignoreDDLError(err error) bool { diff --git a/dm/syncer/error_test.go b/dm/syncer/error_test.go index 424292ff489..a6ca84e91f9 100644 --- a/dm/syncer/error_test.go +++ b/dm/syncer/error_test.go @@ -24,9 +24,9 @@ import ( "github.com/pingcap/tidb/infoschema" tmysql "github.com/pingcap/tidb/parser/mysql" - "github.com/pingcap/ticdc/dm/pkg/conn" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/syncer/dbconn" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/syncer/dbconn" ) func newMysqlErr(number uint16, message string) *mysql.MySQLError { diff --git a/dm/syncer/expr_filter_group.go b/dm/syncer/expr_filter_group.go index ee0904beb62..dee664b62bc 100644 --- a/dm/syncer/expr_filter_group.go +++ b/dm/syncer/expr_filter_group.go @@ -23,9 +23,9 @@ import ( "github.com/pingcap/tidb/util/chunk" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // ExprFilterGroup groups many related fields about expression filter. diff --git a/dm/syncer/expr_filter_group_test.go b/dm/syncer/expr_filter_group_test.go index c86ab6cbb95..b5b8924774f 100644 --- a/dm/syncer/expr_filter_group_test.go +++ b/dm/syncer/expr_filter_group_test.go @@ -19,10 +19,10 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb-tools/pkg/filter" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/schema" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/schema" + "github.com/pingcap/tiflow/dm/pkg/utils" ) func (s *testFilterSuite) TestSkipDMLByExpression(c *C) { diff --git a/dm/syncer/filter.go b/dm/syncer/filter.go index 0021febf393..b224a5ee99c 100644 --- a/dm/syncer/filter.go +++ b/dm/syncer/filter.go @@ -19,9 +19,9 @@ import ( "github.com/pingcap/tidb-tools/pkg/filter" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" - onlineddl "github.com/pingcap/ticdc/dm/syncer/online-ddl-tools" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" + onlineddl "github.com/pingcap/tiflow/dm/syncer/online-ddl-tools" ) // skipQueryEvent if skip by binlog-filter: diff --git a/dm/syncer/filter_test.go b/dm/syncer/filter_test.go index b927b6b3d6a..383362f257b 100644 --- a/dm/syncer/filter_test.go +++ b/dm/syncer/filter_test.go @@ -24,12 +24,12 @@ import ( "github.com/pingcap/tidb-tools/pkg/filter" "github.com/pingcap/tidb/parser" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/conn" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/schema" - "github.com/pingcap/ticdc/dm/pkg/utils" - "github.com/pingcap/ticdc/dm/syncer/dbconn" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/schema" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/syncer/dbconn" ) type testFilterSuite struct { diff --git a/dm/syncer/handle_error.go b/dm/syncer/handle_error.go index e9fb163f0ea..86a481f204f 100644 --- a/dm/syncer/handle_error.go +++ b/dm/syncer/handle_error.go @@ -22,10 +22,10 @@ import ( "github.com/go-mysql-org/go-mysql/replication" "github.com/pingcap/tidb/parser/ast" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/binlog" - parserpkg "github.com/pingcap/ticdc/dm/pkg/parser" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/binlog" + parserpkg "github.com/pingcap/tiflow/dm/pkg/parser" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // HandleError handle error for syncer. diff --git a/dm/syncer/handle_error_test.go b/dm/syncer/handle_error_test.go index 99430545bb9..fd94d01c3a8 100644 --- a/dm/syncer/handle_error_test.go +++ b/dm/syncer/handle_error_test.go @@ -19,9 +19,9 @@ import ( . "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/conn" - "github.com/pingcap/ticdc/dm/syncer/dbconn" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/conn" + "github.com/pingcap/tiflow/dm/syncer/dbconn" ) func (s *testSyncerSuite) TestHandleError(c *C) { diff --git a/dm/syncer/job.go b/dm/syncer/job.go index 927dc91fd65..c907c302d76 100644 --- a/dm/syncer/job.go +++ b/dm/syncer/job.go @@ -20,7 +20,7 @@ import ( "github.com/go-mysql-org/go-mysql/replication" "github.com/pingcap/tidb-tools/pkg/filter" - "github.com/pingcap/ticdc/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/binlog" ) type opType byte diff --git a/dm/syncer/job_test.go b/dm/syncer/job_test.go index 798f62d1054..dcd43ef2324 100644 --- a/dm/syncer/job_test.go +++ b/dm/syncer/job_test.go @@ -19,7 +19,7 @@ import ( "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/util/mock" - "github.com/pingcap/ticdc/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/binlog" ) var _ = Suite(&testJobSuite{}) diff --git a/dm/syncer/metrics/metrics.go b/dm/syncer/metrics/metrics.go index 0e4c1739c61..979571bfd5b 100644 --- a/dm/syncer/metrics/metrics.go +++ b/dm/syncer/metrics/metrics.go @@ -16,7 +16,7 @@ package metrics import ( "github.com/prometheus/client_golang/prometheus" - "github.com/pingcap/ticdc/dm/pkg/metricsproxy" + "github.com/pingcap/tiflow/dm/pkg/metricsproxy" ) // for BinlogEventCost metric stage field. diff --git a/dm/syncer/mode.go b/dm/syncer/mode.go index 097e0ff2a9c..ba57b3722e0 100644 --- a/dm/syncer/mode.go +++ b/dm/syncer/mode.go @@ -19,8 +19,8 @@ import ( "github.com/pingcap/failpoint" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/unit" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" + "github.com/pingcap/tiflow/dm/dm/unit" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" ) func (s *Syncer) enableSafeModeInitializationPhase(tctx *tcontext.Context) { diff --git a/dm/syncer/online-ddl-tools/online_ddl.go b/dm/syncer/online-ddl-tools/online_ddl.go index d485d6f6805..6a27ae93c9e 100644 --- a/dm/syncer/online-ddl-tools/online_ddl.go +++ b/dm/syncer/online-ddl-tools/online_ddl.go @@ -19,14 +19,14 @@ import ( "regexp" "sync" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/conn" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/cputil" - parserpkg "github.com/pingcap/ticdc/dm/pkg/parser" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" - "github.com/pingcap/ticdc/dm/syncer/dbconn" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/cputil" + parserpkg "github.com/pingcap/tiflow/dm/pkg/parser" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/syncer/dbconn" "github.com/pingcap/failpoint" "github.com/pingcap/tidb-tools/pkg/dbutil" diff --git a/dm/syncer/optimist.go b/dm/syncer/optimist.go index ae383a728c6..aa8ac8885c8 100644 --- a/dm/syncer/optimist.go +++ b/dm/syncer/optimist.go @@ -22,9 +22,9 @@ import ( "github.com/pingcap/tidb/parser/model" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/shardddl/optimism" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/shardddl/optimism" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // initOptimisticShardDDL initializes the shard DDL support in the optimistic mode. diff --git a/dm/syncer/relay.go b/dm/syncer/relay.go index b067d41fe64..09ad4a50fda 100644 --- a/dm/syncer/relay.go +++ b/dm/syncer/relay.go @@ -20,11 +20,11 @@ import ( "github.com/go-mysql-org/go-mysql/mysql" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) func (s *Syncer) setInitActiveRelayLog(ctx context.Context) error { diff --git a/dm/syncer/safe-mode/mode.go b/dm/syncer/safe-mode/mode.go index 0adc1fddeb1..6550bdf0727 100644 --- a/dm/syncer/safe-mode/mode.go +++ b/dm/syncer/safe-mode/mode.go @@ -19,9 +19,9 @@ import ( "github.com/pingcap/tidb-tools/pkg/filter" "go.uber.org/zap" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // SafeMode controls whether enable safe-mode through a mechanism similar to reference-count diff --git a/dm/syncer/safe-mode/mode_test.go b/dm/syncer/safe-mode/mode_test.go index c50080e67ab..0ea7e33d59e 100644 --- a/dm/syncer/safe-mode/mode_test.go +++ b/dm/syncer/safe-mode/mode_test.go @@ -19,7 +19,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb-tools/pkg/filter" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" ) var _ = Suite(&testModeSuite{}) diff --git a/dm/syncer/schema.go b/dm/syncer/schema.go index f23b6fa2154..a7d1c2360f5 100644 --- a/dm/syncer/schema.go +++ b/dm/syncer/schema.go @@ -24,12 +24,12 @@ import ( "github.com/pingcap/tidb/parser/model" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/schema" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/schema" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // OperateSchema operates schema for an upstream table. diff --git a/dm/syncer/shardddl/optimist.go b/dm/syncer/shardddl/optimist.go index 6cfd690d5ea..027626d3726 100644 --- a/dm/syncer/shardddl/optimist.go +++ b/dm/syncer/shardddl/optimist.go @@ -21,8 +21,8 @@ import ( "go.etcd.io/etcd/clientv3" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/shardddl/optimism" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/shardddl/optimism" ) // Optimist used to coordinate the shard DDL migration in optimism mode. diff --git a/dm/syncer/shardddl/optimist_test.go b/dm/syncer/shardddl/optimist_test.go index 2e4cbd20c5d..fa616f51bae 100644 --- a/dm/syncer/shardddl/optimist_test.go +++ b/dm/syncer/shardddl/optimist_test.go @@ -25,9 +25,9 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/util/mock" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/shardddl/optimism" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/shardddl/optimism" + "github.com/pingcap/tiflow/dm/pkg/terror" ) type testOptimist struct{} diff --git a/dm/syncer/shardddl/pessimist.go b/dm/syncer/shardddl/pessimist.go index c83a782a78a..4b7d104b68c 100644 --- a/dm/syncer/shardddl/pessimist.go +++ b/dm/syncer/shardddl/pessimist.go @@ -20,9 +20,9 @@ import ( "go.etcd.io/etcd/clientv3" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/shardddl/pessimism" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/shardddl/pessimism" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // Pessimist used to coordinate the shard DDL migration in pessimism mode. diff --git a/dm/syncer/shardddl/pessimist_test.go b/dm/syncer/shardddl/pessimist_test.go index e897277f210..a36450f8aa7 100644 --- a/dm/syncer/shardddl/pessimist_test.go +++ b/dm/syncer/shardddl/pessimist_test.go @@ -22,10 +22,10 @@ import ( "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/integration" - "github.com/pingcap/ticdc/dm/dm/common" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/shardddl/pessimism" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/dm/common" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/shardddl/pessimism" + "github.com/pingcap/tiflow/dm/pkg/terror" ) var etcdTestCli *clientv3.Client diff --git a/dm/syncer/sharding-meta/shardmeta.go b/dm/syncer/sharding-meta/shardmeta.go index b433d764d43..55bc1967f53 100644 --- a/dm/syncer/sharding-meta/shardmeta.go +++ b/dm/syncer/sharding-meta/shardmeta.go @@ -22,11 +22,11 @@ import ( "github.com/pingcap/tidb-tools/pkg/filter" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" ) // DDLItem records ddl information used in sharding sequence organization. diff --git a/dm/syncer/sharding-meta/shardmeta_test.go b/dm/syncer/sharding-meta/shardmeta_test.go index dee9b34e720..d861fad6465 100644 --- a/dm/syncer/sharding-meta/shardmeta_test.go +++ b/dm/syncer/sharding-meta/shardmeta_test.go @@ -20,7 +20,7 @@ import ( "github.com/go-mysql-org/go-mysql/mysql" "github.com/pingcap/check" - "github.com/pingcap/ticdc/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/binlog" ) var _ = check.Suite(&testShardMetaSuite{}) diff --git a/dm/syncer/sharding_group.go b/dm/syncer/sharding_group.go index 447f94efb7b..d330642a9f2 100644 --- a/dm/syncer/sharding_group.go +++ b/dm/syncer/sharding_group.go @@ -74,16 +74,16 @@ import ( "fmt" "sync" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/conn" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/cputil" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" - "github.com/pingcap/ticdc/dm/syncer/dbconn" - shardmeta "github.com/pingcap/ticdc/dm/syncer/sharding-meta" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/cputil" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/syncer/dbconn" + shardmeta "github.com/pingcap/tiflow/dm/syncer/sharding-meta" "github.com/pingcap/tidb-tools/pkg/dbutil" "github.com/pingcap/tidb-tools/pkg/filter" diff --git a/dm/syncer/sharding_group_test.go b/dm/syncer/sharding_group_test.go index 96d5b8b1028..10e54e96ae4 100644 --- a/dm/syncer/sharding_group_test.go +++ b/dm/syncer/sharding_group_test.go @@ -24,16 +24,16 @@ import ( "github.com/pingcap/tidb-tools/pkg/dbutil" "github.com/pingcap/tidb-tools/pkg/filter" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/conn" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/cputil" - "github.com/pingcap/ticdc/dm/pkg/retry" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" - "github.com/pingcap/ticdc/dm/syncer/dbconn" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/cputil" + "github.com/pingcap/tiflow/dm/pkg/retry" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/syncer/dbconn" ) var _ = Suite(&testShardingGroupSuite{}) diff --git a/dm/syncer/status.go b/dm/syncer/status.go index a95efba687e..4d564cf3173 100644 --- a/dm/syncer/status.go +++ b/dm/syncer/status.go @@ -20,11 +20,11 @@ import ( "github.com/pingcap/failpoint" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/syncer/metrics" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/syncer/metrics" ) // Status implements Unit.Status. diff --git a/dm/syncer/status_test.go b/dm/syncer/status_test.go index 97c1d2a3a99..eeff9034ee9 100644 --- a/dm/syncer/status_test.go +++ b/dm/syncer/status_test.go @@ -20,12 +20,12 @@ import ( . "github.com/pingcap/check" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/binlog" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/syncer/shardddl" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/binlog" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/syncer/shardddl" ) var _ = Suite(&statusSuite{}) diff --git a/dm/syncer/streamer_controller.go b/dm/syncer/streamer_controller.go index 063b8950903..06b1b46b744 100644 --- a/dm/syncer/streamer_controller.go +++ b/dm/syncer/streamer_controller.go @@ -25,15 +25,15 @@ import ( "github.com/pingcap/failpoint" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/binlog/common" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/retry" - "github.com/pingcap/ticdc/dm/pkg/streamer" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" - "github.com/pingcap/ticdc/dm/syncer/dbconn" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/binlog/common" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/retry" + "github.com/pingcap/tiflow/dm/pkg/streamer" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/syncer/dbconn" ) // the minimal interval to retry reset binlog streamer. diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 96437d3bf67..a73a160102d 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -45,31 +45,31 @@ import ( "go.uber.org/atomic" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/dm/config" - common2 "github.com/pingcap/ticdc/dm/dm/ctl/common" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/dm/unit" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/binlog/common" - "github.com/pingcap/ticdc/dm/pkg/binlog/event" - "github.com/pingcap/ticdc/dm/pkg/binlog/reader" - "github.com/pingcap/ticdc/dm/pkg/conn" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - fr "github.com/pingcap/ticdc/dm/pkg/func-rollback" - "github.com/pingcap/ticdc/dm/pkg/ha" - "github.com/pingcap/ticdc/dm/pkg/log" - parserpkg "github.com/pingcap/ticdc/dm/pkg/parser" - "github.com/pingcap/ticdc/dm/pkg/schema" - "github.com/pingcap/ticdc/dm/pkg/shardddl/pessimism" - "github.com/pingcap/ticdc/dm/pkg/streamer" - "github.com/pingcap/ticdc/dm/pkg/terror" - "github.com/pingcap/ticdc/dm/pkg/utils" - "github.com/pingcap/ticdc/dm/syncer/dbconn" - operator "github.com/pingcap/ticdc/dm/syncer/err-operator" - "github.com/pingcap/ticdc/dm/syncer/metrics" - onlineddl "github.com/pingcap/ticdc/dm/syncer/online-ddl-tools" - sm "github.com/pingcap/ticdc/dm/syncer/safe-mode" - "github.com/pingcap/ticdc/dm/syncer/shardddl" + "github.com/pingcap/tiflow/dm/dm/config" + common2 "github.com/pingcap/tiflow/dm/dm/ctl/common" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/unit" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/binlog/common" + "github.com/pingcap/tiflow/dm/pkg/binlog/event" + "github.com/pingcap/tiflow/dm/pkg/binlog/reader" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + fr "github.com/pingcap/tiflow/dm/pkg/func-rollback" + "github.com/pingcap/tiflow/dm/pkg/ha" + "github.com/pingcap/tiflow/dm/pkg/log" + parserpkg "github.com/pingcap/tiflow/dm/pkg/parser" + "github.com/pingcap/tiflow/dm/pkg/schema" + "github.com/pingcap/tiflow/dm/pkg/shardddl/pessimism" + "github.com/pingcap/tiflow/dm/pkg/streamer" + "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/syncer/dbconn" + operator "github.com/pingcap/tiflow/dm/syncer/err-operator" + "github.com/pingcap/tiflow/dm/syncer/metrics" + onlineddl "github.com/pingcap/tiflow/dm/syncer/online-ddl-tools" + sm "github.com/pingcap/tiflow/dm/syncer/safe-mode" + "github.com/pingcap/tiflow/dm/syncer/shardddl" ) var ( diff --git a/dm/syncer/syncer_test.go b/dm/syncer/syncer_test.go index 6bfb8564b68..f60f6a4581f 100644 --- a/dm/syncer/syncer_test.go +++ b/dm/syncer/syncer_test.go @@ -25,21 +25,21 @@ import ( "testing" "time" - "github.com/pingcap/ticdc/dm/dm/config" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/binlog/event" - "github.com/pingcap/ticdc/dm/pkg/conn" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/cputil" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/pkg/log" - parserpkg "github.com/pingcap/ticdc/dm/pkg/parser" - "github.com/pingcap/ticdc/dm/pkg/retry" - "github.com/pingcap/ticdc/dm/pkg/schema" - streamer2 "github.com/pingcap/ticdc/dm/pkg/streamer" - "github.com/pingcap/ticdc/dm/pkg/utils" - "github.com/pingcap/ticdc/dm/syncer/dbconn" + "github.com/pingcap/tiflow/dm/dm/config" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/binlog/event" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/cputil" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/pkg/log" + parserpkg "github.com/pingcap/tiflow/dm/pkg/parser" + "github.com/pingcap/tiflow/dm/pkg/retry" + "github.com/pingcap/tiflow/dm/pkg/schema" + streamer2 "github.com/pingcap/tiflow/dm/pkg/streamer" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/syncer/dbconn" sqlmock "github.com/DATA-DOG/go-sqlmock" "github.com/go-mysql-org/go-mysql/mysql" @@ -1082,7 +1082,7 @@ func (s *testSyncerSuite) TestExitSafeModeByConfig(c *C) { checkPointMock.ExpectExec(".*INSERT INTO .* VALUES.* ON DUPLICATE KEY UPDATE.*").WillReturnResult(sqlmock.NewResult(0, 1)) checkPointMock.ExpectCommit() // disable 1-minute safe mode - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/dm/syncer/SafeModeInitPhaseSeconds", "return(0)"), IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/dm/syncer/SafeModeInitPhaseSeconds", "return(0)"), IsNil) go syncer.Process(ctx, resultCh) expectJobs := []*expectJob{ @@ -1155,7 +1155,7 @@ func (s *testSyncerSuite) TestExitSafeModeByConfig(c *C) { if err := checkPointMock.ExpectationsWereMet(); err != nil { c.Errorf("checkpointDB unfulfilled expectations: %s", err) } - c.Assert(failpoint.Disable("github.com/pingcap/ticdc/dm/syncer/SafeModeInitPhaseSeconds"), IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tiflow/dm/syncer/SafeModeInitPhaseSeconds"), IsNil) } func (s *testSyncerSuite) TestRemoveMetadataIsFine(c *C) { diff --git a/dm/syncer/test_injector.go b/dm/syncer/test_injector.go index e19b917de48..0f77839b889 100644 --- a/dm/syncer/test_injector.go +++ b/dm/syncer/test_injector.go @@ -16,8 +16,8 @@ package syncer import ( "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/log" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/terror" ) // TestInjector is used to support inject test cases into syncer. diff --git a/dm/syncer/util.go b/dm/syncer/util.go index 4fc914437d3..c9b37893560 100644 --- a/dm/syncer/util.go +++ b/dm/syncer/util.go @@ -23,9 +23,9 @@ import ( "github.com/pingcap/tidb/parser/ast" "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/conn" - tcontext "github.com/pingcap/ticdc/dm/pkg/context" - "github.com/pingcap/ticdc/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + "github.com/pingcap/tiflow/dm/pkg/terror" ) func toBinlogType(enableRelay bool) BinlogType { diff --git a/dm/tests/_dmctl_tools/check_exit_safe_binlog.go b/dm/tests/_dmctl_tools/check_exit_safe_binlog.go index 8459353d03b..338a1cdfe3d 100644 --- a/dm/tests/_dmctl_tools/check_exit_safe_binlog.go +++ b/dm/tests/_dmctl_tools/check_exit_safe_binlog.go @@ -18,15 +18,15 @@ import ( "fmt" "os" - "github.com/pingcap/ticdc/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/log" "github.com/go-mysql-org/go-mysql/mysql" _ "github.com/go-sql-driver/mysql" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/dm/pkg/binlog" - "github.com/pingcap/ticdc/dm/pkg/gtid" - "github.com/pingcap/ticdc/dm/tests/utils" + "github.com/pingcap/tiflow/dm/pkg/binlog" + "github.com/pingcap/tiflow/dm/pkg/gtid" + "github.com/pingcap/tiflow/dm/tests/utils" ) var compareMap = map[string]map[int]struct{}{ diff --git a/dm/tests/_dmctl_tools/check_master_http_apis.go b/dm/tests/_dmctl_tools/check_master_http_apis.go index 9aa3a421ec4..5f4995fb878 100644 --- a/dm/tests/_dmctl_tools/check_master_http_apis.go +++ b/dm/tests/_dmctl_tools/check_master_http_apis.go @@ -19,7 +19,7 @@ import ( "net/http" "os" - "github.com/pingcap/ticdc/dm/tests/utils" + "github.com/pingcap/tiflow/dm/tests/utils" toolutils "github.com/pingcap/tidb-tools/pkg/utils" ) diff --git a/dm/tests/_dmctl_tools/check_master_online.go b/dm/tests/_dmctl_tools/check_master_online.go index b3977f73e04..eadea8ba9d3 100644 --- a/dm/tests/_dmctl_tools/check_master_online.go +++ b/dm/tests/_dmctl_tools/check_master_online.go @@ -22,8 +22,8 @@ import ( toolutils "github.com/pingcap/tidb-tools/pkg/utils" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/tests/utils" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/tests/utils" ) // use show-ddl-locks request to test DM-master is online diff --git a/dm/tests/_dmctl_tools/check_master_online_http.go b/dm/tests/_dmctl_tools/check_master_online_http.go index 62f33620c86..cdfee1ebc76 100644 --- a/dm/tests/_dmctl_tools/check_master_online_http.go +++ b/dm/tests/_dmctl_tools/check_master_online_http.go @@ -19,7 +19,7 @@ import ( "net/http" "os" - "github.com/pingcap/ticdc/dm/tests/utils" + "github.com/pingcap/tiflow/dm/tests/utils" toolutils "github.com/pingcap/tidb-tools/pkg/utils" ) diff --git a/dm/tests/_dmctl_tools/check_worker_online.go b/dm/tests/_dmctl_tools/check_worker_online.go index 5bc89eabc8c..a3ae201a641 100644 --- a/dm/tests/_dmctl_tools/check_worker_online.go +++ b/dm/tests/_dmctl_tools/check_worker_online.go @@ -22,8 +22,8 @@ import ( toolutils "github.com/pingcap/tidb-tools/pkg/utils" - "github.com/pingcap/ticdc/dm/dm/pb" - "github.com/pingcap/ticdc/dm/tests/utils" + "github.com/pingcap/tiflow/dm/dm/pb" + "github.com/pingcap/tiflow/dm/tests/utils" ) // use query status request to test DM-worker is online diff --git a/dm/tests/_utils/ha_cases_lib.sh b/dm/tests/_utils/ha_cases_lib.sh index dabfd7b05da..d9fb774a90b 100644 --- a/dm/tests/_utils/ha_cases_lib.sh +++ b/dm/tests/_utils/ha_cases_lib.sh @@ -160,7 +160,7 @@ function cleanup() { function isolate_master() { port=${master_ports[$(($1 - 1))]} if [ $2 = "isolate" ]; then - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/dm/master/FailToElect=return(\"master$1\")" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/dm/master/FailToElect=return(\"master$1\")" fi echo "kill dm-master$1" ps aux | grep dm-master$1 | awk '{print $2}' | xargs kill || true @@ -172,7 +172,7 @@ function isolate_master() { function isolate_worker() { port=${worker_ports[$(($1 - 1))]} if [ $2 = "isolate" ]; then - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/dm/worker/FailToKeepAlive=return(\"worker$1\")" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/dm/worker/FailToKeepAlive=return(\"worker$1\")" fi echo "kill dm-worker$1" ps aux | grep dm-worker$1 | awk '{print $2}' | xargs kill || true diff --git a/dm/tests/adjust_gtid/run.sh b/dm/tests/adjust_gtid/run.sh index f51a28a6e9d..88e493b05c2 100755 --- a/dm/tests/adjust_gtid/run.sh +++ b/dm/tests/adjust_gtid/run.sh @@ -58,7 +58,7 @@ function run() { check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT check_metric $MASTER_PORT 'start_leader_counter' 3 0 2 - export GO_FAILPOINTS='github.com/pingcap/ticdc/dm/syncer/AdjustGTIDExit=return(true)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/syncer/AdjustGTIDExit=return(true)' run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT diff --git a/dm/tests/all_mode/run.sh b/dm/tests/all_mode/run.sh index 3d57c1fef24..8618c6700d6 100755 --- a/dm/tests/all_mode/run.sh +++ b/dm/tests/all_mode/run.sh @@ -65,7 +65,7 @@ function test_session_config() { function test_query_timeout() { echo "[$(date)] <<<<<< start test_query_timeout >>>>>>" - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/syncer/BlockSyncStatus=return(\"5s\")" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/syncer/BlockSyncStatus=return(\"5s\")" cp $cur/conf/dm-master.toml $WORK_DIR/dm-master.toml sed -i 's/rpc-timeout = "30s"/rpc-timeout = "3s"/g' $WORK_DIR/dm-master.toml @@ -160,7 +160,7 @@ function test_stop_task_before_checkpoint() { check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT check_metric $MASTER_PORT 'start_leader_counter' 3 0 2 - export GO_FAILPOINTS='github.com/pingcap/ticdc/dm/loader/WaitLoaderStopAfterInitCheckpoint=return(5)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/loader/WaitLoaderStopAfterInitCheckpoint=return(5)' run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml @@ -187,7 +187,7 @@ function test_stop_task_before_checkpoint() { check_port_offline $WORKER1_PORT 20 check_port_offline $WORKER2_PORT 20 - export GO_FAILPOINTS='github.com/pingcap/ticdc/dm/loader/WaitLoaderStopBeforeLoadCheckpoint=return(5)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/loader/WaitLoaderStopBeforeLoadCheckpoint=return(5)' run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT @@ -234,10 +234,10 @@ function test_fail_job_between_event() { # worker1 will be bound to source1 and fail when see the second row change in an event inject_points=( - "github.com/pingcap/ticdc/dm/dm/worker/TaskCheckInterval=return(\"500ms\")" - "github.com/pingcap/ticdc/dm/syncer/countJobFromOneEvent=return()" - "github.com/pingcap/ticdc/dm/syncer/flushFirstJob=return()" - "github.com/pingcap/ticdc/dm/syncer/failSecondJob=return()" + "github.com/pingcap/tiflow/dm/dm/worker/TaskCheckInterval=return(\"500ms\")" + "github.com/pingcap/tiflow/dm/syncer/countJobFromOneEvent=return()" + "github.com/pingcap/tiflow/dm/syncer/flushFirstJob=return()" + "github.com/pingcap/tiflow/dm/syncer/failSecondJob=return()" ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml @@ -246,10 +246,10 @@ function test_fail_job_between_event() { # worker2 will be bound to source2 and fail when see the second event in a GTID inject_points=( - "github.com/pingcap/ticdc/dm/dm/worker/TaskCheckInterval=return(\"500ms\")" - "github.com/pingcap/ticdc/dm/syncer/countJobFromOneGTID=return()" - "github.com/pingcap/ticdc/dm/syncer/flushFirstJob=return()" - "github.com/pingcap/ticdc/dm/syncer/failSecondJob=return()" + "github.com/pingcap/tiflow/dm/dm/worker/TaskCheckInterval=return(\"500ms\")" + "github.com/pingcap/tiflow/dm/syncer/countJobFromOneGTID=return()" + "github.com/pingcap/tiflow/dm/syncer/flushFirstJob=return()" + "github.com/pingcap/tiflow/dm/syncer/failSecondJob=return()" ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml @@ -330,8 +330,8 @@ function run() { test_stop_task_before_checkpoint inject_points=( - "github.com/pingcap/ticdc/dm/dm/worker/TaskCheckInterval=return(\"500ms\")" - "github.com/pingcap/ticdc/dm/relay/NewUpstreamServer=return(true)" + "github.com/pingcap/tiflow/dm/dm/worker/TaskCheckInterval=return(\"500ms\")" + "github.com/pingcap/tiflow/dm/relay/NewUpstreamServer=return(true)" ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" diff --git a/dm/tests/case_sensitive/run.sh b/dm/tests/case_sensitive/run.sh index 8c06720aee9..3fd1fdaea3d 100755 --- a/dm/tests/case_sensitive/run.sh +++ b/dm/tests/case_sensitive/run.sh @@ -10,8 +10,8 @@ API_VERSION="v1alpha1" function run() { run_sql_both_source "SET @@GLOBAL.SQL_MODE='ANSI_QUOTES,NO_AUTO_VALUE_ON_ZERO'" inject_points=( - "github.com/pingcap/ticdc/dm/dm/worker/TaskCheckInterval=return(\"500ms\")" - "github.com/pingcap/ticdc/dm/relay/NewUpstreamServer=return(true)" + "github.com/pingcap/tiflow/dm/dm/worker/TaskCheckInterval=return(\"500ms\")" + "github.com/pingcap/tiflow/dm/relay/NewUpstreamServer=return(true)" ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" diff --git a/dm/tests/checkpoint_transaction/run.sh b/dm/tests/checkpoint_transaction/run.sh index 2487558b220..5771945384a 100755 --- a/dm/tests/checkpoint_transaction/run.sh +++ b/dm/tests/checkpoint_transaction/run.sh @@ -7,7 +7,7 @@ source $cur/../_utils/test_prepare WORK_DIR=$TEST_DIR/$TEST_NAME function run() { - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/syncer/checkCheckpointInMiddleOfTransaction=return" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/syncer/checkCheckpointInMiddleOfTransaction=return" run_sql_file $cur/data/db1.prepare.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 check_contains 'Query OK, 1 row affected' diff --git a/dm/tests/dmctl_basic/run.sh b/dm/tests/dmctl_basic/run.sh index 7af71ca292f..25b56a061f8 100755 --- a/dm/tests/dmctl_basic/run.sh +++ b/dm/tests/dmctl_basic/run.sh @@ -162,7 +162,7 @@ function test_operate_task_bound_to_a_source() { function run() { inject_points=( - "github.com/pingcap/ticdc/dm/pkg/streamer/SetHeartbeatInterval=return(1)" + "github.com/pingcap/tiflow/dm/pkg/streamer/SetHeartbeatInterval=return(1)" ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" diff --git a/dm/tests/drop_column_with_index/run.sh b/dm/tests/drop_column_with_index/run.sh index 3b7f95dc567..8c0ed0308c0 100755 --- a/dm/tests/drop_column_with_index/run.sh +++ b/dm/tests/drop_column_with_index/run.sh @@ -10,8 +10,8 @@ function run() { run_sql_file $cur/data/db1.prepare.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 # need to return error three times, first for switch to remote binlog, second for auto retry inject_points=( - "github.com/pingcap/ticdc/dm/syncer/SyncerGetEventError=return" - "github.com/pingcap/ticdc/dm/syncer/GetEventError=3*return" + "github.com/pingcap/tiflow/dm/syncer/SyncerGetEventError=return" + "github.com/pingcap/tiflow/dm/syncer/GetEventError=3*return" ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" diff --git a/dm/tests/duplicate_event/run.sh b/dm/tests/duplicate_event/run.sh index f70b80c5805..1df98bc42c3 100644 --- a/dm/tests/duplicate_event/run.sh +++ b/dm/tests/duplicate_event/run.sh @@ -11,7 +11,7 @@ function run() { # with a 5 rows insert txn: 1 * FormatDesc + 1 * PreviousGTID + 1 * GTID + 1 * BEGIN + 5 * (Table_map + Write_rows) + 1 * XID # here we fail at the third write rows event, sync should retry and auto recover without any duplicate event - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/syncer/GetEventErrorInTxn=13*return(3)" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/syncer/GetEventErrorInTxn=13*return(3)" run_sql_file $cur/data/db1.prepare.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 check_contains 'Query OK, 2 rows affected' @@ -41,7 +41,7 @@ function run() { # with a 5 rows insert txn: 1 * FormatDesc + 1 * PreviousGTID + 1 * GTID + 1 * BEGIN + 5 * (Table_map + Write_rows) + 1 * XID # here we fail at the third write rows event, sync should retry and auto recover without any duplicate event - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/relay/RelayGetEventFailed=15*return(3);github.com/pingcap/ticdc/dm/relay/retry/RelayAllowRetry=return" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/relay/RelayGetEventFailed=15*return(3);github.com/pingcap/tiflow/dm/relay/retry/RelayAllowRetry=return" run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT diff --git a/dm/tests/fake_rotate_event/run.sh b/dm/tests/fake_rotate_event/run.sh index 704ccf28f16..041f0acf04c 100755 --- a/dm/tests/fake_rotate_event/run.sh +++ b/dm/tests/fake_rotate_event/run.sh @@ -36,7 +36,7 @@ function run() { check_port_offline $WORKER1_PORT 20 # make fake rotate event and rewrite binlog filename to mysql-bin.000001 - export GO_FAILPOINTS='github.com/pingcap/ticdc/dm/syncer/MakeFakeRotateEvent=return("mysql-bin.000001")' + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/syncer/MakeFakeRotateEvent=return("mysql-bin.000001")' run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT diff --git a/dm/tests/full_mode/run.sh b/dm/tests/full_mode/run.sh index 30a8344eadc..c72e7e46813 100755 --- a/dm/tests/full_mode/run.sh +++ b/dm/tests/full_mode/run.sh @@ -7,7 +7,7 @@ source $cur/../_utils/test_prepare WORK_DIR=$TEST_DIR/$TEST_NAME function fail_acquire_global_lock() { - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/dm/worker/TaskCheckInterval=return(\"500ms\")" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/dm/worker/TaskCheckInterval=return(\"500ms\")" run_sql_file $cur/data/db1.prepare.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 check_contains 'Query OK, 2 rows affected' @@ -87,7 +87,7 @@ function escape_schema() { run_sql_file $cur/data/db2.prepare.user.sql $MYSQL_HOST2 $MYSQL_PORT2 $MYSQL_PASSWORD2 check_count 'Query OK, 0 rows affected' 7 - export GO_FAILPOINTS='github.com/pingcap/ticdc/dm/dumpling/SkipRemovingDumplingMetrics=return("")' + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/dumpling/SkipRemovingDumplingMetrics=return("")' run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT diff --git a/dm/tests/generate-mock.sh b/dm/tests/generate-mock.sh index ebdbbc79d47..cc4200a2779 100755 --- a/dm/tests/generate-mock.sh +++ b/dm/tests/generate-mock.sh @@ -19,7 +19,7 @@ for file in ./dm/dm/pb/*pb.go; do # extract public interface from pb source file ifs=$(grep -E "type [[:upper:]].*interface" "$file" | awk '{print $2}' 'ORS=,' | rev | cut -c 2- | rev) echo "generate mock for file $file" - "$MOCKGEN" -destination ./dm/dm/pbmock/"$prefix".go -package "$PACKAGE" github.com/pingcap/ticdc/dm/dm/pb "$ifs" + "$MOCKGEN" -destination ./dm/dm/pbmock/"$prefix".go -package "$PACKAGE" github.com/pingcap/tiflow/dm/dm/pb "$ifs" done echo "generate grpc mock code successfully" diff --git a/dm/tests/ha_cases_1/run.sh b/dm/tests/ha_cases_1/run.sh index 21dc947c5f6..85542699e8b 100755 --- a/dm/tests/ha_cases_1/run.sh +++ b/dm/tests/ha_cases_1/run.sh @@ -71,8 +71,8 @@ function test_kill_master() { } function test_kill_and_isolate_worker() { - inject_points=("github.com/pingcap/ticdc/dm/dm/worker/defaultKeepAliveTTL=return(1)" - "github.com/pingcap/ticdc/dm/dm/worker/defaultRelayKeepAliveTTL=return(2)" + inject_points=("github.com/pingcap/tiflow/dm/dm/worker/defaultKeepAliveTTL=return(1)" + "github.com/pingcap/tiflow/dm/dm/worker/defaultRelayKeepAliveTTL=return(2)" ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" echo "[$(date)] <<<<<< start test_kill_and_isolate_worker >>>>>>" diff --git a/dm/tests/import_goroutine_leak/run.sh b/dm/tests/import_goroutine_leak/run.sh index 35aa9544e9a..76c3854a35a 100644 --- a/dm/tests/import_goroutine_leak/run.sh +++ b/dm/tests/import_goroutine_leak/run.sh @@ -27,11 +27,11 @@ function run() { echo "dm-worker panic, doJob of import unit workers don't exit" # send to closed `runFatalChan` - inject_points=("github.com/pingcap/ticdc/dm/loader/LoadDataSlowDown=sleep(1000)" - "github.com/pingcap/ticdc/dm/loader/dispatchError=return(1)" - "github.com/pingcap/ticdc/dm/loader/executeSQLError=return(1)" - "github.com/pingcap/ticdc/dm/loader/returnDoJobError=return(1)" - "github.com/pingcap/ticdc/dm/loader/workerCantClose=return(1)" + inject_points=("github.com/pingcap/tiflow/dm/loader/LoadDataSlowDown=sleep(1000)" + "github.com/pingcap/tiflow/dm/loader/dispatchError=return(1)" + "github.com/pingcap/tiflow/dm/loader/executeSQLError=return(1)" + "github.com/pingcap/tiflow/dm/loader/returnDoJobError=return(1)" + "github.com/pingcap/tiflow/dm/loader/workerCantClose=return(1)" ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" @@ -59,11 +59,11 @@ function run() { echo "dm-worker panic again, workers of import unit don't exit" # send to closed `runFatalChan` - inject_points=("github.com/pingcap/ticdc/dm/loader/LoadDataSlowDown=sleep(1000)" - "github.com/pingcap/ticdc/dm/loader/dispatchError=return(1)" - "github.com/pingcap/ticdc/dm/loader/executeSQLError=return(1)" - "github.com/pingcap/ticdc/dm/loader/returnDoJobError=return(1)" - "github.com/pingcap/ticdc/dm/loader/dontWaitWorkerExit=return(1)" + inject_points=("github.com/pingcap/tiflow/dm/loader/LoadDataSlowDown=sleep(1000)" + "github.com/pingcap/tiflow/dm/loader/dispatchError=return(1)" + "github.com/pingcap/tiflow/dm/loader/executeSQLError=return(1)" + "github.com/pingcap/tiflow/dm/loader/returnDoJobError=return(1)" + "github.com/pingcap/tiflow/dm/loader/dontWaitWorkerExit=return(1)" ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml @@ -81,10 +81,10 @@ function run() { echo "restart dm-workers with errros to pause" # paused with injected error - inject_points=("github.com/pingcap/ticdc/dm/loader/LoadDataSlowDown=sleep(1000)" - "github.com/pingcap/ticdc/dm/loader/dispatchError=return(1)" - "github.com/pingcap/ticdc/dm/loader/executeSQLError=return(1)" - "github.com/pingcap/ticdc/dm/loader/returnDoJobError=return(1)" + inject_points=("github.com/pingcap/tiflow/dm/loader/LoadDataSlowDown=sleep(1000)" + "github.com/pingcap/tiflow/dm/loader/dispatchError=return(1)" + "github.com/pingcap/tiflow/dm/loader/executeSQLError=return(1)" + "github.com/pingcap/tiflow/dm/loader/returnDoJobError=return(1)" ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml @@ -99,10 +99,10 @@ function run() { check_port_offline $WORKER1_PORT 20 # use a small job chan size to block the sender - inject_points=("github.com/pingcap/ticdc/dm/loader/LoadDataSlowDown=sleep(1000)" - "github.com/pingcap/ticdc/dm/loader/executeSQLError=return(1)" - "github.com/pingcap/ticdc/dm/loader/returnDoJobError=return(1)" - "github.com/pingcap/ticdc/dm/loader/workerChanSize=return(10)" + inject_points=("github.com/pingcap/tiflow/dm/loader/LoadDataSlowDown=sleep(1000)" + "github.com/pingcap/tiflow/dm/loader/executeSQLError=return(1)" + "github.com/pingcap/tiflow/dm/loader/returnDoJobError=return(1)" + "github.com/pingcap/tiflow/dm/loader/workerChanSize=return(10)" ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml diff --git a/dm/tests/incremental_mode/run.sh b/dm/tests/incremental_mode/run.sh index b8b7dbbf171..34d5aab8b3b 100755 --- a/dm/tests/incremental_mode/run.sh +++ b/dm/tests/incremental_mode/run.sh @@ -15,7 +15,7 @@ function run() { run_sql_file $cur/data/db2.prepare.sql $MYSQL_HOST2 $MYSQL_PORT2 $MYSQL_PASSWORD2 check_contains 'Query OK, 3 rows affected' - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/dm/worker/defaultKeepAliveTTL=return(1)" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/dm/worker/defaultKeepAliveTTL=return(1)" run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT @@ -155,7 +155,7 @@ function run() { sed -i "s/binlog-gtid-placeholder-2/$gtid2/g" $WORK_DIR/dm-task.yaml # test graceful display error - export GO_FAILPOINTS='github.com/pingcap/ticdc/dm/syncer/GetEventError=return' + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/syncer/GetEventError=return' run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml @@ -171,7 +171,7 @@ function run() { check_port_offline $WORKER2_PORT 20 # only mock pull binlog failed once - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/syncer/WaitUserCancel=return(8);github.com/pingcap/ticdc/dm/syncer/GetEventError=1*return" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/syncer/WaitUserCancel=return(8);github.com/pingcap/tiflow/dm/syncer/GetEventError=1*return" run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml @@ -212,7 +212,7 @@ function run() { check_port_offline $WORKER1_PORT 20 check_port_offline $WORKER2_PORT 20 - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/syncer/FlushCheckpointStage=return(100)" # for all stages + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/syncer/FlushCheckpointStage=return(100)" # for all stages run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml diff --git a/dm/tests/initial_unit/run.sh b/dm/tests/initial_unit/run.sh index 1d499b8fc65..8260396a5c7 100644 --- a/dm/tests/initial_unit/run.sh +++ b/dm/tests/initial_unit/run.sh @@ -19,8 +19,8 @@ function prepare_data() { function run() { failpoints=( # 1152 is ErrAbortingConnection - "github.com/pingcap/ticdc/dm/syncer/LoadCheckpointFailed=return(1152)" - "github.com/pingcap/ticdc/dm/syncer/dbconn/GetMasterStatusFailed=return(1152)" + "github.com/pingcap/tiflow/dm/syncer/LoadCheckpointFailed=return(1152)" + "github.com/pingcap/tiflow/dm/syncer/dbconn/GetMasterStatusFailed=return(1152)" ) for ((i = 0; i < ${#failpoints[@]}; i++)); do diff --git a/dm/tests/lightning_mode/run.sh b/dm/tests/lightning_mode/run.sh index b15ce6fbc4a..11b26645816 100755 --- a/dm/tests/lightning_mode/run.sh +++ b/dm/tests/lightning_mode/run.sh @@ -12,8 +12,8 @@ function run() { run_sql_both_source "SET @@GLOBAL.SQL_MODE='ANSI_QUOTES,NO_AUTO_VALUE_ON_ZERO'" inject_points=( - "github.com/pingcap/ticdc/dm/dm/worker/TaskCheckInterval=return(\"500ms\")" - "github.com/pingcap/ticdc/dm/relay/NewUpstreamServer=return(true)" + "github.com/pingcap/tiflow/dm/dm/worker/TaskCheckInterval=return(\"500ms\")" + "github.com/pingcap/tiflow/dm/relay/NewUpstreamServer=return(true)" ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" diff --git a/dm/tests/load_interrupt/run.sh b/dm/tests/load_interrupt/run.sh index 7113b487dbb..79f8c79c44c 100755 --- a/dm/tests/load_interrupt/run.sh +++ b/dm/tests/load_interrupt/run.sh @@ -37,7 +37,7 @@ function test_save_checkpoint_failed() { prepare_datafile run_sql_file $WORK_DIR/db1.prepare.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/loader/loaderCPUpdateOffsetError=return()" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/loaderCPUpdateOffsetError=return()" run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT @@ -70,7 +70,7 @@ function run() { prepare_datafile run_sql_file $WORK_DIR/db1.prepare.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 THRESHOLD=1024 - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/loader/LoadExceedOffsetExit=return($THRESHOLD)" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LoadExceedOffsetExit=return($THRESHOLD)" run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT @@ -98,7 +98,7 @@ function run() { # check_row_count 1 # only failed at the first two time, will retry later and success - export GO_FAILPOINTS='github.com/pingcap/ticdc/dm/loader/LoadExecCreateTableFailed=3*return("1213")' # ER_LOCK_DEADLOCK, retryable error code + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/loader/LoadExecCreateTableFailed=3*return("1213")' # ER_LOCK_DEADLOCK, retryable error code run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT diff --git a/dm/tests/load_task/run.sh b/dm/tests/load_task/run.sh index 10b8d0d77db..9bd2c791920 100755 --- a/dm/tests/load_task/run.sh +++ b/dm/tests/load_task/run.sh @@ -31,7 +31,7 @@ function test_worker_restart() { "Please check if the previous worker is online." 1 # worker1 online - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/loader/LoadDataSlowDownByTask=return(\"load_task1\")" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LoadDataSlowDownByTask=return(\"load_task1\")" run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT @@ -80,7 +80,7 @@ function test_transfer_two_sources() { "\"unit\": \"Load\"" 1 # worker2 online - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/loader/LoadDataSlowDown=sleep(15000)" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LoadDataSlowDown=sleep(15000)" run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT @@ -182,7 +182,7 @@ function run() { check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT1 # worker1 loading load_task1 - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/loader/LoadDataSlowDownByTask=return(\"load_task1\")" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LoadDataSlowDownByTask=return(\"load_task1\")" run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT cp $cur/conf/source1.yaml $WORK_DIR/source1.yaml @@ -190,7 +190,7 @@ function run() { dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 # worker2 loading load_task2 - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/loader/LoadDataSlowDownByTask=return(\"load_task2\")" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LoadDataSlowDownByTask=return(\"load_task2\")" run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT cp $cur/conf/source2.yaml $WORK_DIR/source2.yaml @@ -198,7 +198,7 @@ function run() { dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 # worker3 loading load_task3 - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/loader/LoadDataSlowDownByTask=return(\"load_task3\")" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LoadDataSlowDownByTask=return(\"load_task3\")" run_dm_worker $WORK_DIR/worker3 $WORKER3_PORT $cur/conf/dm-worker3.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER3_PORT diff --git a/dm/tests/metrics/run.sh b/dm/tests/metrics/run.sh index c21f72c10f3..9372497d489 100755 --- a/dm/tests/metrics/run.sh +++ b/dm/tests/metrics/run.sh @@ -18,8 +18,8 @@ function run() { check_dashboard_datasource inject_points=( - "github.com/pingcap/ticdc/dm/syncer/BlockDDLJob=return(1)" - "github.com/pingcap/ticdc/dm/syncer/ShowLagInLog=return(1)" # test lag metric >= 1 beacuse we inject BlockDDLJob(ddl) to sleep(1) + "github.com/pingcap/tiflow/dm/syncer/BlockDDLJob=return(1)" + "github.com/pingcap/tiflow/dm/syncer/ShowLagInLog=return(1)" # test lag metric >= 1 beacuse we inject BlockDDLJob(ddl) to sleep(1) ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" @@ -75,8 +75,8 @@ function run() { rm -rf $WORK_DIR/worker1/log/dm-worker.log # clean up the old log rm -rf $WORK_DIR/worker2/log/dm-worker.log # clean up the old log inject_points=( - "github.com/pingcap/ticdc/dm/syncer/BlockExecuteSQLs=return(2)" - "github.com/pingcap/ticdc/dm/syncer/ShowLagInLog=return(2)" # test lag metric >= 2 beacuse we inject BlockExecuteSQLs to sleep(2) although skip lag is 0 (locally), but we use that lag of all dml/skip lag, so lag still >= 2 + "github.com/pingcap/tiflow/dm/syncer/BlockExecuteSQLs=return(2)" + "github.com/pingcap/tiflow/dm/syncer/ShowLagInLog=return(2)" # test lag metric >= 2 beacuse we inject BlockExecuteSQLs to sleep(2) although skip lag is 0 (locally), but we use that lag of all dml/skip lag, so lag still >= 2 ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" @@ -137,8 +137,8 @@ function run() { wait_pattern_exit dm-worker1.toml inject_points=( - "github.com/pingcap/ticdc/dm/syncer/noJobInQueueLog=return()" - "github.com/pingcap/ticdc/dm/syncer/IgnoreSomeTypeEvent=return(\"HeartbeatEvent\")" + "github.com/pingcap/tiflow/dm/syncer/noJobInQueueLog=return()" + "github.com/pingcap/tiflow/dm/syncer/IgnoreSomeTypeEvent=return(\"HeartbeatEvent\")" ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" rm -rf $WORK_DIR/worker1/log/dm-worker.log # clean up the old log diff --git a/dm/tests/new_relay/run.sh b/dm/tests/new_relay/run.sh index 43a2156185a..92e6b1053b8 100755 --- a/dm/tests/new_relay/run.sh +++ b/dm/tests/new_relay/run.sh @@ -28,7 +28,7 @@ function test_cant_dail_upstream() { kill_dm_worker - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/pkg/conn/failDBPing=return()" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/pkg/conn/failDBPing=return()" run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT @@ -85,7 +85,7 @@ function test_kill_dump_connection() { function run() { test_cant_dail_upstream - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/relay/ReportRelayLogSpaceInBackground=return(1)" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/relay/ReportRelayLogSpaceInBackground=return(1)" run_sql_file $cur/data/db1.prepare.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 check_contains 'Query OK, 2 rows affected' diff --git a/dm/tests/online_ddl/run.sh b/dm/tests/online_ddl/run.sh index 69e6764c307..cbcca27f7fb 100755 --- a/dm/tests/online_ddl/run.sh +++ b/dm/tests/online_ddl/run.sh @@ -30,8 +30,8 @@ function run() { dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 inject_points=( - "github.com/pingcap/ticdc/dm/syncer/online-ddl-tools/ExitAfterSaveOnlineDDL=return()" - "github.com/pingcap/ticdc/dm/syncer/ExitAfterSaveOnlineDDL=return()" + "github.com/pingcap/tiflow/dm/syncer/online-ddl-tools/ExitAfterSaveOnlineDDL=return()" + "github.com/pingcap/tiflow/dm/syncer/ExitAfterSaveOnlineDDL=return()" ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml diff --git a/dm/tests/only_dml/run.sh b/dm/tests/only_dml/run.sh index 52c666b7ad0..d735fb644b4 100755 --- a/dm/tests/only_dml/run.sh +++ b/dm/tests/only_dml/run.sh @@ -42,7 +42,7 @@ function insert_data() { } function run() { - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/pkg/streamer/SetHeartbeatInterval=return(1);github.com/pingcap/ticdc/dm/syncer/syncDMLBatchNotFull=return(true)" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/pkg/streamer/SetHeartbeatInterval=return(1);github.com/pingcap/tiflow/dm/syncer/syncDMLBatchNotFull=return(true)" run_sql_file $cur/data/db1.prepare.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 check_contains 'Query OK, 1 row affected' diff --git a/dm/tests/print_status/run.sh b/dm/tests/print_status/run.sh index 891709ac6e4..539602c4300 100755 --- a/dm/tests/print_status/run.sh +++ b/dm/tests/print_status/run.sh @@ -14,9 +14,9 @@ function run() { # TableMapEvent, QueryEvent, GTIDEvent, and a specific Event in each group. # so we slow down 460 * 4 ms. Besides the log may be not flushed to disk asap, # we need to add some retry mechanism - inject_points=("github.com/pingcap/ticdc/dm/dm/worker/PrintStatusCheckSeconds=return(1)" - "github.com/pingcap/ticdc/dm/loader/LoadDataSlowDown=sleep(100)" - "github.com/pingcap/ticdc/dm/syncer/ProcessBinlogSlowDown=sleep(4)") + inject_points=("github.com/pingcap/tiflow/dm/dm/worker/PrintStatusCheckSeconds=return(1)" + "github.com/pingcap/tiflow/dm/loader/LoadDataSlowDown=sleep(100)" + "github.com/pingcap/tiflow/dm/syncer/ProcessBinlogSlowDown=sleep(4)") export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml diff --git a/dm/tests/relay_interrupt/run.sh b/dm/tests/relay_interrupt/run.sh index 6d9abd39c8b..4ba58be24a9 100644 --- a/dm/tests/relay_interrupt/run.sh +++ b/dm/tests/relay_interrupt/run.sh @@ -23,8 +23,8 @@ function prepare_data2() { function run() { failpoints=( # 1152 is ErrAbortingConnection - "github.com/pingcap/ticdc/dm/pkg/utils/GetGlobalVariableFailed=return(\"server_uuid,1152\")" - "github.com/pingcap/ticdc/dm/pkg/utils/GetSessionVariableFailed=return(\"sql_mode,1152\")" + "github.com/pingcap/tiflow/dm/pkg/utils/GetGlobalVariableFailed=return(\"server_uuid,1152\")" + "github.com/pingcap/tiflow/dm/pkg/utils/GetSessionVariableFailed=return(\"sql_mode,1152\")" ) for ((i = 0; i < ${#failpoints[@]}; i++)); do @@ -103,7 +103,7 @@ function run() { echo "read binlog from relay log failed, and will use remote binlog" kill_dm_worker - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/pkg/streamer/GetEventFromLocalFailed=return()" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/pkg/streamer/GetEventFromLocalFailed=return()" run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT prepare_data2 $i diff --git a/dm/tests/retry_cancel/run.sh.todo b/dm/tests/retry_cancel/run.sh.todo index a185af81143..979854a85d7 100755 --- a/dm/tests/retry_cancel/run.sh.todo +++ b/dm/tests/retry_cancel/run.sh.todo @@ -14,7 +14,7 @@ function run() { check_contains 'Query OK, 2 rows affected' # inject error for loading data - export GO_FAILPOINTS='github.com/pingcap/ticdc/dm/pkg/conn/retryableError=return("retry_cancel")' + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/pkg/conn/retryableError=return("retry_cancel")' run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT @@ -55,7 +55,7 @@ function run() { # stop DM-worker, then update failpoint for checkpoint kill_dm_worker - export GO_FAILPOINTS='github.com/pingcap/ticdc/dm/pkg/conn/retryableError=return("UPDATE `dm_meta`.`test_loader_checkpoint`")' + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/pkg/conn/retryableError=return("UPDATE `dm_meta`.`test_loader_checkpoint`")' # start DM-worker again run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml @@ -107,7 +107,7 @@ function run() { # ---------- test for incremental replication ---------- # stop DM-worker, then enable failponits kill_dm_worker - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/pkg/conn/retryableError=return(\"retry_cancel\")" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/pkg/conn/retryableError=return(\"retry_cancel\")" # run sql files to trigger incremental replication run_sql_file $cur/data/db1.increment.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 diff --git a/dm/tests/safe_mode/run.sh b/dm/tests/safe_mode/run.sh index 4e50f8ad355..4d10cc0914b 100755 --- a/dm/tests/safe_mode/run.sh +++ b/dm/tests/safe_mode/run.sh @@ -97,7 +97,7 @@ function safe_mode_recover() { check_port_offline $WORKER1_PORT 20 check_port_offline $WORKER2_PORT 20 - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/syncer/SafeModeExit=return($i)" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/syncer/SafeModeExit=return($i)" run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml @@ -195,7 +195,7 @@ function run() { run_sql_file $cur/data/db2.prepare.sql $MYSQL_HOST2 $MYSQL_PORT2 $MYSQL_PASSWORD2 check_contains 'Query OK, 3 rows affected' - export GO_FAILPOINTS='github.com/pingcap/ticdc/dm/syncer/ReSyncExit=return(true)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/syncer/ReSyncExit=return(true)' run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml @@ -220,7 +220,7 @@ function run() { check_port_offline $WORKER1_PORT 20 check_port_offline $WORKER2_PORT 20 - export GO_FAILPOINTS='github.com/pingcap/ticdc/dm/syncer/ShardSyncedExecutionExit=return(true);github.com/pingcap/ticdc/dm/syncer/SafeModeInitPhaseSeconds=return(300)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/syncer/ShardSyncedExecutionExit=return(true);github.com/pingcap/tiflow/dm/syncer/SafeModeInitPhaseSeconds=return(300)' run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml @@ -242,7 +242,7 @@ function run() { # DM-worker1 is sharding lock owner and exits if [ "$(check_port_return $WORKER1_PORT)" == "0" ]; then echo "DM-worker1 is sharding lock owner and detects it offline" - export GO_FAILPOINTS='github.com/pingcap/ticdc/dm/syncer/SafeModeInitPhaseSeconds=return(0)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/syncer/SafeModeInitPhaseSeconds=return(0)' run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT check_instance_id="1" @@ -252,7 +252,7 @@ function run() { # DM-worker2 is sharding lock owner and exits if [ "$(check_port_return $WORKER2_PORT)" == "0" ]; then echo "DM-worker2 is sharding lock owner and detects it offline" - export GO_FAILPOINTS='github.com/pingcap/ticdc/dm/syncer/SafeModeInitPhaseSeconds=return(0)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/syncer/SafeModeInitPhaseSeconds=return(0)' run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT check_instance_id="2" diff --git a/dm/tests/sequence_safe_mode/run.sh b/dm/tests/sequence_safe_mode/run.sh index 7461f4c6844..5a7f5d70793 100755 --- a/dm/tests/sequence_safe_mode/run.sh +++ b/dm/tests/sequence_safe_mode/run.sh @@ -12,7 +12,7 @@ function run() { run_sql_file $cur/data/db2.prepare.sql $MYSQL_HOST2 $MYSQL_PORT2 $MYSQL_PASSWORD2 check_contains 'Query OK, 3 rows affected' - export GO_FAILPOINTS='github.com/pingcap/ticdc/dm/syncer/ReSyncExit=return(true)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/syncer/ReSyncExit=return(true)' run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml @@ -50,7 +50,7 @@ function run() { pkill -hup dm-worker.test 2>/dev/null || true wait_process_exit dm-worker.test - export GO_FAILPOINTS='github.com/pingcap/ticdc/dm/syncer/SequenceShardSyncedExecutionExit=return(true);github.com/pingcap/ticdc/dm/syncer/SafeModeInitPhaseSeconds=return(300)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/syncer/SequenceShardSyncedExecutionExit=return(true);github.com/pingcap/tiflow/dm/syncer/SafeModeInitPhaseSeconds=return(300)' echo "restart DM-worker after set SequenceShardSyncedExecutionExit failpoint" run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml @@ -71,7 +71,7 @@ function run() { # DM-worker1 is sharding lock owner and exits if [ "$(check_port_return $WORKER1_PORT)" == "0" ]; then echo "DM-worker1 is sharding lock owner and detects it offline" - export GO_FAILPOINTS='github.com/pingcap/ticdc/dm/syncer/SafeModeInitPhaseSeconds=return(0)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/syncer/SafeModeInitPhaseSeconds=return(0)' run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT check_instance_id="1" @@ -81,7 +81,7 @@ function run() { # DM-worker2 is sharding lock owner and exits if [ "$(check_port_return $WORKER2_PORT)" == "0" ]; then echo "DM-worker2 is sharding lock owner and detects it offline" - export GO_FAILPOINTS='github.com/pingcap/ticdc/dm/syncer/SafeModeInitPhaseSeconds=return(0)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/syncer/SafeModeInitPhaseSeconds=return(0)' run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT check_instance_id="2" diff --git a/dm/tests/sequence_sharding_optimistic/run.sh b/dm/tests/sequence_sharding_optimistic/run.sh index 0624684ab80..d414646e899 100755 --- a/dm/tests/sequence_sharding_optimistic/run.sh +++ b/dm/tests/sequence_sharding_optimistic/run.sh @@ -84,7 +84,7 @@ run() { # now, for optimistic shard DDL, different sources will reach a stage often not at the same time, # in order to simply the check and resume flow, only enable the failpoint for one DM-worker. - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/syncer/FlushCheckpointStage=return(100)" # for all stages + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/syncer/FlushCheckpointStage=return(100)" # for all stages echo "restart dm-worker1" ps aux | grep dm-worker1 | awk '{print $2}' | xargs kill || true check_port_offline $WORKER1_PORT 20 diff --git a/dm/tests/shardddl1/run.sh b/dm/tests/shardddl1/run.sh index 4b63801670d..3b81c3f8ba1 100644 --- a/dm/tests/shardddl1/run.sh +++ b/dm/tests/shardddl1/run.sh @@ -261,7 +261,7 @@ function DM_RemoveLock_CASE() { function DM_RemoveLock() { ps aux | grep dm-master | awk '{print $2}' | xargs kill || true check_master_port_offline 1 - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/dm/master/shardddl/SleepWhenRemoveLock=return(30)" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/dm/master/shardddl/SleepWhenRemoveLock=return(30)" run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ @@ -597,7 +597,7 @@ function DM_COMPACT() { ps aux | grep dm-worker | awk '{print $2}' | xargs kill || true check_port_offline $WORKER1_PORT 20 check_port_offline $WORKER2_PORT 20 - export GO_FAILPOINTS='github.com/pingcap/ticdc/dm/syncer/BlockExecuteSQLs=return(1)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/syncer/BlockExecuteSQLs=return(1)' run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT diff --git a/dm/tests/shardddl2/run.sh b/dm/tests/shardddl2/run.sh index 5f095e78588..2c95319431c 100644 --- a/dm/tests/shardddl2/run.sh +++ b/dm/tests/shardddl2/run.sh @@ -248,7 +248,7 @@ function DM_DROP_COLUMN_EXEC_ERROR_CASE() { w="2" fi - restart_worker $w "github.com/pingcap/ticdc/dm/syncer/ExecDDLError=return()" + restart_worker $w "github.com/pingcap/tiflow/dm/syncer/ExecDDLError=return()" run_sql_source1 "insert into ${shardddl1}.${tb1} values(1,'aaa');" run_sql_source2 "insert into ${shardddl1}.${tb1} values(2,'bbb');" @@ -309,7 +309,7 @@ function DM_DROP_COLUMN_ALL_DONE_CASE() { w="2" fi - restart_worker $w "github.com/pingcap/ticdc/dm/syncer/ExecDDLError=return()" + restart_worker $w "github.com/pingcap/tiflow/dm/syncer/ExecDDLError=return()" run_sql_source1 "insert into ${shardddl1}.${tb1} values(1,'aaa');" run_sql_source2 "insert into ${shardddl1}.${tb1} values(2,'bbb');" diff --git a/dm/tests/sharding/run.sh b/dm/tests/sharding/run.sh index 45e8b99c54c..87805d50ab9 100755 --- a/dm/tests/sharding/run.sh +++ b/dm/tests/sharding/run.sh @@ -42,7 +42,7 @@ function run() { # # now, for pessimistic shard DDL, owner and non-owner will reach a stage often not at the same time, # in order to simply the check and resume flow, only enable the failpoint for one DM-worker. - export GO_FAILPOINTS="github.com/pingcap/ticdc/dm/syncer/FlushCheckpointStage=return(2)" + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/syncer/FlushCheckpointStage=return(2)" run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 diff --git a/dm/tests/sharding2/run.sh b/dm/tests/sharding2/run.sh index cf8c6530add..171c318a1c7 100755 --- a/dm/tests/sharding2/run.sh +++ b/dm/tests/sharding2/run.sh @@ -61,7 +61,7 @@ function run() { check_port_offline $WORKER1_PORT 20 check_port_offline $WORKER2_PORT 20 - export GO_FAILPOINTS='github.com/pingcap/ticdc/dm/syncer/ExitAfterDDLBeforeFlush=return(true)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/syncer/ExitAfterDDLBeforeFlush=return(true)' run_sql_file $cur/data/db1.increment3.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 run_sql_file $cur/data/db2.increment3.sql $MYSQL_HOST2 $MYSQL_PORT2 $MYSQL_PASSWORD2 @@ -75,7 +75,7 @@ function run() { check_port_offline $WORKER2_PORT 20 # mock recover relay writer - export GO_FAILPOINTS='github.com/pingcap/ticdc/dm/relay/writer/MockRecoverRelayWriter=return(true)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/relay/writer/MockRecoverRelayWriter=return(true)' run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT diff --git a/dm/tests/start_task/run.sh b/dm/tests/start_task/run.sh index 08d08952bc6..90e51cb24f7 100644 --- a/dm/tests/start_task/run.sh +++ b/dm/tests/start_task/run.sh @@ -66,8 +66,8 @@ function run() { lazy_init_tracker failpoints=( # 1152 is ErrAbortingConnection - "github.com/pingcap/ticdc/dm/pkg/utils/FetchTargetDoTablesFailed=return(1152)" - "github.com/pingcap/ticdc/dm/pkg/utils/FetchAllDoTablesFailed=return(1152)" + "github.com/pingcap/tiflow/dm/pkg/utils/FetchTargetDoTablesFailed=return(1152)" + "github.com/pingcap/tiflow/dm/pkg/utils/FetchAllDoTablesFailed=return(1152)" ) for ((i = 0; i < ${#failpoints[@]}; i++)); do diff --git a/dm/tests/utils/dmctl.go b/dm/tests/utils/dmctl.go index febae30f22c..b72f2f9d4e2 100644 --- a/dm/tests/utils/dmctl.go +++ b/dm/tests/utils/dmctl.go @@ -22,7 +22,7 @@ import ( "github.com/pingcap/errors" "google.golang.org/grpc" - "github.com/pingcap/ticdc/dm/dm/pb" + "github.com/pingcap/tiflow/dm/dm/pb" ) // CreateDmCtl creates a gRPC client to DM-master. diff --git a/dm/tests/utils/log.go b/dm/tests/utils/log.go index 3a1ed2dc6cd..051ce709931 100644 --- a/dm/tests/utils/log.go +++ b/dm/tests/utils/log.go @@ -18,7 +18,7 @@ import ( "go.uber.org/zap" - "github.com/pingcap/ticdc/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/log" ) // ExitWithError forces to exist the process, it's often used in integration tests. diff --git a/docs/design/2020-03-25-ticdc-high-availability-new-design-cn.md b/docs/design/2020-03-25-ticdc-high-availability-new-design-cn.md index 7ea25a8fd9c..f71e2975278 100644 --- a/docs/design/2020-03-25-ticdc-high-availability-new-design-cn.md +++ b/docs/design/2020-03-25-ticdc-high-availability-new-design-cn.md @@ -13,7 +13,7 @@ TiCDC 高可用是指任何一个 cdc 节点挂掉,都不影响集群整体的 本文总结已有代码发现的一些问题,并给出解决方案。 -> 注意,本文分析代码基于 [v4.0.0-beta.2](https://github.com/pingcap/ticdc/tree/v4.0.0-beta.2) 版本 +> 注意,本文分析代码基于 [v4.0.0-beta.2](https://github.com/pingcap/tiflow/tree/v4.0.0-beta.2) 版本 ## 问题 @@ -21,9 +21,9 @@ TiCDC 高可用是指任何一个 cdc 节点挂掉,都不影响集群整体的 一个 Session 是指节点与 etcd 之间维持的一种保持连接的状态。Session 中含有 Lease,并在节点存活期间保持心跳。节点挂掉后,Session 失效,与其中 Lease 关联的 Key 都被删除。这常用来服务发现中对服务节点是否存活的标识。 -起初,Capture 并没有使用 Session,而是仅仅在竞选 Owner 的时候创建了 Session。这样导致 Capture 挂掉后,其注册信息是无法被清理的。Capture 节点挂掉后(非正常退出),其负责的任务没有及时重新分配。这个问题在 PR [Refactor/capture watcher](https://github.com/pingcap/ticdc/pull/319) 中解决。 +起初,Capture 并没有使用 Session,而是仅仅在竞选 Owner 的时候创建了 Session。这样导致 Capture 挂掉后,其注册信息是无法被清理的。Capture 节点挂掉后(非正常退出),其负责的任务没有及时重新分配。这个问题在 PR [Refactor/capture watcher](https://github.com/pingcap/tiflow/pull/319) 中解决。 -另外,为了减少 RTO,我们引入了 Processor 的 Session,用来及时发现挂掉的 Processor。PR:[Reduce the RTO by watching the liveness of processors](https://github.com/pingcap/ticdc/pull/312) +另外,为了减少 RTO,我们引入了 Processor 的 Session,用来及时发现挂掉的 Processor。PR:[Reduce the RTO by watching the liveness of processors](https://github.com/pingcap/tiflow/pull/312) 因此,我们目前有三个 Session,一个是 Capture Session,用来维持 Capture 存活信息,一个是 Processor Session,用来维护 Processor 存活信息,还有一个是原来存在的 Manager Session,用来竞选 Owner。 diff --git a/docs/design/2021-10-13-ticdc-mq-sink-column-selector.md b/docs/design/2021-10-13-ticdc-mq-sink-column-selector.md index 7192a2453b9..f26d8ec7af0 100644 --- a/docs/design/2021-10-13-ticdc-mq-sink-column-selector.md +++ b/docs/design/2021-10-13-ticdc-mq-sink-column-selector.md @@ -1,7 +1,7 @@ # TiCDC Design Documents - Author(s): [hi-rustin](https://github.com/hi-rustin) -- Tracking Issue: https://github.com/pingcap/ticdc/issues/3082 +- Tracking Issue: https://github.com/pingcap/tiflow/issues/3082 ## Table of Contents diff --git a/docs/design/README.md b/docs/design/README.md index 70f8de37446..3eda8a36a09 100644 --- a/docs/design/README.md +++ b/docs/design/README.md @@ -16,14 +16,14 @@ A hastily-proposed design document can hurt its chances of acceptance. Low-quali Although there is no single way to prepare for submitting a design document, it is generally a good idea to pursue feedback from other project developers beforehand, to ascertain that the design document may be desirable; having a consistent impact on the project requires concerted effort toward consensus-building. -The most common preparations for writing and submitting a design document for now is [creating a discuss issue](https://github.com/pingcap/ticdc/issues/new/choose), which is going to be converted into a tracking issue of the design implementation. +The most common preparations for writing and submitting a design document for now is [creating a discuss issue](https://github.com/pingcap/tiflow/issues/new/choose), which is going to be converted into a tracking issue of the design implementation. ## What is the process? 1. Create a pull request with a design document based on the [template](./TEMPLATE.md) under this directory as `YYYY-MM-DD-my-feature.md`. 2. Discussion takes place, and the text is revised in response. 3. The design document is accepted or rejected when at least two committers reach consensus and no objection from the committer. -4. If accepted, [create a tracking issue](https://github.com/pingcap/ticdc/issues/new/choose) for the design document or convert one from a previous discuss issue. The tracking issue basically tracks subtasks and progress. And refer the tracking issue in the design document replacing placeholder in the template. +4. If accepted, [create a tracking issue](https://github.com/pingcap/tiflow/issues/new/choose) for the design document or convert one from a previous discuss issue. The tracking issue basically tracks subtasks and progress. And refer the tracking issue in the design document replacing placeholder in the template. 5. Merge the pull request of design. Please update the tracking issue according to the progress of succeeding implementation pull requests. diff --git a/docs/design/TEMPLATE.md b/docs/design/TEMPLATE.md index 4e4380e500f..a721bc68616 100644 --- a/docs/design/TEMPLATE.md +++ b/docs/design/TEMPLATE.md @@ -1,8 +1,8 @@ # TiCDC Design Documents - Author(s): [Author Name](http://github.com/your-github-id), [Co-Author Name](http://github.com/your-github-id), ... -- Discussion PR: https://github.com/pingcap/ticdc/pull/XXX -- Tracking Issue: https://github.com/pingcap/ticdc/issues/XXX +- Discussion PR: https://github.com/pingcap/tiflow/pull/XXX +- Tracking Issue: https://github.com/pingcap/tiflow/issues/XXX ## Table of Contents diff --git a/docs/design/package.json b/docs/design/package.json index 9cedcd2b15d..5a8d53b2360 100644 --- a/docs/design/package.json +++ b/docs/design/package.json @@ -8,7 +8,7 @@ }, "repository": { "type": "git", - "url": "git+https://github.com:pingcap/ticdc.git" + "url": "git+https://github.com:pingcap/tiflow.git" }, "keywords": [ "rfc" @@ -16,9 +16,9 @@ "author": "TiCDC Authors", "license": "Apache-2.0", "bugs": { - "url": "https://github.com/pingcap/ticdc/issues" + "url": "https://github.com/pingcap/tiflow/issues" }, - "homepage": "https://github.com/pingcap/ticdc/docs/design#readme", + "homepage": "https://github.com/pingcap/tiflow/docs/design#readme", "devDependencies": { "husky": "^5.0.9", "prettier": "^2.2.1" diff --git a/go.mod b/go.mod index 90b9c296732..e89a34a8f47 100644 --- a/go.mod +++ b/go.mod @@ -1,4 +1,4 @@ -module github.com/pingcap/ticdc +module github.com/pingcap/tiflow go 1.16 diff --git a/integration/framework/avro/kafka_docker_env.go b/integration/framework/avro/kafka_docker_env.go index c7de72441e7..9f504e9f368 100644 --- a/integration/framework/avro/kafka_docker_env.go +++ b/integration/framework/avro/kafka_docker_env.go @@ -22,7 +22,7 @@ import ( "github.com/integralist/go-findroot/find" "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/integration/framework" + "github.com/pingcap/tiflow/integration/framework" "go.uber.org/zap" ) diff --git a/integration/framework/avro/kafka_docker_env_test.go b/integration/framework/avro/kafka_docker_env_test.go index f6bf90755db..2662a55a3b9 100644 --- a/integration/framework/avro/kafka_docker_env_test.go +++ b/integration/framework/avro/kafka_docker_env_test.go @@ -18,7 +18,7 @@ import ( "testing" "github.com/pingcap/log" - "github.com/pingcap/ticdc/integration/framework" + "github.com/pingcap/tiflow/integration/framework" "github.com/stretchr/testify/require" ) diff --git a/integration/framework/avro/kafka_single_table.go b/integration/framework/avro/kafka_single_table.go index 7d9084038e5..0ace67e3c56 100644 --- a/integration/framework/avro/kafka_single_table.go +++ b/integration/framework/avro/kafka_single_table.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/integration/framework" + "github.com/pingcap/tiflow/integration/framework" "go.uber.org/zap" ) diff --git a/integration/framework/avro/kafka_single_table_test.go b/integration/framework/avro/kafka_single_table_test.go index bc09561e820..7d4f266e790 100644 --- a/integration/framework/avro/kafka_single_table_test.go +++ b/integration/framework/avro/kafka_single_table_test.go @@ -17,7 +17,7 @@ import ( "database/sql" "testing" - "github.com/pingcap/ticdc/integration/framework" + "github.com/pingcap/tiflow/integration/framework" "github.com/stretchr/testify/require" ) diff --git a/integration/framework/canal/kafka_docker_env.go b/integration/framework/canal/kafka_docker_env.go index f8c21bdf8d4..d3ce414e42e 100644 --- a/integration/framework/canal/kafka_docker_env.go +++ b/integration/framework/canal/kafka_docker_env.go @@ -21,7 +21,7 @@ import ( "github.com/integralist/go-findroot/find" "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/integration/framework" + "github.com/pingcap/tiflow/integration/framework" "go.uber.org/zap" ) diff --git a/integration/framework/canal/kafka_docker_env_test.go b/integration/framework/canal/kafka_docker_env_test.go index 57d7b340355..e150076feaa 100644 --- a/integration/framework/canal/kafka_docker_env_test.go +++ b/integration/framework/canal/kafka_docker_env_test.go @@ -18,7 +18,7 @@ import ( "testing" "github.com/pingcap/log" - "github.com/pingcap/ticdc/integration/framework" + "github.com/pingcap/tiflow/integration/framework" "github.com/stretchr/testify/require" ) diff --git a/integration/framework/canal/kafka_single_table.go b/integration/framework/canal/kafka_single_table.go index ad15240af15..c7c256a7efc 100644 --- a/integration/framework/canal/kafka_single_table.go +++ b/integration/framework/canal/kafka_single_table.go @@ -18,7 +18,7 @@ import ( "time" "github.com/pingcap/log" - "github.com/pingcap/ticdc/integration/framework" + "github.com/pingcap/tiflow/integration/framework" ) const ( diff --git a/integration/framework/canal/kafka_single_table_test.go b/integration/framework/canal/kafka_single_table_test.go index 91a96e67d7a..ed999aac93f 100644 --- a/integration/framework/canal/kafka_single_table_test.go +++ b/integration/framework/canal/kafka_single_table_test.go @@ -17,7 +17,7 @@ import ( "database/sql" "testing" - "github.com/pingcap/ticdc/integration/framework" + "github.com/pingcap/tiflow/integration/framework" "github.com/stretchr/testify/require" ) diff --git a/integration/framework/docker_compose_op.go b/integration/framework/docker_compose_op.go index e42a7a570f4..8f729be09dc 100644 --- a/integration/framework/docker_compose_op.go +++ b/integration/framework/docker_compose_op.go @@ -21,8 +21,8 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/retry" + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/retry" "go.uber.org/zap" "golang.org/x/net/context" ) diff --git a/integration/framework/docker_env.go b/integration/framework/docker_env.go index 1e687912839..40c3405ea0c 100644 --- a/integration/framework/docker_env.go +++ b/integration/framework/docker_env.go @@ -20,8 +20,8 @@ import ( "os/exec" "github.com/pingcap/log" - cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/retry" + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/retry" "go.uber.org/zap" ) diff --git a/integration/framework/mysql/docker_env.go b/integration/framework/mysql/docker_env.go index 94a9ce16c8e..28e2ce66917 100644 --- a/integration/framework/mysql/docker_env.go +++ b/integration/framework/mysql/docker_env.go @@ -19,7 +19,7 @@ import ( "github.com/integralist/go-findroot/find" "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/integration/framework" + "github.com/pingcap/tiflow/integration/framework" "go.uber.org/zap" ) diff --git a/integration/framework/mysql/docker_env_test.go b/integration/framework/mysql/docker_env_test.go index 3ae307821c3..e99b0271190 100644 --- a/integration/framework/mysql/docker_env_test.go +++ b/integration/framework/mysql/docker_env_test.go @@ -18,7 +18,7 @@ import ( "testing" "github.com/pingcap/log" - "github.com/pingcap/ticdc/integration/framework" + "github.com/pingcap/tiflow/integration/framework" "github.com/stretchr/testify/require" ) diff --git a/integration/framework/mysql/single_table.go b/integration/framework/mysql/single_table.go index 3f6addc0b95..dc1ef419c49 100644 --- a/integration/framework/mysql/single_table.go +++ b/integration/framework/mysql/single_table.go @@ -18,7 +18,7 @@ import ( "time" "github.com/pingcap/log" - "github.com/pingcap/ticdc/integration/framework" + "github.com/pingcap/tiflow/integration/framework" ) const ( diff --git a/integration/framework/mysql/single_table_test.go b/integration/framework/mysql/single_table_test.go index ef1f8b8c19d..188f4e95ed0 100644 --- a/integration/framework/mysql/single_table_test.go +++ b/integration/framework/mysql/single_table_test.go @@ -17,7 +17,7 @@ import ( "database/sql" "testing" - "github.com/pingcap/ticdc/integration/framework" + "github.com/pingcap/tiflow/integration/framework" "github.com/stretchr/testify/require" ) diff --git a/integration/framework/sql_helper.go b/integration/framework/sql_helper.go index 2769c03c7d0..69a087110b1 100644 --- a/integration/framework/sql_helper.go +++ b/integration/framework/sql_helper.go @@ -24,7 +24,7 @@ import ( "github.com/jmoiron/sqlx" "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/pkg/quotes" + "github.com/pingcap/tiflow/pkg/quotes" "go.uber.org/zap" "go.uber.org/zap/zapcore" "upper.io/db.v3/lib/sqlbuilder" diff --git a/integration/integration.go b/integration/integration.go index e77d59df08e..63527f4d797 100644 --- a/integration/integration.go +++ b/integration/integration.go @@ -17,11 +17,11 @@ import ( "flag" "github.com/pingcap/log" - "github.com/pingcap/ticdc/integration/framework" - "github.com/pingcap/ticdc/integration/framework/avro" - "github.com/pingcap/ticdc/integration/framework/canal" - "github.com/pingcap/ticdc/integration/framework/mysql" - "github.com/pingcap/ticdc/integration/tests" + "github.com/pingcap/tiflow/integration/framework" + "github.com/pingcap/tiflow/integration/framework/avro" + "github.com/pingcap/tiflow/integration/framework/canal" + "github.com/pingcap/tiflow/integration/framework/mysql" + "github.com/pingcap/tiflow/integration/tests" "go.uber.org/zap" "go.uber.org/zap/zapcore" ) @@ -97,7 +97,7 @@ func testMySQL() { func testMySQLWithCheckingOldvValue() { env := mysql.NewDockerEnv(*dockerComposeFile) - env.DockerComposeOperator.ExecEnv = []string{"GO_FAILPOINTS=github.com/pingcap/ticdc/cdc/sink/SimpleMySQLSinkTester=return(ture)"} + env.DockerComposeOperator.ExecEnv = []string{"GO_FAILPOINTS=github.com/pingcap/tiflow/cdc/sink/SimpleMySQLSinkTester=return(ture)"} task := &mysql.SingleTableTask{TableName: "test", CheckOleValue: true} testCases := []framework.Task{ tests.NewSimpleCase(task), diff --git a/integration/tests/case_alter.go b/integration/tests/case_alter.go index 4dc972c88aa..34819c227c2 100644 --- a/integration/tests/case_alter.go +++ b/integration/tests/case_alter.go @@ -18,7 +18,7 @@ import ( "math/rand" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/integration/framework" + "github.com/pingcap/tiflow/integration/framework" ) // AlterCase is base impl of test case for alter operation diff --git a/integration/tests/case_composite_pkey.go b/integration/tests/case_composite_pkey.go index a969b47f4cd..b0252f8b5e0 100644 --- a/integration/tests/case_composite_pkey.go +++ b/integration/tests/case_composite_pkey.go @@ -15,7 +15,7 @@ package tests import ( "github.com/pingcap/errors" - "github.com/pingcap/ticdc/integration/framework" + "github.com/pingcap/tiflow/integration/framework" ) // CompositePKeyCase is base impl of test case for composite primary keys diff --git a/integration/tests/case_date_time.go b/integration/tests/case_date_time.go index ac060b94036..a278a9a4965 100644 --- a/integration/tests/case_date_time.go +++ b/integration/tests/case_date_time.go @@ -18,10 +18,10 @@ import ( "time" "github.com/pingcap/log" - "github.com/pingcap/ticdc/integration/framework" - "github.com/pingcap/ticdc/integration/framework/avro" - "github.com/pingcap/ticdc/integration/framework/canal" - "github.com/pingcap/ticdc/integration/framework/mysql" + "github.com/pingcap/tiflow/integration/framework" + "github.com/pingcap/tiflow/integration/framework/avro" + "github.com/pingcap/tiflow/integration/framework/canal" + "github.com/pingcap/tiflow/integration/framework/mysql" ) // DateTimeCase is base impl of test case for different types data diff --git a/integration/tests/case_delete.go b/integration/tests/case_delete.go index e301abb3325..b48a2ea8ed2 100644 --- a/integration/tests/case_delete.go +++ b/integration/tests/case_delete.go @@ -17,7 +17,7 @@ import ( "errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/integration/framework" + "github.com/pingcap/tiflow/integration/framework" "go.uber.org/zap" ) diff --git a/integration/tests/case_many_types.go b/integration/tests/case_many_types.go index bcca8313891..5af2a25e815 100644 --- a/integration/tests/case_many_types.go +++ b/integration/tests/case_many_types.go @@ -18,10 +18,10 @@ import ( "math" "time" - "github.com/pingcap/ticdc/integration/framework" - "github.com/pingcap/ticdc/integration/framework/avro" - "github.com/pingcap/ticdc/integration/framework/canal" - "github.com/pingcap/ticdc/integration/framework/mysql" + "github.com/pingcap/tiflow/integration/framework" + "github.com/pingcap/tiflow/integration/framework/avro" + "github.com/pingcap/tiflow/integration/framework/canal" + "github.com/pingcap/tiflow/integration/framework/mysql" ) // ManyTypesCase is base impl of test case for different types data diff --git a/integration/tests/case_simple.go b/integration/tests/case_simple.go index dc78560020b..834a3233d5f 100644 --- a/integration/tests/case_simple.go +++ b/integration/tests/case_simple.go @@ -15,7 +15,7 @@ package tests import ( "github.com/pingcap/errors" - "github.com/pingcap/ticdc/integration/framework" + "github.com/pingcap/tiflow/integration/framework" ) // SimpleCase is base impl of simple test case diff --git a/integration/tests/case_unsigned.go b/integration/tests/case_unsigned.go index 79660403ca0..7978678a5a5 100644 --- a/integration/tests/case_unsigned.go +++ b/integration/tests/case_unsigned.go @@ -14,7 +14,7 @@ package tests import ( - "github.com/pingcap/ticdc/integration/framework" + "github.com/pingcap/tiflow/integration/framework" ) // UnsignedCase is base impl of test case for unsigned int type data diff --git a/kafka_consumer/Dockerfile b/kafka_consumer/Dockerfile index 4eb646c42c7..d19b51b6007 100644 --- a/kafka_consumer/Dockerfile +++ b/kafka_consumer/Dockerfile @@ -1,9 +1,9 @@ FROM golang:1.13 as builder -WORKDIR /go/src/github.com/pingcap/ticdc +WORKDIR /go/src/github.com/pingcap/tiflow COPY . . RUN go mod download RUN make kafka_consumer FROM alpine:3.11 -COPY --from=builder /go/src/github.com/pingcap/ticdc/bin/cdc_kafka_consumer /cdc_kafka_consumer +COPY --from=builder /go/src/github.com/pingcap/tiflow/bin/cdc_kafka_consumer /cdc_kafka_consumer CMD [ "/cdc_kafka_consumer" ] diff --git a/pkg/actor/actor.go b/pkg/actor/actor.go index 7e5c609e77f..c3f559ee796 100644 --- a/pkg/actor/actor.go +++ b/pkg/actor/actor.go @@ -16,8 +16,8 @@ package actor import ( "context" - "github.com/pingcap/ticdc/pkg/actor/message" - cerrors "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/tiflow/pkg/actor/message" + cerrors "github.com/pingcap/tiflow/pkg/errors" ) var errMailboxFull = cerrors.ErrMailboxFull.FastGenByArgs() diff --git a/pkg/actor/actor_test.go b/pkg/actor/actor_test.go index 709168728bf..a47dedb28e5 100644 --- a/pkg/actor/actor_test.go +++ b/pkg/actor/actor_test.go @@ -19,8 +19,8 @@ import ( "testing" "time" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/actor/message" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/actor/message" "github.com/stretchr/testify/require" ) diff --git a/pkg/actor/message/message.go b/pkg/actor/message/message.go index 3e44a4f4604..4dd7d11fd0a 100644 --- a/pkg/actor/message/message.go +++ b/pkg/actor/message/message.go @@ -14,7 +14,7 @@ package message import ( - "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/tiflow/cdc/model" ) // Type is the type of Message diff --git a/pkg/actor/message/message_test.go b/pkg/actor/message/message_test.go index 4551ee9dfa1..10ad190ba79 100644 --- a/pkg/actor/message/message_test.go +++ b/pkg/actor/message/message_test.go @@ -17,7 +17,7 @@ import ( "encoding/json" "testing" - "github.com/pingcap/ticdc/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/leakutil" "github.com/stretchr/testify/require" ) diff --git a/pkg/actor/system.go b/pkg/actor/system.go index ef5dfe118a4..76a72ba3703 100644 --- a/pkg/actor/system.go +++ b/pkg/actor/system.go @@ -22,8 +22,8 @@ import ( "time" "github.com/pingcap/log" - "github.com/pingcap/ticdc/pkg/actor/message" - cerrors "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/tiflow/pkg/actor/message" + cerrors "github.com/pingcap/tiflow/pkg/errors" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" "golang.org/x/sync/errgroup" diff --git a/pkg/actor/system_test.go b/pkg/actor/system_test.go index daeb0647039..63a038ffda0 100644 --- a/pkg/actor/system_test.go +++ b/pkg/actor/system_test.go @@ -23,8 +23,8 @@ import ( "testing" "time" - "github.com/pingcap/ticdc/pkg/actor/message" - "github.com/pingcap/ticdc/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/actor/message" + "github.com/pingcap/tiflow/pkg/leakutil" dto "github.com/prometheus/client_model/go" "github.com/stretchr/testify/require" ) @@ -645,7 +645,7 @@ func TestSendAfterClose(t *testing.T) { } // Run the benchmark -// go test -benchmem -run='^$' -bench '^(BenchmarkActorSendReceive)$' github.com/pingcap/ticdc/pkg/actor +// go test -benchmem -run='^$' -bench '^(BenchmarkActorSendReceive)$' github.com/pingcap/tiflow/pkg/actor func BenchmarkActorSendReceive(b *testing.B) { ctx := context.Background() sys, router := makeTestSystem(b.Name(), b) @@ -687,7 +687,7 @@ func BenchmarkActorSendReceive(b *testing.B) { } // Run the benchmark -// go test -benchmem -run='^$' -bench '^(BenchmarkPollActor)$' github.com/pingcap/ticdc/pkg/actor +// go test -benchmem -run='^$' -bench '^(BenchmarkPollActor)$' github.com/pingcap/tiflow/pkg/actor func BenchmarkPollActor(b *testing.B) { ctx := context.Background() sys, router := makeTestSystem(b.Name(), b) diff --git a/pkg/applier/redo.go b/pkg/applier/redo.go index a4421250846..4538c739fc1 100644 --- a/pkg/applier/redo.go +++ b/pkg/applier/redo.go @@ -19,13 +19,13 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/redo" - "github.com/pingcap/ticdc/cdc/redo/reader" - "github.com/pingcap/ticdc/cdc/sink" - "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/filter" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/redo" + "github.com/pingcap/tiflow/cdc/redo/reader" + "github.com/pingcap/tiflow/cdc/sink" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/filter" "go.uber.org/zap" "golang.org/x/sync/errgroup" ) diff --git a/pkg/applier/redo_test.go b/pkg/applier/redo_test.go index e3a44270e88..b6572bdc65b 100644 --- a/pkg/applier/redo_test.go +++ b/pkg/applier/redo_test.go @@ -21,10 +21,10 @@ import ( "github.com/DATA-DOG/go-sqlmock" "github.com/phayes/freeport" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/redo" - "github.com/pingcap/ticdc/cdc/redo/reader" - "github.com/pingcap/ticdc/cdc/sink" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/redo" + "github.com/pingcap/tiflow/cdc/redo/reader" + "github.com/pingcap/tiflow/cdc/sink" "github.com/stretchr/testify/require" ) diff --git a/pkg/cmd/cli/cli.go b/pkg/cmd/cli/cli.go index 5386f084327..71e8d81b56c 100644 --- a/pkg/cmd/cli/cli.go +++ b/pkg/cmd/cli/cli.go @@ -20,9 +20,9 @@ import ( "github.com/chzyer/readline" "github.com/mattn/go-shellwords" - "github.com/pingcap/ticdc/pkg/cmd/factory" - "github.com/pingcap/ticdc/pkg/cmd/util" - "github.com/pingcap/ticdc/pkg/logutil" + "github.com/pingcap/tiflow/pkg/cmd/factory" + "github.com/pingcap/tiflow/pkg/cmd/util" + "github.com/pingcap/tiflow/pkg/logutil" "github.com/spf13/cobra" ) diff --git a/pkg/cmd/cli/cli_capture.go b/pkg/cmd/cli/cli_capture.go index 0ab748182f4..c490acd6b00 100644 --- a/pkg/cmd/cli/cli_capture.go +++ b/pkg/cmd/cli/cli_capture.go @@ -14,9 +14,9 @@ package cli import ( - cmdcontext "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/cmd/factory" - "github.com/pingcap/ticdc/pkg/cmd/util" + cmdcontext "github.com/pingcap/tiflow/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/cmd/factory" + "github.com/pingcap/tiflow/pkg/cmd/util" "github.com/spf13/cobra" ) diff --git a/pkg/cmd/cli/cli_capture_list.go b/pkg/cmd/cli/cli_capture_list.go index 4dc601c5e03..9f122b5b290 100644 --- a/pkg/cmd/cli/cli_capture_list.go +++ b/pkg/cmd/cli/cli_capture_list.go @@ -17,11 +17,11 @@ import ( "context" "github.com/pingcap/errors" - cmdcontext "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/cmd/factory" - "github.com/pingcap/ticdc/pkg/cmd/util" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/etcd" + cmdcontext "github.com/pingcap/tiflow/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/cmd/factory" + "github.com/pingcap/tiflow/pkg/cmd/util" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/etcd" "github.com/spf13/cobra" "go.etcd.io/etcd/clientv3/concurrency" ) diff --git a/pkg/cmd/cli/cli_changefeed.go b/pkg/cmd/cli/cli_changefeed.go index 88513074adf..99083ac9a86 100644 --- a/pkg/cmd/cli/cli_changefeed.go +++ b/pkg/cmd/cli/cli_changefeed.go @@ -14,9 +14,9 @@ package cli import ( - cmdcontext "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/cmd/factory" - "github.com/pingcap/ticdc/pkg/cmd/util" + cmdcontext "github.com/pingcap/tiflow/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/cmd/factory" + "github.com/pingcap/tiflow/pkg/cmd/util" "github.com/spf13/cobra" ) diff --git a/pkg/cmd/cli/cli_changefeed_create.go b/pkg/cmd/cli/cli_changefeed_create.go index c49e4c8cb2e..47f4bac173a 100644 --- a/pkg/cmd/cli/cli_changefeed_create.go +++ b/pkg/cmd/cli/cli_changefeed_create.go @@ -23,20 +23,20 @@ import ( "github.com/google/uuid" "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/sink" - cmdcontext "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/cmd/factory" - "github.com/pingcap/ticdc/pkg/cmd/util" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/cyclic" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/filter" - "github.com/pingcap/ticdc/pkg/security" - "github.com/pingcap/ticdc/pkg/txnutil/gc" - ticdcutil "github.com/pingcap/ticdc/pkg/util" - "github.com/pingcap/ticdc/pkg/version" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sink" + cmdcontext "github.com/pingcap/tiflow/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/cmd/factory" + "github.com/pingcap/tiflow/pkg/cmd/util" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/cyclic" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/filter" + "github.com/pingcap/tiflow/pkg/security" + "github.com/pingcap/tiflow/pkg/txnutil/gc" + ticdcutil "github.com/pingcap/tiflow/pkg/util" + "github.com/pingcap/tiflow/pkg/version" "github.com/spf13/cobra" "github.com/tikv/client-go/v2/oracle" pd "github.com/tikv/pd/client" diff --git a/pkg/cmd/cli/cli_changefeed_create_test.go b/pkg/cmd/cli/cli_changefeed_create_test.go index 229600ef98e..658346d38e6 100644 --- a/pkg/cmd/cli/cli_changefeed_create_test.go +++ b/pkg/cmd/cli/cli_changefeed_create_test.go @@ -20,8 +20,8 @@ import ( "testing" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/util/testleak" "github.com/spf13/cobra" ) diff --git a/pkg/cmd/cli/cli_changefeed_cyclic.go b/pkg/cmd/cli/cli_changefeed_cyclic.go index 971be8e710c..cc565666e70 100644 --- a/pkg/cmd/cli/cli_changefeed_cyclic.go +++ b/pkg/cmd/cli/cli_changefeed_cyclic.go @@ -14,7 +14,7 @@ package cli import ( - "github.com/pingcap/ticdc/pkg/cmd/factory" + "github.com/pingcap/tiflow/pkg/cmd/factory" "github.com/spf13/cobra" ) diff --git a/pkg/cmd/cli/cli_changefeed_cyclic_create_marktables.go b/pkg/cmd/cli/cli_changefeed_cyclic_create_marktables.go index 889b93f465f..490db7257df 100644 --- a/pkg/cmd/cli/cli_changefeed_cyclic_create_marktables.go +++ b/pkg/cmd/cli/cli_changefeed_cyclic_create_marktables.go @@ -14,11 +14,11 @@ package cli import ( - "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/cmd/factory" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/cyclic/mark" - "github.com/pingcap/ticdc/pkg/security" + "github.com/pingcap/tiflow/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/cmd/factory" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/cyclic/mark" + "github.com/pingcap/tiflow/pkg/security" "github.com/spf13/cobra" "github.com/tikv/client-go/v2/oracle" pd "github.com/tikv/pd/client" diff --git a/pkg/cmd/cli/cli_changefeed_helper.go b/pkg/cmd/cli/cli_changefeed_helper.go index b3028fae76c..97599c32d88 100644 --- a/pkg/cmd/cli/cli_changefeed_helper.go +++ b/pkg/cmd/cli/cli_changefeed_helper.go @@ -21,16 +21,16 @@ import ( "time" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc" - "github.com/pingcap/ticdc/cdc/entry" - "github.com/pingcap/ticdc/cdc/kv" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/cmd/util" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/filter" - "github.com/pingcap/ticdc/pkg/httputil" - "github.com/pingcap/ticdc/pkg/security" + "github.com/pingcap/tiflow/cdc" + "github.com/pingcap/tiflow/cdc/entry" + "github.com/pingcap/tiflow/cdc/kv" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/cmd/util" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/filter" + "github.com/pingcap/tiflow/pkg/httputil" + "github.com/pingcap/tiflow/pkg/security" "github.com/spf13/cobra" "github.com/tikv/client-go/v2/oracle" ) diff --git a/pkg/cmd/cli/cli_changefeed_helper_test.go b/pkg/cmd/cli/cli_changefeed_helper_test.go index c4eba8ea1c6..d75651f0a7c 100644 --- a/pkg/cmd/cli/cli_changefeed_helper_test.go +++ b/pkg/cmd/cli/cli_changefeed_helper_test.go @@ -18,7 +18,7 @@ import ( "path/filepath" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" "github.com/spf13/cobra" ) diff --git a/pkg/cmd/cli/cli_changefeed_list.go b/pkg/cmd/cli/cli_changefeed_list.go index 7225b0aa6c5..4e123a1a617 100644 --- a/pkg/cmd/cli/cli_changefeed_list.go +++ b/pkg/cmd/cli/cli_changefeed_list.go @@ -17,12 +17,12 @@ import ( "encoding/json" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc" - "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/cmd/factory" - "github.com/pingcap/ticdc/pkg/cmd/util" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/security" + "github.com/pingcap/tiflow/cdc" + "github.com/pingcap/tiflow/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/cmd/factory" + "github.com/pingcap/tiflow/pkg/cmd/util" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/security" "github.com/spf13/cobra" "go.uber.org/zap" ) diff --git a/pkg/cmd/cli/cli_changefeed_pause.go b/pkg/cmd/cli/cli_changefeed_pause.go index c7868d67bbc..99cb9ff4e0c 100644 --- a/pkg/cmd/cli/cli_changefeed_pause.go +++ b/pkg/cmd/cli/cli_changefeed_pause.go @@ -14,11 +14,11 @@ package cli import ( - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/cmd/factory" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/security" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/cmd/factory" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/security" "github.com/spf13/cobra" ) diff --git a/pkg/cmd/cli/cli_changefeed_query.go b/pkg/cmd/cli/cli_changefeed_query.go index 06058ff654c..aae9b471709 100644 --- a/pkg/cmd/cli/cli_changefeed_query.go +++ b/pkg/cmd/cli/cli_changefeed_query.go @@ -15,13 +15,13 @@ package cli import ( "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/cmd/factory" - "github.com/pingcap/ticdc/pkg/cmd/util" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/security" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/cmd/factory" + "github.com/pingcap/tiflow/pkg/cmd/util" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/security" "github.com/spf13/cobra" "go.uber.org/zap" ) diff --git a/pkg/cmd/cli/cli_changefeed_remove.go b/pkg/cmd/cli/cli_changefeed_remove.go index 86cc137ec5f..2abf814bfc4 100644 --- a/pkg/cmd/cli/cli_changefeed_remove.go +++ b/pkg/cmd/cli/cli_changefeed_remove.go @@ -14,11 +14,11 @@ package cli import ( - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/cmd/factory" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/security" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/cmd/factory" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/security" "github.com/spf13/cobra" ) diff --git a/pkg/cmd/cli/cli_changefeed_resume.go b/pkg/cmd/cli/cli_changefeed_resume.go index 71760060aea..f5c20886a4c 100644 --- a/pkg/cmd/cli/cli_changefeed_resume.go +++ b/pkg/cmd/cli/cli_changefeed_resume.go @@ -17,12 +17,12 @@ import ( "context" "encoding/json" - "github.com/pingcap/ticdc/cdc" - "github.com/pingcap/ticdc/cdc/model" - cmdcontext "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/cmd/factory" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/security" + "github.com/pingcap/tiflow/cdc" + "github.com/pingcap/tiflow/cdc/model" + cmdcontext "github.com/pingcap/tiflow/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/cmd/factory" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/security" "github.com/spf13/cobra" pd "github.com/tikv/pd/client" ) diff --git a/pkg/cmd/cli/cli_changefeed_statistics.go b/pkg/cmd/cli/cli_changefeed_statistics.go index 7d396b6697f..148304a9027 100644 --- a/pkg/cmd/cli/cli_changefeed_statistics.go +++ b/pkg/cmd/cli/cli_changefeed_statistics.go @@ -17,11 +17,11 @@ import ( "fmt" "time" - "github.com/pingcap/ticdc/pkg/etcd" + "github.com/pingcap/tiflow/pkg/etcd" - "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/cmd/factory" - "github.com/pingcap/ticdc/pkg/cmd/util" + "github.com/pingcap/tiflow/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/cmd/factory" + "github.com/pingcap/tiflow/pkg/cmd/util" "github.com/spf13/cobra" "github.com/tikv/client-go/v2/oracle" pd "github.com/tikv/pd/client" diff --git a/pkg/cmd/cli/cli_changefeed_update.go b/pkg/cmd/cli/cli_changefeed_update.go index 7387685fee0..31c2e8905c4 100644 --- a/pkg/cmd/cli/cli_changefeed_update.go +++ b/pkg/cmd/cli/cli_changefeed_update.go @@ -17,15 +17,15 @@ import ( "fmt" "strings" - "github.com/pingcap/ticdc/pkg/etcd" + "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - cmdcontext "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/cmd/factory" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/security" + "github.com/pingcap/tiflow/cdc/model" + cmdcontext "github.com/pingcap/tiflow/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/cmd/factory" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/security" "github.com/r3labs/diff" "github.com/spf13/cobra" "github.com/spf13/pflag" diff --git a/pkg/cmd/cli/cli_changefeed_update_test.go b/pkg/cmd/cli/cli_changefeed_update_test.go index 5808d299198..0ff3445be8a 100644 --- a/pkg/cmd/cli/cli_changefeed_update_test.go +++ b/pkg/cmd/cli/cli_changefeed_update_test.go @@ -20,8 +20,8 @@ import ( "github.com/pingcap/check" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type changefeedUpdateSuite struct{} diff --git a/pkg/cmd/cli/cli_processor.go b/pkg/cmd/cli/cli_processor.go index fd5dd036656..f2aa924cb20 100644 --- a/pkg/cmd/cli/cli_processor.go +++ b/pkg/cmd/cli/cli_processor.go @@ -14,9 +14,9 @@ package cli import ( - cmdcontext "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/cmd/factory" - "github.com/pingcap/ticdc/pkg/cmd/util" + cmdcontext "github.com/pingcap/tiflow/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/cmd/factory" + "github.com/pingcap/tiflow/pkg/cmd/util" "github.com/spf13/cobra" ) diff --git a/pkg/cmd/cli/cli_processor_list.go b/pkg/cmd/cli/cli_processor_list.go index 64c71b60684..bd8a1ff18e2 100644 --- a/pkg/cmd/cli/cli_processor_list.go +++ b/pkg/cmd/cli/cli_processor_list.go @@ -14,10 +14,10 @@ package cli import ( - "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/cmd/factory" - "github.com/pingcap/ticdc/pkg/cmd/util" - "github.com/pingcap/ticdc/pkg/etcd" + "github.com/pingcap/tiflow/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/cmd/factory" + "github.com/pingcap/tiflow/pkg/cmd/util" + "github.com/pingcap/tiflow/pkg/etcd" "github.com/spf13/cobra" ) diff --git a/pkg/cmd/cli/cli_processor_query.go b/pkg/cmd/cli/cli_processor_query.go index f52a4958942..0d4d704449c 100644 --- a/pkg/cmd/cli/cli_processor_query.go +++ b/pkg/cmd/cli/cli_processor_query.go @@ -14,12 +14,12 @@ package cli import ( - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/cmd/factory" - "github.com/pingcap/ticdc/pkg/cmd/util" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/etcd" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/cmd/factory" + "github.com/pingcap/tiflow/pkg/cmd/util" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/etcd" "github.com/spf13/cobra" ) diff --git a/pkg/cmd/cli/cli_test.go b/pkg/cmd/cli/cli_test.go index 2ffcee94206..8e900b70707 100644 --- a/pkg/cmd/cli/cli_test.go +++ b/pkg/cmd/cli/cli_test.go @@ -17,7 +17,7 @@ import ( "os" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type cliSuite struct{} diff --git a/pkg/cmd/cli/cli_tso.go b/pkg/cmd/cli/cli_tso.go index 61b31c2c127..45fdd81b640 100644 --- a/pkg/cmd/cli/cli_tso.go +++ b/pkg/cmd/cli/cli_tso.go @@ -14,7 +14,7 @@ package cli import ( - "github.com/pingcap/ticdc/pkg/cmd/factory" + "github.com/pingcap/tiflow/pkg/cmd/factory" "github.com/spf13/cobra" ) diff --git a/pkg/cmd/cli/cli_tso_query.go b/pkg/cmd/cli/cli_tso_query.go index 0a57924371f..2bfb2764abf 100644 --- a/pkg/cmd/cli/cli_tso_query.go +++ b/pkg/cmd/cli/cli_tso_query.go @@ -14,8 +14,8 @@ package cli import ( - "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/cmd/factory" + "github.com/pingcap/tiflow/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/cmd/factory" "github.com/spf13/cobra" "github.com/tikv/client-go/v2/oracle" pd "github.com/tikv/pd/client" diff --git a/pkg/cmd/cli/cli_unsafe.go b/pkg/cmd/cli/cli_unsafe.go index d8ea6c11077..354fb3db993 100644 --- a/pkg/cmd/cli/cli_unsafe.go +++ b/pkg/cmd/cli/cli_unsafe.go @@ -18,7 +18,7 @@ import ( "strings" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/pkg/cmd/factory" + "github.com/pingcap/tiflow/pkg/cmd/factory" "github.com/spf13/cobra" ) diff --git a/pkg/cmd/cli/cli_unsafe_delete_service_gc_safepoint.go b/pkg/cmd/cli/cli_unsafe_delete_service_gc_safepoint.go index 3be4ce95c06..0ccd0cc2e26 100644 --- a/pkg/cmd/cli/cli_unsafe_delete_service_gc_safepoint.go +++ b/pkg/cmd/cli/cli_unsafe_delete_service_gc_safepoint.go @@ -15,9 +15,9 @@ package cli import ( "github.com/pingcap/errors" - "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/cmd/factory" - "github.com/pingcap/ticdc/pkg/txnutil/gc" + "github.com/pingcap/tiflow/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/cmd/factory" + "github.com/pingcap/tiflow/pkg/txnutil/gc" "github.com/spf13/cobra" pd "github.com/tikv/pd/client" ) diff --git a/pkg/cmd/cli/cli_unsafe_reset.go b/pkg/cmd/cli/cli_unsafe_reset.go index 5d1bf957f87..98a0d76beaa 100644 --- a/pkg/cmd/cli/cli_unsafe_reset.go +++ b/pkg/cmd/cli/cli_unsafe_reset.go @@ -15,10 +15,10 @@ package cli import ( "github.com/pingcap/errors" - "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/cmd/factory" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/txnutil/gc" + "github.com/pingcap/tiflow/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/cmd/factory" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/txnutil/gc" "github.com/spf13/cobra" pd "github.com/tikv/pd/client" ) diff --git a/pkg/cmd/cli/cli_unsafe_show_metadata.go b/pkg/cmd/cli/cli_unsafe_show_metadata.go index 6040d62adce..3fa726696e0 100644 --- a/pkg/cmd/cli/cli_unsafe_show_metadata.go +++ b/pkg/cmd/cli/cli_unsafe_show_metadata.go @@ -15,9 +15,9 @@ package cli import ( "github.com/pingcap/errors" - "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/cmd/factory" - "github.com/pingcap/ticdc/pkg/etcd" + "github.com/pingcap/tiflow/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/cmd/factory" + "github.com/pingcap/tiflow/pkg/etcd" "github.com/spf13/cobra" ) diff --git a/pkg/cmd/cmd.go b/pkg/cmd/cmd.go index 7d4fda63517..681ce28b2a0 100644 --- a/pkg/cmd/cmd.go +++ b/pkg/cmd/cmd.go @@ -16,10 +16,10 @@ package cmd import ( "os" - "github.com/pingcap/ticdc/pkg/cmd/cli" - "github.com/pingcap/ticdc/pkg/cmd/redo" - "github.com/pingcap/ticdc/pkg/cmd/server" - "github.com/pingcap/ticdc/pkg/cmd/version" + "github.com/pingcap/tiflow/pkg/cmd/cli" + "github.com/pingcap/tiflow/pkg/cmd/redo" + "github.com/pingcap/tiflow/pkg/cmd/server" + "github.com/pingcap/tiflow/pkg/cmd/version" "github.com/spf13/cobra" ) diff --git a/pkg/cmd/factory/factory.go b/pkg/cmd/factory/factory.go index 1a9429267cc..48eb31be6d5 100644 --- a/pkg/cmd/factory/factory.go +++ b/pkg/cmd/factory/factory.go @@ -17,9 +17,9 @@ import ( "crypto/tls" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/pkg/cmd/util" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/security" + "github.com/pingcap/tiflow/pkg/cmd/util" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/security" "github.com/spf13/cobra" pd "github.com/tikv/pd/client" "google.golang.org/grpc" diff --git a/pkg/cmd/factory/factory_impl.go b/pkg/cmd/factory/factory_impl.go index 0f13c21cf9d..fc4e9c77518 100644 --- a/pkg/cmd/factory/factory_impl.go +++ b/pkg/cmd/factory/factory_impl.go @@ -19,10 +19,10 @@ import ( "time" "github.com/pingcap/errors" - cmdconetxt "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/security" - "github.com/pingcap/ticdc/pkg/version" + cmdconetxt "github.com/pingcap/tiflow/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/security" + "github.com/pingcap/tiflow/pkg/version" pd "github.com/tikv/pd/client" "go.etcd.io/etcd/clientv3" etcdlogutil "go.etcd.io/etcd/pkg/logutil" @@ -102,7 +102,7 @@ func (f *factoryImpl) EtcdClient() (*etcd.CDCEtcdClient, error) { LogConfig: &logConfig, DialTimeout: 30 * time.Second, // TODO(hi-rustin): add gRPC metrics to Options. - // See also: https://github.com/pingcap/ticdc/pull/2341#discussion_r673018537. + // See also: https://github.com/pingcap/tiflow/pull/2341#discussion_r673018537. DialOptions: []grpc.DialOption{ grpcTLSOption, grpc.WithBlock(), @@ -141,7 +141,7 @@ func (f factoryImpl) PdClient() (pd.Client, error) { pdClient, err := pd.NewClientWithContext( ctx, pdEndpoints, credential.PDSecurityOption(), // TODO(hi-rustin): add gRPC metrics to Options. - // See also: https://github.com/pingcap/ticdc/pull/2341#discussion_r673032407. + // See also: https://github.com/pingcap/tiflow/pull/2341#discussion_r673032407. pd.WithGRPCDialOptions( grpcTLSOption, grpc.WithBlock(), diff --git a/pkg/cmd/redo/apply.go b/pkg/cmd/redo/apply.go index 1cbcc6ee92a..55993334694 100644 --- a/pkg/cmd/redo/apply.go +++ b/pkg/cmd/redo/apply.go @@ -14,8 +14,8 @@ package redo import ( - "github.com/pingcap/ticdc/pkg/applier" - cmdcontext "github.com/pingcap/ticdc/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/applier" + cmdcontext "github.com/pingcap/tiflow/pkg/cmd/context" "github.com/spf13/cobra" ) diff --git a/pkg/cmd/redo/meta.go b/pkg/cmd/redo/meta.go index 1e87e9f1a97..e0ab7fd9ec8 100644 --- a/pkg/cmd/redo/meta.go +++ b/pkg/cmd/redo/meta.go @@ -14,8 +14,8 @@ package redo import ( - "github.com/pingcap/ticdc/pkg/applier" - cmdcontext "github.com/pingcap/ticdc/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/applier" + cmdcontext "github.com/pingcap/tiflow/pkg/cmd/context" "github.com/spf13/cobra" ) diff --git a/pkg/cmd/redo/redo.go b/pkg/cmd/redo/redo.go index 5da0cbaf5c9..597c68126ae 100644 --- a/pkg/cmd/redo/redo.go +++ b/pkg/cmd/redo/redo.go @@ -14,8 +14,8 @@ package redo import ( - "github.com/pingcap/ticdc/pkg/cmd/util" - "github.com/pingcap/ticdc/pkg/logutil" + "github.com/pingcap/tiflow/pkg/cmd/util" + "github.com/pingcap/tiflow/pkg/logutil" "github.com/spf13/cobra" ) diff --git a/pkg/cmd/server/server.go b/pkg/cmd/server/server.go index 38ed5220165..f7d0e67f327 100644 --- a/pkg/cmd/server/server.go +++ b/pkg/cmd/server/server.go @@ -22,17 +22,17 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc" - "github.com/pingcap/ticdc/cdc/sorter/unified" - cmdcontext "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/cmd/util" - "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/logutil" - "github.com/pingcap/ticdc/pkg/security" - ticdcutil "github.com/pingcap/ticdc/pkg/util" - "github.com/pingcap/ticdc/pkg/version" ticonfig "github.com/pingcap/tidb/config" + "github.com/pingcap/tiflow/cdc" + "github.com/pingcap/tiflow/cdc/sorter/unified" + cmdcontext "github.com/pingcap/tiflow/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/cmd/util" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/logutil" + "github.com/pingcap/tiflow/pkg/security" + ticdcutil "github.com/pingcap/tiflow/pkg/util" + "github.com/pingcap/tiflow/pkg/version" "github.com/spf13/cobra" "github.com/spf13/pflag" "go.uber.org/zap" diff --git a/pkg/cmd/server/server_test.go b/pkg/cmd/server/server_test.go index 08cac96c4cf..e0be7fb6600 100644 --- a/pkg/cmd/server/server_test.go +++ b/pkg/cmd/server/server_test.go @@ -21,9 +21,9 @@ import ( "time" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/util/testleak" ticonfig "github.com/pingcap/tidb/config" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/util/testleak" "github.com/spf13/cobra" ) diff --git a/pkg/cmd/util/helper.go b/pkg/cmd/util/helper.go index 4ecce522704..94348175270 100644 --- a/pkg/cmd/util/helper.go +++ b/pkg/cmd/util/helper.go @@ -25,10 +25,10 @@ import ( "github.com/BurntSushi/toml" "github.com/pingcap/errors" "github.com/pingcap/log" - cmdconetxt "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/logutil" - "github.com/pingcap/ticdc/pkg/version" + cmdconetxt "github.com/pingcap/tiflow/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/logutil" + "github.com/pingcap/tiflow/pkg/version" "github.com/spf13/cobra" "go.uber.org/zap" "golang.org/x/net/http/httpproxy" diff --git a/pkg/cmd/util/helper_test.go b/pkg/cmd/util/helper_test.go index fdf45cdcc02..e16367b9ee4 100644 --- a/pkg/cmd/util/helper_test.go +++ b/pkg/cmd/util/helper_test.go @@ -21,8 +21,8 @@ import ( "testing" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/util/testleak" "github.com/spf13/cobra" ) diff --git a/pkg/cmd/version/version.go b/pkg/cmd/version/version.go index f08e10a58fa..c5c31efc089 100644 --- a/pkg/cmd/version/version.go +++ b/pkg/cmd/version/version.go @@ -14,7 +14,7 @@ package version import ( - "github.com/pingcap/ticdc/pkg/version" + "github.com/pingcap/tiflow/pkg/version" "github.com/spf13/cobra" ) diff --git a/pkg/config/config.go b/pkg/config/config.go index c1c41b61481..f0b9d8a508b 100644 --- a/pkg/config/config.go +++ b/pkg/config/config.go @@ -23,9 +23,9 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/pkg/config/outdated" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/security" + "github.com/pingcap/tiflow/pkg/config/outdated" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/security" "go.uber.org/zap" ) diff --git a/pkg/config/main_test.go b/pkg/config/main_test.go index 7dc1da8e834..ce570fc725a 100644 --- a/pkg/config/main_test.go +++ b/pkg/config/main_test.go @@ -16,7 +16,7 @@ package config import ( "testing" - "github.com/pingcap/ticdc/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/leakutil" ) func TestMain(m *testing.M) { diff --git a/pkg/config/sorter.go b/pkg/config/sorter.go index 7ddef8b9218..8639dcc313b 100644 --- a/pkg/config/sorter.go +++ b/pkg/config/sorter.go @@ -13,7 +13,7 @@ package config -import cerror "github.com/pingcap/ticdc/pkg/errors" +import cerror "github.com/pingcap/tiflow/pkg/errors" // SorterConfig represents sorter config for a changefeed type SorterConfig struct { diff --git a/pkg/context/context.go b/pkg/context/context.go index 1930d3f5db4..c840be8778f 100644 --- a/pkg/context/context.go +++ b/pkg/context/context.go @@ -18,13 +18,13 @@ import ( "time" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/kv" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/pdtime" - "github.com/pingcap/ticdc/pkg/version" tidbkv "github.com/pingcap/tidb/kv" + "github.com/pingcap/tiflow/cdc/kv" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/pdtime" + "github.com/pingcap/tiflow/pkg/version" "github.com/tikv/client-go/v2/oracle" pd "github.com/tikv/pd/client" "go.uber.org/zap" diff --git a/pkg/context/context_test.go b/pkg/context/context_test.go index 184459e5b18..069fb0c9dcc 100644 --- a/pkg/context/context_test.go +++ b/pkg/context/context_test.go @@ -19,8 +19,8 @@ import ( "time" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" "github.com/stretchr/testify/require" ) diff --git a/pkg/context/main_test.go b/pkg/context/main_test.go index 7b35690d10e..5a362d91cee 100644 --- a/pkg/context/main_test.go +++ b/pkg/context/main_test.go @@ -16,7 +16,7 @@ package context import ( "testing" - "github.com/pingcap/ticdc/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/leakutil" ) func TestMain(m *testing.M) { diff --git a/pkg/cyclic/filter.go b/pkg/cyclic/filter.go index daf4b1f3a0b..816c027284c 100644 --- a/pkg/cyclic/filter.go +++ b/pkg/cyclic/filter.go @@ -15,8 +15,8 @@ package cyclic import ( "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/cyclic/mark" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/cyclic/mark" "go.uber.org/zap" ) diff --git a/pkg/cyclic/filter_test.go b/pkg/cyclic/filter_test.go index 755c22b4605..c5351a6454e 100644 --- a/pkg/cyclic/filter_test.go +++ b/pkg/cyclic/filter_test.go @@ -17,8 +17,8 @@ import ( "testing" "github.com/davecgh/go-spew/spew" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/cyclic/mark" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/cyclic/mark" "github.com/stretchr/testify/require" ) diff --git a/pkg/cyclic/main_test.go b/pkg/cyclic/main_test.go index f5518e600ec..4412d154447 100644 --- a/pkg/cyclic/main_test.go +++ b/pkg/cyclic/main_test.go @@ -16,7 +16,7 @@ package cyclic import ( "testing" - "github.com/pingcap/ticdc/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/leakutil" ) func TestMain(m *testing.M) { diff --git a/pkg/cyclic/mark/main_test.go b/pkg/cyclic/mark/main_test.go index 7592c6e95a4..6606ac1c1d9 100644 --- a/pkg/cyclic/mark/main_test.go +++ b/pkg/cyclic/mark/main_test.go @@ -16,7 +16,7 @@ package mark import ( "testing" - "github.com/pingcap/ticdc/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/leakutil" ) func TestMain(m *testing.M) { diff --git a/pkg/cyclic/mark/mark.go b/pkg/cyclic/mark/mark.go index aa39ddaa259..6a027d04f10 100644 --- a/pkg/cyclic/mark/mark.go +++ b/pkg/cyclic/mark/mark.go @@ -22,9 +22,9 @@ import ( "github.com/go-sql-driver/mysql" "github.com/pingcap/errors" "github.com/pingcap/log" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/quotes" - "github.com/pingcap/ticdc/pkg/security" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/quotes" + "github.com/pingcap/tiflow/pkg/security" "go.uber.org/zap" ) diff --git a/pkg/cyclic/replication.go b/pkg/cyclic/replication.go index 1c8f3dedfbb..7392f3c3add 100644 --- a/pkg/cyclic/replication.go +++ b/pkg/cyclic/replication.go @@ -23,10 +23,10 @@ import ( "fmt" "strings" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/cyclic/mark" - "github.com/pingcap/ticdc/pkg/quotes" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/cyclic/mark" + "github.com/pingcap/tiflow/pkg/quotes" ) // RelaxSQLMode returns relaxed SQL mode, "STRICT_TRANS_TABLES" is removed. diff --git a/pkg/cyclic/replication_test.go b/pkg/cyclic/replication_test.go index 4017f5390ed..5855c7fd386 100644 --- a/pkg/cyclic/replication_test.go +++ b/pkg/cyclic/replication_test.go @@ -16,9 +16,9 @@ package cyclic import ( "testing" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/cyclic/mark" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/cyclic/mark" "github.com/stretchr/testify/require" ) diff --git a/pkg/errors/main_test.go b/pkg/errors/main_test.go index 5f60c43031f..95cbe6c4b89 100644 --- a/pkg/errors/main_test.go +++ b/pkg/errors/main_test.go @@ -16,7 +16,7 @@ package errors import ( "testing" - "github.com/pingcap/ticdc/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/leakutil" ) func TestMain(m *testing.M) { diff --git a/pkg/etcd/client.go b/pkg/etcd/client.go index 94a693da8d0..23a0e219a58 100644 --- a/pkg/etcd/client.go +++ b/pkg/etcd/client.go @@ -18,8 +18,8 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/retry" + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/retry" "github.com/prometheus/client_golang/prometheus" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/etcdserver/api/v3rpc/rpctypes" diff --git a/pkg/etcd/client_test.go b/pkg/etcd/client_test.go index dd310bad3ec..fbc9b3268b9 100644 --- a/pkg/etcd/client_test.go +++ b/pkg/etcd/client_test.go @@ -19,7 +19,7 @@ import ( "github.com/pingcap/check" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" "go.etcd.io/etcd/clientv3" ) diff --git a/pkg/etcd/etcd.go b/pkg/etcd/etcd.go index 2e4bd6f838b..f16841563b9 100644 --- a/pkg/etcd/etcd.go +++ b/pkg/etcd/etcd.go @@ -20,9 +20,9 @@ import ( "time" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/tiflow/cdc/model" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/util" "github.com/prometheus/client_golang/prometheus" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/clientv3/concurrency" diff --git a/pkg/etcd/etcd_test.go b/pkg/etcd/etcd_test.go index 3ba1e422e5e..0aac6963e07 100644 --- a/pkg/etcd/etcd_test.go +++ b/pkg/etcd/etcd_test.go @@ -20,9 +20,9 @@ import ( "testing" "time" - "github.com/pingcap/ticdc/cdc/model" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/tiflow/cdc/model" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/util" "go.etcd.io/etcd/clientv3/concurrency" "go.etcd.io/etcd/pkg/logutil" "go.uber.org/zap" @@ -30,7 +30,7 @@ import ( "golang.org/x/sync/errgroup" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/embed" ) diff --git a/pkg/etcd/etcdkey.go b/pkg/etcd/etcdkey.go index 8b91dc36921..b08d14758e9 100644 --- a/pkg/etcd/etcdkey.go +++ b/pkg/etcd/etcdkey.go @@ -17,7 +17,7 @@ import ( "strings" "github.com/pingcap/log" - cerror "github.com/pingcap/ticdc/pkg/errors" + cerror "github.com/pingcap/tiflow/pkg/errors" ) const ( diff --git a/pkg/etcd/etcdkey_test.go b/pkg/etcd/etcdkey_test.go index ba137f96f46..3c754638749 100644 --- a/pkg/etcd/etcdkey_test.go +++ b/pkg/etcd/etcdkey_test.go @@ -15,7 +15,7 @@ package etcd import ( "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type etcdkeySuite struct{} diff --git a/pkg/filelock/filelock.go b/pkg/filelock/filelock.go index 9f9630b6fc4..51dffa5a003 100644 --- a/pkg/filelock/filelock.go +++ b/pkg/filelock/filelock.go @@ -18,7 +18,7 @@ import ( "syscall" "github.com/pingcap/errors" - cerrors "github.com/pingcap/ticdc/pkg/errors" + cerrors "github.com/pingcap/tiflow/pkg/errors" ) // FileLock represents a file lock created by `flock`. diff --git a/pkg/filelock/filelock_test.go b/pkg/filelock/filelock_test.go index e414580b6cb..b4a033858ea 100644 --- a/pkg/filelock/filelock_test.go +++ b/pkg/filelock/filelock_test.go @@ -17,7 +17,7 @@ import ( "testing" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type fileLockSuite struct{} diff --git a/pkg/filter/filter.go b/pkg/filter/filter.go index afe5913c5cf..7ed7ffe14ea 100644 --- a/pkg/filter/filter.go +++ b/pkg/filter/filter.go @@ -14,12 +14,12 @@ package filter import ( - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/cyclic/mark" - cerror "github.com/pingcap/ticdc/pkg/errors" filterV1 "github.com/pingcap/tidb-tools/pkg/filter" filterV2 "github.com/pingcap/tidb-tools/pkg/table-filter" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/cyclic/mark" + cerror "github.com/pingcap/tiflow/pkg/errors" ) // Filter is a event filter implementation. diff --git a/pkg/filter/filter_test.go b/pkg/filter/filter_test.go index 89978efba57..619c64adb76 100644 --- a/pkg/filter/filter_test.go +++ b/pkg/filter/filter_test.go @@ -16,7 +16,7 @@ package filter import ( "testing" - "github.com/pingcap/ticdc/pkg/config" + "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tidb/parser/model" "github.com/stretchr/testify/require" diff --git a/pkg/filter/main_test.go b/pkg/filter/main_test.go index 63ca3092f24..78b342c14ef 100644 --- a/pkg/filter/main_test.go +++ b/pkg/filter/main_test.go @@ -16,7 +16,7 @@ package filter import ( "testing" - "github.com/pingcap/ticdc/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/leakutil" ) func TestMain(m *testing.M) { diff --git a/pkg/flags/main_test.go b/pkg/flags/main_test.go index 8ab6dc50cdd..87c2c6bb44b 100644 --- a/pkg/flags/main_test.go +++ b/pkg/flags/main_test.go @@ -16,7 +16,7 @@ package flags import ( "testing" - "github.com/pingcap/ticdc/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/leakutil" ) func TestMain(m *testing.M) { diff --git a/pkg/flags/urls.go b/pkg/flags/urls.go index 2cbd26cdc5e..912e745a344 100644 --- a/pkg/flags/urls.go +++ b/pkg/flags/urls.go @@ -17,7 +17,7 @@ import ( "strings" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/pkg/types" + "github.com/pingcap/tiflow/pkg/types" ) // URLsValue define a slice of URLs as a type diff --git a/pkg/httputil/httputil.go b/pkg/httputil/httputil.go index d60298307a0..012a84dc4d6 100644 --- a/pkg/httputil/httputil.go +++ b/pkg/httputil/httputil.go @@ -16,7 +16,7 @@ package httputil import ( "net/http" - "github.com/pingcap/ticdc/pkg/security" + "github.com/pingcap/tiflow/pkg/security" ) // Client wraps an HTTP client and support TLS requests. diff --git a/pkg/httputil/httputil_test.go b/pkg/httputil/httputil_test.go index 38899ba1741..7c76ae7f0f2 100644 --- a/pkg/httputil/httputil_test.go +++ b/pkg/httputil/httputil_test.go @@ -24,7 +24,7 @@ import ( "path/filepath" "testing" - "github.com/pingcap/ticdc/pkg/security" + "github.com/pingcap/tiflow/pkg/security" "github.com/stretchr/testify/require" "github.com/pingcap/tidb-tools/pkg/utils" diff --git a/pkg/httputil/main_test.go b/pkg/httputil/main_test.go index 7b585e9c242..b9ba503f7fc 100644 --- a/pkg/httputil/main_test.go +++ b/pkg/httputil/main_test.go @@ -16,7 +16,7 @@ package httputil import ( "testing" - "github.com/pingcap/ticdc/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/leakutil" ) func TestMain(m *testing.M) { diff --git a/pkg/leakutil/leak_helper_test.go b/pkg/leakutil/leak_helper_test.go index bb753690127..940cc065cbd 100644 --- a/pkg/leakutil/leak_helper_test.go +++ b/pkg/leakutil/leak_helper_test.go @@ -29,7 +29,7 @@ func TestSetUpLeakTest(t *testing.T) { func TestMain(m *testing.M) { opts := []goleak.Option{ - goleak.IgnoreTopFunction("github.com/pingcap/ticdc/pkg/leakutil.TestSetUpLeakTest.func1"), + goleak.IgnoreTopFunction("github.com/pingcap/tiflow/pkg/leakutil.TestSetUpLeakTest.func1"), } SetUpLeakTest(m, opts...) diff --git a/pkg/logutil/log.go b/pkg/logutil/log.go index 7790a2090a8..4daa3eb8f54 100644 --- a/pkg/logutil/log.go +++ b/pkg/logutil/log.go @@ -194,8 +194,8 @@ func initGRPCLogger(level zapcore.Level) error { logger := log.L().With(zap.String("name", "grpc")) // For gRPC 1.26.0, logging call stack: // - // github.com/pingcap/ticdc/pkg/util.levelToFunc.func1 - // github.com/pingcap/ticdc/pkg/util.(*grpcLoggerWriter).Write + // github.com/pingcap/tiflow/pkg/util.levelToFunc.func1 + // github.com/pingcap/tiflow/pkg/util.(*grpcLoggerWriter).Write // log.(*Logger).Output // log.(*Logger).Printf // google.golang.org/grpc/grpclog.(*loggerT).Infof diff --git a/pkg/logutil/log_test.go b/pkg/logutil/log_test.go index 3d41eea5bed..22acf2d4eb9 100644 --- a/pkg/logutil/log_test.go +++ b/pkg/logutil/log_test.go @@ -21,7 +21,7 @@ import ( "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" "go.uber.org/zap" "go.uber.org/zap/zapcore" ) diff --git a/pkg/notify/main_test.go b/pkg/notify/main_test.go index 59abd4ea67c..90da6a0613e 100644 --- a/pkg/notify/main_test.go +++ b/pkg/notify/main_test.go @@ -16,7 +16,7 @@ package notify import ( "testing" - "github.com/pingcap/ticdc/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/leakutil" ) func TestMain(m *testing.M) { diff --git a/pkg/notify/notify.go b/pkg/notify/notify.go index 477aa67d0cd..5b58a6759c6 100644 --- a/pkg/notify/notify.go +++ b/pkg/notify/notify.go @@ -17,7 +17,7 @@ import ( "sync" "time" - "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/tiflow/pkg/errors" ) // Notifier provides a one-to-many notification mechanism diff --git a/pkg/notify/notify_test.go b/pkg/notify/notify_test.go index 0ab41aeb61c..ddf2ae9403d 100644 --- a/pkg/notify/notify_test.go +++ b/pkg/notify/notify_test.go @@ -18,7 +18,7 @@ import ( "testing" "time" - "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/tiflow/pkg/errors" "github.com/stretchr/testify/require" ) diff --git a/pkg/orchestrator/etcd_worker.go b/pkg/orchestrator/etcd_worker.go index 102fe84db11..077f9ceb27e 100644 --- a/pkg/orchestrator/etcd_worker.go +++ b/pkg/orchestrator/etcd_worker.go @@ -21,9 +21,9 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/orchestrator/util" + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/orchestrator/util" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/clientv3/concurrency" "go.etcd.io/etcd/mvcc/mvccpb" diff --git a/pkg/orchestrator/etcd_worker_bank_test.go b/pkg/orchestrator/etcd_worker_bank_test.go index 6a7f3daec32..49a7561a50e 100644 --- a/pkg/orchestrator/etcd_worker_bank_test.go +++ b/pkg/orchestrator/etcd_worker_bank_test.go @@ -24,9 +24,9 @@ import ( "github.com/pingcap/check" "github.com/pingcap/log" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/orchestrator/util" - "github.com/pingcap/ticdc/pkg/util/testleak" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/orchestrator/util" + "github.com/pingcap/tiflow/pkg/util/testleak" "go.uber.org/zap" ) diff --git a/pkg/orchestrator/etcd_worker_test.go b/pkg/orchestrator/etcd_worker_test.go index ad0711f386d..1161399e877 100644 --- a/pkg/orchestrator/etcd_worker_test.go +++ b/pkg/orchestrator/etcd_worker_test.go @@ -26,10 +26,10 @@ import ( "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/log" - cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/orchestrator/util" - "github.com/pingcap/ticdc/pkg/util/testleak" + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/orchestrator/util" + "github.com/pingcap/tiflow/pkg/util/testleak" "github.com/prometheus/client_golang/prometheus" "go.etcd.io/etcd/clientv3" "go.uber.org/zap" diff --git a/pkg/orchestrator/interfaces.go b/pkg/orchestrator/interfaces.go index 6bf0eaf7b6e..4ab21ba6420 100644 --- a/pkg/orchestrator/interfaces.go +++ b/pkg/orchestrator/interfaces.go @@ -16,7 +16,7 @@ package orchestrator import ( "context" - "github.com/pingcap/ticdc/pkg/orchestrator/util" + "github.com/pingcap/tiflow/pkg/orchestrator/util" ) // Reactor is a stateful transform of states. diff --git a/pkg/orchestrator/reactor_state.go b/pkg/orchestrator/reactor_state.go index 9dc513d541a..3ee96d15a6f 100644 --- a/pkg/orchestrator/reactor_state.go +++ b/pkg/orchestrator/reactor_state.go @@ -19,10 +19,10 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/orchestrator/util" + "github.com/pingcap/tiflow/cdc/model" + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/orchestrator/util" "go.uber.org/zap" ) diff --git a/pkg/orchestrator/reactor_state_test.go b/pkg/orchestrator/reactor_state_test.go index 2541388d2ea..0124e31f28f 100644 --- a/pkg/orchestrator/reactor_state_test.go +++ b/pkg/orchestrator/reactor_state_test.go @@ -19,10 +19,10 @@ import ( "github.com/google/go-cmp/cmp" "github.com/google/go-cmp/cmp/cmpopts" "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/orchestrator/util" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/orchestrator/util" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type stateSuite struct{} diff --git a/pkg/orchestrator/reactor_state_tester.go b/pkg/orchestrator/reactor_state_tester.go index 5149cd34097..72b5f5e2c0a 100644 --- a/pkg/orchestrator/reactor_state_tester.go +++ b/pkg/orchestrator/reactor_state_tester.go @@ -16,8 +16,8 @@ package orchestrator import ( "github.com/pingcap/check" "github.com/pingcap/errors" - cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/orchestrator/util" + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/orchestrator/util" ) // ReactorStateTester is a helper struct for unit-testing an implementer of ReactorState diff --git a/pkg/orchestrator/util/key_utils_test.go b/pkg/orchestrator/util/key_utils_test.go index baa74594819..d84e3304382 100644 --- a/pkg/orchestrator/util/key_utils_test.go +++ b/pkg/orchestrator/util/key_utils_test.go @@ -17,7 +17,7 @@ import ( "testing" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" ) func Test(t *testing.T) { check.TestingT(t) } diff --git a/pkg/pdtime/acquirer.go b/pkg/pdtime/acquirer.go index 3fae739fa9d..4f9a3ad93ad 100644 --- a/pkg/pdtime/acquirer.go +++ b/pkg/pdtime/acquirer.go @@ -20,7 +20,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/pkg/retry" + "github.com/pingcap/tiflow/pkg/retry" "github.com/tikv/client-go/v2/oracle" pd "github.com/tikv/pd/client" "go.uber.org/zap" diff --git a/pkg/pdtime/main_test.go b/pkg/pdtime/main_test.go index 229d3e567f5..24c9b4e26ad 100644 --- a/pkg/pdtime/main_test.go +++ b/pkg/pdtime/main_test.go @@ -16,7 +16,7 @@ package pdtime import ( "testing" - "github.com/pingcap/ticdc/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/leakutil" ) func TestMain(m *testing.M) { diff --git a/pkg/pipeline/context.go b/pkg/pipeline/context.go index e1c927e04a3..12c2f69d6c8 100644 --- a/pkg/pipeline/context.go +++ b/pkg/pipeline/context.go @@ -13,7 +13,7 @@ package pipeline -import "github.com/pingcap/ticdc/pkg/context" +import "github.com/pingcap/tiflow/pkg/context" // NodeContext adds two functions from `coutext.Context` and created by pipeline type NodeContext interface { diff --git a/pkg/pipeline/main_test.go b/pkg/pipeline/main_test.go index f030ba777fb..218b606dcf5 100644 --- a/pkg/pipeline/main_test.go +++ b/pkg/pipeline/main_test.go @@ -16,7 +16,7 @@ package pipeline import ( "testing" - "github.com/pingcap/ticdc/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/leakutil" ) func TestMain(m *testing.M) { diff --git a/pkg/pipeline/message.go b/pkg/pipeline/message.go index 37d26a9ddbb..c7602fe3fab 100644 --- a/pkg/pipeline/message.go +++ b/pkg/pipeline/message.go @@ -13,7 +13,7 @@ package pipeline -import "github.com/pingcap/ticdc/cdc/model" +import "github.com/pingcap/tiflow/cdc/model" // MessageType is the type of Message type MessageType int diff --git a/pkg/pipeline/pipeline.go b/pkg/pipeline/pipeline.go index ead3f445b22..2155984d373 100644 --- a/pkg/pipeline/pipeline.go +++ b/pkg/pipeline/pipeline.go @@ -19,8 +19,8 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/pkg/context" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/tiflow/pkg/context" + cerror "github.com/pingcap/tiflow/pkg/errors" "go.uber.org/zap" ) diff --git a/pkg/pipeline/pipeline_test.go b/pkg/pipeline/pipeline_test.go index 65402b13bae..26ad07362e2 100644 --- a/pkg/pipeline/pipeline_test.go +++ b/pkg/pipeline/pipeline_test.go @@ -21,9 +21,9 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/context" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/context" + cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/stretchr/testify/require" "go.uber.org/zap" ) @@ -446,7 +446,7 @@ func (n *forward) Destroy(ctx NodeContext) error { } // Run the benchmark -// go test -benchmem -run='^$' -bench '^(BenchmarkPipeline)$' github.com/pingcap/ticdc/pkg/pipeline +// go test -benchmem -run='^$' -bench '^(BenchmarkPipeline)$' github.com/pingcap/tiflow/pkg/pipeline func BenchmarkPipeline(b *testing.B) { ctx := context.NewContext(stdCtx.Background(), &context.GlobalVars{}) runnersSize, outputChannelSize := 2, 64 diff --git a/pkg/pipeline/runner.go b/pkg/pipeline/runner.go index ce8b32f595b..3de79a3c6f5 100644 --- a/pkg/pipeline/runner.go +++ b/pkg/pipeline/runner.go @@ -18,7 +18,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/pkg/context" + "github.com/pingcap/tiflow/pkg/context" "go.uber.org/zap" ) diff --git a/pkg/pipeline/test.go b/pkg/pipeline/test.go index c640a4a27dd..ae9373c7fb8 100644 --- a/pkg/pipeline/test.go +++ b/pkg/pipeline/test.go @@ -13,7 +13,7 @@ package pipeline -import "github.com/pingcap/ticdc/pkg/context" +import "github.com/pingcap/tiflow/pkg/context" // SendMessageToNode4Test sends messages to specified `Node` through `Receive` in order. // This function is only for testing. diff --git a/pkg/quotes/main_test.go b/pkg/quotes/main_test.go index f24c77d5755..379e87fe51d 100644 --- a/pkg/quotes/main_test.go +++ b/pkg/quotes/main_test.go @@ -16,7 +16,7 @@ package quotes import ( "testing" - "github.com/pingcap/ticdc/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/leakutil" ) func TestMain(m *testing.M) { diff --git a/pkg/regionspan/span.go b/pkg/regionspan/span.go index 79fcb5eef7e..ed1dc39cf0c 100644 --- a/pkg/regionspan/span.go +++ b/pkg/regionspan/span.go @@ -19,10 +19,10 @@ import ( "fmt" "github.com/pingcap/log" - cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/codec" + cerror "github.com/pingcap/tiflow/pkg/errors" "go.uber.org/zap" ) diff --git a/pkg/retry/main_test.go b/pkg/retry/main_test.go index 69b296d23ff..e9f7b820f77 100644 --- a/pkg/retry/main_test.go +++ b/pkg/retry/main_test.go @@ -16,7 +16,7 @@ package retry import ( "testing" - "github.com/pingcap/ticdc/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/leakutil" ) func TestMain(m *testing.M) { diff --git a/pkg/retry/retry_with_opt.go b/pkg/retry/retry_with_opt.go index 7bf666777b1..c5482ebe130 100644 --- a/pkg/retry/retry_with_opt.go +++ b/pkg/retry/retry_with_opt.go @@ -20,7 +20,7 @@ import ( "time" "github.com/pingcap/errors" - cerror "github.com/pingcap/ticdc/pkg/errors" + cerror "github.com/pingcap/tiflow/pkg/errors" ) // Operation is the action need to retry diff --git a/pkg/scheduler/interface.go b/pkg/scheduler/interface.go index b3c83188bc2..c444f1ca0ae 100644 --- a/pkg/scheduler/interface.go +++ b/pkg/scheduler/interface.go @@ -15,7 +15,7 @@ package scheduler import ( "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/tiflow/cdc/model" ) // Scheduler is an abstraction for anything that provide the schedule table feature diff --git a/pkg/scheduler/main_test.go b/pkg/scheduler/main_test.go index 144a3f1f152..a98b8c7c955 100644 --- a/pkg/scheduler/main_test.go +++ b/pkg/scheduler/main_test.go @@ -16,7 +16,7 @@ package scheduler import ( "testing" - "github.com/pingcap/ticdc/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/leakutil" ) func TestMain(m *testing.M) { diff --git a/pkg/scheduler/table_number.go b/pkg/scheduler/table_number.go index 99ae38c5f18..7bfe59b9ace 100644 --- a/pkg/scheduler/table_number.go +++ b/pkg/scheduler/table_number.go @@ -13,7 +13,7 @@ package scheduler -import "github.com/pingcap/ticdc/cdc/model" +import "github.com/pingcap/tiflow/cdc/model" // TableNumberScheduler provides a feature that scheduling by the table number type TableNumberScheduler struct { diff --git a/pkg/scheduler/table_number_test.go b/pkg/scheduler/table_number_test.go index 2a65e2d38c8..dcba54d9599 100644 --- a/pkg/scheduler/table_number_test.go +++ b/pkg/scheduler/table_number_test.go @@ -17,7 +17,7 @@ import ( "fmt" "testing" - "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/tiflow/cdc/model" "github.com/stretchr/testify/require" ) diff --git a/pkg/scheduler/workload.go b/pkg/scheduler/workload.go index ce84e22369b..a340d8db55b 100644 --- a/pkg/scheduler/workload.go +++ b/pkg/scheduler/workload.go @@ -16,7 +16,7 @@ package scheduler import ( "math" - "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/tiflow/cdc/model" ) type workloads map[model.CaptureID]model.TaskWorkload diff --git a/pkg/scheduler/workload_test.go b/pkg/scheduler/workload_test.go index 3c752e413b8..9c924623cc7 100644 --- a/pkg/scheduler/workload_test.go +++ b/pkg/scheduler/workload_test.go @@ -17,7 +17,7 @@ import ( "fmt" "testing" - "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/tiflow/cdc/model" "github.com/stretchr/testify/require" ) diff --git a/pkg/security/credential.go b/pkg/security/credential.go index 31908ce8169..0961c6be6dd 100644 --- a/pkg/security/credential.go +++ b/pkg/security/credential.go @@ -19,8 +19,8 @@ import ( "encoding/pem" "os" - cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/tidb-tools/pkg/utils" + cerror "github.com/pingcap/tiflow/pkg/errors" pd "github.com/tikv/pd/client" "google.golang.org/grpc" "google.golang.org/grpc/credentials" diff --git a/pkg/tcpserver/main_test.go b/pkg/tcpserver/main_test.go index 6008181e554..6e0c2d2220a 100644 --- a/pkg/tcpserver/main_test.go +++ b/pkg/tcpserver/main_test.go @@ -16,7 +16,7 @@ package tcpserver import ( "testing" - "github.com/pingcap/ticdc/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/leakutil" ) func TestMain(m *testing.M) { diff --git a/pkg/tcpserver/tcp_server.go b/pkg/tcpserver/tcp_server.go index 6115ead1ede..24ce5b102ee 100644 --- a/pkg/tcpserver/tcp_server.go +++ b/pkg/tcpserver/tcp_server.go @@ -22,8 +22,8 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/security" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/security" "github.com/soheilhy/cmux" "go.uber.org/zap" "golang.org/x/sync/errgroup" diff --git a/pkg/tcpserver/tcp_server_test.go b/pkg/tcpserver/tcp_server_test.go index 654fade8489..8e873219cc8 100644 --- a/pkg/tcpserver/tcp_server_test.go +++ b/pkg/tcpserver/tcp_server_test.go @@ -27,8 +27,8 @@ import ( grpcTestingProto "github.com/grpc-ecosystem/go-grpc-middleware/testing/testproto" "github.com/integralist/go-findroot/find" "github.com/phayes/freeport" - "github.com/pingcap/ticdc/pkg/httputil" - "github.com/pingcap/ticdc/pkg/security" + "github.com/pingcap/tiflow/pkg/httputil" + "github.com/pingcap/tiflow/pkg/security" "github.com/stretchr/testify/require" "google.golang.org/grpc" ) diff --git a/pkg/txnutil/gc/gc_manager.go b/pkg/txnutil/gc/gc_manager.go index 1a158fa6f9d..0e4887b2261 100644 --- a/pkg/txnutil/gc/gc_manager.go +++ b/pkg/txnutil/gc/gc_manager.go @@ -20,10 +20,10 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" - cdcContext "github.com/pingcap/ticdc/pkg/context" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + cdcContext "github.com/pingcap/tiflow/pkg/context" + cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/tikv/client-go/v2/oracle" pd "github.com/tikv/pd/client" "go.uber.org/zap" diff --git a/pkg/txnutil/gc/gc_manager_test.go b/pkg/txnutil/gc/gc_manager_test.go index d650aa72443..368f879cd39 100644 --- a/pkg/txnutil/gc/gc_manager_test.go +++ b/pkg/txnutil/gc/gc_manager_test.go @@ -18,13 +18,13 @@ import ( "testing" "time" - "github.com/pingcap/ticdc/pkg/pdtime" + "github.com/pingcap/tiflow/pkg/pdtime" "github.com/pingcap/check" "github.com/pingcap/errors" - cdcContext "github.com/pingcap/ticdc/pkg/context" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/util/testleak" + cdcContext "github.com/pingcap/tiflow/pkg/context" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/util/testleak" "github.com/tikv/client-go/v2/oracle" ) diff --git a/pkg/txnutil/gc/gc_service.go b/pkg/txnutil/gc/gc_service.go index 56f51ee3d24..85e21d91909 100644 --- a/pkg/txnutil/gc/gc_service.go +++ b/pkg/txnutil/gc/gc_service.go @@ -19,8 +19,8 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/retry" + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/retry" pd "github.com/tikv/pd/client" "go.uber.org/zap" ) diff --git a/pkg/txnutil/gc/gc_service_test.go b/pkg/txnutil/gc/gc_service_test.go index 97aed0edddd..d32214a8b3e 100644 --- a/pkg/txnutil/gc/gc_service_test.go +++ b/pkg/txnutil/gc/gc_service_test.go @@ -19,7 +19,7 @@ import ( "github.com/pingcap/check" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" pd "github.com/tikv/pd/client" ) diff --git a/pkg/types/main_test.go b/pkg/types/main_test.go index 3d61dcf48e4..fd2845744b6 100644 --- a/pkg/types/main_test.go +++ b/pkg/types/main_test.go @@ -16,7 +16,7 @@ package types import ( "testing" - "github.com/pingcap/ticdc/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/leakutil" ) func TestMain(m *testing.M) { diff --git a/pkg/types/urls.go b/pkg/types/urls.go index 824e4ceadd9..b1d02927917 100644 --- a/pkg/types/urls.go +++ b/pkg/types/urls.go @@ -20,7 +20,7 @@ import ( "strings" "github.com/pingcap/errors" - cerror "github.com/pingcap/ticdc/pkg/errors" + cerror "github.com/pingcap/tiflow/pkg/errors" ) // URLs defines a slice of URLs as a type diff --git a/pkg/util/bitflag_test.go b/pkg/util/bitflag_test.go index e2a86b3ce99..b0308c32608 100644 --- a/pkg/util/bitflag_test.go +++ b/pkg/util/bitflag_test.go @@ -15,7 +15,7 @@ package util import ( "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" ) const ( diff --git a/pkg/util/ctx_test.go b/pkg/util/ctx_test.go index 889c8ac799b..9ab83004a61 100644 --- a/pkg/util/ctx_test.go +++ b/pkg/util/ctx_test.go @@ -17,8 +17,8 @@ import ( "context" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/util/testleak" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tiflow/pkg/util/testleak" "go.uber.org/zap" ) diff --git a/pkg/util/fileutil.go b/pkg/util/fileutil.go index 1ed97fe1c2f..20e64e5d738 100644 --- a/pkg/util/fileutil.go +++ b/pkg/util/fileutil.go @@ -22,8 +22,8 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" ) const ( diff --git a/pkg/util/fileutil_test.go b/pkg/util/fileutil_test.go index bc52919bba8..8b1809de80b 100644 --- a/pkg/util/fileutil_test.go +++ b/pkg/util/fileutil_test.go @@ -21,8 +21,8 @@ import ( "github.com/pingcap/check" "github.com/pingcap/failpoint" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type fileUtilSuite struct{} @@ -97,8 +97,8 @@ func (s *fileUtilSuite) TestCheckDataDirSatisfied(c *check.C) { conf.DataDir = dir config.StoreGlobalServerConfig(conf) - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/pkg/util/InjectCheckDataDirSatisfied", ""), check.IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/pkg/util/InjectCheckDataDirSatisfied", ""), check.IsNil) err := CheckDataDirSatisfied() c.Assert(err, check.IsNil) - c.Assert(failpoint.Disable("github.com/pingcap/ticdc/pkg/util/InjectCheckDataDirSatisfied"), check.IsNil) + c.Assert(failpoint.Disable("github.com/pingcap/tiflow/pkg/util/InjectCheckDataDirSatisfied"), check.IsNil) } diff --git a/pkg/util/overlap_merge_test.go b/pkg/util/overlap_merge_test.go index 5c84674718d..db8f329cf03 100644 --- a/pkg/util/overlap_merge_test.go +++ b/pkg/util/overlap_merge_test.go @@ -18,7 +18,7 @@ import ( "fmt" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type overlapSuite struct{} diff --git a/pkg/util/test_helper_test.go b/pkg/util/test_helper_test.go index 5b78d437dc2..07f9307c3fb 100644 --- a/pkg/util/test_helper_test.go +++ b/pkg/util/test_helper_test.go @@ -21,7 +21,7 @@ import ( "time" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" ) func Test(t *testing.T) { diff --git a/pkg/util/testleak/leaktest.go b/pkg/util/testleak/leaktest.go index b7cb125148e..9bbb7a6122c 100644 --- a/pkg/util/testleak/leaktest.go +++ b/pkg/util/testleak/leaktest.go @@ -53,11 +53,11 @@ func interestingGoroutines() (gs []string) { // library used by sarama, ref: https://github.com/rcrowley/go-metrics/pull/266 "github.com/rcrowley/go-metrics.(*meterArbiter).tick", // TODO: remove these two lines after unified sorter is fixed - "github.com/pingcap/ticdc/cdc/sorter/unified.newBackEndPool", - "github.com/pingcap/ticdc/cdc/sorter/unified.(*heapSorter).flush", + "github.com/pingcap/tiflow/cdc/sorter/unified.newBackEndPool", + "github.com/pingcap/tiflow/cdc/sorter/unified.(*heapSorter).flush", // kv client region worker pool - "github.com/pingcap/ticdc/cdc/kv.RunWorkerPool", - "github.com/pingcap/ticdc/pkg/workerpool.(*defaultPoolImpl).Run", + "github.com/pingcap/tiflow/cdc/kv.RunWorkerPool", + "github.com/pingcap/tiflow/pkg/workerpool.(*defaultPoolImpl).Run", // library used by log "gopkg.in/natefinch/lumberjack%2ev2.(*Logger).millRun", } diff --git a/pkg/util/tz.go b/pkg/util/tz.go index b1e18043542..77d03e17306 100644 --- a/pkg/util/tz.go +++ b/pkg/util/tz.go @@ -18,8 +18,8 @@ import ( "strings" "time" - cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/tidb/util/timeutil" + cerror "github.com/pingcap/tiflow/pkg/errors" ) // GetTimezone returns the timezone specified by the name diff --git a/pkg/util/tz_test.go b/pkg/util/tz_test.go index aa25a33c285..885b3fa1404 100644 --- a/pkg/util/tz_test.go +++ b/pkg/util/tz_test.go @@ -15,7 +15,7 @@ package util import ( "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type tzSuite struct{} diff --git a/pkg/version/check.go b/pkg/version/check.go index 29e4a1be5d9..63e9e29c2b5 100644 --- a/pkg/version/check.go +++ b/pkg/version/check.go @@ -25,10 +25,10 @@ import ( "github.com/coreos/go-semver/semver" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/httputil" - "github.com/pingcap/ticdc/pkg/security" + "github.com/pingcap/tiflow/cdc/model" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/httputil" + "github.com/pingcap/tiflow/pkg/security" pd "github.com/tikv/pd/client" "go.uber.org/zap" ) diff --git a/pkg/version/check_test.go b/pkg/version/check_test.go index 545a4d1e9b3..bd9c54404e0 100644 --- a/pkg/version/check_test.go +++ b/pkg/version/check_test.go @@ -23,7 +23,7 @@ import ( "github.com/coreos/go-semver/semver" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/tiflow/cdc/model" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" pd "github.com/tikv/pd/client" diff --git a/pkg/version/main_test.go b/pkg/version/main_test.go index 943c56a67bf..74534184dad 100644 --- a/pkg/version/main_test.go +++ b/pkg/version/main_test.go @@ -16,7 +16,7 @@ package version import ( "testing" - "github.com/pingcap/ticdc/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/leakutil" ) func TestMain(m *testing.M) { diff --git a/pkg/version/version.go b/pkg/version/version.go index 3c6d4b960cc..ba4d2f78e8a 100644 --- a/pkg/version/version.go +++ b/pkg/version/version.go @@ -18,7 +18,7 @@ import ( "github.com/coreos/go-semver/semver" "github.com/pingcap/log" - "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" ) diff --git a/pkg/workerpool/async_pool_impl.go b/pkg/workerpool/async_pool_impl.go index e2bf3ab6113..87c027e56b8 100644 --- a/pkg/workerpool/async_pool_impl.go +++ b/pkg/workerpool/async_pool_impl.go @@ -19,8 +19,8 @@ import ( "sync/atomic" "github.com/pingcap/errors" - cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/retry" + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/retry" "golang.org/x/sync/errgroup" ) diff --git a/pkg/workerpool/main_test.go b/pkg/workerpool/main_test.go index 97d7fb65de2..33b6d827a5f 100644 --- a/pkg/workerpool/main_test.go +++ b/pkg/workerpool/main_test.go @@ -16,7 +16,7 @@ package workerpool import ( "testing" - "github.com/pingcap/ticdc/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/leakutil" ) func TestMain(m *testing.M) { diff --git a/pkg/workerpool/pool_impl.go b/pkg/workerpool/pool_impl.go index 5927c12069d..2f10a4c8b65 100644 --- a/pkg/workerpool/pool_impl.go +++ b/pkg/workerpool/pool_impl.go @@ -22,8 +22,8 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/notify" + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/notify" "go.uber.org/zap" "golang.org/x/sync/errgroup" "golang.org/x/time/rate" diff --git a/pkg/workerpool/pool_test.go b/pkg/workerpool/pool_test.go index 3aad1b0b1f7..9e2592f89d2 100644 --- a/pkg/workerpool/pool_test.go +++ b/pkg/workerpool/pool_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - cerror "github.com/pingcap/ticdc/pkg/errors" + cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/stretchr/testify/require" "go.uber.org/zap" "golang.org/x/sync/errgroup" @@ -196,10 +196,10 @@ func TestCancelHandle(t *testing.T) { } } - err := failpoint.Enable("github.com/pingcap/ticdc/pkg/workerpool/addEventDelayPoint", "1*sleep(500)") + err := failpoint.Enable("github.com/pingcap/tiflow/pkg/workerpool/addEventDelayPoint", "1*sleep(500)") require.Nil(t, err) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/pkg/workerpool/addEventDelayPoint") + _ = failpoint.Disable("github.com/pingcap/tiflow/pkg/workerpool/addEventDelayPoint") }() handle.Unregister() @@ -228,10 +228,10 @@ func TestCancelTimer(t *testing.T) { return pool.Run(ctx) }) - err := failpoint.Enable("github.com/pingcap/ticdc/pkg/workerpool/unregisterDelayPoint", "sleep(5000)") + err := failpoint.Enable("github.com/pingcap/tiflow/pkg/workerpool/unregisterDelayPoint", "sleep(5000)") require.Nil(t, err) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/pkg/workerpool/unregisterDelayPoint") + _ = failpoint.Disable("github.com/pingcap/tiflow/pkg/workerpool/unregisterDelayPoint") }() handle := pool.RegisterEvent(func(ctx context.Context, event interface{}) error { @@ -541,7 +541,7 @@ func TestSynchronizeLog(t *testing.T) { } // Benchmark workerpool with ping-pong workflow. -// go test -benchmem -run='^$' -bench '^(BenchmarkWorkerpool)$' github.com/pingcap/ticdc/pkg/workerpool +// go test -benchmem -run='^$' -bench '^(BenchmarkWorkerpool)$' github.com/pingcap/tiflow/pkg/workerpool func BenchmarkWorkerpool(b *testing.B) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() diff --git a/testing_utils/cdc_state_checker/cdc_monitor.go b/testing_utils/cdc_state_checker/cdc_monitor.go index 5eac68220a3..5c2ee8a6dcf 100644 --- a/testing_utils/cdc_state_checker/cdc_monitor.go +++ b/testing_utils/cdc_state_checker/cdc_monitor.go @@ -19,9 +19,9 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/orchestrator" - "github.com/pingcap/ticdc/pkg/security" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/orchestrator" + "github.com/pingcap/tiflow/pkg/security" "github.com/prometheus/client_golang/prometheus" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/pkg/logutil" diff --git a/testing_utils/cdc_state_checker/main.go b/testing_utils/cdc_state_checker/main.go index 096f9aadca2..13c6364027b 100644 --- a/testing_utils/cdc_state_checker/main.go +++ b/testing_utils/cdc_state_checker/main.go @@ -19,7 +19,7 @@ import ( "strings" "github.com/pingcap/log" - "github.com/pingcap/ticdc/pkg/security" + "github.com/pingcap/tiflow/pkg/security" "go.uber.org/zap" "go.uber.org/zap/zapcore" ) diff --git a/testing_utils/cdc_state_checker/reactor.go b/testing_utils/cdc_state_checker/reactor.go index 2c0acd8d1e8..619b19fdf00 100644 --- a/testing_utils/cdc_state_checker/reactor.go +++ b/testing_utils/cdc_state_checker/reactor.go @@ -18,8 +18,8 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/orchestrator" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/orchestrator" "go.uber.org/zap" ) diff --git a/testing_utils/cdc_state_checker/state.go b/testing_utils/cdc_state_checker/state.go index f0bdd75810e..313f24d41fc 100644 --- a/testing_utils/cdc_state_checker/state.go +++ b/testing_utils/cdc_state_checker/state.go @@ -19,10 +19,10 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/orchestrator" - "github.com/pingcap/ticdc/pkg/orchestrator/util" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/orchestrator" + "github.com/pingcap/tiflow/pkg/orchestrator/util" "go.uber.org/zap" ) diff --git a/testing_utils/many_sorters_test/many_sorters.go b/testing_utils/many_sorters_test/many_sorters.go index 1772bc6f37d..dc4f3ae314c 100644 --- a/testing_utils/many_sorters_test/many_sorters.go +++ b/testing_utils/many_sorters_test/many_sorters.go @@ -26,11 +26,11 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/sorter" - "github.com/pingcap/ticdc/cdc/sorter/unified" - "github.com/pingcap/ticdc/pkg/config" - cerrors "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sorter" + "github.com/pingcap/tiflow/cdc/sorter/unified" + "github.com/pingcap/tiflow/pkg/config" + cerrors "github.com/pingcap/tiflow/pkg/errors" "go.uber.org/zap" "go.uber.org/zap/zapcore" "golang.org/x/sync/errgroup" @@ -45,7 +45,7 @@ var ( func main() { flag.Parse() - err := failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/sorterDebug", "return(true)") + err := failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterDebug", "return(true)") if err != nil { log.Fatal("Could not enable failpoint", zap.Error(err)) } diff --git a/testing_utils/sorter_stress_test/sorter_stress.go b/testing_utils/sorter_stress_test/sorter_stress.go index 57ace85345f..f32988cf2e4 100644 --- a/testing_utils/sorter_stress_test/sorter_stress.go +++ b/testing_utils/sorter_stress_test/sorter_stress.go @@ -25,10 +25,10 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/sorter" - "github.com/pingcap/ticdc/cdc/sorter/unified" - "github.com/pingcap/ticdc/pkg/config" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sorter" + "github.com/pingcap/tiflow/cdc/sorter/unified" + "github.com/pingcap/tiflow/pkg/config" "go.uber.org/zap" "golang.org/x/sync/errgroup" ) @@ -43,7 +43,7 @@ var ( func main() { flag.Parse() log.SetLevel(zap.DebugLevel) - err := failpoint.Enable("github.com/pingcap/ticdc/cdc/sorter/unified/sorterDebug", "return(true)") + err := failpoint.Enable("github.com/pingcap/tiflow/cdc/sorter/unified/sorterDebug", "return(true)") if err != nil { log.Fatal("Could not enable failpoint", zap.Error(err)) } diff --git a/tests/integration_tests/_utils/run_cdc_cli_tso_query b/tests/integration_tests/_utils/run_cdc_cli_tso_query index 33ab8282e28..fe30a544ff5 100755 --- a/tests/integration_tests/_utils/run_cdc_cli_tso_query +++ b/tests/integration_tests/_utils/run_cdc_cli_tso_query @@ -23,7 +23,7 @@ pd_port=${2} tso=$(run_cdc_cli tso query --pd=http://${pd_host}:${pd_port}) # make sure get tso only # the tso got above is: -# "427768583921860609 PASS coverage: 2.7% of statements in github.com/pingcap/ticdc/..." +# "427768583921860609 PASS coverage: 2.7% of statements in github.com/pingcap/tiflow/..." # and only "427768583921860609" is a real tso echo $tso | awk -F " " '{print $1}' diff --git a/tests/integration_tests/availability/owner.sh b/tests/integration_tests/availability/owner.sh index 5eed09aae63..d8975521862 100755 --- a/tests/integration_tests/availability/owner.sh +++ b/tests/integration_tests/availability/owner.sh @@ -159,7 +159,7 @@ function test_owner_cleanup_stale_tasks() { function test_owner_retryable_error() { echo "run test case test_owner_retryable_error" - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/capture/capture-campaign-compacted-error=1*return(true)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/capture/capture-campaign-compacted-error=1*return(true)' # start a capture server run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix test_owner_retryable_error.server1 @@ -171,7 +171,7 @@ function test_owner_retryable_error() { echo "owner pid:" $owner_pid echo "owner id" $owner_id - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/owner/owner-run-with-error=1*return(true);github.com/pingcap/ticdc/cdc/capture/capture-resign-failed=1*return(true)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/owner-run-with-error=1*return(true);github.com/pingcap/tiflow/cdc/capture/capture-resign-failed=1*return(true)' # run another server run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix test_owner_retryable_error.server2 --addr "127.0.0.1:8301" @@ -196,7 +196,7 @@ function test_owner_retryable_error() { function test_gap_between_watch_capture() { echo "run test case test_gap_between_watch_capture" - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/owner/sleep-in-owner-tick=1*sleep(6000)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/sleep-in-owner-tick=1*sleep(6000)' # start a capture server run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix test_gap_between_watch_capture.server1 diff --git a/tests/integration_tests/bank/case.go b/tests/integration_tests/bank/case.go index 54220be7559..c3aa41fab05 100644 --- a/tests/integration_tests/bank/case.go +++ b/tests/integration_tests/bank/case.go @@ -25,8 +25,8 @@ import ( _ "github.com/go-sql-driver/mysql" // MySQL driver "github.com/pingcap/errors" "github.com/pingcap/log" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/retry" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/retry" "go.uber.org/zap" "golang.org/x/sync/errgroup" ) diff --git a/tests/integration_tests/capture_session_done_during_task/run.sh b/tests/integration_tests/capture_session_done_during_task/run.sh index 89c5ad36f3b..0205d74943d 100644 --- a/tests/integration_tests/capture_session_done_during_task/run.sh +++ b/tests/integration_tests/capture_session_done_during_task/run.sh @@ -28,7 +28,7 @@ function run() { run_sql "CREATE table capture_session_done_during_task.t (id int primary key auto_increment, a int)" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) run_sql "INSERT INTO capture_session_done_during_task.t values (),(),(),(),(),(),()" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/processor/processorManagerHandleNewChangefeedDelay=sleep(2000)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/processor/processorManagerHandleNewChangefeedDelay=sleep(2000)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8300" --pd $pd_addr changefeed_id=$(cdc cli changefeed create --pd=$pd_addr --start-ts=$start_ts --sink-uri="$SINK_URI" 2>&1 | tail -n2 | head -n1 | awk '{print $2}') # wait task is dispatched diff --git a/tests/integration_tests/capture_suicide_while_balance_table/run.sh b/tests/integration_tests/capture_suicide_while_balance_table/run.sh index 8e0597921bd..3accdee5b4f 100644 --- a/tests/integration_tests/capture_suicide_while_balance_table/run.sh +++ b/tests/integration_tests/capture_suicide_while_balance_table/run.sh @@ -36,7 +36,7 @@ function run() { pd_addr="http://$UP_PD_HOST_1:$UP_PD_PORT_1" run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --pd $pd_addr --logsuffix 1 --addr "127.0.0.1:8300" - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/sink/MySQLSinkHangLongTime=1*return(true)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/sink/MySQLSinkHangLongTime=1*return(true)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --pd $pd_addr --logsuffix 2 --addr "127.0.0.1:8301" SINK_URI="mysql://normal:123456@127.0.0.1:3306/?max-txn-row=1" diff --git a/tests/integration_tests/cdc/cdc.go b/tests/integration_tests/cdc/cdc.go index a62dcad8b98..6b7f56cb36f 100644 --- a/tests/integration_tests/cdc/cdc.go +++ b/tests/integration_tests/cdc/cdc.go @@ -20,8 +20,8 @@ import ( _ "github.com/go-sql-driver/mysql" "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/tests/integration_tests/dailytest" - "github.com/pingcap/ticdc/tests/integration_tests/util" + "github.com/pingcap/tiflow/tests/integration_tests/dailytest" + "github.com/pingcap/tiflow/tests/integration_tests/util" ) func main() { diff --git a/tests/integration_tests/changefeed_auto_stop/run.sh b/tests/integration_tests/changefeed_auto_stop/run.sh index f6a55182405..19ce910a00b 100755 --- a/tests/integration_tests/changefeed_auto_stop/run.sh +++ b/tests/integration_tests/changefeed_auto_stop/run.sh @@ -44,7 +44,7 @@ function run() { done run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "1" --addr "127.0.0.1:8301" --pd "http://${UP_PD_HOST_1}:${UP_PD_PORT_1}" - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/processor/pipeline/ProcessorSyncResolvedError=1*return(true);github.com/pingcap/ticdc/cdc/processor/ProcessorUpdatePositionDelaying=sleep(1000)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/processor/pipeline/ProcessorSyncResolvedError=1*return(true);github.com/pingcap/tiflow/cdc/processor/ProcessorUpdatePositionDelaying=sleep(1000)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "2" --addr "127.0.0.1:8302" --pd "http://${UP_PD_HOST_1}:${UP_PD_PORT_1}" export GO_FAILPOINTS='' diff --git a/tests/integration_tests/changefeed_error/run.sh b/tests/integration_tests/changefeed_error/run.sh index c4a32f48088..6c5b878c187 100755 --- a/tests/integration_tests/changefeed_error/run.sh +++ b/tests/integration_tests/changefeed_error/run.sh @@ -114,7 +114,7 @@ function run() { start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) run_sql "CREATE DATABASE changefeed_error;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} go-ycsb load mysql -P $CUR/conf/workload -p mysql.host=${UP_TIDB_HOST} -p mysql.port=${UP_TIDB_PORT} -p mysql.user=root -p mysql.db=changefeed_error - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/owner/NewChangefeedNoRetryError=1*return(true)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/NewChangefeedNoRetryError=1*return(true)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY capture_pid=$(ps -C $CDC_BINARY -o pid= | awk '{print $1}') @@ -138,7 +138,7 @@ function run() { go-ycsb load mysql -P $CUR/conf/workload -p mysql.host=${UP_TIDB_HOST} -p mysql.port=${UP_TIDB_PORT} -p mysql.user=root -p mysql.db=changefeed_error check_sync_diff $WORK_DIR $CUR/conf/diff_config.toml - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/owner/NewChangefeedRetryError=return(true)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/NewChangefeedRetryError=return(true)' kill $capture_pid ensure $MAX_RETRIES check_no_capture http://${UP_PD_HOST_1}:${UP_PD_PORT_1} run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY @@ -151,7 +151,7 @@ function run() { cleanup_process $CDC_BINARY # owner DDL error case - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/owner/InjectChangefeedDDLError=return(true)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/InjectChangefeedDDLError=return(true)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY changefeedid_1="changefeed-error-1" run_cdc_cli changefeed create --start-ts=$start_ts --sink-uri="$SINK_URI" -c $changefeedid_1 @@ -163,7 +163,7 @@ function run() { cleanup_process $CDC_BINARY # updating GC safepoint failure case - export GO_FAILPOINTS='github.com/pingcap/ticdc/pkg/txnutil/gc/InjectActualGCSafePoint=return(9223372036854775807)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/pkg/txnutil/gc/InjectActualGCSafePoint=return(9223372036854775807)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY changefeedid_2="changefeed-error-2" diff --git a/tests/integration_tests/changefeed_fast_fail/run.sh b/tests/integration_tests/changefeed_fast_fail/run.sh index 273368f5130..a5bdaa60d17 100644 --- a/tests/integration_tests/changefeed_fast_fail/run.sh +++ b/tests/integration_tests/changefeed_fast_fail/run.sh @@ -44,7 +44,7 @@ function run() { start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) run_sql "CREATE DATABASE changefeed_error;" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/owner/InjectChangefeedFastFailError=return(true)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/InjectChangefeedFastFailError=return(true)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY SINK_URI="mysql://normal:123456@127.0.0.1:3306/?max-txn-row=1" diff --git a/tests/integration_tests/consistent_replicate_nfs/run.sh b/tests/integration_tests/consistent_replicate_nfs/run.sh index ccbef125e92..3219102055a 100644 --- a/tests/integration_tests/consistent_replicate_nfs/run.sh +++ b/tests/integration_tests/consistent_replicate_nfs/run.sh @@ -57,7 +57,7 @@ function run() { # Inject the failpoint to prevent sink execution, but the global resolved can be moved forward. # Then we can apply redo log to reach an eventual consistent state in downstream. cleanup_process $CDC_BINARY - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/sink/MySQLSinkHangLongTime=return(true)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/sink/MySQLSinkHangLongTime=return(true)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY run_sql "create table consistent_replicate_nfs.USERTABLE2 like consistent_replicate_nfs.USERTABLE" ${UP_TIDB_HOST} ${UP_TIDB_PORT} diff --git a/tests/integration_tests/consistent_replicate_s3/run.sh b/tests/integration_tests/consistent_replicate_s3/run.sh index e9fef40ae2b..653bb21101c 100644 --- a/tests/integration_tests/consistent_replicate_s3/run.sh +++ b/tests/integration_tests/consistent_replicate_s3/run.sh @@ -90,7 +90,7 @@ function run() { # Inject the failpoint to prevent sink execution, but the global resolved can be moved forward. # Then we can apply redo log to reach an eventual consistent state in downstream. cleanup_process $CDC_BINARY - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/sink/MySQLSinkHangLongTime=return(true)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/sink/MySQLSinkHangLongTime=return(true)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY run_sql "create table consistent_replicate_s3.USERTABLE2 like consistent_replicate_s3.USERTABLE" ${UP_TIDB_HOST} ${UP_TIDB_PORT} diff --git a/tests/integration_tests/dailytest/db.go b/tests/integration_tests/dailytest/db.go index c0c7694bc79..e3831054186 100644 --- a/tests/integration_tests/dailytest/db.go +++ b/tests/integration_tests/dailytest/db.go @@ -23,8 +23,8 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/tests/integration_tests/util" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tiflow/tests/integration_tests/util" "go.uber.org/zap/zapcore" ) diff --git a/tests/integration_tests/ddl_puller_lag/run.sh b/tests/integration_tests/ddl_puller_lag/run.sh index e25f38489de..8f9595d7e0a 100644 --- a/tests/integration_tests/ddl_puller_lag/run.sh +++ b/tests/integration_tests/ddl_puller_lag/run.sh @@ -21,7 +21,7 @@ function prepare() { run_sql "CREATE table test.ddl_puller_lag1(id int primary key, val int);" run_sql "CREATE table test.ddl_puller_lag2(id int primary key, val int);" - run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --failpoint 'github.com/pingcap/ticdc/cdc/processor/processorDDLResolved=1*sleep(180000)' + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --failpoint 'github.com/pingcap/tiflow/cdc/processor/processorDDLResolved=1*sleep(180000)' TOPIC_NAME="ticdc-ddl-puller-lag-test-$RANDOM" case $SINK_TYPE in diff --git a/tests/integration_tests/force_replicate_table/run.sh b/tests/integration_tests/force_replicate_table/run.sh index 76e5581be2f..1b3e7ea04ef 100755 --- a/tests/integration_tests/force_replicate_table/run.sh +++ b/tests/integration_tests/force_replicate_table/run.sh @@ -72,7 +72,7 @@ function run() { table="force_replicate_table.t$i" check_table_exists $table ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} done - # data could be duplicated due to https://github.com/pingcap/ticdc/issues/964, + # data could be duplicated due to https://github.com/pingcap/tiflow/issues/964, # so we just check downstream contains all data in upstream. for i in $(seq 0 6); do ensure 5 check_data_subset "force_replicate_table.t$i" \ diff --git a/tests/integration_tests/gc_safepoint/run.sh b/tests/integration_tests/gc_safepoint/run.sh index c483b21c74e..fa28076e00b 100755 --- a/tests/integration_tests/gc_safepoint/run.sh +++ b/tests/integration_tests/gc_safepoint/run.sh @@ -86,7 +86,7 @@ function run() { kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}&max-message-bytes=10485760" ;; *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/?max-txn-row=1" ;; esac - export GO_FAILPOINTS='github.com/pingcap/ticdc/pkg/txnutil/gc/InjectGcSafepointUpdateInterval=return(500)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/pkg/txnutil/gc/InjectGcSafepointUpdateInterval=return(500)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8300" --pd $pd_addr changefeed_id=$(cdc cli changefeed create --pd=$pd_addr --sink-uri="$SINK_URI" 2>&1 | tail -n2 | head -n1 | awk '{print $2}') if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/integration_tests/kafka_sink_error_resume/run.sh b/tests/integration_tests/kafka_sink_error_resume/run.sh index 7039c16c042..1020d5cd233 100755 --- a/tests/integration_tests/kafka_sink_error_resume/run.sh +++ b/tests/integration_tests/kafka_sink_error_resume/run.sh @@ -39,7 +39,7 @@ function run() { TOPIC_NAME="ticdc-kafka-sink-error-resume-test-$RANDOM" SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}&max-message-bytes=10485760" - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/sink/producer/kafka/KafkaSinkAsyncSendError=4*return(true)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/sink/producer/kafka/KafkaSinkAsyncSendError=4*return(true)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8300" --pd $pd_addr changefeed_id=$(cdc cli changefeed create --pd=$pd_addr --sink-uri="$SINK_URI" 2>&1 | tail -n2 | head -n1 | awk '{print $2}') run_kafka_consumer $WORK_DIR "kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&version=${KAFKA_VERSION}&max-message-bytes=10485760" diff --git a/tests/integration_tests/kill_owner_with_ddl/run.sh b/tests/integration_tests/kill_owner_with_ddl/run.sh index 22757000f23..b13f8dbc8fc 100755 --- a/tests/integration_tests/kill_owner_with_ddl/run.sh +++ b/tests/integration_tests/kill_owner_with_ddl/run.sh @@ -56,7 +56,7 @@ function run() { run_sql "CREATE table kill_owner_with_ddl.t1 (id int primary key auto_increment, val int);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} check_table_exists "kill_owner_with_ddl.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/sink/MySQLSinkExecDDLDelay=return(true);github.com/pingcap/ticdc/cdc/capture/ownerFlushIntervalInject=return(10)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/sink/MySQLSinkExecDDLDelay=return(true);github.com/pingcap/tiflow/cdc/capture/ownerFlushIntervalInject=return(10)' kill_cdc_and_restart $pd_addr $WORK_DIR $CDC_BINARY for i in $(seq 2 3); do diff --git a/tests/integration_tests/kv_client_stream_reconnect/run.sh b/tests/integration_tests/kv_client_stream_reconnect/run.sh index ce4c102b463..ab39329419e 100644 --- a/tests/integration_tests/kv_client_stream_reconnect/run.sh +++ b/tests/integration_tests/kv_client_stream_reconnect/run.sh @@ -24,7 +24,7 @@ function run() { esac # this will be triggered every 5s in kv client - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/kv/kvClientForceReconnect=return(true)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/kv/kvClientForceReconnect=return(true)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8300" --pd $pd_addr changefeed_id=$(cdc cli changefeed create --pd=$pd_addr --sink-uri="$SINK_URI" 2>&1 | tail -n2 | head -n1 | awk '{print $2}') if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/integration_tests/many_pk_or_uk/main.go b/tests/integration_tests/many_pk_or_uk/main.go index 152e40acaba..aa00fe72812 100644 --- a/tests/integration_tests/many_pk_or_uk/main.go +++ b/tests/integration_tests/many_pk_or_uk/main.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/tests/integration_tests/util" + "github.com/pingcap/tiflow/tests/integration_tests/util" ) func main() { diff --git a/tests/integration_tests/move_table/main.go b/tests/integration_tests/move_table/main.go index 5b0e84cd1cc..4510fa90ed6 100644 --- a/tests/integration_tests/move_table/main.go +++ b/tests/integration_tests/move_table/main.go @@ -24,12 +24,12 @@ import ( "strings" "time" - "github.com/pingcap/ticdc/pkg/etcd" + "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/errors" "github.com/pingcap/log" - cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/retry" + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/retry" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/pkg/logutil" "go.uber.org/zap" diff --git a/tests/integration_tests/multi_source/main.go b/tests/integration_tests/multi_source/main.go index 3324d5ee5ba..db14e10ff76 100644 --- a/tests/integration_tests/multi_source/main.go +++ b/tests/integration_tests/multi_source/main.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/tests/integration_tests/util" + "github.com/pingcap/tiflow/tests/integration_tests/util" "go.uber.org/zap" ) diff --git a/tests/integration_tests/owner_remove_table_error/run.sh b/tests/integration_tests/owner_remove_table_error/run.sh index bae27f0a874..5ba75eb1ef1 100644 --- a/tests/integration_tests/owner_remove_table_error/run.sh +++ b/tests/integration_tests/owner_remove_table_error/run.sh @@ -25,7 +25,7 @@ function run() { pd_addr="http://$UP_PD_HOST_1:$UP_PD_PORT_1" SINK_URI="mysql://normal:123456@127.0.0.1:3306/?max-txn-row=1" - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/owner/OwnerRemoveTableError=1*return(true)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/OwnerRemoveTableError=1*return(true)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8300" --pd $pd_addr changefeed_id=$(cdc cli changefeed create --pd=$pd_addr --sink-uri="$SINK_URI" 2>&1 | tail -n2 | head -n1 | awk '{print $2}') diff --git a/tests/integration_tests/processor_err_chan/run.sh b/tests/integration_tests/processor_err_chan/run.sh index f4962fb3c52..7e9ca97176c 100644 --- a/tests/integration_tests/processor_err_chan/run.sh +++ b/tests/integration_tests/processor_err_chan/run.sh @@ -47,7 +47,7 @@ function run() { run_sql "CREATE table processor_err_chan.t$i (id int primary key auto_increment)" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} done - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/processor/pipeline/ProcessorAddTableError=1*return(true)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/processor/pipeline/ProcessorAddTableError=1*return(true)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8300" --pd $pd_addr changefeed_id=$(cdc cli changefeed create --pd=$pd_addr --sink-uri="$SINK_URI" 2>&1 | tail -n2 | head -n1 | awk '{print $2}') diff --git a/tests/integration_tests/processor_panic/main.go b/tests/integration_tests/processor_panic/main.go index c6aef27aa22..430a87d5b8d 100644 --- a/tests/integration_tests/processor_panic/main.go +++ b/tests/integration_tests/processor_panic/main.go @@ -22,8 +22,8 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/pkg/quotes" - "github.com/pingcap/ticdc/tests/integration_tests/util" + "github.com/pingcap/tiflow/pkg/quotes" + "github.com/pingcap/tiflow/tests/integration_tests/util" "go.uber.org/zap" "golang.org/x/sync/errgroup" ) diff --git a/tests/integration_tests/processor_panic/run.sh b/tests/integration_tests/processor_panic/run.sh index ac4167a3572..20f9aa6fd8b 100644 --- a/tests/integration_tests/processor_panic/run.sh +++ b/tests/integration_tests/processor_panic/run.sh @@ -19,7 +19,7 @@ function prepare() { start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix 1 --addr 127.0.0.1:8300 --restart true \ - --failpoint 'github.com/pingcap/ticdc/cdc/processor/pipeline/ProcessorSyncResolvedPreEmit=return(true)' + --failpoint 'github.com/pingcap/tiflow/cdc/processor/pipeline/ProcessorSyncResolvedPreEmit=return(true)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix 2 --addr 127.0.0.1:8301 diff --git a/tests/integration_tests/processor_resolved_ts_fallback/run.sh b/tests/integration_tests/processor_resolved_ts_fallback/run.sh index 251c058d40e..b99754b42d6 100755 --- a/tests/integration_tests/processor_resolved_ts_fallback/run.sh +++ b/tests/integration_tests/processor_resolved_ts_fallback/run.sh @@ -18,7 +18,7 @@ function run() { start_tidb_cluster --workdir $WORK_DIR cd $WORK_DIR - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/sink/SinkFlushDMLPanic=return(true);github.com/pingcap/ticdc/cdc/sink/producer/kafka/SinkFlushDMLPanic=return(true)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/sink/SinkFlushDMLPanic=return(true);github.com/pingcap/tiflow/cdc/sink/producer/kafka/SinkFlushDMLPanic=return(true)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix "1" --addr "127.0.0.1:8301" --pd "http://${UP_PD_HOST_1}:${UP_PD_PORT_1}" TOPIC_NAME="ticdc-processor-resolved-ts-fallback-test-$RANDOM" diff --git a/tests/integration_tests/processor_stop_delay/run.sh b/tests/integration_tests/processor_stop_delay/run.sh index 55a8f57e48f..89d23b963fb 100644 --- a/tests/integration_tests/processor_stop_delay/run.sh +++ b/tests/integration_tests/processor_stop_delay/run.sh @@ -20,7 +20,7 @@ function run() { kafka) SINK_URI="kafka://127.0.0.1:9092/$TOPIC_NAME?partition-num=4&kafka-version=${KAFKA_VERSION}&max-message-bytes=10485760" ;; *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/?max-txn-row=1" ;; esac - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/processor/processorStopDelay=1*sleep(10000)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/processor/processorStopDelay=1*sleep(10000)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8300" --pd $pd_addr changefeed_id=$(cdc cli changefeed create --pd=$pd_addr --sink-uri="$SINK_URI" 2>&1 | tail -n2 | head -n1 | awk '{print $2}') diff --git a/tests/integration_tests/resolve_lock/main.go b/tests/integration_tests/resolve_lock/main.go index f02ed745270..f75da489d54 100644 --- a/tests/integration_tests/resolve_lock/main.go +++ b/tests/integration_tests/resolve_lock/main.go @@ -31,11 +31,11 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/log" - "github.com/pingcap/ticdc/tests/integration_tests/util" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/store/driver" "github.com/pingcap/tidb/tablecodec" + "github.com/pingcap/tiflow/tests/integration_tests/util" "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/tikv" "github.com/tikv/client-go/v2/tikvrpc" diff --git a/tests/integration_tests/sink_hang/run.sh b/tests/integration_tests/sink_hang/run.sh index ac386512890..83d78c9c050 100644 --- a/tests/integration_tests/sink_hang/run.sh +++ b/tests/integration_tests/sink_hang/run.sh @@ -42,7 +42,7 @@ function run() { *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/?max-txn-row=1" ;; esac - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/sink/MySQLSinkHangLongTime=1*return(true);github.com/pingcap/ticdc/cdc/sink/MySQLSinkExecDMLError=9*return(true)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/sink/MySQLSinkHangLongTime=1*return(true);github.com/pingcap/tiflow/cdc/sink/MySQLSinkExecDMLError=9*return(true)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8300" --pd $pd_addr changefeed_id=$(cdc cli changefeed create --pd=$pd_addr --sink-uri="$SINK_URI" 2>&1 | tail -n2 | head -n1 | awk '{print $2}') if [ "$SINK_TYPE" == "kafka" ]; then diff --git a/tests/integration_tests/sink_retry/run.sh b/tests/integration_tests/sink_retry/run.sh index 0f19d3fb016..88b0219257a 100755 --- a/tests/integration_tests/sink_retry/run.sh +++ b/tests/integration_tests/sink_retry/run.sh @@ -21,7 +21,7 @@ function run() { start_ts=$(run_cdc_cli_tso_query ${UP_PD_HOST_1} ${UP_PD_PORT_1}) run_sql "CREATE DATABASE sink_retry;" go-ycsb load mysql -P $CUR/conf/workload -p mysql.host=${UP_TIDB_HOST} -p mysql.port=${UP_TIDB_PORT} -p mysql.user=root -p mysql.db=sink_retry - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/sink/MySQLSinkTxnRandomError=25%return(true)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/sink/MySQLSinkTxnRandomError=25%return(true)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY TOPIC_NAME="ticdc-sink-retry-test-$RANDOM" diff --git a/tools/check/check-errdoc.sh b/tools/check/check-errdoc.sh index 243a0668ab1..3ebf7269eca 100755 --- a/tools/check/check-errdoc.sh +++ b/tools/check/check-errdoc.sh @@ -17,5 +17,5 @@ set -euo pipefail cd -P . cp errors.toml /tmp/errors.toml.before -./tools/bin/errdoc-gen --source . --module github.com/pingcap/ticdc --output errors.toml --ignore proto,dm +./tools/bin/errdoc-gen --source . --module github.com/pingcap/tiflow --output errors.toml --ignore proto,dm diff -q errors.toml /tmp/errors.toml.before From 84a4d264a1870c1b56f937f3d42e8187ee87421a Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 20 Dec 2021 14:47:46 +0800 Subject: [PATCH 13/48] http_*: add log for http api and refine the err handle logic (#2997) (#3307) --- cdc/capture/http_errors.go | 51 +++++++++++ cdc/capture/http_errors_test.go | 33 +++++++ cdc/capture/http_handler.go | 150 ++++++++++---------------------- cdc/capture/http_validator.go | 2 +- cdc/http_router.go | 56 +++++++++++- cdc/http_router_test.go | 3 +- cdc/http_status.go | 2 +- cdc/sink/mysql.go | 24 +++-- cdc/sink/simple_mysql_tester.go | 2 +- cdc/sink/sink_test.go | 2 +- pkg/config/config_test.go | 1 + 11 files changed, 203 insertions(+), 123 deletions(-) create mode 100644 cdc/capture/http_errors.go create mode 100644 cdc/capture/http_errors_test.go diff --git a/cdc/capture/http_errors.go b/cdc/capture/http_errors.go new file mode 100644 index 00000000000..ed58b71e108 --- /dev/null +++ b/cdc/capture/http_errors.go @@ -0,0 +1,51 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package capture + +import ( + "strings" + + "github.com/pingcap/errors" + cerror "github.com/pingcap/tiflow/pkg/errors" +) + +// httpBadRequestError is some errors that will cause a BadRequestError in http handler +var httpBadRequestError = []*errors.Error{ + cerror.ErrAPIInvalidParam, cerror.ErrSinkURIInvalid, cerror.ErrStartTsBeforeGC, + cerror.ErrChangeFeedNotExists, cerror.ErrTargetTsBeforeStartTs, cerror.ErrTableIneligible, + cerror.ErrFilterRuleInvalid, cerror.ErrChangefeedUpdateRefused, cerror.ErrMySQLConnectionError, + cerror.ErrMySQLInvalidConfig, +} + +// IsHTTPBadRequestError check if a error is a http bad request error +func IsHTTPBadRequestError(err error) bool { + if err == nil { + return false + } + for _, e := range httpBadRequestError { + if e.Equal(err) { + return true + } + + rfcCode, ok := cerror.RFCCode(err) + if ok && e.RFCCode() == rfcCode { + return true + } + + if strings.Contains(err.Error(), string(e.RFCCode())) { + return true + } + } + return false +} diff --git a/cdc/capture/http_errors_test.go b/cdc/capture/http_errors_test.go new file mode 100644 index 00000000000..8437577a20d --- /dev/null +++ b/cdc/capture/http_errors_test.go @@ -0,0 +1,33 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package capture + +import ( + "testing" + + "github.com/pingcap/errors" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/stretchr/testify/require" +) + +func TestIsHTTPBadRequestError(t *testing.T) { + err := cerror.ErrAPIInvalidParam.GenWithStack("aa") + require.Equal(t, true, IsHTTPBadRequestError(err)) + err = cerror.ErrAPIInvalidParam.Wrap(errors.New("aa")) + require.Equal(t, true, IsHTTPBadRequestError(err)) + err = cerror.ErrPDEtcdAPIError.GenWithStack("aa") + require.Equal(t, false, IsHTTPBadRequestError(err)) + err = nil + require.Equal(t, false, IsHTTPBadRequestError(err)) +} diff --git a/cdc/capture/http_handler.go b/cdc/capture/http_handler.go index 1b340c7477e..133e7cc9e92 100644 --- a/cdc/capture/http_handler.go +++ b/cdc/capture/http_handler.go @@ -78,13 +78,15 @@ func (h *HTTPHandler) ListChangefeed(c *gin.Context) { // get all changefeed status statuses, err := statusProvider.GetAllChangeFeedStatuses(ctx) if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } // get all changefeed infos infos, err := statusProvider.GetAllChangeFeedInfo(ctx) if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + // this call will return a parsedError generated by the error we passed in + // so it is no need to check the parsedError + _ = c.Error(err) return } @@ -140,38 +142,25 @@ func (h *HTTPHandler) GetChangefeed(c *gin.Context) { ctx := c.Request.Context() changefeedID := c.Param(apiOpVarChangefeedID) if err := model.ValidateChangefeedID(changefeedID); err != nil { - c.IndentedJSON(http.StatusBadRequest, - model.NewHTTPError(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID))) + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID)) return } info, err := statusProvider.GetChangeFeedInfo(ctx, changefeedID) if err != nil { - if cerror.ErrChangeFeedNotExists.Equal(err) { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) - return - } - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } status, err := statusProvider.GetChangeFeedStatus(ctx, changefeedID) if err != nil { - if cerror.ErrChangeFeedNotExists.Equal(err) { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) - return - } - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } processorInfos, err := statusProvider.GetAllTaskStatuses(ctx, changefeedID) if err != nil { - if cerror.ErrChangeFeedNotExists.Equal(err) { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) - return - } - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } @@ -197,7 +186,7 @@ func (h *HTTPHandler) GetChangefeed(c *gin.Context) { TaskStatus: taskStatus, } - c.JSON(http.StatusOK, changefeedDetail) + c.IndentedJSON(http.StatusOK, changefeedDetail) } // CreateChangefeed creates a changefeed @@ -219,29 +208,25 @@ func (h *HTTPHandler) CreateChangefeed(c *gin.Context) { ctx := c.Request.Context() var changefeedConfig model.ChangefeedConfig if err := c.BindJSON(&changefeedConfig); err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(cerror.ErrAPIInvalidParam.Wrap(err)) return } info, err := verifyCreateChangefeedConfig(c, changefeedConfig, h.capture) if err != nil { - if cerror.ErrPDEtcdAPIError.Equal(err) { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) - return - } - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) + _ = c.Error(err) return } infoStr, err := info.Marshal() if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } err = h.capture.etcdClient.CreateChangefeedInfo(ctx, info, changefeedConfig.ID) if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } @@ -269,18 +254,13 @@ func (h *HTTPHandler) PauseChangefeed(c *gin.Context) { changefeedID := c.Param(apiOpVarChangefeedID) if err := model.ValidateChangefeedID(changefeedID); err != nil { - c.IndentedJSON(http.StatusBadRequest, - model.NewHTTPError(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID))) + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID)) return } // check if the changefeed exists _, err := statusProvider.GetChangeFeedStatus(ctx, changefeedID) if err != nil { - if cerror.ErrChangeFeedNotExists.Equal(err) { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) - return - } - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } @@ -316,18 +296,13 @@ func (h *HTTPHandler) ResumeChangefeed(c *gin.Context) { ctx := c.Request.Context() changefeedID := c.Param(apiOpVarChangefeedID) if err := model.ValidateChangefeedID(changefeedID); err != nil { - c.IndentedJSON(http.StatusBadRequest, - model.NewHTTPError(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID))) + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID)) return } // check if the changefeed exists _, err := statusProvider.GetChangeFeedStatus(ctx, changefeedID) if err != nil { - if cerror.ErrChangeFeedNotExists.Equal(err) { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) - return - } - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } @@ -370,21 +345,16 @@ func (h *HTTPHandler) UpdateChangefeed(c *gin.Context) { changefeedID := c.Param(apiOpVarChangefeedID) if err := model.ValidateChangefeedID(changefeedID); err != nil { - c.IndentedJSON(http.StatusBadRequest, - model.NewHTTPError(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID))) + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID)) return } info, err := statusProvider.GetChangeFeedInfo(ctx, changefeedID) if err != nil { - if cerror.ErrChangeFeedNotExists.Equal(err) { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) - return - } - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } if info.State != model.StateStopped { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(cerror.ErrChangefeedUpdateRefused.GenWithStackByArgs("can only update changefeed config when it is stopped"))) + _ = c.Error(cerror.ErrChangefeedUpdateRefused.GenWithStackByArgs("can only update changefeed config when it is stopped")) return } @@ -392,19 +362,19 @@ func (h *HTTPHandler) UpdateChangefeed(c *gin.Context) { // filter_rules, ignore_txn_start_ts, mounter_worker_num, sink_config var changefeedConfig model.ChangefeedConfig if err = c.BindJSON(&changefeedConfig); err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } newInfo, err := verifyUpdateChangefeedConfig(ctx, changefeedConfig, info) if err != nil { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) + _ = c.Error(err) return } err = h.capture.etcdClient.SaveChangeFeedInfo(ctx, newInfo, changefeedID) if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } @@ -430,18 +400,13 @@ func (h *HTTPHandler) RemoveChangefeed(c *gin.Context) { ctx := c.Request.Context() changefeedID := c.Param(apiOpVarChangefeedID) if err := model.ValidateChangefeedID(changefeedID); err != nil { - c.IndentedJSON(http.StatusBadRequest, - model.NewHTTPError(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID))) + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID)) return } // check if the changefeed exists _, err := statusProvider.GetChangeFeedStatus(ctx, changefeedID) if err != nil { - if cerror.ErrChangeFeedNotExists.Equal(err) { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) - return - } - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } @@ -478,18 +443,13 @@ func (h *HTTPHandler) RebalanceTable(c *gin.Context) { changefeedID := c.Param(apiOpVarChangefeedID) if err := model.ValidateChangefeedID(changefeedID); err != nil { - c.IndentedJSON(http.StatusBadRequest, - model.NewHTTPError(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID))) + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID)) return } // check if the changefeed exists _, err := statusProvider.GetChangeFeedStatus(ctx, changefeedID) if err != nil { - if cerror.ErrChangeFeedNotExists.Equal(err) { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) - return - } - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } @@ -522,18 +482,13 @@ func (h *HTTPHandler) MoveTable(c *gin.Context) { ctx := c.Request.Context() changefeedID := c.Param(apiOpVarChangefeedID) if err := model.ValidateChangefeedID(changefeedID); err != nil { - c.IndentedJSON(http.StatusBadRequest, - model.NewHTTPError(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID))) + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID)) return } // check if the changefeed exists _, err := statusProvider.GetChangeFeedStatus(ctx, changefeedID) if err != nil { - if cerror.ErrChangeFeedNotExists.Equal(err) { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) - return - } - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } @@ -543,12 +498,12 @@ func (h *HTTPHandler) MoveTable(c *gin.Context) { }{} err = c.BindJSON(&data) if err != nil { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) + _ = c.Error(cerror.ErrAPIInvalidParam.Wrap(err)) + return } if err := model.ValidateChangefeedID(data.CaptureID); err != nil { - c.IndentedJSON(http.StatusBadRequest, - model.NewHTTPError(cerror.ErrAPIInvalidParam.GenWithStack("invalid capture_id: %s", data.CaptureID))) + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid capture_id: %s", data.CaptureID)) return } @@ -603,42 +558,34 @@ func (h *HTTPHandler) GetProcessor(c *gin.Context) { changefeedID := c.Param(apiOpVarChangefeedID) if err := model.ValidateChangefeedID(changefeedID); err != nil { - c.IndentedJSON(http.StatusBadRequest, - model.NewHTTPError(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID))) + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid changefeed_id: %s", changefeedID)) return } captureID := c.Param(apiOpVarCaptureID) if err := model.ValidateChangefeedID(captureID); err != nil { - c.IndentedJSON(http.StatusBadRequest, - model.NewHTTPError(cerror.ErrAPIInvalidParam.GenWithStack("invalid capture_id: %s", changefeedID))) + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid capture_id: %s", captureID)) return } statuses, err := statusProvider.GetAllTaskStatuses(ctx, changefeedID) if err != nil { - if cerror.ErrChangeFeedNotExists.Equal(err) { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) - } - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } status, exist := statuses[captureID] if !exist { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(cerror.ErrCaptureNotExist.GenWithStackByArgs(captureID))) + _ = c.Error(cerror.ErrCaptureNotExist.GenWithStackByArgs(captureID)) } positions, err := statusProvider.GetTaskPositions(ctx, changefeedID) if err != nil { - if cerror.ErrChangeFeedNotExists.Equal(err) { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) - } - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } position, exist := positions[captureID] if !exist { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(cerror.ErrCaptureNotExist.GenWithStackByArgs(captureID))) + _ = c.Error(cerror.ErrCaptureNotExist.GenWithStackByArgs(captureID)) } processorDetail := &model.ProcessorDetail{CheckPointTs: position.CheckPointTs, ResolvedTs: position.ResolvedTs, Error: position.Error} @@ -647,7 +594,7 @@ func (h *HTTPHandler) GetProcessor(c *gin.Context) { tables = append(tables, tableID) } processorDetail.Tables = tables - c.JSON(http.StatusOK, processorDetail) + c.IndentedJSON(http.StatusOK, processorDetail) } // ListProcessor lists all processors in the TiCDC cluster @@ -669,7 +616,7 @@ func (h *HTTPHandler) ListProcessor(c *gin.Context) { ctx := c.Request.Context() infos, err := statusProvider.GetProcessors(ctx) if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } resps := make([]*model.ProcessorCommonInfo, len(infos)) @@ -699,7 +646,7 @@ func (h *HTTPHandler) ListCapture(c *gin.Context) { ctx := c.Request.Context() captureInfos, err := statusProvider.GetCaptures(ctx) if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } @@ -781,14 +728,13 @@ func SetLogLevel(c *gin.Context) { }{} err := c.BindJSON(&data) if err != nil { - c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("invalid log level: %s", err.Error())) return } err = logutil.SetLogLevel(data.Level) if err != nil { - c.IndentedJSON(http.StatusBadRequest, - model.NewHTTPError(cerror.ErrAPIInvalidParam.GenWithStack("fail to change log level: %s", err))) + _ = c.Error(cerror.ErrAPIInvalidParam.GenWithStack("fail to change log level: %s", data.Level)) return } log.Warn("log level changed", zap.String("level", data.Level)) @@ -800,7 +746,7 @@ func (h *HTTPHandler) forwardToOwner(c *gin.Context) { ctx := c.Request.Context() // every request can only forward to owner one time if len(c.GetHeader(forWardFromCapture)) != 0 { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(cerror.ErrRequestForwardErr.FastGenByArgs())) + _ = c.Error(cerror.ErrRequestForwardErr.FastGenByArgs()) return } c.Header(forWardFromCapture, h.capture.Info().ID) @@ -817,13 +763,13 @@ func (h *HTTPHandler) forwardToOwner(c *gin.Context) { return nil }, retry.WithBackoffBaseDelay(300), retry.WithMaxTries(getOwnerRetryMaxTime)) if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } tslConfig, err := config.GetGlobalServerConfig().Security.ToTLSConfigWithVerify() if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } @@ -845,7 +791,7 @@ func (h *HTTPHandler) forwardToOwner(c *gin.Context) { cli := httputil.NewClient(tslConfig) resp, err := cli.Do(req) if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } @@ -863,7 +809,7 @@ func (h *HTTPHandler) forwardToOwner(c *gin.Context) { defer resp.Body.Close() _, err = bufio.NewReader(resp.Body).WriteTo(c.Writer) if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } } diff --git a/cdc/capture/http_validator.go b/cdc/capture/http_validator.go index 0281a23d63f..029cfd68368 100644 --- a/cdc/capture/http_validator.go +++ b/cdc/capture/http_validator.go @@ -139,7 +139,7 @@ func verifyCreateChangefeedConfig(ctx context.Context, changefeedConfig model.Ch tz, err := util.GetTimezone(changefeedConfig.TimeZone) if err != nil { - return nil, errors.Annotate(err, "invalid timezone:"+changefeedConfig.TimeZone) + return nil, cerror.ErrAPIInvalidParam.Wrap(errors.Annotatef(err, "invalid timezone:%s", changefeedConfig.TimeZone)) } ctx = util.PutTimezoneInCtx(ctx, tz) if err := sink.Validate(ctx, info.SinkURI, info.Config, info.Opts); err != nil { diff --git a/cdc/http_router.go b/cdc/http_router.go index 51019609629..dd75deef268 100644 --- a/cdc/http_router.go +++ b/cdc/http_router.go @@ -21,23 +21,29 @@ import ( "time" "github.com/gin-gonic/gin" + "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/capture" + "github.com/pingcap/tiflow/cdc/model" swaggerFiles "github.com/swaggo/files" ginSwagger "github.com/swaggo/gin-swagger" + "go.uber.org/zap" // use for OpenAPI online docs _ "github.com/pingcap/tiflow/docs/api" ) // newRouter create a router for OpenAPI + func newRouter(captureHandler capture.HTTPHandler) *gin.Engine { // discard gin log output gin.DefaultWriter = io.Discard router := gin.New() + router.Use(logMiddleware()) // request will timeout after 10 second router.Use(timeoutMiddleware(time.Second * 10)) + router.Use(errorHandleMiddleware()) // OpenAPI online docs router.GET("/swagger/*any", ginSwagger.WrapHandler(swaggerFiles.Handler)) @@ -96,7 +102,7 @@ func newRouter(captureHandler capture.HTTPHandler) *gin.Engine { } // timeoutMiddleware wraps the request context with a timeout -func timeoutMiddleware(timeout time.Duration) func(c *gin.Context) { +func timeoutMiddleware(timeout time.Duration) gin.HandlerFunc { return func(c *gin.Context) { // wrap the request context with a timeout ctx, cancel := context.WithTimeout(c.Request.Context(), timeout) @@ -119,3 +125,51 @@ func timeoutMiddleware(timeout time.Duration) func(c *gin.Context) { c.Next() } } + +func logMiddleware() gin.HandlerFunc { + return func(c *gin.Context) { + start := time.Now() + path := c.Request.URL.Path + query := c.Request.URL.RawQuery + c.Next() + + cost := time.Since(start) + + err := c.Errors.Last() + var stdErr error + if err != nil { + stdErr = err.Err + } + + log.Info(path, + zap.Int("status", c.Writer.Status()), + zap.String("method", c.Request.Method), + zap.String("path", path), + zap.String("query", query), + zap.String("ip", c.ClientIP()), + zap.String("user-agent", c.Request.UserAgent()), + zap.Error(stdErr), + zap.Duration("cost", cost), + ) + } +} + +func errorHandleMiddleware() gin.HandlerFunc { + return func(c *gin.Context) { + c.Next() + // because we will return immediately after an error occurs in http_handler + // there wil be only one error in c.Errors + lastError := c.Errors.Last() + if lastError != nil { + err := lastError.Err + // put the error into response + if capture.IsHTTPBadRequestError(err) { + c.IndentedJSON(http.StatusBadRequest, model.NewHTTPError(err)) + } else { + c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + } + c.Abort() + return + } + } +} diff --git a/cdc/http_router_test.go b/cdc/http_router_test.go index 47ebb4245b9..4f779c4adec 100644 --- a/cdc/http_router_test.go +++ b/cdc/http_router_test.go @@ -23,9 +23,8 @@ import ( "github.com/stretchr/testify/require" ) -func TestPProfRouter(t *testing.T) { +func TestPProfPath(t *testing.T) { t.Parallel() - router := newRouter(capture.NewHTTPHandler(nil)) apis := []*openAPI{ diff --git a/cdc/http_status.go b/cdc/http_status.go index 7b534b502ec..4b3890b3238 100644 --- a/cdc/http_status.go +++ b/cdc/http_status.go @@ -39,6 +39,7 @@ import ( ) func (s *Server) startStatusHTTP() error { + conf := config.GetGlobalServerConfig() router := newRouter(capture.NewHTTPHandler(s.capture)) router.GET("/status", gin.WrapF(s.handleStatus)) @@ -58,7 +59,6 @@ func (s *Server) startStatusHTTP() error { prometheus.DefaultGatherer = registry router.Any("/metrics", gin.WrapH(promhttp.Handler())) - conf := config.GetGlobalServerConfig() err := conf.Security.AddSelfCommonName() if err != nil { log.Error("status server set tls config failed", zap.Error(err)) diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index d0704ba8a92..aeab222697d 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -334,7 +334,7 @@ func checkTiDBVariable(ctx context.Context, db *sql.DB, variableName, defaultVal err := db.QueryRowContext(ctx, querySQL).Scan(&name, &value) if err != nil && err != sql.ErrNoRows { errMsg := "fail to query session variable " + variableName - return "", errors.Annotate(cerror.WrapError(cerror.ErrMySQLQueryError, err), errMsg) + return "", cerror.ErrMySQLQueryError.Wrap(err).GenWithStack(errMsg) } // session variable works, use given default value if err == nil { @@ -438,13 +438,12 @@ func parseSinkURI(ctx context.Context, sinkURI *url.URL, opts map[string]string) } tlsCfg, err := credential.ToTLSConfig() if err != nil { - return nil, errors.Annotate(err, "fail to open MySQL connection") + return nil, errors.Trace(err) } name := "cdc_mysql_tls" + params.changefeedID err = dmysql.RegisterTLSConfig(name, tlsCfg) if err != nil { - return nil, errors.Annotate( - cerror.WrapError(cerror.ErrMySQLConnectionError, err), "fail to open MySQL connection") + return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") } params.tls = "?tls=" + name } @@ -513,8 +512,7 @@ var GetDBConnImpl = getDBConn func getDBConn(ctx context.Context, dsnStr string) (*sql.DB, error) { db, err := sql.Open("mysql", dsnStr) if err != nil { - return nil, errors.Annotate( - cerror.WrapError(cerror.ErrMySQLConnectionError, err), "Open database connection failed") + return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") } err = db.PingContext(ctx) if err != nil { @@ -522,8 +520,7 @@ func getDBConn(ctx context.Context, dsnStr string) (*sql.DB, error) { if closeErr := db.Close(); closeErr != nil { log.Warn("close db failed", zap.Error(err)) } - return nil, errors.Annotate( - cerror.WrapError(cerror.ErrMySQLConnectionError, err), "fail to open MySQL connection") + return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") } return db, nil } @@ -1361,12 +1358,12 @@ func newMySQLSyncpointStore(ctx context.Context, id string, sinkURI *url.URL) (S } tlsCfg, err := credential.ToTLSConfig() if err != nil { - return nil, errors.Annotate(err, "fail to open MySQL connection") + return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") } name := "cdc_mysql_tls" + "syncpoint" + id err = dmysql.RegisterTLSConfig(name, tlsCfg) if err != nil { - return nil, errors.Annotate(err, "fail to open MySQL connection") + return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") } tlsParam = "?tls=" + name } @@ -1406,8 +1403,7 @@ func newMySQLSyncpointStore(ctx context.Context, id string, sinkURI *url.URL) (S } testDB, err := sql.Open("mysql", dsn.FormatDSN()) if err != nil { - return nil, errors.Annotate( - cerror.WrapError(cerror.ErrMySQLConnectionError, err), "fail to open MySQL connection when configuring sink") + return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection when configuring sink") } defer testDB.Close() dsnStr, err = configureSinkURI(ctx, dsn, params, testDB) @@ -1416,11 +1412,11 @@ func newMySQLSyncpointStore(ctx context.Context, id string, sinkURI *url.URL) (S } syncDB, err = sql.Open("mysql", dsnStr) if err != nil { - return nil, errors.Annotate(err, "Open database connection failed") + return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") } err = syncDB.PingContext(ctx) if err != nil { - return nil, errors.Annotate(err, "fail to open MySQL connection") + return nil, cerror.ErrMySQLConnectionError.Wrap(err).GenWithStack("fail to open MySQL connection") } log.Info("Start mysql syncpoint sink") diff --git a/cdc/sink/simple_mysql_tester.go b/cdc/sink/simple_mysql_tester.go index 2bb46c14bb9..4bfd5abf7ae 100644 --- a/cdc/sink/simple_mysql_tester.go +++ b/cdc/sink/simple_mysql_tester.go @@ -85,7 +85,7 @@ func newSimpleMySQLSink(ctx context.Context, sinkURI *url.URL, config *config.Re db, err = sql.Open("mysql", dsnStr) if err != nil { return nil, errors.Annotate( - cerror.WrapError(cerror.ErrMySQLConnectionError, err), "Open database connection failed") + cerror.WrapError(cerror.ErrMySQLConnectionError, err), "fail to open MySQL connection") } err = db.PingContext(ctx) if err != nil { diff --git a/cdc/sink/sink_test.go b/cdc/sink/sink_test.go index ae68c9f3984..63acc486138 100644 --- a/cdc/sink/sink_test.go +++ b/cdc/sink/sink_test.go @@ -35,7 +35,7 @@ func TestValidateSink(t *testing.T) { sinkURI := "mysql://root:111@127.0.0.1:3306/" err := Validate(ctx, sinkURI, replicateConfig, opts) require.NotNil(t, err) - require.Regexp(t, "fail to open MySQL connection.*ErrMySQLConnectionError.*", err) + require.Contains(t, err.Error(), "fail to open MySQL connection") // test sink uri right sinkURI = "blackhole://" diff --git a/pkg/config/config_test.go b/pkg/config/config_test.go index 05f523b33b7..09430135d3a 100644 --- a/pkg/config/config_test.go +++ b/pkg/config/config_test.go @@ -69,6 +69,7 @@ func TestReplicaConfigOutDated(t *testing.T) { func TestServerConfigMarshal(t *testing.T) { t.Parallel() + rawConfig := `{"addr":"192.155.22.33:8887","advertise-addr":"","log-file":"","log-level":"info","log":{"file":{"max-size":300,"max-days":0,"max-backups":0}},"data-dir":"","gc-ttl":86400,"tz":"System","capture-session-ttl":10,"owner-flush-interval":200000000,"processor-flush-interval":100000000,"sorter":{"num-concurrent-worker":4,"chunk-size-limit":999,"max-memory-percentage":30,"max-memory-consumption":17179869184,"num-workerpool-goroutine":16,"sort-dir":"/tmp/sorter"},"security":{"ca-path":"","cert-path":"","key-path":"","cert-allowed-cn":null},"per-table-memory-quota":10485760,"kv-client":{"worker-concurrent":8,"worker-pool-size":0,"region-scan-limit":40}}` conf := GetDefaultServerConfig() From 7d57e2ffca4b6d1137c955d60cceff894a3168f9 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 20 Dec 2021 16:37:46 +0800 Subject: [PATCH 14/48] etcd_worker: batch etcd patch (#3277) (#3389) --- cdc/capture/capture.go | 3 +- cdc/metrics.go | 2 + errors.toml | 10 + go.mod | 1 + metrics/grafana/ticdc.json | 1117 ++++++++++++++++- pkg/errors/errors.go | 8 +- pkg/etcd/client.go | 77 +- pkg/orchestrator/batch.go | 91 ++ pkg/orchestrator/batch_test.go | 75 ++ pkg/orchestrator/etcd_worker.go | 173 ++- pkg/orchestrator/etcd_worker_bank_test.go | 9 +- pkg/orchestrator/etcd_worker_test.go | 16 +- pkg/orchestrator/interfaces.go | 6 +- pkg/orchestrator/metrics.go | 52 + pkg/orchestrator/reactor_state.go | 1 + .../cdc_state_checker/cdc_monitor.go | 2 +- 16 files changed, 1574 insertions(+), 69 deletions(-) create mode 100644 pkg/orchestrator/batch.go create mode 100644 pkg/orchestrator/batch_test.go create mode 100644 pkg/orchestrator/metrics.go diff --git a/cdc/capture/capture.go b/cdc/capture/capture.go index de7b8ba3982..e7d8cf47e46 100644 --- a/cdc/capture/capture.go +++ b/cdc/capture/capture.go @@ -283,7 +283,8 @@ func (c *Capture) runEtcdWorker(ctx cdcContext.Context, reactor orchestrator.Rea if err != nil { return errors.Trace(err) } - if err := etcdWorker.Run(ctx, c.session, timerInterval); err != nil { + captureAddr := c.info.AdvertiseAddr + if err := etcdWorker.Run(ctx, c.session, timerInterval, captureAddr); err != nil { // We check ttl of lease instead of check `session.Done`, because // `session.Done` is only notified when etcd client establish a // new keepalive request, there could be a time window as long as diff --git a/cdc/metrics.go b/cdc/metrics.go index f27d4e1de66..6f6a85408a7 100644 --- a/cdc/metrics.go +++ b/cdc/metrics.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tiflow/cdc/sorter/unified" "github.com/pingcap/tiflow/pkg/actor" "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/orchestrator" "github.com/prometheus/client_golang/prometheus" ) @@ -44,6 +45,7 @@ func init() { etcd.InitMetrics(registry) initServerMetrics(registry) actor.InitMetrics(registry) + orchestrator.InitMetrics(registry) // Sorter metrics memory.InitMetrics(registry) unified.InitMetrics(registry) diff --git a/errors.toml b/errors.toml index d35160a82b3..6a219f446a8 100755 --- a/errors.toml +++ b/errors.toml @@ -241,6 +241,16 @@ error = ''' the etcd txn should be aborted and retried immediately ''' +["CDC:ErrEtcdTxnOpsExceed"] +error = ''' +patch ops:%d of a single changefeed exceed etcd txn max ops:%d +''' + +["CDC:ErrEtcdTxnSizeExceed"] +error = ''' +patch size:%d of a single changefeed exceed etcd txn max size:%d +''' + ["CDC:ErrEventFeedAborted"] error = ''' single event feed aborted diff --git a/go.mod b/go.mod index e89a34a8f47..0553961b3a7 100644 --- a/go.mod +++ b/go.mod @@ -9,6 +9,7 @@ require ( github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751 github.com/apache/pulsar-client-go v0.6.0 github.com/aws/aws-sdk-go v1.35.3 + github.com/benbjohnson/clock v1.1.0 github.com/bradleyjkemp/grpc-tools v0.2.5 github.com/cenkalti/backoff v2.2.1+incompatible github.com/chaos-mesh/go-sqlsmith v0.0.0-20211025024535-03ae33408684 diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index 73840c7a08d..f1e6bda773c 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -929,6 +929,1119 @@ "title": "Server", "type": "row" }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 1 + }, + "id": 266, + "panels": [ + { + "cards": { + "cardPadding": 1, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": null, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 2 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 262, + "legend": { + "show": true + }, + "pluginVersion": "6.1.6", + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(ticdc_etcd_worker_tick_reactor_duration_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, capture)", + "format": "heatmap", + "interval": "1", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "title": "EtcdWorker tick reactor duration", + "tooltip": { + "show": true, + "showHistogram": false + }, + "tooltipDecimals": null, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": null, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "auto", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 2 + }, + "hiddenSeries": false, + "id": 264, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "6.1.6", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.95, sum(rate(ticdc_etcd_worker_tick_reactor_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", + "interval": "", + "legendFormat": "{{capture}}-95", + "queryType": "randomWalk", + "refId": "A" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(ticdc_etcd_worker_tick_reactor_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", + "hide": false, + "interval": "", + "legendFormat": "{{capture}}-99", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "EtcdWorker tick reactor duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:2612", + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "$$hashKey": "object:2613", + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "max": null, + "min": 1, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 10 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 256, + "legend": { + "show": true + }, + "pluginVersion": "6.1.6", + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(ticdc_etcd_worker_etcd_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "EtcdWorker exec etcd txn duration", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": { + "unit": "s" + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 10 + }, + "hiddenSeries": false, + "id": 258, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "6.1.6", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.95, sum(rate(ticdc_etcd_worker_etcd_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", + "format": "time_series", + "interval": "", + "legendFormat": "{{capture}}-p95", + "queryType": "randomWalk", + "refId": "A" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(ticdc_etcd_worker_etcd_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", + "hide": false, + "interval": "", + "legendFormat": "{{lcapture}-p99}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "EtcdWorker exec etcd txn duration percentile", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:1612", + "format": "s", + "label": null, + "logBase": 2, + "max": null, + "min": null, + "show": true + }, + { + "$$hashKey": "object:1613", + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolatePurples", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 18 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 254, + "legend": { + "show": true + }, + "pluginVersion": "6.1.6", + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(ticdc_etcd_worker_etcd_txn_size_bytes_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, capture)", + "format": "heatmap", + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{le}}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "title": "EtcdWorker txn size ", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": null, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": null, + "format": "decbytes", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "auto", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 18 + }, + "hiddenSeries": false, + "id": 260, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "6.1.6", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.95, sum(rate(ticdc_etcd_worker_etcd_txn_size_bytes_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", + "interval": "", + "legendFormat": "{{capture}}-p95", + "queryType": "randomWalk", + "refId": "A" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(ticdc_etcd_worker_etcd_txn_size_bytes_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", + "hide": false, + "interval": "", + "legendFormat": "{{capture}}-p99", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "EtcdWorker txn size percentile", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:2055", + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "$$hashKey": "object:2056", + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "title": "EtcdWorker", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 1 + }, + "id": 266, + "panels": [ + { + "cards": { + "cardPadding": 1, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": null, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 2 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 262, + "legend": { + "show": true + }, + "pluginVersion": "6.1.6", + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(ticdc_etcd_worker_tick_reactor_duration_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, capture)", + "format": "heatmap", + "interval": "1", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "title": "EtcdWorker tick reactor duration", + "tooltip": { + "show": true, + "showHistogram": false + }, + "tooltipDecimals": null, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": null, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "auto", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 2 + }, + "hiddenSeries": false, + "id": 264, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "6.1.6", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.95, sum(rate(ticdc_etcd_worker_tick_reactor_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", + "interval": "", + "legendFormat": "{{capture}}-95", + "queryType": "randomWalk", + "refId": "A" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(ticdc_etcd_worker_tick_reactor_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", + "hide": false, + "interval": "", + "legendFormat": "{{capture}}-99", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "EtcdWorker tick reactor duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:2612", + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "$$hashKey": "object:2613", + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "max": null, + "min": 1, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 10 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 256, + "legend": { + "show": true + }, + "pluginVersion": "6.1.6", + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(ticdc_etcd_worker_etcd_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "EtcdWorker exec etcd txn duration", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": { + "unit": "s" + }, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 10 + }, + "hiddenSeries": false, + "id": 258, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "6.1.6", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.95, sum(rate(ticdc_etcd_worker_etcd_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", + "format": "time_series", + "interval": "", + "legendFormat": "{{capture}}-p95", + "queryType": "randomWalk", + "refId": "A" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(ticdc_etcd_worker_etcd_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", + "hide": false, + "interval": "", + "legendFormat": "{{capture}-p99}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "EtcdWorker exec etcd txn duration percentile", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:1612", + "format": "s", + "label": null, + "logBase": 2, + "max": null, + "min": null, + "show": true + }, + { + "$$hashKey": "object:1613", + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolatePurples", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 18 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 254, + "legend": { + "show": true + }, + "pluginVersion": "6.1.6", + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(ticdc_etcd_worker_etcd_txn_size_bytes_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, capture)", + "format": "heatmap", + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{le}}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "title": "EtcdWorker txn size ", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": null, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": null, + "format": "decbytes", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "auto", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 18 + }, + "hiddenSeries": false, + "id": 260, + "legend": { + "avg": false, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "6.1.6", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.95, sum(rate(ticdc_etcd_worker_etcd_txn_size_bytes_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", + "interval": "", + "legendFormat": "{{capture}}-p95", + "queryType": "randomWalk", + "refId": "A" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(ticdc_etcd_worker_etcd_txn_size_bytes_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", + "hide": false, + "interval": "", + "legendFormat": "{{capture}}-p99", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "EtcdWorker txn size percentile", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "$$hashKey": "object:2055", + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "$$hashKey": "object:2056", + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "title": "EtcdWorker", + "type": "row" + }, { "collapsed": true, "gridPos": { @@ -9681,5 +10794,5 @@ "timezone": "browser", "title": "Test-Cluster-TiCDC", "uid": "YiGL8hBZ1", - "version": 25 -} \ No newline at end of file + "version": 26 +} diff --git a/pkg/errors/errors.go b/pkg/errors/errors.go index ef42086d356..4e1011662d1 100644 --- a/pkg/errors/errors.go +++ b/pkg/errors/errors.go @@ -211,9 +211,11 @@ var ( // ErrEtcdSessionDone is used by etcd worker to signal a session done ErrEtcdSessionDone = errors.Normalize("the etcd session is done", errors.RFCCodeText("CDC:ErrEtcdSessionDone")) // ErrReactorFinished is used by reactor to signal a **normal** exit. - ErrReactorFinished = errors.Normalize("the reactor has done its job and should no longer be executed", errors.RFCCodeText("CDC:ErrReactorFinished")) - ErrLeaseTimeout = errors.Normalize("owner lease timeout", errors.RFCCodeText("CDC:ErrLeaseTimeout")) - ErrLeaseExpired = errors.Normalize("owner lease expired ", errors.RFCCodeText("CDC:ErrLeaseExpired")) + ErrReactorFinished = errors.Normalize("the reactor has done its job and should no longer be executed", errors.RFCCodeText("CDC:ErrReactorFinished")) + ErrLeaseTimeout = errors.Normalize("owner lease timeout", errors.RFCCodeText("CDC:ErrLeaseTimeout")) + ErrLeaseExpired = errors.Normalize("owner lease expired ", errors.RFCCodeText("CDC:ErrLeaseExpired")) + ErrEtcdTxnSizeExceed = errors.Normalize("patch size:%d of a single changefeed exceed etcd txn max size:%d", errors.RFCCodeText("CDC:ErrEtcdTxnSizeExceed")) + ErrEtcdTxnOpsExceed = errors.Normalize("patch ops:%d of a single changefeed exceed etcd txn max ops:%d", errors.RFCCodeText("CDC:ErrEtcdTxnOpsExceed")) // pipeline errors ErrSendToClosedPipeline = errors.Normalize("pipeline is closed, cannot send message", errors.RFCCodeText("CDC:ErrSendToClosedPipeline")) diff --git a/pkg/etcd/client.go b/pkg/etcd/client.go index 23a0e219a58..e1e286decb5 100644 --- a/pkg/etcd/client.go +++ b/pkg/etcd/client.go @@ -15,7 +15,9 @@ package etcd import ( "context" + "time" + "github.com/benbjohnson/clock" "github.com/pingcap/errors" "github.com/pingcap/log" cerrors "github.com/pingcap/tiflow/pkg/errors" @@ -41,6 +43,14 @@ const ( backoffBaseDelayInMs = 500 // in previous/backoff retry pkg, the DefaultMaxInterval = 60 * time.Second backoffMaxDelayInMs = 60 * 1000 + // If no msg comes from a etcd watchCh for etcdWatchChTimeoutDuration long, + // we should cancel the watchCh and request a new watchCh from etcd client + etcdWatchChTimeoutDuration = 10 * time.Second + // If no msg comes from a etcd watchCh for etcdRequestProgressDuration long, + // we should call RequestProgress of etcd client + etcdRequestProgressDuration = 1 * time.Second + // etcdWatchChBufferSize is arbitrarily specified, it will be modified in the future + etcdWatchChBufferSize = 16 ) // set to var instead of const for mocking the value to speedup test @@ -50,11 +60,13 @@ var maxTries int64 = 8 type Client struct { cli *clientv3.Client metrics map[string]prometheus.Counter + // clock is for making it easier to mock time-related data structures in unit tests + clock clock.Clock } // Wrap warps a clientv3.Client that provides etcd APIs required by TiCDC. func Wrap(cli *clientv3.Client, metrics map[string]prometheus.Counter) *Client { - return &Client{cli: cli, metrics: metrics} + return &Client{cli: cli, metrics: metrics, clock: clock.New()} } // Unwrap returns a clientv3.Client @@ -165,7 +177,68 @@ func (c *Client) TimeToLive(ctx context.Context, lease clientv3.LeaseID, opts .. // Watch delegates request to clientv3.Watcher.Watch func (c *Client) Watch(ctx context.Context, key string, opts ...clientv3.OpOption) clientv3.WatchChan { - return c.cli.Watch(ctx, key, opts...) + watchCh := make(chan clientv3.WatchResponse, etcdWatchChBufferSize) + go c.WatchWithChan(ctx, watchCh, key, opts...) + return watchCh +} + +// WatchWithChan maintains a watchCh and sends all msg from the watchCh to outCh +func (c *Client) WatchWithChan(ctx context.Context, outCh chan<- clientv3.WatchResponse, key string, opts ...clientv3.OpOption) { + defer func() { + close(outCh) + log.Info("WatchWithChan exited") + }() + var lastRevision int64 + watchCtx, cancel := context.WithCancel(ctx) + defer cancel() + watchCh := c.cli.Watch(watchCtx, key, opts...) + + ticker := c.clock.Ticker(etcdRequestProgressDuration) + defer ticker.Stop() + lastReceivedResponseTime := c.clock.Now() + + for { + select { + case <-ctx.Done(): + cancel() + return + case response := <-watchCh: + lastReceivedResponseTime = c.clock.Now() + if response.Err() == nil && !response.IsProgressNotify() { + lastRevision = response.Header.Revision + } + + Loop: + // we must loop here until the response is sent to outCh + // or otherwise the response will be lost + for { + select { + case <-ctx.Done(): + cancel() + return + case outCh <- response: // it may block here + break Loop + case <-ticker.C: + if c.clock.Since(lastReceivedResponseTime) >= etcdWatchChTimeoutDuration { + log.Warn("etcd client outCh blocking too long, the etcdWorker may be stuck", zap.Duration("duration", c.clock.Since(lastReceivedResponseTime))) + } + } + } + case <-ticker.C: + if err := c.RequestProgress(ctx); err != nil { + log.Warn("failed to request progress for etcd watcher", zap.Error(err)) + } + if c.clock.Since(lastReceivedResponseTime) >= etcdWatchChTimeoutDuration { + // cancel the last cancel func to reset it + log.Warn("etcd client watchCh blocking too long, reset the watchCh", zap.Duration("duration", c.clock.Since(lastReceivedResponseTime)), zap.Stack("stack")) + cancel() + watchCtx, cancel = context.WithCancel(ctx) + watchCh = c.cli.Watch(watchCtx, key, clientv3.WithPrefix(), clientv3.WithRev(lastRevision+1)) + // we need to reset lastReceivedResponseTime after reset Watch + lastReceivedResponseTime = c.clock.Now() + } + } + } } // RequestProgress requests a progress notify response be sent in all watch channels. diff --git a/pkg/orchestrator/batch.go b/pkg/orchestrator/batch.go new file mode 100644 index 00000000000..3bc98706167 --- /dev/null +++ b/pkg/orchestrator/batch.go @@ -0,0 +1,91 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package orchestrator + +import ( + "github.com/pingcap/errors" + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/orchestrator/util" +) + +const ( + // 1.25 MiB + // Ref: https://etcd.io/docs/v3.3/dev-guide/limit/ + etcdTxnMaxSize = 1024 * (1024 + 256) + // Ref: https://etcd.io/docs/v3.3/op-guide/configuration/#--max-txn-ops + etcdTxnMaxOps = 128 +) + +// getBatchChangedState has 4 return values: +// 1.batchChangedSate +// 2.number of patch apply to batchChangedState +// 3.size of batchChangedState in byte +// 4.error +func getBatchChangedState(state map[util.EtcdKey][]byte, patchGroups [][]DataPatch) (map[util.EtcdKey][]byte, int, int, error) { + num := 0 + totalSize := 0 + // store changedState of multiple changefeed + batchChangedState := make(map[util.EtcdKey][]byte) + for i, patches := range patchGroups { + changedState, changedSize, err := getChangedState(state, patches) + if err != nil { + return nil, 0, 0, err + } + // if a changefeed's changedState size is larger than etcdTxnMaxSize + // or the length of changedState is larger than etcdTxnMaxOps + // we should return an error instantly + if i == 0 { + if changedSize > etcdTxnMaxSize { + return nil, 0, 0, cerrors.ErrEtcdTxnSizeExceed.GenWithStackByArgs(changedSize, etcdTxnMaxSize) + } + if len(changedState) > etcdTxnMaxOps { + return nil, 0, 0, cerrors.ErrEtcdTxnOpsExceed.GenWithStackByArgs(len(changedState), etcdTxnMaxOps) + } + } + + // batchChangedState size should not exceeds the etcdTxnMaxSize limit + // and keys numbers should not exceeds the etcdTxnMaxOps limit + if totalSize+changedSize >= etcdTxnMaxSize || + len(batchChangedState)+len(changedState) >= etcdTxnMaxOps { + break + } + for k, v := range changedState { + batchChangedState[k] = v + } + num++ + totalSize += changedSize + } + return batchChangedState, num, totalSize, nil +} + +func getChangedState(state map[util.EtcdKey][]byte, patches []DataPatch) (map[util.EtcdKey][]byte, int, error) { + changedSet := make(map[util.EtcdKey]struct{}) + changeState := make(map[util.EtcdKey][]byte) + changedSize := 0 + for _, patch := range patches { + err := patch.Patch(state, changedSet) + if err != nil { + if cerrors.ErrEtcdIgnore.Equal(errors.Cause(err)) { + continue + } + return nil, 0, errors.Trace(err) + } + } + for k := range changedSet { + v := state[k] + changedSize += len(k.String())*2 + len(v) + changeState[k] = v + } + return changeState, changedSize, nil +} diff --git a/pkg/orchestrator/batch_test.go b/pkg/orchestrator/batch_test.go new file mode 100644 index 00000000000..95a7721872f --- /dev/null +++ b/pkg/orchestrator/batch_test.go @@ -0,0 +1,75 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package orchestrator + +import ( + "fmt" + "testing" + + "github.com/pingcap/tiflow/pkg/orchestrator/util" + "github.com/stretchr/testify/require" +) + +func TestGetBatchChangeState(t *testing.T) { + t.Parallel() + patchGroupSize := 1000 + patchGroup := make([][]DataPatch, patchGroupSize) + for i := 0; i < patchGroupSize; i++ { + i := i + patches := []DataPatch{&SingleDataPatch{ + Key: util.NewEtcdKey(fmt.Sprintf("/key%d", i)), + Func: func(old []byte) (newValue []byte, changed bool, err error) { + newValue = []byte(fmt.Sprintf("abc%d", i)) + return newValue, true, nil + }, + }} + patchGroup[i] = patches + } + rawState := make(map[util.EtcdKey][]byte) + changedState, n, size, err := getBatchChangedState(rawState, patchGroup) + require.Nil(t, err) + require.LessOrEqual(t, n, len(patchGroup)) + require.LessOrEqual(t, size, etcdTxnMaxSize) + require.LessOrEqual(t, len(changedState), etcdTxnMaxOps) + require.Equal(t, []byte(fmt.Sprintf("abc%d", 0)), changedState[util.NewEtcdKey("/key0")]) + + // test single patch exceed txn max size + largeSizePatches := []DataPatch{&SingleDataPatch{ + Key: util.NewEtcdKey("largePatch"), + Func: func(old []byte) (newValue []byte, changed bool, err error) { + newValue = make([]byte, etcdTxnMaxSize) + return newValue, true, nil + }, + }} + patchGroup = [][]DataPatch{largeSizePatches} + _, _, _, err = getBatchChangedState(rawState, patchGroup) + require.NotNil(t, err) + require.Contains(t, err.Error(), "a single changefeed exceed etcd txn max size") + + // test single patch exceed txn max ops + manyOpsPatches := make([]DataPatch, 0) + for i := 0; i <= etcdTxnMaxOps*2; i++ { + manyOpsPatches = append(manyOpsPatches, &SingleDataPatch{ + Key: util.NewEtcdKey(fmt.Sprintf("/key%d", i)), + Func: func(old []byte) (newValue []byte, changed bool, err error) { + newValue = []byte(fmt.Sprintf("abc%d", i)) + return newValue, true, nil + }, + }) + } + patchGroup = [][]DataPatch{manyOpsPatches} + _, _, _, err = getBatchChangedState(rawState, patchGroup) + require.NotNil(t, err) + require.Contains(t, err.Error(), "a single changefeed exceed etcd txn max ops") +} diff --git a/pkg/orchestrator/etcd_worker.go b/pkg/orchestrator/etcd_worker.go index 077f9ceb27e..cb402edac0c 100644 --- a/pkg/orchestrator/etcd_worker.go +++ b/pkg/orchestrator/etcd_worker.go @@ -20,18 +20,31 @@ import ( "time" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/log" cerrors "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/orchestrator/util" + "github.com/prometheus/client_golang/prometheus" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/clientv3/concurrency" + "go.etcd.io/etcd/etcdserver/etcdserverpb" "go.etcd.io/etcd/mvcc/mvccpb" "go.uber.org/zap" "go.uber.org/zap/zapcore" "golang.org/x/time/rate" ) +const ( + // etcdTxnTimeoutDuration represents the timeout duration for committing a + // transaction to Etcd + etcdTxnTimeoutDuration = 30 * time.Second + // etcdWorkerLogsWarnDuration when EtcdWorker commits a txn to etcd or ticks + // it reactor takes more than etcdWorkerLogsWarnDuration, it will print a log + etcdWorkerLogsWarnDuration = 1 * time.Second + deletionCounterKey = "/meta/ticdc-delete-etcd-key-count" +) + // EtcdWorker handles all interactions with Etcd type EtcdWorker struct { client *etcd.Client @@ -58,6 +71,15 @@ type EtcdWorker struct { // a `compare-and-swap` semantics, which is essential for implementing // snapshot isolation for Reactor ticks. deleteCounter int64 + + metrics *etcdWorkerMetrics +} + +type etcdWorkerMetrics struct { + // kv events related metrics + metricEtcdTxnSize prometheus.Observer + metricEtcdTxnDuration prometheus.Observer + metricEtcdWorkerTickDuration prometheus.Observer } type etcdUpdate struct { @@ -85,30 +107,35 @@ func NewEtcdWorker(client *etcd.Client, prefix string, reactor Reactor, initStat }, nil } -const ( - etcdRequestProgressDuration = 2 * time.Second - deletionCounterKey = "/meta/ticdc-delete-etcd-key-count" -) +func (worker *EtcdWorker) initMetrics(captureAddr string) { + metrics := &etcdWorkerMetrics{} + metrics.metricEtcdTxnSize = etcdTxnSize.WithLabelValues(captureAddr) + metrics.metricEtcdTxnDuration = etcdTxnExecDuration.WithLabelValues(captureAddr) + metrics.metricEtcdWorkerTickDuration = etcdWorkerTickDuration.WithLabelValues(captureAddr) + worker.metrics = metrics +} // Run starts the EtcdWorker event loop. // A tick is generated either on a timer whose interval is timerInterval, or on an Etcd event. // If the specified etcd session is Done, this Run function will exit with cerrors.ErrEtcdSessionDone. // And the specified etcd session is nil-safety. -func (worker *EtcdWorker) Run(ctx context.Context, session *concurrency.Session, timerInterval time.Duration) error { +func (worker *EtcdWorker) Run(ctx context.Context, session *concurrency.Session, timerInterval time.Duration, captureAddr string) error { defer worker.cleanUp() + worker.initMetrics(captureAddr) + err := worker.syncRawState(ctx) if err != nil { return errors.Trace(err) } - ctx1, cancel := context.WithCancel(ctx) - defer cancel() - ticker := time.NewTicker(timerInterval) defer ticker.Stop() - watchCh := worker.client.Watch(ctx1, worker.prefix.String(), clientv3.WithPrefix(), clientv3.WithRev(worker.revision+1)) + watchCtx, cancel := context.WithCancel(ctx) + defer cancel() + watchCh := worker.client.Watch(watchCtx, worker.prefix.String(), clientv3.WithPrefix(), clientv3.WithRev(worker.revision+1)) + var ( pendingPatches [][]DataPatch exiting bool @@ -120,14 +147,12 @@ func (worker *EtcdWorker) Run(ctx context.Context, session *concurrency.Session, // should never be closed sessionDone = make(chan struct{}) } - lastReceivedEventTime := time.Now() // tickRate represents the number of times EtcdWorker can tick // the reactor per second tickRate := time.Second / timerInterval rl := rate.NewLimiter(rate.Limit(tickRate), 1) for { - var response clientv3.WatchResponse select { case <-ctx.Done(): return ctx.Err() @@ -135,33 +160,46 @@ func (worker *EtcdWorker) Run(ctx context.Context, session *concurrency.Session, return cerrors.ErrEtcdSessionDone.GenWithStackByArgs() case <-ticker.C: // There is no new event to handle on timer ticks, so we have nothing here. - if time.Since(lastReceivedEventTime) > etcdRequestProgressDuration { - if err := worker.client.RequestProgress(ctx); err != nil { - log.Warn("failed to request progress for etcd watcher", zap.Error(err)) - } - } - case response = <-watchCh: + case response := <-watchCh: // In this select case, we receive new events from Etcd, and call handleEvent if appropriate. if err := response.Err(); err != nil { return errors.Trace(err) } - lastReceivedEventTime = time.Now() - // Check whether the response is stale. - if worker.revision >= response.Header.GetRevision() { + // ProgressNotify implies no new events. + if response.IsProgressNotify() { + log.Debug("Etcd progress notification", + zap.Int64("revision", response.Header.GetRevision())) + // Note that we don't need to update the revision here, and we + // should not do so, because the revision of the progress notification + // may not satisfy the strict monotonicity we have expected. + // + // Updating `worker.revision` can cause a useful event with the + // same revision to be dropped erroneously. + // + // Refer to https://etcd.io/docs/v3.3/dev-guide/interacting_v3/#watch-progress + // "Note: The revision number in the progress notify response is the revision + // from the local etcd server node that the watch stream is connected to. [...]" + // This implies that the progress notification will NOT go through the raft + // consensus, thereby NOT affecting the revision (index). continue } - worker.revision = response.Header.GetRevision() - // ProgressNotify implies no new events. - if response.IsProgressNotify() { + // Check whether the response is stale. + if worker.revision >= response.Header.GetRevision() { + log.Info("Stale Etcd event dropped", + zap.Int64("event-revision", response.Header.GetRevision()), + zap.Int64("previous-revision", worker.revision), + zap.Any("events", response.Events)) continue } + worker.revision = response.Header.GetRevision() for _, event := range response.Events { // handleEvent will apply the event to our internal `rawState`. worker.handleEvent(ctx, event) } + } if len(pendingPatches) > 0 { @@ -196,8 +234,14 @@ func (worker *EtcdWorker) Run(ctx context.Context, session *concurrency.Session, if !rl.Allow() { continue } + startTime := time.Now() // it is safe that a batch of updates has been applied to worker.state before worker.reactor.Tick nextState, err := worker.reactor.Tick(ctx, worker.state) + costTime := time.Since(startTime) + if costTime > etcdWorkerLogsWarnDuration { + log.Warn("EtcdWorker reactor tick took too long", zap.Duration("duration", costTime)) + } + worker.metrics.metricEtcdWorkerTickDuration.Observe(costTime.Seconds()) if err != nil { if !cerrors.ErrReactorFinished.Equal(errors.Cause(err)) { return errors.Trace(err) @@ -284,33 +328,31 @@ func (worker *EtcdWorker) cloneRawState() map[util.EtcdKey][]byte { } func (worker *EtcdWorker) applyPatchGroups(ctx context.Context, patchGroups [][]DataPatch) ([][]DataPatch, error) { + state := worker.cloneRawState() for len(patchGroups) > 0 { - patches := patchGroups[0] - err := worker.applyPatches(ctx, patches) + changeSate, n, size, err := getBatchChangedState(state, patchGroups) if err != nil { return patchGroups, err } - patchGroups = patchGroups[1:] + err = worker.commitChangedState(ctx, changeSate, size) + if err != nil { + return patchGroups, err + } + patchGroups = patchGroups[n:] } return patchGroups, nil } -func (worker *EtcdWorker) applyPatches(ctx context.Context, patches []DataPatch) error { - state := worker.cloneRawState() - changedSet := make(map[util.EtcdKey]struct{}) - for _, patch := range patches { - err := patch.Patch(state, changedSet) - if err != nil { - if cerrors.ErrEtcdIgnore.Equal(errors.Cause(err)) { - continue - } - return errors.Trace(err) - } +func (worker *EtcdWorker) commitChangedState(ctx context.Context, changedState map[util.EtcdKey][]byte, size int) error { + if len(changedState) == 0 { + return nil } - cmps := make([]clientv3.Cmp, 0, len(changedSet)) - ops := make([]clientv3.Op, 0, len(changedSet)) + + cmps := make([]clientv3.Cmp, 0, len(changedState)) + ops := make([]clientv3.Op, 0, len(changedState)) hasDelete := false - for key := range changedSet { + + for key, value := range changedState { // make sure someone else has not updated the key after the last snapshot var cmp clientv3.Cmp if entry, ok := worker.rawState[key]; ok { @@ -322,7 +364,6 @@ func (worker *EtcdWorker) applyPatches(ctx context.Context, patches []DataPatch) } cmps = append(cmps, cmp) - value := state[key] var op clientv3.Op if value != nil { op = clientv3.OpPut(key.String(), string(value)) @@ -344,7 +385,26 @@ func (worker *EtcdWorker) applyPatches(ctx context.Context, patches []DataPatch) panic("unreachable") } - resp, err := worker.client.Txn(ctx).If(cmps...).Then(ops...).Commit() + worker.metrics.metricEtcdTxnSize.Observe(float64(size)) + startTime := time.Now() + + txnCtx, cancel := context.WithTimeout(ctx, etcdTxnTimeoutDuration) + resp, err := worker.client.Txn(txnCtx).If(cmps...).Then(ops...).Commit() + cancel() + + // For testing the situation where we have a progress notification that + // has the same revision as the committed Etcd transaction. + failpoint.Inject("InjectProgressRequestAfterCommit", func() { + if err := worker.client.RequestProgress(ctx); err != nil { + failpoint.Return(errors.Trace(err)) + } + }) + + costTime := time.Since(startTime) + if costTime > etcdWorkerLogsWarnDuration { + log.Warn("Etcd transaction took too long", zap.Duration("duration", costTime)) + } + worker.metrics.metricEtcdTxnDuration.Observe(costTime.Seconds()) if err != nil { return errors.Trace(err) } @@ -355,6 +415,8 @@ func (worker *EtcdWorker) applyPatches(ctx context.Context, patches []DataPatch) return nil } + // Logs the conditions for the failed Etcd transaction. + worker.logEtcdCmps(cmps) return cerrors.ErrEtcdTryAgain.GenWithStackByArgs() } @@ -370,19 +432,34 @@ func (worker *EtcdWorker) applyUpdates() error { return nil } -func logEtcdOps(ops []clientv3.Op, commited bool) { - if log.GetLevel() != zapcore.DebugLevel || len(ops) == 0 { +func logEtcdOps(ops []clientv3.Op, committed bool) { + if committed && (log.GetLevel() != zapcore.DebugLevel || len(ops) == 0) { return } - log.Debug("[etcd worker] ==========Update State to ETCD==========") + logFn := log.Debug + if !committed { + logFn = log.Info + } + + logFn("[etcd worker] ==========Update State to ETCD==========") for _, op := range ops { if op.IsDelete() { - log.Debug("[etcd worker] delete key", zap.ByteString("key", op.KeyBytes())) + logFn("[etcd worker] delete key", zap.ByteString("key", op.KeyBytes())) } else { - log.Debug("[etcd worker] put key", zap.ByteString("key", op.KeyBytes()), zap.ByteString("value", op.ValueBytes())) + logFn("[etcd worker] put key", zap.ByteString("key", op.KeyBytes()), zap.ByteString("value", op.ValueBytes())) } } - log.Debug("[etcd worker] ============State Commit=============", zap.Bool("committed", commited)) + logFn("[etcd worker] ============State Commit=============", zap.Bool("committed", committed)) +} + +func (worker *EtcdWorker) logEtcdCmps(cmps []clientv3.Cmp) { + log.Info("[etcd worker] ==========Failed Etcd Txn Cmps==========") + for _, cmp := range cmps { + cmp := etcdserverpb.Compare(cmp) + log.Info("[etcd worker] compare", + zap.String("cmp", cmp.String())) + } + log.Info("[etcd worker] ============End Failed Etcd Txn Cmps=============") } func (worker *EtcdWorker) cleanUp() { diff --git a/pkg/orchestrator/etcd_worker_bank_test.go b/pkg/orchestrator/etcd_worker_bank_test.go index 49a7561a50e..7321ce6033c 100644 --- a/pkg/orchestrator/etcd_worker_bank_test.go +++ b/pkg/orchestrator/etcd_worker_bank_test.go @@ -23,6 +23,7 @@ import ( "time" "github.com/pingcap/check" + "github.com/pingcap/failpoint" "github.com/pingcap/log" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/orchestrator/util" @@ -121,6 +122,12 @@ func (b *bankReactor) Tick(ctx context.Context, state ReactorState) (nextState R func (s *etcdWorkerSuite) TestEtcdBank(c *check.C) { defer testleak.AfterTest(c)() + + _ = failpoint.Enable("github.com/pingcap/ticdc/pkg/orchestrator/InjectProgressRequestAfterCommit", "10%return(true)") + defer func() { + _ = failpoint.Disable("github.com/pingcap/ticdc/pkg/orchestrator/InjectProgressRequestAfterCommit") + }() + totalAccountNumber := 25 workerNumber := 10 var wg sync.WaitGroup @@ -150,7 +157,7 @@ func (s *etcdWorkerSuite) TestEtcdBank(c *check.C) { accountNumber: totalAccountNumber, }, &bankReactorState{c: c, index: i, account: make([]int, totalAccountNumber)}) c.Assert(err, check.IsNil) - err = worker.Run(ctx, nil, 100*time.Millisecond) + err = worker.Run(ctx, nil, 100*time.Millisecond, "127.0.0.1") if err == nil || err.Error() == "etcdserver: request timed out" { continue } diff --git a/pkg/orchestrator/etcd_worker_test.go b/pkg/orchestrator/etcd_worker_test.go index 1161399e877..d7715eb8d0b 100644 --- a/pkg/orchestrator/etcd_worker_test.go +++ b/pkg/orchestrator/etcd_worker_test.go @@ -267,7 +267,7 @@ func (s *etcdWorkerSuite) TestEtcdSum(c *check.C) { return errors.Trace(err) } - return errors.Trace(etcdWorker.Run(ctx, nil, 10*time.Millisecond)) + return errors.Trace(etcdWorker.Run(ctx, nil, 10*time.Millisecond, "127.0.0.1")) }) } @@ -352,7 +352,7 @@ func (s *etcdWorkerSuite) TestLinearizability(c *check.C) { c.Assert(err, check.IsNil) errg := &errgroup.Group{} errg.Go(func() error { - return reactor.Run(ctx, nil, 10*time.Millisecond) + return reactor.Run(ctx, nil, 10*time.Millisecond, "127.0.0.1") }) time.Sleep(500 * time.Millisecond) @@ -437,7 +437,7 @@ func (s *etcdWorkerSuite) TestFinished(c *check.C) { state: make(map[string]string), }) c.Assert(err, check.IsNil) - err = reactor.Run(ctx, nil, 10*time.Millisecond) + err = reactor.Run(ctx, nil, 10*time.Millisecond, "127.0.0.1") c.Assert(err, check.IsNil) resp, err := cli.Get(ctx, prefix+"/key1") c.Assert(err, check.IsNil) @@ -506,7 +506,7 @@ func (s *etcdWorkerSuite) TestCover(c *check.C) { state: make(map[string]string), }) c.Assert(err, check.IsNil) - err = reactor.Run(ctx, nil, 10*time.Millisecond) + err = reactor.Run(ctx, nil, 10*time.Millisecond, "127.0.0.1") c.Assert(err, check.IsNil) resp, err := cli.Get(ctx, prefix+"/key1") c.Assert(err, check.IsNil) @@ -585,7 +585,7 @@ func (s *etcdWorkerSuite) TestEmptyTxn(c *check.C) { state: make(map[string]string), }) c.Assert(err, check.IsNil) - err = reactor.Run(ctx, nil, 10*time.Millisecond) + err = reactor.Run(ctx, nil, 10*time.Millisecond, "127.0.0.1") c.Assert(err, check.IsNil) resp, err := cli.Get(ctx, prefix+"/key1") c.Assert(err, check.IsNil) @@ -652,7 +652,7 @@ func (s *etcdWorkerSuite) TestEmptyOrNil(c *check.C) { state: make(map[string]string), }) c.Assert(err, check.IsNil) - err = reactor.Run(ctx, nil, 10*time.Millisecond) + err = reactor.Run(ctx, nil, 10*time.Millisecond, "127.0.0.1") c.Assert(err, check.IsNil) resp, err := cli.Get(ctx, prefix+"/key1") c.Assert(err, check.IsNil) @@ -733,7 +733,7 @@ func (s *etcdWorkerSuite) TestModifyAfterDelete(c *check.C) { wg.Add(1) go func() { defer wg.Done() - err := worker1.Run(ctx, nil, time.Millisecond*100) + err := worker1.Run(ctx, nil, time.Millisecond*100, "127.0.0.1") c.Assert(err, check.IsNil) }() @@ -748,7 +748,7 @@ func (s *etcdWorkerSuite) TestModifyAfterDelete(c *check.C) { }) c.Assert(err, check.IsNil) - err = worker2.Run(ctx, nil, time.Millisecond*100) + err = worker2.Run(ctx, nil, time.Millisecond*100, "127.0.0.1") c.Assert(err, check.IsNil) modifyReactor.waitOnCh <- struct{}{} diff --git a/pkg/orchestrator/interfaces.go b/pkg/orchestrator/interfaces.go index 4ab21ba6420..5e74fab592b 100644 --- a/pkg/orchestrator/interfaces.go +++ b/pkg/orchestrator/interfaces.go @@ -68,10 +68,10 @@ func (s *SingleDataPatch) Patch(valueMap map[util.EtcdKey][]byte, changedSet map return nil } -// MultiDatePatch represents an update to many keys -type MultiDatePatch func(valueMap map[util.EtcdKey][]byte, changedSet map[util.EtcdKey]struct{}) error +// MultiDataPatch represents an update to many keys +type MultiDataPatch func(valueMap map[util.EtcdKey][]byte, changedSet map[util.EtcdKey]struct{}) error // Patch implements the DataPatch interface -func (m MultiDatePatch) Patch(valueMap map[util.EtcdKey][]byte, changedSet map[util.EtcdKey]struct{}) error { +func (m MultiDataPatch) Patch(valueMap map[util.EtcdKey][]byte, changedSet map[util.EtcdKey]struct{}) error { return m(valueMap, changedSet) } diff --git a/pkg/orchestrator/metrics.go b/pkg/orchestrator/metrics.go new file mode 100644 index 00000000000..efbb242871a --- /dev/null +++ b/pkg/orchestrator/metrics.go @@ -0,0 +1,52 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package orchestrator + +import "github.com/prometheus/client_golang/prometheus" + +var ( + etcdTxnSize = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: "ticdc", + Subsystem: "etcd_worker", + Name: "etcd_txn_size_bytes", + Help: "Bucketed histogram of a etcd txn size.", + Buckets: prometheus.ExponentialBuckets(1, 2, 18), + }, []string{"capture"}) + + etcdTxnExecDuration = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: "ticdc", + Subsystem: "etcd_worker", + Name: "etcd_txn_exec_duration", + Help: "Bucketed histogram of processing time (s) of a etcd txn.", + Buckets: prometheus.ExponentialBuckets(0.002 /* 2 ms */, 2, 18), + }, []string{"capture"}) + + etcdWorkerTickDuration = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: "ticdc", + Subsystem: "etcd_worker", + Name: "tick_reactor_duration", + Help: "Bucketed histogram of etcdWorker tick reactor time (s).", + Buckets: prometheus.ExponentialBuckets(0.01 /* 10 ms */, 2, 18), + }, []string{"capture"}) +) + +// InitMetrics registers all metrics in this file +func InitMetrics(registry *prometheus.Registry) { + registry.MustRegister(etcdTxnSize) + registry.MustRegister(etcdTxnExecDuration) + registry.MustRegister(etcdWorkerTickDuration) +} diff --git a/pkg/orchestrator/reactor_state.go b/pkg/orchestrator/reactor_state.go index 3ee96d15a6f..c1d253e2308 100644 --- a/pkg/orchestrator/reactor_state.go +++ b/pkg/orchestrator/reactor_state.go @@ -100,6 +100,7 @@ func (s *GlobalReactorState) Update(key util.EtcdKey, value []byte, _ bool) erro } // GetPatches implements the ReactorState interface +// Every []DataPatch slice in [][]DataPatch slice is the patches of a ChangefeedReactorState func (s *GlobalReactorState) GetPatches() [][]DataPatch { pendingPatches := s.pendingPatches for _, changefeedState := range s.Changefeeds { diff --git a/testing_utils/cdc_state_checker/cdc_monitor.go b/testing_utils/cdc_state_checker/cdc_monitor.go index 5c2ee8a6dcf..b5bc997860e 100644 --- a/testing_utils/cdc_state_checker/cdc_monitor.go +++ b/testing_utils/cdc_state_checker/cdc_monitor.go @@ -89,7 +89,7 @@ func newCDCMonitor(ctx context.Context, pd string, credential *security.Credenti func (m *cdcMonitor) run(ctx context.Context) error { log.Debug("start running cdcMonitor") - err := m.etcdWorker.Run(ctx, nil, 200*time.Millisecond) + err := m.etcdWorker.Run(ctx, nil, 200*time.Millisecond, "127.0.0.1") log.Error("etcdWorker exited: test-case-failed", zap.Error(err)) log.Info("CDC state", zap.Reflect("state", m.reactor.state)) return err From e85d1e167b0503c3fe9a707e0488df13fce38ca0 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 20 Dec 2021 18:07:46 +0800 Subject: [PATCH 15/48] http_api (ticdc): check --cert-allowed-cn before add server common name (#3628) (#3882) --- cdc/capture/capture.go | 6 ++ cdc/capture/http_handler.go | 10 --- cdc/http_status.go | 14 +++- cdc/http_status_test.go | 146 ++++++++++++++++++++++++++++++++++++ pkg/security/test_util.go | 132 ++++++++++++++++++++++++++++++++ 5 files changed, 294 insertions(+), 14 deletions(-) create mode 100644 pkg/security/test_util.go diff --git a/cdc/capture/capture.go b/cdc/capture/capture.go index e7d8cf47e46..a351d30622b 100644 --- a/cdc/capture/capture.go +++ b/cdc/capture/capture.go @@ -81,6 +81,12 @@ func NewCapture(pdClient pd.Client, kvStorage tidbkv.Storage, etcdClient *etcd.C } } +func NewCapture4Test() *Capture { + return &Capture{ + info: &model.CaptureInfo{ID: "capture-for-test", AdvertiseAddr: "127.0.0.1", Version: "test"}, + } +} + func (c *Capture) reset(ctx context.Context) error { c.captureMu.Lock() defer c.captureMu.Unlock() diff --git a/cdc/capture/http_handler.go b/cdc/capture/http_handler.go index 133e7cc9e92..a13f2ffffe2 100644 --- a/cdc/capture/http_handler.go +++ b/cdc/capture/http_handler.go @@ -672,11 +672,6 @@ func (h *HTTPHandler) ListCapture(c *gin.Context) { // @Failure 500,400 {object} model.HTTPError // @Router /api/v1/status [get] func (h *HTTPHandler) ServerStatus(c *gin.Context) { - if !h.capture.IsOwner() { - h.forwardToOwner(c) - return - } - status := model.ServerStatus{ Version: version.ReleaseVersion, GitHash: version.GitHash, @@ -697,11 +692,6 @@ func (h *HTTPHandler) ServerStatus(c *gin.Context) { // @Failure 500 {object} model.HTTPError // @Router /api/v1/health [get] func (h *HTTPHandler) Health(c *gin.Context) { - if !h.capture.IsOwner() { - h.forwardToOwner(c) - return - } - ctx := c.Request.Context() if _, err := h.capture.GetOwner(ctx); err != nil { diff --git a/cdc/http_status.go b/cdc/http_status.go index 4b3890b3238..f7e6e2d0e59 100644 --- a/cdc/http_status.go +++ b/cdc/http_status.go @@ -59,11 +59,17 @@ func (s *Server) startStatusHTTP() error { prometheus.DefaultGatherer = registry router.Any("/metrics", gin.WrapH(promhttp.Handler())) - err := conf.Security.AddSelfCommonName() - if err != nil { - log.Error("status server set tls config failed", zap.Error(err)) - return errors.Trace(err) + // if CertAllowedCN was specified, we should add server's common name + // otherwise, https requests sent to non-owner capture can't be forward + // to owner + if len(conf.Security.CertAllowedCN) != 0 { + err := conf.Security.AddSelfCommonName() + if err != nil { + log.Error("status server set tls config failed", zap.Error(err)) + return errors.Trace(err) + } } + tlsConfig, err := conf.Security.ToTLSConfigWithVerify() if err != nil { log.Error("status server get tls config failed", zap.Error(err)) diff --git a/cdc/http_status_test.go b/cdc/http_status_test.go index 3e758b62351..9150a141ce1 100644 --- a/cdc/http_status_test.go +++ b/cdc/http_status_test.go @@ -15,16 +15,27 @@ package cdc import ( "bytes" + "context" + "crypto/tls" + "encoding/json" "fmt" "io" "net/http" "net/url" + "strings" "time" "github.com/pingcap/check" "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/br/pkg/httputil" + "github.com/pingcap/tiflow/cdc/capture" + "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/retry" + security2 "github.com/pingcap/tiflow/pkg/security" "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/tikv/pd/pkg/tempurl" "go.etcd.io/etcd/clientv3/concurrency" ) @@ -159,3 +170,138 @@ func testHandleFailpoint(c *check.C) { }) c.Assert(failpointHit, check.IsFalse) } + +func (s *httpStatusSuite) TestServerTLSWithoutCommonName(c *check.C) { + defer testleak.AfterTest(c) + addr := tempurl.Alloc()[len("http://"):] + // Do not specify common name + security, err := security2.NewCredential4Test("") + c.Assert(err, check.IsNil) + conf := config.GetDefaultServerConfig() + conf.Addr = addr + conf.AdvertiseAddr = addr + conf.Security = &security + config.StoreGlobalServerConfig(conf) + + server, err := NewServer([]string{"https://127.0.0.1:2379"}) + server.capture = capture.NewCapture4Test() + c.Assert(err, check.IsNil) + err = server.startStatusHTTP() + c.Assert(err, check.IsNil) + defer func() { + c.Assert(server.statusServer.Close(), check.IsNil) + }() + + statusURL := fmt.Sprintf("https://%s/api/v1/status", addr) + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + + // test cli sends request without a cert will success + err = retry.Do(ctx, func() error { + tr := &http.Transport{ + TLSClientConfig: &tls.Config{InsecureSkipVerify: true}, + } + cli := &http.Client{Transport: tr} + resp, err := cli.Get(statusURL) + if err != nil { + return err + } + decoder := json.NewDecoder(resp.Body) + captureInfo := &model.CaptureInfo{} + err = decoder.Decode(captureInfo) + c.Assert(err, check.IsNil) + c.Assert(captureInfo.ID, check.Equals, server.capture.Info().ID) + resp.Body.Close() + return nil + }, retry.WithMaxTries(retryTime), retry.WithBackoffBaseDelay(50), retry.WithIsRetryableErr(cerrors.IsRetryableError)) + c.Assert(err, check.IsNil) + + // test cli sends request with a cert will success + err = retry.Do(ctx, func() error { + tlsConfig, err := security.ToTLSConfigWithVerify() + if err != nil { + c.Assert(err, check.IsNil) + } + cli := httputil.NewClient(tlsConfig) + resp, err := cli.Get(statusURL) + if err != nil { + return err + } + decoder := json.NewDecoder(resp.Body) + captureInfo := &model.CaptureInfo{} + err = decoder.Decode(captureInfo) + c.Assert(err, check.IsNil) + c.Assert(captureInfo.ID, check.Equals, server.capture.Info().ID) + resp.Body.Close() + return nil + }, retry.WithMaxTries(retryTime), retry.WithBackoffBaseDelay(50), retry.WithIsRetryableErr(cerrors.IsRetryableError)) + c.Assert(err, check.IsNil) +} + +// +func (s *httpStatusSuite) TestServerTLSWithCommonName(c *check.C) { + defer testleak.AfterTest(c) + addr := tempurl.Alloc()[len("http://"):] + // specify a common name + security, err := security2.NewCredential4Test("test") + c.Assert(err, check.IsNil) + conf := config.GetDefaultServerConfig() + conf.Addr = addr + conf.AdvertiseAddr = addr + conf.Security = &security + config.StoreGlobalServerConfig(conf) + + server, err := NewServer([]string{"https://127.0.0.1:2379"}) + server.capture = capture.NewCapture4Test() + c.Assert(err, check.IsNil) + err = server.startStatusHTTP() + c.Assert(err, check.IsNil) + defer func() { + c.Assert(server.statusServer.Close(), check.IsNil) + }() + + statusURL := fmt.Sprintf("https://%s/api/v1/status", addr) + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) + defer cancel() + + // test cli sends request without a cert will fail + err = retry.Do(ctx, func() error { + tr := &http.Transport{ + TLSClientConfig: &tls.Config{InsecureSkipVerify: true}, + } + cli := &http.Client{Transport: tr} + resp, err := cli.Get(statusURL) + if err != nil { + return err + } + decoder := json.NewDecoder(resp.Body) + captureInfo := &model.CaptureInfo{} + err = decoder.Decode(captureInfo) + c.Assert(err, check.IsNil) + c.Assert(captureInfo.ID, check.Equals, server.capture.Info().ID) + resp.Body.Close() + return nil + }, retry.WithMaxTries(retryTime), retry.WithBackoffBaseDelay(50), retry.WithIsRetryableErr(cerrors.IsRetryableError)) + c.Assert(strings.Contains(err.Error(), "remote error: tls: bad certificate"), check.IsTrue) + + // test cli sends request with a cert will success + err = retry.Do(ctx, func() error { + tlsConfig, err := security.ToTLSConfigWithVerify() + if err != nil { + c.Assert(err, check.IsNil) + } + cli := httputil.NewClient(tlsConfig) + resp, err := cli.Get(statusURL) + if err != nil { + return err + } + decoder := json.NewDecoder(resp.Body) + captureInfo := &model.CaptureInfo{} + err = decoder.Decode(captureInfo) + c.Assert(err, check.IsNil) + c.Assert(captureInfo.ID, check.Equals, server.capture.Info().ID) + resp.Body.Close() + return nil + }, retry.WithMaxTries(retryTime), retry.WithBackoffBaseDelay(50), retry.WithIsRetryableErr(cerrors.IsRetryableError)) + c.Assert(err, check.IsNil) +} diff --git a/pkg/security/test_util.go b/pkg/security/test_util.go new file mode 100644 index 00000000000..652c3549842 --- /dev/null +++ b/pkg/security/test_util.go @@ -0,0 +1,132 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package security + +import "os" + +// All these certificates or keys are use for testing only. +var certPem = `-----BEGIN CERTIFICATE----- +MIIDjzCCAnegAwIBAgIUWBTDQm4xOYDxZBTkpCQouREtT8QwDQYJKoZIhvcNAQEL +BQAwVzELMAkGA1UEBhMCQ04xEDAOBgNVBAgTB0JlaWppbmcxEDAOBgNVBAcTB0Jl +aWppbmcxEDAOBgNVBAoTB1BpbmdDQVAxEjAQBgNVBAMTCU15IG93biBDQTAgFw0y +MDAyMTgwOTExMDBaGA8yMTIwMDEyNTA5MTEwMFowFjEUMBIGA1UEAxMLdGlkYi1z +ZXJ2ZXIwggEiMA0GCSqGSIb3DQEBAQUAA4IBDwAwggEKAoIBAQCr2ZxAb+dItEQz +avuza0IoIT/UolC9XTGaQiCUUPZUMN9hb4KYEwTks1ZthHovTIJUdTwHtpWfDUWx +uIXhOlRjfD+viY4aXtBsaK8xi9F7o2HbFQ5O9y3AXK/YW+u0FfWtnn/xAtvPUgUc +61NXtBTMvNard9ICIXW+FWxLcFSaHpC9ZTyr13KWmZRDbai1JFeaKvATMW30r7Dd +Ur1npppzt7ZdG6tU/FuqBBSrZtuVSGKLwVx0JQDw16eVan4friY3ZPNVoZvkKyvt +I1LsBYMQ8p+ijtbcftvMqWZFVw95F1+3C2JIjWN9ujGmvJr+dtPIE8T/J8tT9Jif +9vz16nOLAgMBAAGjgZEwgY4wDgYDVR0PAQH/BAQDAgWgMB0GA1UdJQQWMBQGCCsG +AQUFBwMBBggrBgEFBQcDAjAMBgNVHRMBAf8EAjAAMB0GA1UdDgQWBBRVB/Bvdzvh +6WQRWpc9SzcbXLz77zAfBgNVHSMEGDAWgBSdAhKsS8BKSOidoGCUYNeaFma4/zAP +BgNVHREECDAGhwR/AAABMA0GCSqGSIb3DQEBCwUAA4IBAQAAqg5pgGQqORKRSdlY +wzVvzKaulpvjZfVMM6YiOUtmlU0CGWq7E3gLFzkvebpU0KsFlbyZ92h/2Fw5Ay2b +kxkCy18mJ4lGkvF0cU4UD3XheFMvD2QWWRX4WPpAhStofrWOXeyq3Div2+fQjMJd +kyeWUzPU7T467IWUHOWNsFAjfVHNsmG45qLGt+XQckHTvASX5IvN+5tkRUCW30vO +b3BdDQUFglGTUFU2epaZGTti0SYiRiY+9R3zFWX4uBcEBYhk9e/0BU8FqdWW5GjI +pFpH9t64CjKIdRQXpIn4cogK/GwyuRuDPV/RkMjrIqOi7pGejXwyDe9avHFVR6re +oowA +-----END CERTIFICATE-----` + +var caPem = `-----BEGIN CERTIFICATE----- +MIIDgDCCAmigAwIBAgIUHWvlRJydvYTR0ot3b8f6IlSHcGUwDQYJKoZIhvcNAQEL +BQAwVzELMAkGA1UEBhMCQ04xEDAOBgNVBAgTB0JlaWppbmcxEDAOBgNVBAcTB0Jl +aWppbmcxEDAOBgNVBAoTB1BpbmdDQVAxEjAQBgNVBAMTCU15IG93biBDQTAgFw0y +MDAyMTgwNzQxMDBaGA8yMTIwMDEyNTA3NDEwMFowVzELMAkGA1UEBhMCQ04xEDAO +BgNVBAgTB0JlaWppbmcxEDAOBgNVBAcTB0JlaWppbmcxEDAOBgNVBAoTB1BpbmdD +QVAxEjAQBgNVBAMTCU15IG93biBDQTCCASIwDQYJKoZIhvcNAQEBBQADggEPADCC +AQoCggEBAOAdNtjanFhPaKJHQjr7+h/Cpps5bLc6S1vmgi/EIi9PKv3eyDgtlW1r +As2sjXRMHjcuZp2hHJ9r9FrMQD1rQQq5vJzQqM+eyWLc2tyZWXNWkZVvpjU4Hy5k +jZFLXoyHgAvps/LGu81F5Lk5CvLHswWTyGQUCFi1l/cYcQg6AExh2pO/WJu4hQhe +1mBBIKsJhZ5b5tWruLeI+YIjD1oo1ADMHYLK1BHON2fUmUHRGbrYKu4yCuyip3wn +rbVlpabn7l1JBMatCUJLHR6VWQ2MNjrOXAEUYm4xGEN+tUYyUOGl5mHFguLl3OIn +wj+1dT3WOr/NraPYlwVOnAd9GNbPJj0CAwEAAaNCMEAwDgYDVR0PAQH/BAQDAgEG +MA8GA1UdEwEB/wQFMAMBAf8wHQYDVR0OBBYEFJ0CEqxLwEpI6J2gYJRg15oWZrj/ +MA0GCSqGSIb3DQEBCwUAA4IBAQCf8xRf7q1xAaGrc9HCPvN4OFkxDwz1CifrvrLR +ZgIWGUdCHDW2D1IiWKZQWeJKC1otA5x0hrS5kEGfkLFhADEU4txwp70DQaBArPti +pSgheIEbaT0H3BUTYSgS3VL2HjxN5OVMN6jNG3rWyxnJpNOCsJhhJXPK50CRZ7fk +Dcodj6FfEM2bfp2bGkxyVtUch7eepfUVbslXa7jE7Y8M3cr9NoLUcSP6D1RJWkNd +dBQoUsb6Ckq27ozEKOgwuBVv4BrrbFN//+7WHP8Vy6sSMyd+dJLBi6wehJjQhIz6 +vqLWE81rSJuxZqjLpCkFdeEF+9SRjWegU0ZDM4V+YeX53BPC +-----END CERTIFICATE----- +` + +var keyPem = `-----BEGIN RSA PRIVATE KEY----- +MIIEogIBAAKCAQEAq9mcQG/nSLREM2r7s2tCKCE/1KJQvV0xmkIglFD2VDDfYW+C +mBME5LNWbYR6L0yCVHU8B7aVnw1FsbiF4TpUY3w/r4mOGl7QbGivMYvRe6Nh2xUO +TvctwFyv2FvrtBX1rZ5/8QLbz1IFHOtTV7QUzLzWq3fSAiF1vhVsS3BUmh6QvWU8 +q9dylpmUQ22otSRXmirwEzFt9K+w3VK9Z6aac7e2XRurVPxbqgQUq2bblUhii8Fc +dCUA8NenlWp+H64mN2TzVaGb5Csr7SNS7AWDEPKfoo7W3H7bzKlmRVcPeRdftwti +SI1jfboxprya/nbTyBPE/yfLU/SYn/b89epziwIDAQABAoIBACPlI08OULgN90Tq +LsLuP3ZUY5nNgaHcKnU3JMj2FE3Hm5ElkpijOF1w3Dep+T+R8pMjnbNavuvnAMy7 +ZzOBVIknNcI7sDPv5AcQ4q8trkbt/I2fW0rBNIw+j/hYUuZdw+BNABpeZ31pe2nr ++Y+TLNkLBKfyMiqBxK88mE81mmZKblyvXCawW0A/iDDJ7fPNqoGF+y9ylTYaNRPk +aJGnaEZobJ4Lm5tSqW4gRX2ft6Hm67RkvVaopPFnlkvfusXUTFUqEVQCURRUqXbf +1ah2chUHxj22UdY9540H5yVNgEP3oR+uS/hbZqxKcJUTznUW5th3CyQPIKMlGlcB +p+zWlTECgYEAxlY4zGJw4QQwGYMKLyWCSHUgKYrKu2Ub2JKJFMTdsoj9H7DI+WHf +lQaO9NCOo2lt0ofYM1MzEpI5Cl/aMrPw+mwquBbxWdMHXK2eSsUQOVo9HtUjgK2t +J2AYFCfsYndo+hCj3ApMHgiY3sghTCXeycvT52bm11VeNVcs3pKxIYMCgYEA3dAJ +PwIfAB8t+6JCP2yYH4ExNjoMNYMdXqhz4vt3UGwgskRqTW6qdd9JvrRQ/JPvGpDy +T375h/+lLw0E4ljsnOPGSzbXNf4bYRHTwPOL+LqVM4Bg90hjclqphElHChxep1di +WcdArB0oae/l4M96z3GjfnXIUVOp8K6BUQCab1kCgYAFFAQUR5j4SfEpVg+WsXEq +hcUzCxixv5785pOX8opynctNWmtq5zSgTjCu2AAu8u4a69t/ROwT16aaO2YM0kqj +Ps3BNOUtFZgkqVVaOL13mnXiKjbkfo3majFzoqoMw13uuSpY4fKc+j9fxOQFXRrd +M9jTHfFfJhJpbzf44uyiHQKBgFIPwzvyVvG+l05/Ky83x9fv/frn4thxV45LmAQj +sHKqbjZFpWZcSOgu4aOSJlwrhsw3T84lVcAAzmXn1STAbVll01jEQz6QciSpacP6 +1pAAx240UqtptpD6BbkROxz8ffA/Hf3E/6Itb2QyAsP3PqI8kpYYkTG1WCvZA7Kq +HHiRAoGAXbUZ25LcrmyuxKWpbty8fck1tjKPvclQB35rOx6vgnfW6pcKMeebYvgq +nJka/QunEReOH/kGxAd/+ymvUBuFQCfFg3Aus+DtAuh9AkBr+cIyPjJqynnIT87J +MbkOw4uEhDJAtGUR9o1j83N1f05bnEwssXiXR0LZPylb9Qzc4tg= +-----END RSA PRIVATE KEY----- +` + +// NewCredential4Test return a Credential for testing +func NewCredential4Test(cn string) (Credential, error) { + res := Credential{} + if cn != "" { + res.CertAllowedCN = append(res.CertAllowedCN, cn) + } + cert, err := os.CreateTemp("", "ticdc-test-cert") + if err != nil { + return res, err + } + _, err = cert.Write([]byte(certPem)) + if err != nil { + return res, err + } + + ca, err := os.CreateTemp("", "ticdc-test-ca") + if err != nil { + return res, err + } + _, err = ca.Write([]byte(caPem)) + if err != nil { + return res, err + } + + key, err := os.CreateTemp("", "ticdc-test-key") + if err != nil { + return res, err + } + _, err = key.Write([]byte(keyPem)) + if err != nil { + return res, err + } + + res.CertPath = cert.Name() + res.CAPath = ca.Name() + res.KeyPath = key.Name() + + return res, nil +} From 998285e2f0480aa14b03218f24b325c21037bc24 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 20 Dec 2021 18:55:46 +0800 Subject: [PATCH 16/48] kvclient(ticdc): fix kvclient takes too long time to recover (#3612) (#3663) --- cdc/kv/client.go | 37 +++++++++--------------------------- cdc/kv/client_test.go | 42 ++++++++++++++++++----------------------- cdc/kv/region_worker.go | 7 +++---- 3 files changed, 30 insertions(+), 56 deletions(-) diff --git a/cdc/kv/client.go b/cdc/kv/client.go index b78601f3719..ff586c5b072 100644 --- a/cdc/kv/client.go +++ b/cdc/kv/client.go @@ -132,21 +132,6 @@ func newSingleRegionInfo(verID tikv.RegionVerID, span regionspan.ComparableSpan, } } -// partialClone clones part fields of singleRegionInfo, this is used when error -// happens, kv client needs to recover region request from singleRegionInfo -func (s *singleRegionInfo) partialClone() singleRegionInfo { - sri := singleRegionInfo{ - verID: s.verID, - span: s.span.Clone(), - ts: s.ts, - rpcCtx: &tikv.RPCContext{}, - } - if s.rpcCtx != nil { - sri.rpcCtx.Addr = s.rpcCtx.Addr - } - return sri -} - type regionErrorInfo struct { singleRegionInfo err error @@ -358,10 +343,6 @@ func (c *CDCClient) newStream(ctx context.Context, addr string, storeID uint64) } err = version.CheckStoreVersion(ctx, c.pd, storeID) if err != nil { - // TODO: we don't close gPRC conn here, let it goes into TransientFailure - // state. If the store recovers, the gPRC conn can be reused. But if - // store goes away forever, the conn will be leaked, we need a better - // connection pool. log.Error("check tikv version failed", zap.Error(err), zap.Uint64("storeID", storeID)) return } @@ -369,10 +350,6 @@ func (c *CDCClient) newStream(ctx context.Context, addr string, storeID uint64) var streamClient cdcpb.ChangeData_EventFeedClient streamClient, err = client.EventFeed(ctx) if err != nil { - // TODO: we don't close gPRC conn here, let it goes into TransientFailure - // state. If the store recovers, the gPRC conn can be reused. But if - // store goes away forever, the conn will be leaked, we need a better - // connection pool. err = cerror.WrapError(cerror.ErrTiKVEventFeed, err) log.Info("establish stream to store failed, retry later", zap.String("addr", addr), zap.Error(err)) return @@ -383,7 +360,7 @@ func (c *CDCClient) newStream(ctx context.Context, addr string, storeID uint64) } log.Debug("created stream to store", zap.String("addr", addr)) return nil - }, retry.WithBackoffBaseDelay(500), retry.WithMaxTries(8), retry.WithIsRetryableErr(cerror.IsRetryableError)) + }, retry.WithBackoffBaseDelay(500), retry.WithMaxTries(2), retry.WithIsRetryableErr(cerror.IsRetryableError)) return } @@ -1025,7 +1002,6 @@ func (s *eventFeedSession) handleError(ctx context.Context, errInfo regionErrorI innerErr := eerr.err if notLeader := innerErr.GetNotLeader(); notLeader != nil { metricFeedNotLeaderCounter.Inc() - // TODO: Handle the case that notleader.GetLeader() is nil. s.regionCache.UpdateLeader(errInfo.verID, notLeader.GetLeader(), errInfo.rpcCtx.AccessIdx) } else if innerErr.GetEpochNotMatch() != nil { // TODO: If only confver is updated, we don't need to reload the region from region cache. @@ -1064,10 +1040,12 @@ func (s *eventFeedSession) handleError(ctx context.Context, errInfo regionErrorI case *sendRequestToStoreErr: metricStoreSendRequestErr.Inc() default: + //[TODO] Move all OnSendFail logic here + // We expect some unknown error to trigger RegionCache recheck its store state and change leader to peer to + // make some detection(peer may tell us where new leader is) + // RegionCache.OnSendFail is thread_safe inner. bo := tikv.NewBackoffer(ctx, tikvRequestMaxBackoff) - if errInfo.rpcCtx.Meta != nil { - s.regionCache.OnSendFail(bo, errInfo.rpcCtx, regionScheduleReload, err) - } + s.regionCache.OnSendFail(bo, errInfo.rpcCtx, regionScheduleReload, err) } failpoint.Inject("kvClientRegionReentrantErrorDelay", nil) @@ -1165,6 +1143,9 @@ func (s *eventFeedSession) receiveFromStream( zap.Uint64("storeID", storeID), zap.Error(err), ) + // Note that pd need at lease 10s+ to tag a kv node as disconnect if kv node down + // tikv raft need wait (raft-base-tick-interval * raft-election-timeout-ticks) 10s to start a new + // election } // Use the same delay mechanism as `stream.Send` error handling, since diff --git a/cdc/kv/client_test.go b/cdc/kv/client_test.go index 067f3682617..895ba2f1d36 100644 --- a/cdc/kv/client_test.go +++ b/cdc/kv/client_test.go @@ -50,6 +50,7 @@ import ( "go.etcd.io/etcd/embed" "go.uber.org/zap" "google.golang.org/grpc" + "google.golang.org/grpc/keepalive" ) func Test(t *testing.T) { @@ -294,7 +295,18 @@ func newMockServiceSpecificAddr( lis, err := lc.Listen(ctx, "tcp", listenAddr) c.Assert(err, check.IsNil) addr = lis.Addr().String() - grpcServer = grpc.NewServer() + kaep := keepalive.EnforcementPolicy{ + MinTime: 60 * time.Second, + PermitWithoutStream: true, + } + kasp := keepalive.ServerParameters{ + MaxConnectionIdle: 60 * time.Second, // If a client is idle for 60 seconds, send a GOAWAY + MaxConnectionAge: 60 * time.Second, // If any connection is alive for more than 60 seconds, send a GOAWAY + MaxConnectionAgeGrace: 5 * time.Second, // Allow 5 seconds for pending RPCs to complete before forcibly closing connections + Time: 5 * time.Second, // Ping the client if it is idle for 5 seconds to ensure the connection is still active + Timeout: 1 * time.Second, // Wait 1 second for the ping ack before assuming the connection is dead + } + grpcServer = grpc.NewServer(grpc.KeepaliveEnforcementPolicy(kaep), grpc.KeepaliveParams(kasp)) cdcpb.RegisterChangeDataServer(grpcServer, srv) wg.Add(1) go func() { @@ -1682,10 +1694,7 @@ func (s *clientSuite) TestIncompatibleTiKV(c *check.C) { var genLock sync.Mutex nextVer := -1 call := int32(0) - // 20 here not too much, since check version itself has 3 time retry, and - // region cache could also call get store API, which will trigger version - // generator too. - versionGenCallBoundary := int32(20) + versionGenCallBoundary := int32(8) gen := func() string { genLock.Lock() defer genLock.Unlock() @@ -1740,7 +1749,8 @@ func (s *clientSuite) TestIncompatibleTiKV(c *check.C) { grpcPool := NewGrpcPoolImpl(ctx, &security.Credential{}) defer grpcPool.Close() cdcClient := NewCDCClient(ctx, pdClient, kvStorage, grpcPool) - eventCh := make(chan model.RegionFeedEvent, 10) + // NOTICE: eventCh may block the main logic of EventFeed + eventCh := make(chan model.RegionFeedEvent, 128) wg.Add(1) go func() { defer wg.Done() @@ -2119,7 +2129,6 @@ func (s *clientSuite) testEventCommitTsFallback(c *check.C, events []*cdcpb.Chan ch1 <- event } clientWg.Wait() - cancel() } @@ -2603,23 +2612,6 @@ func (s *clientSuite) TestOutOfRegionRangeEvent(c *check.C) { cancel() } -func (s *clientSuite) TestSingleRegionInfoClone(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - sri := newSingleRegionInfo( - tikv.RegionVerID{}, - regionspan.ComparableSpan{Start: []byte("a"), End: []byte("c")}, - 1000, &tikv.RPCContext{}) - sri2 := sri.partialClone() - sri2.ts = 2000 - sri2.span.End[0] = 'b' - c.Assert(sri.ts, check.Equals, uint64(1000)) - c.Assert(sri.span.String(), check.Equals, "[61, 63)") - c.Assert(sri2.ts, check.Equals, uint64(2000)) - c.Assert(sri2.span.String(), check.Equals, "[61, 62)") - c.Assert(sri2.rpcCtx, check.DeepEquals, &tikv.RPCContext{}) -} - // TestResolveLockNoCandidate tests the resolved ts manager can work normally // when no region exceeds reslove lock interval, that is what candidate means. func (s *clientSuite) TestResolveLockNoCandidate(c *check.C) { @@ -2941,6 +2933,7 @@ func (s *clientSuite) testKVClientForceReconnect(c *check.C) { server1Stopped <- struct{}{} }() for { + // Currently no msg more than 60s will cause a GoAway msg to end the connection _, err := server.Recv() if err != nil { log.Error("mock server error", zap.Error(err)) @@ -2989,6 +2982,7 @@ func (s *clientSuite) testKVClientForceReconnect(c *check.C) { initialized := mockInitializedEvent(regionID3, currentRequestID()) ch1 <- initialized + // Connection close for timeout <-server1Stopped var requestIds sync.Map diff --git a/cdc/kv/region_worker.go b/cdc/kv/region_worker.go index dead08c2626..ad37a3eee5b 100644 --- a/cdc/kv/region_worker.go +++ b/cdc/kv/region_worker.go @@ -784,9 +784,8 @@ func (w *regionWorker) evictAllRegions() error { } state.markStopped() w.delRegionState(state.sri.verID.GetID()) - singleRegionInfo := state.sri.partialClone() - if state.lastResolvedTs > singleRegionInfo.ts { - singleRegionInfo.ts = state.lastResolvedTs + if state.lastResolvedTs > state.sri.ts { + state.sri.ts = state.lastResolvedTs } revokeToken := !state.initialized state.lock.Unlock() @@ -794,7 +793,7 @@ func (w *regionWorker) evictAllRegions() error { // region worker exits, we must use the parent context to prevent // regionErrorInfo loss. err = w.session.onRegionFail(w.parentCtx, regionErrorInfo{ - singleRegionInfo: singleRegionInfo, + singleRegionInfo: state.sri, err: cerror.ErrEventFeedAborted.FastGenByArgs(), }, revokeToken) return err == nil From 99eaa82012266fd5dc8bc7d17fb38d579d08240e Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 20 Dec 2021 19:17:46 +0800 Subject: [PATCH 17/48] owner: fix owner tick block http request (#3490) (#3530) --- cdc/owner/owner.go | 27 +++++++---- cdc/owner/owner_test.go | 101 +++++++++++++++++++++++++++++++++++++++- 2 files changed, 118 insertions(+), 10 deletions(-) diff --git a/cdc/owner/owner.go b/cdc/owner/owner.go index c830f5081f7..8cd40ad1a6e 100644 --- a/cdc/owner/owner.go +++ b/cdc/owner/owner.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tiflow/pkg/version" pd "github.com/tikv/pd/client" "go.uber.org/zap" + "golang.org/x/time/rate" ) type ownerJobType int @@ -46,6 +47,10 @@ const ( ownerJobTypeQuery ) +// versionInconsistentLogRate represents the rate of log output when there are +// captures with versions different from that of the owner +const versionInconsistentLogRate = 1 + type ownerJob struct { tp ownerJobType changefeedID model.ChangeFeedID @@ -77,10 +82,10 @@ type Owner struct { ownerJobQueueMu sync.Mutex ownerJobQueue []*ownerJob - + // logLimiter controls cluster version check log output rate + logLimiter *rate.Limiter lastTickTime time.Time - - closed int32 + closed int32 newChangefeed func(id model.ChangeFeedID, gcManager gc.Manager) *changefeed } @@ -92,6 +97,7 @@ func NewOwner(pdClient pd.Client) *Owner { gcManager: gc.NewManager(pdClient), lastTickTime: time.Now(), newChangefeed: newChangefeed, + logLimiter: rate.NewLimiter(versionInconsistentLogRate, versionInconsistentLogRate), } } @@ -118,12 +124,16 @@ func (o *Owner) Tick(stdCtx context.Context, rawState orchestrator.ReactorState) state := rawState.(*orchestrator.GlobalReactorState) o.captures = state.Captures o.updateMetrics(state) + + // handleJobs() should be called before clusterVersionConsistent(), because + // when there are different versions of cdc nodes in the cluster, + // the admin job may not be processed all the time. And http api relies on + // admin job, which will cause all http api unavailable. + o.handleJobs() + if !o.clusterVersionConsistent(state.Captures) { - // sleep one second to avoid printing too much log - time.Sleep(1 * time.Second) return state, nil } - // Owner should update GC safepoint before initializing changefeed, so // changefeed can remove its "ticdc-creating" service GC safepoint during // initializing. @@ -133,7 +143,6 @@ func (o *Owner) Tick(stdCtx context.Context, rawState orchestrator.ReactorState) return nil, errors.Trace(err) } - o.handleJobs() for changefeedID, changefeedState := range state.Changefeeds { if changefeedState.Info == nil { o.cleanUpChangefeed(changefeedState) @@ -275,7 +284,9 @@ func (o *Owner) clusterVersionConsistent(captures map[model.CaptureID]*model.Cap myVersion := version.ReleaseVersion for _, capture := range captures { if myVersion != capture.Version { - log.Warn("the capture version is different with the owner", zap.Reflect("capture", capture), zap.String("my-version", myVersion)) + if o.logLimiter.Allow() { + log.Warn("the capture version is different with the owner", zap.Reflect("capture", capture), zap.String("owner-version", myVersion)) + } return false } } diff --git a/cdc/owner/owner_test.go b/cdc/owner/owner_test.go index 21fdaa43937..fe18e2e69a9 100644 --- a/cdc/owner/owner_test.go +++ b/cdc/owner/owner_test.go @@ -54,7 +54,7 @@ func createOwner4Test(ctx cdcContext.Context, c *check.C) (*Owner, *orchestrator return safePoint, nil }, } - cf := NewOwner4Test(func(ctx cdcContext.Context, startTs uint64) (DDLPuller, error) { + owner := NewOwner4Test(func(ctx cdcContext.Context, startTs uint64) (DDLPuller, error) { return &mockDDLPuller{resolvedTs: startTs - 1}, nil }, func(ctx cdcContext.Context) (AsyncSink, error) { return &mockAsyncSink{}, nil @@ -72,13 +72,14 @@ func createOwner4Test(ctx cdcContext.Context, c *check.C) (*Owner, *orchestrator captureBytes, err := ctx.GlobalVars().CaptureInfo.Marshal() c.Assert(err, check.IsNil) tester.MustUpdate(cdcKey.String(), captureBytes) - return cf, state, tester + return owner, state, tester } func (s *ownerSuite) TestCreateRemoveChangefeed(c *check.C) { defer testleak.AfterTest(c)() ctx := cdcContext.NewBackendContext4Test(false) owner, state, tester := createOwner4Test(ctx, c) + changefeedID := "test-changefeed" changefeedInfo := &model.ChangeFeedInfo{ StartTs: oracle.GoTimeToTS(time.Now()), @@ -362,3 +363,99 @@ func (s *ownerSuite) TestUpdateGCSafePoint(c *check.C) { case <-ch: } } + +// make sure handleJobs works well even if there is two different +// version of captures in the cluster +func (s *ownerSuite) TestHandleJobsDontBlock(c *check.C) { + defer testleak.AfterTest(c)() + ctx := cdcContext.NewBackendContext4Test(false) + ctx, cancel := cdcContext.WithCancel(ctx) + defer cancel() + owner, state, tester := createOwner4Test(ctx, c) + + statusProvider := owner.StatusProvider() + // work well + cf1 := "test-changefeed" + cfInfo1 := &model.ChangeFeedInfo{ + StartTs: oracle.GoTimeToTS(time.Now()), + Config: config.GetDefaultReplicaConfig(), + State: model.StateNormal, + } + changefeedStr, err := cfInfo1.Marshal() + c.Assert(err, check.IsNil) + cdcKey := etcd.CDCKey{ + Tp: etcd.CDCKeyTypeChangefeedInfo, + ChangefeedID: cf1, + } + tester.MustUpdate(cdcKey.String(), []byte(changefeedStr)) + _, err = owner.Tick(ctx, state) + tester.MustApplyPatches() + c.Assert(err, check.IsNil) + + c.Assert(owner.changefeeds, check.HasKey, cf1) + + // add an non-consistent version capture + captureInfo := &model.CaptureInfo{ + ID: "capture-id-owner-test", + AdvertiseAddr: "127.0.0.1:0000", + Version: " v0.0.1-test-only", + } + cdcKey = etcd.CDCKey{ + Tp: etcd.CDCKeyTypeCapture, + CaptureID: captureInfo.ID, + } + v, err := captureInfo.Marshal() + c.Assert(err, check.IsNil) + tester.MustUpdate(cdcKey.String(), v) + + // try to add another changefeed + cf2 := "test-changefeed1" + cfInfo2 := &model.ChangeFeedInfo{ + StartTs: oracle.GoTimeToTS(time.Now()), + Config: config.GetDefaultReplicaConfig(), + State: model.StateNormal, + } + changefeedStr1, err := cfInfo2.Marshal() + c.Assert(err, check.IsNil) + cdcKey = etcd.CDCKey{ + Tp: etcd.CDCKeyTypeChangefeedInfo, + ChangefeedID: cf2, + } + tester.MustUpdate(cdcKey.String(), []byte(changefeedStr1)) + _, err = owner.Tick(ctx, state) + tester.MustApplyPatches() + c.Assert(err, check.IsNil) + // make sure this changefeed add failed, which means that owner are return + // in clusterVersionConsistent check + c.Assert(owner.changefeeds[cf2], check.IsNil) + + // make sure statusProvider works well + ctx1, cancel := context.WithTimeout(context.Background(), time.Second*5) + defer cancel() + + var errIn error + var infos map[model.ChangeFeedID]*model.ChangeFeedInfo + done := make(chan struct{}) + go func() { + infos, errIn = statusProvider.GetAllChangeFeedInfo(ctx1) + done <- struct{}{} + }() + + ticker := time.NewTicker(20 * time.Millisecond) + defer ticker.Stop() +WorkLoop: + for { + select { + case <-done: + break WorkLoop + case <-ctx1.Done(): + c.Fatal(ctx1.Err()) + case <-ticker.C: + _, err = owner.Tick(ctx, state) + c.Assert(err, check.IsNil) + } + } + c.Assert(errIn, check.IsNil) + c.Assert(infos[cf1], check.NotNil) + c.Assert(infos[cf2], check.IsNil) +} From 9f32dd5736f906d4e94f5e7a5fe941a41bcc3a71 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Tue, 21 Dec 2021 12:25:46 +0800 Subject: [PATCH 18/48] dm/syncer: use downstream PK/UK to generate DML (#3168) (#3256) --- dm/_utils/terror_gen/errors_release.txt | 4 + dm/errors.toml | 24 + dm/pkg/schema/tracker.go | 270 ++++++++++- dm/pkg/schema/tracker_test.go | 439 +++++++++++++++++- dm/pkg/terror/error_list.go | 14 +- dm/pkg/utils/db.go | 12 + dm/syncer/causality_test.go | 16 +- dm/syncer/compactor_test.go | 64 ++- dm/syncer/dml.go | 106 +++-- dm/syncer/dml_test.go | 157 +++++-- dm/syncer/expr_filter_group_test.go | 11 +- dm/syncer/filter_test.go | 2 +- dm/syncer/job_test.go | 16 +- dm/syncer/syncer.go | 113 ++--- dm/syncer/syncer_test.go | 39 +- .../downstream_diff_index/conf/dm-master.toml | 4 + .../downstream_diff_index/conf/dm-task.yaml | 61 +++ .../conf/dm-worker1.toml | 2 + .../conf/dm-worker2.toml | 2 + .../downstream_diff_index/conf/source1.yaml | 10 + .../downstream_diff_index/conf/source2.yaml | 10 + .../data/db1.increment.sql | 3 + .../data/db1.prepare.sql | 7 + .../data/db2.increment.sql | 3 + .../data/db2.prepare.sql | 7 + .../data/tidb.prepare.sql | 4 + dm/tests/downstream_diff_index/run.sh | 78 ++++ dm/tests/others_integration_1.txt | 2 + 28 files changed, 1281 insertions(+), 199 deletions(-) create mode 100644 dm/tests/downstream_diff_index/conf/dm-master.toml create mode 100644 dm/tests/downstream_diff_index/conf/dm-task.yaml create mode 100644 dm/tests/downstream_diff_index/conf/dm-worker1.toml create mode 100644 dm/tests/downstream_diff_index/conf/dm-worker2.toml create mode 100644 dm/tests/downstream_diff_index/conf/source1.yaml create mode 100644 dm/tests/downstream_diff_index/conf/source2.yaml create mode 100644 dm/tests/downstream_diff_index/data/db1.increment.sql create mode 100644 dm/tests/downstream_diff_index/data/db1.prepare.sql create mode 100644 dm/tests/downstream_diff_index/data/db2.increment.sql create mode 100644 dm/tests/downstream_diff_index/data/db2.prepare.sql create mode 100644 dm/tests/downstream_diff_index/data/tidb.prepare.sql create mode 100755 dm/tests/downstream_diff_index/run.sh diff --git a/dm/_utils/terror_gen/errors_release.txt b/dm/_utils/terror_gen/errors_release.txt index 660eef46495..c81497876da 100644 --- a/dm/_utils/terror_gen/errors_release.txt +++ b/dm/_utils/terror_gen/errors_release.txt @@ -495,6 +495,10 @@ ErrSchemaTrackerInvalidCreateTableStmt,[code=44009:class=schema-tracker:scope=in ErrSchemaTrackerRestoreStmtFail,[code=44010:class=schema-tracker:scope=internal:level=medium], "Message: fail to restore the statement" ErrSchemaTrackerCannotDropTable,[code=44011:class=schema-tracker:scope=internal:level=high], "Message: failed to drop table for %v in schema tracker" ErrSchemaTrackerInit,[code=44012:class=schema-tracker:scope=internal:level=high], "Message: failed to create schema tracker" +ErrSchemaTrackerCannotSetDownstreamSQLMode,[code=44016:class=schema-tracker:scope=internal:level=high], "Message: failed to set default downstream sql_mode %v in schema tracker" +ErrSchemaTrackerCannotInitDownstreamParser,[code=44017:class=schema-tracker:scope=internal:level=high], "Message: failed to init downstream parser by sql_mode %v in schema tracker" +ErrSchemaTrackerCannotMockDownstreamTable,[code=44018:class=schema-tracker:scope=internal:level=high], "Message: failed to mock downstream table by create table statement %v in schema tracker" +ErrSchemaTrackerCannotFetchDownstreamCreateTableStmt,[code=44019:class=schema-tracker:scope=internal:level=high], "Message: failed to fetch downstream table %v by show create table statement in schema tracker" ErrSchedulerNotStarted,[code=46001:class=scheduler:scope=internal:level=high], "Message: the scheduler has not started" ErrSchedulerStarted,[code=46002:class=scheduler:scope=internal:level=medium], "Message: the scheduler has already started" ErrSchedulerWorkerExist,[code=46003:class=scheduler:scope=internal:level=medium], "Message: dm-worker with name %s already exists" diff --git a/dm/errors.toml b/dm/errors.toml index 8e795adf873..9fd7af01ece 100644 --- a/dm/errors.toml +++ b/dm/errors.toml @@ -2980,6 +2980,30 @@ description = "" workaround = "" tags = ["downstream", "high"] +[error.DM-schema-tracker-44016] +message = "failed to set default downstream sql_mode %v in schema tracker" +description = "" +workaround = "" +tags = ["internal", "high"] + +[error.DM-schema-tracker-44017] +message = "failed to init downstream parser by sql_mode %v in schema tracker" +description = "" +workaround = "" +tags = ["internal", "high"] + +[error.DM-schema-tracker-44018] +message = "failed to mock downstream table by create table statement %v in schema tracker" +description = "" +workaround = "" +tags = ["internal", "high"] + +[error.DM-schema-tracker-44019] +message = "failed to fetch downstream table %v by show create table statement in schema tracker" +description = "" +workaround = "" +tags = ["internal", "high"] + [error.DM-scheduler-46001] message = "the scheduler has not started" description = "" diff --git a/dm/pkg/schema/tracker.go b/dm/pkg/schema/tracker.go index 2f7d3624091..a5e7a9f45c3 100644 --- a/dm/pkg/schema/tracker.go +++ b/dm/pkg/schema/tracker.go @@ -25,23 +25,30 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/parser/terror" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/store/mockstore" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/mock" "go.uber.org/zap" - "github.com/pingcap/tiflow/dm/pkg/conn" tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/log" dmterror "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/syncer/dbconn" ) const ( // TiDBClusteredIndex is the variable name for clustered index. TiDBClusteredIndex = "tidb_enable_clustered_index" + // downstream mock table id, consists of serial numbers of letters. + mockTableID = 121402101900011104 ) var ( @@ -55,15 +62,30 @@ var ( // Tracker is used to track schema locally. type Tracker struct { - store kv.Storage - dom *domain.Domain - se session.Session + store kv.Storage + dom *domain.Domain + se session.Session + dsTracker *downstreamTracker +} + +// downstreamTracker tracks downstream schema. +type downstreamTracker struct { + downstreamConn *dbconn.DBConn // downstream connection + stmtParser *parser.Parser // statement parser + tableInfos map[string]*downstreamTableInfo // downstream table infos +} + +// downstreamTableInfo contains tableinfo and index cache. +type downstreamTableInfo struct { + tableInfo *model.TableInfo // tableInfo which comes from parse create statement syntaxtree + indexCache *model.IndexInfo // index cache include pk/uk(not null) + availableUKCache []*model.IndexInfo // index cache include uks(data not null) } // NewTracker creates a new tracker. `sessionCfg` will be set as tracker's session variables if specified, or retrieve -// some variable from downstream TiDB using `tidbConn`. +// some variable from downstream using `downstreamConn`. // NOTE **sessionCfg is a reference to caller**. -func NewTracker(ctx context.Context, task string, sessionCfg map[string]string, tidbConn *conn.BaseConn) (*Tracker, error) { +func NewTracker(ctx context.Context, task string, sessionCfg map[string]string, downstreamConn *dbconn.DBConn) (*Tracker, error) { // NOTE: tidb uses a **global** config so can't isolate tracker's config from each other. If that isolation is needed, // we might SetGlobalConfig before every call to tracker, or use some patch like https://github.com/bouk/monkey tidbConfig.UpdateGlobal(func(conf *tidbConfig.Config) { @@ -82,7 +104,7 @@ func NewTracker(ctx context.Context, task string, sessionCfg map[string]string, for _, k := range downstreamVars { if _, ok := sessionCfg[k]; !ok { var ignoredColumn interface{} - rows, err2 := tidbConn.QuerySQL(tctx, fmt.Sprintf("SHOW VARIABLES LIKE '%s'", k)) + rows, err2 := downstreamConn.QuerySQL(tctx, fmt.Sprintf("SHOW VARIABLES LIKE '%s'", k)) if err2 != nil { return nil, err2 } @@ -156,10 +178,17 @@ func NewTracker(ctx context.Context, task string, sessionCfg map[string]string, return nil, err } + // init downstreamTracker + dsTracker := &downstreamTracker{ + downstreamConn: downstreamConn, + tableInfos: make(map[string]*downstreamTableInfo), + } + return &Tracker{ - store: store, - dom: dom, - se: se, + store: store, + dom: dom, + se: se, + dsTracker: dsTracker, }, nil } @@ -345,3 +374,224 @@ func (tr *Tracker) CreateTableIfNotExists(table *filter.Table, ti *model.TableIn func (tr *Tracker) GetSystemVar(name string) (string, bool) { return tr.se.GetSessionVars().GetSystemVar(name) } + +// GetDownStreamIndexInfo gets downstream PK/UK(not null) Index. +// note. this function will init downstreamTrack's table info. +func (tr *Tracker) GetDownStreamIndexInfo(tctx *tcontext.Context, tableID string, originTi *model.TableInfo) (*model.IndexInfo, error) { + dti, ok := tr.dsTracker.tableInfos[tableID] + if !ok { + tctx.Logger.Info("Downstream schema tracker init. ", zap.String("tableID", tableID)) + ti, err := tr.getTableInfoByCreateStmt(tctx, tableID) + if err != nil { + tctx.Logger.Error("Init dowstream schema info error. ", zap.String("tableID", tableID), zap.Error(err)) + return nil, err + } + + dti = getDownStreamTi(ti, originTi) + tr.dsTracker.tableInfos[tableID] = dti + } + return dti.indexCache, nil +} + +// GetAvailableDownStreamUKIndexInfo gets available downstream UK whose data is not null. +// note. this function will not init downstreamTrack. +func (tr *Tracker) GetAvailableDownStreamUKIndexInfo(tableID string, data []interface{}) *model.IndexInfo { + dti, ok := tr.dsTracker.tableInfos[tableID] + + if !ok || len(dti.availableUKCache) == 0 { + return nil + } + // func for check data is not null + fn := func(i int) bool { + return data[i] != nil + } + + for i, uk := range dti.availableUKCache { + // check uk's column data is not null + if isSpecifiedIndexColumn(uk, fn) { + if i != 0 { + // exchange available uk to the first of the array to reduce judgements for next row + dti.availableUKCache[0], dti.availableUKCache[i] = dti.availableUKCache[i], dti.availableUKCache[0] + } + return uk + } + } + return nil +} + +// RemoveDownstreamSchema just remove schema or table in downstreamTrack. +func (tr *Tracker) RemoveDownstreamSchema(tctx *tcontext.Context, targetTables []*filter.Table) { + if len(targetTables) == 0 { + return + } + + for _, targetTable := range targetTables { + tableID := utils.GenTableID(targetTable) + _, ok := tr.dsTracker.tableInfos[tableID] + if !ok { + // handle just have schema + if targetTable.Schema != "" && targetTable.Name == "" { + for k := range tr.dsTracker.tableInfos { + if strings.HasPrefix(k, tableID+".") { + delete(tr.dsTracker.tableInfos, k) + tctx.Logger.Info("Remove downstream schema tracker", zap.String("tableID", k)) + } + } + } + } else { + delete(tr.dsTracker.tableInfos, tableID) + tctx.Logger.Info("Remove downstream schema tracker", zap.String("tableID", tableID)) + } + } +} + +// getTableInfoByCreateStmt get downstream tableInfo by "SHOW CREATE TABLE" stmt. +func (tr *Tracker) getTableInfoByCreateStmt(tctx *tcontext.Context, tableID string) (*model.TableInfo, error) { + if tr.dsTracker.stmtParser == nil { + err := tr.initDownStreamSQLModeAndParser(tctx) + if err != nil { + return nil, err + } + } + createStr, err := utils.GetTableCreateSQL(tctx.Ctx, tr.dsTracker.downstreamConn.BaseConn.DBConn, tableID) + if err != nil { + return nil, dmterror.ErrSchemaTrackerCannotFetchDownstreamCreateTableStmt.Delegate(err, tableID) + } + + tctx.Logger.Info("Show create table info", zap.String("tableID", tableID), zap.String("create string", createStr)) + // parse create table stmt. + stmtNode, err := tr.dsTracker.stmtParser.ParseOneStmt(createStr, "", "") + if err != nil { + return nil, dmterror.ErrSchemaTrackerInvalidCreateTableStmt.Delegate(err, createStr) + } + + ti, err := ddl.MockTableInfo(mock.NewContext(), stmtNode.(*ast.CreateTableStmt), mockTableID) + if err != nil { + return nil, dmterror.ErrSchemaTrackerCannotMockDownstreamTable.Delegate(err, createStr) + } + return ti, nil +} + +// initDownStreamTrackerParser init downstream tracker parser by default sql_mode. +func (tr *Tracker) initDownStreamSQLModeAndParser(tctx *tcontext.Context) error { + setSQLMode := fmt.Sprintf("SET SESSION SQL_MODE = '%s'", mysql.DefaultSQLMode) + _, err := tr.dsTracker.downstreamConn.ExecuteSQL(tctx, []string{setSQLMode}) + if err != nil { + return dmterror.ErrSchemaTrackerCannotSetDownstreamSQLMode.Delegate(err, mysql.DefaultSQLMode) + } + stmtParser, err := utils.GetParserFromSQLModeStr(mysql.DefaultSQLMode) + if err != nil { + return dmterror.ErrSchemaTrackerCannotInitDownstreamParser.Delegate(err, mysql.DefaultSQLMode) + } + tr.dsTracker.stmtParser = stmtParser + return nil +} + +// getDownStreamTi constructs downstreamTable index cache by tableinfo. +func getDownStreamTi(ti *model.TableInfo, originTi *model.TableInfo) *downstreamTableInfo { + var ( + indexCache *model.IndexInfo + availableUKCache = make([]*model.IndexInfo, 0, len(ti.Indices)) + hasPk = false + ) + + // func for check not null constraint + fn := func(i int) bool { + return mysql.HasNotNullFlag(ti.Columns[i].Flag) + } + + for _, idx := range ti.Indices { + if !idx.Primary && !idx.Unique { + continue + } + indexRedirect := redirectIndexKeys(idx, originTi) + if indexRedirect == nil { + continue + } + if idx.Primary { + indexCache = indexRedirect + hasPk = true + } else if idx.Unique { + // second check not null unique key + if !hasPk && isSpecifiedIndexColumn(idx, fn) { + indexCache = indexRedirect + } else { + availableUKCache = append(availableUKCache, indexRedirect) + } + } + } + + // handle pk exceptional case. + // e.g. "create table t(a int primary key, b int)". + if !hasPk { + exPk := redirectIndexKeys(handlePkExCase(ti), originTi) + if exPk != nil { + indexCache = exPk + } + } + + return &downstreamTableInfo{ + tableInfo: ti, + indexCache: indexCache, + availableUKCache: availableUKCache, + } +} + +// redirectIndexKeys redirect index's columns offset in origin tableinfo. +func redirectIndexKeys(index *model.IndexInfo, originTi *model.TableInfo) *model.IndexInfo { + if index == nil || originTi == nil { + return nil + } + + columns := make([]*model.IndexColumn, 0, len(index.Columns)) + for _, key := range index.Columns { + originColumn := model.FindColumnInfo(originTi.Columns, key.Name.L) + if originColumn == nil { + return nil + } + column := &model.IndexColumn{ + Name: key.Name, + Offset: originColumn.Offset, + Length: key.Length, + } + columns = append(columns, column) + } + return &model.IndexInfo{ + Table: index.Table, + Unique: index.Unique, + Primary: index.Primary, + State: index.State, + Tp: index.Tp, + Columns: columns, + } +} + +// handlePkExCase is handle pk exceptional case. +// e.g. "create table t(a int primary key, b int)". +func handlePkExCase(ti *model.TableInfo) *model.IndexInfo { + if pk := ti.GetPkColInfo(); pk != nil { + return &model.IndexInfo{ + Table: ti.Name, + Unique: true, + Primary: true, + State: model.StatePublic, + Tp: model.IndexTypeBtree, + Columns: []*model.IndexColumn{{ + Name: pk.Name, + Offset: pk.Offset, + Length: types.UnspecifiedLength, + }}, + } + } + return nil +} + +// isSpecifiedIndexColumn checks all of index's columns are matching 'fn'. +func isSpecifiedIndexColumn(index *model.IndexInfo, fn func(i int) bool) bool { + for _, col := range index.Columns { + if !fn(col.Offset) { + return false + } + } + return true +} diff --git a/dm/pkg/schema/tracker_test.go b/dm/pkg/schema/tracker_test.go index 621fdf48780..0004bd04983 100644 --- a/dm/pkg/schema/tracker_test.go +++ b/dm/pkg/schema/tracker_test.go @@ -26,11 +26,20 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/log" "github.com/pingcap/tidb-tools/pkg/filter" + "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/mysql" + timock "github.com/pingcap/tidb/util/mock" "go.uber.org/zap/zapcore" + "github.com/pingcap/tiflow/dm/dm/config" "github.com/pingcap/tiflow/dm/pkg/conn" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" + dlog "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/pkg/terror" + "github.com/pingcap/tiflow/dm/syncer/dbconn" ) func Test(t *testing.T) { @@ -42,12 +51,14 @@ var _ = Suite(&trackerSuite{}) var defaultTestSessionCfg = map[string]string{"sql_mode": "ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION"} type trackerSuite struct { - baseConn *conn.BaseConn + dbConn *dbconn.DBConn db *sql.DB backupKeys []string + cfg *config.SubTaskConfig } func (s *trackerSuite) SetUpSuite(c *C) { + s.cfg = &config.SubTaskConfig{} s.backupKeys = downstreamVars downstreamVars = []string{"sql_mode"} db, _, err := sqlmock.New() @@ -55,7 +66,7 @@ func (s *trackerSuite) SetUpSuite(c *C) { c.Assert(err, IsNil) con, err := db.Conn(context.Background()) c.Assert(err, IsNil) - s.baseConn = conn.NewBaseConn(con, nil) + s.dbConn = &dbconn.DBConn{Cfg: s.cfg, BaseConn: conn.NewBaseConn(con, nil)} } func (s *trackerSuite) TearDownSuite(c *C) { @@ -76,28 +87,29 @@ func (s *trackerSuite) TestTiDBAndSessionCfg(c *C) { con, err := db.Conn(context.Background()) c.Assert(err, IsNil) baseConn := conn.NewBaseConn(con, nil) - + dbConn := &dbconn.DBConn{Cfg: s.cfg, BaseConn: baseConn} // user give correct session config - _, err = NewTracker(context.Background(), "test-tracker", defaultTestSessionCfg, baseConn) + + _, err = NewTracker(context.Background(), "test-tracker", defaultTestSessionCfg, dbConn) c.Assert(err, IsNil) // user give wrong session session, will return error sessionCfg := map[string]string{"sql_mode": "HaHa"} - _, err = NewTracker(context.Background(), "test-tracker", sessionCfg, baseConn) + _, err = NewTracker(context.Background(), "test-tracker", sessionCfg, dbConn) c.Assert(err, NotNil) // discover session config failed, will return error mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows( sqlmock.NewRows([]string{"Variable_name", "Value"}). AddRow("sql_mode", "HaHa")) - _, err = NewTracker(context.Background(), "test-tracker", nil, baseConn) + _, err = NewTracker(context.Background(), "test-tracker", nil, dbConn) c.Assert(err, NotNil) // empty or default config in downstream mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows( sqlmock.NewRows([]string{"Variable_name", "Value"}). - AddRow("sql_mode", "")) - tracker, err := NewTracker(context.Background(), "test-tracker", nil, baseConn) + AddRow("sql_mode", defaultTestSessionCfg["sql_mode"])) + tracker, err := NewTracker(context.Background(), "test-tracker", nil, dbConn) c.Assert(err, IsNil) c.Assert(mock.ExpectationsWereMet(), IsNil) err = tracker.Exec(context.Background(), "", "create database testdb;") @@ -107,7 +119,7 @@ func (s *trackerSuite) TestTiDBAndSessionCfg(c *C) { mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows( sqlmock.NewRows([]string{"Variable_name", "Value"}). AddRow("sql_mode", "ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_DATE,NO_ZERO_IN_DATE")) - tracker, err = NewTracker(context.Background(), "test-tracker", nil, baseConn) + tracker, err = NewTracker(context.Background(), "test-tracker", nil, dbConn) c.Assert(err, IsNil) c.Assert(mock.ExpectationsWereMet(), IsNil) c.Assert(tracker.se.GetSessionVars().SQLMode.HasOnlyFullGroupBy(), IsTrue) @@ -124,7 +136,7 @@ func (s *trackerSuite) TestTiDBAndSessionCfg(c *C) { // user set session config, get tracker config from downstream // no `STRICT_TRANS_TABLES`, no error now sessionCfg = map[string]string{"sql_mode": "NO_ZERO_DATE,NO_ZERO_IN_DATE,ANSI_QUOTES"} - tracker, err = NewTracker(context.Background(), "test-tracker", sessionCfg, baseConn) + tracker, err = NewTracker(context.Background(), "test-tracker", sessionCfg, dbConn) c.Assert(err, IsNil) c.Assert(mock.ExpectationsWereMet(), IsNil) @@ -156,7 +168,7 @@ func (s *trackerSuite) TestTiDBAndSessionCfg(c *C) { "sql_mode": "NO_ZERO_DATE,NO_ZERO_IN_DATE,ANSI_QUOTES", "tidb_enable_clustered_index": "ON", } - tracker, err = NewTracker(context.Background(), "test-tracker", sessionCfg, baseConn) + tracker, err = NewTracker(context.Background(), "test-tracker", sessionCfg, dbConn) c.Assert(err, IsNil) c.Assert(mock.ExpectationsWereMet(), IsNil) @@ -176,7 +188,7 @@ func (s *trackerSuite) TestDDL(c *C) { Name: "foo", } - tracker, err := NewTracker(context.Background(), "test-tracker", defaultTestSessionCfg, s.baseConn) + tracker, err := NewTracker(context.Background(), "test-tracker", defaultTestSessionCfg, s.dbConn) c.Assert(err, IsNil) // Table shouldn't exist before initialization. @@ -242,7 +254,7 @@ func (s *trackerSuite) TestDDL(c *C) { func (s *trackerSuite) TestGetSingleColumnIndices(c *C) { log.SetLevel(zapcore.ErrorLevel) - tracker, err := NewTracker(context.Background(), "test-tracker", defaultTestSessionCfg, s.baseConn) + tracker, err := NewTracker(context.Background(), "test-tracker", defaultTestSessionCfg, s.dbConn) c.Assert(err, IsNil) ctx := context.Background() @@ -281,7 +293,7 @@ func (s *trackerSuite) TestGetSingleColumnIndices(c *C) { func (s *trackerSuite) TestCreateSchemaIfNotExists(c *C) { log.SetLevel(zapcore.ErrorLevel) - tracker, err := NewTracker(context.Background(), "test-tracker", defaultTestSessionCfg, s.baseConn) + tracker, err := NewTracker(context.Background(), "test-tracker", defaultTestSessionCfg, s.dbConn) c.Assert(err, IsNil) // We cannot create a table without a database. @@ -309,7 +321,7 @@ func (s *trackerSuite) TestCreateSchemaIfNotExists(c *C) { func (s *trackerSuite) TestMultiDrop(c *C) { log.SetLevel(zapcore.ErrorLevel) - tracker, err := NewTracker(context.Background(), "test-tracker", defaultTestSessionCfg, s.baseConn) + tracker, err := NewTracker(context.Background(), "test-tracker", defaultTestSessionCfg, s.dbConn) c.Assert(err, IsNil) ctx := context.Background() @@ -357,7 +369,7 @@ func (s *trackerSuite) TestCreateTableIfNotExists(c *C) { Name: "foo", } - tracker, err := NewTracker(context.Background(), "test-tracker", defaultTestSessionCfg, s.baseConn) + tracker, err := NewTracker(context.Background(), "test-tracker", defaultTestSessionCfg, s.dbConn) c.Assert(err, IsNil) // Create some sort of complicated table. @@ -437,7 +449,7 @@ func (s *trackerSuite) TestAllSchemas(c *C) { log.SetLevel(zapcore.ErrorLevel) ctx := context.Background() - tracker, err := NewTracker(context.Background(), "test-tracker", defaultTestSessionCfg, s.baseConn) + tracker, err := NewTracker(context.Background(), "test-tracker", defaultTestSessionCfg, s.dbConn) c.Assert(err, IsNil) // nothing should exist... @@ -520,15 +532,404 @@ func (s *trackerSuite) TestNotSupportedVariable(c *C) { con, err := db.Conn(context.Background()) c.Assert(err, IsNil) baseConn := conn.NewBaseConn(con, nil) + dbConn := &dbconn.DBConn{Cfg: s.cfg, BaseConn: baseConn} mock.ExpectQuery("SHOW VARIABLES LIKE 'sql_mode'").WillReturnRows( sqlmock.NewRows([]string{"Variable_name", "Value"}). - AddRow("sql_mode", "")) + AddRow("sql_mode", defaultTestSessionCfg["sql_mode"])) oldSessionVar := map[string]string{ "tidb_enable_change_column_type": "ON", } + _, err = NewTracker(context.Background(), "test-tracker", oldSessionVar, dbConn) + c.Assert(err, IsNil) +} + +func (s *trackerSuite) TestInitDownStreamSQLModeAndParser(c *C) { + log.SetLevel(zapcore.ErrorLevel) + + // tracker and sqlmock + db, mock, err := sqlmock.New() + c.Assert(err, IsNil) + defer db.Close() + con, err := db.Conn(context.Background()) + c.Assert(err, IsNil) + baseConn := conn.NewBaseConn(con, nil) + dbConn := &dbconn.DBConn{Cfg: s.cfg, BaseConn: baseConn} + + tracker, err := NewTracker(context.Background(), "test-tracker", defaultTestSessionCfg, dbConn) + c.Assert(err, IsNil) + + mock.ExpectBegin() + mock.ExpectExec(fmt.Sprintf("SET SESSION SQL_MODE = '%s'", mysql.DefaultSQLMode)).WillReturnResult(sqlmock.NewResult(0, 0)) + mock.ExpectCommit() + + tctx := tcontext.NewContext(context.Background(), dlog.L()) + + err = tracker.initDownStreamSQLModeAndParser(tctx) + c.Assert(err, IsNil) + c.Assert(tracker.dsTracker.stmtParser, NotNil) +} + +func (s *trackerSuite) TestGetDownStreamIndexInfo(c *C) { + log.SetLevel(zapcore.ErrorLevel) + + // origin table info + p := parser.New() + se := timock.NewContext() + node, err := p.ParseOneStmt("create table t(a int, b int, c varchar(10))", "utf8mb4", "utf8mb4_bin") + c.Assert(err, IsNil) + oriTi, err := ddl.MockTableInfo(se, node.(*ast.CreateTableStmt), 1) + c.Assert(err, IsNil) + + // tracker and sqlmock + db, mock, err := sqlmock.New() + c.Assert(err, IsNil) + defer db.Close() + con, err := db.Conn(context.Background()) + c.Assert(err, IsNil) + baseConn := conn.NewBaseConn(con, nil) + dbConn := &dbconn.DBConn{Cfg: s.cfg, BaseConn: baseConn} + tracker, err := NewTracker(context.Background(), "test-tracker", defaultTestSessionCfg, dbConn) + c.Assert(err, IsNil) + + mock.ExpectBegin() + mock.ExpectExec(fmt.Sprintf("SET SESSION SQL_MODE = '%s'", mysql.DefaultSQLMode)).WillReturnResult(sqlmock.NewResult(0, 0)) + mock.ExpectCommit() + + tableID := "`test`.`test`" + + // downstream has no pk/uk + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int, b int, c varchar(10))")) + indexinfo, err := tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + c.Assert(err, IsNil) + _, ok := tracker.dsTracker.tableInfos[tableID] + c.Assert(ok, IsTrue) + c.Assert(indexinfo, IsNil) + delete(tracker.dsTracker.tableInfos, tableID) + + // downstream has pk(not constraints like "create table t(a int primary key,b int not null)" + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int, b int, c varchar(10), PRIMARY KEY (c))")) + indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + c.Assert(err, IsNil) + _, ok = tracker.dsTracker.tableInfos[tableID] + c.Assert(ok, IsTrue) + c.Assert(indexinfo, NotNil) + delete(tracker.dsTracker.tableInfos, tableID) + + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int primary key, b int, c varchar(10))")) + indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + c.Assert(err, IsNil) + _, ok = tracker.dsTracker.tableInfos[tableID] + c.Assert(ok, IsTrue) + c.Assert(indexinfo, NotNil) + delete(tracker.dsTracker.tableInfos, tableID) + + // downstream has composite pks + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int, b int, c varchar(10), PRIMARY KEY (a,b))")) + indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + c.Assert(err, IsNil) + _, ok = tracker.dsTracker.tableInfos[tableID] + c.Assert(ok, IsTrue) + c.Assert(len(indexinfo.Columns) == 2, IsTrue) + delete(tracker.dsTracker.tableInfos, tableID) + + // downstream has uk(not null) + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int unique not null, b int, c varchar(10))")) + indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + c.Assert(err, IsNil) + _, ok = tracker.dsTracker.tableInfos[tableID] + c.Assert(ok, IsTrue) + c.Assert(indexinfo.Columns, NotNil) + delete(tracker.dsTracker.tableInfos, tableID) + + // downstream has uk(without not null) + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int unique, b int, c varchar(10))")) + indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + c.Assert(err, IsNil) + dti, ok := tracker.dsTracker.tableInfos[tableID] + c.Assert(ok, IsTrue) + c.Assert(indexinfo, IsNil) + c.Assert(dti.availableUKCache, NotNil) + delete(tracker.dsTracker.tableInfos, tableID) + + // downstream has uks + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int unique, b int unique, c varchar(10) unique not null)")) + indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + c.Assert(err, IsNil) + dti, ok = tracker.dsTracker.tableInfos[tableID] + c.Assert(ok, IsTrue) + c.Assert(indexinfo, NotNil) + c.Assert(len(dti.availableUKCache) == 2, IsTrue) + delete(tracker.dsTracker.tableInfos, tableID) + + // downstream has pk and uk, pk has priority + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int unique not null , b int, c varchar(10), PRIMARY KEY (c))")) + indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + c.Assert(err, IsNil) + c.Assert(indexinfo.Primary, IsTrue) + delete(tracker.dsTracker.tableInfos, tableID) + + // downstream has more columns than upstream, and that column in used in PK + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int , d int PRIMARY KEY, c varchar(10), b int unique not null)")) + indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + c.Assert(err, IsNil) + c.Assert(indexinfo, NotNil) + c.Assert(indexinfo.Primary, IsFalse) + delete(tracker.dsTracker.tableInfos, tableID) + + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int , d int PRIMARY KEY, c varchar(10), b int unique)")) + indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + c.Assert(err, IsNil) + c.Assert(indexinfo, IsNil) + dti, ok = tracker.dsTracker.tableInfos[tableID] + c.Assert(ok, IsTrue) + c.Assert(len(dti.availableUKCache) == 1, IsTrue) + delete(tracker.dsTracker.tableInfos, tableID) + + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int , d int PRIMARY KEY, c varchar(10), b int)")) + indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + c.Assert(err, IsNil) + c.Assert(indexinfo, IsNil) + delete(tracker.dsTracker.tableInfos, tableID) + + // downstream has more columns than upstream, and that column in used in UK(not null) + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int , d int unique not null, c varchar(10), b int unique not null)")) + indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + c.Assert(err, IsNil) + c.Assert(indexinfo, NotNil) + c.Assert(indexinfo.Columns[0].Name.L == "b", IsTrue) + delete(tracker.dsTracker.tableInfos, tableID) + + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int , d int unique not null, c varchar(10), b int unique)")) + indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + c.Assert(err, IsNil) + c.Assert(indexinfo, IsNil) + dti, ok = tracker.dsTracker.tableInfos[tableID] + c.Assert(ok, IsTrue) + c.Assert(len(dti.availableUKCache) == 1, IsTrue) + delete(tracker.dsTracker.tableInfos, tableID) + + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int , d int unique not null, c varchar(10), b int)")) + indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + c.Assert(err, IsNil) + c.Assert(indexinfo, IsNil) + delete(tracker.dsTracker.tableInfos, tableID) +} + +func (s *trackerSuite) TestGetAvailableDownStreanUKIndexInfo(c *C) { + log.SetLevel(zapcore.ErrorLevel) + + // origin table info + p := parser.New() + se := timock.NewContext() + node, err := p.ParseOneStmt("create table t(a int, b int, c varchar(10))", "utf8mb4", "utf8mb4_bin") + c.Assert(err, IsNil) + oriTi, err := ddl.MockTableInfo(se, node.(*ast.CreateTableStmt), 1) + c.Assert(err, IsNil) + + // tracker and sqlmock + db, mock, err := sqlmock.New() + c.Assert(err, IsNil) + defer db.Close() + con, err := db.Conn(context.Background()) + c.Assert(err, IsNil) + baseConn := conn.NewBaseConn(con, nil) + dbConn := &dbconn.DBConn{Cfg: s.cfg, BaseConn: baseConn} + tracker, err := NewTracker(context.Background(), "test-tracker", defaultTestSessionCfg, dbConn) + c.Assert(err, IsNil) + + mock.ExpectBegin() + mock.ExpectExec(fmt.Sprintf("SET SESSION SQL_MODE = '%s'", mysql.DefaultSQLMode)).WillReturnResult(sqlmock.NewResult(0, 0)) + mock.ExpectCommit() + + tableID := "`test`.`test`" + + // downstream has no uk + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int, b int, c varchar(10))")) + indexinfo, err := tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + c.Assert(err, IsNil) + c.Assert(indexinfo, IsNil) + data := []interface{}{1, 2, 3} + indexinfo = tracker.GetAvailableDownStreamUKIndexInfo(tableID, data) + c.Assert(indexinfo, IsNil) + delete(tracker.dsTracker.tableInfos, tableID) + + // downstream has uk but data is null + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int unique, b int, c varchar(10))")) + indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + c.Assert(err, IsNil) + c.Assert(indexinfo, IsNil) + data = []interface{}{nil, 2, 3} + indexinfo = tracker.GetAvailableDownStreamUKIndexInfo(tableID, data) + c.Assert(indexinfo, IsNil) + delete(tracker.dsTracker.tableInfos, tableID) + + // downstream has uk and data is not null + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int unique, b int, c varchar(10))")) + indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + c.Assert(err, IsNil) + c.Assert(indexinfo, IsNil) + data = []interface{}{1, 2, 3} + indexinfo = tracker.GetAvailableDownStreamUKIndexInfo(tableID, data) + c.Assert(indexinfo, NotNil) + delete(tracker.dsTracker.tableInfos, tableID) + + // downstream has union uk but data has null + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int, b int, c varchar(10), unique key(a, b))")) + indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + c.Assert(err, IsNil) + c.Assert(indexinfo, IsNil) + data = []interface{}{1, nil, 3} + indexinfo = tracker.GetAvailableDownStreamUKIndexInfo(tableID, data) + c.Assert(indexinfo, IsNil) + delete(tracker.dsTracker.tableInfos, tableID) + + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int, b int, c varchar(10), unique key(a, b))")) + indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + c.Assert(err, IsNil) + c.Assert(indexinfo, IsNil) + data = []interface{}{1, nil, nil} + indexinfo = tracker.GetAvailableDownStreamUKIndexInfo(tableID, data) + c.Assert(indexinfo, IsNil) + delete(tracker.dsTracker.tableInfos, tableID) + + // downstream has union uk but data has null + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int, b int, c varchar(10), unique key(a, b))")) + indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + c.Assert(err, IsNil) + c.Assert(indexinfo, IsNil) + data = []interface{}{1, 2, 3} + indexinfo = tracker.GetAvailableDownStreamUKIndexInfo(tableID, data) + c.Assert(indexinfo, NotNil) + delete(tracker.dsTracker.tableInfos, tableID) +} + +func (s *trackerSuite) TestReTrackDownStreamIndex(c *C) { + log.SetLevel(zapcore.ErrorLevel) + + // origin table info + p := parser.New() + se := timock.NewContext() + node, err := p.ParseOneStmt("create table t(a int, b int, c varchar(10))", "utf8mb4", "utf8mb4_bin") + c.Assert(err, IsNil) + oriTi, err := ddl.MockTableInfo(se, node.(*ast.CreateTableStmt), 1) + c.Assert(err, IsNil) + + // tracker and sqlmock + db, mock, err := sqlmock.New() + c.Assert(err, IsNil) + defer db.Close() + con, err := db.Conn(context.Background()) + c.Assert(err, IsNil) + baseConn := conn.NewBaseConn(con, nil) + dbConn := &dbconn.DBConn{Cfg: s.cfg, BaseConn: baseConn} + tracker, err := NewTracker(context.Background(), "test-tracker", defaultTestSessionCfg, dbConn) + c.Assert(err, IsNil) + + mock.ExpectBegin() + mock.ExpectExec(fmt.Sprintf("SET SESSION SQL_MODE = '%s'", mysql.DefaultSQLMode)).WillReturnResult(sqlmock.NewResult(0, 0)) + mock.ExpectCommit() + + tableID := "`test`.`test`" + + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int, b int, c varchar(10), PRIMARY KEY (a,b))")) + _, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + c.Assert(err, IsNil) + _, ok := tracker.dsTracker.tableInfos[tableID] + c.Assert(ok, IsTrue) + + // just table + targetTables := []*filter.Table{{Schema: "test", Name: "a"}, {Schema: "test", Name: "test"}} + tracker.RemoveDownstreamSchema(tcontext.Background(), targetTables) + _, ok = tracker.dsTracker.tableInfos[tableID] + c.Assert(ok, IsFalse) + + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int, b int, c varchar(10), PRIMARY KEY (a,b))")) + _, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + c.Assert(err, IsNil) + _, ok = tracker.dsTracker.tableInfos[tableID] + c.Assert(ok, IsTrue) + + tracker.RemoveDownstreamSchema(tcontext.Background(), targetTables) + _, ok = tracker.dsTracker.tableInfos[tableID] + c.Assert(ok, IsFalse) + + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int primary key, b int, c varchar(10))")) + _, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + c.Assert(err, IsNil) + _, ok = tracker.dsTracker.tableInfos[tableID] + c.Assert(ok, IsTrue) + + // just schema + targetTables = []*filter.Table{{Schema: "test", Name: "a"}, {Schema: "test", Name: ""}} + tracker.RemoveDownstreamSchema(tcontext.Background(), targetTables) + _, ok = tracker.dsTracker.tableInfos[tableID] + c.Assert(ok, IsFalse) + + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int, b int, c varchar(10), PRIMARY KEY (a,b))")) + _, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + c.Assert(err, IsNil) + _, ok = tracker.dsTracker.tableInfos[tableID] + c.Assert(ok, IsTrue) + + tracker.RemoveDownstreamSchema(tcontext.Background(), targetTables) + _, ok = tracker.dsTracker.tableInfos[tableID] + c.Assert(ok, IsFalse) - _, err = NewTracker(context.Background(), "test-tracker", oldSessionVar, baseConn) + mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("test", "create table t(a int primary key, b int, c varchar(10))")) + _, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) + _, ok = tracker.dsTracker.tableInfos[tableID] + c.Assert(ok, IsTrue) } diff --git a/dm/pkg/terror/error_list.go b/dm/pkg/terror/error_list.go index 74d721499b9..347192f5719 100644 --- a/dm/pkg/terror/error_list.go +++ b/dm/pkg/terror/error_list.go @@ -608,6 +608,10 @@ const ( codeSchemaTrackerMarshalJSON codeSchemaTrackerUnMarshalJSON codeSchemaTrackerUnSchemaNotExist + codeSchemaTrackerCannotSetDownstreamSQLMode + codeSchemaTrackerCannotInitDownstreamParser + codeSchemaTrackerCannotMockDownstreamTable + codeSchemaTrackerCannotFetchDownstreamCreateTableStmt ) // HA scheduler. @@ -1243,7 +1247,15 @@ var ( "fail to restore the statement", "") ErrSchemaTrackerCannotDropTable = New(codeSchemaTrackerCannotDropTable, ClassSchemaTracker, ScopeInternal, LevelHigh, "failed to drop table for %v in schema tracker", "") - ErrSchemaTrackerInit = New(codeSchemaTrackerInit, ClassSchemaTracker, ScopeInternal, LevelHigh, "failed to create schema tracker", "") + ErrSchemaTrackerInit = New(codeSchemaTrackerInit, ClassSchemaTracker, ScopeInternal, LevelHigh, "failed to create schema tracker", "") + ErrSchemaTrackerCannotSetDownstreamSQLMode = New(codeSchemaTrackerCannotSetDownstreamSQLMode, ClassSchemaTracker, ScopeInternal, LevelHigh, + "failed to set default downstream sql_mode %v in schema tracker", "") + ErrSchemaTrackerCannotInitDownstreamParser = New(codeSchemaTrackerCannotInitDownstreamParser, ClassSchemaTracker, ScopeInternal, LevelHigh, + "failed to init downstream parser by sql_mode %v in schema tracker", "") + ErrSchemaTrackerCannotMockDownstreamTable = New(codeSchemaTrackerCannotMockDownstreamTable, ClassSchemaTracker, ScopeInternal, LevelHigh, + "failed to mock downstream table by create table statement %v in schema tracker", "") + ErrSchemaTrackerCannotFetchDownstreamCreateTableStmt = New(codeSchemaTrackerCannotFetchDownstreamCreateTableStmt, ClassSchemaTracker, ScopeInternal, LevelHigh, + "failed to fetch downstream table %v by show create table statement in schema tracker", "") // HA scheduler. ErrSchedulerNotStarted = New(codeSchedulerNotStarted, ClassScheduler, ScopeInternal, LevelHigh, "the scheduler has not started", "") diff --git a/dm/pkg/utils/db.go b/dm/pkg/utils/db.go index b9519b28a48..c582a016823 100644 --- a/dm/pkg/utils/db.go +++ b/dm/pkg/utils/db.go @@ -620,3 +620,15 @@ func GetSQLModeStrBySQLMode(sqlMode tmysql.SQLMode) string { } return strings.Join(sqlModeStr, ",") } + +// GetTableCreateSQL gets table create sql by 'show create table schema.table'. +func GetTableCreateSQL(ctx context.Context, conn *sql.Conn, tableID string) (sql string, err error) { + querySQL := fmt.Sprintf("SHOW CREATE TABLE %s", tableID) + var table, createStr string + row := conn.QueryRowContext(ctx, querySQL) + err = row.Scan(&table, &createStr) + if err != nil { + return "", terror.DBErrorAdapt(err, terror.ErrDBDriverError) + } + return createStr, nil +} diff --git a/dm/syncer/causality_test.go b/dm/syncer/causality_test.go index 13e87eddb2d..be775bf9d26 100644 --- a/dm/syncer/causality_test.go +++ b/dm/syncer/causality_test.go @@ -19,6 +19,8 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb-tools/pkg/filter" "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/types" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tiflow/dm/dm/config" @@ -57,6 +59,18 @@ func (s *testSyncerSuite) TestCasuality(c *C) { schema := "create table tb(a int primary key, b int unique);" ti, err := createTableInfo(p, se, int64(0), schema) c.Assert(err, IsNil) + tiIndex := &model.IndexInfo{ + Table: ti.Name, + Unique: true, + Primary: true, + State: model.StatePublic, + Tp: model.IndexTypeBtree, + Columns: []*model.IndexColumn{{ + Name: ti.Columns[0].Name, + Offset: ti.Columns[0].Offset, + Length: types.UnspecifiedLength, + }}, + } jobCh := make(chan *job, 10) syncer := &Syncer{ @@ -103,7 +117,7 @@ func (s *testSyncerSuite) TestCasuality(c *C) { ec := &eventContext{startLocation: &location, currentLocation: &location, lastLocation: &location} for _, tc := range testCases { - job := newDMLJob(tc.op, table, table, newDML(tc.op, false, "", table, tc.oldVals, tc.vals, tc.oldVals, tc.vals, ti.Columns, ti), ec) + job := newDMLJob(tc.op, table, table, newDML(tc.op, false, "", table, tc.oldVals, tc.vals, tc.oldVals, tc.vals, ti.Columns, ti, tiIndex), ec) jobCh <- job } diff --git a/dm/syncer/compactor_test.go b/dm/syncer/compactor_test.go index 99100e13727..901bd40ffd8 100644 --- a/dm/syncer/compactor_test.go +++ b/dm/syncer/compactor_test.go @@ -22,6 +22,8 @@ import ( "github.com/pingcap/failpoint" filter "github.com/pingcap/tidb-tools/pkg/table-filter" "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/types" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tiflow/dm/dm/config" @@ -76,6 +78,18 @@ func (s *testSyncerSuite) TestCompactJob(c *C) { schema := "create table test.tb(id int primary key, col1 int, name varchar(24))" ti, err := createTableInfo(p, se, 0, schema) c.Assert(err, IsNil) + tiIndex := &model.IndexInfo{ + Table: ti.Name, + Unique: true, + Primary: true, + State: model.StatePublic, + Tp: model.IndexTypeBtree, + Columns: []*model.IndexColumn{{ + Name: ti.Columns[0].Name, + Offset: ti.Columns[0].Offset, + Length: types.UnspecifiedLength, + }}, + } var dml *DML var dmls []*DML @@ -94,7 +108,7 @@ func (s *testSyncerSuite) TestCompactJob(c *C) { oldValues, ok := kv[newID] if !ok { // insert - dml = newDML(insert, false, targetTableID, sourceTable, nil, values, nil, values, ti.Columns, ti) + dml = newDML(insert, false, targetTableID, sourceTable, nil, values, nil, values, ti.Columns, ti, tiIndex) } else { if rand.Int()%2 > 0 { // update @@ -108,10 +122,10 @@ func (s *testSyncerSuite) TestCompactJob(c *C) { } } } - dml = newDML(update, false, targetTableID, sourceTable, oldValues, values, oldValues, values, ti.Columns, ti) + dml = newDML(update, false, targetTableID, sourceTable, oldValues, values, oldValues, values, ti.Columns, ti, tiIndex) } else { // delete - dml = newDML(del, false, targetTableID, sourceTable, nil, oldValues, nil, oldValues, ti.Columns, ti) + dml = newDML(del, false, targetTableID, sourceTable, nil, oldValues, nil, oldValues, ti.Columns, ti, tiIndex) } } @@ -179,6 +193,18 @@ func (s *testSyncerSuite) TestCompactorSafeMode(c *C) { schema := "create table test.tb(id int primary key, col1 int, name varchar(24))" ti, err := createTableInfo(p, se, 0, schema) c.Assert(err, IsNil) + tiIndex := &model.IndexInfo{ + Table: ti.Name, + Unique: true, + Primary: true, + State: model.StatePublic, + Tp: model.IndexTypeBtree, + Columns: []*model.IndexColumn{{ + Name: ti.Columns[0].Name, + Offset: ti.Columns[0].Offset, + Length: types.UnspecifiedLength, + }}, + } testCases := []struct { input []*DML @@ -187,31 +213,31 @@ func (s *testSyncerSuite) TestCompactorSafeMode(c *C) { // nolint:dupl { input: []*DML{ - newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti), - newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti), - newDML(update, true, targetTableID, sourceTable, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, ti.Columns, ti), - newDML(del, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti), - newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti), + newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex), + newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex), + newDML(update, true, targetTableID, sourceTable, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, ti.Columns, ti, tiIndex), + newDML(del, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex), + newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex), }, output: []*DML{ - newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti.Columns, ti), - newDML(del, true, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti), - newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti), + newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti.Columns, ti, tiIndex), + newDML(del, true, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex), + newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex), }, }, // nolint:dupl { input: []*DML{ - newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti), - newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti), - newDML(update, false, targetTableID, sourceTable, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, ti.Columns, ti), - newDML(del, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti), - newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti), + newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex), + newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex), + newDML(update, false, targetTableID, sourceTable, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, ti.Columns, ti, tiIndex), + newDML(del, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex), + newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex), }, output: []*DML{ - newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti.Columns, ti), - newDML(del, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti), - newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti), + newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti.Columns, ti, tiIndex), + newDML(del, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex), + newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex), }, }, } diff --git a/dm/syncer/dml.go b/dm/syncer/dml.go index 0323407754f..539755c6d9d 100644 --- a/dm/syncer/dml.go +++ b/dm/syncer/dml.go @@ -19,6 +19,7 @@ import ( "strconv" "strings" + "github.com/pingcap/failpoint" "github.com/pingcap/tidb-tools/pkg/dbutil" "github.com/pingcap/tidb-tools/pkg/filter" "github.com/pingcap/tidb/expression" @@ -27,6 +28,7 @@ import ( "github.com/pingcap/tidb/parser/types" "go.uber.org/zap" + tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/pkg/terror" ) @@ -106,18 +108,20 @@ RowLoop: } } - dmls = append(dmls, newDML(insert, param.safeMode, param.targetTableID, param.sourceTable, nil, value, nil, originalValue, columns, ti)) + dmls = append(dmls, newDML(insert, param.safeMode, param.targetTableID, param.sourceTable, nil, value, nil, originalValue, columns, ti, nil)) } return dmls, nil } func (s *Syncer) genAndFilterUpdateDMLs( + tctx *tcontext.Context, param *genDMLParam, oldValueFilters []expression.Expression, newValueFilters []expression.Expression, ) ([]*DML, error) { var ( + tableID = param.targetTableID data = param.data originalData = param.originalData columns = param.columns @@ -125,6 +129,12 @@ func (s *Syncer) genAndFilterUpdateDMLs( dmls = make([]*DML, 0, len(data)/2) ) + // if downstream pk/uk(not null) exits, then use downstream pk/uk(not null) + downstreamIndexColumns, err := s.schemaTracker.GetDownStreamIndexInfo(tctx, tableID, ti) + if err != nil { + return nil, err + } + RowLoop: for i := 0; i < len(data); i += 2 { oldData := data[i] @@ -170,19 +180,30 @@ RowLoop: } } - dmls = append(dmls, newDML(update, param.safeMode, param.targetTableID, param.sourceTable, oldValues, changedValues, oriOldValues, oriChangedValues, columns, ti)) + if downstreamIndexColumns == nil { + downstreamIndexColumns = s.schemaTracker.GetAvailableDownStreamUKIndexInfo(tableID, oriOldValues) + } + + dmls = append(dmls, newDML(update, param.safeMode, param.targetTableID, param.sourceTable, oldValues, changedValues, oriOldValues, oriChangedValues, columns, ti, downstreamIndexColumns)) } return dmls, nil } -func (s *Syncer) genAndFilterDeleteDMLs(param *genDMLParam, filterExprs []expression.Expression) ([]*DML, error) { +func (s *Syncer) genAndFilterDeleteDMLs(tctx *tcontext.Context, param *genDMLParam, filterExprs []expression.Expression) ([]*DML, error) { var ( + tableID = param.targetTableID dataSeq = param.originalData ti = param.sourceTableInfo dmls = make([]*DML, 0, len(dataSeq)) ) + // if downstream pk/uk(not null) exits, then use downstream pk/uk(not null) + downstreamIndexColumns, err := s.schemaTracker.GetDownStreamIndexInfo(tctx, tableID, ti) + if err != nil { + return nil, err + } + RowLoop: for _, data := range dataSeq { if len(data) != len(ti.Columns) { @@ -201,7 +222,12 @@ RowLoop: continue RowLoop } } - dmls = append(dmls, newDML(del, false, param.targetTableID, param.sourceTable, nil, value, nil, value, ti.Columns, ti)) + + if downstreamIndexColumns == nil { + downstreamIndexColumns = s.schemaTracker.GetAvailableDownStreamUKIndexInfo(tableID, value) + } + + dmls = append(dmls, newDML(del, false, param.targetTableID, param.sourceTable, nil, value, nil, value, ti.Columns, ti, downstreamIndexColumns)) } return dmls, nil @@ -312,14 +338,6 @@ func findFitIndex(ti *model.TableInfo) *model.IndexInfo { return getSpecifiedIndexColumn(ti, fn) } -func getAvailableIndexColumn(ti *model.TableInfo, data []interface{}) *model.IndexInfo { - fn := func(i int) bool { - return data[i] == nil - } - - return getSpecifiedIndexColumn(ti, fn) -} - func getSpecifiedIndexColumn(ti *model.TableInfo, fn func(i int) bool) *model.IndexInfo { for _, indexCols := range ti.Indices { if !indexCols.Unique { @@ -433,32 +451,34 @@ func checkLogColumns(skipped [][]int) error { // DML stores param for DML. type DML struct { - targetTableID string - sourceTable *filter.Table - op opType - oldValues []interface{} // only for update SQL - values []interface{} - columns []*model.ColumnInfo - sourceTableInfo *model.TableInfo - originOldValues []interface{} // only for update SQL - originValues []interface{} // use to gen key and `WHERE` - safeMode bool - key string // use to detect causality + targetTableID string + sourceTable *filter.Table + op opType + oldValues []interface{} // only for update SQL + values []interface{} + columns []*model.ColumnInfo + sourceTableInfo *model.TableInfo + originOldValues []interface{} // only for update SQL + originValues []interface{} // use to gen key and `WHERE` + safeMode bool + key string // use to detect causality + downstreamIndexInfo *model.IndexInfo } // newDML creates DML. -func newDML(op opType, safeMode bool, targetTableID string, sourceTable *filter.Table, oldValues, values, originOldValues, originValues []interface{}, columns []*model.ColumnInfo, sourceTableInfo *model.TableInfo) *DML { +func newDML(op opType, safeMode bool, targetTableID string, sourceTable *filter.Table, oldValues, values, originOldValues, originValues []interface{}, columns []*model.ColumnInfo, sourceTableInfo *model.TableInfo, downstreamIndexInfo *model.IndexInfo) *DML { return &DML{ - op: op, - safeMode: safeMode, - targetTableID: targetTableID, - sourceTable: sourceTable, - oldValues: oldValues, - values: values, - columns: columns, - sourceTableInfo: sourceTableInfo, - originOldValues: originOldValues, - originValues: originValues, + op: op, + safeMode: safeMode, + targetTableID: targetTableID, + sourceTable: sourceTable, + oldValues: oldValues, + values: values, + columns: columns, + sourceTableInfo: sourceTableInfo, + originOldValues: originOldValues, + originValues: originValues, + downstreamIndexInfo: downstreamIndexInfo, } } @@ -590,19 +610,23 @@ func (dml *DML) whereColumnsAndValues() ([]string, []interface{}) { values = dml.originOldValues } - defaultIndexColumns := findFitIndex(dml.sourceTableInfo) - - if defaultIndexColumns == nil { - defaultIndexColumns = getAvailableIndexColumn(dml.sourceTableInfo, values) - } - if defaultIndexColumns != nil { - columns, values = getColumnData(dml.sourceTableInfo.Columns, defaultIndexColumns, values) + if dml.downstreamIndexInfo != nil { + columns, values = getColumnData(dml.sourceTableInfo.Columns, dml.downstreamIndexInfo, values) } columnNames := make([]string, 0, len(columns)) for _, column := range columns { columnNames = append(columnNames, column.Name.O) } + + failpoint.Inject("DownstreamTrackerWhereCheck", func() { + if dml.op == update { + log.L().Info("UpdateWhereColumnsCheck", zap.String("Columns", fmt.Sprintf("%v", columnNames))) + } else if dml.op == del { + log.L().Info("DeleteWhereColumnsCheck", zap.String("Columns", fmt.Sprintf("%v", columnNames))) + } + }) + return columnNames, values } diff --git a/dm/syncer/dml_test.go b/dm/syncer/dml_test.go index 2175bcd43e4..3d892066a9e 100644 --- a/dm/syncer/dml_test.go +++ b/dm/syncer/dml_test.go @@ -235,6 +235,18 @@ func (s *testSyncerSuite) TestGenWhere(c *C) { schema2 := "create table test.tb(id int, col1 int, col2 int, name varchar(24))" ti2, err := createTableInfo(p, se, 0, schema2) c.Assert(err, IsNil) + ti1Index := &model.IndexInfo{ + Table: ti1.Name, + Unique: true, + Primary: true, + State: model.StatePublic, + Tp: model.IndexTypeBtree, + Columns: []*model.IndexColumn{{ + Name: ti1.Columns[0].Name, + Offset: ti1.Columns[0].Offset, + Length: types.UnspecifiedLength, + }}, + } testCases := []struct { dml *DML @@ -242,26 +254,27 @@ func (s *testSyncerSuite) TestGenWhere(c *C) { values []interface{} }{ { - newDML(del, false, "", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti1.Columns, ti1), + newDML(del, false, "", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti1.Columns, ti1, ti1Index), "`id` = ?", []interface{}{1}, }, { - newDML(update, false, "", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, ti1.Columns, ti1), + newDML(update, false, "", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, ti1.Columns, ti1, ti1Index), "`id` = ?", []interface{}{1}, }, { - newDML(del, false, "", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti2.Columns, ti2), + newDML(del, false, "", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti2.Columns, ti2, nil), "`id` = ? AND `col1` = ? AND `col2` = ? AND `name` = ?", []interface{}{1, 2, 3, "haha"}, }, { - newDML(update, false, "", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, ti2.Columns, ti2), + newDML(update, false, "", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, ti2.Columns, ti2, nil), "`id` = ? AND `col1` = ? AND `col2` = ? AND `name` = ?", []interface{}{1, 2, 3, "haha"}, }, } + for _, tc := range testCases { var buf strings.Builder whereValues := tc.dml.genWhere(&buf) @@ -276,6 +289,18 @@ func (s *testSyncerSuite) TestGenSQL(c *C) { schema := "create table test.tb(id int primary key, col1 int unique not null, col2 int unique, name varchar(24))" ti, err := createTableInfo(p, se, 0, schema) c.Assert(err, IsNil) + tiIndex := &model.IndexInfo{ + Table: ti.Name, + Unique: true, + Primary: true, + State: model.StatePublic, + Tp: model.IndexTypeBtree, + Columns: []*model.IndexColumn{{ + Name: ti.Columns[0].Name, + Offset: ti.Columns[0].Offset, + Length: types.UnspecifiedLength, + }}, + } testCases := []struct { dml *DML @@ -283,27 +308,27 @@ func (s *testSyncerSuite) TestGenSQL(c *C) { args [][]interface{} }{ { - newDML(insert, false, "`targetSchema`.`targetTable`", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti), + newDML(insert, false, "`targetSchema`.`targetTable`", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex), []string{"INSERT INTO `targetSchema`.`targetTable` (`id`,`col1`,`col2`,`name`) VALUES (?,?,?,?)"}, [][]interface{}{{1, 2, 3, "haha"}}, }, { - newDML(insert, true, "`targetSchema`.`targetTable`", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti), + newDML(insert, true, "`targetSchema`.`targetTable`", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex), []string{"INSERT INTO `targetSchema`.`targetTable` (`id`,`col1`,`col2`,`name`) VALUES (?,?,?,?) ON DUPLICATE KEY UPDATE `id`=VALUES(`id`),`col1`=VALUES(`col1`),`col2`=VALUES(`col2`),`name`=VALUES(`name`)"}, [][]interface{}{{1, 2, 3, "haha"}}, }, { - newDML(del, false, "`targetSchema`.`targetTable`", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti), + newDML(del, false, "`targetSchema`.`targetTable`", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex), []string{"DELETE FROM `targetSchema`.`targetTable` WHERE `id` = ? LIMIT 1"}, [][]interface{}{{1}}, }, { - newDML(update, false, "`targetSchema`.`targetTable`", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti), + newDML(update, false, "`targetSchema`.`targetTable`", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex), []string{"UPDATE `targetSchema`.`targetTable` SET `id` = ?, `col1` = ?, `col2` = ?, `name` = ? WHERE `id` = ? LIMIT 1"}, [][]interface{}{{4, 5, 6, "hihi", 1}}, }, { - newDML(update, true, "`targetSchema`.`targetTable`", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti), + newDML(update, true, "`targetSchema`.`targetTable`", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex), []string{"DELETE FROM `targetSchema`.`targetTable` WHERE `id` = ? LIMIT 1", "INSERT INTO `targetSchema`.`targetTable` (`id`,`col1`,`col2`,`name`) VALUES (?,?,?,?) ON DUPLICATE KEY UPDATE `id`=VALUES(`id`),`col1`=VALUES(`col1`),`col2`=VALUES(`col2`),`name`=VALUES(`name`)"}, [][]interface{}{{1}, {4, 5, 6, "hihi"}}, }, @@ -338,62 +363,110 @@ func (s *testSyncerSuite) TestGenDMLWithSameOp(c *C) { schema22 := "create table dbb.tbb(id int primary key, col3 int unique not null, name varchar(24))" ti11, err := createTableInfo(p, se, 0, schema11) c.Assert(err, IsNil) + ti11Index := &model.IndexInfo{ + Table: ti11.Name, + Unique: true, + Primary: true, + State: model.StatePublic, + Tp: model.IndexTypeBtree, + Columns: []*model.IndexColumn{{ + Name: ti11.Columns[0].Name, + Offset: ti11.Columns[0].Offset, + Length: types.UnspecifiedLength, + }}, + } ti12, err := createTableInfo(p, se, 0, schema12) c.Assert(err, IsNil) + ti12Index := &model.IndexInfo{ + Table: ti12.Name, + Unique: true, + Primary: true, + State: model.StatePublic, + Tp: model.IndexTypeBtree, + Columns: []*model.IndexColumn{{ + Name: ti12.Columns[0].Name, + Offset: ti12.Columns[0].Offset, + Length: types.UnspecifiedLength, + }}, + } ti21, err := createTableInfo(p, se, 0, schema21) c.Assert(err, IsNil) + ti21Index := &model.IndexInfo{ + Table: ti21.Name, + Unique: true, + Primary: true, + State: model.StatePublic, + Tp: model.IndexTypeBtree, + Columns: []*model.IndexColumn{{ + Name: ti21.Columns[0].Name, + Offset: ti21.Columns[0].Offset, + Length: types.UnspecifiedLength, + }}, + } ti22, err := createTableInfo(p, se, 0, schema22) c.Assert(err, IsNil) + ti22Index := &model.IndexInfo{ + Table: ti22.Name, + Unique: true, + Primary: true, + State: model.StatePublic, + Tp: model.IndexTypeBtree, + Columns: []*model.IndexColumn{{ + Name: ti22.Columns[0].Name, + Offset: ti22.Columns[0].Offset, + Length: types.UnspecifiedLength, + }}, + } dmls := []*DML{ // insert - newDML(insert, true, targetTableID1, sourceTable11, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti11.Columns, ti11), - newDML(insert, true, targetTableID1, sourceTable11, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti11.Columns, ti11), - newDML(insert, true, targetTableID1, sourceTable12, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti12.Columns, ti12), + newDML(insert, true, targetTableID1, sourceTable11, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti11.Columns, ti11, ti11Index), + newDML(insert, true, targetTableID1, sourceTable11, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti11.Columns, ti11, ti11Index), + newDML(insert, true, targetTableID1, sourceTable12, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti12.Columns, ti12, ti12Index), // update no index - newDML(update, true, targetTableID1, sourceTable11, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, ti11.Columns, ti11), - newDML(update, true, targetTableID1, sourceTable11, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, ti11.Columns, ti11), - newDML(update, true, targetTableID1, sourceTable12, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, ti12.Columns, ti12), + newDML(update, true, targetTableID1, sourceTable11, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, ti11.Columns, ti11, ti11Index), + newDML(update, true, targetTableID1, sourceTable11, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, ti11.Columns, ti11, ti11Index), + newDML(update, true, targetTableID1, sourceTable12, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, ti12.Columns, ti12, ti12Index), // update uk - newDML(update, true, targetTableID1, sourceTable11, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, ti11.Columns, ti11), - newDML(update, true, targetTableID1, sourceTable11, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, ti11.Columns, ti11), - newDML(update, true, targetTableID1, sourceTable12, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, ti12.Columns, ti12), + newDML(update, true, targetTableID1, sourceTable11, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, ti11.Columns, ti11, ti11Index), + newDML(update, true, targetTableID1, sourceTable11, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, ti11.Columns, ti11, ti11Index), + newDML(update, true, targetTableID1, sourceTable12, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, ti12.Columns, ti12, ti12Index), // update pk - newDML(update, true, targetTableID1, sourceTable11, []interface{}{1, 4, "aa"}, []interface{}{4, 4, "aa"}, []interface{}{1, 1, "aa"}, []interface{}{4, 4, "aa"}, ti11.Columns, ti11), - newDML(update, true, targetTableID1, sourceTable11, []interface{}{2, 5, "bb"}, []interface{}{5, 5, "bb"}, []interface{}{2, 2, "bb"}, []interface{}{5, 5, "bb"}, ti11.Columns, ti11), - newDML(update, true, targetTableID1, sourceTable12, []interface{}{3, 6, "cc"}, []interface{}{6, 6, "cc"}, []interface{}{3, 3, "cc"}, []interface{}{6, 6, "cc"}, ti12.Columns, ti12), + newDML(update, true, targetTableID1, sourceTable11, []interface{}{1, 4, "aa"}, []interface{}{4, 4, "aa"}, []interface{}{1, 1, "aa"}, []interface{}{4, 4, "aa"}, ti11.Columns, ti11, ti11Index), + newDML(update, true, targetTableID1, sourceTable11, []interface{}{2, 5, "bb"}, []interface{}{5, 5, "bb"}, []interface{}{2, 2, "bb"}, []interface{}{5, 5, "bb"}, ti11.Columns, ti11, ti11Index), + newDML(update, true, targetTableID1, sourceTable12, []interface{}{3, 6, "cc"}, []interface{}{6, 6, "cc"}, []interface{}{3, 3, "cc"}, []interface{}{6, 6, "cc"}, ti12.Columns, ti12, ti12Index), // delete - newDML(del, true, targetTableID1, sourceTable11, nil, []interface{}{4, 4, "aa"}, nil, []interface{}{4, 4, "aa"}, ti11.Columns, ti11), - newDML(del, true, targetTableID1, sourceTable11, nil, []interface{}{5, 5, "bb"}, nil, []interface{}{5, 5, "bb"}, ti11.Columns, ti11), - newDML(del, true, targetTableID1, sourceTable12, nil, []interface{}{6, 6, "cc"}, nil, []interface{}{6, 6, "cc"}, ti12.Columns, ti12), + newDML(del, true, targetTableID1, sourceTable11, nil, []interface{}{4, 4, "aa"}, nil, []interface{}{4, 4, "aa"}, ti11.Columns, ti11, ti11Index), + newDML(del, true, targetTableID1, sourceTable11, nil, []interface{}{5, 5, "bb"}, nil, []interface{}{5, 5, "bb"}, ti11.Columns, ti11, ti11Index), + newDML(del, true, targetTableID1, sourceTable12, nil, []interface{}{6, 6, "cc"}, nil, []interface{}{6, 6, "cc"}, ti12.Columns, ti12, ti12Index), // target table 2 // insert - newDML(insert, true, targetTableID2, sourceTable21, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti21.Columns, ti21), - newDML(insert, false, targetTableID2, sourceTable21, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti21.Columns, ti21), - newDML(insert, false, targetTableID2, sourceTable22, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti22.Columns, ti22), + newDML(insert, true, targetTableID2, sourceTable21, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti21.Columns, ti21, ti21Index), + newDML(insert, false, targetTableID2, sourceTable21, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti21.Columns, ti21, ti21Index), + newDML(insert, false, targetTableID2, sourceTable22, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti22.Columns, ti22, ti22Index), // update no index - newDML(update, false, targetTableID2, sourceTable21, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, ti21.Columns, ti21), - newDML(update, false, targetTableID2, sourceTable21, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, ti21.Columns, ti21), - newDML(update, false, targetTableID2, sourceTable22, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, ti22.Columns, ti22), + newDML(update, false, targetTableID2, sourceTable21, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, ti21.Columns, ti21, ti21Index), + newDML(update, false, targetTableID2, sourceTable21, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, ti21.Columns, ti21, ti21Index), + newDML(update, false, targetTableID2, sourceTable22, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, ti22.Columns, ti22, ti21Index), // update uk - newDML(update, false, targetTableID2, sourceTable21, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, ti21.Columns, ti21), - newDML(update, false, targetTableID2, sourceTable21, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, ti21.Columns, ti21), - newDML(update, false, targetTableID2, sourceTable22, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, ti22.Columns, ti22), + newDML(update, false, targetTableID2, sourceTable21, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, ti21.Columns, ti21, ti21Index), + newDML(update, false, targetTableID2, sourceTable21, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, ti21.Columns, ti21, ti21Index), + newDML(update, false, targetTableID2, sourceTable22, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, ti22.Columns, ti22, ti22Index), // update pk - newDML(update, false, targetTableID2, sourceTable21, []interface{}{1, 4, "aa"}, []interface{}{4, 4, "aa"}, []interface{}{1, 1, "aa"}, []interface{}{4, 4, "aa"}, ti21.Columns, ti21), - newDML(update, false, targetTableID2, sourceTable21, []interface{}{2, 5, "bb"}, []interface{}{5, 5, "bb"}, []interface{}{2, 2, "bb"}, []interface{}{5, 5, "bb"}, ti21.Columns, ti21), - newDML(update, false, targetTableID2, sourceTable22, []interface{}{3, 6, "cc"}, []interface{}{6, 6, "cc"}, []interface{}{3, 3, "cc"}, []interface{}{6, 6, "cc"}, ti22.Columns, ti22), + newDML(update, false, targetTableID2, sourceTable21, []interface{}{1, 4, "aa"}, []interface{}{4, 4, "aa"}, []interface{}{1, 1, "aa"}, []interface{}{4, 4, "aa"}, ti21.Columns, ti21, ti21Index), + newDML(update, false, targetTableID2, sourceTable21, []interface{}{2, 5, "bb"}, []interface{}{5, 5, "bb"}, []interface{}{2, 2, "bb"}, []interface{}{5, 5, "bb"}, ti21.Columns, ti21, ti21Index), + newDML(update, false, targetTableID2, sourceTable22, []interface{}{3, 6, "cc"}, []interface{}{6, 6, "cc"}, []interface{}{3, 3, "cc"}, []interface{}{6, 6, "cc"}, ti22.Columns, ti22, ti22Index), // delete - newDML(del, false, targetTableID2, sourceTable21, nil, []interface{}{4, 4, "aa"}, nil, []interface{}{4, 4, "aa"}, ti21.Columns, ti21), - newDML(del, false, targetTableID2, sourceTable21, nil, []interface{}{5, 5, "bb"}, nil, []interface{}{5, 5, "bb"}, ti21.Columns, ti21), - newDML(del, false, targetTableID2, sourceTable22, nil, []interface{}{6, 6, "cc"}, nil, []interface{}{6, 6, "cc"}, ti22.Columns, ti22), + newDML(del, false, targetTableID2, sourceTable21, nil, []interface{}{4, 4, "aa"}, nil, []interface{}{4, 4, "aa"}, ti21.Columns, ti21, ti21Index), + newDML(del, false, targetTableID2, sourceTable21, nil, []interface{}{5, 5, "bb"}, nil, []interface{}{5, 5, "bb"}, ti21.Columns, ti21, ti21Index), + newDML(del, false, targetTableID2, sourceTable22, nil, []interface{}{6, 6, "cc"}, nil, []interface{}{6, 6, "cc"}, ti22.Columns, ti22, ti22Index), // table1 // detele - newDML(del, false, targetTableID1, sourceTable11, nil, []interface{}{44, 44, "aaa"}, nil, []interface{}{44, 44, "aaa"}, ti11.Columns, ti11), - newDML(del, false, targetTableID1, sourceTable11, nil, []interface{}{55, 55, "bbb"}, nil, []interface{}{55, 55, "bbb"}, ti11.Columns, ti11), - newDML(del, false, targetTableID1, sourceTable12, nil, []interface{}{66, 66, "ccc"}, nil, []interface{}{66, 66, "ccc"}, ti12.Columns, ti12), + newDML(del, false, targetTableID1, sourceTable11, nil, []interface{}{44, 44, "aaa"}, nil, []interface{}{44, 44, "aaa"}, ti11.Columns, ti11, ti11Index), + newDML(del, false, targetTableID1, sourceTable11, nil, []interface{}{55, 55, "bbb"}, nil, []interface{}{55, 55, "bbb"}, ti11.Columns, ti11, ti11Index), + newDML(del, false, targetTableID1, sourceTable12, nil, []interface{}{66, 66, "ccc"}, nil, []interface{}{66, 66, "ccc"}, ti12.Columns, ti12, ti12Index), } expectQueries := []string{ diff --git a/dm/syncer/expr_filter_group_test.go b/dm/syncer/expr_filter_group_test.go index b5b8924774f..819386b4bab 100644 --- a/dm/syncer/expr_filter_group_test.go +++ b/dm/syncer/expr_filter_group_test.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/pkg/schema" "github.com/pingcap/tiflow/dm/pkg/utils" + "github.com/pingcap/tiflow/dm/syncer/dbconn" ) func (s *testFilterSuite) TestSkipDMLByExpression(c *C) { @@ -92,8 +93,9 @@ create table t ( ) c.Assert(log.InitLogger(&log.Config{Level: "debug"}), IsNil) + dbConn := &dbconn.DBConn{Cfg: &config.SubTaskConfig{}, BaseConn: s.baseConn} for _, ca := range cases { - schemaTracker, err := schema.NewTracker(ctx, "unit-test", defaultTestSessionCfg, s.baseConn) + schemaTracker, err := schema.NewTracker(ctx, "unit-test", defaultTestSessionCfg, dbConn) c.Assert(err, IsNil) c.Assert(schemaTracker.CreateSchemaIfNotExists(dbName), IsNil) c.Assert(schemaTracker.Exec(ctx, dbName, ca.tableStr), IsNil) @@ -350,9 +352,10 @@ create table t ( ) c.Assert(log.InitLogger(&log.Config{Level: "debug"}), IsNil) + dbConn := &dbconn.DBConn{Cfg: &config.SubTaskConfig{}, BaseConn: s.baseConn} for _, ca := range cases { c.Log(ca.tableStr) - schemaTracker, err := schema.NewTracker(ctx, "unit-test", defaultTestSessionCfg, s.baseConn) + schemaTracker, err := schema.NewTracker(ctx, "unit-test", defaultTestSessionCfg, dbConn) c.Assert(err, IsNil) c.Assert(schemaTracker.CreateSchemaIfNotExists(dbName), IsNil) c.Assert(schemaTracker.Exec(ctx, dbName, ca.tableStr), IsNil) @@ -401,7 +404,9 @@ create table t ( );` exprStr = "d > 1" ) - schemaTracker, err := schema.NewTracker(ctx, "unit-test", defaultTestSessionCfg, s.baseConn) + + dbConn := &dbconn.DBConn{Cfg: &config.SubTaskConfig{}, BaseConn: s.baseConn} + schemaTracker, err := schema.NewTracker(ctx, "unit-test", defaultTestSessionCfg, dbConn) c.Assert(err, IsNil) c.Assert(schemaTracker.CreateSchemaIfNotExists(dbName), IsNil) c.Assert(schemaTracker.Exec(ctx, dbName, tableStr), IsNil) diff --git a/dm/syncer/filter_test.go b/dm/syncer/filter_test.go index 383362f257b..9ad9c4d37bb 100644 --- a/dm/syncer/filter_test.go +++ b/dm/syncer/filter_test.go @@ -67,7 +67,7 @@ func (s *testFilterSuite) TestSkipQueryEvent(c *C) { c.Assert(err, IsNil) syncer.ddlDBConn = &dbconn.DBConn{Cfg: syncer.cfg, BaseConn: s.baseConn} - syncer.schemaTracker, err = schema.NewTracker(context.Background(), syncer.cfg.Name, defaultTestSessionCfg, syncer.ddlDBConn.BaseConn) + syncer.schemaTracker, err = schema.NewTracker(context.Background(), syncer.cfg.Name, defaultTestSessionCfg, syncer.ddlDBConn) c.Assert(err, IsNil) syncer.exprFilterGroup = NewExprFilterGroup(utils.NewSessionCtx(nil), nil) diff --git a/dm/syncer/job_test.go b/dm/syncer/job_test.go index dcd43ef2324..a40c8e99e56 100644 --- a/dm/syncer/job_test.go +++ b/dm/syncer/job_test.go @@ -17,6 +17,8 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/tidb-tools/pkg/filter" "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/parser/model" + "github.com/pingcap/tidb/parser/types" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tiflow/dm/pkg/binlog" @@ -87,13 +89,25 @@ func (t *testJobSuite) TestJob(c *C) { se := mock.NewContext() ti, err := createTableInfo(p, se, 0, schema) c.Assert(err, IsNil) + tiIndex := &model.IndexInfo{ + Table: ti.Name, + Unique: true, + Primary: true, + State: model.StatePublic, + Tp: model.IndexTypeBtree, + Columns: []*model.IndexColumn{{ + Name: ti.Columns[0].Name, + Offset: ti.Columns[0].Offset, + Length: types.UnspecifiedLength, + }}, + } testCases := []struct { job *job jobStr string }{ { - newDMLJob(insert, table, table, newDML(insert, true, "targetTable", table, nil, []interface{}{2, 2}, nil, []interface{}{2, 2}, ti.Columns, ti), ec), + newDMLJob(insert, table, table, newDML(insert, true, "targetTable", table, nil, []interface{}{2, 2}, nil, []interface{}{2, 2}, ti.Columns, ti, tiIndex), ec), "tp: insert, dml: [safemode: true, targetTableID: targetTable, op: insert, columns: [id col1], oldValues: [], values: [2 2]], ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", }, { newDDLJob(qec), diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index a73a160102d..22eaf285c5c 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -141,10 +141,11 @@ type Syncer struct { fromDB *dbconn.UpStreamConn - toDB *conn.BaseDB - toDBConns []*dbconn.DBConn - ddlDB *conn.BaseDB - ddlDBConn *dbconn.DBConn + toDB *conn.BaseDB + toDBConns []*dbconn.DBConn + ddlDB *conn.BaseDB + ddlDBConn *dbconn.DBConn + downstreamTrackConn *dbconn.DBConn dmlJobCh chan *job ddlJobCh chan *job @@ -320,7 +321,7 @@ func (s *Syncer) Init(ctx context.Context) (err error) { } rollbackHolder.Add(fr.FuncRollback{Name: "close-DBs", Fn: s.closeDBs}) - s.schemaTracker, err = schema.NewTracker(ctx, s.cfg.Name, s.cfg.To.Session, s.ddlDBConn.BaseConn) + s.schemaTracker, err = schema.NewTracker(ctx, s.cfg.Name, s.cfg.To.Session, s.downstreamTrackConn) if err != nil { return terror.ErrSchemaTrackerInit.Delegate(err) } @@ -583,6 +584,11 @@ func (s *Syncer) resetDBs(tctx *tcontext.Context) error { return terror.WithScope(err, terror.ScopeDownstream) } + err = s.downstreamTrackConn.ResetConn(tctx) + if err != nil { + return terror.WithScope(err, terror.ScopeDownstream) + } + err = s.checkpoint.ResetConn(tctx) if err != nil { return terror.WithScope(err, terror.ScopeDownstream) @@ -735,62 +741,51 @@ func (s *Syncer) trackTableInfoFromDownstream(tctx *tcontext.Context, sourceTabl return terror.ErrSchemaTrackerCannotParseDownstreamTable.Delegate(err, targetTable, sourceTable) } - rows, err := s.ddlDBConn.QuerySQL(tctx, "SHOW CREATE TABLE "+targetTable.String()) + createSQL, err := utils.GetTableCreateSQL(tctx.Ctx, s.ddlDBConn.BaseConn.DBConn, targetTable.String()) if err != nil { return terror.ErrSchemaTrackerCannotFetchDownstreamTable.Delegate(err, targetTable, sourceTable) } - defer rows.Close() - - for rows.Next() { - var tableName, createSQL string - if err = rows.Scan(&tableName, &createSQL); err != nil { - return terror.WithScope(terror.DBErrorAdapt(err, terror.ErrDBDriverError), terror.ScopeDownstream) - } - // rename the table back to original. - var createNode ast.StmtNode - createNode, err = parser2.ParseOneStmt(createSQL, "", "") - if err != nil { - return terror.ErrSchemaTrackerCannotParseDownstreamTable.Delegate(err, targetTable, sourceTable) - } - createStmt := createNode.(*ast.CreateTableStmt) - createStmt.IfNotExists = true - createStmt.Table.Schema = model.NewCIStr(sourceTable.Schema) - createStmt.Table.Name = model.NewCIStr(sourceTable.Name) - - // schema tracker sets non-clustered index, so can't handle auto_random. - if v, _ := s.schemaTracker.GetSystemVar(schema.TiDBClusteredIndex); v == "OFF" { - for _, col := range createStmt.Cols { - for i, opt := range col.Options { - if opt.Tp == ast.ColumnOptionAutoRandom { - // col.Options is unordered - col.Options[i] = col.Options[len(col.Options)-1] - col.Options = col.Options[:len(col.Options)-1] - break - } + // rename the table back to original. + var createNode ast.StmtNode + createNode, err = parser2.ParseOneStmt(createSQL, "", "") + if err != nil { + return terror.ErrSchemaTrackerCannotParseDownstreamTable.Delegate(err, targetTable, sourceTable) + } + createStmt := createNode.(*ast.CreateTableStmt) + createStmt.IfNotExists = true + createStmt.Table.Schema = model.NewCIStr(sourceTable.Schema) + createStmt.Table.Name = model.NewCIStr(sourceTable.Name) + + // schema tracker sets non-clustered index, so can't handle auto_random. + if v, _ := s.schemaTracker.GetSystemVar(schema.TiDBClusteredIndex); v == "OFF" { + for _, col := range createStmt.Cols { + for i, opt := range col.Options { + if opt.Tp == ast.ColumnOptionAutoRandom { + // col.Options is unordered + col.Options[i] = col.Options[len(col.Options)-1] + col.Options = col.Options[:len(col.Options)-1] + break } } } - - var newCreateSQLBuilder strings.Builder - restoreCtx := format.NewRestoreCtx(format.DefaultRestoreFlags, &newCreateSQLBuilder) - if err = createStmt.Restore(restoreCtx); err != nil { - return terror.ErrSchemaTrackerCannotParseDownstreamTable.Delegate(err, targetTable, sourceTable) - } - newCreateSQL := newCreateSQLBuilder.String() - tctx.L().Debug("reverse-synchronized table schema", - zap.Stringer("sourceTable", sourceTable), - zap.Stringer("targetTable", targetTable), - zap.String("sql", newCreateSQL), - ) - if err = s.schemaTracker.Exec(tctx.Ctx, sourceTable.Schema, newCreateSQL); err != nil { - return terror.ErrSchemaTrackerCannotCreateTable.Delegate(err, sourceTable) - } } - if err = rows.Err(); err != nil { - return terror.WithScope(terror.DBErrorAdapt(err, terror.ErrDBDriverError), terror.ScopeDownstream) + var newCreateSQLBuilder strings.Builder + restoreCtx := format.NewRestoreCtx(format.DefaultRestoreFlags, &newCreateSQLBuilder) + if err = createStmt.Restore(restoreCtx); err != nil { + return terror.ErrSchemaTrackerCannotParseDownstreamTable.Delegate(err, targetTable, sourceTable) } + newCreateSQL := newCreateSQLBuilder.String() + tctx.L().Debug("reverse-synchronized table schema", + zap.Stringer("sourceTable", sourceTable), + zap.Stringer("targetTable", targetTable), + zap.String("sql", newCreateSQL), + ) + if err = s.schemaTracker.Exec(tctx.Ctx, sourceTable.Schema, newCreateSQL); err != nil { + return terror.ErrSchemaTrackerCannotCreateTable.Delegate(err, sourceTable) + } + return nil } @@ -2043,7 +2038,7 @@ func (s *Syncer) handleRowsEvent(ev *replication.RowsEvent, ec eventContext) err } param.safeMode = ec.safeMode - dmls, err = s.genAndFilterUpdateDMLs(param, oldExprFilter, newExprFilter) + dmls, err = s.genAndFilterUpdateDMLs(ec.tctx, param, oldExprFilter, newExprFilter) if err != nil { return terror.Annotatef(err, "gen update sqls failed, sourceTable: %v, targetTable: %v", sourceTable, targetTable) } @@ -2056,7 +2051,7 @@ func (s *Syncer) handleRowsEvent(ev *replication.RowsEvent, ec eventContext) err return err2 } - dmls, err = s.genAndFilterDeleteDMLs(param, exprFilter) + dmls, err = s.genAndFilterDeleteDMLs(ec.tctx, param, exprFilter) if err != nil { return terror.Annotatef(err, "gen delete sqls failed, sourceTable: %v, targetTable: %v", sourceTable, targetTable) } @@ -2635,6 +2630,7 @@ func (s *Syncer) trackDDL(usedSchema string, trackInfo *ddlInfo, ec *eventContex shouldTableExistNum int // tableNames[:shouldTableExistNum] should exist shouldRefTableExistNum int // tableNames[1:shouldTableExistNum] should exist, since first one is "caller table" tryFetchDownstreamTable bool // to make sure if not exists will execute correctly + shouldReTrackDownstreamIndex bool // retrack downstreamIndex ) switch node := trackInfo.originStmt.(type) { @@ -2645,6 +2641,7 @@ func (s *Syncer) trackDDL(usedSchema string, trackInfo *ddlInfo, ec *eventContex shouldSchemaExist = true case *ast.DropDatabaseStmt: shouldExecDDLOnSchemaTracker = true + shouldReTrackDownstreamIndex = true if s.cfg.ShardMode == "" { if err := s.checkpoint.DeleteSchemaPoint(ec.tctx, srcTable.Schema); err != nil { return err @@ -2661,6 +2658,7 @@ func (s *Syncer) trackDDL(usedSchema string, trackInfo *ddlInfo, ec *eventContex tryFetchDownstreamTable = true case *ast.DropTableStmt: shouldExecDDLOnSchemaTracker = true + shouldReTrackDownstreamIndex = true if err := s.checkpoint.DeleteTablePoint(ec.tctx, srcTable); err != nil { return err } @@ -2668,8 +2666,10 @@ func (s *Syncer) trackDDL(usedSchema string, trackInfo *ddlInfo, ec *eventContex shouldExecDDLOnSchemaTracker = true shouldSchemaExist = true shouldTableExistNum = 1 + shouldReTrackDownstreamIndex = true case *ast.AlterTableStmt: shouldSchemaExist = true + shouldReTrackDownstreamIndex = true // for DDL that adds FK, since TiDB doesn't fully support it yet, we simply ignore execution of this DDL. switch { case len(node.Specs) == 1 && node.Specs[0].Constraint != nil && node.Specs[0].Constraint.Tp == ast.ConstraintForeignKey: @@ -2688,6 +2688,10 @@ func (s *Syncer) trackDDL(usedSchema string, trackInfo *ddlInfo, ec *eventContex ec.tctx.L().DPanic("unhandled DDL type cannot be tracked", zap.Stringer("type", reflect.TypeOf(trackInfo.originStmt))) } + if shouldReTrackDownstreamIndex { + s.schemaTracker.RemoveDownstreamSchema(s.tctx, targetTables) + } + if shouldSchemaExist { if err := s.schemaTracker.CreateSchemaIfNotExists(srcTable.Schema); err != nil { return terror.ErrSchemaTrackerCannotCreateSchema.Delegate(err, srcTable.Schema) @@ -2870,13 +2874,14 @@ func (s *Syncer) createDBs(ctx context.Context) error { dbCfg.RawDBCfg = config.DefaultRawDBConfig().SetReadTimeout(maxDDLConnectionTimeout) var ddlDBConns []*dbconn.DBConn - s.ddlDB, ddlDBConns, err = dbconn.CreateConns(s.tctx, s.cfg, &dbCfg, 1) + s.ddlDB, ddlDBConns, err = dbconn.CreateConns(s.tctx, s.cfg, &dbCfg, 2) if err != nil { dbconn.CloseUpstreamConn(s.tctx, s.fromDB) dbconn.CloseBaseDB(s.tctx, s.toDB) return err } s.ddlDBConn = ddlDBConns[0] + s.downstreamTrackConn = ddlDBConns[1] printServerVersion(s.tctx, s.fromDB.BaseDB, "upstream") printServerVersion(s.tctx, s.toDB, "downstream") diff --git a/dm/syncer/syncer_test.go b/dm/syncer/syncer_test.go index f60f6a4581f..3fa0b82f227 100644 --- a/dm/syncer/syncer_test.go +++ b/dm/syncer/syncer_test.go @@ -54,6 +54,7 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" + pmysql "github.com/pingcap/tidb/parser/mysql" "go.uber.org/zap" ) @@ -779,8 +780,20 @@ func (s *testSyncerSuite) TestRun(c *C) { {Cfg: s.cfg, BaseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})}, } syncer.ddlDBConn = &dbconn.DBConn{Cfg: s.cfg, BaseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})} - syncer.schemaTracker, err = schema.NewTracker(context.Background(), s.cfg.Name, defaultTestSessionCfg, syncer.ddlDBConn.BaseConn) + syncer.downstreamTrackConn = &dbconn.DBConn{Cfg: s.cfg, BaseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})} + syncer.schemaTracker, err = schema.NewTracker(context.Background(), s.cfg.Name, defaultTestSessionCfg, syncer.downstreamTrackConn) + mock.ExpectBegin() + mock.ExpectExec(fmt.Sprintf("SET SESSION SQL_MODE = '%s'", pmysql.DefaultSQLMode)).WillReturnResult(sqlmock.NewResult(0, 0)) + mock.ExpectCommit() + mock.ExpectQuery("SHOW CREATE TABLE " + "`test_1`.`t_1`").WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("t_1", "create table t_1(id int primary key, name varchar(24))")) + s.mockGetServerUnixTS(mock) + mock.ExpectQuery("SHOW CREATE TABLE " + "`test_1`.`t_2`").WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("t_2", "create table t_2(id int primary key, name varchar(24))")) syncer.exprFilterGroup = NewExprFilterGroup(utils.NewSessionCtx(nil), nil) + c.Assert(err, IsNil) c.Assert(syncer.Type(), Equals, pb.UnitType_Sync) @@ -791,7 +804,6 @@ func (s *testSyncerSuite) TestRun(c *C) { syncer.setupMockCheckpoint(c, checkPointDBConn, checkPointMock) syncer.reset() - s.mockGetServerUnixTS(mock) events1 := mockBinlogEvents{ mockBinlogEvent{typ: DBCreate, args: []interface{}{"test_1"}}, mockBinlogEvent{typ: TableCreate, args: []interface{}{"test_1", "create table test_1.t_1(id int primary key, name varchar(24))"}}, @@ -1026,8 +1038,18 @@ func (s *testSyncerSuite) TestExitSafeModeByConfig(c *C) { {Cfg: s.cfg, BaseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})}, } syncer.ddlDBConn = &dbconn.DBConn{Cfg: s.cfg, BaseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})} - syncer.schemaTracker, err = schema.NewTracker(context.Background(), s.cfg.Name, defaultTestSessionCfg, syncer.ddlDBConn.BaseConn) + syncer.downstreamTrackConn = &dbconn.DBConn{Cfg: s.cfg, BaseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})} + mock.ExpectBegin() + mock.ExpectExec(fmt.Sprintf("SET SESSION SQL_MODE = '%s'", pmysql.DefaultSQLMode)).WillReturnResult(sqlmock.NewResult(0, 0)) + mock.ExpectCommit() + + mock.ExpectQuery("SHOW CREATE TABLE " + "`test_1`.`t_1`").WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("t_1", "create table t_1(id int primary key, name varchar(24))")) + + syncer.schemaTracker, err = schema.NewTracker(context.Background(), s.cfg.Name, defaultTestSessionCfg, syncer.ddlDBConn) syncer.exprFilterGroup = NewExprFilterGroup(utils.NewSessionCtx(nil), nil) + c.Assert(err, IsNil) c.Assert(syncer.Type(), Equals, pb.UnitType_Sync) @@ -1215,8 +1237,9 @@ func (s *testSyncerSuite) TestTrackDDL(c *C) { } syncer.ddlDBConn = &dbconn.DBConn{Cfg: s.cfg, BaseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})} syncer.checkpoint.(*RemoteCheckPoint).dbConn = &dbconn.DBConn{Cfg: s.cfg, BaseConn: conn.NewBaseConn(checkPointDBConn, &retry.FiniteRetryStrategy{})} - syncer.schemaTracker, err = schema.NewTracker(context.Background(), s.cfg.Name, defaultTestSessionCfg, syncer.ddlDBConn.BaseConn) + syncer.schemaTracker, err = schema.NewTracker(context.Background(), s.cfg.Name, defaultTestSessionCfg, syncer.ddlDBConn) syncer.exprFilterGroup = NewExprFilterGroup(utils.NewSessionCtx(nil), nil) + c.Assert(syncer.genRouter(), IsNil) c.Assert(err, IsNil) @@ -1479,7 +1502,8 @@ func (s *testSyncerSuite) TestTrackDownstreamTableWontOverwrite(c *C) { c.Assert(err, IsNil) baseConn := conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{}) syncer.ddlDBConn = &dbconn.DBConn{Cfg: s.cfg, BaseConn: baseConn} - syncer.schemaTracker, err = schema.NewTracker(ctx, s.cfg.Name, defaultTestSessionCfg, baseConn) + syncer.downstreamTrackConn = &dbconn.DBConn{Cfg: s.cfg, BaseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})} + syncer.schemaTracker, err = schema.NewTracker(ctx, s.cfg.Name, defaultTestSessionCfg, syncer.downstreamTrackConn) c.Assert(err, IsNil) upTable := &filter.Table{ @@ -1521,7 +1545,8 @@ func (s *testSyncerSuite) TestDownstreamTableHasAutoRandom(c *C) { c.Assert(err, IsNil) baseConn := conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{}) syncer.ddlDBConn = &dbconn.DBConn{Cfg: s.cfg, BaseConn: baseConn} - syncer.schemaTracker, err = schema.NewTracker(ctx, s.cfg.Name, defaultTestSessionCfg, baseConn) + syncer.downstreamTrackConn = &dbconn.DBConn{Cfg: s.cfg, BaseConn: conn.NewBaseConn(dbConn, &retry.FiniteRetryStrategy{})} + syncer.schemaTracker, err = schema.NewTracker(ctx, s.cfg.Name, defaultTestSessionCfg, syncer.downstreamTrackConn) c.Assert(err, IsNil) schemaName := "test" @@ -1561,7 +1586,7 @@ func (s *testSyncerSuite) TestDownstreamTableHasAutoRandom(c *C) { "tidb_skip_utf8_check": "0", schema.TiDBClusteredIndex: "ON", } - syncer.schemaTracker, err = schema.NewTracker(ctx, s.cfg.Name, sessionCfg, baseConn) + syncer.schemaTracker, err = schema.NewTracker(ctx, s.cfg.Name, sessionCfg, syncer.downstreamTrackConn) c.Assert(err, IsNil) v, ok := syncer.schemaTracker.GetSystemVar(schema.TiDBClusteredIndex) c.Assert(v, Equals, "ON") diff --git a/dm/tests/downstream_diff_index/conf/dm-master.toml b/dm/tests/downstream_diff_index/conf/dm-master.toml new file mode 100644 index 00000000000..7cecf59ad86 --- /dev/null +++ b/dm/tests/downstream_diff_index/conf/dm-master.toml @@ -0,0 +1,4 @@ +# Master Configuration. +master-addr = ":8261" +advertise-addr = "127.0.0.1:8261" +auto-compaction-retention = "3s" diff --git a/dm/tests/downstream_diff_index/conf/dm-task.yaml b/dm/tests/downstream_diff_index/conf/dm-task.yaml new file mode 100644 index 00000000000..5dc73b1ecf9 --- /dev/null +++ b/dm/tests/downstream_diff_index/conf/dm-task.yaml @@ -0,0 +1,61 @@ +--- +name: test +task-mode: all +is-sharding: true +shard-mode: "pessimistic" +meta-schema: "dm_meta" +# enable-heartbeat: true +heartbeat-update-interval: 1 +heartbeat-report-interval: 1 + +target-database: + host: "127.0.0.1" + port: 4000 + user: "root" + password: "" + +mysql-instances: + - source-id: "mysql-replica-01" + black-white-list: "instance" # compatible with deprecated config + route-rules: [ "downstream-table-rules","downstream-schema-rules" ] + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + + - source-id: "mysql-replica-02" + black-white-list: "instance" # compatible with deprecated config + route-rules: [ "downstream-table-rules","downstream-schema-rules" ] + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + +black-white-list: # compatible with deprecated config + instance: + do-dbs: ["downstream_diff_index*"] + +routes: + downstream-table-rules: + schema-pattern: "downstream_diff_index*" + target-schema: "downstream_diff_index" + table-pattern: "t*" + target-table: "t" + downstream-schema-rules: + schema-pattern: "downstream_diff_index*" + target-schema: "downstream_diff_index" + +mydumpers: + global: + threads: 4 + chunk-filesize: 64 + skip-tz-utc: true + extra-args: "" + +loaders: + global: + pool-size: 16 + dir: "./dumped_data" + +syncers: + global: + worker-count: 16 + batch: 100 diff --git a/dm/tests/downstream_diff_index/conf/dm-worker1.toml b/dm/tests/downstream_diff_index/conf/dm-worker1.toml new file mode 100644 index 00000000000..7a72ea72bf8 --- /dev/null +++ b/dm/tests/downstream_diff_index/conf/dm-worker1.toml @@ -0,0 +1,2 @@ +name = "worker1" +join = "127.0.0.1:8261" diff --git a/dm/tests/downstream_diff_index/conf/dm-worker2.toml b/dm/tests/downstream_diff_index/conf/dm-worker2.toml new file mode 100644 index 00000000000..010e21c73eb --- /dev/null +++ b/dm/tests/downstream_diff_index/conf/dm-worker2.toml @@ -0,0 +1,2 @@ +name = "worker2" +join = "127.0.0.1:8261" diff --git a/dm/tests/downstream_diff_index/conf/source1.yaml b/dm/tests/downstream_diff_index/conf/source1.yaml new file mode 100644 index 00000000000..c3b58180140 --- /dev/null +++ b/dm/tests/downstream_diff_index/conf/source1.yaml @@ -0,0 +1,10 @@ +source-id: mysql-replica-01 +flavor: '' +enable-gtid: true +from: + host: 127.0.0.1 + user: root + password: /Q7B9DizNLLTTfiZHv9WoEAKamfpIUs= + port: 3306 +checker: + check-enable: false diff --git a/dm/tests/downstream_diff_index/conf/source2.yaml b/dm/tests/downstream_diff_index/conf/source2.yaml new file mode 100644 index 00000000000..b5ef1489cfe --- /dev/null +++ b/dm/tests/downstream_diff_index/conf/source2.yaml @@ -0,0 +1,10 @@ +source-id: mysql-replica-02 +flavor: '' +enable-gtid: true +from: + host: 127.0.0.1 + user: root + password: /Q7B9DizNLLTTfiZHv9WoEAKamfpIUs= + port: 3307 +checker: + check-enable: false diff --git a/dm/tests/downstream_diff_index/data/db1.increment.sql b/dm/tests/downstream_diff_index/data/db1.increment.sql new file mode 100644 index 00000000000..24ceb2365a8 --- /dev/null +++ b/dm/tests/downstream_diff_index/data/db1.increment.sql @@ -0,0 +1,3 @@ +use downstream_diff_index1; +update t1 set c3 = '111' where c1 = 1; +delete from t1 where c1 = 2; diff --git a/dm/tests/downstream_diff_index/data/db1.prepare.sql b/dm/tests/downstream_diff_index/data/db1.prepare.sql new file mode 100644 index 00000000000..4d3d8443227 --- /dev/null +++ b/dm/tests/downstream_diff_index/data/db1.prepare.sql @@ -0,0 +1,7 @@ +drop database if exists `downstream_diff_index1`; +create database `downstream_diff_index1`; +use `downstream_diff_index1`; +create table t1 (c1 int, c2 int, c3 varchar(10), primary key(c1)); +insert into t1 values(1, 1, '1'); +insert into t1 values(2, 2, '2'); +insert into t1 values(3, 3, '3'); diff --git a/dm/tests/downstream_diff_index/data/db2.increment.sql b/dm/tests/downstream_diff_index/data/db2.increment.sql new file mode 100644 index 00000000000..4e367799586 --- /dev/null +++ b/dm/tests/downstream_diff_index/data/db2.increment.sql @@ -0,0 +1,3 @@ +use downstream_diff_index2; +update t2 set c3 = '333' where c1 = 3; +delete from t2 where c1 = 1; diff --git a/dm/tests/downstream_diff_index/data/db2.prepare.sql b/dm/tests/downstream_diff_index/data/db2.prepare.sql new file mode 100644 index 00000000000..13fac5510f1 --- /dev/null +++ b/dm/tests/downstream_diff_index/data/db2.prepare.sql @@ -0,0 +1,7 @@ +drop database if exists `downstream_diff_index2`; +create database `downstream_diff_index2`; +use `downstream_diff_index2`; +create table t2 (c1 int, c2 int, c3 varchar(10), primary key(c1)); +insert into t2 values(1, 12, '13'); +insert into t2 values(2, 22, '23'); +insert into t2 values(3, 32, '33'); diff --git a/dm/tests/downstream_diff_index/data/tidb.prepare.sql b/dm/tests/downstream_diff_index/data/tidb.prepare.sql new file mode 100644 index 00000000000..b5b0d0ccb85 --- /dev/null +++ b/dm/tests/downstream_diff_index/data/tidb.prepare.sql @@ -0,0 +1,4 @@ +drop database if exists `downstream_diff_index`; +create database `downstream_diff_index`; +use `downstream_diff_index`; +create table t (c1 int, c2 int, c3 varchar(10)); diff --git a/dm/tests/downstream_diff_index/run.sh b/dm/tests/downstream_diff_index/run.sh new file mode 100755 index 00000000000..eab1c4f87b9 --- /dev/null +++ b/dm/tests/downstream_diff_index/run.sh @@ -0,0 +1,78 @@ +#!/bin/bash + +set -eu + +cur=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +source $cur/../_utils/test_prepare +WORK_DIR=$TEST_DIR/$TEST_NAME +db1="downstream_diff_index1" +tb1="t1" +db2="downstream_diff_index2" +tb2="t2" +db="downstream_diff_index" +tb="t" + +function run() { + # create table in mysql with pk + run_sql_file $cur/data/db1.prepare.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 + run_sql_file $cur/data/db2.prepare.sql $MYSQL_HOST2 $MYSQL_PORT2 $MYSQL_PASSWORD2 + # create table in tidb with different pk + run_sql_file $cur/data/tidb.prepare.sql $TIDB_HOST $TIDB_PORT $TIDB_PASSWORD + + # start DM worker and master + run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml + check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT + + # worker will inject delete/update sql check + inject_points=( + "github.com/pingcap/tiflow/dm/syncer/DownstreamTrackerWhereCheck=return()" + ) + export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})" + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT + dmctl_operate_source create $cur/conf/source1.yaml $SOURCE_ID1 + + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT + dmctl_operate_source create $cur/conf/source2.yaml $SOURCE_ID2 + + # start DM task + dmctl_start_task "$cur/conf/dm-task.yaml" "--remove-meta" + # check full load data + run_sql_tidb_with_retry "select count(1) from ${db}.${tb} where c1<100;" "count(1): 6" + + # downstream create diff uk + run_sql_tidb "alter table ${db}.${tb} add unique key(c2);" + + # db1 increment data with update and delete + run_sql_file $cur/data/db1.increment.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 + # check update data + run_sql_tidb_with_retry "select count(1) from ${db}.${tb} where c1=1 and c3='111';" "count(1): 1" + check_log_contain_with_retry '\[UpdateWhereColumnsCheck\] \[Columns="\[c2\]"\]' $WORK_DIR/worker1/log/dm-worker.log + # check delete data + run_sql_tidb_with_retry "select count(1) from ${db}.${tb} where c1=2;" "count(1): 1" + check_log_contain_with_retry '\[DeleteWhereColumnsCheck\] \[Columns="\[c2\]"\]' $WORK_DIR/worker1/log/dm-worker.log + + # alter schema to test pk + run_sql_tidb "alter table ${db}.${tb} add primary key(c3);" + run_sql_source1 "alter table ${db1}.${tb1} drop column c2;" + run_sql_source2 "alter table ${db2}.${tb2} drop column c2;" + + # db2 increment data with update and delete + run_sql_file $cur/data/db2.increment.sql $MYSQL_HOST2 $MYSQL_PORT2 $MYSQL_PASSWORD2 + # check update data + run_sql_tidb_with_retry "select count(1) from ${db}.${tb} where c1=3 and c3='333';" "count(1): 1" + check_log_contain_with_retry '\[UpdateWhereColumnsCheck\] \[Columns="\[c3\]"\]' $WORK_DIR/worker2/log/dm-worker.log + # check delete data + run_sql_tidb_with_retry "select count(1) from ${db}.${tb} where c1=1;" "count(1): 1" + check_log_contain_with_retry '\[DeleteWhereColumnsCheck\] \[Columns="\[c3\]"\]' $WORK_DIR/worker2/log/dm-worker.log +} + +cleanup_data downstream_diff_index +# also cleanup dm processes in case of last run failed +cleanup_process $* +run $* +cleanup_process $* +export GO_FAILPOINTS='' + +echo "[$(date)] <<<<<< test case $TEST_NAME success! >>>>>>" diff --git a/dm/tests/others_integration_1.txt b/dm/tests/others_integration_1.txt index d5361c58d4f..374b8bdeff2 100644 --- a/dm/tests/others_integration_1.txt +++ b/dm/tests/others_integration_1.txt @@ -8,3 +8,5 @@ only_dml adjust_gtid checkpoint_transaction lightning_mode +downstream_diff_index + From 167d9c5619ba8a2c53447aacce7b9d6b17724af4 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Tue, 21 Dec 2021 14:43:47 +0800 Subject: [PATCH 19/48] dep(dm): update go-mysql (#3914) (#3934) --- dm/tests/all_mode/run.sh | 3 +++ go.mod | 2 +- go.sum | 4 ++-- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/dm/tests/all_mode/run.sh b/dm/tests/all_mode/run.sh index 8618c6700d6..4eba97d890f 100755 --- a/dm/tests/all_mode/run.sh +++ b/dm/tests/all_mode/run.sh @@ -380,6 +380,9 @@ function run() { # use sync_diff_inspector to check full dump loader check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + run_sql_source1 "SHOW SLAVE HOSTS;" + check_contains 'Slave_UUID' + run_sql_tidb "set time_zone = '+04:00';SELECT count(*) from all_mode.no_diff where dt = ts;" check_contains "count(*): 3" diff --git a/go.mod b/go.mod index 0553961b3a7..77977624bb0 100644 --- a/go.mod +++ b/go.mod @@ -25,7 +25,7 @@ require ( github.com/frankban/quicktest v1.11.1 // indirect github.com/getkin/kin-openapi v0.79.0 github.com/gin-gonic/gin v1.7.0 - github.com/go-mysql-org/go-mysql v1.1.3-0.20210705101833-83965e516929 + github.com/go-mysql-org/go-mysql v1.4.1-0.20211217061939-06f932768788 github.com/go-sql-driver/mysql v1.6.0 github.com/gogo/gateway v1.1.0 github.com/gogo/protobuf v1.3.2 diff --git a/go.sum b/go.sum index e333ecd701f..d9448e91b09 100644 --- a/go.sum +++ b/go.sum @@ -314,8 +314,8 @@ github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9 github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= github.com/go-logr/logr v0.1.0/go.mod h1:ixOQHD9gLJUVQQ2ZOR7zLEifBX6tGkNJF4QyIY7sIas= github.com/go-martini/martini v0.0.0-20170121215854-22fa46961aab/go.mod h1:/P9AEU963A2AYjv4d1V5eVL1CQbEJq6aCNHDDjibzu8= -github.com/go-mysql-org/go-mysql v1.1.3-0.20210705101833-83965e516929 h1:cjv3hcFlmma66+fYTvhHt/sbwZWWJs09iv2ipVRIr0I= -github.com/go-mysql-org/go-mysql v1.1.3-0.20210705101833-83965e516929/go.mod h1:3lFZKf7l95Qo70+3XB2WpiSf9wu2s3na3geLMaIIrqQ= +github.com/go-mysql-org/go-mysql v1.4.1-0.20211217061939-06f932768788 h1:0IsP4ViNmA7ZElbCE4/lINdTppdw3jdcAiJaPDyeHx8= +github.com/go-mysql-org/go-mysql v1.4.1-0.20211217061939-06f932768788/go.mod h1:3lFZKf7l95Qo70+3XB2WpiSf9wu2s3na3geLMaIIrqQ= github.com/go-ole/go-ole v1.2.4 h1:nNBDSCOigTSiarFpYE9J/KtEA1IOW4CNeqT9TQDqCxI= github.com/go-ole/go-ole v1.2.4/go.mod h1:XCwSNxSkXRo4vlyPy93sltvi/qJq0jqQhjqQNIwKuxM= github.com/go-openapi/jsonpointer v0.17.0/go.mod h1:cOnomiV+CVVwFLk0A/MExoFMjwdsUdVpsRhURCKh+3M= From d99b5a6146e3a51b37fcb53bd2fb90f6127e4ad0 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Tue, 21 Dec 2021 17:39:47 +0800 Subject: [PATCH 20/48] dm/syncer: multiple rows use downstream schema (#3308) (#3953) --- dm/pkg/schema/tracker.go | 73 +++++++++--------- dm/pkg/schema/tracker_test.go | 135 ++++++++++++++++------------------ dm/syncer/causality_test.go | 9 ++- dm/syncer/compactor_test.go | 51 +++++++------ dm/syncer/dml.go | 117 +++++++++++++++-------------- dm/syncer/dml_test.go | 107 +++++++++++++++------------ dm/syncer/job_test.go | 2 +- dm/syncer/syncer.go | 2 +- dm/syncer/syncer_test.go | 3 + 9 files changed, 264 insertions(+), 235 deletions(-) diff --git a/dm/pkg/schema/tracker.go b/dm/pkg/schema/tracker.go index a5e7a9f45c3..941ad3512fc 100644 --- a/dm/pkg/schema/tracker.go +++ b/dm/pkg/schema/tracker.go @@ -72,14 +72,14 @@ type Tracker struct { type downstreamTracker struct { downstreamConn *dbconn.DBConn // downstream connection stmtParser *parser.Parser // statement parser - tableInfos map[string]*downstreamTableInfo // downstream table infos + tableInfos map[string]*DownstreamTableInfo // downstream table infos } -// downstreamTableInfo contains tableinfo and index cache. -type downstreamTableInfo struct { - tableInfo *model.TableInfo // tableInfo which comes from parse create statement syntaxtree - indexCache *model.IndexInfo // index cache include pk/uk(not null) - availableUKCache []*model.IndexInfo // index cache include uks(data not null) +// DownstreamTableInfo contains tableinfo and index cache. +type DownstreamTableInfo struct { + TableInfo *model.TableInfo // tableInfo which comes from parse create statement syntaxtree + AbsoluteUKIndexInfo *model.IndexInfo // absolute uk index is a pk/uk(not null) + AvailableUKIndexList []*model.IndexInfo // index list which is all uks } // NewTracker creates a new tracker. `sessionCfg` will be set as tracker's session variables if specified, or retrieve @@ -181,7 +181,7 @@ func NewTracker(ctx context.Context, task string, sessionCfg map[string]string, // init downstreamTracker dsTracker := &downstreamTracker{ downstreamConn: downstreamConn, - tableInfos: make(map[string]*downstreamTableInfo), + tableInfos: make(map[string]*DownstreamTableInfo), } return &Tracker{ @@ -375,9 +375,9 @@ func (tr *Tracker) GetSystemVar(name string) (string, bool) { return tr.se.GetSessionVars().GetSystemVar(name) } -// GetDownStreamIndexInfo gets downstream PK/UK(not null) Index. +// GetDownStreamTableInfo gets downstream table info. // note. this function will init downstreamTrack's table info. -func (tr *Tracker) GetDownStreamIndexInfo(tctx *tcontext.Context, tableID string, originTi *model.TableInfo) (*model.IndexInfo, error) { +func (tr *Tracker) GetDownStreamTableInfo(tctx *tcontext.Context, tableID string, originTi *model.TableInfo) (*DownstreamTableInfo, error) { dti, ok := tr.dsTracker.tableInfos[tableID] if !ok { tctx.Logger.Info("Downstream schema tracker init. ", zap.String("tableID", tableID)) @@ -387,10 +387,10 @@ func (tr *Tracker) GetDownStreamIndexInfo(tctx *tcontext.Context, tableID string return nil, err } - dti = getDownStreamTi(ti, originTi) + dti = GetDownStreamTi(ti, originTi) tr.dsTracker.tableInfos[tableID] = dti } - return dti.indexCache, nil + return dti, nil } // GetAvailableDownStreamUKIndexInfo gets available downstream UK whose data is not null. @@ -398,7 +398,7 @@ func (tr *Tracker) GetDownStreamIndexInfo(tctx *tcontext.Context, tableID string func (tr *Tracker) GetAvailableDownStreamUKIndexInfo(tableID string, data []interface{}) *model.IndexInfo { dti, ok := tr.dsTracker.tableInfos[tableID] - if !ok || len(dti.availableUKCache) == 0 { + if !ok || len(dti.AvailableUKIndexList) == 0 { return nil } // func for check data is not null @@ -406,13 +406,9 @@ func (tr *Tracker) GetAvailableDownStreamUKIndexInfo(tableID string, data []inte return data[i] != nil } - for i, uk := range dti.availableUKCache { + for _, uk := range dti.AvailableUKIndexList { // check uk's column data is not null if isSpecifiedIndexColumn(uk, fn) { - if i != 0 { - // exchange available uk to the first of the array to reduce judgements for next row - dti.availableUKCache[0], dti.availableUKCache[i] = dti.availableUKCache[i], dti.availableUKCache[0] - } return uk } } @@ -487,12 +483,13 @@ func (tr *Tracker) initDownStreamSQLModeAndParser(tctx *tcontext.Context) error return nil } -// getDownStreamTi constructs downstreamTable index cache by tableinfo. -func getDownStreamTi(ti *model.TableInfo, originTi *model.TableInfo) *downstreamTableInfo { +// GetDownStreamTi constructs downstreamTable index cache by tableinfo. +func GetDownStreamTi(ti *model.TableInfo, originTi *model.TableInfo) *DownstreamTableInfo { var ( - indexCache *model.IndexInfo - availableUKCache = make([]*model.IndexInfo, 0, len(ti.Indices)) - hasPk = false + absoluteUKIndexInfo *model.IndexInfo + availableUKIndexList = []*model.IndexInfo{} + hasPk = false + absoluteUKPosition = -1 ) // func for check not null constraint @@ -500,7 +497,7 @@ func getDownStreamTi(ti *model.TableInfo, originTi *model.TableInfo) *downstream return mysql.HasNotNullFlag(ti.Columns[i].Flag) } - for _, idx := range ti.Indices { + for i, idx := range ti.Indices { if !idx.Primary && !idx.Unique { continue } @@ -508,16 +505,15 @@ func getDownStreamTi(ti *model.TableInfo, originTi *model.TableInfo) *downstream if indexRedirect == nil { continue } + availableUKIndexList = append(availableUKIndexList, indexRedirect) if idx.Primary { - indexCache = indexRedirect + absoluteUKIndexInfo = indexRedirect + absoluteUKPosition = i hasPk = true - } else if idx.Unique { + } else if absoluteUKIndexInfo == nil && isSpecifiedIndexColumn(idx, fn) { // second check not null unique key - if !hasPk && isSpecifiedIndexColumn(idx, fn) { - indexCache = indexRedirect - } else { - availableUKCache = append(availableUKCache, indexRedirect) - } + absoluteUKIndexInfo = indexRedirect + absoluteUKPosition = i } } @@ -526,14 +522,21 @@ func getDownStreamTi(ti *model.TableInfo, originTi *model.TableInfo) *downstream if !hasPk { exPk := redirectIndexKeys(handlePkExCase(ti), originTi) if exPk != nil { - indexCache = exPk + absoluteUKIndexInfo = exPk + absoluteUKPosition = len(availableUKIndexList) + availableUKIndexList = append(availableUKIndexList, absoluteUKIndexInfo) } } - return &downstreamTableInfo{ - tableInfo: ti, - indexCache: indexCache, - availableUKCache: availableUKCache, + // move absoluteUKIndexInfo to the first in availableUKIndexList + if absoluteUKPosition != -1 && len(availableUKIndexList) > 1 { + availableUKIndexList[0], availableUKIndexList[absoluteUKPosition] = availableUKIndexList[absoluteUKPosition], availableUKIndexList[0] + } + + return &DownstreamTableInfo{ + TableInfo: ti, + AbsoluteUKIndexInfo: absoluteUKIndexInfo, + AvailableUKIndexList: availableUKIndexList, } } diff --git a/dm/pkg/schema/tracker_test.go b/dm/pkg/schema/tracker_test.go index 0004bd04983..5e6d9f12e97 100644 --- a/dm/pkg/schema/tracker_test.go +++ b/dm/pkg/schema/tracker_test.go @@ -603,145 +603,140 @@ func (s *trackerSuite) TestGetDownStreamIndexInfo(c *C) { mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int, b int, c varchar(10))")) - indexinfo, err := tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + dti, err := tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) - _, ok := tracker.dsTracker.tableInfos[tableID] - c.Assert(ok, IsTrue) - c.Assert(indexinfo, IsNil) + c.Assert(dti, NotNil) + c.Assert(dti.AbsoluteUKIndexInfo, IsNil) delete(tracker.dsTracker.tableInfos, tableID) // downstream has pk(not constraints like "create table t(a int primary key,b int not null)" mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int, b int, c varchar(10), PRIMARY KEY (c))")) - indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + dti, err = tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) - _, ok = tracker.dsTracker.tableInfos[tableID] - c.Assert(ok, IsTrue) - c.Assert(indexinfo, NotNil) + c.Assert(dti, NotNil) + c.Assert(dti.AbsoluteUKIndexInfo, NotNil) delete(tracker.dsTracker.tableInfos, tableID) mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int primary key, b int, c varchar(10))")) - indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + dti, err = tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) - _, ok = tracker.dsTracker.tableInfos[tableID] - c.Assert(ok, IsTrue) - c.Assert(indexinfo, NotNil) + c.Assert(dti, NotNil) + c.Assert(dti.AbsoluteUKIndexInfo, NotNil) delete(tracker.dsTracker.tableInfos, tableID) // downstream has composite pks mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int, b int, c varchar(10), PRIMARY KEY (a,b))")) - indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + dti, err = tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) - _, ok = tracker.dsTracker.tableInfos[tableID] - c.Assert(ok, IsTrue) - c.Assert(len(indexinfo.Columns) == 2, IsTrue) + c.Assert(dti, NotNil) + c.Assert(dti.AbsoluteUKIndexInfo, NotNil) + c.Assert(len(dti.AbsoluteUKIndexInfo.Columns) == 2, IsTrue) delete(tracker.dsTracker.tableInfos, tableID) // downstream has uk(not null) mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int unique not null, b int, c varchar(10))")) - indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + dti, err = tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) - _, ok = tracker.dsTracker.tableInfos[tableID] - c.Assert(ok, IsTrue) - c.Assert(indexinfo.Columns, NotNil) + c.Assert(dti, NotNil) + c.Assert(dti.AbsoluteUKIndexInfo, NotNil) + c.Assert(dti.AbsoluteUKIndexInfo.Columns, NotNil) delete(tracker.dsTracker.tableInfos, tableID) // downstream has uk(without not null) mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int unique, b int, c varchar(10))")) - indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + dti, err = tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) - dti, ok := tracker.dsTracker.tableInfos[tableID] - c.Assert(ok, IsTrue) - c.Assert(indexinfo, IsNil) - c.Assert(dti.availableUKCache, NotNil) + c.Assert(dti, NotNil) + c.Assert(dti.AbsoluteUKIndexInfo, IsNil) + c.Assert(dti.AvailableUKIndexList, NotNil) delete(tracker.dsTracker.tableInfos, tableID) // downstream has uks mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int unique, b int unique, c varchar(10) unique not null)")) - indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + dti, err = tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) - dti, ok = tracker.dsTracker.tableInfos[tableID] - c.Assert(ok, IsTrue) - c.Assert(indexinfo, NotNil) - c.Assert(len(dti.availableUKCache) == 2, IsTrue) + c.Assert(dti, NotNil) + c.Assert(dti.AbsoluteUKIndexInfo, NotNil) + c.Assert(len(dti.AvailableUKIndexList) == 3, IsTrue) + c.Assert(dti.AvailableUKIndexList[0] == dti.AbsoluteUKIndexInfo, IsTrue) delete(tracker.dsTracker.tableInfos, tableID) // downstream has pk and uk, pk has priority mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int unique not null , b int, c varchar(10), PRIMARY KEY (c))")) - indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + dti, err = tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) - c.Assert(indexinfo.Primary, IsTrue) + c.Assert(dti.AbsoluteUKIndexInfo.Primary, IsTrue) + c.Assert(len(dti.AvailableUKIndexList) == 2, IsTrue) + c.Assert(dti.AvailableUKIndexList[0] == dti.AbsoluteUKIndexInfo, IsTrue) delete(tracker.dsTracker.tableInfos, tableID) // downstream has more columns than upstream, and that column in used in PK mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int , d int PRIMARY KEY, c varchar(10), b int unique not null)")) - indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + dti, err = tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) - c.Assert(indexinfo, NotNil) - c.Assert(indexinfo.Primary, IsFalse) + c.Assert(dti.AbsoluteUKIndexInfo, NotNil) + c.Assert(dti.AbsoluteUKIndexInfo.Primary, IsFalse) + c.Assert(len(dti.AvailableUKIndexList) == 1, IsTrue) delete(tracker.dsTracker.tableInfos, tableID) mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int , d int PRIMARY KEY, c varchar(10), b int unique)")) - indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + dti, err = tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) - c.Assert(indexinfo, IsNil) - dti, ok = tracker.dsTracker.tableInfos[tableID] - c.Assert(ok, IsTrue) - c.Assert(len(dti.availableUKCache) == 1, IsTrue) + c.Assert(dti.AbsoluteUKIndexInfo, IsNil) + c.Assert(len(dti.AvailableUKIndexList) == 1, IsTrue) delete(tracker.dsTracker.tableInfos, tableID) mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int , d int PRIMARY KEY, c varchar(10), b int)")) - indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + dti, err = tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) - c.Assert(indexinfo, IsNil) + c.Assert(dti.AbsoluteUKIndexInfo, IsNil) delete(tracker.dsTracker.tableInfos, tableID) // downstream has more columns than upstream, and that column in used in UK(not null) mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int , d int unique not null, c varchar(10), b int unique not null)")) - indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + dti, err = tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) - c.Assert(indexinfo, NotNil) - c.Assert(indexinfo.Columns[0].Name.L == "b", IsTrue) + c.Assert(dti.AbsoluteUKIndexInfo, NotNil) + c.Assert(dti.AbsoluteUKIndexInfo.Columns[0].Name.L == "b", IsTrue) delete(tracker.dsTracker.tableInfos, tableID) mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int , d int unique not null, c varchar(10), b int unique)")) - indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + dti, err = tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) - c.Assert(indexinfo, IsNil) - dti, ok = tracker.dsTracker.tableInfos[tableID] - c.Assert(ok, IsTrue) - c.Assert(len(dti.availableUKCache) == 1, IsTrue) + c.Assert(dti.AbsoluteUKIndexInfo, IsNil) + c.Assert(len(dti.AvailableUKIndexList) == 1, IsTrue) delete(tracker.dsTracker.tableInfos, tableID) mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int , d int unique not null, c varchar(10), b int)")) - indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + dti, err = tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) - c.Assert(indexinfo, IsNil) + c.Assert(dti.AbsoluteUKIndexInfo, IsNil) delete(tracker.dsTracker.tableInfos, tableID) } @@ -777,11 +772,11 @@ func (s *trackerSuite) TestGetAvailableDownStreanUKIndexInfo(c *C) { mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int, b int, c varchar(10))")) - indexinfo, err := tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + dti, err := tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) - c.Assert(indexinfo, IsNil) + c.Assert(dti.AbsoluteUKIndexInfo, IsNil) data := []interface{}{1, 2, 3} - indexinfo = tracker.GetAvailableDownStreamUKIndexInfo(tableID, data) + indexinfo := tracker.GetAvailableDownStreamUKIndexInfo(tableID, data) c.Assert(indexinfo, IsNil) delete(tracker.dsTracker.tableInfos, tableID) @@ -789,9 +784,9 @@ func (s *trackerSuite) TestGetAvailableDownStreanUKIndexInfo(c *C) { mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int unique, b int, c varchar(10))")) - indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + dti, err = tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) - c.Assert(indexinfo, IsNil) + c.Assert(dti.AbsoluteUKIndexInfo, IsNil) data = []interface{}{nil, 2, 3} indexinfo = tracker.GetAvailableDownStreamUKIndexInfo(tableID, data) c.Assert(indexinfo, IsNil) @@ -801,9 +796,9 @@ func (s *trackerSuite) TestGetAvailableDownStreanUKIndexInfo(c *C) { mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int unique, b int, c varchar(10))")) - indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + dti, err = tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) - c.Assert(indexinfo, IsNil) + c.Assert(dti.AbsoluteUKIndexInfo, IsNil) data = []interface{}{1, 2, 3} indexinfo = tracker.GetAvailableDownStreamUKIndexInfo(tableID, data) c.Assert(indexinfo, NotNil) @@ -813,9 +808,9 @@ func (s *trackerSuite) TestGetAvailableDownStreanUKIndexInfo(c *C) { mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int, b int, c varchar(10), unique key(a, b))")) - indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + dti, err = tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) - c.Assert(indexinfo, IsNil) + c.Assert(dti.AbsoluteUKIndexInfo, IsNil) data = []interface{}{1, nil, 3} indexinfo = tracker.GetAvailableDownStreamUKIndexInfo(tableID, data) c.Assert(indexinfo, IsNil) @@ -824,9 +819,9 @@ func (s *trackerSuite) TestGetAvailableDownStreanUKIndexInfo(c *C) { mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int, b int, c varchar(10), unique key(a, b))")) - indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + dti, err = tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) - c.Assert(indexinfo, IsNil) + c.Assert(dti.AbsoluteUKIndexInfo, IsNil) data = []interface{}{1, nil, nil} indexinfo = tracker.GetAvailableDownStreamUKIndexInfo(tableID, data) c.Assert(indexinfo, IsNil) @@ -836,9 +831,9 @@ func (s *trackerSuite) TestGetAvailableDownStreanUKIndexInfo(c *C) { mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int, b int, c varchar(10), unique key(a, b))")) - indexinfo, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + dti, err = tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) - c.Assert(indexinfo, IsNil) + c.Assert(dti.AbsoluteUKIndexInfo, IsNil) data = []interface{}{1, 2, 3} indexinfo = tracker.GetAvailableDownStreamUKIndexInfo(tableID, data) c.Assert(indexinfo, NotNil) @@ -876,7 +871,7 @@ func (s *trackerSuite) TestReTrackDownStreamIndex(c *C) { mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int, b int, c varchar(10), PRIMARY KEY (a,b))")) - _, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + _, err = tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) _, ok := tracker.dsTracker.tableInfos[tableID] c.Assert(ok, IsTrue) @@ -890,7 +885,7 @@ func (s *trackerSuite) TestReTrackDownStreamIndex(c *C) { mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int, b int, c varchar(10), PRIMARY KEY (a,b))")) - _, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + _, err = tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) _, ok = tracker.dsTracker.tableInfos[tableID] c.Assert(ok, IsTrue) @@ -902,7 +897,7 @@ func (s *trackerSuite) TestReTrackDownStreamIndex(c *C) { mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int primary key, b int, c varchar(10))")) - _, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + _, err = tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) _, ok = tracker.dsTracker.tableInfos[tableID] c.Assert(ok, IsTrue) @@ -916,7 +911,7 @@ func (s *trackerSuite) TestReTrackDownStreamIndex(c *C) { mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int, b int, c varchar(10), PRIMARY KEY (a,b))")) - _, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + _, err = tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) _, ok = tracker.dsTracker.tableInfos[tableID] c.Assert(ok, IsTrue) @@ -928,7 +923,7 @@ func (s *trackerSuite) TestReTrackDownStreamIndex(c *C) { mock.ExpectQuery("SHOW CREATE TABLE " + tableID).WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("test", "create table t(a int primary key, b int, c varchar(10))")) - _, err = tracker.GetDownStreamIndexInfo(tcontext.Background(), tableID, oriTi) + _, err = tracker.GetDownStreamTableInfo(tcontext.Background(), tableID, oriTi) c.Assert(err, IsNil) _, ok = tracker.dsTracker.tableInfos[tableID] c.Assert(ok, IsTrue) diff --git a/dm/syncer/causality_test.go b/dm/syncer/causality_test.go index be775bf9d26..db07e204e21 100644 --- a/dm/syncer/causality_test.go +++ b/dm/syncer/causality_test.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tiflow/dm/pkg/binlog" tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/schema" "github.com/pingcap/tiflow/dm/pkg/utils" ) @@ -56,8 +57,8 @@ func (s *testSyncerSuite) TestDetectConflict(c *C) { func (s *testSyncerSuite) TestCasuality(c *C) { p := parser.New() se := mock.NewContext() - schema := "create table tb(a int primary key, b int unique);" - ti, err := createTableInfo(p, se, int64(0), schema) + schemaStr := "create table tb(a int primary key, b int unique);" + ti, err := createTableInfo(p, se, int64(0), schemaStr) c.Assert(err, IsNil) tiIndex := &model.IndexInfo{ Table: ti.Name, @@ -71,6 +72,8 @@ func (s *testSyncerSuite) TestCasuality(c *C) { Length: types.UnspecifiedLength, }}, } + downTi := schema.GetDownStreamTi(ti, ti) + c.Assert(downTi, NotNil) jobCh := make(chan *job, 10) syncer := &Syncer{ @@ -117,7 +120,7 @@ func (s *testSyncerSuite) TestCasuality(c *C) { ec := &eventContext{startLocation: &location, currentLocation: &location, lastLocation: &location} for _, tc := range testCases { - job := newDMLJob(tc.op, table, table, newDML(tc.op, false, "", table, tc.oldVals, tc.vals, tc.oldVals, tc.vals, ti.Columns, ti, tiIndex), ec) + job := newDMLJob(tc.op, table, table, newDML(tc.op, false, "", table, tc.oldVals, tc.vals, tc.oldVals, tc.vals, ti.Columns, ti, tiIndex, downTi), ec) jobCh <- job } diff --git a/dm/syncer/compactor_test.go b/dm/syncer/compactor_test.go index 901bd40ffd8..19d374257a3 100644 --- a/dm/syncer/compactor_test.go +++ b/dm/syncer/compactor_test.go @@ -30,6 +30,7 @@ import ( "github.com/pingcap/tiflow/dm/pkg/binlog" tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/schema" "github.com/pingcap/tiflow/dm/pkg/utils" ) @@ -75,8 +76,8 @@ func (s *testSyncerSuite) TestCompactJob(c *C) { targetTableID := "`test`.`tb`" sourceTable := &filter.Table{Schema: "test", Name: "tb1"} targetTable := &filter.Table{Schema: "test", Name: "tb"} - schema := "create table test.tb(id int primary key, col1 int, name varchar(24))" - ti, err := createTableInfo(p, se, 0, schema) + schemaStr := "create table test.tb(id int primary key, col1 int, name varchar(24))" + ti, err := createTableInfo(p, se, 0, schemaStr) c.Assert(err, IsNil) tiIndex := &model.IndexInfo{ Table: ti.Name, @@ -90,6 +91,8 @@ func (s *testSyncerSuite) TestCompactJob(c *C) { Length: types.UnspecifiedLength, }}, } + downTi := schema.GetDownStreamTi(ti, ti) + c.Assert(downTi, NotNil) var dml *DML var dmls []*DML @@ -108,7 +111,7 @@ func (s *testSyncerSuite) TestCompactJob(c *C) { oldValues, ok := kv[newID] if !ok { // insert - dml = newDML(insert, false, targetTableID, sourceTable, nil, values, nil, values, ti.Columns, ti, tiIndex) + dml = newDML(insert, false, targetTableID, sourceTable, nil, values, nil, values, ti.Columns, ti, tiIndex, downTi) } else { if rand.Int()%2 > 0 { // update @@ -122,10 +125,10 @@ func (s *testSyncerSuite) TestCompactJob(c *C) { } } } - dml = newDML(update, false, targetTableID, sourceTable, oldValues, values, oldValues, values, ti.Columns, ti, tiIndex) + dml = newDML(update, false, targetTableID, sourceTable, oldValues, values, oldValues, values, ti.Columns, ti, tiIndex, downTi) } else { // delete - dml = newDML(del, false, targetTableID, sourceTable, nil, oldValues, nil, oldValues, ti.Columns, ti, tiIndex) + dml = newDML(del, false, targetTableID, sourceTable, nil, oldValues, nil, oldValues, ti.Columns, ti, tiIndex, downTi) } } @@ -190,8 +193,8 @@ func (s *testSyncerSuite) TestCompactorSafeMode(c *C) { targetTableID := "`test`.`tb`" sourceTable := &filter.Table{Schema: "test", Name: "tb1"} targetTable := &filter.Table{Schema: "test", Name: "tb"} - schema := "create table test.tb(id int primary key, col1 int, name varchar(24))" - ti, err := createTableInfo(p, se, 0, schema) + schemaStr := "create table test.tb(id int primary key, col1 int, name varchar(24))" + ti, err := createTableInfo(p, se, 0, schemaStr) c.Assert(err, IsNil) tiIndex := &model.IndexInfo{ Table: ti.Name, @@ -205,6 +208,8 @@ func (s *testSyncerSuite) TestCompactorSafeMode(c *C) { Length: types.UnspecifiedLength, }}, } + downTi := schema.GetDownStreamTi(ti, ti) + c.Assert(downTi, NotNil) testCases := []struct { input []*DML @@ -213,31 +218,31 @@ func (s *testSyncerSuite) TestCompactorSafeMode(c *C) { // nolint:dupl { input: []*DML{ - newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex), - newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex), - newDML(update, true, targetTableID, sourceTable, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, ti.Columns, ti, tiIndex), - newDML(del, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex), - newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex), + newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex, downTi), + newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex, downTi), + newDML(update, true, targetTableID, sourceTable, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, ti.Columns, ti, tiIndex, downTi), + newDML(del, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex, downTi), + newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex, downTi), }, output: []*DML{ - newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti.Columns, ti, tiIndex), - newDML(del, true, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex), - newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex), + newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti.Columns, ti, tiIndex, downTi), + newDML(del, true, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex, downTi), + newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex, downTi), }, }, // nolint:dupl { input: []*DML{ - newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex), - newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex), - newDML(update, false, targetTableID, sourceTable, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, ti.Columns, ti, tiIndex), - newDML(del, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex), - newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex), + newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex, downTi), + newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex, downTi), + newDML(update, false, targetTableID, sourceTable, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, ti.Columns, ti, tiIndex, downTi), + newDML(del, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex, downTi), + newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex, downTi), }, output: []*DML{ - newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti.Columns, ti, tiIndex), - newDML(del, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex), - newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex), + newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti.Columns, ti, tiIndex, downTi), + newDML(del, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex, downTi), + newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex, downTi), }, }, } diff --git a/dm/syncer/dml.go b/dm/syncer/dml.go index 539755c6d9d..d4a04245ed7 100644 --- a/dm/syncer/dml.go +++ b/dm/syncer/dml.go @@ -30,6 +30,7 @@ import ( tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/log" + "github.com/pingcap/tiflow/dm/pkg/schema" "github.com/pingcap/tiflow/dm/pkg/terror" ) @@ -76,8 +77,9 @@ func extractValueFromData(data []interface{}, columns []*model.ColumnInfo) []int return value } -func (s *Syncer) genAndFilterInsertDMLs(param *genDMLParam, filterExprs []expression.Expression) ([]*DML, error) { +func (s *Syncer) genAndFilterInsertDMLs(tctx *tcontext.Context, param *genDMLParam, filterExprs []expression.Expression) ([]*DML, error) { var ( + tableID = param.targetTableID dataSeq = param.data originalDataSeq = param.originalData columns = param.columns @@ -85,6 +87,13 @@ func (s *Syncer) genAndFilterInsertDMLs(param *genDMLParam, filterExprs []expres dmls = make([]*DML, 0, len(dataSeq)) ) + // if downstream pk/uk(not null) exits, then use downstream pk/uk(not null) + downstreamTableInfo, err := s.schemaTracker.GetDownStreamTableInfo(tctx, tableID, ti) + if err != nil { + return nil, err + } + downstreamIndexColumns := downstreamTableInfo.AbsoluteUKIndexInfo + RowLoop: for dataIdx, data := range dataSeq { if len(data) != len(columns) { @@ -108,7 +117,11 @@ RowLoop: } } - dmls = append(dmls, newDML(insert, param.safeMode, param.targetTableID, param.sourceTable, nil, value, nil, originalValue, columns, ti, nil)) + if downstreamIndexColumns == nil { + downstreamIndexColumns = s.schemaTracker.GetAvailableDownStreamUKIndexInfo(tableID, value) + } + + dmls = append(dmls, newDML(insert, param.safeMode, tableID, param.sourceTable, nil, value, nil, originalValue, columns, ti, downstreamIndexColumns, downstreamTableInfo)) } return dmls, nil @@ -130,10 +143,11 @@ func (s *Syncer) genAndFilterUpdateDMLs( ) // if downstream pk/uk(not null) exits, then use downstream pk/uk(not null) - downstreamIndexColumns, err := s.schemaTracker.GetDownStreamIndexInfo(tctx, tableID, ti) + downstreamTableInfo, err := s.schemaTracker.GetDownStreamTableInfo(tctx, tableID, ti) if err != nil { return nil, err } + downstreamIndexColumns := downstreamTableInfo.AbsoluteUKIndexInfo RowLoop: for i := 0; i < len(data); i += 2 { @@ -184,7 +198,7 @@ RowLoop: downstreamIndexColumns = s.schemaTracker.GetAvailableDownStreamUKIndexInfo(tableID, oriOldValues) } - dmls = append(dmls, newDML(update, param.safeMode, param.targetTableID, param.sourceTable, oldValues, changedValues, oriOldValues, oriChangedValues, columns, ti, downstreamIndexColumns)) + dmls = append(dmls, newDML(update, param.safeMode, param.targetTableID, param.sourceTable, oldValues, changedValues, oriOldValues, oriChangedValues, columns, ti, downstreamIndexColumns, downstreamTableInfo)) } return dmls, nil @@ -199,10 +213,11 @@ func (s *Syncer) genAndFilterDeleteDMLs(tctx *tcontext.Context, param *genDMLPar ) // if downstream pk/uk(not null) exits, then use downstream pk/uk(not null) - downstreamIndexColumns, err := s.schemaTracker.GetDownStreamIndexInfo(tctx, tableID, ti) + downstreamTableInfo, err := s.schemaTracker.GetDownStreamTableInfo(tctx, tableID, ti) if err != nil { return nil, err } + downstreamIndexColumns := downstreamTableInfo.AbsoluteUKIndexInfo RowLoop: for _, data := range dataSeq { @@ -227,7 +242,7 @@ RowLoop: downstreamIndexColumns = s.schemaTracker.GetAvailableDownStreamUKIndexInfo(tableID, value) } - dmls = append(dmls, newDML(del, false, param.targetTableID, param.sourceTable, nil, value, nil, value, ti.Columns, ti, downstreamIndexColumns)) + dmls = append(dmls, newDML(del, false, param.targetTableID, param.sourceTable, nil, value, nil, value, ti.Columns, ti, downstreamIndexColumns, downstreamTableInfo)) } return dmls, nil @@ -451,34 +466,36 @@ func checkLogColumns(skipped [][]int) error { // DML stores param for DML. type DML struct { - targetTableID string - sourceTable *filter.Table - op opType - oldValues []interface{} // only for update SQL - values []interface{} - columns []*model.ColumnInfo - sourceTableInfo *model.TableInfo - originOldValues []interface{} // only for update SQL - originValues []interface{} // use to gen key and `WHERE` - safeMode bool - key string // use to detect causality - downstreamIndexInfo *model.IndexInfo + targetTableID string + sourceTable *filter.Table + op opType + oldValues []interface{} // only for update SQL + values []interface{} + columns []*model.ColumnInfo + sourceTableInfo *model.TableInfo + originOldValues []interface{} // only for update SQL + originValues []interface{} // use to gen key and `WHERE` + safeMode bool + key string // use to detect causality + pickedDownstreamIndexInfo *model.IndexInfo // pick an index from downstream which comes from pk/uk not null/uk value not null and is only used in genWhere + downstreamTableInfo *schema.DownstreamTableInfo // downstream table info } // newDML creates DML. -func newDML(op opType, safeMode bool, targetTableID string, sourceTable *filter.Table, oldValues, values, originOldValues, originValues []interface{}, columns []*model.ColumnInfo, sourceTableInfo *model.TableInfo, downstreamIndexInfo *model.IndexInfo) *DML { +func newDML(op opType, safeMode bool, targetTableID string, sourceTable *filter.Table, oldValues, values, originOldValues, originValues []interface{}, columns []*model.ColumnInfo, sourceTableInfo *model.TableInfo, pickedDownstreamIndexInfo *model.IndexInfo, downstreamTableInfo *schema.DownstreamTableInfo) *DML { return &DML{ - op: op, - safeMode: safeMode, - targetTableID: targetTableID, - sourceTable: sourceTable, - oldValues: oldValues, - values: values, - columns: columns, - sourceTableInfo: sourceTableInfo, - originOldValues: originOldValues, - originValues: originValues, - downstreamIndexInfo: downstreamIndexInfo, + op: op, + safeMode: safeMode, + targetTableID: targetTableID, + sourceTable: sourceTable, + oldValues: oldValues, + values: values, + columns: columns, + sourceTableInfo: sourceTableInfo, + originOldValues: originOldValues, + originValues: originValues, + pickedDownstreamIndexInfo: pickedDownstreamIndexInfo, + downstreamTableInfo: downstreamTableInfo, } } @@ -511,7 +528,7 @@ func updateToDelAndInsert(updateDML *DML) (*DML, *DML) { // identifyColumns gets columns of unique not null index. // This is used for compact. func (dml *DML) identifyColumns() []string { - if defaultIndexColumns := findFitIndex(dml.sourceTableInfo); defaultIndexColumns != nil { + if defaultIndexColumns := dml.downstreamTableInfo.AbsoluteUKIndexInfo; defaultIndexColumns != nil { columns := make([]string, 0, len(defaultIndexColumns.Columns)) for _, column := range defaultIndexColumns.Columns { columns = append(columns, column.Name.O) @@ -524,7 +541,7 @@ func (dml *DML) identifyColumns() []string { // identifyValues gets values of unique not null index. // This is used for compact. func (dml *DML) identifyValues() []interface{} { - if defaultIndexColumns := findFitIndex(dml.sourceTableInfo); defaultIndexColumns != nil { + if defaultIndexColumns := dml.downstreamTableInfo.AbsoluteUKIndexInfo; defaultIndexColumns != nil { values := make([]interface{}, 0, len(defaultIndexColumns.Columns)) for _, column := range defaultIndexColumns.Columns { values = append(values, dml.values[column.Offset]) @@ -537,7 +554,7 @@ func (dml *DML) identifyValues() []interface{} { // oldIdentifyValues gets old values of unique not null index. // only for update SQL. func (dml *DML) oldIdentifyValues() []interface{} { - if defaultIndexColumns := findFitIndex(dml.sourceTableInfo); defaultIndexColumns != nil { + if defaultIndexColumns := dml.downstreamTableInfo.AbsoluteUKIndexInfo; defaultIndexColumns != nil { values := make([]interface{}, 0, len(defaultIndexColumns.Columns)) for _, column := range defaultIndexColumns.Columns { values = append(values, dml.oldValues[column.Offset]) @@ -583,11 +600,11 @@ func (dml *DML) identifyKeys() []string { var keys []string // for UPDATE statement if dml.originOldValues != nil { - keys = append(keys, genMultipleKeys(dml.sourceTableInfo, dml.originOldValues, dml.targetTableID)...) + keys = append(keys, genMultipleKeys(dml.downstreamTableInfo, dml.sourceTableInfo, dml.originOldValues, dml.targetTableID)...) } if dml.originValues != nil { - keys = append(keys, genMultipleKeys(dml.sourceTableInfo, dml.originValues, dml.targetTableID)...) + keys = append(keys, genMultipleKeys(dml.downstreamTableInfo, dml.sourceTableInfo, dml.originValues, dml.targetTableID)...) } return keys } @@ -610,8 +627,8 @@ func (dml *DML) whereColumnsAndValues() ([]string, []interface{}) { values = dml.originOldValues } - if dml.downstreamIndexInfo != nil { - columns, values = getColumnData(dml.sourceTableInfo.Columns, dml.downstreamIndexInfo, values) + if dml.pickedDownstreamIndexInfo != nil { + columns, values = getColumnData(dml.sourceTableInfo.Columns, dml.pickedDownstreamIndexInfo, values) } columnNames := make([]string, 0, len(columns)) @@ -668,21 +685,10 @@ func genKeyList(table string, columns []*model.ColumnInfo, dataSeq []interface{} // genMultipleKeys gens keys with UNIQUE NOT NULL value. // if not UNIQUE NOT NULL value, use table name instead. -func genMultipleKeys(ti *model.TableInfo, value []interface{}, table string) []string { - multipleKeys := make([]string, 0, len(ti.Indices)+1) - if ti.PKIsHandle { - if pk := ti.GetPkColInfo(); pk != nil { - cols := []*model.ColumnInfo{pk} - vals := []interface{}{value[pk.Offset]} - multipleKeys = append(multipleKeys, genKeyList(table, cols, vals)) - } - } +func genMultipleKeys(downstreamTableInfo *schema.DownstreamTableInfo, ti *model.TableInfo, value []interface{}, table string) []string { + multipleKeys := make([]string, 0, len(downstreamTableInfo.AvailableUKIndexList)) - for _, indexCols := range ti.Indices { - // PK also has a true Unique - if !indexCols.Unique { - continue - } + for _, indexCols := range downstreamTableInfo.AvailableUKIndexList { cols, vals := getColumnData(ti.Columns, indexCols, value) key := genKeyList(table, cols, vals) if len(key) > 0 { // ignore `null` value. @@ -933,16 +939,15 @@ func genSQLMultipleRows(op dmlOpType, dmls []*DML) (queries []string, args [][]i // sameColumns check whether two DMLs have same columns. func sameColumns(lhs *DML, rhs *DML) bool { - // if source table is same, columns will be same. - if lhs.sourceTable.Schema == rhs.sourceTable.Schema && lhs.sourceTable.Name == rhs.sourceTable.Name { - return true - } - var lhsCols, rhsCols []string if lhs.op == del { lhsCols, _ = lhs.whereColumnsAndValues() rhsCols, _ = rhs.whereColumnsAndValues() } else { + // if source table is same, columns will be same. + if lhs.sourceTable.Schema == rhs.sourceTable.Schema && lhs.sourceTable.Name == rhs.sourceTable.Name { + return true + } lhsCols = lhs.columnNames() rhsCols = rhs.columnNames() } diff --git a/dm/syncer/dml_test.go b/dm/syncer/dml_test.go index 3d892066a9e..bd4883bc634 100644 --- a/dm/syncer/dml_test.go +++ b/dm/syncer/dml_test.go @@ -18,6 +18,7 @@ import ( "strings" . "github.com/pingcap/check" + "github.com/pingcap/tiflow/dm/pkg/schema" "github.com/pingcap/tidb-tools/pkg/filter" tiddl "github.com/pingcap/tidb/ddl" @@ -171,7 +172,7 @@ func (s *testSyncerSuite) TestGenMultipleKeys(c *C) { // non-integer primary key schema: `create table t65(a int unique, b varchar(16) primary key)`, values: []interface{}{16, "xyz"}, - keys: []string{"16.a.table", "xyz.b.table"}, + keys: []string{"xyz.b.table", "16.a.table"}, }, { // primary key of multiple columns @@ -214,14 +215,16 @@ func (s *testSyncerSuite) TestGenMultipleKeys(c *C) { } for i, tc := range testCases { - schema := tc.schema + schemaStr := tc.schema assert := func(obtained interface{}, checker Checker, args ...interface{}) { - c.Assert(obtained, checker, append(args, Commentf("test case schema: %s", schema))...) + c.Assert(obtained, checker, append(args, Commentf("test case schema: %s", schemaStr))...) } ti, err := createTableInfo(p, se, int64(i+1), tc.schema) assert(err, IsNil) - keys := genMultipleKeys(ti, tc.values, "table") + dti := schema.GetDownStreamTi(ti, ti) + assert(dti, NotNil) + keys := genMultipleKeys(dti, ti, tc.values, "table") assert(keys, DeepEquals, tc.keys) } } @@ -254,22 +257,22 @@ func (s *testSyncerSuite) TestGenWhere(c *C) { values []interface{} }{ { - newDML(del, false, "", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti1.Columns, ti1, ti1Index), + newDML(del, false, "", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti1.Columns, ti1, ti1Index, nil), "`id` = ?", []interface{}{1}, }, { - newDML(update, false, "", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, ti1.Columns, ti1, ti1Index), + newDML(update, false, "", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, ti1.Columns, ti1, ti1Index, nil), "`id` = ?", []interface{}{1}, }, { - newDML(del, false, "", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti2.Columns, ti2, nil), + newDML(del, false, "", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti2.Columns, ti2, nil, nil), "`id` = ? AND `col1` = ? AND `col2` = ? AND `name` = ?", []interface{}{1, 2, 3, "haha"}, }, { - newDML(update, false, "", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, ti2.Columns, ti2, nil), + newDML(update, false, "", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, ti2.Columns, ti2, nil, nil), "`id` = ? AND `col1` = ? AND `col2` = ? AND `name` = ?", []interface{}{1, 2, 3, "haha"}, }, @@ -308,27 +311,27 @@ func (s *testSyncerSuite) TestGenSQL(c *C) { args [][]interface{} }{ { - newDML(insert, false, "`targetSchema`.`targetTable`", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex), + newDML(insert, false, "`targetSchema`.`targetTable`", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex, nil), []string{"INSERT INTO `targetSchema`.`targetTable` (`id`,`col1`,`col2`,`name`) VALUES (?,?,?,?)"}, [][]interface{}{{1, 2, 3, "haha"}}, }, { - newDML(insert, true, "`targetSchema`.`targetTable`", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex), + newDML(insert, true, "`targetSchema`.`targetTable`", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex, nil), []string{"INSERT INTO `targetSchema`.`targetTable` (`id`,`col1`,`col2`,`name`) VALUES (?,?,?,?) ON DUPLICATE KEY UPDATE `id`=VALUES(`id`),`col1`=VALUES(`col1`),`col2`=VALUES(`col2`),`name`=VALUES(`name`)"}, [][]interface{}{{1, 2, 3, "haha"}}, }, { - newDML(del, false, "`targetSchema`.`targetTable`", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex), + newDML(del, false, "`targetSchema`.`targetTable`", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex, nil), []string{"DELETE FROM `targetSchema`.`targetTable` WHERE `id` = ? LIMIT 1"}, [][]interface{}{{1}}, }, { - newDML(update, false, "`targetSchema`.`targetTable`", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex), + newDML(update, false, "`targetSchema`.`targetTable`", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex, nil), []string{"UPDATE `targetSchema`.`targetTable` SET `id` = ?, `col1` = ?, `col2` = ?, `name` = ? WHERE `id` = ? LIMIT 1"}, [][]interface{}{{4, 5, 6, "hihi", 1}}, }, { - newDML(update, true, "`targetSchema`.`targetTable`", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex), + newDML(update, true, "`targetSchema`.`targetTable`", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex, nil), []string{"DELETE FROM `targetSchema`.`targetTable` WHERE `id` = ? LIMIT 1", "INSERT INTO `targetSchema`.`targetTable` (`id`,`col1`,`col2`,`name`) VALUES (?,?,?,?) ON DUPLICATE KEY UPDATE `id`=VALUES(`id`),`col1`=VALUES(`col1`),`col2`=VALUES(`col2`),`name`=VALUES(`name`)"}, [][]interface{}{{1}, {4, 5, 6, "hihi"}}, }, @@ -375,6 +378,9 @@ func (s *testSyncerSuite) TestGenDMLWithSameOp(c *C) { Length: types.UnspecifiedLength, }}, } + downTi11 := &schema.DownstreamTableInfo{ + AbsoluteUKIndexInfo: ti11Index, + } ti12, err := createTableInfo(p, se, 0, schema12) c.Assert(err, IsNil) ti12Index := &model.IndexInfo{ @@ -389,6 +395,9 @@ func (s *testSyncerSuite) TestGenDMLWithSameOp(c *C) { Length: types.UnspecifiedLength, }}, } + downTi12 := &schema.DownstreamTableInfo{ + AbsoluteUKIndexInfo: ti12Index, + } ti21, err := createTableInfo(p, se, 0, schema21) c.Assert(err, IsNil) ti21Index := &model.IndexInfo{ @@ -403,6 +412,9 @@ func (s *testSyncerSuite) TestGenDMLWithSameOp(c *C) { Length: types.UnspecifiedLength, }}, } + downTi21 := &schema.DownstreamTableInfo{ + AbsoluteUKIndexInfo: ti21Index, + } ti22, err := createTableInfo(p, se, 0, schema22) c.Assert(err, IsNil) ti22Index := &model.IndexInfo{ @@ -417,56 +429,59 @@ func (s *testSyncerSuite) TestGenDMLWithSameOp(c *C) { Length: types.UnspecifiedLength, }}, } + downTi22 := &schema.DownstreamTableInfo{ + AbsoluteUKIndexInfo: ti22Index, + } dmls := []*DML{ // insert - newDML(insert, true, targetTableID1, sourceTable11, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti11.Columns, ti11, ti11Index), - newDML(insert, true, targetTableID1, sourceTable11, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti11.Columns, ti11, ti11Index), - newDML(insert, true, targetTableID1, sourceTable12, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti12.Columns, ti12, ti12Index), + newDML(insert, true, targetTableID1, sourceTable11, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti11.Columns, ti11, ti11Index, downTi11), + newDML(insert, true, targetTableID1, sourceTable11, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti11.Columns, ti11, ti11Index, downTi11), + newDML(insert, true, targetTableID1, sourceTable12, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti12.Columns, ti12, ti12Index, downTi12), // update no index - newDML(update, true, targetTableID1, sourceTable11, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, ti11.Columns, ti11, ti11Index), - newDML(update, true, targetTableID1, sourceTable11, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, ti11.Columns, ti11, ti11Index), - newDML(update, true, targetTableID1, sourceTable12, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, ti12.Columns, ti12, ti12Index), + newDML(update, true, targetTableID1, sourceTable11, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, ti11.Columns, ti11, ti11Index, downTi11), + newDML(update, true, targetTableID1, sourceTable11, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, ti11.Columns, ti11, ti11Index, downTi11), + newDML(update, true, targetTableID1, sourceTable12, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, ti12.Columns, ti12, ti12Index, downTi12), // update uk - newDML(update, true, targetTableID1, sourceTable11, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, ti11.Columns, ti11, ti11Index), - newDML(update, true, targetTableID1, sourceTable11, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, ti11.Columns, ti11, ti11Index), - newDML(update, true, targetTableID1, sourceTable12, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, ti12.Columns, ti12, ti12Index), + newDML(update, true, targetTableID1, sourceTable11, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, ti11.Columns, ti11, ti11Index, downTi11), + newDML(update, true, targetTableID1, sourceTable11, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, ti11.Columns, ti11, ti11Index, downTi11), + newDML(update, true, targetTableID1, sourceTable12, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, ti12.Columns, ti12, ti12Index, downTi12), // update pk - newDML(update, true, targetTableID1, sourceTable11, []interface{}{1, 4, "aa"}, []interface{}{4, 4, "aa"}, []interface{}{1, 1, "aa"}, []interface{}{4, 4, "aa"}, ti11.Columns, ti11, ti11Index), - newDML(update, true, targetTableID1, sourceTable11, []interface{}{2, 5, "bb"}, []interface{}{5, 5, "bb"}, []interface{}{2, 2, "bb"}, []interface{}{5, 5, "bb"}, ti11.Columns, ti11, ti11Index), - newDML(update, true, targetTableID1, sourceTable12, []interface{}{3, 6, "cc"}, []interface{}{6, 6, "cc"}, []interface{}{3, 3, "cc"}, []interface{}{6, 6, "cc"}, ti12.Columns, ti12, ti12Index), + newDML(update, true, targetTableID1, sourceTable11, []interface{}{1, 4, "aa"}, []interface{}{4, 4, "aa"}, []interface{}{1, 1, "aa"}, []interface{}{4, 4, "aa"}, ti11.Columns, ti11, ti11Index, downTi11), + newDML(update, true, targetTableID1, sourceTable11, []interface{}{2, 5, "bb"}, []interface{}{5, 5, "bb"}, []interface{}{2, 2, "bb"}, []interface{}{5, 5, "bb"}, ti11.Columns, ti11, ti11Index, downTi11), + newDML(update, true, targetTableID1, sourceTable12, []interface{}{3, 6, "cc"}, []interface{}{6, 6, "cc"}, []interface{}{3, 3, "cc"}, []interface{}{6, 6, "cc"}, ti12.Columns, ti12, ti12Index, downTi12), // delete - newDML(del, true, targetTableID1, sourceTable11, nil, []interface{}{4, 4, "aa"}, nil, []interface{}{4, 4, "aa"}, ti11.Columns, ti11, ti11Index), - newDML(del, true, targetTableID1, sourceTable11, nil, []interface{}{5, 5, "bb"}, nil, []interface{}{5, 5, "bb"}, ti11.Columns, ti11, ti11Index), - newDML(del, true, targetTableID1, sourceTable12, nil, []interface{}{6, 6, "cc"}, nil, []interface{}{6, 6, "cc"}, ti12.Columns, ti12, ti12Index), + newDML(del, true, targetTableID1, sourceTable11, nil, []interface{}{4, 4, "aa"}, nil, []interface{}{4, 4, "aa"}, ti11.Columns, ti11, ti11Index, downTi11), + newDML(del, true, targetTableID1, sourceTable11, nil, []interface{}{5, 5, "bb"}, nil, []interface{}{5, 5, "bb"}, ti11.Columns, ti11, ti11Index, downTi11), + newDML(del, true, targetTableID1, sourceTable12, nil, []interface{}{6, 6, "cc"}, nil, []interface{}{6, 6, "cc"}, ti12.Columns, ti12, ti12Index, downTi12), // target table 2 // insert - newDML(insert, true, targetTableID2, sourceTable21, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti21.Columns, ti21, ti21Index), - newDML(insert, false, targetTableID2, sourceTable21, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti21.Columns, ti21, ti21Index), - newDML(insert, false, targetTableID2, sourceTable22, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti22.Columns, ti22, ti22Index), + newDML(insert, true, targetTableID2, sourceTable21, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti21.Columns, ti21, ti21Index, downTi21), + newDML(insert, false, targetTableID2, sourceTable21, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti21.Columns, ti21, ti21Index, downTi21), + newDML(insert, false, targetTableID2, sourceTable22, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti22.Columns, ti22, ti22Index, downTi22), // update no index - newDML(update, false, targetTableID2, sourceTable21, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, ti21.Columns, ti21, ti21Index), - newDML(update, false, targetTableID2, sourceTable21, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, ti21.Columns, ti21, ti21Index), - newDML(update, false, targetTableID2, sourceTable22, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, ti22.Columns, ti22, ti21Index), + newDML(update, false, targetTableID2, sourceTable21, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, ti21.Columns, ti21, ti21Index, downTi21), + newDML(update, false, targetTableID2, sourceTable21, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, ti21.Columns, ti21, ti21Index, downTi21), + newDML(update, false, targetTableID2, sourceTable22, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, ti22.Columns, ti22, ti22Index, downTi22), // update uk - newDML(update, false, targetTableID2, sourceTable21, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, ti21.Columns, ti21, ti21Index), - newDML(update, false, targetTableID2, sourceTable21, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, ti21.Columns, ti21, ti21Index), - newDML(update, false, targetTableID2, sourceTable22, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, ti22.Columns, ti22, ti22Index), + newDML(update, false, targetTableID2, sourceTable21, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, []interface{}{1, 1, "aa"}, []interface{}{1, 4, "aa"}, ti21.Columns, ti21, ti21Index, downTi21), + newDML(update, false, targetTableID2, sourceTable21, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, []interface{}{2, 2, "bb"}, []interface{}{2, 5, "bb"}, ti21.Columns, ti21, ti21Index, downTi21), + newDML(update, false, targetTableID2, sourceTable22, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, []interface{}{3, 3, "cc"}, []interface{}{3, 6, "cc"}, ti22.Columns, ti22, ti22Index, downTi22), // update pk - newDML(update, false, targetTableID2, sourceTable21, []interface{}{1, 4, "aa"}, []interface{}{4, 4, "aa"}, []interface{}{1, 1, "aa"}, []interface{}{4, 4, "aa"}, ti21.Columns, ti21, ti21Index), - newDML(update, false, targetTableID2, sourceTable21, []interface{}{2, 5, "bb"}, []interface{}{5, 5, "bb"}, []interface{}{2, 2, "bb"}, []interface{}{5, 5, "bb"}, ti21.Columns, ti21, ti21Index), - newDML(update, false, targetTableID2, sourceTable22, []interface{}{3, 6, "cc"}, []interface{}{6, 6, "cc"}, []interface{}{3, 3, "cc"}, []interface{}{6, 6, "cc"}, ti22.Columns, ti22, ti22Index), + newDML(update, false, targetTableID2, sourceTable21, []interface{}{1, 4, "aa"}, []interface{}{4, 4, "aa"}, []interface{}{1, 1, "aa"}, []interface{}{4, 4, "aa"}, ti21.Columns, ti21, ti21Index, downTi21), + newDML(update, false, targetTableID2, sourceTable21, []interface{}{2, 5, "bb"}, []interface{}{5, 5, "bb"}, []interface{}{2, 2, "bb"}, []interface{}{5, 5, "bb"}, ti21.Columns, ti21, ti21Index, downTi21), + newDML(update, false, targetTableID2, sourceTable22, []interface{}{3, 6, "cc"}, []interface{}{6, 6, "cc"}, []interface{}{3, 3, "cc"}, []interface{}{6, 6, "cc"}, ti22.Columns, ti22, ti22Index, downTi22), // delete - newDML(del, false, targetTableID2, sourceTable21, nil, []interface{}{4, 4, "aa"}, nil, []interface{}{4, 4, "aa"}, ti21.Columns, ti21, ti21Index), - newDML(del, false, targetTableID2, sourceTable21, nil, []interface{}{5, 5, "bb"}, nil, []interface{}{5, 5, "bb"}, ti21.Columns, ti21, ti21Index), - newDML(del, false, targetTableID2, sourceTable22, nil, []interface{}{6, 6, "cc"}, nil, []interface{}{6, 6, "cc"}, ti22.Columns, ti22, ti22Index), + newDML(del, false, targetTableID2, sourceTable21, nil, []interface{}{4, 4, "aa"}, nil, []interface{}{4, 4, "aa"}, ti21.Columns, ti21, ti21Index, downTi21), + newDML(del, false, targetTableID2, sourceTable21, nil, []interface{}{5, 5, "bb"}, nil, []interface{}{5, 5, "bb"}, ti21.Columns, ti21, ti21Index, downTi21), + newDML(del, false, targetTableID2, sourceTable22, nil, []interface{}{6, 6, "cc"}, nil, []interface{}{6, 6, "cc"}, ti22.Columns, ti22, ti22Index, downTi22), // table1 // detele - newDML(del, false, targetTableID1, sourceTable11, nil, []interface{}{44, 44, "aaa"}, nil, []interface{}{44, 44, "aaa"}, ti11.Columns, ti11, ti11Index), - newDML(del, false, targetTableID1, sourceTable11, nil, []interface{}{55, 55, "bbb"}, nil, []interface{}{55, 55, "bbb"}, ti11.Columns, ti11, ti11Index), - newDML(del, false, targetTableID1, sourceTable12, nil, []interface{}{66, 66, "ccc"}, nil, []interface{}{66, 66, "ccc"}, ti12.Columns, ti12, ti12Index), + newDML(del, false, targetTableID1, sourceTable11, nil, []interface{}{44, 44, "aaa"}, nil, []interface{}{44, 44, "aaa"}, ti11.Columns, ti11, ti11Index, downTi11), + newDML(del, false, targetTableID1, sourceTable11, nil, []interface{}{55, 55, "bbb"}, nil, []interface{}{55, 55, "bbb"}, ti11.Columns, ti11, ti11Index, downTi11), + newDML(del, false, targetTableID1, sourceTable12, nil, []interface{}{66, 66, "ccc"}, nil, []interface{}{66, 66, "ccc"}, ti12.Columns, ti12, ti12Index, downTi12), } expectQueries := []string{ diff --git a/dm/syncer/job_test.go b/dm/syncer/job_test.go index a40c8e99e56..637eb1a7c89 100644 --- a/dm/syncer/job_test.go +++ b/dm/syncer/job_test.go @@ -107,7 +107,7 @@ func (t *testJobSuite) TestJob(c *C) { jobStr string }{ { - newDMLJob(insert, table, table, newDML(insert, true, "targetTable", table, nil, []interface{}{2, 2}, nil, []interface{}{2, 2}, ti.Columns, ti, tiIndex), ec), + newDMLJob(insert, table, table, newDML(insert, true, "targetTable", table, nil, []interface{}{2, 2}, nil, []interface{}{2, 2}, ti.Columns, ti, tiIndex, nil), ec), "tp: insert, dml: [safemode: true, targetTableID: targetTable, op: insert, columns: [id col1], oldValues: [], values: [2 2]], ddls: [], last_location: position: (, 4), gtid-set: , start_location: position: (, 4), gtid-set: , current_location: position: (, 4), gtid-set: ", }, { newDDLJob(qec), diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 22eaf285c5c..585f3cfa674 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -2024,7 +2024,7 @@ func (s *Syncer) handleRowsEvent(ev *replication.RowsEvent, ec eventContext) err } param.safeMode = ec.safeMode - dmls, err = s.genAndFilterInsertDMLs(param, exprFilter) + dmls, err = s.genAndFilterInsertDMLs(ec.tctx, param, exprFilter) if err != nil { return terror.Annotatef(err, "gen insert sqls failed, sourceTable: %v, targetTable: %v", sourceTable, targetTable) } diff --git a/dm/syncer/syncer_test.go b/dm/syncer/syncer_test.go index 3fa0b82f227..dbec3bbde03 100644 --- a/dm/syncer/syncer_test.go +++ b/dm/syncer/syncer_test.go @@ -788,6 +788,9 @@ func (s *testSyncerSuite) TestRun(c *C) { mock.ExpectQuery("SHOW CREATE TABLE " + "`test_1`.`t_1`").WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). AddRow("t_1", "create table t_1(id int primary key, name varchar(24))")) + mock.ExpectQuery("SHOW CREATE TABLE " + "`test_1`.`t_1`").WillReturnRows( + sqlmock.NewRows([]string{"Table", "Create Table"}). + AddRow("t_1", "create table t_1(id int primary key, name varchar(24), KEY `index1` (`name`))")) s.mockGetServerUnixTS(mock) mock.ExpectQuery("SHOW CREATE TABLE " + "`test_1`.`t_2`").WillReturnRows( sqlmock.NewRows([]string{"Table", "Create Table"}). From d358c28c61ca997c5aba8882cf8715a7ca68e185 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Tue, 21 Dec 2021 18:11:47 +0800 Subject: [PATCH 21/48] errorutil,sink,syncer: add errorutil to handle ignorable error (#3264) (#3995) --- cdc/sink/mysql.go | 41 ++++++++++------------------ cdc/sink/simple_mysql_tester.go | 6 +++-- dm/syncer/error.go | 25 ++--------------- dm/syncer/error_test.go | 18 ------------- dm/syncer/syncer.go | 3 ++- dm/syncer/syncer_test.go | 3 ++- pkg/errorutil/ignore.go | 48 +++++++++++++++++++++++++++++++++ pkg/errorutil/ignore_test.go | 47 ++++++++++++++++++++++++++++++++ 8 files changed, 119 insertions(+), 72 deletions(-) create mode 100644 pkg/errorutil/ignore.go create mode 100644 pkg/errorutil/ignore_test.go diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index aeab222697d..044face3385 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -29,8 +29,6 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - tddl "github.com/pingcap/tidb/ddl" - "github.com/pingcap/tidb/infoschema" timodel "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tiflow/cdc/model" @@ -39,7 +37,7 @@ import ( "github.com/pingcap/tiflow/pkg/cyclic" "github.com/pingcap/tiflow/pkg/cyclic/mark" cerror "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/filter" + "github.com/pingcap/tiflow/pkg/errorutil" tifilter "github.com/pingcap/tiflow/pkg/filter" "github.com/pingcap/tiflow/pkg/notify" "github.com/pingcap/tiflow/pkg/quotes" @@ -91,7 +89,7 @@ type mysqlSink struct { db *sql.DB params *sinkParams - filter *filter.Filter + filter *tifilter.Filter cyclic *cyclic.Cyclic txnCache *common.UnresolvedTxnCache @@ -114,6 +112,16 @@ type mysqlSink struct { cancel func() } +func needSwitchDB(ddl *model.DDLEvent) bool { + if len(ddl.TableInfo.Schema) > 0 { + if ddl.Type == timodel.ActionCreateSchema || ddl.Type == timodel.ActionDropSchema { + return false + } + return true + } + return false +} + func (s *mysqlSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { count := s.txnCache.Append(s.filter, rows...) s.statistics.AddRowsCount(count) @@ -214,7 +222,7 @@ func (s *mysqlSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTab func (s *mysqlSink) execDDLWithMaxRetries(ctx context.Context, ddl *model.DDLEvent) error { return retry.Do(ctx, func() error { err := s.execDDL(ctx, ddl) - if isIgnorableDDLError(err) { + if errorutil.IsIgnorableMySQLDDLError(err) { log.Info("execute DDL failed, but error can be ignored", zap.String("query", ddl.Query), zap.Error(err)) return nil } @@ -226,7 +234,7 @@ func (s *mysqlSink) execDDLWithMaxRetries(ctx context.Context, ddl *model.DDLEve } func (s *mysqlSink) execDDL(ctx context.Context, ddl *model.DDLEvent) error { - shouldSwitchDB := len(ddl.TableInfo.Schema) > 0 && ddl.Type != timodel.ActionCreateSchema + shouldSwitchDB := needSwitchDB(ddl) failpoint.Inject("MySQLSinkExecDDLDelay", func() { select { @@ -1288,27 +1296,6 @@ func whereSlice(cols []*model.Column, forceReplicate bool) (colNames []string, a return } -func isIgnorableDDLError(err error) bool { - errCode, ok := getSQLErrCode(err) - if !ok { - return false - } - // we can get error code from: - // infoschema's error definition: https://github.com/pingcap/tidb/blob/master/infoschema/infoschema.go - // DDL's error definition: https://github.com/pingcap/tidb/blob/master/ddl/ddl.go - // tidb/mysql error code definition: https://github.com/pingcap/tidb/blob/master/mysql/errcode.go - switch errCode { - case infoschema.ErrDatabaseExists.Code(), infoschema.ErrDatabaseNotExists.Code(), infoschema.ErrDatabaseDropExists.Code(), - infoschema.ErrTableExists.Code(), infoschema.ErrTableNotExists.Code(), infoschema.ErrTableDropExists.Code(), - infoschema.ErrColumnExists.Code(), infoschema.ErrColumnNotExists.Code(), infoschema.ErrIndexExists.Code(), - infoschema.ErrKeyNotExists.Code(), tddl.ErrCantDropFieldOrKey.Code(), mysql.ErrDupKeyName, mysql.ErrSameNamePartition, - mysql.ErrDropPartitionNonExistent, mysql.ErrMultiplePriKey: - return true - default: - return false - } -} - func getSQLErrCode(err error) (errors.ErrCode, bool) { mysqlErr, ok := errors.Cause(err).(*dmysql.MySQLError) if !ok { diff --git a/cdc/sink/simple_mysql_tester.go b/cdc/sink/simple_mysql_tester.go index 4bfd5abf7ae..95a8eb9cba4 100644 --- a/cdc/sink/simple_mysql_tester.go +++ b/cdc/sink/simple_mysql_tester.go @@ -25,12 +25,14 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" + "go.uber.org/zap" + "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/errorutil" "github.com/pingcap/tiflow/pkg/filter" "github.com/pingcap/tiflow/pkg/quotes" - "go.uber.org/zap" ) func init() { @@ -176,7 +178,7 @@ func (s *simpleMySQLSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) sql = fmt.Sprintf("use %s;%s", ddl.TableInfo.Schema, ddl.Query) } _, err := s.db.ExecContext(ctx, sql) - if err != nil && isIgnorableDDLError(err) { + if err != nil && errorutil.IsIgnorableMySQLDDLError(err) { log.Info("execute DDL failed, but error can be ignored", zap.String("query", ddl.Query), zap.Error(err)) return nil } diff --git a/dm/syncer/error.go b/dm/syncer/error.go index a1638b932a5..c1aec276d8e 100644 --- a/dm/syncer/error.go +++ b/dm/syncer/error.go @@ -21,9 +21,7 @@ import ( "github.com/go-sql-driver/mysql" "github.com/pingcap/errors" "github.com/pingcap/tidb-tools/pkg/dbutil" - tddl "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/errno" - "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/parser/ast" tmysql "github.com/pingcap/tidb/parser/mysql" @@ -32,28 +30,9 @@ import ( tcontext "github.com/pingcap/tiflow/dm/pkg/context" "github.com/pingcap/tiflow/dm/pkg/log" "github.com/pingcap/tiflow/dm/syncer/dbconn" + "github.com/pingcap/tiflow/pkg/errorutil" ) -func ignoreDDLError(err error) bool { - err = errors.Cause(err) - mysqlErr, ok := err.(*mysql.MySQLError) - if !ok { - return false - } - - errCode := errors.ErrCode(mysqlErr.Number) - switch errCode { - case infoschema.ErrDatabaseExists.Code(), infoschema.ErrDatabaseDropExists.Code(), - infoschema.ErrTableExists.Code(), infoschema.ErrTableDropExists.Code(), - infoschema.ErrColumnExists.Code(), - infoschema.ErrIndexExists.Code(), - infoschema.ErrKeyNameDuplicate.Code(), tddl.ErrCantDropFieldOrKey.Code(): - return true - default: - return false - } -} - func isDropColumnWithIndexError(err error) bool { mysqlErr, ok := errors.Cause(err).(*mysql.MySQLError) if !ok { @@ -207,7 +186,7 @@ func (s *Syncer) handleSpecialDDLError(tctx *tcontext.Context, err error, ddls [ } tctx.L().Info("drop index success, now try to drop column", zap.Strings("index", idx2Drop)) - if _, err2 = conn.ExecuteSQLWithIgnore(tctx, ignoreDDLError, ddls[index:]); err2 != nil { + if _, err2 = conn.ExecuteSQLWithIgnore(tctx, errorutil.IsIgnorableMySQLDDLError, ddls[index:]); err2 != nil { return err2 } diff --git a/dm/syncer/error_test.go b/dm/syncer/error_test.go index a6ca84e91f9..9f84997bec3 100644 --- a/dm/syncer/error_test.go +++ b/dm/syncer/error_test.go @@ -21,8 +21,6 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/tidb/errno" - "github.com/pingcap/tidb/infoschema" - tmysql "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tiflow/dm/pkg/conn" tcontext "github.com/pingcap/tiflow/dm/pkg/context" @@ -36,22 +34,6 @@ func newMysqlErr(number uint16, message string) *mysql.MySQLError { } } -func (s *testSyncerSuite) TestIgnoreDDLError(c *C) { - cases := []struct { - err error - ret bool - }{ - {errors.New("raw error"), false}, - {newMysqlErr(tmysql.ErrDupKeyName, "Error: Duplicate key name 'some_key'"), true}, - {newMysqlErr(uint16(infoschema.ErrDatabaseExists.Code()), "Can't create database"), true}, - {newMysqlErr(uint16(infoschema.ErrAccessDenied.Code()), "Access denied for user"), false}, - } - - for _, t := range cases { - c.Assert(ignoreDDLError(t.err), Equals, t.ret) - } -} - func (s *testSyncerSuite) TestHandleSpecialDDLError(c *C) { var ( syncer = NewSyncer(s.cfg, nil, nil) diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 585f3cfa674..ea3c22f73f1 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -70,6 +70,7 @@ import ( onlineddl "github.com/pingcap/tiflow/dm/syncer/online-ddl-tools" sm "github.com/pingcap/tiflow/dm/syncer/safe-mode" "github.com/pingcap/tiflow/dm/syncer/shardddl" + "github.com/pingcap/tiflow/pkg/errorutil" ) var ( @@ -1162,7 +1163,7 @@ func (s *Syncer) syncDDL(tctx *tcontext.Context, queueBucket string, db *dbconn. if !ignore { var affected int - affected, err = db.ExecuteSQLWithIgnore(tctx, ignoreDDLError, ddlJob.ddls) + affected, err = db.ExecuteSQLWithIgnore(tctx, errorutil.IsIgnorableMySQLDDLError, ddlJob.ddls) if err != nil { err = s.handleSpecialDDLError(tctx, err, ddlJob.ddls, affected, db) err = terror.WithScope(err, terror.ScopeDownstream) diff --git a/dm/syncer/syncer_test.go b/dm/syncer/syncer_test.go index dbec3bbde03..3467cd1e8e9 100644 --- a/dm/syncer/syncer_test.go +++ b/dm/syncer/syncer_test.go @@ -40,6 +40,7 @@ import ( streamer2 "github.com/pingcap/tiflow/dm/pkg/streamer" "github.com/pingcap/tiflow/dm/pkg/utils" "github.com/pingcap/tiflow/dm/syncer/dbconn" + "github.com/pingcap/tiflow/pkg/errorutil" sqlmock "github.com/DATA-DOG/go-sqlmock" "github.com/go-mysql-org/go-mysql/mysql" @@ -1644,7 +1645,7 @@ func (s *testSyncerSuite) TestExecuteSQLSWithIgnore(c *C) { mock.ExpectCommit() tctx := tcontext.Background().WithLogger(log.With(zap.String("test", "TestExecuteSQLSWithIgnore"))) - n, err := conn.ExecuteSQLWithIgnore(tctx, ignoreDDLError, sqls) + n, err := conn.ExecuteSQLWithIgnore(tctx, errorutil.IsIgnorableMySQLDDLError, sqls) c.Assert(err, IsNil) c.Assert(n, Equals, 2) diff --git a/pkg/errorutil/ignore.go b/pkg/errorutil/ignore.go new file mode 100644 index 00000000000..6a48dd203f6 --- /dev/null +++ b/pkg/errorutil/ignore.go @@ -0,0 +1,48 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package errorutil + +import ( + dmysql "github.com/go-sql-driver/mysql" + "github.com/pingcap/errors" + tddl "github.com/pingcap/tidb/ddl" + "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/parser/mysql" +) + +// IsIgnorableMySQLDDLError is used to check what error can be ignored +// we can get error code from: +// infoschema's error definition: https://github.com/pingcap/tidb/blob/master/infoschema/infoschema.go +// DDL's error definition: https://github.com/pingcap/tidb/blob/master/ddl/ddl.go +// tidb/mysql error code definition: https://github.com/pingcap/tidb/blob/master/mysql/errcode.go +func IsIgnorableMySQLDDLError(err error) bool { + err = errors.Cause(err) + mysqlErr, ok := err.(*dmysql.MySQLError) + if !ok { + return false + } + + errCode := errors.ErrCode(mysqlErr.Number) + switch errCode { + case infoschema.ErrDatabaseExists.Code(), infoschema.ErrDatabaseDropExists.Code(), + infoschema.ErrTableExists.Code(), infoschema.ErrTableDropExists.Code(), + infoschema.ErrColumnExists.Code(), infoschema.ErrIndexExists.Code(), + infoschema.ErrKeyNotExists.Code(), tddl.ErrCantDropFieldOrKey.Code(), + mysql.ErrDupKeyName, mysql.ErrSameNamePartition, + mysql.ErrDropPartitionNonExistent, mysql.ErrMultiplePriKey: + return true + default: + return false + } +} diff --git a/pkg/errorutil/ignore_test.go b/pkg/errorutil/ignore_test.go new file mode 100644 index 00000000000..7c2dbd0f7e0 --- /dev/null +++ b/pkg/errorutil/ignore_test.go @@ -0,0 +1,47 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package errorutil + +import ( + "errors" + "testing" + + "github.com/go-sql-driver/mysql" + "github.com/pingcap/tidb/infoschema" + tmysql "github.com/pingcap/tidb/parser/mysql" + "github.com/stretchr/testify/assert" +) + +func newMysqlErr(number uint16, message string) *mysql.MySQLError { + return &mysql.MySQLError{ + Number: number, + Message: message, + } +} + +func TestIgnoreMysqlDDLError(t *testing.T) { + cases := []struct { + err error + ret bool + }{ + {errors.New("raw error"), false}, + {newMysqlErr(tmysql.ErrDupKeyName, "Error: Duplicate key name 'some_key'"), true}, + {newMysqlErr(uint16(infoschema.ErrDatabaseExists.Code()), "Can't create database"), true}, + {newMysqlErr(uint16(infoschema.ErrAccessDenied.Code()), "Access denied for user"), false}, + } + + for _, item := range cases { + assert.Equal(t, item.ret, IsIgnorableMySQLDDLError(item.err)) + } +} From 05802b3eda895d93d5f18a8939951e7e909e9042 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Wed, 22 Dec 2021 09:55:46 +0800 Subject: [PATCH 22/48] dm/worker: don't exit when failed to read checkpoint in relay (#3345) (#4005) --- dm/dm/worker/server.go | 1 + dm/dm/worker/source_worker.go | 4 ++-- dm/tests/new_relay/run.sh | 41 +++++++++++++++++++++++++++++++++++ 3 files changed, 44 insertions(+), 2 deletions(-) diff --git a/dm/dm/worker/server.go b/dm/dm/worker/server.go index cac9331fdf4..b3f9b0ce51e 100644 --- a/dm/dm/worker/server.go +++ b/dm/dm/worker/server.go @@ -89,6 +89,7 @@ func NewServer(cfg *Config) *Server { } // Start starts to serving. +// this function should only exit when can't dail DM-master, for other errors it should not exit. func (s *Server) Start() error { log.L().Info("starting dm-worker server") RegistryMetrics() diff --git a/dm/dm/worker/source_worker.go b/dm/dm/worker/source_worker.go index 9ea3e9385cc..b9125c5c1b2 100644 --- a/dm/dm/worker/source_worker.go +++ b/dm/dm/worker/source_worker.go @@ -301,11 +301,11 @@ func (w *SourceWorker) EnableRelay() (err error) { defer dcancel() minLoc, err1 := getMinLocInAllSubTasks(dctx, subTaskCfgs) if err1 != nil { - return err1 + w.l.Error("meet error when EnableRelay", zap.Error(err1)) } if minLoc != nil { - log.L().Info("get min location in all subtasks", zap.Stringer("location", *minLoc)) + w.l.Info("get min location in all subtasks", zap.Stringer("location", *minLoc)) w.cfg.RelayBinLogName = binlog.AdjustPosition(minLoc.Position).Name w.cfg.RelayBinlogGTID = minLoc.GTIDSetStr() // set UUIDSuffix when bound to a source diff --git a/dm/tests/new_relay/run.sh b/dm/tests/new_relay/run.sh index 92e6b1053b8..1a30f297e7e 100755 --- a/dm/tests/new_relay/run.sh +++ b/dm/tests/new_relay/run.sh @@ -43,6 +43,46 @@ function test_cant_dail_upstream() { cleanup_data $TEST_NAME } +function test_cant_dail_downstream() { + cleanup_data $TEST_NAME + cleanup_process + + run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml + check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT + + cp $cur/conf/source1.yaml $WORK_DIR/source1.yaml + dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID1 worker1" \ + "\"result\": true" 1 + dmctl_start_task_standalone $cur/conf/dm-task.yaml "--remove-meta" + + kill_dm_worker + # kill tidb + pkill -hup tidb-server 2>/dev/null || true + wait_process_exit tidb-server + + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT + + # make sure DM-worker doesn't exit + sleep 2 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status -s $SOURCE_ID1" \ + "\"relayCatchUpMaster\": true" 1 \ + "dial tcp 127.0.0.1:4000: connect: connection refused" 1 + + # restart tidb + run_tidb_server 4000 $TIDB_PASSWORD + sleep 2 + + cleanup_process + cleanup_data $TEST_NAME +} + function test_kill_dump_connection() { cleanup_data $TEST_NAME cleanup_process @@ -83,6 +123,7 @@ function test_kill_dump_connection() { } function run() { + test_cant_dail_downstream test_cant_dail_upstream export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/relay/ReportRelayLogSpaceInBackground=return(1)" From 5a9e78fab4ad629a0fd41ddff14779fb7119ae10 Mon Sep 17 00:00:00 2001 From: lance6716 Date: Wed, 22 Dec 2021 14:19:47 +0800 Subject: [PATCH 23/48] syncer(dm): use an early location to reset binlog and open safemode (#3860) --- dm/syncer/syncer.go | 21 ++++++++++++-- dm/tests/_utils/check_sync_diff | 2 +- dm/tests/duplicate_event/run.sh | 46 ------------------------------- dm/tests/others_integration_2.txt | 1 + 4 files changed, 21 insertions(+), 49 deletions(-) diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index ea3c22f73f1..9ff458af78b 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -1550,6 +1550,9 @@ func (s *Syncer) Run(ctx context.Context) (err error) { return nil } + inFinerRetry := false + // in release branch, we only use eventIndex to test a bug + eventIndex := 0 for { if s.execError.Load() != nil { return nil @@ -1591,6 +1594,14 @@ func (s *Syncer) Run(ctx context.Context) (err error) { err = errors.New("connect: connection refused") } }) + failpoint.Inject("GetEventErrorInTxn", func(val failpoint.Value) { + if intVal, ok := val.(int); ok && intVal == eventIndex { + err = errors.New("failpoint triggered") + s.tctx.L().Warn("failed to get event", zap.Int("event_index", eventIndex), + zap.Any("cur_pos", currentLocation), zap.Any("las_pos", lastLocation), + zap.Any("pos", e.Header.LogPos), log.ShortError(err)) + } + }) switch { case err == context.Canceled: tctx.L().Info("binlog replication main routine quit(context canceled)!", zap.Stringer("last location", lastLocation)) @@ -1617,11 +1628,12 @@ func (s *Syncer) Run(ctx context.Context) (err error) { if s.streamerController.CanRetry(err) { // lastLocation is the last finished GTID - err = s.streamerController.ResetReplicationSyncer(tctx, lastLocation) + err = s.streamerController.ResetReplicationSyncer(tctx, s.checkpoint.GlobalPoint()) if err != nil { return err } log.L().Info("reset replication binlog puller", zap.Any("pos", s.checkpoint.GlobalPoint())) + inFinerRetry = true continue } @@ -1760,7 +1772,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { shardingReSync: shardingReSync, closeShardingResync: closeShardingResync, traceSource: traceSource, - safeMode: s.safeMode.Enable(), + safeMode: s.safeMode.Enable() || inFinerRetry, tryReSync: tryReSync, startTime: startTime, shardingReSyncCh: &shardingReSyncCh, @@ -1773,12 +1785,17 @@ func (s *Syncer) Run(ctx context.Context) (err error) { case *replication.RotateEvent: err2 = s.handleRotateEvent(ev, ec) case *replication.RowsEvent: + eventIndex++ metrics.BinlogEventRowHistogram.WithLabelValues(s.cfg.WorkerName, s.cfg.Name, s.cfg.SourceID).Observe(float64(len(ev.Rows))) err2 = s.handleRowsEvent(ev, ec) case *replication.QueryEvent: originSQL = strings.TrimSpace(string(ev.Query)) err2 = s.handleQueryEvent(ev, ec, originSQL) case *replication.XIDEvent: + eventIndex = 0 + if inFinerRetry { + inFinerRetry = false + } if shardingReSync != nil { shardingReSync.currLocation.Position.Pos = e.Header.LogPos shardingReSync.currLocation.Suffix = currentLocation.Suffix diff --git a/dm/tests/_utils/check_sync_diff b/dm/tests/_utils/check_sync_diff index e4ed59f3245..2b7bcd17bdf 100755 --- a/dm/tests/_utils/check_sync_diff +++ b/dm/tests/_utils/check_sync_diff @@ -23,7 +23,7 @@ cd $workdir i=0 while [ $i -lt $check_time ]; do rm -rf $OUTPUT_DIR - $binary --config=$conf >>$LOG 2>&1 + $binary --config=$conf >$LOG 2>&1 ret=$? if [ "$ret" == 0 ]; then echo "check diff successfully" diff --git a/dm/tests/duplicate_event/run.sh b/dm/tests/duplicate_event/run.sh index 1df98bc42c3..7e401898ff3 100644 --- a/dm/tests/duplicate_event/run.sh +++ b/dm/tests/duplicate_event/run.sh @@ -34,53 +34,7 @@ function run() { run_sql_file $cur/data/db1.increment.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 check_log_contain_with_retry "reset replication binlog puller" $WORK_DIR/worker1/log/dm-worker.log - check_log_contain_with_retry "discard event already consumed" $WORK_DIR/worker1/log/dm-worker.log check_sync_diff $WORK_DIR $cur/conf/diff_config.toml - - # 2. test relay log retry relay with GTID - - # with a 5 rows insert txn: 1 * FormatDesc + 1 * PreviousGTID + 1 * GTID + 1 * BEGIN + 5 * (Table_map + Write_rows) + 1 * XID - # here we fail at the third write rows event, sync should retry and auto recover without any duplicate event - export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/relay/RelayGetEventFailed=15*return(3);github.com/pingcap/tiflow/dm/relay/retry/RelayAllowRetry=return" - - run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml - check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT - - cp $cur/conf/source2.yaml $WORK_DIR/source2.yaml - dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 - - run_sql_file $cur/data/db1.prepare.sql $MYSQL_HOST2 $MYSQL_PORT2 $MYSQL_PASSWORD2 - check_contains 'Query OK, 2 rows affected' - - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "start-task $cur/conf/dm-task-relay.yaml --remove-meta" - check_metric $WORKER2_PORT "dm_worker_task_state{source_id=\"mysql-replica-02\",task=\"test_relay\",worker=\"worker2\"}" 10 1 3 - - check_sync_diff $WORK_DIR $cur/conf/diff_relay_config.toml - - run_sql_source2 "flush logs;" - run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "start-relay -s $SOURCE_ID2 worker2" \ - "\"result\": true" 1 - run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "query-status -s $SOURCE_ID2" \ - "\"relayCatchUpMaster\": true" 1 - - run_sql_file $cur/data/db1.increment.sql $MYSQL_HOST2 $MYSQL_PORT2 $MYSQL_PASSWORD2 - - check_log_contain_with_retry "retrying to read binlog" $WORK_DIR/worker2/log/dm-worker.log - check_log_contain_with_retry "discard duplicate event" $WORK_DIR/worker2/log/dm-worker.log - - check_sync_diff $WORK_DIR $cur/conf/diff_relay_config.toml - - # check relay log binlog file size is the same as master size - run_sql_source2 "show master status;" - binlog_file=$(grep "File" $TEST_DIR/sql_res.$TEST_NAME.txt | awk -F: '{print $2}' | xargs) - binlog_pos=$(grep "Position" $TEST_DIR/sql_res.$TEST_NAME.txt | awk -F: '{print $2}' | xargs) - - server_uuid=$(tail -n 1 $WORK_DIR/worker2/relay-dir/server-uuid.index) - relay_log_size=$(ls -al $WORK_DIR/worker2/relay-dir/$server_uuid/$binlog_file | awk '{print $5}') - [ "$binlog_pos" -eq "$relay_log_size" ] } # also cleanup dm processes in case of last run failed diff --git a/dm/tests/others_integration_2.txt b/dm/tests/others_integration_2.txt index eb7c5d37db7..c8959ae28bb 100644 --- a/dm/tests/others_integration_2.txt +++ b/dm/tests/others_integration_2.txt @@ -7,4 +7,5 @@ case_sensitive sql_mode http_proxies openapi +duplicate_event tracker_ignored_ddl From d9b5b4b19433e08837f507d4546186bb8a6a7284 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Wed, 22 Dec 2021 16:17:47 +0800 Subject: [PATCH 24/48] ticdc/owner: Fix ddl special comment syntax error (#3845) (#3978) --- cdc/owner/changefeed.go | 34 ++++- cdc/owner/changefeed_test.go | 145 ++++++++++++++++++- tests/integration_tests/ddl_reentrant/run.sh | 64 ++++---- 3 files changed, 209 insertions(+), 34 deletions(-) diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go index 2ee838cb178..a152d2382a4 100644 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -15,13 +15,15 @@ package owner import ( "context" + "strings" "sync" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" + "github.com/pingcap/tidb/parser" + "github.com/pingcap/tidb/parser/format" timodel "github.com/pingcap/tidb/parser/model" - "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/redo" cdcContext "github.com/pingcap/tiflow/pkg/context" @@ -442,7 +444,11 @@ func (c *changefeed) asyncExecDDL(ctx cdcContext.Context, job *timodel.Job) (don if err != nil { return false, errors.Trace(err) } - ddlEvent.Query = binloginfo.AddSpecialComment(ddlEvent.Query) + ddlEvent.Query, err = addSpecialComment(ddlEvent.Query) + if err != nil { + return false, errors.Trace(err) + } + c.ddlEventCache = ddlEvent if c.redoManager.Enabled() { err = c.redoManager.EmitDDLEvent(ctx, ddlEvent) @@ -506,3 +512,27 @@ func (c *changefeed) updateStatus(currentTs int64, barrierTs model.Ts) { func (c *changefeed) Close(ctx context.Context) { c.releaseResources(ctx) } + +// addSpecialComment translate tidb feature to comment +func addSpecialComment(ddlQuery string) (string, error) { + stms, _, err := parser.New().ParseSQL(ddlQuery) + if err != nil { + return "", errors.Trace(err) + } + if len(stms) != 1 { + log.Panic("invalid ddlQuery statement size", zap.String("ddlQuery", ddlQuery)) + } + var sb strings.Builder + // translate TiDB feature to special comment + restoreFlags := format.RestoreTiDBSpecialComment + // escape the keyword + restoreFlags |= format.RestoreNameBackQuotes + // upper case keyword + restoreFlags |= format.RestoreKeyWordUppercase + // wrap string with single quote + restoreFlags |= format.RestoreStringSingleQuotes + if err = stms[0].Restore(format.NewRestoreCtx(restoreFlags, &sb)); err != nil { + return "", errors.Trace(err) + } + return sb.String(), nil +} diff --git a/cdc/owner/changefeed_test.go b/cdc/owner/changefeed_test.go index 2657ce5fa8f..ec927e94a45 100644 --- a/cdc/owner/changefeed_test.go +++ b/cdc/owner/changefeed_test.go @@ -270,7 +270,7 @@ func (s *changefeedSuite) TestExecDDL(c *check.C) { mockDDLPuller.ddlQueue = append(mockDDLPuller.ddlQueue, job) tickThreeTime() c.Assert(state.Status.CheckpointTs, check.Equals, mockDDLPuller.resolvedTs) - c.Assert(mockAsyncSink.ddlExecuting.Query, check.Equals, "create database test1") + c.Assert(mockAsyncSink.ddlExecuting.Query, check.Equals, "CREATE DATABASE `test1`") // executing the ddl finished mockAsyncSink.ddlDone = true @@ -285,7 +285,7 @@ func (s *changefeedSuite) TestExecDDL(c *check.C) { mockDDLPuller.ddlQueue = append(mockDDLPuller.ddlQueue, job) tickThreeTime() c.Assert(state.Status.CheckpointTs, check.Equals, mockDDLPuller.resolvedTs) - c.Assert(mockAsyncSink.ddlExecuting.Query, check.Equals, "create table test1.test1(id int primary key)") + c.Assert(mockAsyncSink.ddlExecuting.Query, check.Equals, "CREATE TABLE `test1`.`test1` (`id` INT PRIMARY KEY)") // executing the ddl finished mockAsyncSink.ddlDone = true @@ -352,3 +352,144 @@ func (s *changefeedSuite) TestFinished(c *check.C) { c.Assert(state.Status.CheckpointTs, check.Equals, state.Info.TargetTs) c.Assert(state.Info.State, check.Equals, model.StateFinished) } + +func (s *changefeedSuite) TestAddSpecialComment(c *check.C) { + defer testleak.AfterTest(c)() + testCase := []struct { + input string + result string + }{ + { + "create table t1 (id int ) shard_row_id_bits=2;", + "CREATE TABLE `t1` (`id` INT) /*T! SHARD_ROW_ID_BITS = 2 */", + }, + { + "create table t1 (id int ) shard_row_id_bits=2 pre_split_regions=2;", + "CREATE TABLE `t1` (`id` INT) /*T! SHARD_ROW_ID_BITS = 2 */ /*T! PRE_SPLIT_REGIONS = 2 */", + }, + { + "create table t1 (id int ) shard_row_id_bits=2 pre_split_regions=2;", + "CREATE TABLE `t1` (`id` INT) /*T! SHARD_ROW_ID_BITS = 2 */ /*T! PRE_SPLIT_REGIONS = 2 */", + }, + { + "create table t1 (id int ) shard_row_id_bits=2 engine=innodb pre_split_regions=2;", + "CREATE TABLE `t1` (`id` INT) /*T! SHARD_ROW_ID_BITS = 2 */ ENGINE = innodb /*T! PRE_SPLIT_REGIONS = 2 */", + }, + { + "create table t1 (id int ) pre_split_regions=2 shard_row_id_bits=2;", + "CREATE TABLE `t1` (`id` INT) /*T! PRE_SPLIT_REGIONS = 2 */ /*T! SHARD_ROW_ID_BITS = 2 */", + }, + { + "create table t6 (id int ) shard_row_id_bits=2 shard_row_id_bits=3 pre_split_regions=2;", + "CREATE TABLE `t6` (`id` INT) /*T! SHARD_ROW_ID_BITS = 2 */ /*T! SHARD_ROW_ID_BITS = 3 */ /*T! PRE_SPLIT_REGIONS = 2 */", + }, + { + "create table t1 (id int primary key auto_random(2));", + "CREATE TABLE `t1` (`id` INT PRIMARY KEY /*T![auto_rand] AUTO_RANDOM(2) */)", + }, + { + "create table t1 (id int primary key auto_random);", + "CREATE TABLE `t1` (`id` INT PRIMARY KEY /*T![auto_rand] AUTO_RANDOM */)", + }, + { + "create table t1 (id int auto_random ( 4 ) primary key);", + "CREATE TABLE `t1` (`id` INT /*T![auto_rand] AUTO_RANDOM(4) */ PRIMARY KEY)", + }, + { + "create table t1 (id int auto_random ( 4 ) primary key);", + "CREATE TABLE `t1` (`id` INT /*T![auto_rand] AUTO_RANDOM(4) */ PRIMARY KEY)", + }, + { + "create table t1 (id int auto_random ( 3 ) primary key) auto_random_base = 100;", + "CREATE TABLE `t1` (`id` INT /*T![auto_rand] AUTO_RANDOM(3) */ PRIMARY KEY) /*T![auto_rand_base] AUTO_RANDOM_BASE = 100 */", + }, + { + "create table t1 (id int auto_random primary key) auto_random_base = 50;", + "CREATE TABLE `t1` (`id` INT /*T![auto_rand] AUTO_RANDOM */ PRIMARY KEY) /*T![auto_rand_base] AUTO_RANDOM_BASE = 50 */", + }, + { + "create table t1 (id int auto_increment key) auto_id_cache 100;", + "CREATE TABLE `t1` (`id` INT AUTO_INCREMENT PRIMARY KEY) /*T![auto_id_cache] AUTO_ID_CACHE = 100 */", + }, + { + "create table t1 (id int auto_increment unique) auto_id_cache 10;", + "CREATE TABLE `t1` (`id` INT AUTO_INCREMENT UNIQUE KEY) /*T![auto_id_cache] AUTO_ID_CACHE = 10 */", + }, + { + "create table t1 (id int) auto_id_cache = 5;", + "CREATE TABLE `t1` (`id` INT) /*T![auto_id_cache] AUTO_ID_CACHE = 5 */", + }, + { + "create table t1 (id int) auto_id_cache=5;", + "CREATE TABLE `t1` (`id` INT) /*T![auto_id_cache] AUTO_ID_CACHE = 5 */", + }, + { + "create table t1 (id int) /*T![auto_id_cache] auto_id_cache=5 */ ;", + "CREATE TABLE `t1` (`id` INT) /*T![auto_id_cache] AUTO_ID_CACHE = 5 */", + }, + { + "create table t1 (id int, a varchar(255), primary key (a, b) clustered);", + "CREATE TABLE `t1` (`id` INT,`a` VARCHAR(255),PRIMARY KEY(`a`, `b`) /*T![clustered_index] CLUSTERED */)", + }, + { + "create table t1(id int, v int, primary key(a) clustered);", + "CREATE TABLE `t1` (`id` INT,`v` INT,PRIMARY KEY(`a`) /*T![clustered_index] CLUSTERED */)", + }, + { + "create table t1(id int primary key clustered, v int);", + "CREATE TABLE `t1` (`id` INT PRIMARY KEY /*T![clustered_index] CLUSTERED */,`v` INT)", + }, + { + "alter table t add primary key(a) clustered;", + "ALTER TABLE `t` ADD PRIMARY KEY(`a`) /*T![clustered_index] CLUSTERED */", + }, + { + "create table t1 (id int, a varchar(255), primary key (a, b) nonclustered);", + "CREATE TABLE `t1` (`id` INT,`a` VARCHAR(255),PRIMARY KEY(`a`, `b`) /*T![clustered_index] NONCLUSTERED */)", + }, + { + "create table t1 (id int, a varchar(255), primary key (a, b) /*T![clustered_index] nonclustered */);", + "CREATE TABLE `t1` (`id` INT,`a` VARCHAR(255),PRIMARY KEY(`a`, `b`) /*T![clustered_index] NONCLUSTERED */)", + }, + { + "create table clustered_test(id int)", + "CREATE TABLE `clustered_test` (`id` INT)", + }, + { + "create database clustered_test", + "CREATE DATABASE `clustered_test`", + }, + { + "create database clustered", + "CREATE DATABASE `clustered`", + }, + { + "create table clustered (id int)", + "CREATE TABLE `clustered` (`id` INT)", + }, + { + "create table t1 (id int, a varchar(255) key clustered);", + "CREATE TABLE `t1` (`id` INT,`a` VARCHAR(255) PRIMARY KEY /*T![clustered_index] CLUSTERED */)", + }, + { + "alter table t force auto_increment = 12;", + "ALTER TABLE `t` /*T![force_inc] FORCE */ AUTO_INCREMENT = 12", + }, + { + "alter table t force, auto_increment = 12;", + "ALTER TABLE `t` FORCE /* AlterTableForce is not supported */ , AUTO_INCREMENT = 12", + }, + { + "create table cdc_test (id varchar(10) primary key ,c1 varchar(10)) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin/*!90000 SHARD_ROW_ID_BITS=4 PRE_SPLIT_REGIONS=3 */", + "CREATE TABLE `cdc_test` (`id` VARCHAR(10) PRIMARY KEY,`c1` VARCHAR(10)) ENGINE = InnoDB DEFAULT CHARACTER SET = UTF8MB4 DEFAULT COLLATE = UTF8MB4_BIN /*T! SHARD_ROW_ID_BITS = 4 */ /*T! PRE_SPLIT_REGIONS = 3 */", + }, + } + for _, ca := range testCase { + re, err := addSpecialComment(ca.input) + c.Check(err, check.IsNil) + c.Check(re, check.Equals, ca.result) + } + c.Assert(func() { + _, _ = addSpecialComment("alter table t force, auto_increment = 12;alter table t force, auto_increment = 12;") + }, check.Panics, "invalid ddlQuery statement size") +} diff --git a/tests/integration_tests/ddl_reentrant/run.sh b/tests/integration_tests/ddl_reentrant/run.sh index 6e788d9dcf6..7867179066d 100644 --- a/tests/integration_tests/ddl_reentrant/run.sh +++ b/tests/integration_tests/ddl_reentrant/run.sh @@ -8,26 +8,27 @@ WORK_DIR=$OUT_DIR/$TEST_NAME CDC_BINARY=cdc.test SINK_TYPE=$1 -ddls=("create database ddl_reentrant" false - "create table ddl_reentrant.t1 (id int primary key, id2 int not null, a varchar(10) not null, unique a(a), unique id2(id2))" false - "alter table ddl_reentrant.t1 add column b int" false - "alter table ddl_reentrant.t1 drop column b" false - "alter table ddl_reentrant.t1 add key index_a(a)" false - "alter table ddl_reentrant.t1 drop index index_a" false - "truncate table ddl_reentrant.t1" true - "alter table ddl_reentrant.t1 modify a varchar(20)" true - "rename table ddl_reentrant.t1 to ddl_reentrant.t2" false - "alter table ddl_reentrant.t2 alter a set default 'hello'" true - "alter table ddl_reentrant.t2 comment='modify comment'" true - "alter table ddl_reentrant.t2 rename index a to idx_a" false - "create table ddl_reentrant.t3 (a int primary key, b int) partition by range(a) (partition p0 values less than (1000), partition p1 values less than (2000))" false - "alter table ddl_reentrant.t3 add partition (partition p2 values less than (3000))" false - "alter table ddl_reentrant.t3 drop partition p2" false - "alter table ddl_reentrant.t3 truncate partition p0" true - "create view ddl_reentrant.t3_view as select a, b from ddl_reentrant.t3" false - "drop view ddl_reentrant.t3_view" false - "alter table ddl_reentrant.t3 default character set utf8mb4 default collate utf8mb4_unicode_ci" true - "alter schema ddl_reentrant default character set utf8mb4 default collate utf8mb4_unicode_ci" true +# cdc parse and restore ddl with flags format.RestoreStringSingleQuotes|format.RestoreNameBackQuotes|format.RestoreKeyWordUppercase|format.RestoreTiDBSpecialComment +ddls=("create database ddl_reentrant" false 'CREATE DATABASE `ddl_reentrant`' + "create table ddl_reentrant.t1 (id int primary key, id2 int not null, a varchar(10) not null, unique a(a), unique id2(id2))" false 'CREATE TABLE `ddl_reentrant`.`t1` (`id` INT PRIMARY KEY,`id2` INT NOT NULL,`a` VARCHAR(10) NOT NULL,UNIQUE `a`(`a`),UNIQUE `id2`(`id2`))' + "alter table ddl_reentrant.t1 add column b int" false 'ALTER TABLE `ddl_reentrant`.`t1` ADD COLUMN `b` INT' + "alter table ddl_reentrant.t1 drop column b" false 'ALTER TABLE `ddl_reentrant`.`t1` DROP COLUMN `b`' + "alter table ddl_reentrant.t1 add key index_a(a)" false 'ALTER TABLE `ddl_reentrant`.`t1` ADD INDEX `index_a`(`a`)' + "alter table ddl_reentrant.t1 drop index index_a" false 'ALTER TABLE `ddl_reentrant`.`t1` DROP INDEX `index_a`' + "truncate table ddl_reentrant.t1" true 'TRUNCATE TABLE `ddl_reentrant`.`t1`' + "alter table ddl_reentrant.t1 modify a varchar(20)" true 'ALTER TABLE `ddl_reentrant`.`t1` MODIFY COLUMN `a` VARCHAR(20)' + "rename table ddl_reentrant.t1 to ddl_reentrant.t2" false 'RENAME TABLE `ddl_reentrant`.`t1` TO `ddl_reentrant`.`t2`' + "alter table ddl_reentrant.t2 alter a set default 'hello'" true 'ALTER TABLE `ddl_reentrant`.`t2` ALTER COLUMN `a` SET DEFAULT _UTF8MB4'"'hello'" + "alter table ddl_reentrant.t2 comment='modify comment'" true 'ALTER TABLE `ddl_reentrant`.`t2` COMMENT = '"'modify comment'" + "alter table ddl_reentrant.t2 rename index a to idx_a" false 'ALTER TABLE `ddl_reentrant`.`t2` RENAME INDEX `a` TO `idx_a`' + "create table ddl_reentrant.t3 (a int primary key, b int) partition by range(a) (partition p0 values less than (1000), partition p1 values less than (2000))" false 'CREATE TABLE `ddl_reentrant`.`t3` (`a` INT PRIMARY KEY,`b` INT) PARTITION BY RANGE (`a`) (PARTITION `p0` VALUES LESS THAN (1000),PARTITION `p1` VALUES LESS THAN (2000))' + "alter table ddl_reentrant.t3 add partition (partition p2 values less than (3000))" false 'ALTER TABLE `ddl_reentrant`.`t3` ADD PARTITION (PARTITION `p2` VALUES LESS THAN (3000))' + "alter table ddl_reentrant.t3 drop partition p2" false 'ALTER TABLE `ddl_reentrant`.`t3` DROP PARTITION `p2`' + "alter table ddl_reentrant.t3 truncate partition p0" true 'ALTER TABLE `ddl_reentrant`.`t3` TRUNCATE PARTITION `p0`' + "create view ddl_reentrant.t3_view as select a, b from ddl_reentrant.t3" false 'CREATE ALGORITHM = UNDEFINED DEFINER = CURRENT_USER SQL SECURITY DEFINER VIEW `ddl_reentrant`.`t3_view` AS SELECT `a`,`b` FROM `ddl_reentrant`.`t3`' + "drop view ddl_reentrant.t3_view" false 'DROP VIEW `ddl_reentrant`.`t3_view`' + "alter table ddl_reentrant.t3 default character set utf8mb4 default collate utf8mb4_unicode_ci" true 'ALTER TABLE `ddl_reentrant`.`t3` CHARACTER SET UTF8MB4 COLLATE UTF8MB4_UNICODE_CI' + "alter schema ddl_reentrant default character set utf8mb4 default collate utf8mb4_unicode_ci" true 'ALTER DATABASE `ddl_reentrant` CHARACTER SET = utf8mb4 COLLATE = utf8mb4_unicode_ci' ) function complete_ddls() { @@ -36,14 +37,14 @@ function complete_ddls() { echo "skip some DDLs in tidb v4.0.x" else # DDLs that are supportted since 5.0 - ddls+=("alter table ddl_reentrant.t2 add column c1 int, add column c2 int, add column c3 int" false) - ddls+=("alter table ddl_reentrant.t2 drop column c1, drop column c2, drop column c3" false) + ddls+=("alter table ddl_reentrant.t2 add column c1 int, add column c2 int, add column c3 int" false 'ALTER TABLE `ddl_reentrant`.`t2` ADD COLUMN `c1` INT, ADD COLUMN `c2` INT, ADD COLUMN `c3` INT') + ddls+=("alter table ddl_reentrant.t2 drop column c1, drop column c2, drop column c3" false 'ALTER TABLE `ddl_reentrant`.`t2` DROP COLUMN `c1`, DROP COLUMN `c2`, DROP COLUMN `c3`') fi - ddls+=("alter table ddl_reentrant.t2 drop primary key" false) - ddls+=("alter table ddl_reentrant.t2 add primary key pk(id)" false) - ddls+=("drop table ddl_reentrant.t2" false) - ddls+=("recover table ddl_reentrant.t2" false) - ddls+=("drop database ddl_reentrant" false) + ddls+=("alter table ddl_reentrant.t2 drop primary key" false 'ALTER TABLE `ddl_reentrant`.`t2` DROP PRIMARY KEY') + ddls+=("alter table ddl_reentrant.t2 add primary key pk(id)" false 'ALTER TABLE `ddl_reentrant`.`t2` ADD PRIMARY KEY `pk`(`id`)') + ddls+=("drop table ddl_reentrant.t2" false 'DROP TABLE `ddl_reentrant`.`t2`') + ddls+=("recover table ddl_reentrant.t2" false 'RECOVER TABLE `ddl_reentrant`.`t2`') + ddls+=("drop database ddl_reentrant" false 'DROP DATABASE `ddl_reentrant`') } changefeedid="" @@ -94,14 +95,15 @@ tidb_build_branch=$(mysql -uroot -h${UP_TIDB_HOST} -P${UP_TIDB_PORT} -e \ function ddl_test() { ddl=$1 is_reentrant=$2 + restored_sql=$3 echo "------------------------------------------" - echo "test ddl $ddl, is_reentrant: $is_reentrant" + echo "test ddl $ddl, is_reentrant: $is_reentrant restored_sql: $restored_sql" run_sql $ddl ${UP_TIDB_HOST} ${UP_TIDB_PORT} ensure 10 check_ts_forward $changefeedid - echo $ddl >${WORK_DIR}/ddl_temp.sql + echo $restored_sql >${WORK_DIR}/ddl_temp.sql ensure 10 check_ddl_executed "${WORK_DIR}/cdc.log" "${WORK_DIR}/ddl_temp.sql" true ddl_finished_ts=$(grep "Execute DDL succeeded" ${WORK_DIR}/cdc.log | tail -n 1 | grep -oE '"CommitTs\\":[0-9]{18}' | awk -F: '{print $(NF)}') cdc cli changefeed remove --changefeed-id=${changefeedid} @@ -146,7 +148,9 @@ function run() { idx=$((idx + 1)) idxs_reentrant=${ddls[$idx]} idx=$((idx + 1)) - ddl_test $ddl $idxs_reentrant + restored_sql=${ddls[$idx]} + idx=$((idx + 1)) + ddl_test $ddl $idxs_reentrant $restored_sql done IFS=$OLDIFS From 091efcaefefd5cd083e38677c6079030af3571fd Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Wed, 22 Dec 2021 20:05:48 +0800 Subject: [PATCH 25/48] dm/scheduler: fix inconsistent of relay status (#3474) (#4009) --- dm/dm/master/scheduler/scheduler.go | 10 ++++ dm/tests/new_relay/conf/source2.yaml | 14 +++++ dm/tests/new_relay/run.sh | 79 +++++++++++++++++++++++++++- 3 files changed, 102 insertions(+), 1 deletion(-) create mode 100644 dm/tests/new_relay/conf/source2.yaml diff --git a/dm/dm/master/scheduler/scheduler.go b/dm/dm/master/scheduler/scheduler.go index 00fe67fd70b..7b7dcb2557a 100644 --- a/dm/dm/master/scheduler/scheduler.go +++ b/dm/dm/master/scheduler/scheduler.go @@ -1780,6 +1780,16 @@ func (s *Scheduler) handleWorkerOnline(ev ha.WorkerEvent, toLock bool) error { // 3. change the stage (from Offline) to Free or Relay. lastRelaySource := w.RelaySourceID() + if lastRelaySource == "" { + // when worker is removed (for example lost keepalive when master scheduler boots up), w.RelaySourceID() is + // of course nothing, so we find the relay source from a better place + for source, workerM := range s.relayWorkers { + if _, ok2 := workerM[w.BaseInfo().Name]; ok2 { + lastRelaySource = source + break + } + } + } w.ToFree() // TODO: rename ToFree to Online and move below logic inside it if lastRelaySource != "" { diff --git a/dm/tests/new_relay/conf/source2.yaml b/dm/tests/new_relay/conf/source2.yaml new file mode 100644 index 00000000000..6c272e728c1 --- /dev/null +++ b/dm/tests/new_relay/conf/source2.yaml @@ -0,0 +1,14 @@ +source-id: mysql-replica-02 +server-id: 123456 +flavor: 'mysql' +enable-gtid: true +relay-binlog-name: '' +relay-binlog-gtid: '' +enable-relay: false +from: + host: 127.0.0.1 + user: root + password: /Q7B9DizNLLTTfiZHv9WoEAKamfpIUs= + port: 3307 +checker: + check-enable: false diff --git a/dm/tests/new_relay/run.sh b/dm/tests/new_relay/run.sh index 1a30f297e7e..42f0fda61ce 100755 --- a/dm/tests/new_relay/run.sh +++ b/dm/tests/new_relay/run.sh @@ -83,6 +83,82 @@ function test_cant_dail_downstream() { cleanup_data $TEST_NAME } +function test_restart_relay_status() { + cleanup_data $TEST_NAME + cleanup_process + + run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml + check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT + + dmctl_operate_source create $cur/conf/source1.yaml $SOURCE_ID1 + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID1 worker1" + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status -s $SOURCE_ID1" \ + "\"result\": true" 2 \ + "\"worker\": \"worker1\"" 1 + + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT + + dmctl_operate_source create $cur/conf/source2.yaml $SOURCE_ID2 + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID2 worker2" + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status -s $SOURCE_ID2" \ + "\"result\": true" 2 \ + "\"worker\": \"worker2\"" 1 + + run_dm_worker $WORK_DIR/worker3 $WORKER3_PORT $cur/conf/dm-worker3.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER3_PORT + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-relay -s $SOURCE_ID2 worker3" + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status -s $SOURCE_ID2" \ + "\"result\": true" 3 \ + "\"worker\": \"worker2\"" 1 \ + "\"worker\": \"worker3\"" 1 + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "list-member -n worker3" \ + "relay" 1 + + kill_dm_worker + kill_dm_master + + run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml + check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT + + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml + run_dm_worker $WORK_DIR/worker3 $WORKER3_PORT $cur/conf/dm-worker3.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER3_PORT + + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status -s $SOURCE_ID1" \ + "\"result\": true" 2 \ + "\"worker\": \"worker1\"" 1 + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status -s $SOURCE_ID2" \ + "\"result\": true" 3 \ + "\"worker\": \"worker2\"" 1 \ + "\"worker\": \"worker3\"" 1 + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "list-member --worker" \ + "relay" 1 \ + "bound" 2 +} + function test_kill_dump_connection() { cleanup_data $TEST_NAME cleanup_process @@ -108,7 +184,7 @@ function test_kill_dump_connection() { "\"worker\": \"worker1\"" 1 run_sql_source1 "show processlist" - # kill dumop connection to test wheather relay will auto reconnect db + # kill dump connection to test whether relay will auto reconnect db dump_conn_id=$(cat $TEST_DIR/sql_res.$TEST_NAME.txt | grep Binlog -B 4 | grep Id | cut -d : -f2) run_sql_source1 "kill ${dump_conn_id}" @@ -123,6 +199,7 @@ function test_kill_dump_connection() { } function run() { + test_restart_relay_status test_cant_dail_downstream test_cant_dail_upstream From 2290a12296e69f5804af20b11940078d0a86e77f Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Fri, 24 Dec 2021 14:37:47 +0800 Subject: [PATCH 26/48] owner,scheduler(cdc): fix nil pointer panic in owner scheduler (#2980) (#4007) (#4016) --- cdc/owner/scheduler.go | 5 +++++ cdc/owner/scheduler_test.go | 19 ++++++++++++++++++- 2 files changed, 23 insertions(+), 1 deletion(-) diff --git a/cdc/owner/scheduler.go b/cdc/owner/scheduler.go index 2bf474790ac..ef468d3a200 100644 --- a/cdc/owner/scheduler.go +++ b/cdc/owner/scheduler.go @@ -302,6 +302,11 @@ func (s *scheduler) handleJobs(jobs []*schedulerJob) { func (s *scheduler) cleanUpFinishedOperations() { for captureID := range s.state.TaskStatuses { s.state.PatchTaskStatus(captureID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { + if status == nil { + log.Warn("task status of the capture is not found, may be the key in etcd was deleted", zap.String("captureID", captureID), zap.String("changeFeedID", s.state.ID)) + return status, false, nil + } + changed := false for tableID, operation := range status.Operation { if operation.Status == model.OperFinished { diff --git a/cdc/owner/scheduler_test.go b/cdc/owner/scheduler_test.go index 320100237a3..c426140400c 100644 --- a/cdc/owner/scheduler_test.go +++ b/cdc/owner/scheduler_test.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/check" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/orchestrator" "github.com/pingcap/tiflow/pkg/util/testleak" ) @@ -83,8 +84,24 @@ func (s *schedulerSuite) finishTableOperation(captureID model.CaptureID, tableID func (s *schedulerSuite) TestScheduleOneCapture(c *check.C) { defer testleak.AfterTest(c)() + + s.reset(c) + captureID := "test-capture-0" + s.addCapture(captureID) + + _, _ = s.scheduler.Tick(s.state, []model.TableID{}, s.captures) + + // Manually simulate the scenario where the corresponding key was deleted in the etcd + key := &etcd.CDCKey{ + Tp: etcd.CDCKeyTypeTaskStatus, + CaptureID: captureID, + ChangefeedID: s.state.ID, + } + s.tester.MustUpdate(key.String(), nil) + s.tester.MustApplyPatches() + s.reset(c) - captureID := "test-capture-1" + captureID = "test-capture-1" s.addCapture(captureID) // add three tables From 5e7e3afcfb635d2461db817ac8c880c1cff816a0 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Fri, 24 Dec 2021 16:03:47 +0800 Subject: [PATCH 27/48] config(ticdc): Fix old value configuration check for maxwell protocol (#3747) (#3783) --- cdc/sink/codec/interface.go | 20 ++++++++++++++++++++ cdc/sink/mq.go | 8 +++++--- pkg/cmd/cli/cli_changefeed_create.go | 8 ++++++-- 3 files changed, 31 insertions(+), 5 deletions(-) diff --git a/cdc/sink/codec/interface.go b/cdc/sink/codec/interface.go index ca8548dae7f..3c017ae86f5 100644 --- a/cdc/sink/codec/interface.go +++ b/cdc/sink/codec/interface.go @@ -194,6 +194,26 @@ func (p *Protocol) FromString(protocol string) { } } +// String converts the Protocol enum type string to string. +func (p Protocol) String() string { + switch p { + case ProtocolDefault: + return "default" + case ProtocolCanal: + return "canal" + case ProtocolAvro: + return "avro" + case ProtocolMaxwell: + return "maxwell" + case ProtocolCanalJSON: + return "canal-json" + case ProtocolCraft: + return "craft" + default: + panic("unreachable") + } +} + type EncoderBuilder interface { Build(ctx context.Context) (EventBatchEncoder, error) } diff --git a/cdc/sink/mq.go b/cdc/sink/mq.go index 0da86336b62..862acf26a7b 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -15,6 +15,7 @@ package sink import ( "context" + "fmt" "net/url" "strings" "sync/atomic" @@ -68,9 +69,10 @@ func newMqSink( ) (*mqSink, error) { var protocol codec.Protocol protocol.FromString(config.Sink.Protocol) - if (protocol == codec.ProtocolCanal || protocol == codec.ProtocolCanalJSON) && !config.EnableOldValue { - log.Error("Old value is not enabled when using Canal protocol. Please update changefeed config") - return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, errors.New("Canal requires old value to be enabled")) + if (protocol == codec.ProtocolCanal || protocol == codec.ProtocolCanalJSON || protocol == codec.ProtocolMaxwell) && !config.EnableOldValue { + log.Error(fmt.Sprintf("Old value is not enabled when using `%s` protocol. "+ + "Please update changefeed config", protocol.String())) + return nil, cerror.WrapError(cerror.ErrKafkaInvalidConfig, errors.New(fmt.Sprintf("%s protocol requires old value to be enabled", protocol.String()))) } encoderBuilder, err := codec.NewEventBatchEncoderBuilder(protocol, credential, opts) diff --git a/pkg/cmd/cli/cli_changefeed_create.go b/pkg/cmd/cli/cli_changefeed_create.go index 47f4bac173a..ddf4a04229b 100644 --- a/pkg/cmd/cli/cli_changefeed_create.go +++ b/pkg/cmd/cli/cli_changefeed_create.go @@ -46,6 +46,7 @@ import ( // forceEnableOldValueProtocols specifies which protocols need to be forced to enable old value. var forceEnableOldValueProtocols = []string{ "canal", + "canal-json", "maxwell", } @@ -205,9 +206,12 @@ func (o *createChangefeedOptions) completeCfg(ctx context.Context, cmd *cobra.Co } protocol := sinkURIParsed.Query().Get("protocol") + if protocol != "" { + cfg.Sink.Protocol = protocol + } for _, fp := range forceEnableOldValueProtocols { - if protocol == fp { - log.Warn("Attempting to replicate without old value enabled. CDC will enable old value and continue.", zap.String("protocol", protocol)) + if cfg.Sink.Protocol == fp { + log.Warn("Attempting to replicate without old value enabled. CDC will enable old value and continue.", zap.String("protocol", cfg.Sink.Protocol)) cfg.EnableOldValue = true break } From a189f0b3627814ae90a837e737484e950f1b1416 Mon Sep 17 00:00:00 2001 From: sdojjy Date: Tue, 28 Dec 2021 17:27:49 +0800 Subject: [PATCH 28/48] sink(ticdc): cherry pick sink bug fix to release 5.3 (#4083) --- cdc/owner/async_sink_test.go | 2 +- cdc/processor/pipeline/sink.go | 10 +- cdc/processor/pipeline/sink_test.go | 20 +- cdc/processor/pipeline/table.go | 2 +- cdc/processor/processor.go | 2 + cdc/processor/processor_test.go | 2 + cdc/sink/black_hole.go | 6 +- cdc/sink/buffer_sink.go | 58 +- cdc/sink/buffer_sink_test.go | 91 +++ cdc/sink/cdclog/file.go | 4 +- cdc/sink/cdclog/s3.go | 4 +- cdc/sink/common/common.go | 33 +- cdc/sink/common/common_test.go | 65 +- cdc/sink/manager.go | 57 +- cdc/sink/manager_test.go | 65 +- cdc/sink/mq.go | 20 +- cdc/sink/mq_test.go | 10 +- cdc/sink/mysql.go | 86 +-- cdc/sink/mysql_test.go | 868 +++++++---------------- cdc/sink/mysql_worker_test.go | 362 ++++++++++ cdc/sink/simple_mysql_tester.go | 4 +- cdc/sink/sink.go | 4 +- cdc/sink/table_sink.go | 12 +- cmd/kafka-consumer/main.go | 68 +- pkg/applier/redo.go | 36 +- tests/integration_tests/sink_hang/run.sh | 4 +- 26 files changed, 1015 insertions(+), 880 deletions(-) create mode 100644 cdc/sink/buffer_sink_test.go create mode 100644 cdc/sink/mysql_worker_test.go diff --git a/cdc/owner/async_sink_test.go b/cdc/owner/async_sink_test.go index 981c1905e6b..74709d31d74 100644 --- a/cdc/owner/async_sink_test.go +++ b/cdc/owner/async_sink_test.go @@ -65,7 +65,7 @@ func (m *mockSink) Close(ctx context.Context) error { return nil } -func (m *mockSink) Barrier(ctx context.Context) error { +func (m *mockSink) Barrier(ctx context.Context, tableID model.TableID) error { return nil } diff --git a/cdc/processor/pipeline/sink.go b/cdc/processor/pipeline/sink.go index b436df2a050..5c09a6736cf 100644 --- a/cdc/processor/pipeline/sink.go +++ b/cdc/processor/pipeline/sink.go @@ -64,8 +64,9 @@ func (s *TableStatus) Store(new TableStatus) { } type sinkNode struct { - sink sink.Sink - status TableStatus + sink sink.Sink + status TableStatus + tableID model.TableID resolvedTs model.Ts checkpointTs model.Ts @@ -78,8 +79,9 @@ type sinkNode struct { flowController tableFlowController } -func newSinkNode(sink sink.Sink, startTs model.Ts, targetTs model.Ts, flowController tableFlowController) *sinkNode { +func newSinkNode(tableID model.TableID, sink sink.Sink, startTs model.Ts, targetTs model.Ts, flowController tableFlowController) *sinkNode { return &sinkNode{ + tableID: tableID, sink: sink, status: TableStatusInitializing, targetTs: targetTs, @@ -136,7 +138,7 @@ func (n *sinkNode) flushSink(ctx pipeline.NodeContext, resolvedTs model.Ts) (err if err := n.emitRow2Sink(ctx); err != nil { return errors.Trace(err) } - checkpointTs, err := n.sink.FlushRowChangedEvents(ctx, resolvedTs) + checkpointTs, err := n.sink.FlushRowChangedEvents(ctx, n.tableID, resolvedTs) if err != nil { return errors.Trace(err) } diff --git a/cdc/processor/pipeline/sink_test.go b/cdc/processor/pipeline/sink_test.go index 2e6b25cce0e..dc87961ca1e 100644 --- a/cdc/processor/pipeline/sink_test.go +++ b/cdc/processor/pipeline/sink_test.go @@ -76,7 +76,7 @@ func (s *mockSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error panic("unreachable") } -func (s *mockSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { +func (s *mockSink) FlushRowChangedEvents(ctx context.Context, _ model.TableID, resolvedTs uint64) (uint64, error) { s.received = append(s.received, struct { resolvedTs model.Ts row *model.RowChangedEvent @@ -92,7 +92,7 @@ func (s *mockSink) Close(ctx context.Context) error { return nil } -func (s *mockSink) Barrier(ctx context.Context) error { +func (s *mockSink) Barrier(ctx context.Context, tableID model.TableID) error { return nil } @@ -137,7 +137,7 @@ func (s *outputSuite) TestStatus(c *check.C) { }) // test stop at targetTs - node := newSinkNode(&mockSink{}, 0, 10, &mockFlowController{}) + node := newSinkNode(1, &mockSink{}, 0, 10, &mockFlowController{}) c.Assert(node.Init(pipeline.MockNodeContext4Test(ctx, pipeline.Message{}, nil)), check.IsNil) c.Assert(node.Status(), check.Equals, TableStatusInitializing) @@ -163,7 +163,7 @@ func (s *outputSuite) TestStatus(c *check.C) { c.Assert(node.CheckpointTs(), check.Equals, uint64(10)) // test the stop at ts command - node = newSinkNode(&mockSink{}, 0, 10, &mockFlowController{}) + node = newSinkNode(1, &mockSink{}, 0, 10, &mockFlowController{}) c.Assert(node.Init(pipeline.MockNodeContext4Test(ctx, pipeline.Message{}, nil)), check.IsNil) c.Assert(node.Status(), check.Equals, TableStatusInitializing) @@ -186,7 +186,7 @@ func (s *outputSuite) TestStatus(c *check.C) { c.Assert(node.CheckpointTs(), check.Equals, uint64(2)) // test the stop at ts command is after then resolvedTs and checkpointTs is greater than stop ts - node = newSinkNode(&mockSink{}, 0, 10, &mockFlowController{}) + node = newSinkNode(1, &mockSink{}, 0, 10, &mockFlowController{}) c.Assert(node.Init(pipeline.MockNodeContext4Test(ctx, pipeline.Message{}, nil)), check.IsNil) c.Assert(node.Status(), check.Equals, TableStatusInitializing) @@ -223,7 +223,7 @@ func (s *outputSuite) TestStopStatus(c *check.C) { }) closeCh := make(chan interface{}, 1) - node := newSinkNode(&mockCloseControlSink{mockSink: mockSink{}, closeCh: closeCh}, 0, 100, &mockFlowController{}) + node := newSinkNode(1, &mockCloseControlSink{mockSink: mockSink{}, closeCh: closeCh}, 0, 100, &mockFlowController{}) c.Assert(node.Init(pipeline.MockNodeContext4Test(ctx, pipeline.Message{}, nil)), check.IsNil) c.Assert(node.Status(), check.Equals, TableStatusInitializing) c.Assert(node.Receive(pipeline.MockNodeContext4Test(ctx, @@ -258,7 +258,7 @@ func (s *outputSuite) TestManyTs(c *check.C) { }, }) sink := &mockSink{} - node := newSinkNode(sink, 0, 10, &mockFlowController{}) + node := newSinkNode(1, sink, 0, 10, &mockFlowController{}) c.Assert(node.Init(pipeline.MockNodeContext4Test(ctx, pipeline.Message{}, nil)), check.IsNil) c.Assert(node.Status(), check.Equals, TableStatusInitializing) @@ -379,7 +379,7 @@ func (s *outputSuite) TestIgnoreEmptyRowChangeEvent(c *check.C) { }, }) sink := &mockSink{} - node := newSinkNode(sink, 0, 10, &mockFlowController{}) + node := newSinkNode(1, sink, 0, 10, &mockFlowController{}) c.Assert(node.Init(pipeline.MockNodeContext4Test(ctx, pipeline.Message{}, nil)), check.IsNil) // empty row, no Columns and PreColumns. @@ -399,7 +399,7 @@ func (s *outputSuite) TestSplitUpdateEventWhenEnableOldValue(c *check.C) { }, }) sink := &mockSink{} - node := newSinkNode(sink, 0, 10, &mockFlowController{}) + node := newSinkNode(1, sink, 0, 10, &mockFlowController{}) c.Assert(node.Init(pipeline.MockNodeContext4Test(ctx, pipeline.Message{}, nil)), check.IsNil) // nil row. @@ -458,7 +458,7 @@ func (s *outputSuite) TestSplitUpdateEventWhenDisableOldValue(c *check.C) { }, }) sink := &mockSink{} - node := newSinkNode(sink, 0, 10, &mockFlowController{}) + node := newSinkNode(1, sink, 0, 10, &mockFlowController{}) c.Assert(node.Init(pipeline.MockNodeContext4Test(ctx, pipeline.Message{}, nil)), check.IsNil) // nil row. diff --git a/cdc/processor/pipeline/table.go b/cdc/processor/pipeline/table.go index eb6b05e8ef5..729b9f58c2e 100644 --- a/cdc/processor/pipeline/table.go +++ b/cdc/processor/pipeline/table.go @@ -206,7 +206,7 @@ func NewTablePipeline(ctx cdcContext.Context, p := pipeline.NewPipeline(ctx, 500*time.Millisecond, runnerSize, defaultOutputChannelSize) sorterNode := newSorterNode(tableName, tableID, replicaInfo.StartTs, flowController, mounter) - sinkNode := newSinkNode(sink, replicaInfo.StartTs, targetTs, flowController) + sinkNode := newSinkNode(tableID, sink, replicaInfo.StartTs, targetTs, flowController) p.AppendNode(ctx, "puller", newPullerNode(tableID, replicaInfo, tableName)) p.AppendNode(ctx, "sorter", sorterNode) diff --git a/cdc/processor/processor.go b/cdc/processor/processor.go index b9ae2843c84..684ccab02f5 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -169,6 +169,8 @@ func (p *processor) tick(ctx cdcContext.Context, state *orchestrator.ChangefeedR if err := p.lazyInit(ctx); err != nil { return nil, errors.Trace(err) } + // sink manager will return this checkpointTs to sink node if sink node resolvedTs flush failed + p.sinkManager.UpdateChangeFeedCheckpointTs(state.Info.GetCheckpointTs(state.Status)) if err := p.handleTableOperation(ctx); err != nil { return nil, errors.Trace(err) } diff --git a/cdc/processor/processor_test.go b/cdc/processor/processor_test.go index d95a737ed70..16f27b9cab5 100644 --- a/cdc/processor/processor_test.go +++ b/cdc/processor/processor_test.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tiflow/cdc/model" tablepipeline "github.com/pingcap/tiflow/cdc/processor/pipeline" "github.com/pingcap/tiflow/cdc/redo" + "github.com/pingcap/tiflow/cdc/sink" cdcContext "github.com/pingcap/tiflow/pkg/context" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/etcd" @@ -48,6 +49,7 @@ func newProcessor4Test( ) *processor { p := newProcessor(ctx) p.lazyInit = func(ctx cdcContext.Context) error { return nil } + p.sinkManager = &sink.Manager{} p.redoManager = redo.NewDisabledManager() p.createTablePipeline = createTablePipeline p.schemaStorage = &mockSchemaStorage{c: c} diff --git a/cdc/sink/black_hole.go b/cdc/sink/black_hole.go index 42e623422f7..051da83bf05 100644 --- a/cdc/sink/black_hole.go +++ b/cdc/sink/black_hole.go @@ -31,7 +31,6 @@ func newBlackHoleSink(ctx context.Context, opts map[string]string) *blackHoleSin type blackHoleSink struct { statistics *Statistics - checkpointTs uint64 accumulated uint64 lastAccumulated uint64 } @@ -46,7 +45,7 @@ func (b *blackHoleSink) EmitRowChangedEvents(ctx context.Context, rows ...*model return nil } -func (b *blackHoleSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { +func (b *blackHoleSink) FlushRowChangedEvents(ctx context.Context, _ model.TableID, resolvedTs uint64) (uint64, error) { log.Debug("BlockHoleSink: FlushRowChangedEvents", zap.Uint64("resolvedTs", resolvedTs)) err := b.statistics.RecordBatchExecution(func() (int, error) { // TODO: add some random replication latency @@ -56,7 +55,6 @@ func (b *blackHoleSink) FlushRowChangedEvents(ctx context.Context, resolvedTs ui return int(batchSize), nil }) b.statistics.PrintStatus(ctx) - atomic.StoreUint64(&b.checkpointTs, resolvedTs) return resolvedTs, err } @@ -79,6 +77,6 @@ func (b *blackHoleSink) Close(ctx context.Context) error { return nil } -func (b *blackHoleSink) Barrier(ctx context.Context) error { +func (b *blackHoleSink) Barrier(ctx context.Context, tableID model.TableID) error { return nil } diff --git a/cdc/sink/buffer_sink.go b/cdc/sink/buffer_sink.go index 73cc3be0b9d..0b19ba96763 100644 --- a/cdc/sink/buffer_sink.go +++ b/cdc/sink/buffer_sink.go @@ -29,11 +29,12 @@ import ( type bufferSink struct { Sink - checkpointTs uint64 - buffer map[model.TableID][]*model.RowChangedEvent - bufferMu sync.Mutex - flushTsChan chan uint64 - drawbackChan chan drawbackMsg + changeFeedCheckpointTs uint64 + tableCheckpointTsMap sync.Map + buffer map[model.TableID][]*model.RowChangedEvent + bufferMu sync.Mutex + flushTsChan chan flushMsg + drawbackChan chan drawbackMsg } func newBufferSink( @@ -42,14 +43,14 @@ func newBufferSink( errCh chan error, checkpointTs model.Ts, drawbackChan chan drawbackMsg, -) Sink { +) *bufferSink { sink := &bufferSink{ Sink: backendSink, // buffer shares the same flow control with table sink - buffer: make(map[model.TableID][]*model.RowChangedEvent), - checkpointTs: checkpointTs, - flushTsChan: make(chan uint64, 128), - drawbackChan: drawbackChan, + buffer: make(map[model.TableID][]*model.RowChangedEvent), + changeFeedCheckpointTs: checkpointTs, + flushTsChan: make(chan flushMsg, 128), + drawbackChan: drawbackChan, } go sink.run(ctx, errCh) return sink @@ -81,8 +82,9 @@ func (b *bufferSink) run(ctx context.Context, errCh chan error) { delete(b.buffer, drawback.tableID) b.bufferMu.Unlock() close(drawback.callback) - case resolvedTs := <-b.flushTsChan: + case flushEvent := <-b.flushTsChan: b.bufferMu.Lock() + resolvedTs := flushEvent.resolvedTs // find all rows before resolvedTs and emit to backend sink for tableID, rows := range b.buffer { i := sort.Search(len(rows), func(i int) bool { @@ -109,14 +111,15 @@ func (b *bufferSink) run(ctx context.Context, errCh chan error) { b.bufferMu.Unlock() start := time.Now() - checkpointTs, err := b.Sink.FlushRowChangedEvents(ctx, resolvedTs) + tableID := flushEvent.tableID + checkpointTs, err := b.Sink.FlushRowChangedEvents(ctx, flushEvent.tableID, resolvedTs) if err != nil { if errors.Cause(err) != context.Canceled { errCh <- err } return } - atomic.StoreUint64(&b.checkpointTs, checkpointTs) + b.tableCheckpointTsMap.Store(tableID, checkpointTs) dur := time.Since(start) metricFlushDuration.Observe(dur.Seconds()) @@ -146,11 +149,32 @@ func (b *bufferSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.Ro return nil } -func (b *bufferSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { +func (b *bufferSink) FlushRowChangedEvents(ctx context.Context, tableID model.TableID, resolvedTs uint64) (uint64, error) { select { case <-ctx.Done(): - return atomic.LoadUint64(&b.checkpointTs), ctx.Err() - case b.flushTsChan <- resolvedTs: + return b.getTableCheckpointTs(tableID), ctx.Err() + case b.flushTsChan <- flushMsg{ + tableID: tableID, + resolvedTs: resolvedTs, + }: } - return atomic.LoadUint64(&b.checkpointTs), nil + return b.getTableCheckpointTs(tableID), nil +} + +type flushMsg struct { + tableID model.TableID + resolvedTs uint64 +} + +func (b *bufferSink) getTableCheckpointTs(tableID model.TableID) uint64 { + checkPoints, ok := b.tableCheckpointTsMap.Load(tableID) + if ok { + return checkPoints.(uint64) + } + return atomic.LoadUint64(&b.changeFeedCheckpointTs) +} + +// UpdateChangeFeedCheckpointTs update the changeFeedCheckpointTs every processor tick +func (b *bufferSink) UpdateChangeFeedCheckpointTs(checkpointTs uint64) { + atomic.StoreUint64(&b.changeFeedCheckpointTs, checkpointTs) } diff --git a/cdc/sink/buffer_sink_test.go b/cdc/sink/buffer_sink_test.go new file mode 100644 index 00000000000..e1fe467a0a5 --- /dev/null +++ b/cdc/sink/buffer_sink_test.go @@ -0,0 +1,91 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package sink + +import ( + "context" + "testing" + "time" + + "github.com/pingcap/tiflow/cdc/model" + "github.com/stretchr/testify/require" +) + +func TestTableIsNotFlushed(t *testing.T) { + b := bufferSink{changeFeedCheckpointTs: 1} + require.Equal(t, uint64(1), b.getTableCheckpointTs(2)) + b.UpdateChangeFeedCheckpointTs(3) + require.Equal(t, uint64(3), b.getTableCheckpointTs(2)) +} + +func TestFlushTable(t *testing.T) { + ctx, cancel := context.WithCancel(context.TODO()) + defer func() { + cancel() + }() + b := newBufferSink(ctx, newBlackHoleSink(ctx, make(map[string]string)), make(chan error), 5, make(chan drawbackMsg)) + require.Equal(t, uint64(5), b.getTableCheckpointTs(2)) + require.Nil(t, b.EmitRowChangedEvents(ctx)) + tbl1 := &model.TableName{TableID: 1} + tbl2 := &model.TableName{TableID: 2} + tbl3 := &model.TableName{TableID: 3} + tbl4 := &model.TableName{TableID: 4} + require.Nil(t, b.EmitRowChangedEvents(ctx, []*model.RowChangedEvent{ + {CommitTs: 6, Table: tbl1}, + {CommitTs: 6, Table: tbl2}, + {CommitTs: 6, Table: tbl3}, + {CommitTs: 6, Table: tbl4}, + {CommitTs: 10, Table: tbl1}, + {CommitTs: 10, Table: tbl2}, + {CommitTs: 10, Table: tbl3}, + {CommitTs: 10, Table: tbl4}, + }...)) + checkpoint, err := b.FlushRowChangedEvents(ctx, 1, 7) + require.True(t, checkpoint <= 7) + require.Nil(t, err) + checkpoint, err = b.FlushRowChangedEvents(ctx, 2, 6) + require.True(t, checkpoint <= 6) + require.Nil(t, err) + checkpoint, err = b.FlushRowChangedEvents(ctx, 3, 8) + require.True(t, checkpoint <= 8) + require.Nil(t, err) + time.Sleep(200 * time.Millisecond) + require.Equal(t, uint64(7), b.getTableCheckpointTs(1)) + require.Equal(t, uint64(6), b.getTableCheckpointTs(2)) + require.Equal(t, uint64(8), b.getTableCheckpointTs(3)) + require.Equal(t, uint64(5), b.getTableCheckpointTs(4)) + b.UpdateChangeFeedCheckpointTs(6) + require.Equal(t, uint64(7), b.getTableCheckpointTs(1)) + require.Equal(t, uint64(6), b.getTableCheckpointTs(2)) + require.Equal(t, uint64(8), b.getTableCheckpointTs(3)) + require.Equal(t, uint64(6), b.getTableCheckpointTs(4)) +} + +func TestFlushFailed(t *testing.T) { + ctx, cancel := context.WithCancel(context.TODO()) + b := newBufferSink(ctx, newBlackHoleSink(ctx, make(map[string]string)), make(chan error), 5, make(chan drawbackMsg)) + checkpoint, err := b.FlushRowChangedEvents(ctx, 3, 8) + require.True(t, checkpoint <= 8) + require.Nil(t, err) + time.Sleep(200 * time.Millisecond) + require.Equal(t, uint64(8), b.getTableCheckpointTs(3)) + cancel() + checkpoint, _ = b.FlushRowChangedEvents(ctx, 3, 18) + require.Equal(t, uint64(8), checkpoint) + checkpoint, _ = b.FlushRowChangedEvents(ctx, 1, 18) + require.Equal(t, uint64(5), checkpoint) + time.Sleep(200 * time.Millisecond) + require.Equal(t, uint64(8), b.getTableCheckpointTs(3)) + require.Equal(t, uint64(5), b.getTableCheckpointTs(1)) +} diff --git a/cdc/sink/cdclog/file.go b/cdc/sink/cdclog/file.go index ee57b4e3450..23367e98b29 100644 --- a/cdc/sink/cdclog/file.go +++ b/cdc/sink/cdclog/file.go @@ -225,7 +225,7 @@ func (f *fileSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowC return f.emitRowChangedEvents(ctx, newTableStream, rows...) } -func (f *fileSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { +func (f *fileSink) FlushRowChangedEvents(ctx context.Context, tableID model.TableID, resolvedTs uint64) (uint64, error) { log.Debug("[FlushRowChangedEvents] enter", zap.Uint64("ts", resolvedTs)) return f.flushRowChangedEvents(ctx, resolvedTs) } @@ -349,7 +349,7 @@ func (f *fileSink) Close(ctx context.Context) error { return nil } -func (f *fileSink) Barrier(ctx context.Context) error { +func (f *fileSink) Barrier(ctx context.Context, tableID model.TableID) error { // Barrier does nothing because FlushRowChangedEvents in file sink has flushed // all buffered events forcedlly. return nil diff --git a/cdc/sink/cdclog/s3.go b/cdc/sink/cdclog/s3.go index 2b6da88cb1f..dc5b26b2791 100644 --- a/cdc/sink/cdclog/s3.go +++ b/cdc/sink/cdclog/s3.go @@ -222,7 +222,7 @@ func (s *s3Sink) flushLogMeta(ctx context.Context) error { return cerror.WrapError(cerror.ErrS3SinkWriteStorage, s.storage.WriteFile(ctx, logMetaFile, data)) } -func (s *s3Sink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { +func (s *s3Sink) FlushRowChangedEvents(ctx context.Context, tableID model.TableID, resolvedTs uint64) (uint64, error) { // we should flush all events before resolvedTs, there are two kind of flush policy // 1. flush row events to a s3 chunk: if the event size is not enough, // TODO: when cdc crashed, we should repair these chunks to a complete file @@ -347,7 +347,7 @@ func (s *s3Sink) Close(ctx context.Context) error { return nil } -func (s *s3Sink) Barrier(ctx context.Context) error { +func (s *s3Sink) Barrier(ctx context.Context, tableID model.TableID) error { // Barrier does nothing because FlushRowChangedEvents in s3 sink has flushed // all buffered events forcedlly. return nil diff --git a/cdc/sink/common/common.go b/cdc/sink/common/common.go index cf018a7c8b5..7fff1ec0082 100644 --- a/cdc/sink/common/common.go +++ b/cdc/sink/common/common.go @@ -16,7 +16,6 @@ package common import ( "sort" "sync" - "sync/atomic" "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" @@ -55,7 +54,6 @@ func (t *txnsWithTheSameCommitTs) Append(row *model.RowChangedEvent) { type UnresolvedTxnCache struct { unresolvedTxnsMu sync.Mutex unresolvedTxns map[model.TableID][]*txnsWithTheSameCommitTs - checkpointTs uint64 } // NewUnresolvedTxnCache returns a new UnresolvedTxnCache @@ -103,32 +101,27 @@ func (c *UnresolvedTxnCache) Append(filter *filter.Filter, rows ...*model.RowCha // Resolved returns resolved txns according to resolvedTs // The returned map contains many txns grouped by tableID. for each table, the each commitTs of txn in txns slice is strictly increasing -func (c *UnresolvedTxnCache) Resolved(resolvedTs uint64) map[model.TableID][]*model.SingleTableTxn { - if resolvedTs <= atomic.LoadUint64(&c.checkpointTs) { - return nil - } - +func (c *UnresolvedTxnCache) Resolved(resolvedTsMap *sync.Map) (map[model.TableID]uint64, map[model.TableID][]*model.SingleTableTxn) { c.unresolvedTxnsMu.Lock() defer c.unresolvedTxnsMu.Unlock() if len(c.unresolvedTxns) == 0 { - return nil + return nil, nil } - _, resolvedTxnsMap := splitResolvedTxn(resolvedTs, c.unresolvedTxns) - return resolvedTxnsMap -} - -// UpdateCheckpoint updates the checkpoint ts -func (c *UnresolvedTxnCache) UpdateCheckpoint(checkpointTs uint64) { - atomic.StoreUint64(&c.checkpointTs, checkpointTs) + return splitResolvedTxn(resolvedTsMap, c.unresolvedTxns) } func splitResolvedTxn( - resolvedTs uint64, unresolvedTxns map[model.TableID][]*txnsWithTheSameCommitTs, -) (minTs uint64, resolvedRowsMap map[model.TableID][]*model.SingleTableTxn) { + resolvedTsMap *sync.Map, unresolvedTxns map[model.TableID][]*txnsWithTheSameCommitTs, +) (flushedResolvedTsMap map[model.TableID]uint64, resolvedRowsMap map[model.TableID][]*model.SingleTableTxn) { resolvedRowsMap = make(map[model.TableID][]*model.SingleTableTxn, len(unresolvedTxns)) - minTs = resolvedTs + flushedResolvedTsMap = make(map[model.TableID]uint64, len(unresolvedTxns)) for tableID, txns := range unresolvedTxns { + v, ok := resolvedTsMap.Load(tableID) + if !ok { + continue + } + resolvedTs := v.(uint64) i := sort.Search(len(txns), func(i int) bool { return txns[i].commitTs > resolvedTs }) @@ -154,9 +147,7 @@ func splitResolvedTxn( } } resolvedRowsMap[tableID] = resolvedTxns - if len(resolvedTxnsWithTheSameCommitTs) > 0 && resolvedTxnsWithTheSameCommitTs[0].commitTs < minTs { - minTs = resolvedTxnsWithTheSameCommitTs[0].commitTs - } + flushedResolvedTsMap[tableID] = resolvedTs } return } diff --git a/cdc/sink/common/common_test.go b/cdc/sink/common/common_test.go index 4cadba85e56..fb8b7f26e6d 100644 --- a/cdc/sink/common/common_test.go +++ b/cdc/sink/common/common_test.go @@ -15,26 +15,22 @@ package common import ( "sort" + "sync" "testing" "github.com/google/go-cmp/cmp" - "github.com/pingcap/check" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" ) -type SinkCommonSuite struct{} +func TestSplitResolvedTxn(test *testing.T) { + defer testleak.AfterTestT(test)() -func Test(t *testing.T) { check.TestingT(t) } - -var _ = check.Suite(&SinkCommonSuite{}) - -func (s SinkCommonSuite) TestSplitResolvedTxn(c *check.C) { - defer testleak.AfterTest(c)() testCases := [][]struct { - input []*model.RowChangedEvent - resolvedTs model.Ts - expected map[model.TableID][]*model.SingleTableTxn + input []*model.RowChangedEvent + resolvedTsMap map[model.TableID]uint64 + expected map[model.TableID][]*model.SingleTableTxn }{{{ // Testing basic transaction collocation, no txns with the same committs input: []*model.RowChangedEvent{ {StartTs: 1, CommitTs: 5, Table: &model.TableName{TableID: 1}}, @@ -45,7 +41,10 @@ func (s SinkCommonSuite) TestSplitResolvedTxn(c *check.C) { {StartTs: 1, CommitTs: 11, Table: &model.TableName{TableID: 1}}, {StartTs: 1, CommitTs: 12, Table: &model.TableName{TableID: 2}}, }, - resolvedTs: 6, + resolvedTsMap: map[model.TableID]uint64{ + 1: uint64(6), + 2: uint64(6), + }, expected: map[model.TableID][]*model.SingleTableTxn{ 1: {{Table: &model.TableName{TableID: 1}, StartTs: 1, CommitTs: 5, Rows: []*model.RowChangedEvent{ {StartTs: 1, CommitTs: 5, Table: &model.TableName{TableID: 1}}, @@ -59,7 +58,11 @@ func (s SinkCommonSuite) TestSplitResolvedTxn(c *check.C) { input: []*model.RowChangedEvent{ {StartTs: 1, CommitTs: 8, Table: &model.TableName{TableID: 3}}, }, - resolvedTs: 13, + resolvedTsMap: map[model.TableID]uint64{ + 1: uint64(13), + 2: uint64(13), + 3: uint64(13), + }, expected: map[model.TableID][]*model.SingleTableTxn{ 1: {{Table: &model.TableName{TableID: 1}, StartTs: 1, CommitTs: 8, Rows: []*model.RowChangedEvent{ {StartTs: 1, CommitTs: 8, Table: &model.TableName{TableID: 1}}, @@ -76,17 +79,24 @@ func (s SinkCommonSuite) TestSplitResolvedTxn(c *check.C) { }}}, }, }}, {{ // Testing the short circuit path - input: []*model.RowChangedEvent{}, - resolvedTs: 6, - expected: nil, + input: []*model.RowChangedEvent{}, + resolvedTsMap: map[model.TableID]uint64{ + 1: uint64(13), + 2: uint64(13), + 3: uint64(13), + }, + expected: nil, }, { input: []*model.RowChangedEvent{ {StartTs: 1, CommitTs: 11, Table: &model.TableName{TableID: 1}}, {StartTs: 1, CommitTs: 12, Table: &model.TableName{TableID: 1}}, {StartTs: 1, CommitTs: 13, Table: &model.TableName{TableID: 2}}, }, - resolvedTs: 6, - expected: map[model.TableID][]*model.SingleTableTxn{}, + resolvedTsMap: map[model.TableID]uint64{ + 1: uint64(6), + 2: uint64(6), + }, + expected: map[model.TableID][]*model.SingleTableTxn{}, }}, {{ // Testing the txns with the same commitTs input: []*model.RowChangedEvent{ {StartTs: 1, CommitTs: 5, Table: &model.TableName{TableID: 1}}, @@ -99,7 +109,10 @@ func (s SinkCommonSuite) TestSplitResolvedTxn(c *check.C) { {StartTs: 1, CommitTs: 6, Table: &model.TableName{TableID: 2}}, {StartTs: 1, CommitTs: 7, Table: &model.TableName{TableID: 2}}, }, - resolvedTs: 6, + resolvedTsMap: map[model.TableID]uint64{ + 1: uint64(6), + 2: uint64(6), + }, expected: map[model.TableID][]*model.SingleTableTxn{ 1: {{Table: &model.TableName{TableID: 1}, StartTs: 1, CommitTs: 5, Rows: []*model.RowChangedEvent{ {StartTs: 1, CommitTs: 5, Table: &model.TableName{TableID: 1}}, @@ -119,7 +132,10 @@ func (s SinkCommonSuite) TestSplitResolvedTxn(c *check.C) { {StartTs: 2, CommitTs: 8, Table: &model.TableName{TableID: 1}}, {StartTs: 1, CommitTs: 9, Table: &model.TableName{TableID: 1}}, }, - resolvedTs: 13, + resolvedTsMap: map[model.TableID]uint64{ + 1: uint64(13), + 2: uint64(13), + }, expected: map[model.TableID][]*model.SingleTableTxn{ 1: {{Table: &model.TableName{TableID: 1}, StartTs: 1, CommitTs: 8, Rows: []*model.RowChangedEvent{ {StartTs: 1, CommitTs: 8, Table: &model.TableName{TableID: 1}}, @@ -144,7 +160,11 @@ func (s SinkCommonSuite) TestSplitResolvedTxn(c *check.C) { cache := NewUnresolvedTxnCache() for _, t := range tc { cache.Append(nil, t.input...) - resolved := cache.Resolved(t.resolvedTs) + resolvedTsMap := sync.Map{} + for tableID, ts := range t.resolvedTsMap { + resolvedTsMap.Store(tableID, ts) + } + _, resolved := cache.Resolved(&resolvedTsMap) for tableID, txns := range resolved { sort.Slice(txns, func(i, j int) bool { if txns[i].CommitTs != txns[j].CommitTs { @@ -154,8 +174,7 @@ func (s SinkCommonSuite) TestSplitResolvedTxn(c *check.C) { }) resolved[tableID] = txns } - c.Assert(resolved, check.DeepEquals, t.expected, - check.Commentf("%s", cmp.Diff(resolved, t.expected))) + require.Equal(test, t.expected, resolved, cmp.Diff(resolved, t.expected)) } } } diff --git a/cdc/sink/manager.go b/cdc/sink/manager.go index dd09ccd412c..c1bfc8e8d1f 100644 --- a/cdc/sink/manager.go +++ b/cdc/sink/manager.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" - redo "github.com/pingcap/tiflow/cdc/redo" + "github.com/pingcap/tiflow/cdc/redo" "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" ) @@ -34,10 +34,11 @@ const ( // Manager manages table sinks, maintains the relationship between table sinks and backendSink. type Manager struct { - backendSink Sink - checkpointTs model.Ts - tableSinks map[model.TableID]*tableSink - tableSinksMu sync.Mutex + backendSink *bufferSink + tableCheckpointTsMap sync.Map + tableSinks map[model.TableID]*tableSink + tableSinksMu sync.Mutex + changeFeedCheckpointTs uint64 flushMu sync.Mutex flushing int64 @@ -57,7 +58,7 @@ func NewManager( drawbackChan := make(chan drawbackMsg, 16) return &Manager{ backendSink: newBufferSink(ctx, backendSink, errCh, checkpointTs, drawbackChan), - checkpointTs: checkpointTs, + changeFeedCheckpointTs: checkpointTs, tableSinks: make(map[model.TableID]*tableSink), drawbackChan: drawbackChan, captureAddr: captureAddr, @@ -87,18 +88,21 @@ func (m *Manager) CreateTableSink(tableID model.TableID, checkpointTs model.Ts, // Close closes the Sink manager and backend Sink, this method can be reentrantly called func (m *Manager) Close(ctx context.Context) error { tableSinkTotalRowsCountCounter.DeleteLabelValues(m.captureAddr, m.changefeedID) - return m.backendSink.Close(ctx) + if m.backendSink != nil { + return m.backendSink.Close(ctx) + } + return nil } -func (m *Manager) getMinEmittedTs() model.Ts { +func (m *Manager) getMinEmittedTs(tableID model.TableID) model.Ts { m.tableSinksMu.Lock() defer m.tableSinksMu.Unlock() if len(m.tableSinks) == 0 { - return m.getCheckpointTs() + return m.getCheckpointTs(tableID) } minTs := model.Ts(math.MaxUint64) - for _, tableSink := range m.tableSinks { - resolvedTs := tableSink.getResolvedTs() + for _, tblSink := range m.tableSinks { + resolvedTs := tblSink.getResolvedTs() if minTs > resolvedTs { minTs = resolvedTs } @@ -106,7 +110,7 @@ func (m *Manager) getMinEmittedTs() model.Ts { return minTs } -func (m *Manager) flushBackendSink(ctx context.Context) (model.Ts, error) { +func (m *Manager) flushBackendSink(ctx context.Context, tableID model.TableID) (model.Ts, error) { // NOTICE: Because all table sinks will try to flush backend sink, // which will cause a lot of lock contention and blocking in high concurrency cases. // So here we use flushing as a lightweight lock to improve the lock competition problem. @@ -114,19 +118,19 @@ func (m *Manager) flushBackendSink(ctx context.Context) (model.Ts, error) { // Do not skip flushing for resolving #3503. // TODO uncomment the following return. // if !atomic.CompareAndSwapInt64(&m.flushing, 0, 1) { - // return m.getCheckpointTs(), nil + // return m.getCheckpointTs(tableID), nil // } m.flushMu.Lock() defer func() { m.flushMu.Unlock() atomic.StoreInt64(&m.flushing, 0) }() - minEmittedTs := m.getMinEmittedTs() - checkpointTs, err := m.backendSink.FlushRowChangedEvents(ctx, minEmittedTs) + minEmittedTs := m.getMinEmittedTs(tableID) + checkpointTs, err := m.backendSink.FlushRowChangedEvents(ctx, tableID, minEmittedTs) if err != nil { - return m.getCheckpointTs(), errors.Trace(err) + return m.getCheckpointTs(tableID), errors.Trace(err) } - atomic.StoreUint64(&m.checkpointTs, checkpointTs) + m.tableCheckpointTsMap.Store(tableID, checkpointTs) return checkpointTs, nil } @@ -145,11 +149,24 @@ func (m *Manager) destroyTableSink(ctx context.Context, tableID model.TableID) e return ctx.Err() case <-callback: } - return m.backendSink.Barrier(ctx) + return m.backendSink.Barrier(ctx, tableID) } -func (m *Manager) getCheckpointTs() uint64 { - return atomic.LoadUint64(&m.checkpointTs) +func (m *Manager) getCheckpointTs(tableID model.TableID) uint64 { + checkPoints, ok := m.tableCheckpointTsMap.Load(tableID) + if ok { + return checkPoints.(uint64) + } + // cannot find table level checkpointTs because of no table level resolvedTs flush task finished successfully, + // for example: first time to flush resolvedTs but cannot get the flush lock, return changefeed level checkpointTs is safe + return atomic.LoadUint64(&m.changeFeedCheckpointTs) +} + +func (m *Manager) UpdateChangeFeedCheckpointTs(checkpointTs uint64) { + atomic.StoreUint64(&m.changeFeedCheckpointTs, checkpointTs) + if m.backendSink != nil { + m.backendSink.UpdateChangeFeedCheckpointTs(checkpointTs) + } } type drawbackMsg struct { diff --git a/cdc/sink/manager_test.go b/cdc/sink/manager_test.go index 912100b9eb6..e04d4ec875e 100644 --- a/cdc/sink/manager_test.go +++ b/cdc/sink/manager_test.go @@ -35,9 +35,17 @@ var _ = check.Suite(&managerSuite{}) type checkSink struct { *check.C - rows []*model.RowChangedEvent + rows map[model.TableID][]*model.RowChangedEvent rowsMu sync.Mutex - lastResolvedTs uint64 + lastResolvedTs map[model.TableID]uint64 +} + +func newCheckSink(c *check.C) *checkSink { + return &checkSink{ + C: c, + rows: make(map[model.TableID][]*model.RowChangedEvent), + lastResolvedTs: make(map[model.TableID]uint64), + } } func (c *checkSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error { @@ -47,7 +55,9 @@ func (c *checkSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTab func (c *checkSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { c.rowsMu.Lock() defer c.rowsMu.Unlock() - c.rows = append(c.rows, rows...) + for _, row := range rows { + c.rows[row.Table.TableID] = append(c.rows[row.Table.TableID], row) + } return nil } @@ -55,24 +65,25 @@ func (c *checkSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error panic("unreachable") } -func (c *checkSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { +func (c *checkSink) FlushRowChangedEvents(ctx context.Context, tableID model.TableID, resolvedTs uint64) (uint64, error) { c.rowsMu.Lock() defer c.rowsMu.Unlock() var newRows []*model.RowChangedEvent - for _, row := range c.rows { - if row.CommitTs <= c.lastResolvedTs { - return c.lastResolvedTs, errors.Errorf("commit-ts(%d) is not greater than lastResolvedTs(%d)", row.CommitTs, c.lastResolvedTs) + rows := c.rows[tableID] + for _, row := range rows { + if row.CommitTs <= c.lastResolvedTs[tableID] { + return c.lastResolvedTs[tableID], errors.Errorf("commit-ts(%d) is not greater than lastResolvedTs(%d)", row.CommitTs, c.lastResolvedTs) } if row.CommitTs > resolvedTs { newRows = append(newRows, row) } } - c.Assert(c.lastResolvedTs, check.LessEqual, resolvedTs) - c.lastResolvedTs = resolvedTs - c.rows = newRows + c.Assert(c.lastResolvedTs[tableID], check.LessEqual, resolvedTs) + c.lastResolvedTs[tableID] = resolvedTs + c.rows[tableID] = newRows - return c.lastResolvedTs, nil + return c.lastResolvedTs[tableID], nil } func (c *checkSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { @@ -83,7 +94,7 @@ func (c *checkSink) Close(ctx context.Context) error { return nil } -func (c *checkSink) Barrier(ctx context.Context) error { +func (c *checkSink) Barrier(ctx context.Context, tableID model.TableID) error { return nil } @@ -92,7 +103,7 @@ func (s *managerSuite) TestManagerRandom(c *check.C) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() errCh := make(chan error, 16) - manager := NewManager(ctx, &checkSink{C: c}, errCh, 0, "", "") + manager := NewManager(ctx, newCheckSink(c), errCh, 0, "", "") defer manager.Close(ctx) goroutineNum := 10 rowNum := 100 @@ -118,7 +129,7 @@ func (s *managerSuite) TestManagerRandom(c *check.C) { for j := 1; j < rowNum; j++ { if rand.Intn(10) == 0 { resolvedTs := lastResolvedTs + uint64(rand.Intn(j-int(lastResolvedTs))) - _, err := tableSink.FlushRowChangedEvents(ctx, resolvedTs) + _, err := tableSink.FlushRowChangedEvents(ctx, model.TableID(i), resolvedTs) c.Assert(err, check.IsNil) lastResolvedTs = resolvedTs } else { @@ -129,7 +140,7 @@ func (s *managerSuite) TestManagerRandom(c *check.C) { c.Assert(err, check.IsNil) } } - _, err := tableSink.FlushRowChangedEvents(ctx, uint64(rowNum)) + _, err := tableSink.FlushRowChangedEvents(ctx, model.TableID(i), uint64(rowNum)) c.Assert(err, check.IsNil) }() } @@ -147,7 +158,7 @@ func (s *managerSuite) TestManagerAddRemoveTable(c *check.C) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() errCh := make(chan error, 16) - manager := NewManager(ctx, &checkSink{C: c}, errCh, 0, "", "") + manager := NewManager(ctx, newCheckSink(c), errCh, 0, "", "") defer manager.Close(ctx) goroutineNum := 200 var wg sync.WaitGroup @@ -180,7 +191,7 @@ func (s *managerSuite) TestManagerAddRemoveTable(c *check.C) { }) c.Assert(err, check.IsNil) } - _, err := sink.FlushRowChangedEvents(ctx, resolvedTs) + _, err := sink.FlushRowChangedEvents(ctx, sink.(*tableSink).tableID, resolvedTs) if err != nil { c.Assert(errors.Cause(err), check.Equals, context.Canceled) } @@ -234,7 +245,7 @@ func (s *managerSuite) TestManagerDestroyTableSink(c *check.C) { defer cancel() errCh := make(chan error, 16) - manager := NewManager(ctx, &checkSink{C: c}, errCh, 0, "", "") + manager := NewManager(ctx, newCheckSink(c), errCh, 0, "", "") defer manager.Close(ctx) tableID := int64(49) @@ -244,7 +255,7 @@ func (s *managerSuite) TestManagerDestroyTableSink(c *check.C) { CommitTs: uint64(110), }) c.Assert(err, check.IsNil) - _, err = tableSink.FlushRowChangedEvents(ctx, 110) + _, err = tableSink.FlushRowChangedEvents(ctx, tableID, 110) c.Assert(err, check.IsNil) err = manager.destroyTableSink(ctx, tableID) c.Assert(err, check.IsNil) @@ -255,7 +266,7 @@ func (s *managerSuite) TestManagerDestroyTableSink(c *check.C) { func BenchmarkManagerFlushing(b *testing.B) { ctx, cancel := context.WithCancel(context.Background()) errCh := make(chan error, 16) - manager := NewManager(ctx, &checkSink{}, errCh, 0, "", "") + manager := NewManager(ctx, newCheckSink(nil), errCh, 0, "", "") // Init table sinks. goroutineNum := 2000 @@ -295,11 +306,11 @@ func BenchmarkManagerFlushing(b *testing.B) { // All tables are flushed concurrently, except table 0. for i := 1; i < goroutineNum; i++ { i := i - tableSink := tableSinks[i] + tblSink := tableSinks[i] go func() { for j := 1; j < rowNum; j++ { if j%2 == 0 { - _, err := tableSink.FlushRowChangedEvents(context.Background(), uint64(j)) + _, err := tblSink.FlushRowChangedEvents(context.Background(), tblSink.(*tableSink).tableID, uint64(j)) if err != nil { b.Error(err) } @@ -310,9 +321,9 @@ func BenchmarkManagerFlushing(b *testing.B) { b.ResetTimer() // Table 0 flush. - tableSink := tableSinks[0] + tblSink := tableSinks[0] for i := 0; i < b.N; i++ { - _, err := tableSink.FlushRowChangedEvents(context.Background(), uint64(rowNum)) + _, err := tblSink.FlushRowChangedEvents(context.Background(), tblSink.(*tableSink).tableID, uint64(rowNum)) if err != nil { b.Error(err) } @@ -345,7 +356,7 @@ func (e *errorSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error panic("unreachable") } -func (e *errorSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { +func (e *errorSink) FlushRowChangedEvents(ctx context.Context, tableID model.TableID, resolvedTs uint64) (uint64, error) { return 0, errors.New("error in flush row changed events") } @@ -357,7 +368,7 @@ func (e *errorSink) Close(ctx context.Context) error { return nil } -func (e *errorSink) Barrier(ctx context.Context) error { +func (e *errorSink) Barrier(ctx context.Context, tableID model.TableID) error { return nil } @@ -374,7 +385,7 @@ func (s *managerSuite) TestManagerError(c *check.C) { Table: &model.TableName{TableID: 1}, }) c.Assert(err, check.IsNil) - _, err = sink.FlushRowChangedEvents(ctx, 2) + _, err = sink.FlushRowChangedEvents(ctx, 1, 2) c.Assert(err, check.IsNil) err = <-errCh c.Assert(err.Error(), check.Equals, "error in emit row changed events") diff --git a/cdc/sink/mq.go b/cdc/sink/mq.go index 862acf26a7b..fa92bb62f75 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -55,10 +55,9 @@ type mqSink struct { partitionNum int32 partitionInput []chan mqEvent partitionResolvedTs []uint64 - - checkpointTs uint64 - resolvedNotifier *notify.Notifier - resolvedReceiver *notify.Receiver + tableCheckpointTs map[model.TableID]uint64 + resolvedNotifier *notify.Notifier + resolvedReceiver *notify.Receiver statistics *Statistics } @@ -111,6 +110,7 @@ func newMqSink( partitionNum: partitionNum, partitionInput: partitionInput, partitionResolvedTs: make([]uint64, partitionNum), + tableCheckpointTs: make(map[model.TableID]uint64), resolvedNotifier: notifier, resolvedReceiver: resolvedReceiver, @@ -153,9 +153,9 @@ func (k *mqSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowCha return nil } -func (k *mqSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { - if resolvedTs <= k.checkpointTs { - return k.checkpointTs, nil +func (k *mqSink) FlushRowChangedEvents(ctx context.Context, tableID model.TableID, resolvedTs uint64) (uint64, error) { + if checkpointTs, ok := k.tableCheckpointTs[tableID]; ok && resolvedTs <= checkpointTs { + return checkpointTs, nil } for i := 0; i < int(k.partitionNum); i++ { @@ -188,9 +188,9 @@ flushLoop: if err != nil { return 0, errors.Trace(err) } - k.checkpointTs = resolvedTs + k.tableCheckpointTs[tableID] = resolvedTs k.statistics.PrintStatus(ctx) - return k.checkpointTs, nil + return resolvedTs, nil } func (k *mqSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { @@ -249,7 +249,7 @@ func (k *mqSink) Close(ctx context.Context) error { return errors.Trace(err) } -func (k *mqSink) Barrier(cxt context.Context) error { +func (k *mqSink) Barrier(cxt context.Context, tableID model.TableID) error { // Barrier does nothing because FlushRowChangedEvents in mq sink has flushed // all buffered events by force. return nil diff --git a/cdc/sink/mq_test.go b/cdc/sink/mq_test.go index c94152b7e05..4a2a6730138 100644 --- a/cdc/sink/mq_test.go +++ b/cdc/sink/mq_test.go @@ -74,10 +74,12 @@ func (s mqSinkSuite) TestKafkaSink(c *check.C) { // mock kafka broker processes 1 row changed event leader.Returns(prodSuccess) + tableID := model.TableID(1) row := &model.RowChangedEvent{ Table: &model.TableName{ - Schema: "test", - Table: "t1", + Schema: "test", + Table: "t1", + TableID: tableID, }, StartTs: 100, CommitTs: 120, @@ -85,11 +87,11 @@ func (s mqSinkSuite) TestKafkaSink(c *check.C) { } err = sink.EmitRowChangedEvents(ctx, row) c.Assert(err, check.IsNil) - checkpointTs, err := sink.FlushRowChangedEvents(ctx, uint64(120)) + checkpointTs, err := sink.FlushRowChangedEvents(ctx, tableID, uint64(120)) c.Assert(err, check.IsNil) c.Assert(checkpointTs, check.Equals, uint64(120)) // flush older resolved ts - checkpointTs, err = sink.FlushRowChangedEvents(ctx, uint64(110)) + checkpointTs, err = sink.FlushRowChangedEvents(ctx, tableID, uint64(110)) c.Assert(err, check.IsNil) c.Assert(checkpointTs, check.Equals, uint64(120)) diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index 044face3385..edfc0cbecec 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -22,7 +22,6 @@ import ( "strconv" "strings" "sync" - "sync/atomic" "time" dmysql "github.com/go-sql-driver/mysql" @@ -92,10 +91,10 @@ type mysqlSink struct { filter *tifilter.Filter cyclic *cyclic.Cyclic - txnCache *common.UnresolvedTxnCache - workers []*mysqlSinkWorker - resolvedTs uint64 - maxResolvedTs uint64 + txnCache *common.UnresolvedTxnCache + workers []*mysqlSinkWorker + tableCheckpointTs sync.Map + tableMaxResolvedTs sync.Map execWaitNotifier *notify.Notifier resolvedNotifier *notify.Notifier @@ -130,13 +129,11 @@ func (s *mysqlSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.Row // FlushRowChangedEvents will flushes all received events, we don't allow mysql // sink to receive events before resolving -func (s *mysqlSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { - if atomic.LoadUint64(&s.maxResolvedTs) < resolvedTs { - atomic.StoreUint64(&s.maxResolvedTs, resolvedTs) +func (s *mysqlSink) FlushRowChangedEvents(ctx context.Context, tableID model.TableID, resolvedTs uint64) (uint64, error) { + v, ok := s.tableMaxResolvedTs.Load(tableID) + if !ok || v.(uint64) < resolvedTs { + s.tableMaxResolvedTs.Store(tableID, resolvedTs) } - // resolvedTs can be fallen back, such as a new table is added into this sink - // with a smaller start-ts - atomic.StoreUint64(&s.resolvedTs, resolvedTs) s.resolvedNotifier.Notify() // check and throw error @@ -146,13 +143,7 @@ func (s *mysqlSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64 default: } - checkpointTs := resolvedTs - for _, worker := range s.workers { - workerCheckpointTs := atomic.LoadUint64(&worker.checkpointTs) - if workerCheckpointTs < checkpointTs { - checkpointTs = workerCheckpointTs - } - } + checkpointTs := s.getTableCheckpointTs(tableID) s.statistics.PrintStatus(ctx) return checkpointTs, nil } @@ -169,13 +160,12 @@ func (s *mysqlSink) flushRowChangedEvents(ctx context.Context, receiver *notify. return case <-receiver.C: } - resolvedTs := atomic.LoadUint64(&s.resolvedTs) - resolvedTxnsMap := s.txnCache.Resolved(resolvedTs) + flushedResolvedTsMap, resolvedTxnsMap := s.txnCache.Resolved(&s.tableMaxResolvedTs) if len(resolvedTxnsMap) == 0 { - for _, worker := range s.workers { - atomic.StoreUint64(&worker.checkpointTs, resolvedTs) - } - s.txnCache.UpdateCheckpoint(resolvedTs) + s.tableMaxResolvedTs.Range(func(key, value interface{}) bool { + s.tableCheckpointTs.Store(key, value) + return true + }) continue } @@ -187,10 +177,9 @@ func (s *mysqlSink) flushRowChangedEvents(ctx context.Context, receiver *notify. } s.dispatchAndExecTxns(ctx, resolvedTxnsMap) - for _, worker := range s.workers { - atomic.StoreUint64(&worker.checkpointTs, resolvedTs) + for tableID, resolvedTs := range flushedResolvedTsMap { + s.tableCheckpointTs.Store(tableID, resolvedTs) } - s.txnCache.UpdateCheckpoint(resolvedTs) } } @@ -743,7 +732,6 @@ type mysqlSinkWorker struct { execDMLs func(context.Context, []*model.RowChangedEvent, uint64, int) error metricBucketSize prometheus.Counter receiver *notify.Receiver - checkpointTs uint64 closedCh chan struct{} } @@ -786,10 +774,9 @@ func (w *mysqlSinkWorker) appendFinishTxn(wg *sync.WaitGroup) { func (w *mysqlSinkWorker) run(ctx context.Context) (err error) { var ( - toExecRows []*model.RowChangedEvent - replicaID uint64 - txnNum int - lastCommitTs uint64 + toExecRows []*model.RowChangedEvent + replicaID uint64 + txnNum int ) // mark FinishWg before worker exits, all data txns can be omitted. @@ -827,7 +814,6 @@ func (w *mysqlSinkWorker) run(ctx context.Context) (err error) { txnNum = 0 return err } - atomic.StoreUint64(&w.checkpointTs, lastCommitTs) toExecRows = toExecRows[:0] w.metricBucketSize.Add(float64(txnNum)) txnNum = 0 @@ -857,7 +843,6 @@ func (w *mysqlSinkWorker) run(ctx context.Context) (err error) { } replicaID = txn.ReplicaID toExecRows = append(toExecRows, txn.Rows...) - lastCommitTs = txn.CommitTs txnNum++ case <-w.receiver.C: if err := flushRows(); err != nil { @@ -894,7 +879,7 @@ func (s *mysqlSink) Close(ctx context.Context) error { return cerror.WrapError(cerror.ErrMySQLConnectionError, err) } -func (s *mysqlSink) Barrier(ctx context.Context) error { +func (s *mysqlSink) Barrier(ctx context.Context, tableID model.TableID) error { warnDuration := 3 * time.Minute ticker := time.NewTicker(warnDuration) defer ticker.Stop() @@ -903,15 +888,23 @@ func (s *mysqlSink) Barrier(ctx context.Context) error { case <-ctx.Done(): return errors.Trace(ctx.Err()) case <-ticker.C: + maxResolvedTs, ok := s.tableMaxResolvedTs.Load(tableID) log.Warn("Barrier doesn't return in time, may be stuck", - zap.Uint64("resolved-ts", atomic.LoadUint64(&s.maxResolvedTs)), - zap.Uint64("checkpoint-ts", s.checkpointTs())) + zap.Int64("tableID", tableID), + zap.Bool("has resolvedTs", ok), + zap.Any("resolvedTs", maxResolvedTs), + zap.Uint64("checkpointTs", s.getTableCheckpointTs(tableID))) default: - maxResolvedTs := atomic.LoadUint64(&s.maxResolvedTs) - if s.checkpointTs() >= maxResolvedTs { + v, ok := s.tableMaxResolvedTs.Load(tableID) + if !ok { + log.Info("No table resolvedTs is found", zap.Int64("table-id", tableID)) return nil } - checkpointTs, err := s.FlushRowChangedEvents(ctx, maxResolvedTs) + maxResolvedTs := v.(uint64) + if s.getTableCheckpointTs(tableID) >= maxResolvedTs { + return nil + } + checkpointTs, err := s.FlushRowChangedEvents(ctx, tableID, maxResolvedTs) if err != nil { return err } @@ -924,15 +917,12 @@ func (s *mysqlSink) Barrier(ctx context.Context) error { } } -func (s *mysqlSink) checkpointTs() uint64 { - checkpointTs := atomic.LoadUint64(&s.resolvedTs) - for _, worker := range s.workers { - workerCheckpointTs := atomic.LoadUint64(&worker.checkpointTs) - if workerCheckpointTs < checkpointTs { - checkpointTs = workerCheckpointTs - } +func (s *mysqlSink) getTableCheckpointTs(tableID model.TableID) uint64 { + v, ok := s.tableCheckpointTs.Load(tableID) + if ok { + return v.(uint64) } - return checkpointTs + return uint64(0) } func logDMLTxnErr(err error) error { diff --git a/cdc/sink/mysql_test.go b/cdc/sink/mysql_test.go index bc242861eba..e9965144446 100644 --- a/cdc/sink/mysql_test.go +++ b/cdc/sink/mysql_test.go @@ -21,15 +21,12 @@ import ( "net" "net/url" "sort" - "strings" "sync" "testing" "time" "github.com/DATA-DOG/go-sqlmock" - "github.com/davecgh/go-spew/spew" dmysql "github.com/go-sql-driver/mysql" - "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/tidb/infoschema" timodel "github.com/pingcap/tidb/parser/model" @@ -40,21 +37,13 @@ import ( "github.com/pingcap/tiflow/pkg/cyclic/mark" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/filter" - "github.com/pingcap/tiflow/pkg/notify" "github.com/pingcap/tiflow/pkg/retry" - "github.com/pingcap/tiflow/pkg/util/testleak" - "golang.org/x/sync/errgroup" + "github.com/stretchr/testify/require" ) -type MySQLSinkSuite struct{} - -func Test(t *testing.T) { check.TestingT(t) } - -var _ = check.Suite(&MySQLSinkSuite{}) - -func newMySQLSink4Test(ctx context.Context, c *check.C) *mysqlSink { +func newMySQLSink4Test(ctx context.Context, t *testing.T) *mysqlSink { f, err := filter.NewFilter(config.GetDefaultReplicaConfig()) - c.Assert(err, check.IsNil) + require.Nil(t, err) params := defaultParams.Clone() params.batchReplaceEnabled = false return &mysqlSink{ @@ -65,301 +54,7 @@ func newMySQLSink4Test(ctx context.Context, c *check.C) *mysqlSink { } } -func (s MySQLSinkSuite) TestMysqlSinkWorker(c *check.C) { - defer testleak.AfterTest(c)() - testCases := []struct { - txns []*model.SingleTableTxn - expectedOutputRows [][]*model.RowChangedEvent - exportedOutputReplicaIDs []uint64 - maxTxnRow int - }{ - { - txns: []*model.SingleTableTxn{}, - maxTxnRow: 4, - }, { - txns: []*model.SingleTableTxn{ - { - CommitTs: 1, - Rows: []*model.RowChangedEvent{{CommitTs: 1}}, - ReplicaID: 1, - }, - }, - expectedOutputRows: [][]*model.RowChangedEvent{{{CommitTs: 1}}}, - exportedOutputReplicaIDs: []uint64{1}, - maxTxnRow: 2, - }, { - txns: []*model.SingleTableTxn{ - { - CommitTs: 1, - Rows: []*model.RowChangedEvent{{CommitTs: 1}, {CommitTs: 1}, {CommitTs: 1}}, - ReplicaID: 1, - }, - }, - expectedOutputRows: [][]*model.RowChangedEvent{ - {{CommitTs: 1}, {CommitTs: 1}, {CommitTs: 1}}, - }, - exportedOutputReplicaIDs: []uint64{1}, - maxTxnRow: 2, - }, { - txns: []*model.SingleTableTxn{ - { - CommitTs: 1, - Rows: []*model.RowChangedEvent{{CommitTs: 1}, {CommitTs: 1}}, - ReplicaID: 1, - }, - { - CommitTs: 2, - Rows: []*model.RowChangedEvent{{CommitTs: 2}}, - ReplicaID: 1, - }, - { - CommitTs: 3, - Rows: []*model.RowChangedEvent{{CommitTs: 3}, {CommitTs: 3}}, - ReplicaID: 1, - }, - }, - expectedOutputRows: [][]*model.RowChangedEvent{ - {{CommitTs: 1}, {CommitTs: 1}, {CommitTs: 2}}, - {{CommitTs: 3}, {CommitTs: 3}}, - }, - exportedOutputReplicaIDs: []uint64{1, 1}, - maxTxnRow: 4, - }, { - txns: []*model.SingleTableTxn{ - { - CommitTs: 1, - Rows: []*model.RowChangedEvent{{CommitTs: 1}}, - ReplicaID: 1, - }, - { - CommitTs: 2, - Rows: []*model.RowChangedEvent{{CommitTs: 2}}, - ReplicaID: 2, - }, - { - CommitTs: 3, - Rows: []*model.RowChangedEvent{{CommitTs: 3}}, - ReplicaID: 3, - }, - }, - expectedOutputRows: [][]*model.RowChangedEvent{ - {{CommitTs: 1}}, - {{CommitTs: 2}}, - {{CommitTs: 3}}, - }, - exportedOutputReplicaIDs: []uint64{1, 2, 3}, - maxTxnRow: 4, - }, { - txns: []*model.SingleTableTxn{ - { - CommitTs: 1, - Rows: []*model.RowChangedEvent{{CommitTs: 1}}, - ReplicaID: 1, - }, - { - CommitTs: 2, - Rows: []*model.RowChangedEvent{{CommitTs: 2}, {CommitTs: 2}, {CommitTs: 2}}, - ReplicaID: 1, - }, - { - CommitTs: 3, - Rows: []*model.RowChangedEvent{{CommitTs: 3}}, - ReplicaID: 1, - }, - { - CommitTs: 4, - Rows: []*model.RowChangedEvent{{CommitTs: 4}}, - ReplicaID: 1, - }, - }, - expectedOutputRows: [][]*model.RowChangedEvent{ - {{CommitTs: 1}}, - {{CommitTs: 2}, {CommitTs: 2}, {CommitTs: 2}}, - {{CommitTs: 3}, {CommitTs: 4}}, - }, - exportedOutputReplicaIDs: []uint64{1, 1, 1}, - maxTxnRow: 2, - }, - } - ctx := context.Background() - - notifier := new(notify.Notifier) - for i, tc := range testCases { - cctx, cancel := context.WithCancel(ctx) - var outputRows [][]*model.RowChangedEvent - var outputReplicaIDs []uint64 - receiver, err := notifier.NewReceiver(-1) - c.Assert(err, check.IsNil) - w := newMySQLSinkWorker(tc.maxTxnRow, 1, - bucketSizeCounter.WithLabelValues("capture", "changefeed", "1"), - receiver, - func(ctx context.Context, events []*model.RowChangedEvent, replicaID uint64, bucket int) error { - outputRows = append(outputRows, events) - outputReplicaIDs = append(outputReplicaIDs, replicaID) - return nil - }) - errg, cctx := errgroup.WithContext(cctx) - errg.Go(func() error { - return w.run(cctx) - }) - for _, txn := range tc.txns { - w.appendTxn(cctx, txn) - } - var wg sync.WaitGroup - w.appendFinishTxn(&wg) - // ensure all txns are fetched from txn channel in sink worker - time.Sleep(time.Millisecond * 100) - notifier.Notify() - wg.Wait() - cancel() - c.Assert(errors.Cause(errg.Wait()), check.Equals, context.Canceled) - c.Assert(outputRows, check.DeepEquals, tc.expectedOutputRows, - check.Commentf("case %v, %s, %s", i, spew.Sdump(outputRows), spew.Sdump(tc.expectedOutputRows))) - c.Assert(outputReplicaIDs, check.DeepEquals, tc.exportedOutputReplicaIDs, - check.Commentf("case %v, %s, %s", i, spew.Sdump(outputReplicaIDs), spew.Sdump(tc.exportedOutputReplicaIDs))) - } -} - -func (s MySQLSinkSuite) TestMySQLSinkWorkerExitWithError(c *check.C) { - defer testleak.AfterTest(c)() - txns1 := []*model.SingleTableTxn{ - { - CommitTs: 1, - Rows: []*model.RowChangedEvent{{CommitTs: 1}}, - }, - { - CommitTs: 2, - Rows: []*model.RowChangedEvent{{CommitTs: 2}}, - }, - { - CommitTs: 3, - Rows: []*model.RowChangedEvent{{CommitTs: 3}}, - }, - { - CommitTs: 4, - Rows: []*model.RowChangedEvent{{CommitTs: 4}}, - }, - } - txns2 := []*model.SingleTableTxn{ - { - CommitTs: 5, - Rows: []*model.RowChangedEvent{{CommitTs: 5}}, - }, - { - CommitTs: 6, - Rows: []*model.RowChangedEvent{{CommitTs: 6}}, - }, - } - maxTxnRow := 1 - ctx := context.Background() - - errExecFailed := errors.New("sink worker exec failed") - notifier := new(notify.Notifier) - cctx, cancel := context.WithCancel(ctx) - receiver, err := notifier.NewReceiver(-1) - c.Assert(err, check.IsNil) - w := newMySQLSinkWorker(maxTxnRow, 1, /*bucket*/ - bucketSizeCounter.WithLabelValues("capture", "changefeed", "1"), - receiver, - func(ctx context.Context, events []*model.RowChangedEvent, replicaID uint64, bucket int) error { - return errExecFailed - }) - errg, cctx := errgroup.WithContext(cctx) - errg.Go(func() error { - return w.run(cctx) - }) - // txn in txns1 will be sent to worker txnCh - for _, txn := range txns1 { - w.appendTxn(cctx, txn) - } - - // simulate notify sink worker to flush existing txns - var wg sync.WaitGroup - w.appendFinishTxn(&wg) - time.Sleep(time.Millisecond * 100) - // txn in txn2 will be blocked since the worker has exited - for _, txn := range txns2 { - w.appendTxn(cctx, txn) - } - notifier.Notify() - - // simulate sink shutdown and send closed singal to sink worker - w.closedCh <- struct{}{} - w.cleanup() - - // the flush notification wait group should be done - wg.Wait() - - cancel() - c.Assert(errg.Wait(), check.Equals, errExecFailed) -} - -func (s MySQLSinkSuite) TestMySQLSinkWorkerExitCleanup(c *check.C) { - defer testleak.AfterTest(c)() - txns1 := []*model.SingleTableTxn{ - { - CommitTs: 1, - Rows: []*model.RowChangedEvent{{CommitTs: 1}}, - }, - { - CommitTs: 2, - Rows: []*model.RowChangedEvent{{CommitTs: 2}}, - }, - } - txns2 := []*model.SingleTableTxn{ - { - CommitTs: 5, - Rows: []*model.RowChangedEvent{{CommitTs: 5}}, - }, - } - - maxTxnRow := 1 - ctx := context.Background() - - errExecFailed := errors.New("sink worker exec failed") - notifier := new(notify.Notifier) - cctx, cancel := context.WithCancel(ctx) - receiver, err := notifier.NewReceiver(-1) - c.Assert(err, check.IsNil) - w := newMySQLSinkWorker(maxTxnRow, 1, /*bucket*/ - bucketSizeCounter.WithLabelValues("capture", "changefeed", "1"), - receiver, - func(ctx context.Context, events []*model.RowChangedEvent, replicaID uint64, bucket int) error { - return errExecFailed - }) - errg, cctx := errgroup.WithContext(cctx) - errg.Go(func() error { - err := w.run(cctx) - return err - }) - for _, txn := range txns1 { - w.appendTxn(cctx, txn) - } - - // sleep to let txns flushed by tick - time.Sleep(time.Millisecond * 100) - - // simulate more txns are sent to txnCh after the sink worker run has exited - for _, txn := range txns2 { - w.appendTxn(cctx, txn) - } - var wg sync.WaitGroup - w.appendFinishTxn(&wg) - notifier.Notify() - - // simulate sink shutdown and send closed singal to sink worker - w.closedCh <- struct{}{} - w.cleanup() - - // the flush notification wait group should be done - wg.Wait() - - cancel() - c.Assert(errg.Wait(), check.Equals, errExecFailed) -} - -func (s MySQLSinkSuite) TestPrepareDML(c *check.C) { - defer testleak.AfterTest(c)() +func TestPrepareDML(t *testing.T) { testCases := []struct { input []*model.RowChangedEvent expected *preparedDMLs @@ -394,15 +89,14 @@ func (s MySQLSinkSuite) TestPrepareDML(c *check.C) { }} ctx, cancel := context.WithCancel(context.Background()) defer cancel() - ms := newMySQLSink4Test(ctx, c) + ms := newMySQLSink4Test(ctx, t) for i, tc := range testCases { dmls := ms.prepareDMLs(tc.input, 0, 0) - c.Assert(dmls, check.DeepEquals, tc.expected, check.Commentf("%d", i)) + require.Equal(t, tc.expected, dmls, tc.expected, fmt.Sprintf("%d", i)) } } -func (s MySQLSinkSuite) TestPrepareUpdate(c *check.C) { - defer testleak.AfterTest(c)() +func TestPrepareUpdate(t *testing.T) { testCases := []struct { quoteTable string preCols []*model.Column @@ -448,13 +142,12 @@ func (s MySQLSinkSuite) TestPrepareUpdate(c *check.C) { } for _, tc := range testCases { query, args := prepareUpdate(tc.quoteTable, tc.preCols, tc.cols, false) - c.Assert(query, check.Equals, tc.expectedSQL) - c.Assert(args, check.DeepEquals, tc.expectedArgs) + require.Equal(t, tc.expectedSQL, query) + require.Equal(t, tc.expectedArgs, args) } } -func (s MySQLSinkSuite) TestPrepareDelete(c *check.C) { - defer testleak.AfterTest(c)() +func TestPrepareDelete(t *testing.T) { testCases := []struct { quoteTable string preCols []*model.Column @@ -489,13 +182,12 @@ func (s MySQLSinkSuite) TestPrepareDelete(c *check.C) { } for _, tc := range testCases { query, args := prepareDelete(tc.quoteTable, tc.preCols, false) - c.Assert(query, check.Equals, tc.expectedSQL) - c.Assert(args, check.DeepEquals, tc.expectedArgs) + require.Equal(t, tc.expectedSQL, query) + require.Equal(t, tc.expectedArgs, args) } } -func (s MySQLSinkSuite) TestWhereSlice(c *check.C) { - defer testleak.AfterTest(c)() +func TestWhereSlice(t *testing.T) { testCases := []struct { cols []*model.Column forceReplicate bool @@ -574,13 +266,12 @@ func (s MySQLSinkSuite) TestWhereSlice(c *check.C) { } for _, tc := range testCases { colNames, args := whereSlice(tc.cols, tc.forceReplicate) - c.Assert(colNames, check.DeepEquals, tc.expectedColNames) - c.Assert(args, check.DeepEquals, tc.expectedArgs) + require.Equal(t, tc.expectedColNames, colNames) + require.Equal(t, tc.expectedArgs, args) } } -func (s MySQLSinkSuite) TestMapReplace(c *check.C) { - defer testleak.AfterTest(c)() +func TestMapReplace(t *testing.T) { testCases := []struct { quoteTable string cols []*model.Column @@ -614,8 +305,8 @@ func (s MySQLSinkSuite) TestMapReplace(c *check.C) { // multiple times to verify the stability of column sequence in query string for i := 0; i < 10; i++ { query, args := prepareReplace(tc.quoteTable, tc.cols, false, false) - c.Assert(query, check.Equals, tc.expectedQuery) - c.Assert(args, check.DeepEquals, tc.expectedArgs) + require.Equal(t, tc.expectedQuery, query) + require.Equal(t, tc.expectedArgs, args) } } } @@ -626,8 +317,7 @@ func (a sqlArgs) Len() int { return len(a) } func (a sqlArgs) Less(i, j int) bool { return fmt.Sprintf("%s", a[i]) < fmt.Sprintf("%s", a[j]) } func (a sqlArgs) Swap(i, j int) { a[i], a[j] = a[j], a[i] } -func (s MySQLSinkSuite) TestReduceReplace(c *check.C) { - defer testleak.AfterTest(c)() +func TestReduceReplace(t *testing.T) { testCases := []struct { replaces map[string][][]interface{} batchSize int @@ -732,211 +422,11 @@ func (s MySQLSinkSuite) TestReduceReplace(c *check.C) { sort.Strings(sqls) sort.Sort(sqlArgs(args)) } - c.Assert(sqls, check.DeepEquals, tc.expectSQLs) - c.Assert(args, check.DeepEquals, tc.expectArgs) - } -} - -func (s MySQLSinkSuite) TestSinkParamsClone(c *check.C) { - defer testleak.AfterTest(c)() - param1 := defaultParams.Clone() - param2 := param1.Clone() - param2.changefeedID = "123" - param2.batchReplaceEnabled = false - param2.maxTxnRow = 1 - c.Assert(param1, check.DeepEquals, &sinkParams{ - workerCount: DefaultWorkerCount, - maxTxnRow: DefaultMaxTxnRow, - tidbTxnMode: defaultTiDBTxnMode, - batchReplaceEnabled: defaultBatchReplaceEnabled, - batchReplaceSize: defaultBatchReplaceSize, - readTimeout: defaultReadTimeout, - writeTimeout: defaultWriteTimeout, - dialTimeout: defaultDialTimeout, - safeMode: defaultSafeMode, - }) - c.Assert(param2, check.DeepEquals, &sinkParams{ - changefeedID: "123", - workerCount: DefaultWorkerCount, - maxTxnRow: 1, - tidbTxnMode: defaultTiDBTxnMode, - batchReplaceEnabled: false, - batchReplaceSize: defaultBatchReplaceSize, - readTimeout: defaultReadTimeout, - writeTimeout: defaultWriteTimeout, - dialTimeout: defaultDialTimeout, - safeMode: defaultSafeMode, - }) -} - -func (s MySQLSinkSuite) TestConfigureSinkURI(c *check.C) { - defer testleak.AfterTest(c)() - - testDefaultParams := func() { - db, err := mockTestDB() - c.Assert(err, check.IsNil) - defer db.Close() - - dsn, err := dmysql.ParseDSN("root:123456@tcp(127.0.0.1:4000)/") - c.Assert(err, check.IsNil) - params := defaultParams.Clone() - dsnStr, err := configureSinkURI(context.TODO(), dsn, params, db) - c.Assert(err, check.IsNil) - expectedParams := []string{ - "tidb_txn_mode=optimistic", - "readTimeout=2m", - "writeTimeout=2m", - "allow_auto_random_explicit_insert=1", - } - for _, param := range expectedParams { - c.Assert(strings.Contains(dsnStr, param), check.IsTrue) - } - c.Assert(strings.Contains(dsnStr, "time_zone"), check.IsFalse) - } - - testTimezoneParam := func() { - db, err := mockTestDB() - c.Assert(err, check.IsNil) - defer db.Close() - - dsn, err := dmysql.ParseDSN("root:123456@tcp(127.0.0.1:4000)/") - c.Assert(err, check.IsNil) - params := defaultParams.Clone() - params.timezone = `"UTC"` - dsnStr, err := configureSinkURI(context.TODO(), dsn, params, db) - c.Assert(err, check.IsNil) - c.Assert(strings.Contains(dsnStr, "time_zone=%22UTC%22"), check.IsTrue) - } - - testTimeoutParams := func() { - db, err := mockTestDB() - c.Assert(err, check.IsNil) - defer db.Close() - - dsn, err := dmysql.ParseDSN("root:123456@tcp(127.0.0.1:4000)/") - c.Assert(err, check.IsNil) - uri, err := url.Parse("mysql://127.0.0.1:3306/?read-timeout=4m&write-timeout=5m&timeout=3m") - c.Assert(err, check.IsNil) - params, err := parseSinkURI(context.TODO(), uri, map[string]string{}) - c.Assert(err, check.IsNil) - dsnStr, err := configureSinkURI(context.TODO(), dsn, params, db) - c.Assert(err, check.IsNil) - expectedParams := []string{ - "readTimeout=4m", - "writeTimeout=5m", - "timeout=3m", - } - for _, param := range expectedParams { - c.Assert(strings.Contains(dsnStr, param), check.IsTrue) - } - } - - testDefaultParams() - testTimezoneParam() - testTimeoutParams() -} - -func (s MySQLSinkSuite) TestParseSinkURI(c *check.C) { - defer testleak.AfterTest(c)() - expected := defaultParams.Clone() - expected.workerCount = 64 - expected.maxTxnRow = 20 - expected.batchReplaceEnabled = true - expected.batchReplaceSize = 50 - expected.safeMode = true - expected.timezone = `"UTC"` - expected.changefeedID = "cf-id" - expected.captureAddr = "127.0.0.1:8300" - expected.tidbTxnMode = "pessimistic" - uriStr := "mysql://127.0.0.1:3306/?worker-count=64&max-txn-row=20" + - "&batch-replace-enable=true&batch-replace-size=50&safe-mode=true" + - "&tidb-txn-mode=pessimistic" - opts := map[string]string{ - OptChangefeedID: expected.changefeedID, - OptCaptureAddr: expected.captureAddr, - } - uri, err := url.Parse(uriStr) - c.Assert(err, check.IsNil) - params, err := parseSinkURI(context.TODO(), uri, opts) - c.Assert(err, check.IsNil) - c.Assert(params, check.DeepEquals, expected) -} - -func (s MySQLSinkSuite) TestParseSinkURITimezone(c *check.C) { - defer testleak.AfterTest(c)() - uris := []string{ - "mysql://127.0.0.1:3306/?time-zone=Asia/Shanghai&worker-count=32", - "mysql://127.0.0.1:3306/?time-zone=&worker-count=32", - "mysql://127.0.0.1:3306/?worker-count=32", - } - expected := []string{ - "\"Asia/Shanghai\"", - "", - "\"UTC\"", - } - ctx := context.TODO() - opts := map[string]string{} - for i, uriStr := range uris { - uri, err := url.Parse(uriStr) - c.Assert(err, check.IsNil) - params, err := parseSinkURI(ctx, uri, opts) - c.Assert(err, check.IsNil) - c.Assert(params.timezone, check.Equals, expected[i]) - } -} - -func (s MySQLSinkSuite) TestParseSinkURIBadQueryString(c *check.C) { - defer testleak.AfterTest(c)() - uris := []string{ - "", - "postgre://127.0.0.1:3306", - "mysql://127.0.0.1:3306/?worker-count=not-number", - "mysql://127.0.0.1:3306/?max-txn-row=not-number", - "mysql://127.0.0.1:3306/?ssl-ca=only-ca-exists", - "mysql://127.0.0.1:3306/?batch-replace-enable=not-bool", - "mysql://127.0.0.1:3306/?batch-replace-enable=true&batch-replace-size=not-number", - "mysql://127.0.0.1:3306/?safe-mode=not-bool", - } - ctx := context.TODO() - opts := map[string]string{OptChangefeedID: "changefeed-01"} - var uri *url.URL - var err error - for _, uriStr := range uris { - if uriStr != "" { - uri, err = url.Parse(uriStr) - c.Assert(err, check.IsNil) - } else { - uri = nil - } - _, err = parseSinkURI(ctx, uri, opts) - c.Assert(err, check.NotNil) + require.Equal(t, tc.expectSQLs, sqls) + require.Equal(t, tc.expectArgs, args) } } -func (s MySQLSinkSuite) TestCheckTiDBVariable(c *check.C) { - defer testleak.AfterTest(c)() - db, mock, err := sqlmock.New() - c.Assert(err, check.IsNil) - defer db.Close() //nolint:errcheck - columns := []string{"Variable_name", "Value"} - - mock.ExpectQuery("show session variables like 'allow_auto_random_explicit_insert';").WillReturnRows( - sqlmock.NewRows(columns).AddRow("allow_auto_random_explicit_insert", "0"), - ) - val, err := checkTiDBVariable(context.TODO(), db, "allow_auto_random_explicit_insert", "1") - c.Assert(err, check.IsNil) - c.Assert(val, check.Equals, "1") - - mock.ExpectQuery("show session variables like 'no_exist_variable';").WillReturnError(sql.ErrNoRows) - val, err = checkTiDBVariable(context.TODO(), db, "no_exist_variable", "0") - c.Assert(err, check.IsNil) - c.Assert(val, check.Equals, "") - - mock.ExpectQuery("show session variables like 'version';").WillReturnError(sql.ErrConnDone) - _, err = checkTiDBVariable(context.TODO(), db, "version", "5.7.25-TiDB-v4.0.0") - c.Assert(err, check.ErrorMatches, ".*"+sql.ErrConnDone.Error()) -} - func mockTestDB() (*sql.DB, error) { // mock for test db, which is used querying TiDB session variable db, mock, err := sqlmock.New() @@ -954,9 +444,7 @@ func mockTestDB() (*sql.DB, error) { return db, nil } -func (s MySQLSinkSuite) TestAdjustSQLMode(c *check.C) { - defer testleak.AfterTest(c)() - +func TestAdjustSQLMode(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -968,12 +456,12 @@ func (s MySQLSinkSuite) TestAdjustSQLMode(c *check.C) { if dbIndex == 0 { // test db db, err := mockTestDB() - c.Assert(err, check.IsNil) + require.Nil(t, err) return db, nil } // normal db db, mock, err := sqlmock.New(sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual)) - c.Assert(err, check.IsNil) + require.Nil(t, err) mock.ExpectQuery("SELECT @@SESSION.sql_mode;"). WillReturnRows(sqlmock.NewRows([]string{"@@SESSION.sql_mode"}). AddRow("ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE")) @@ -990,7 +478,8 @@ func (s MySQLSinkSuite) TestAdjustSQLMode(c *check.C) { changefeed := "test-changefeed" sinkURI, err := url.Parse("mysql://127.0.0.1:4000/?time-zone=UTC&worker-count=4") - c.Assert(err, check.IsNil) + require.Nil(t, err) + require.Nil(t, err) rc := config.GetDefaultReplicaConfig() rc.Cyclic = &config.CyclicConfig{ Enable: true, @@ -998,17 +487,17 @@ func (s MySQLSinkSuite) TestAdjustSQLMode(c *check.C) { FilterReplicaID: []uint64{2}, } f, err := filter.NewFilter(rc) - c.Assert(err, check.IsNil) + require.Nil(t, err) cyclicConfig, err := rc.Cyclic.Marshal() - c.Assert(err, check.IsNil) + require.Nil(t, err) opts := map[string]string{ mark.OptCyclicConfig: cyclicConfig, } sink, err := newMySQLSink(ctx, changefeed, sinkURI, f, rc, opts) - c.Assert(err, check.IsNil) + require.Nil(t, err) err = sink.Close(ctx) - c.Assert(err, check.IsNil) + require.Nil(t, err) } type mockUnavailableMySQL struct { @@ -1017,19 +506,19 @@ type mockUnavailableMySQL struct { wg sync.WaitGroup } -func newMockUnavailableMySQL(addr string, c *check.C) *mockUnavailableMySQL { +func newMockUnavailableMySQL(addr string, t *testing.T) *mockUnavailableMySQL { s := &mockUnavailableMySQL{ quit: make(chan interface{}), } l, err := net.Listen("tcp", addr) - c.Assert(err, check.IsNil) + require.Nil(t, err) s.listener = l s.wg.Add(1) - go s.serve(c) + go s.serve(t) return s } -func (s *mockUnavailableMySQL) serve(c *check.C) { +func (s *mockUnavailableMySQL) serve(t *testing.T) { defer s.wg.Done() for { @@ -1039,7 +528,7 @@ func (s *mockUnavailableMySQL) serve(c *check.C) { case <-s.quit: return default: - c.Error(err) + require.Error(t, err) } } else { s.wg.Add(1) @@ -1058,28 +547,24 @@ func (s *mockUnavailableMySQL) Stop() { s.wg.Wait() } -func (s MySQLSinkSuite) TestNewMySQLTimeout(c *check.C) { - defer testleak.AfterTest(c)() - +func TestNewMySQLTimeout(t *testing.T) { addr := "127.0.0.1:33333" - mockMySQL := newMockUnavailableMySQL(addr, c) + mockMySQL := newMockUnavailableMySQL(addr, t) defer mockMySQL.Stop() ctx, cancel := context.WithCancel(context.Background()) defer cancel() changefeed := "test-changefeed" sinkURI, err := url.Parse(fmt.Sprintf("mysql://%s/?read-timeout=2s&timeout=2s", addr)) - c.Assert(err, check.IsNil) + require.Nil(t, err) rc := config.GetDefaultReplicaConfig() f, err := filter.NewFilter(rc) - c.Assert(err, check.IsNil) + require.Nil(t, err) _, err = newMySQLSink(ctx, changefeed, sinkURI, f, rc, map[string]string{}) - c.Assert(errors.Cause(err), check.Equals, driver.ErrBadConn) + require.Equal(t, driver.ErrBadConn, errors.Cause(err)) } -func (s MySQLSinkSuite) TestNewMySQLSinkExecDML(c *check.C) { - defer testleak.AfterTest(c)() - +func TestNewMySQLSinkExecDML(t *testing.T) { dbIndex := 0 mockGetDBConn := func(ctx context.Context, dsnStr string) (*sql.DB, error) { defer func() { @@ -1088,12 +573,12 @@ func (s MySQLSinkSuite) TestNewMySQLSinkExecDML(c *check.C) { if dbIndex == 0 { // test db db, err := mockTestDB() - c.Assert(err, check.IsNil) + require.Nil(t, err) return db, nil } // normal db db, mock, err := sqlmock.New(sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual)) - c.Assert(err, check.IsNil) + require.Nil(t, err) mock.ExpectBegin() mock.ExpectExec("REPLACE INTO `s1`.`t1`(`a`,`b`) VALUES (?,?),(?,?)"). WithArgs(1, "test", 2, "test"). @@ -1117,12 +602,12 @@ func (s MySQLSinkSuite) TestNewMySQLSinkExecDML(c *check.C) { defer cancel() changefeed := "test-changefeed" sinkURI, err := url.Parse("mysql://127.0.0.1:4000/?time-zone=UTC&worker-count=4") - c.Assert(err, check.IsNil) + require.Nil(t, err) rc := config.GetDefaultReplicaConfig() f, err := filter.NewFilter(rc) - c.Assert(err, check.IsNil) + require.Nil(t, err) sink, err := newMySQLSink(ctx, changefeed, sinkURI, f, rc, map[string]string{}) - c.Assert(err, check.IsNil) + require.Nil(t, err) rows := []*model.RowChangedEvent{ { @@ -1173,39 +658,38 @@ func (s MySQLSinkSuite) TestNewMySQLSinkExecDML(c *check.C) { } err = sink.EmitRowChangedEvents(ctx, rows...) - c.Assert(err, check.IsNil) + require.Nil(t, err) + // retry to make sure event is flushed err = retry.Do(context.Background(), func() error { - ts, err := sink.FlushRowChangedEvents(ctx, uint64(2)) - c.Assert(err, check.IsNil) + ts, err := sink.FlushRowChangedEvents(ctx, 1, uint64(2)) + require.Nil(t, err) if ts < uint64(2) { return errors.Errorf("checkpoint ts %d less than resolved ts %d", ts, 2) } return nil }, retry.WithBackoffBaseDelay(20), retry.WithMaxTries(10), retry.WithIsRetryableErr(cerror.IsRetryableError)) - c.Assert(err, check.IsNil) + require.Nil(t, err) err = retry.Do(context.Background(), func() error { - ts, err := sink.FlushRowChangedEvents(ctx, uint64(4)) - c.Assert(err, check.IsNil) + ts, err := sink.FlushRowChangedEvents(ctx, 2, uint64(4)) + require.Nil(t, err) if ts < uint64(4) { return errors.Errorf("checkpoint ts %d less than resolved ts %d", ts, 4) } return nil }, retry.WithBackoffBaseDelay(20), retry.WithMaxTries(10), retry.WithIsRetryableErr(cerror.IsRetryableError)) - c.Assert(err, check.IsNil) + require.Nil(t, err) - err = sink.Barrier(ctx) - c.Assert(err, check.IsNil) + err = sink.Barrier(ctx, 2) + require.Nil(t, err) err = sink.Close(ctx) - c.Assert(err, check.IsNil) + require.Nil(t, err) } -func (s MySQLSinkSuite) TestExecDMLRollbackErrDatabaseNotExists(c *check.C) { - defer testleak.AfterTest(c)() - +func TestExecDMLRollbackErrDatabaseNotExists(t *testing.T) { rows := []*model.RowChangedEvent{ { Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, @@ -1233,12 +717,12 @@ func (s MySQLSinkSuite) TestExecDMLRollbackErrDatabaseNotExists(c *check.C) { if dbIndex == 0 { // test db db, err := mockTestDB() - c.Assert(err, check.IsNil) + require.Nil(t, err) return db, nil } // normal db db, mock, err := sqlmock.New(sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual)) - c.Assert(err, check.IsNil) + require.Nil(t, err) mock.ExpectBegin() mock.ExpectExec("REPLACE INTO `s1`.`t1`(`a`) VALUES (?),(?)"). WithArgs(1, 2). @@ -1257,23 +741,21 @@ func (s MySQLSinkSuite) TestExecDMLRollbackErrDatabaseNotExists(c *check.C) { defer cancel() changefeed := "test-changefeed" sinkURI, err := url.Parse("mysql://127.0.0.1:4000/?time-zone=UTC&worker-count=1") - c.Assert(err, check.IsNil) + require.Nil(t, err) rc := config.GetDefaultReplicaConfig() f, err := filter.NewFilter(rc) - c.Assert(err, check.IsNil) + require.Nil(t, err) sink, err := newMySQLSink(ctx, changefeed, sinkURI, f, rc, map[string]string{}) - c.Assert(err, check.IsNil) + require.Nil(t, err) err = sink.(*mysqlSink).execDMLs(ctx, rows, 1 /* replicaID */, 1 /* bucket */) - c.Assert(errors.Cause(err), check.Equals, errDatabaseNotExists) + require.Equal(t, errDatabaseNotExists, errors.Cause(err)) err = sink.Close(ctx) - c.Assert(err, check.IsNil) + require.Nil(t, err) } -func (s MySQLSinkSuite) TestExecDMLRollbackErrTableNotExists(c *check.C) { - defer testleak.AfterTest(c)() - +func TestExecDMLRollbackErrTableNotExists(t *testing.T) { rows := []*model.RowChangedEvent{ { Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, @@ -1301,12 +783,12 @@ func (s MySQLSinkSuite) TestExecDMLRollbackErrTableNotExists(c *check.C) { if dbIndex == 0 { // test db db, err := mockTestDB() - c.Assert(err, check.IsNil) + require.Nil(t, err) return db, nil } // normal db db, mock, err := sqlmock.New(sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual)) - c.Assert(err, check.IsNil) + require.Nil(t, err) mock.ExpectBegin() mock.ExpectExec("REPLACE INTO `s1`.`t1`(`a`) VALUES (?),(?)"). WithArgs(1, 2). @@ -1325,23 +807,21 @@ func (s MySQLSinkSuite) TestExecDMLRollbackErrTableNotExists(c *check.C) { defer cancel() changefeed := "test-changefeed" sinkURI, err := url.Parse("mysql://127.0.0.1:4000/?time-zone=UTC&worker-count=1") - c.Assert(err, check.IsNil) + require.Nil(t, err) rc := config.GetDefaultReplicaConfig() f, err := filter.NewFilter(rc) - c.Assert(err, check.IsNil) + require.Nil(t, err) sink, err := newMySQLSink(ctx, changefeed, sinkURI, f, rc, map[string]string{}) - c.Assert(err, check.IsNil) + require.Nil(t, err) err = sink.(*mysqlSink).execDMLs(ctx, rows, 1 /* replicaID */, 1 /* bucket */) - c.Assert(errors.Cause(err), check.Equals, errTableNotExists) + require.Equal(t, errTableNotExists, errors.Cause(err)) err = sink.Close(ctx) - c.Assert(err, check.IsNil) + require.Nil(t, err) } -func (s MySQLSinkSuite) TestExecDMLRollbackErrRetryable(c *check.C) { - defer testleak.AfterTest(c)() - +func TestExecDMLRollbackErrRetryable(t *testing.T) { rows := []*model.RowChangedEvent{ { Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, @@ -1369,12 +849,12 @@ func (s MySQLSinkSuite) TestExecDMLRollbackErrRetryable(c *check.C) { if dbIndex == 0 { // test db db, err := mockTestDB() - c.Assert(err, check.IsNil) + require.Nil(t, err) return db, nil } // normal db db, mock, err := sqlmock.New(sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual)) - c.Assert(err, check.IsNil) + require.Nil(t, err) for i := 0; i < defaultDMLMaxRetryTime; i++ { mock.ExpectBegin() mock.ExpectExec("REPLACE INTO `s1`.`t1`(`a`) VALUES (?),(?)"). @@ -1395,23 +875,21 @@ func (s MySQLSinkSuite) TestExecDMLRollbackErrRetryable(c *check.C) { defer cancel() changefeed := "test-changefeed" sinkURI, err := url.Parse("mysql://127.0.0.1:4000/?time-zone=UTC&worker-count=1") - c.Assert(err, check.IsNil) + require.Nil(t, err) rc := config.GetDefaultReplicaConfig() f, err := filter.NewFilter(rc) - c.Assert(err, check.IsNil) + require.Nil(t, err) sink, err := newMySQLSink(ctx, changefeed, sinkURI, f, rc, map[string]string{}) - c.Assert(err, check.IsNil) + require.Nil(t, err) err = sink.(*mysqlSink).execDMLs(ctx, rows, 1 /* replicaID */, 1 /* bucket */) - c.Assert(errors.Cause(err), check.Equals, errLockDeadlock) + require.Equal(t, errLockDeadlock, errors.Cause(err)) err = sink.Close(ctx) - c.Assert(err, check.IsNil) + require.Nil(t, err) } -func (s MySQLSinkSuite) TestNewMySQLSinkExecDDL(c *check.C) { - defer testleak.AfterTest(c)() - +func TestNewMySQLSinkExecDDL(t *testing.T) { dbIndex := 0 mockGetDBConn := func(ctx context.Context, dsnStr string) (*sql.DB, error) { defer func() { @@ -1420,12 +898,12 @@ func (s MySQLSinkSuite) TestNewMySQLSinkExecDDL(c *check.C) { if dbIndex == 0 { // test db db, err := mockTestDB() - c.Assert(err, check.IsNil) + require.Nil(t, err) return db, nil } // normal db db, mock, err := sqlmock.New(sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual)) - c.Assert(err, check.IsNil) + require.Nil(t, err) mock.ExpectBegin() mock.ExpectExec("USE `test`;").WillReturnResult(sqlmock.NewResult(1, 1)) mock.ExpectExec("ALTER TABLE test.t1 ADD COLUMN a int").WillReturnResult(sqlmock.NewResult(1, 1)) @@ -1450,15 +928,15 @@ func (s MySQLSinkSuite) TestNewMySQLSinkExecDDL(c *check.C) { defer cancel() changefeed := "test-changefeed" sinkURI, err := url.Parse("mysql://127.0.0.1:4000/?time-zone=UTC&worker-count=4") - c.Assert(err, check.IsNil) + require.Nil(t, err) rc := config.GetDefaultReplicaConfig() rc.Filter = &config.FilterConfig{ Rules: []string{"test.t1"}, } f, err := filter.NewFilter(rc) - c.Assert(err, check.IsNil) + require.Nil(t, err) sink, err := newMySQLSink(ctx, changefeed, sinkURI, f, rc, map[string]string{}) - c.Assert(err, check.IsNil) + require.Nil(t, err) ddl1 := &model.DDLEvent{ StartTs: 1000, @@ -1482,20 +960,66 @@ func (s MySQLSinkSuite) TestNewMySQLSinkExecDDL(c *check.C) { } err = sink.EmitDDLEvent(ctx, ddl1) - c.Assert(err, check.IsNil) + require.Nil(t, err) err = sink.EmitDDLEvent(ctx, ddl2) - c.Assert(cerror.ErrDDLEventIgnored.Equal(err), check.IsTrue) + require.True(t, cerror.ErrDDLEventIgnored.Equal(err)) // DDL execute failed, but error can be ignored err = sink.EmitDDLEvent(ctx, ddl1) - c.Assert(err, check.IsNil) + require.Nil(t, err) err = sink.Close(ctx) - c.Assert(err, check.IsNil) + require.Nil(t, err) } -func (s MySQLSinkSuite) TestNewMySQLSink(c *check.C) { - defer testleak.AfterTest(c)() +func TestNeedSwitchDB(t *testing.T) { + testCases := []struct { + ddl *model.DDLEvent + needSwitch bool + }{ + { + &model.DDLEvent{ + TableInfo: &model.SimpleTableInfo{ + Schema: "", + }, + Type: timodel.ActionCreateTable, + }, + false, + }, + { + &model.DDLEvent{ + TableInfo: &model.SimpleTableInfo{ + Schema: "golang", + }, + Type: timodel.ActionCreateSchema, + }, + false, + }, + { + &model.DDLEvent{ + TableInfo: &model.SimpleTableInfo{ + Schema: "golang", + }, + Type: timodel.ActionDropSchema, + }, + false, + }, + { + &model.DDLEvent{ + TableInfo: &model.SimpleTableInfo{ + Schema: "golang", + }, + Type: timodel.ActionCreateTable, + }, + true, + }, + } + for _, tc := range testCases { + require.Equal(t, tc.needSwitch, needSwitchDB(tc.ddl)) + } +} + +func TestNewMySQLSink(t *testing.T) { dbIndex := 0 mockGetDBConn := func(ctx context.Context, dsnStr string) (*sql.DB, error) { defer func() { @@ -1504,13 +1028,13 @@ func (s MySQLSinkSuite) TestNewMySQLSink(c *check.C) { if dbIndex == 0 { // test db db, err := mockTestDB() - c.Assert(err, check.IsNil) + require.Nil(t, err) return db, nil } // normal db db, mock, err := sqlmock.New(sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual)) mock.ExpectClose() - c.Assert(err, check.IsNil) + require.Nil(t, err) return db, nil } backupGetDBConn := GetDBConnImpl @@ -1524,19 +1048,17 @@ func (s MySQLSinkSuite) TestNewMySQLSink(c *check.C) { changefeed := "test-changefeed" sinkURI, err := url.Parse("mysql://127.0.0.1:4000/?time-zone=UTC&worker-count=4") - c.Assert(err, check.IsNil) + require.Nil(t, err) rc := config.GetDefaultReplicaConfig() f, err := filter.NewFilter(rc) - c.Assert(err, check.IsNil) + require.Nil(t, err) sink, err := newMySQLSink(ctx, changefeed, sinkURI, f, rc, map[string]string{}) - c.Assert(err, check.IsNil) + require.Nil(t, err) err = sink.Close(ctx) - c.Assert(err, check.IsNil) + require.Nil(t, err) } -func (s MySQLSinkSuite) TestMySQLSinkClose(c *check.C) { - defer testleak.AfterTest(c)() - +func TestMySQLSinkClose(t *testing.T) { dbIndex := 0 mockGetDBConn := func(ctx context.Context, dsnStr string) (*sql.DB, error) { defer func() { @@ -1545,13 +1067,13 @@ func (s MySQLSinkSuite) TestMySQLSinkClose(c *check.C) { if dbIndex == 0 { // test db db, err := mockTestDB() - c.Assert(err, check.IsNil) + require.Nil(t, err) return db, nil } // normal db db, mock, err := sqlmock.New(sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual)) mock.ExpectClose() - c.Assert(err, check.IsNil) + require.Nil(t, err) return db, nil } backupGetDBConn := GetDBConnImpl @@ -1564,14 +1086,100 @@ func (s MySQLSinkSuite) TestMySQLSinkClose(c *check.C) { changefeed := "test-changefeed" sinkURI, err := url.Parse("mysql://127.0.0.1:4000/?time-zone=UTC&worker-count=4") - c.Assert(err, check.IsNil) + require.Nil(t, err) rc := config.GetDefaultReplicaConfig() f, err := filter.NewFilter(rc) - c.Assert(err, check.IsNil) + require.Nil(t, err) // test sink.Close will work correctly even if the ctx pass in has not been cancel sink, err := newMySQLSink(ctx, changefeed, sinkURI, f, rc, map[string]string{}) - c.Assert(err, check.IsNil) + require.Nil(t, err) + err = sink.Close(ctx) + require.Nil(t, err) +} + +func TestMySQLSinkFlushResovledTs(t *testing.T) { + dbIndex := 0 + mockGetDBConn := func(ctx context.Context, dsnStr string) (*sql.DB, error) { + defer func() { + dbIndex++ + }() + if dbIndex == 0 { + // test db + db, err := mockTestDB() + require.Nil(t, err) + return db, nil + } + // normal db + db, mock, err := sqlmock.New(sqlmock.QueryMatcherOption(sqlmock.QueryMatcherEqual)) + mock.ExpectBegin() + mock.ExpectExec("REPLACE INTO `s1`.`t1`(`a`) VALUES (?)"). + WithArgs(1). + WillReturnResult(sqlmock.NewResult(1, 1)) + mock.ExpectCommit() + mock.ExpectBegin() + mock.ExpectExec("REPLACE INTO `s1`.`t2`(`a`) VALUES (?)"). + WithArgs(1). + WillReturnResult(sqlmock.NewResult(1, 1)) + mock.ExpectCommit() + mock.ExpectClose() + require.Nil(t, err) + return db, nil + } + backupGetDBConn := GetDBConnImpl + GetDBConnImpl = mockGetDBConn + defer func() { + GetDBConnImpl = backupGetDBConn + }() + + ctx := context.Background() + + changefeed := "test-changefeed" + sinkURI, err := url.Parse("mysql://127.0.0.1:4000/?time-zone=UTC&worker-count=4") + require.Nil(t, err) + rc := config.GetDefaultReplicaConfig() + f, err := filter.NewFilter(rc) + require.Nil(t, err) + + // test sink.Close will work correctly even if the ctx pass in has not been cancel + si, err := newMySQLSink(ctx, changefeed, sinkURI, f, rc, map[string]string{}) + sink := si.(*mysqlSink) + require.Nil(t, err) + checkpoint, err := sink.FlushRowChangedEvents(ctx, model.TableID(1), 1) + require.Nil(t, err) + require.Equal(t, uint64(0), checkpoint) + rows := []*model.RowChangedEvent{ + { + Table: &model.TableName{Schema: "s1", Table: "t1", TableID: 1}, + CommitTs: 5, + Columns: []*model.Column{ + {Name: "a", Type: mysql.TypeLong, Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, Value: 1}, + }, + }, + } + err = sink.EmitRowChangedEvents(ctx, rows...) + require.Nil(t, err) + checkpoint, err = sink.FlushRowChangedEvents(ctx, model.TableID(1), 6) + require.True(t, checkpoint <= 5) + time.Sleep(500 * time.Millisecond) + require.Nil(t, err) + require.Equal(t, uint64(6), sink.getTableCheckpointTs(model.TableID(1))) + rows = []*model.RowChangedEvent{ + { + Table: &model.TableName{Schema: "s1", Table: "t2", TableID: 2}, + CommitTs: 4, + Columns: []*model.Column{ + {Name: "a", Type: mysql.TypeLong, Flag: model.HandleKeyFlag | model.PrimaryKeyFlag, Value: 1}, + }, + }, + } + err = sink.EmitRowChangedEvents(ctx, rows...) + require.Nil(t, err) + checkpoint, err = sink.FlushRowChangedEvents(ctx, model.TableID(2), 5) + require.True(t, checkpoint <= 5) + time.Sleep(500 * time.Millisecond) + require.Nil(t, err) + require.Equal(t, uint64(5), sink.getTableCheckpointTs(model.TableID(2))) err = sink.Close(ctx) - c.Assert(err, check.IsNil) + require.Nil(t, err) } diff --git a/cdc/sink/mysql_worker_test.go b/cdc/sink/mysql_worker_test.go new file mode 100644 index 00000000000..03b6b794a0b --- /dev/null +++ b/cdc/sink/mysql_worker_test.go @@ -0,0 +1,362 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package sink + +import ( + "context" + "fmt" + "sync" + "testing" + "time" + + "github.com/davecgh/go-spew/spew" + "github.com/pingcap/errors" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/notify" + "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" + "golang.org/x/sync/errgroup" +) + +func TestMysqlSinkWorker(t *testing.T) { + defer testleak.AfterTestT(t)() + tbl := &model.TableName{ + Schema: "test", + Table: "user", + TableID: 1, + IsPartition: false, + } + testCases := []struct { + txns []*model.SingleTableTxn + expectedOutputRows [][]*model.RowChangedEvent + exportedOutputReplicaIDs []uint64 + maxTxnRow int + }{ + { + txns: []*model.SingleTableTxn{}, + maxTxnRow: 4, + }, { + txns: []*model.SingleTableTxn{ + { + Table: tbl, + CommitTs: 1, + Rows: []*model.RowChangedEvent{{CommitTs: 1}}, + ReplicaID: 1, + }, + }, + expectedOutputRows: [][]*model.RowChangedEvent{{{CommitTs: 1}}}, + exportedOutputReplicaIDs: []uint64{1}, + maxTxnRow: 2, + }, { + txns: []*model.SingleTableTxn{ + { + Table: tbl, + CommitTs: 1, + Rows: []*model.RowChangedEvent{{CommitTs: 1}, {CommitTs: 1}, {CommitTs: 1}}, + ReplicaID: 1, + }, + }, + expectedOutputRows: [][]*model.RowChangedEvent{ + {{CommitTs: 1}, {CommitTs: 1}, {CommitTs: 1}}, + }, + exportedOutputReplicaIDs: []uint64{1}, + maxTxnRow: 2, + }, { + txns: []*model.SingleTableTxn{ + { + Table: tbl, + CommitTs: 1, + Rows: []*model.RowChangedEvent{{CommitTs: 1}, {CommitTs: 1}}, + ReplicaID: 1, + }, + { + Table: tbl, + CommitTs: 2, + Rows: []*model.RowChangedEvent{{CommitTs: 2}}, + ReplicaID: 1, + }, + { + Table: tbl, + CommitTs: 3, + Rows: []*model.RowChangedEvent{{CommitTs: 3}, {CommitTs: 3}}, + ReplicaID: 1, + }, + }, + expectedOutputRows: [][]*model.RowChangedEvent{ + {{CommitTs: 1}, {CommitTs: 1}, {CommitTs: 2}}, + {{CommitTs: 3}, {CommitTs: 3}}, + }, + exportedOutputReplicaIDs: []uint64{1, 1}, + maxTxnRow: 4, + }, { + txns: []*model.SingleTableTxn{ + { + Table: tbl, + CommitTs: 1, + Rows: []*model.RowChangedEvent{{CommitTs: 1}}, + ReplicaID: 1, + }, + { + Table: tbl, + CommitTs: 2, + Rows: []*model.RowChangedEvent{{CommitTs: 2}}, + ReplicaID: 2, + }, + { + Table: tbl, + CommitTs: 3, + Rows: []*model.RowChangedEvent{{CommitTs: 3}}, + ReplicaID: 3, + }, + }, + expectedOutputRows: [][]*model.RowChangedEvent{ + {{CommitTs: 1}}, + {{CommitTs: 2}}, + {{CommitTs: 3}}, + }, + exportedOutputReplicaIDs: []uint64{1, 2, 3}, + maxTxnRow: 4, + }, { + txns: []*model.SingleTableTxn{ + { + Table: tbl, + CommitTs: 1, + Rows: []*model.RowChangedEvent{{CommitTs: 1}}, + ReplicaID: 1, + }, + { + Table: tbl, + CommitTs: 2, + Rows: []*model.RowChangedEvent{{CommitTs: 2}, {CommitTs: 2}, {CommitTs: 2}}, + ReplicaID: 1, + }, + { + Table: tbl, + CommitTs: 3, + Rows: []*model.RowChangedEvent{{CommitTs: 3}}, + ReplicaID: 1, + }, + { + Table: tbl, + CommitTs: 4, + Rows: []*model.RowChangedEvent{{CommitTs: 4}}, + ReplicaID: 1, + }, + }, + expectedOutputRows: [][]*model.RowChangedEvent{ + {{CommitTs: 1}}, + {{CommitTs: 2}, {CommitTs: 2}, {CommitTs: 2}}, + {{CommitTs: 3}, {CommitTs: 4}}, + }, + exportedOutputReplicaIDs: []uint64{1, 1, 1}, + maxTxnRow: 2, + }, + } + ctx := context.Background() + + notifier := new(notify.Notifier) + for i, tc := range testCases { + cctx, cancel := context.WithCancel(ctx) + var outputRows [][]*model.RowChangedEvent + var outputReplicaIDs []uint64 + receiver, err := notifier.NewReceiver(-1) + require.Nil(t, err) + w := newMySQLSinkWorker(tc.maxTxnRow, 1, + bucketSizeCounter.WithLabelValues("capture", "changefeed", "1"), + receiver, + func(ctx context.Context, events []*model.RowChangedEvent, replicaID uint64, bucket int) error { + outputRows = append(outputRows, events) + outputReplicaIDs = append(outputReplicaIDs, replicaID) + return nil + }) + errg, cctx := errgroup.WithContext(cctx) + errg.Go(func() error { + return w.run(cctx) + }) + for _, txn := range tc.txns { + w.appendTxn(cctx, txn) + } + var wg sync.WaitGroup + w.appendFinishTxn(&wg) + // ensure all txns are fetched from txn channel in sink worker + time.Sleep(time.Millisecond * 100) + notifier.Notify() + wg.Wait() + cancel() + require.Equal(t, context.Canceled, errors.Cause(errg.Wait())) + require.Equal(t, tc.expectedOutputRows, outputRows, + fmt.Sprintf("case %v, %s, %s", i, spew.Sdump(outputRows), spew.Sdump(tc.expectedOutputRows))) + require.Equal(t, tc.exportedOutputReplicaIDs, outputReplicaIDs, tc.exportedOutputReplicaIDs, + fmt.Sprintf("case %v, %s, %s", i, spew.Sdump(outputReplicaIDs), spew.Sdump(tc.exportedOutputReplicaIDs))) + } +} + +func TestMySQLSinkWorkerExitWithError(t *testing.T) { + defer testleak.AfterTestT(t)() + tbl := &model.TableName{ + Schema: "test", + Table: "user", + TableID: 1, + IsPartition: false, + } + txns1 := []*model.SingleTableTxn{ + { + Table: tbl, + CommitTs: 1, + Rows: []*model.RowChangedEvent{{CommitTs: 1}}, + }, + { + Table: tbl, + CommitTs: 2, + Rows: []*model.RowChangedEvent{{CommitTs: 2}}, + }, + { + Table: tbl, + CommitTs: 3, + Rows: []*model.RowChangedEvent{{CommitTs: 3}}, + }, + { + Table: tbl, + CommitTs: 4, + Rows: []*model.RowChangedEvent{{CommitTs: 4}}, + }, + } + txns2 := []*model.SingleTableTxn{ + { + Table: tbl, + CommitTs: 5, + Rows: []*model.RowChangedEvent{{CommitTs: 5}}, + }, + { + Table: tbl, + CommitTs: 6, + Rows: []*model.RowChangedEvent{{CommitTs: 6}}, + }, + } + maxTxnRow := 1 + ctx := context.Background() + + errExecFailed := errors.New("sink worker exec failed") + notifier := new(notify.Notifier) + cctx, cancel := context.WithCancel(ctx) + receiver, err := notifier.NewReceiver(-1) + require.Nil(t, err) + w := newMySQLSinkWorker(maxTxnRow, 1, /*bucket*/ + bucketSizeCounter.WithLabelValues("capture", "changefeed", "1"), + receiver, + func(ctx context.Context, events []*model.RowChangedEvent, replicaID uint64, bucket int) error { + return errExecFailed + }) + errg, cctx := errgroup.WithContext(cctx) + errg.Go(func() error { + return w.run(cctx) + }) + // txn in txns1 will be sent to worker txnCh + for _, txn := range txns1 { + w.appendTxn(cctx, txn) + } + + // simulate notify sink worker to flush existing txns + var wg sync.WaitGroup + w.appendFinishTxn(&wg) + time.Sleep(time.Millisecond * 100) + // txn in txn2 will be blocked since the worker has exited + for _, txn := range txns2 { + w.appendTxn(cctx, txn) + } + notifier.Notify() + + // simulate sink shutdown and send closed singal to sink worker + w.closedCh <- struct{}{} + w.cleanup() + + // the flush notification wait group should be done + wg.Wait() + + cancel() + require.Equal(t, errExecFailed, errg.Wait()) +} + +func TestMySQLSinkWorkerExitCleanup(t *testing.T) { + defer testleak.AfterTestT(t)() + tbl := &model.TableName{ + Schema: "test", + Table: "user", + TableID: 1, + IsPartition: false, + } + txns1 := []*model.SingleTableTxn{ + { + Table: tbl, + CommitTs: 1, + Rows: []*model.RowChangedEvent{{CommitTs: 1}}, + }, + { + Table: tbl, + CommitTs: 2, + Rows: []*model.RowChangedEvent{{CommitTs: 2}}, + }, + } + txns2 := []*model.SingleTableTxn{ + { + Table: tbl, + CommitTs: 5, + Rows: []*model.RowChangedEvent{{CommitTs: 5}}, + }, + } + + maxTxnRow := 1 + ctx := context.Background() + + errExecFailed := errors.New("sink worker exec failed") + notifier := new(notify.Notifier) + cctx, cancel := context.WithCancel(ctx) + receiver, err := notifier.NewReceiver(-1) + require.Nil(t, err) + w := newMySQLSinkWorker(maxTxnRow, 1, /*bucket*/ + bucketSizeCounter.WithLabelValues("capture", "changefeed", "1"), + receiver, + func(ctx context.Context, events []*model.RowChangedEvent, replicaID uint64, bucket int) error { + return errExecFailed + }) + errg, cctx := errgroup.WithContext(cctx) + errg.Go(func() error { + err := w.run(cctx) + return err + }) + for _, txn := range txns1 { + w.appendTxn(cctx, txn) + } + + // sleep to let txns flushed by tick + time.Sleep(time.Millisecond * 100) + + // simulate more txns are sent to txnCh after the sink worker run has exited + for _, txn := range txns2 { + w.appendTxn(cctx, txn) + } + var wg sync.WaitGroup + w.appendFinishTxn(&wg) + notifier.Notify() + + // simulate sink shutdown and send closed singal to sink worker + w.closedCh <- struct{}{} + w.cleanup() + + // the flush notification wait group should be done + wg.Wait() + + cancel() + require.Equal(t, errExecFailed, errg.Wait()) +} diff --git a/cdc/sink/simple_mysql_tester.go b/cdc/sink/simple_mysql_tester.go index 95a8eb9cba4..75f71639401 100644 --- a/cdc/sink/simple_mysql_tester.go +++ b/cdc/sink/simple_mysql_tester.go @@ -187,7 +187,7 @@ func (s *simpleMySQLSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) // FlushRowChangedEvents flushes each row which of commitTs less than or equal to `resolvedTs` into downstream. // TiCDC guarantees that all of Event which of commitTs less than or equal to `resolvedTs` are sent to Sink through `EmitRowChangedEvents` -func (s *simpleMySQLSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { +func (s *simpleMySQLSink) FlushRowChangedEvents(ctx context.Context, _ model.TableID, resolvedTs uint64) (uint64, error) { s.rowsBufferLock.Lock() defer s.rowsBufferLock.Unlock() newBuffer := make([]*model.RowChangedEvent, 0, len(s.rowsBuffer)) @@ -217,7 +217,7 @@ func (s *simpleMySQLSink) Close(ctx context.Context) error { return s.db.Close() } -func (s *simpleMySQLSink) Barrier(ctx context.Context) error { +func (s *simpleMySQLSink) Barrier(ctx context.Context, tableID model.TableID) error { return nil } diff --git a/cdc/sink/sink.go b/cdc/sink/sink.go index 155012bc28a..013aeae1035 100644 --- a/cdc/sink/sink.go +++ b/cdc/sink/sink.go @@ -45,7 +45,7 @@ type Sink interface { // FlushRowChangedEvents flushes each row which of commitTs less than or equal to `resolvedTs` into downstream. // TiCDC guarantees that all of Event which of commitTs less than or equal to `resolvedTs` are sent to Sink through `EmitRowChangedEvents` - FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) + FlushRowChangedEvents(ctx context.Context, tableID model.TableID, resolvedTs uint64) (uint64, error) // EmitCheckpointTs sends CheckpointTs to Sink // TiCDC guarantees that all Events **in the cluster** which of commitTs less than or equal `checkpointTs` are sent to downstream successfully. @@ -56,7 +56,7 @@ type Sink interface { // Barrier is a synchronous function to wait all events to be flushed in underlying sink // Note once Barrier is called, the resolved ts won't be pushed until the Barrier call returns. - Barrier(ctx context.Context) error + Barrier(ctx context.Context, tableID model.TableID) error } var sinkIniterMap = make(map[string]sinkInitFunc) diff --git a/cdc/sink/table_sink.go b/cdc/sink/table_sink.go index 83a1acb8c86..3a5d1a205a6 100644 --- a/cdc/sink/table_sink.go +++ b/cdc/sink/table_sink.go @@ -56,7 +56,7 @@ func (t *tableSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error // FlushRowChangedEvents flushes sorted rows to sink manager, note the resolvedTs // is required to be no more than global resolvedTs, table barrierTs and table // redo log watermarkTs. -func (t *tableSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, error) { +func (t *tableSink) FlushRowChangedEvents(ctx context.Context, tableID model.TableID, resolvedTs uint64) (uint64, error) { // Log abnormal checkpoint that is large than resolved ts. logAbnormalCheckpoint := func(ckpt uint64) { if ckpt > resolvedTs { @@ -76,7 +76,7 @@ func (t *tableSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64 if err != nil { return ckpt, err } - ckpt, err = t.manager.flushBackendSink(ctx) + ckpt, err = t.manager.flushBackendSink(ctx, tableID) if err != nil { return ckpt, err } @@ -88,14 +88,14 @@ func (t *tableSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64 err := t.manager.backendSink.EmitRowChangedEvents(ctx, resolvedRows...) if err != nil { - return t.manager.getCheckpointTs(), errors.Trace(err) + return t.manager.getCheckpointTs(tableID), errors.Trace(err) } atomic.StoreUint64(&t.emittedTs, resolvedTs) ckpt, err := t.flushRedoLogs(ctx, resolvedTs) if err != nil { return ckpt, err } - ckpt, err = t.manager.flushBackendSink(ctx) + ckpt, err = t.manager.flushBackendSink(ctx, tableID) if err != nil { return ckpt, err } @@ -107,7 +107,7 @@ func (t *tableSink) flushRedoLogs(ctx context.Context, resolvedTs uint64) (uint6 if t.redoManager.Enabled() { err := t.redoManager.FlushLog(ctx, t.tableID, resolvedTs) if err != nil { - return t.manager.getCheckpointTs(), err + return t.manager.getCheckpointTs(t.tableID), err } } return 0, nil @@ -138,6 +138,6 @@ func (t *tableSink) Close(ctx context.Context) error { } // Barrier is not used in table sink -func (t *tableSink) Barrier(ctx context.Context) error { +func (t *tableSink) Barrier(ctx context.Context, tableID model.TableID) error { return nil } diff --git a/cmd/kafka-consumer/main.go b/cmd/kafka-consumer/main.go index 6d1d3ff324e..fc05c04d8e9 100644 --- a/cmd/kafka-consumer/main.go +++ b/cmd/kafka-consumer/main.go @@ -290,6 +290,13 @@ func main() { } } +type partitionSink struct { + sink.Sink + resolvedTs uint64 + partitionNo int + tablesMap sync.Map +} + // Consumer represents a Sarama consumer group consumer type Consumer struct { ready chan bool @@ -298,10 +305,7 @@ type Consumer struct { maxDDLReceivedTs uint64 ddlListMu sync.Mutex - sinks []*struct { - sink.Sink - resolvedTs uint64 - } + sinks []*partitionSink sinksMu sync.Mutex ddlSink sink.Sink @@ -326,10 +330,7 @@ func NewConsumer(ctx context.Context) (*Consumer, error) { c.fakeTableIDGenerator = &fakeTableIDGenerator{ tableIDs: make(map[string]int64), } - c.sinks = make([]*struct { - sink.Sink - resolvedTs uint64 - }, kafkaPartitionNum) + c.sinks = make([]*partitionSink, kafkaPartitionNum) ctx, cancel := context.WithCancel(ctx) errCh := make(chan error, 1) opts := map[string]string{} @@ -339,10 +340,7 @@ func NewConsumer(ctx context.Context) (*Consumer, error) { cancel() return nil, errors.Trace(err) } - c.sinks[i] = &struct { - sink.Sink - resolvedTs uint64 - }{Sink: s} + c.sinks[i] = &partitionSink{Sink: s, partitionNo: i} } sink, err := sink.New(ctx, "kafka-consumer", downstreamURIStr, filter, config.GetDefaultReplicaConfig(), opts, errCh) if err != nil { @@ -443,6 +441,10 @@ ClaimMessages: if err != nil { log.Fatal("emit row changed event failed", zap.Error(err)) } + lastCommitTs, ok := sink.tablesMap.Load(row.Table.TableID) + if !ok || lastCommitTs.(uint64) < row.CommitTs { + sink.tablesMap.Store(row.Table.TableID, row.CommitTs) + } case model.MqMessageTypeResolved: ts, err := batchDecoder.NextResolvedEvent() if err != nil { @@ -503,10 +505,7 @@ func (c *Consumer) popDDL() *model.DDLEvent { return nil } -func (c *Consumer) forEachSink(fn func(sink *struct { - sink.Sink - resolvedTs uint64 -}) error) error { +func (c *Consumer) forEachSink(fn func(sink *partitionSink) error) error { c.sinksMu.Lock() defer c.sinksMu.Unlock() for _, sink := range c.sinks { @@ -529,10 +528,7 @@ func (c *Consumer) Run(ctx context.Context) error { time.Sleep(100 * time.Millisecond) // handle ddl globalResolvedTs := uint64(math.MaxUint64) - err := c.forEachSink(func(sink *struct { - sink.Sink - resolvedTs uint64 - }) error { + err := c.forEachSink(func(sink *partitionSink) error { resolvedTs := atomic.LoadUint64(&sink.resolvedTs) if resolvedTs < globalResolvedTs { globalResolvedTs = resolvedTs @@ -545,10 +541,7 @@ func (c *Consumer) Run(ctx context.Context) error { todoDDL := c.getFrontDDL() if todoDDL != nil && globalResolvedTs >= todoDDL.CommitTs { // flush DMLs - err := c.forEachSink(func(sink *struct { - sink.Sink - resolvedTs uint64 - }) error { + err := c.forEachSink(func(sink *partitionSink) error { return syncFlushRowChangedEvents(ctx, sink, todoDDL.CommitTs) }) if err != nil { @@ -574,10 +567,7 @@ func (c *Consumer) Run(ctx context.Context) error { atomic.StoreUint64(&c.globalResolvedTs, globalResolvedTs) log.Info("update globalResolvedTs", zap.Uint64("ts", globalResolvedTs)) - err = c.forEachSink(func(sink *struct { - sink.Sink - resolvedTs uint64 - }) error { + err = c.forEachSink(func(sink *partitionSink) error { return syncFlushRowChangedEvents(ctx, sink, globalResolvedTs) }) if err != nil { @@ -586,18 +576,34 @@ func (c *Consumer) Run(ctx context.Context) error { } } -func syncFlushRowChangedEvents(ctx context.Context, sink sink.Sink, resolvedTs uint64) error { +func syncFlushRowChangedEvents(ctx context.Context, sink *partitionSink, resolvedTs uint64) error { for { select { case <-ctx.Done(): return ctx.Err() default: } - checkpointTs, err := sink.FlushRowChangedEvents(ctx, resolvedTs) + // tables are flushed + var ( + err error + checkpointTs uint64 + ) + flushedResolvedTs := true + sink.tablesMap.Range(func(key, value interface{}) bool { + tableID := key.(int64) + checkpointTs, err = sink.FlushRowChangedEvents(ctx, tableID, resolvedTs) + if err != nil { + return false + } + if checkpointTs < resolvedTs { + flushedResolvedTs = false + } + return true + }) if err != nil { return err } - if checkpointTs >= resolvedTs { + if flushedResolvedTs { return nil } } diff --git a/pkg/applier/redo.go b/pkg/applier/redo.go index 4538c739fc1..cbe8f46b60e 100644 --- a/pkg/applier/redo.go +++ b/pkg/applier/redo.go @@ -129,6 +129,7 @@ func (ra *RedoApplier) consumeLogs(ctx context.Context) error { // lastResolvedTs records the max resolved ts we have seen from redo logs. lastResolvedTs := checkpointTs cachedRows := make([]*model.RowChangedEvent, 0, emitBatch) + tableResolvedTsMap := make(map[model.TableID]model.Ts) for { redoLogs, err := ra.rd.ReadNextLog(ctx, readBatch) if err != nil { @@ -139,6 +140,10 @@ func (ra *RedoApplier) consumeLogs(ctx context.Context) error { } for _, redoLog := range redoLogs { + tableID := redoLog.Row.Table.TableID + if _, ok := tableResolvedTsMap[redoLog.Row.Table.TableID]; !ok { + tableResolvedTsMap[tableID] = lastSafeResolvedTs + } if len(cachedRows) >= emitBatch { err := s.EmitRowChangedEvents(ctx, cachedRows...) if err != nil { @@ -147,26 +152,33 @@ func (ra *RedoApplier) consumeLogs(ctx context.Context) error { cachedRows = make([]*model.RowChangedEvent, 0, emitBatch) } cachedRows = append(cachedRows, redo.LogToRow(redoLog)) - if redoLog.Row.CommitTs > lastResolvedTs { - lastSafeResolvedTs, lastResolvedTs = lastResolvedTs, redoLog.Row.CommitTs + + if redoLog.Row.CommitTs > tableResolvedTsMap[tableID] { + tableResolvedTsMap[tableID], lastResolvedTs = lastResolvedTs, redoLog.Row.CommitTs } } - _, err = s.FlushRowChangedEvents(ctx, lastSafeResolvedTs) - if err != nil { - return err + + for tableID, tableLastResolvedTs := range tableResolvedTsMap { + _, err = s.FlushRowChangedEvents(ctx, tableID, tableLastResolvedTs) + if err != nil { + return err + } } } err = s.EmitRowChangedEvents(ctx, cachedRows...) if err != nil { return err } - _, err = s.FlushRowChangedEvents(ctx, resolvedTs) - if err != nil { - return err - } - err = s.Barrier(ctx) - if err != nil { - return err + + for tableID := range tableResolvedTsMap { + _, err = s.FlushRowChangedEvents(ctx, tableID, resolvedTs) + if err != nil { + return err + } + err = s.Barrier(ctx, tableID) + if err != nil { + return err + } } return errApplyFinished } diff --git a/tests/integration_tests/sink_hang/run.sh b/tests/integration_tests/sink_hang/run.sh index 83d78c9c050..8a3303956e3 100644 --- a/tests/integration_tests/sink_hang/run.sh +++ b/tests/integration_tests/sink_hang/run.sh @@ -42,7 +42,7 @@ function run() { *) SINK_URI="mysql://normal:123456@127.0.0.1:3306/?max-txn-row=1" ;; esac - export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/sink/MySQLSinkHangLongTime=1*return(true);github.com/pingcap/tiflow/cdc/sink/MySQLSinkExecDMLError=9*return(true)' + export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/sink/MySQLSinkExecDMLError=2*return(true)' run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --addr "127.0.0.1:8300" --pd $pd_addr changefeed_id=$(cdc cli changefeed create --pd=$pd_addr --sink-uri="$SINK_URI" 2>&1 | tail -n2 | head -n1 | awk '{print $2}') if [ "$SINK_TYPE" == "kafka" ]; then @@ -54,8 +54,6 @@ function run() { run_sql "CREATE table sink_hang.t2(id int primary key auto_increment, val int);" ${UP_TIDB_HOST} ${UP_TIDB_PORT} run_sql "BEGIN; INSERT INTO sink_hang.t1 VALUES (),(),(); INSERT INTO sink_hang.t2 VALUES (),(),(); COMMIT" ${UP_TIDB_HOST} ${UP_TIDB_PORT} - ensure $MAX_RETRIES check_changefeed_state $pd_addr $changefeed_id "error" - cdc cli changefeed resume --changefeed-id=$changefeed_id --pd=$pd_addr ensure $MAX_RETRIES check_changefeed_state $pd_addr $changefeed_id "normal" check_table_exists "sink_hang.t1" ${DOWN_TIDB_HOST} ${DOWN_TIDB_PORT} From 1ba52909e323189d9f06423819617b4c943e59e4 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Thu, 30 Dec 2021 12:57:50 +0800 Subject: [PATCH 29/48] master(dm): clean and treat invalid load task (#4004) (#4145) --- dm/dm/master/scheduler/scheduler.go | 57 ++++++++++--- dm/dm/master/scheduler/scheduler_test.go | 12 +++ dm/dm/master/server.go | 2 + dm/tests/_utils/test_prepare | 7 +- .../load_task/conf/dm-task-standalone.yaml | 41 ++++++++++ .../load_task/conf/dm-task2-standalone.yaml | 41 ++++++++++ dm/tests/load_task/run.sh | 81 +++++++++++++++++++ 7 files changed, 227 insertions(+), 14 deletions(-) create mode 100644 dm/tests/load_task/conf/dm-task-standalone.yaml create mode 100644 dm/tests/load_task/conf/dm-task2-standalone.yaml diff --git a/dm/dm/master/scheduler/scheduler.go b/dm/dm/master/scheduler/scheduler.go index 7b7dcb2557a..519fabcb804 100644 --- a/dm/dm/master/scheduler/scheduler.go +++ b/dm/dm/master/scheduler/scheduler.go @@ -2269,16 +2269,56 @@ func (s *Scheduler) getNextLoadTaskTransfer(worker, source string) (string, stri return "", "" } -// hasLoadTaskByWorkerAndSource check whether there is a load subtask for the worker and source. +// hasLoadTaskByWorkerAndSource check whether there is an existing load subtask for the worker and source. func (s *Scheduler) hasLoadTaskByWorkerAndSource(worker, source string) bool { - for _, sourceWorkerMap := range s.loadTasks { - if workerName, ok := sourceWorkerMap[source]; ok && workerName == worker { + for taskName, sourceWorkerMap := range s.loadTasks { + // don't consider removed subtask + subtasksV, ok := s.subTaskCfgs.Load(taskName) + if !ok { + continue + } + subtasks := subtasksV.(map[string]config.SubTaskConfig) + if _, ok2 := subtasks[source]; !ok2 { + continue + } + + if workerName, ok2 := sourceWorkerMap[source]; ok2 && workerName == worker { return true } } return false } +// TryResolveLoadTask checks if there are sources whose load task has local files and not bound to the worker which is +// accessible to the local files. If so, trigger a transfer source. +func (s *Scheduler) TryResolveLoadTask(sources []string) { + for _, source := range sources { + s.mu.Lock() + worker, ok := s.bounds[source] + if !ok { + s.mu.Unlock() + continue + } + if err := s.tryResolveLoadTask(worker.baseInfo.Name, source); err != nil { + s.logger.Error("tryResolveLoadTask failed", zap.Error(err)) + } + s.mu.Unlock() + } +} + +func (s *Scheduler) tryResolveLoadTask(originWorker, originSource string) error { + if s.hasLoadTaskByWorkerAndSource(originWorker, originSource) { + return nil + } + + worker, source := s.getNextLoadTaskTransfer(originWorker, originSource) + if worker == "" && source == "" { + return nil + } + + return s.transferWorkerAndSource(originWorker, originSource, worker, source) +} + func (s *Scheduler) handleLoadTaskDel(loadTask ha.LoadTask) error { s.mu.Lock() defer s.mu.Unlock() @@ -2296,16 +2336,7 @@ func (s *Scheduler) handleLoadTaskDel(loadTask ha.LoadTask) error { delete(s.loadTasks, loadTask.Task) } - if s.hasLoadTaskByWorkerAndSource(originWorker, loadTask.Source) { - return nil - } - - worker, source := s.getNextLoadTaskTransfer(originWorker, loadTask.Source) - if worker == "" && source == "" { - return nil - } - - return s.transferWorkerAndSource(originWorker, loadTask.Source, worker, source) + return s.tryResolveLoadTask(originWorker, loadTask.Source) } func (s *Scheduler) handleLoadTaskPut(loadTask ha.LoadTask) { diff --git a/dm/dm/master/scheduler/scheduler_test.go b/dm/dm/master/scheduler/scheduler_test.go index 5335a9daf47..11f4a3f2ec2 100644 --- a/dm/dm/master/scheduler/scheduler_test.go +++ b/dm/dm/master/scheduler/scheduler_test.go @@ -1578,6 +1578,13 @@ func (t *testScheduler) TestWatchLoadTask(c *C) { s.workers[workerName4] = worker4 s.sourceCfgs[sourceID1] = &config.SourceConfig{} s.sourceCfgs[sourceID2] = &config.SourceConfig{} + s.subTaskCfgs.Store(task1, map[string]config.SubTaskConfig{ + sourceID1: {}, + }) + s.subTaskCfgs.Store(task2, map[string]config.SubTaskConfig{ + sourceID1: {}, + sourceID2: {}, + }) worker1.ToFree() c.Assert(s.boundSourceToWorker(sourceID1, worker1), IsNil) @@ -1651,6 +1658,11 @@ func (t *testScheduler) TestWatchLoadTask(c *C) { c.Assert(s.bounds[sourceID2], DeepEquals, worker4) c.Assert(worker2.stage, Equals, WorkerFree) + // after stop-task, hasLoadTaskByWorkerAndSource is no longer valid + c.Assert(s.hasLoadTaskByWorkerAndSource(workerName4, sourceID2), IsTrue) + s.subTaskCfgs.Delete(task2) + c.Assert(s.hasLoadTaskByWorkerAndSource(workerName4, sourceID2), IsFalse) + cancel1() wg.Wait() } diff --git a/dm/dm/master/server.go b/dm/dm/master/server.go index 1f50386f448..47d7508fa9a 100644 --- a/dm/dm/master/server.go +++ b/dm/dm/master/server.go @@ -500,6 +500,8 @@ func (s *Server) StartTask(ctx context.Context, req *pb.StartTaskRequest) (*pb.S release() } + go s.scheduler.TryResolveLoadTask(sources) + resp.Result = true if cfg.RemoveMeta { resp.Msg = "`remove-meta` in task config is deprecated, please use `start-task ... --remove-meta` instead" diff --git a/dm/tests/_utils/test_prepare b/dm/tests/_utils/test_prepare index 04e3f298fcd..3e581f76d01 100644 --- a/dm/tests/_utils/test_prepare +++ b/dm/tests/_utils/test_prepare @@ -59,13 +59,18 @@ function join_string() { # shortcut for start task on one DM-worker function dmctl_start_task_standalone() { + if [ $# -ge 2 ]; then + remove_meta=$2 + else + remove_meta="" + fi if [ $# -ge 1 ]; then task_conf=$1 else task_conf="$cur/conf/dm-task.yaml" fi run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "start-task $task_conf" \ + "start-task $task_conf $remove_meta" \ "\"result\": true" 2 \ "\"source\": \"$SOURCE_ID1\"" 1 } diff --git a/dm/tests/load_task/conf/dm-task-standalone.yaml b/dm/tests/load_task/conf/dm-task-standalone.yaml new file mode 100644 index 00000000000..0d293423e43 --- /dev/null +++ b/dm/tests/load_task/conf/dm-task-standalone.yaml @@ -0,0 +1,41 @@ +--- +name: load_task1 +task-mode: all +is-sharding: false +meta-schema: "dm_meta" +heartbeat-update-interval: 1 +heartbeat-report-interval: 1 + +target-database: + host: "127.0.0.1" + port: 4000 + user: "test" + password: "/Q7B9DizNLLTTfiZHv9WoEAKamfpIUs=" + +mysql-instances: + - source-id: "mysql-replica-01" + block-allow-list: "instance" + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + +block-allow-list: + instance: + do-dbs: ["load_task1"] + +mydumpers: + global: + threads: 4 + chunk-filesize: 64 + skip-tz-utc: true + extra-args: "" + +loaders: + global: + pool-size: 16 + dir: "./dumped_data" + +syncers: + global: + worker-count: 16 + batch: 100 diff --git a/dm/tests/load_task/conf/dm-task2-standalone.yaml b/dm/tests/load_task/conf/dm-task2-standalone.yaml new file mode 100644 index 00000000000..bc98e4efac3 --- /dev/null +++ b/dm/tests/load_task/conf/dm-task2-standalone.yaml @@ -0,0 +1,41 @@ +--- +name: load_task2 +task-mode: all +is-sharding: false +meta-schema: "dm_meta" +heartbeat-update-interval: 1 +heartbeat-report-interval: 1 + +target-database: + host: "127.0.0.1" + port: 4000 + user: "test" + password: "/Q7B9DizNLLTTfiZHv9WoEAKamfpIUs=" + +mysql-instances: + - source-id: "mysql-replica-01" + block-allow-list: "instance" + mydumper-config-name: "global" + loader-config-name: "global" + syncer-config-name: "global" + +block-allow-list: + instance: + do-dbs: ["load_task2"] + +mydumpers: + global: + threads: 4 + chunk-filesize: 64 + skip-tz-utc: true + extra-args: "" + +loaders: + global: + pool-size: 16 + dir: "./dumped_data" + +syncers: + global: + worker-count: 16 + batch: 100 diff --git a/dm/tests/load_task/run.sh b/dm/tests/load_task/run.sh index 9bd2c791920..24251f6ada6 100755 --- a/dm/tests/load_task/run.sh +++ b/dm/tests/load_task/run.sh @@ -170,6 +170,85 @@ function test_transfer_two_sources() { "\"taskStatus\": \"Running\"" 4 } +function stop_task_left_load() { + echo "start DM master, workers and sources" + run_dm_master $WORK_DIR/master $MASTER_PORT1 $cur/conf/dm-master.toml + check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT1 + + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LoadDataSlowDownByTask=return(\"load_task1\")" + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT + cp $cur/conf/source1.yaml $WORK_DIR/source1.yaml + sed -i "/relay-binlog-name/i\relay-dir: $WORK_DIR/worker1/relay_log" $WORK_DIR/source1.yaml + dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 + + dmctl_start_task_standalone "$cur/conf/dm-task-standalone.yaml" "--remove-meta" + + export GO_FAILPOINTS="" + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT + + # kill worker1, load_task1 will be transferred to worker2, but lack local files + ps aux | grep dm-worker1 | awk '{print $2}' | xargs kill || true + check_port_offline $WORKER1_PORT 20 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status load_task1" \ + "different worker in load stage, previous worker: worker1, current worker: worker2" 1 + + # now stop this task without clean meta (left a load_task KV in etcd) + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-task load_task1" \ + "\"result\": true" 2 + + dmctl_start_task_standalone "$cur/conf/dm-task2-standalone.yaml" "--remove-meta" + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status load_task2" \ + "\"unit\": \"Sync\"" 1 + + # after worker1 goes online, although it has unfinished load_task1, but load_task1 is stopped so should not rebound + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/loader/LoadDataSlowDownByTask=return(\"load_task1\")" + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "list-member --name worker1" \ + "\"source\": \"\"" 1 + + # start-task again, expect the source is auto transferred back + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "start-task $cur/conf/dm-task-standalone.yaml" + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "list-member --name worker1" \ + "\"source\": \"mysql-replica-01\"" 1 + + # repeat again and check start-task --remove-meta will not cause transfer + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "stop-task load_task1" \ + "\"result\": true" 2 + ps aux | grep dm-worker1 | awk '{print $2}' | xargs kill || true + check_port_offline $WORKER1_PORT 20 + + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status load_task2" \ + "\"unit\": \"Sync\"" 1 + + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT + + dmctl_start_task_standalone "$cur/conf/dm-task-standalone.yaml" "--remove-meta" + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status load_task1" \ + "\"unit\": \"Sync\"" 1 + run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "list-member --name worker1" \ + "\"source\": \"\"" 1 + + cleanup_process $* + cleanup_data load_task1 + cleanup_data load_task2 +} + function run() { echo "import prepare data" run_sql_file $cur/data/db1.prepare.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 @@ -177,6 +256,8 @@ function run() { run_sql_file $cur/data/db2.prepare.sql $MYSQL_HOST2 $MYSQL_PORT2 $MYSQL_PASSWORD2 check_contains 'Query OK, 3 rows affected' + stop_task_left_load + echo "start DM master, workers and sources" run_dm_master $WORK_DIR/master $MASTER_PORT1 $cur/conf/dm-master.toml check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT1 From 8e1aaabf2f707d3575ce9f89758b8323e20a3904 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Thu, 30 Dec 2021 14:11:51 +0800 Subject: [PATCH 30/48] loader: fix wrong progress in query-status for loader (#4093) (#4143) close pingcap/tiflow#3252 --- dm/loader/loader.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dm/loader/loader.go b/dm/loader/loader.go index ae03d03876c..3731a943984 100644 --- a/dm/loader/loader.go +++ b/dm/loader/loader.go @@ -222,7 +222,7 @@ func (w *Worker) run(ctx context.Context, fileJobQueue chan *fileJob, runFatalCh continue } // update finished offset after checkpoint updated - w.loader.finishedDataSize.Store(job.offset) + w.loader.finishedDataSize.Add(job.offset - job.lastOffset) if _, ok := w.loader.dbTableDataFinishedSize[job.sourceSchema]; ok { if _, ok := w.loader.dbTableDataFinishedSize[job.sourceSchema][job.sourceTable]; ok { w.loader.dbTableDataFinishedSize[job.sourceSchema][job.sourceTable].Store(job.offset) From fcbf280b82db80b47d2cd1dc3f03a2384604f5b3 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Thu, 30 Dec 2021 14:49:50 +0800 Subject: [PATCH 31/48] ticdc/processor: Fix backoff base delay misconfiguration (#3992) (#4028) --- cdc/processor/processor.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cdc/processor/processor.go b/cdc/processor/processor.go index 684ccab02f5..7395b79fc18 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -707,7 +707,7 @@ func (p *processor) createTablePipelineImpl(ctx cdcContext.Context, tableID mode } markTableID = tableInfo.ID return nil - }, retry.WithBackoffMaxDelay(50), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(20)) + }, retry.WithBackoffBaseDelay(50), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(20)) if err != nil { return nil, errors.Trace(err) } From 76de75d799ce7391899c0f59ea849c70d97396cd Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Thu, 30 Dec 2021 17:51:50 +0800 Subject: [PATCH 32/48] dm: load table structure from dump files (#3295) (#4163) --- dm/loader/util.go | 1 - dm/syncer/syncer.go | 7 +++++-- dm/tests/downstream_more_column/run.sh | 11 +---------- dm/tests/shardddl1/run.sh | 5 ++++- 4 files changed, 10 insertions(+), 14 deletions(-) diff --git a/dm/loader/util.go b/dm/loader/util.go index 523daae7b39..9abd144d0ed 100644 --- a/dm/loader/util.go +++ b/dm/loader/util.go @@ -158,7 +158,6 @@ func cleanDumpFiles(cfg *config.SubTaskConfig) { var lastErr error for f := range files { if strings.HasSuffix(f, ".sql") { - // TODO: table structure files are not used now, but we plan to used them in future so not delete them if strings.HasSuffix(f, "-schema-create.sql") || strings.HasSuffix(f, "-schema.sql") { continue } diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 9ff458af78b..3dfe640503b 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -1351,7 +1351,11 @@ func (s *Syncer) Run(ctx context.Context) (err error) { if s.cfg.Mode == config.ModeAll && fresh { delLoadTask = true flushCheckpoint = true - // TODO: loadTableStructureFromDump in future + err = s.loadTableStructureFromDump(ctx) + if err != nil { + tctx.L().Warn("error happened when load table structure from dump files", zap.Error(err)) + cleanDumpFile = false + } } else { cleanDumpFile = false } @@ -2766,7 +2770,6 @@ func (s *Syncer) genRouter() error { return nil } -//nolint:unused func (s *Syncer) loadTableStructureFromDump(ctx context.Context) error { logger := s.tctx.L() diff --git a/dm/tests/downstream_more_column/run.sh b/dm/tests/downstream_more_column/run.sh index d8f4a04eae4..eb8eca00da0 100755 --- a/dm/tests/downstream_more_column/run.sh +++ b/dm/tests/downstream_more_column/run.sh @@ -32,17 +32,8 @@ function run() { run_sql_tidb_with_retry "select count(1) from ${db}.${tb} where c1<100;" "count(1): 2" run_sql_file $cur/data/db1.increment.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 - run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "query-status test" \ - "sourceTable: \`${db1}\`.\`${tb1}\`" 1 \ - "targetTable: \`${db}\`.\`${tb}\`" 1 \ - "Column count doesn't match value count" 1 - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "operate-schema set -s mysql-replica-01 test -d ${db1} -t ${tb1} $cur/data/schema.sql" \ - "\"result\": true" 2 + # we load table structure from dump files, so there's no error now - run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ - "resume-task test" # check incremental data run_sql_tidb_with_retry "select count(1) from ${db}.${tb} where c1>100 and c1<1000;" "count(1): 2" diff --git a/dm/tests/shardddl1/run.sh b/dm/tests/shardddl1/run.sh index 3b81c3f8ba1..1b16959c9c0 100644 --- a/dm/tests/shardddl1/run.sh +++ b/dm/tests/shardddl1/run.sh @@ -13,7 +13,10 @@ function DM_001_CASE() { # schema tracker could track per table without error run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "query-status test" \ - "Duplicate column name 'new_col1'" 1 + "\"result\": true" 2 \ + "\"synced\": true" 1 + # only downstream sees a duplicate error, but currently ignored by DM + check_log_contain_with_retry "Duplicate column name 'new_col1'" $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log } function DM_001() { From 263f8525c351a15002e3ccaee6be897f822720f4 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Thu, 30 Dec 2021 18:37:50 +0800 Subject: [PATCH 33/48] compactor: fix duplicate entry in safemode (#3432) (#3434) (#4088) --- dm/syncer/compactor.go | 13 ++++ dm/syncer/compactor_test.go | 4 +- dm/syncer/dml.go | 50 +++++++------- dm/syncer/dml_test.go | 42 +++++++++--- dm/syncer/syncer_test.go | 10 +-- dm/tests/_utils/test_prepare | 21 ++++++ dm/tests/shardddl1/run.sh | 127 +++++++++++++++++++++++++++++++++-- 7 files changed, 221 insertions(+), 46 deletions(-) diff --git a/dm/syncer/compactor.go b/dm/syncer/compactor.go index e8b96dbd2a4..e49681aa4c7 100644 --- a/dm/syncer/compactor.go +++ b/dm/syncer/compactor.go @@ -14,6 +14,8 @@ package syncer import ( + "fmt" + "strconv" "time" "github.com/pingcap/failpoint" @@ -165,6 +167,15 @@ func (c *compactor) compactJob(j *job) { } key := j.dml.identifyKey() + + failpoint.Inject("DownstreamIdentifyKeyCheckInCompact", func(v failpoint.Value) { + value, err := strconv.Atoi(key) + upper := v.(int) + if err != nil || value > upper { + panic(fmt.Sprintf("downstream identifyKey check failed. key value %v should less than %v", value, upper)) + } + }) + prevPos, ok := tableKeyMap[key] // if no such key in the buffer, add it if !ok { @@ -184,6 +195,8 @@ func (c *compactor) compactJob(j *job) { j.dml.oldValues = nil j.dml.originOldValues = nil j.dml.op = insert + // DELETE + INSERT + UPDATE => INSERT with safemode + j.dml.safeMode = prevJob.dml.safeMode } else if prevJob.tp == update { // UPDATE + UPDATE => UPDATE j.dml.oldValues = prevJob.dml.oldValues diff --git a/dm/syncer/compactor_test.go b/dm/syncer/compactor_test.go index 19d374257a3..94a5b659a98 100644 --- a/dm/syncer/compactor_test.go +++ b/dm/syncer/compactor_test.go @@ -238,11 +238,13 @@ func (s *testSyncerSuite) TestCompactorSafeMode(c *C) { newDML(update, false, targetTableID, sourceTable, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, []interface{}{1, 1, "a"}, []interface{}{3, 3, "c"}, ti.Columns, ti, tiIndex, downTi), newDML(del, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex, downTi), newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex, downTi), + newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex, downTi), + newDML(update, false, targetTableID, sourceTable, []interface{}{2, 2, "b"}, []interface{}{2, 2, "c"}, []interface{}{2, 2, "b"}, []interface{}{2, 2, "c"}, ti.Columns, ti, tiIndex, downTi), }, output: []*DML{ newDML(insert, false, targetTableID, sourceTable, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti.Columns, ti, tiIndex, downTi), - newDML(del, false, targetTableID, sourceTable, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti.Columns, ti, tiIndex, downTi), newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti.Columns, ti, tiIndex, downTi), + newDML(insert, true, targetTableID, sourceTable, nil, []interface{}{2, 2, "c"}, nil, []interface{}{2, 2, "c"}, ti.Columns, ti, tiIndex, downTi), }, }, } diff --git a/dm/syncer/dml.go b/dm/syncer/dml.go index d4a04245ed7..a9641a851f9 100644 --- a/dm/syncer/dml.go +++ b/dm/syncer/dml.go @@ -42,6 +42,7 @@ const ( updateDML = dmlOpType(update) deleteDML = dmlOpType(del) insertOnDuplicateDML dmlOpType = iota + 1 + replaceDML ) func (op dmlOpType) String() (str string) { @@ -54,6 +55,8 @@ func (op dmlOpType) String() (str string) { return "delete" case insertOnDuplicateDML: return "insert on duplicate update" + case replaceDML: + return "replace" } return } @@ -785,11 +788,15 @@ func (dml *DML) genDeleteSQL() ([]string, [][]interface{}) { } // genInsertSQL generates a `INSERT`. -// if in safemode, generates a `INSERT ON DUPLICATE UPDATE` statement. +// if in safemode, generates a `REPLACE` statement. func (dml *DML) genInsertSQL() ([]string, [][]interface{}) { var buf strings.Builder buf.Grow(1024) - buf.WriteString("INSERT INTO ") + if dml.safeMode { + buf.WriteString("REPLACE INTO ") + } else { + buf.WriteString("INSERT INTO ") + } buf.WriteString(dml.targetTableID) buf.WriteString(" (") for i, column := range dml.columns { @@ -810,16 +817,6 @@ func (dml *DML) genInsertSQL() ([]string, [][]interface{}) { buf.WriteString("?)") } } - if dml.safeMode { - buf.WriteString(" ON DUPLICATE KEY UPDATE ") - for i, column := range dml.columns { - col := dbutil.ColumnName(column.Name.O) - buf.WriteString(col + "=VALUES(" + col + ")") - if i != len(dml.columns)-1 { - buf.WriteByte(',') - } - } - } return []string{buf.String()}, [][]interface{}{dml.values} } @@ -837,16 +834,21 @@ func valuesHolder(n int) string { return builder.String() } -// genInsertOnDuplicateSQLMultipleRows generates a `INSERT` with multiple rows like 'INSERT INTO tb(a,b) VALUES (1,1),(2,2)' +// genInsertSQLMultipleRows generates a `INSERT` with multiple rows like 'INSERT INTO tb(a,b) VALUES (1,1),(2,2)' +// if replace, generates a `REPLACE' with multiple rows like 'REPLACE INTO tb(a,b) VALUES (1,1),(2,2)' // if onDuplicate, generates a `INSERT ON DUPLICATE KEY UPDATE` statement like 'INSERT INTO tb(a,b) VALUES (1,1),(2,2) ON DUPLICATE KEY UPDATE a=VALUES(a),b=VALUES(b)'. -func genInsertOnDuplicateSQLMultipleRows(onDuplicate bool, dmls []*DML) ([]string, [][]interface{}) { +func genInsertSQLMultipleRows(op dmlOpType, dmls []*DML) ([]string, [][]interface{}) { if len(dmls) == 0 { return nil, nil } var buf strings.Builder buf.Grow(1024) - buf.WriteString("INSERT INTO") + if op == replaceDML { + buf.WriteString("REPLACE INTO") + } else { + buf.WriteString("INSERT INTO") + } buf.WriteString(" " + dmls[0].targetTableID + " (") for i, column := range dmls[0].columns { buf.WriteString(dbutil.ColumnName(column.Name.O)) @@ -866,7 +868,7 @@ func genInsertOnDuplicateSQLMultipleRows(onDuplicate bool, dmls []*DML) ([]strin buf.WriteString(holder) } - if onDuplicate { + if op == insertOnDuplicateDML { buf.WriteString(" ON DUPLICATE KEY UPDATE ") for i, column := range dmls[0].columns { col := dbutil.ColumnName(column.Name.O) @@ -927,10 +929,8 @@ func genSQLMultipleRows(op dmlOpType, dmls []*DML) (queries []string, args [][]i log.L().Debug("generate DMLs with multiple rows", zap.Stringer("op", op), zap.Stringer("original op", dmls[0].op), zap.Int("rows", len(dmls))) } switch op { - case insertDML: - return genInsertOnDuplicateSQLMultipleRows(false, dmls) - case insertOnDuplicateDML: - return genInsertOnDuplicateSQLMultipleRows(true, dmls) + case insertDML, replaceDML, insertOnDuplicateDML: + return genInsertSQLMultipleRows(op, dmls) case deleteDML: return genDeleteSQLMultipleRows(dmls) } @@ -1052,17 +1052,19 @@ func genDMLsWithSameOp(dmls []*DML) ([]string, [][]interface{}) { // group dmls with same dmlOp for i, dml := range dmls { curOp := dmlOpType(dml.op) - // if update statement didn't update identify values, regard it as insert on duplicate. - // if insert with safemode, regard it as insert on duplicate. - if (curOp == updateDML && !dml.updateIdentify()) || (curOp == insertDML && dml.safeMode) { + if curOp == updateDML && !dml.updateIdentify() && !dml.safeMode { + // if update statement didn't update identify values and not in safemode, regard it as insert on duplicate. curOp = insertOnDuplicateDML + } else if curOp == insertDML && dml.safeMode { + // if insert with safemode, regard it as replace + curOp = replaceDML } if i == 0 { lastOp = curOp } - // now there are 4 situations: [insert, insert on duplicate(insert with safemode/update without identify keys), update(update identify keys), delete] + // now there are 5 situations: [insert, replace(insert with safemode), insert on duplicate(update without identify keys), update(update identify keys/update with safemode), delete] if lastOp != curOp { query, arg := genDMLsWithSameTable(lastOp, groupDMLs) queries = append(queries, query...) diff --git a/dm/syncer/dml_test.go b/dm/syncer/dml_test.go index bd4883bc634..b1a6910ff1d 100644 --- a/dm/syncer/dml_test.go +++ b/dm/syncer/dml_test.go @@ -317,7 +317,7 @@ func (s *testSyncerSuite) TestGenSQL(c *C) { }, { newDML(insert, true, "`targetSchema`.`targetTable`", &filter.Table{}, nil, []interface{}{1, 2, 3, "haha"}, nil, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex, nil), - []string{"INSERT INTO `targetSchema`.`targetTable` (`id`,`col1`,`col2`,`name`) VALUES (?,?,?,?) ON DUPLICATE KEY UPDATE `id`=VALUES(`id`),`col1`=VALUES(`col1`),`col2`=VALUES(`col2`),`name`=VALUES(`name`)"}, + []string{"REPLACE INTO `targetSchema`.`targetTable` (`id`,`col1`,`col2`,`name`) VALUES (?,?,?,?)"}, [][]interface{}{{1, 2, 3, "haha"}}, }, { @@ -332,7 +332,7 @@ func (s *testSyncerSuite) TestGenSQL(c *C) { }, { newDML(update, true, "`targetSchema`.`targetTable`", &filter.Table{}, []interface{}{1, 2, 3, "haha"}, []interface{}{4, 5, 6, "hihi"}, []interface{}{1, 2, 3, "haha"}, []interface{}{1, 2, 3, "haha"}, ti.Columns, ti, tiIndex, nil), - []string{"DELETE FROM `targetSchema`.`targetTable` WHERE `id` = ? LIMIT 1", "INSERT INTO `targetSchema`.`targetTable` (`id`,`col1`,`col2`,`name`) VALUES (?,?,?,?) ON DUPLICATE KEY UPDATE `id`=VALUES(`id`),`col1`=VALUES(`col1`),`col2`=VALUES(`col2`),`name`=VALUES(`name`)"}, + []string{"DELETE FROM `targetSchema`.`targetTable` WHERE `id` = ? LIMIT 1", "REPLACE INTO `targetSchema`.`targetTable` (`id`,`col1`,`col2`,`name`) VALUES (?,?,?,?)"}, [][]interface{}{{1}, {4, 5, 6, "hihi"}}, }, } @@ -438,7 +438,7 @@ func (s *testSyncerSuite) TestGenDMLWithSameOp(c *C) { newDML(insert, true, targetTableID1, sourceTable11, nil, []interface{}{1, 1, "a"}, nil, []interface{}{1, 1, "a"}, ti11.Columns, ti11, ti11Index, downTi11), newDML(insert, true, targetTableID1, sourceTable11, nil, []interface{}{2, 2, "b"}, nil, []interface{}{2, 2, "b"}, ti11.Columns, ti11, ti11Index, downTi11), newDML(insert, true, targetTableID1, sourceTable12, nil, []interface{}{3, 3, "c"}, nil, []interface{}{3, 3, "c"}, ti12.Columns, ti12, ti12Index, downTi12), - // update no index + // update no index but safemode newDML(update, true, targetTableID1, sourceTable11, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, []interface{}{1, 1, "a"}, []interface{}{1, 1, "aa"}, ti11.Columns, ti11, ti11Index, downTi11), newDML(update, true, targetTableID1, sourceTable11, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, []interface{}{2, 2, "b"}, []interface{}{2, 2, "bb"}, ti11.Columns, ti11, ti11Index, downTi11), newDML(update, true, targetTableID1, sourceTable12, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, []interface{}{3, 3, "c"}, []interface{}{3, 3, "cc"}, ti12.Columns, ti12, ti12Index, downTi12), @@ -486,17 +486,29 @@ func (s *testSyncerSuite) TestGenDMLWithSameOp(c *C) { expectQueries := []string{ // table1 - "INSERT INTO `db1`.`tb1` (`id`,`col1`,`name`) VALUES (?,?,?),(?,?,?),(?,?,?),(?,?,?),(?,?,?),(?,?,?),(?,?,?),(?,?,?),(?,?,?) ON DUPLICATE KEY UPDATE `id`=VALUES(`id`),`col1`=VALUES(`col1`),`name`=VALUES(`name`)", + "REPLACE INTO `db1`.`tb1` (`id`,`col1`,`name`) VALUES (?,?,?),(?,?,?),(?,?,?)", + "DELETE FROM `db1`.`tb1` WHERE `id` = ? LIMIT 1", + "REPLACE INTO `db1`.`tb1` (`id`,`col1`,`name`) VALUES (?,?,?)", + "DELETE FROM `db1`.`tb1` WHERE `id` = ? LIMIT 1", + "REPLACE INTO `db1`.`tb1` (`id`,`col1`,`name`) VALUES (?,?,?)", + "DELETE FROM `db1`.`tb1` WHERE `id` = ? LIMIT 1", + "REPLACE INTO `db1`.`tb1` (`id`,`col1`,`name`) VALUES (?,?,?)", "DELETE FROM `db1`.`tb1` WHERE `id` = ? LIMIT 1", - "INSERT INTO `db1`.`tb1` (`id`,`col1`,`name`) VALUES (?,?,?) ON DUPLICATE KEY UPDATE `id`=VALUES(`id`),`col1`=VALUES(`col1`),`name`=VALUES(`name`)", + "REPLACE INTO `db1`.`tb1` (`id`,`col1`,`name`) VALUES (?,?,?)", "DELETE FROM `db1`.`tb1` WHERE `id` = ? LIMIT 1", - "INSERT INTO `db1`.`tb1` (`id`,`col1`,`name`) VALUES (?,?,?) ON DUPLICATE KEY UPDATE `id`=VALUES(`id`),`col1`=VALUES(`col1`),`name`=VALUES(`name`)", + "REPLACE INTO `db1`.`tb1` (`id`,`col1`,`name`) VALUES (?,?,?)", "DELETE FROM `db1`.`tb1` WHERE `id` = ? LIMIT 1", - "INSERT INTO `db1`.`tb1` (`id`,`col1`,`name`) VALUES (?,?,?) ON DUPLICATE KEY UPDATE `id`=VALUES(`id`),`col1`=VALUES(`col1`),`name`=VALUES(`name`)", + "REPLACE INTO `db1`.`tb1` (`id`,`col1`,`name`) VALUES (?,?,?)", + "DELETE FROM `db1`.`tb1` WHERE `id` = ? LIMIT 1", + "REPLACE INTO `db1`.`tb1` (`id`,`col1`,`name`) VALUES (?,?,?)", + "DELETE FROM `db1`.`tb1` WHERE `id` = ? LIMIT 1", + "REPLACE INTO `db1`.`tb1` (`id`,`col1`,`name`) VALUES (?,?,?)", + "DELETE FROM `db1`.`tb1` WHERE `id` = ? LIMIT 1", + "REPLACE INTO `db1`.`tb1` (`id`,`col1`,`name`) VALUES (?,?,?)", "DELETE FROM `db1`.`tb1` WHERE (`id`) IN ((?),(?),(?))", // table2 - "INSERT INTO `db2`.`tb2` (`id`,`col2`,`name`) VALUES (?,?,?) ON DUPLICATE KEY UPDATE `id`=VALUES(`id`),`col2`=VALUES(`col2`),`name`=VALUES(`name`)", + "REPLACE INTO `db2`.`tb2` (`id`,`col2`,`name`) VALUES (?,?,?)", "INSERT INTO `db2`.`tb2` (`id`,`col2`,`name`) VALUES (?,?,?)", "INSERT INTO `db2`.`tb2` (`id`,`col3`,`name`) VALUES (?,?,?)", "INSERT INTO `db2`.`tb2` (`id`,`col2`,`name`) VALUES (?,?,?),(?,?,?) ON DUPLICATE KEY UPDATE `id`=VALUES(`id`),`col2`=VALUES(`col2`),`name`=VALUES(`name`)", @@ -514,7 +526,19 @@ func (s *testSyncerSuite) TestGenDMLWithSameOp(c *C) { expectArgs := [][]interface{}{ // table1 - {1, 1, "a", 2, 2, "b", 3, 3, "c", 1, 1, "aa", 2, 2, "bb", 3, 3, "cc", 1, 4, "aa", 2, 5, "bb", 3, 6, "cc"}, + {1, 1, "a", 2, 2, "b", 3, 3, "c"}, + {1}, + {1, 1, "aa"}, + {2}, + {2, 2, "bb"}, + {3}, + {3, 3, "cc"}, + {1}, + {1, 4, "aa"}, + {2}, + {2, 5, "bb"}, + {3}, + {3, 6, "cc"}, {1}, {4, 4, "aa"}, {2}, diff --git a/dm/syncer/syncer_test.go b/dm/syncer/syncer_test.go index 3467cd1e8e9..b924a935546 100644 --- a/dm/syncer/syncer_test.go +++ b/dm/syncer/syncer_test.go @@ -867,7 +867,7 @@ func (s *testSyncerSuite) TestRun(c *C) { nil, }, { insert, - []string{"INSERT INTO `test_1`.`t_1` (`id`,`name`) VALUES (?,?) ON DUPLICATE KEY UPDATE `id`=VALUES(`id`),`name`=VALUES(`name`)"}, + []string{"REPLACE INTO `test_1`.`t_1` (`id`,`name`) VALUES (?,?)"}, [][]interface{}{{int64(580981944116838401), "a"}}, }, { flush, @@ -879,7 +879,7 @@ func (s *testSyncerSuite) TestRun(c *C) { nil, }, { insert, - []string{"INSERT INTO `test_1`.`t_1` (`id`,`name`) VALUES (?,?) ON DUPLICATE KEY UPDATE `id`=VALUES(`id`),`name`=VALUES(`name`)"}, + []string{"REPLACE INTO `test_1`.`t_1` (`id`,`name`) VALUES (?,?)"}, [][]interface{}{{int64(580981944116838402), "b"}}, }, { del, @@ -888,7 +888,7 @@ func (s *testSyncerSuite) TestRun(c *C) { }, { // safe mode is true, will split update to delete + replace update, - []string{"DELETE FROM `test_1`.`t_1` WHERE `id` = ? LIMIT 1", "INSERT INTO `test_1`.`t_1` (`id`,`name`) VALUES (?,?) ON DUPLICATE KEY UPDATE `id`=VALUES(`id`),`name`=VALUES(`name`)"}, + []string{"DELETE FROM `test_1`.`t_1` WHERE `id` = ? LIMIT 1", "REPLACE INTO `test_1`.`t_1` (`id`,`name`) VALUES (?,?)"}, [][]interface{}{{int64(580981944116838402)}, {int64(580981944116838401), "b"}}, }, { flush, @@ -1131,7 +1131,7 @@ func (s *testSyncerSuite) TestExitSafeModeByConfig(c *C) { nil, }, { insert, - []string{"INSERT INTO `test_1`.`t_1` (`id`,`name`) VALUES (?,?) ON DUPLICATE KEY UPDATE `id`=VALUES(`id`),`name`=VALUES(`name`)"}, + []string{"REPLACE INTO `test_1`.`t_1` (`id`,`name`) VALUES (?,?)"}, [][]interface{}{{int32(1), "a"}}, }, { del, @@ -1139,7 +1139,7 @@ func (s *testSyncerSuite) TestExitSafeModeByConfig(c *C) { [][]interface{}{{int32(1)}}, }, { update, - []string{"DELETE FROM `test_1`.`t_1` WHERE `id` = ? LIMIT 1", "INSERT INTO `test_1`.`t_1` (`id`,`name`) VALUES (?,?) ON DUPLICATE KEY UPDATE `id`=VALUES(`id`),`name`=VALUES(`name`)"}, + []string{"DELETE FROM `test_1`.`t_1` WHERE `id` = ? LIMIT 1", "REPLACE INTO `test_1`.`t_1` (`id`,`name`) VALUES (?,?)"}, [][]interface{}{{int32(2)}, {int32(1), "b"}}, }, { // start from this event, location passes safeModeExitLocation and safe mode should exit diff --git a/dm/tests/_utils/test_prepare b/dm/tests/_utils/test_prepare index 3e581f76d01..ff24c716c5c 100644 --- a/dm/tests/_utils/test_prepare +++ b/dm/tests/_utils/test_prepare @@ -241,6 +241,27 @@ function run_sql_tidb_with_retry() { fi } +# shortcut for run tidb sql and check result with retry +function run_sql_tidb_with_retry_times() { + rc=0 + for ((k=1; k<$3; k++)); do + run_sql_tidb "$1" + if grep -Fq "$2" "$TEST_DIR/sql_res.$TEST_NAME.txt"; then + rc=1 + break + fi + echo "run tidb sql failed $k-th time, retry later" + sleep 2 + done + if [[ $rc = 0 ]]; then + echo "TEST FAILED: OUTPUT DOES NOT CONTAIN '$2'" + echo "____________________________________" + cat "$TEST_DIR/sql_res.$TEST_NAME.txt" + echo "^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^" + exit 1 + fi +} + # shortcut for check log contain with retry function check_log_contain_with_retry() { text=$1 diff --git a/dm/tests/shardddl1/run.sh b/dm/tests/shardddl1/run.sh index 1b16959c9c0..fa009a003b0 100644 --- a/dm/tests/shardddl1/run.sh +++ b/dm/tests/shardddl1/run.sh @@ -600,7 +600,7 @@ function DM_COMPACT() { ps aux | grep dm-worker | awk '{print $2}' | xargs kill || true check_port_offline $WORKER1_PORT 20 check_port_offline $WORKER2_PORT 20 - export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/syncer/BlockExecuteSQLs=return(1)' + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/syncer/BlockExecuteSQLs=return(1);github.com/pingcap/tiflow/dm/syncer/SafeModeInitPhaseSeconds=return(5)' run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT @@ -611,6 +611,54 @@ function DM_COMPACT() { "clean_table" "" } +function DM_COMPACT_USE_DOWNSTREAM_SCHEMA_CASE() { + END=10 + # As this kind of sql is no use, like "update tb1 set c=1 where a=100" which is behind of "insert into tb1(a,b,c) values(100,1,1)" + # We should avoid this kind of sql to make sure the count of dmls + for i in $(seq 0 $END); do + run_sql_source1 "insert into ${shardddl1}.${tb1}(a,b,c) values($((i + 100)),$i,$i)" + run_sql_source1 "update ${shardddl1}.${tb1} set c=20 where a=$((i + 100))" + run_sql_source1 "update ${shardddl1}.${tb1} set c=c+1 where a=$((i + 100))" + # Use downstream uk 'b' as key and this sql which modifiies 'b' will be splited to two job(delete+insert) + run_sql_source1 "update ${shardddl1}.${tb1} set b=b+1 where a=$((i + 100))" + run_sql_source1 "update ${shardddl1}.${tb1} set a=a+100 where a=$((i + 100))" + run_sql_source1 "delete from ${shardddl1}.${tb1} where a=$((i + 200))" + run_sql_source1 "insert into ${shardddl1}.${tb1}(a,b,c) values($((i + 100)),$i,$i)" + done + run_sql_tidb_with_retry_times "select count(1) from ${shardddl}.${tb};" "count(1): 11" 30 + run_sql_tidb "create table ${shardddl}.${tb}_temp (a int primary key auto_increment, b int unique not null, c int) auto_increment = 100; + insert into ${shardddl}.${tb}_temp (a, b, c) select a, b, c from ${shardddl}.${tb}; + drop table ${shardddl}.${tb}; rename table ${shardddl}.${tb}_temp to ${shardddl}.${tb};" + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml 30 + compactCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep "finish to compact" | wc -l) + # As compact is affected by "j.tp == flush", the check count of compact use "-le 50" + if [[ "$compactCnt" -le 50 ]]; then + echo "compact $compactCnt dmls which is less than 50" + exit 1 + fi +} + +function DM_COMPACT_USE_DOWNSTREAM_SCHEMA() { + # downstream pk/uk/column is diffrent with upstream, compact use downstream schema. + ps aux | grep dm-worker | awk '{print $2}' | xargs kill || true + check_port_offline $WORKER1_PORT 20 + check_port_offline $WORKER2_PORT 20 + # DownstreamIdentifyKeyCheckInCompact=return(20) will check whether the key value in compact is less than 20, if false, it will be panic. + # This goal is check whether it use downstream schema in compator. + # if use downstream schema, key will be 'b' with value less than 20. + # If use upstream schema, key will be 'a' with value greater than 100. + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/syncer/SkipFlushCompactor=return();github.com/pingcap/tiflow/dm/syncer/DownstreamIdentifyKeyCheckInCompact=return(20)' + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT + + run_case COMPACT_USE_DOWNSTREAM_SCHEMA "single-source-no-sharding" \ + "run_sql_source1 \"create table ${shardddl1}.${tb1} (a int primary key, b int unique not null, c int);\"; + run_sql_tidb \"drop database if exists ${shardddl}; create database ${shardddl}; create table ${shardddl}.${tb} (a int, b int unique not null, c int, d int primary key auto_increment) auto_increment = 100;\"" \ + "clean_table" "" +} + function DM_MULTIPLE_ROWS_CASE() { END=100 for i in $(seq 1 10 $END); do @@ -630,22 +678,56 @@ function DM_MULTIPLE_ROWS_CASE() { run_sql_source1 "delete from ${shardddl1}.${tb1} where a<=$((0 - i)) and a>$((-10 - i))" done + # wait safemode exit + check_log_contain_with_retry "disable safe-mode after task initialization finished" $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log + + # insert again without safmode + for i in $(seq 1 10 $END); do + run_sql_source1 "insert into ${shardddl1}.${tb1}(a,b) values($i,$i),($((i + 1)),$((i + 1))),($((i + 2)),$((i + 2))),($((i + 3)),$((i + 3))),($((i + 4)),$((i + 4))),\ + ($((i + 5)),$((i + 5))),($((i + 6)),$((i + 6))),($((i + 7)),$((i + 7))),($((i + 8)),$((i + 8))),($((i + 9)),$((i + 9)))" + done + for i in $(seq 1 10 $END); do + run_sql_source1 "update ${shardddl1}.${tb1} set c=1 where a>=$i and a<$((i + 10))" + done + for i in $(seq 1 10 $END); do + run_sql_source1 "update ${shardddl1}.${tb1} set b = 0 - b where a>=$i and a<$((i + 10))" + done + for i in $(seq 1 10 $END); do + run_sql_source1 "update ${shardddl1}.${tb1} set a = 0 - a where a>=$i and a<$((i + 10))" + done + for i in $(seq 1 10 $END); do + run_sql_source1 "delete from ${shardddl1}.${tb1} where a<=$((0 - i)) and a>$((-10 - i))" + done + # insert new values, otherwise there may not be any data in downstream in middle stage and check_sync_diff return true immediately - for i in $(seq 100 110 $END); do + for i in $(seq 101 10 200); do run_sql_source1 "insert into ${shardddl1}.${tb1}(a,b) values($i,$i),($((i + 1)),$((i + 1))),($((i + 2)),$((i + 2))),($((i + 3)),$((i + 3))),($((i + 4)),$((i + 4))),\ ($((i + 5)),$((i + 5))),($((i + 6)),$((i + 6))),($((i + 7)),$((i + 7))),($((i + 8)),$((i + 8))),($((i + 9)),$((i + 9)))" done + + run_sql_tidb_with_retry "select count(1) from ${shardddl}.${tb} where a>100 and a<=200;" "count(1): 100" check_sync_diff $WORK_DIR $cur/conf/diff_config.toml 30 - insertMergeCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep '"original op"=insert' | wc -l) - updateMergeCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep '"original op"=update' | wc -l) - deleteMergeCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep '"original op"=delete' | wc -l) - if [[ "$insertMergeCnt" -le 5 || "$updateMergeCnt" -le 5 || "$deleteMergeCnt" -le 5 ]]; then - echo "merge dmls less than 5, insertMergeCnt: $insertMergeCnt, updateMergeCnt: $updateMergeCnt, deleteMergeCnt: $deleteMergeCnt" + insertMergeCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep '\[op=insert\]' | wc -l) + replaceMergeCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep '\[op=replace\]' | wc -l) + updateMergeCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep '\[op="insert on duplicate update"\]' | wc -l) + deleteMergeCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep '\[op=delete\]' | wc -l) + echo $insertMergeCnt $replaceMergeCnt $updateMergeCnt $deleteMergeCnt + if [[ "$insertMergeCnt" -le 5 || "$updateMergeCnt" -le 5 || "$deleteMergeCnt" -le 5 || "$replaceMergeCnt" -le 5 ]]; then + echo "merge dmls less than 5, insertMergeCnt: $insertMergeCnt, replaceMergeCnt: $replaceMergeCnt, updateMergeCnt: $updateMergeCnt, deleteMergeCnt: $deleteMergeCnt" exit 1 fi } function DM_MULTIPLE_ROWS() { + ps aux | grep dm-worker | awk '{print $2}' | xargs kill || true + check_port_offline $WORKER1_PORT 20 + check_port_offline $WORKER2_PORT 20 + export GO_FAILPOINTS='github.com/pingcap/tiflow/dm/syncer/BlockExecuteSQLs=return(1);github.com/pingcap/tiflow/dm/syncer/SafeModeInitPhaseSeconds=return(5)' + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT + run_case MULTIPLE_ROWS "single-source-no-sharding" \ "run_sql_source1 \"create table ${shardddl1}.${tb1} (a int primary key, b int unique, c int);\"" \ "clean_table" "" @@ -677,13 +759,43 @@ function DM_CAUSALITY() { "clean_table" "" } +function DM_CAUSALITY_USE_DOWNSTREAM_SCHEMA_CASE() { + run_sql_source1 "insert into ${shardddl1}.${tb1} values(1,2)" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(2,3)" + run_sql_source1 "update ${shardddl1}.${tb1} set a=3, b=4 where b=3" + run_sql_source1 "delete from ${shardddl1}.${tb1} where a=1" + run_sql_source1 "insert into ${shardddl1}.${tb1} values(1,3)" + + run_sql_tidb_with_retry_times "select count(1) from ${shardddl}.${tb} where a =1 and b=3;" "count(1): 1" 30 + run_sql_tidb "create table ${shardddl}.${tb}_temp (a int primary key, b int unique); + insert into ${shardddl}.${tb}_temp (a, b) select a, b from ${shardddl}.${tb}; + drop table ${shardddl}.${tb}; rename table ${shardddl}.${tb}_temp to ${shardddl}.${tb};" + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + + causalityCnt=$(cat $WORK_DIR/worker1/log/dm-worker.log $WORK_DIR/worker2/log/dm-worker.log | grep "meet causality key, will generate a conflict job to flush all sqls" | wc -l) + if [[ "$causalityCnt" -ne 0 ]]; then + echo "causalityCnt is $causalityCnt, but it should be 0" + exit 1 + fi +} + +function DM_CAUSALITY_USE_DOWNSTREAM_SCHEMA() { + # downstream pk/uk/column is diffrent with upstream, causality use downstream schema. + run_case CAUSALITY_USE_DOWNSTREAM_SCHEMA "single-source-no-sharding" \ + "run_sql_source1 \"create table ${shardddl1}.${tb1} (a int primary key, b int unique);\"; + run_sql_tidb \"drop database if exists ${shardddl}; create database ${shardddl}; create table ${shardddl}.${tb} (a int, b int unique, c int primary key auto_increment) auto_increment = 100;\"" \ + "clean_table" "" +} + function run() { init_cluster init_database DM_COMPACT + DM_COMPACT_USE_DOWNSTREAM_SCHEMA DM_MULTIPLE_ROWS DM_CAUSALITY + DM_CAUSALITY_USE_DOWNSTREAM_SCHEMA DM_UpdateBARule DM_RENAME_TABLE DM_RENAME_COLUMN_OPTIMISTIC @@ -697,6 +809,7 @@ function run() { DM_${i} sleep 1 done + } cleanup_data $shardddl From f9c7619bd0f88c781e5cdc0b0b17540a04db2164 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Thu, 30 Dec 2021 20:45:50 +0800 Subject: [PATCH 34/48] kv(ticdc): reduce eventfeed rate limited log (#4072) (#4111) close pingcap/tiflow#4006 --- cdc/kv/client.go | 113 +++++++++++++++++++++++----------------- cdc/kv/client_test.go | 17 ++++++ cdc/kv/region_worker.go | 21 +++----- 3 files changed, 87 insertions(+), 64 deletions(-) diff --git a/cdc/kv/client.go b/cdc/kv/client.go index ff586c5b072..0feaddcf623 100644 --- a/cdc/kv/client.go +++ b/cdc/kv/client.go @@ -77,8 +77,12 @@ const ( // frequency of creating new goroutine. defaultRegionChanSize = 128 - // initial size for region rate limit queue + // initial size for region rate limit queue. defaultRegionRateLimitQueueSize = 128 + // Interval of check region retry rate limit queue. + defaultCheckRegionRateLimitInterval = 50 * time.Millisecond + // Duration of warning region retry rate limited too long. + defaultLogRegionRateLimitDuration = 10 * time.Second ) // time interval to force kv client to terminate gRPC stream and reconnect @@ -135,6 +139,33 @@ func newSingleRegionInfo(verID tikv.RegionVerID, span regionspan.ComparableSpan, type regionErrorInfo struct { singleRegionInfo err error + + retryLimitTime *time.Time + logRateLimitDuration time.Duration +} + +func newRegionErrorInfo(info singleRegionInfo, err error) regionErrorInfo { + return regionErrorInfo{ + singleRegionInfo: info, + err: err, + + logRateLimitDuration: defaultLogRegionRateLimitDuration, + } +} + +func (r *regionErrorInfo) logRateLimitedHint() bool { + now := time.Now() + if r.retryLimitTime == nil { + // Caller should log on the first rate limited. + r.retryLimitTime = &now + return true + } + if now.Sub(*r.retryLimitTime) > r.logRateLimitDuration { + // Caller should log if it lasts too long. + r.retryLimitTime = &now + return true + } + return false } type regionFeedState struct { @@ -518,9 +549,10 @@ func (s *eventFeedSession) eventFeed(ctx context.Context, ts uint64) error { } }) + tableID, tableName := util.TableIDFromCtx(ctx) + cfID := util.ChangefeedIDFromCtx(ctx) g.Go(func() error { - checkRateLimitInterval := 50 * time.Millisecond - timer := time.NewTimer(checkRateLimitInterval) + timer := time.NewTimer(defaultCheckRegionRateLimitInterval) defer timer.Stop() for { select { @@ -528,12 +560,27 @@ func (s *eventFeedSession) eventFeed(ctx context.Context, ts uint64) error { return ctx.Err() case <-timer.C: s.handleRateLimit(ctx) - timer.Reset(checkRateLimitInterval) + timer.Reset(defaultCheckRegionRateLimitInterval) case errInfo := <-s.errCh: s.errChSizeGauge.Dec() allowed := s.checkRateLimit(errInfo.singleRegionInfo.verID.GetID()) if !allowed { - // rate limit triggers, add the error info to the rate limit queue + if errInfo.logRateLimitedHint() { + zapFieldAddr := zap.Skip() + if errInfo.singleRegionInfo.rpcCtx != nil { + // rpcCtx may be nil if we fails to get region info + // from pd. It could cause by pd down or the region + // has been merged. + zapFieldAddr = zap.String("addr", errInfo.singleRegionInfo.rpcCtx.Addr) + } + log.Info("EventFeed retry rate limited", + zap.Uint64("regionID", errInfo.singleRegionInfo.verID.GetID()), + zap.Uint64("ts", errInfo.singleRegionInfo.ts), + zap.String("changefeed", cfID), zap.Stringer("span", errInfo.span), + zap.Int64("tableID", tableID), zap.String("tableName", tableName), + zapFieldAddr) + } + // rate limit triggers, add the error info to the rate limit queue. s.rateLimitQueue = append(s.rateLimitQueue, errInfo) } else { err := s.handleError(ctx, errInfo) @@ -626,14 +673,13 @@ func (s *eventFeedSession) scheduleRegionRequest(ctx context.Context, sri single // onRegionFail handles a region's failure, which means, unlock the region's range and send the error to the errCh for // error handling. This function is non blocking even if error channel is full. // CAUTION: Note that this should only be called in a context that the region has locked it's range. -func (s *eventFeedSession) onRegionFail(ctx context.Context, errorInfo regionErrorInfo, revokeToken bool) error { +func (s *eventFeedSession) onRegionFail(ctx context.Context, errorInfo regionErrorInfo, revokeToken bool) { log.Debug("region failed", zap.Uint64("regionID", errorInfo.verID.GetID()), zap.Error(errorInfo.err)) s.rangeLock.UnlockRange(errorInfo.span.Start, errorInfo.span.End, errorInfo.verID.GetID(), errorInfo.verID.GetVer(), errorInfo.ts) if revokeToken { s.regionRouter.Release(errorInfo.rpcCtx.Addr) } s.enqueueError(ctx, errorInfo) - return nil } // requestRegionToStore gets singleRegionInfo from regionRouter, which is a token @@ -727,13 +773,8 @@ func (s *eventFeedSession) requestRegionToStore( } bo := tikv.NewBackoffer(ctx, tikvRequestMaxBackoff) s.client.regionCache.OnSendFail(bo, rpcCtx, regionScheduleReload, err) - err = s.onRegionFail(ctx, regionErrorInfo{ - singleRegionInfo: sri, - err: &connectToStoreErr{}, - }, false /* revokeToken */) - if err != nil { - return errors.Trace(err) - } + errInfo := newRegionErrorInfo(sri, &connectToStoreErr{}) + s.onRegionFail(ctx, errInfo, false /* revokeToken */) continue } s.addStream(rpcCtx.Addr, stream, streamCancel) @@ -787,15 +828,8 @@ func (s *eventFeedSession) requestRegionToStore( continue } - // Wait for a while and retry sending the request - time.Sleep(time.Millisecond * time.Duration(rand.Intn(100))) - err = s.onRegionFail(ctx, regionErrorInfo{ - singleRegionInfo: sri, - err: &sendRequestToStoreErr{}, - }, false /* revokeToken */) - if err != nil { - return errors.Trace(err) - } + errInfo := newRegionErrorInfo(sri, &sendRequestToStoreErr{}) + s.onRegionFail(ctx, errInfo, false /* revokeToken */) } else { s.regionRouter.Acquire(rpcCtx.Addr) } @@ -856,15 +890,8 @@ func (s *eventFeedSession) dispatchRequest( log.Info("cannot get rpcCtx, retry span", zap.Uint64("regionID", sri.verID.GetID()), zap.Stringer("span", sri.span)) - err = s.onRegionFail(ctx, regionErrorInfo{ - singleRegionInfo: sri, - err: &rpcCtxUnavailableErr{ - verID: sri.verID, - }, - }, false /* revokeToken */) - if err != nil { - return errors.Trace(err) - } + errInfo := newRegionErrorInfo(sri, &rpcCtxUnavailableErr{verID: sri.verID}) + s.onRegionFail(ctx, errInfo, false /* revokeToken */) continue } sri.rpcCtx = rpcCtx @@ -980,15 +1007,11 @@ func (s *eventFeedSession) handleRateLimit(ctx context.Context) { } // checkRateLimit checks whether a region can be reconnected based on its rate limiter -func (s *eventFeedSession) checkRateLimit(regionID uint64) (allowed bool) { +func (s *eventFeedSession) checkRateLimit(regionID uint64) bool { limiter := s.client.getRegionLimiter(regionID) // use Limiter.Allow here since if exceed the rate limit, we skip this region // and try it later. - allowed = limiter.Allow() - if !allowed { - log.Info("EventFeed retry rate limited", zap.Uint64("regionID", regionID)) - } - return + return limiter.Allow() } // handleError handles error returned by a region. If some new EventFeed connection should be established, the region @@ -1086,14 +1109,8 @@ func (s *eventFeedSession) receiveFromStream( remainingRegions := pendingRegions.takeAll() for _, state := range remainingRegions { - err := s.onRegionFail(ctx, regionErrorInfo{ - singleRegionInfo: state.sri, - err: cerror.ErrPendingRegionCancel.GenWithStackByArgs(), - }, true /* revokeToken */) - if err != nil { - // The only possible is that the ctx is cancelled. Simply return. - return - } + errInfo := newRegionErrorInfo(state.sri, cerror.ErrPendingRegionCancel.FastGenByArgs()) + s.onRegionFail(ctx, errInfo, true /* revokeToken */) } }() @@ -1105,9 +1122,7 @@ func (s *eventFeedSession) receiveFromStream( // to call exactly once from outter code logic worker := newRegionWorker(s, addr) - defer func() { - worker.evictAllRegions() //nolint:errcheck - }() + defer worker.evictAllRegions() g.Go(func() error { return worker.run(ctx) diff --git a/cdc/kv/client_test.go b/cdc/kv/client_test.go index 895ba2f1d36..5fa97d76e6e 100644 --- a/cdc/kv/client_test.go +++ b/cdc/kv/client_test.go @@ -44,6 +44,7 @@ import ( "github.com/pingcap/tiflow/pkg/txnutil" "github.com/pingcap/tiflow/pkg/util" "github.com/pingcap/tiflow/pkg/util/testleak" + "github.com/stretchr/testify/require" "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/testutils" "github.com/tikv/client-go/v2/tikv" @@ -3585,3 +3586,19 @@ func (s *clientSuite) TestHandleRateLimit(c *check.C) { c.Assert(session.rateLimitQueue, check.HasLen, 0) c.Assert(cap(session.rateLimitQueue), check.Equals, 128) } + +func TestRegionErrorInfoLogRateLimitedHint(t *testing.T) { + t.Parallel() + + errInfo := newRegionErrorInfo(singleRegionInfo{}, nil) + errInfo.logRateLimitDuration = time.Second + + // True on the first rate limited. + require.True(t, errInfo.logRateLimitedHint()) + require.False(t, errInfo.logRateLimitedHint()) + + // True if it lasts too long. + time.Sleep(2 * errInfo.logRateLimitDuration) + require.True(t, errInfo.logRateLimitedHint()) + require.False(t, errInfo.logRateLimitedHint()) +} diff --git a/cdc/kv/region_worker.go b/cdc/kv/region_worker.go index ad37a3eee5b..d0f55a4dbee 100644 --- a/cdc/kv/region_worker.go +++ b/cdc/kv/region_worker.go @@ -275,13 +275,8 @@ func (w *regionWorker) handleSingleRegionError(err error, state *regionFeedState revokeToken := !state.initialized // since the context used in region worker will be cancelled after region // worker exits, we must use the parent context to prevent regionErrorInfo loss. - err2 := w.session.onRegionFail(w.parentCtx, regionErrorInfo{ - singleRegionInfo: state.sri, - err: err, - }, revokeToken) - if err2 != nil { - return err2 - } + errInfo := newRegionErrorInfo(state.sri, err) + w.session.onRegionFail(w.parentCtx, errInfo, revokeToken) return retErr } @@ -771,8 +766,7 @@ func (w *regionWorker) handleResolvedTs( // evictAllRegions is used when gRPC stream meets error and re-establish, notify // all existing regions to re-establish -func (w *regionWorker) evictAllRegions() error { - var err error +func (w *regionWorker) evictAllRegions() { for _, states := range w.statesManager.states { states.Range(func(_, value interface{}) bool { state := value.(*regionFeedState) @@ -792,14 +786,11 @@ func (w *regionWorker) evictAllRegions() error { // since the context used in region worker will be cancelled after // region worker exits, we must use the parent context to prevent // regionErrorInfo loss. - err = w.session.onRegionFail(w.parentCtx, regionErrorInfo{ - singleRegionInfo: state.sri, - err: cerror.ErrEventFeedAborted.FastGenByArgs(), - }, revokeToken) - return err == nil + errInfo := newRegionErrorInfo(state.sri, cerror.ErrEventFeedAborted.FastGenByArgs()) + w.session.onRegionFail(w.parentCtx, errInfo, revokeToken) + return true }) } - return err } func getWorkerPoolSize() (size int) { From ed96ff4e3f163fc587522e161bf6c91f09747498 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Thu, 30 Dec 2021 22:39:50 +0800 Subject: [PATCH 35/48] metrics(ticdc): add resolved ts and add changefeed to dataflow (#4038) (#4104) --- cdc/owner/changefeed.go | 21 +- cdc/owner/metrics.go | 18 +- metrics/grafana/ticdc.json | 7655 ++++++++++++++++++------------------ 3 files changed, 3971 insertions(+), 3723 deletions(-) diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go index a152d2382a4..a65039fd46a 100644 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -70,6 +70,8 @@ type changefeed struct { metricsChangefeedCheckpointTsGauge prometheus.Gauge metricsChangefeedCheckpointTsLagGauge prometheus.Gauge + metricsChangefeedResolvedTsGauge prometheus.Gauge + metricsChangefeedResolvedTsLagGauge prometheus.Gauge newDDLPuller func(ctx cdcContext.Context, startTs uint64) (DDLPuller, error) newSink func(ctx cdcContext.Context) (AsyncSink, error) @@ -284,6 +286,9 @@ LOOP: // init metrics c.metricsChangefeedCheckpointTsGauge = changefeedCheckpointTsGauge.WithLabelValues(c.id) c.metricsChangefeedCheckpointTsLagGauge = changefeedCheckpointTsLagGauge.WithLabelValues(c.id) + c.metricsChangefeedResolvedTsGauge = changefeedResolvedTsGauge.WithLabelValues(c.id) + c.metricsChangefeedResolvedTsLagGauge = changefeedResolvedTsLagGauge.WithLabelValues(c.id) + c.initialized = true return nil } @@ -311,10 +316,17 @@ func (c *changefeed) releaseResources(ctx context.Context) { log.Warn("Closing sink failed in Owner", zap.String("changefeedID", c.state.ID), zap.Error(err)) } c.wg.Wait() + changefeedCheckpointTsGauge.DeleteLabelValues(c.id) changefeedCheckpointTsLagGauge.DeleteLabelValues(c.id) c.metricsChangefeedCheckpointTsGauge = nil c.metricsChangefeedCheckpointTsLagGauge = nil + + changefeedResolvedTsGauge.DeleteLabelValues(c.id) + changefeedResolvedTsLagGauge.DeleteLabelValues(c.id) + c.metricsChangefeedResolvedTsGauge = nil + c.metricsChangefeedResolvedTsLagGauge = nil + c.initialized = false } @@ -503,10 +515,13 @@ func (c *changefeed) updateStatus(currentTs int64, barrierTs model.Ts) { } return status, changed, nil }) - phyTs := oracle.ExtractPhysical(checkpointTs) + phyCkpTs := oracle.ExtractPhysical(checkpointTs) + c.metricsChangefeedCheckpointTsGauge.Set(float64(phyCkpTs)) + c.metricsChangefeedCheckpointTsLagGauge.Set(float64(currentTs-phyCkpTs) / 1e3) - c.metricsChangefeedCheckpointTsGauge.Set(float64(phyTs)) - c.metricsChangefeedCheckpointTsLagGauge.Set(float64(currentTs-phyTs) / 1e3) + phyRTs := oracle.ExtractPhysical(resolvedTs) + c.metricsChangefeedResolvedTsGauge.Set(float64(phyRTs)) + c.metricsChangefeedResolvedTsLagGauge.Set(float64(currentTs-phyRTs) / 1e3) } func (c *changefeed) Close(ctx context.Context) { diff --git a/cdc/owner/metrics.go b/cdc/owner/metrics.go index 4aeb513649b..bc1bff8d0aa 100644 --- a/cdc/owner/metrics.go +++ b/cdc/owner/metrics.go @@ -28,7 +28,21 @@ var ( Namespace: "ticdc", Subsystem: "owner", Name: "checkpoint_ts_lag", - Help: "checkpoint ts lag of changefeeds", + Help: "checkpoint ts lag of changefeeds in seconds", + }, []string{"changefeed"}) + changefeedResolvedTsGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "owner", + Name: "resolved_ts", + Help: "resolved ts of changefeeds", + }, []string{"changefeed"}) + changefeedResolvedTsLagGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "owner", + Name: "resolved_ts_lag", + Help: "resolved ts lag of changefeeds in seconds", }, []string{"changefeed"}) ownershipCounter = prometheus.NewCounter( prometheus.CounterOpts{ @@ -63,7 +77,9 @@ const ( // InitMetrics registers all metrics used in owner func InitMetrics(registry *prometheus.Registry) { registry.MustRegister(changefeedCheckpointTsGauge) + registry.MustRegister(changefeedResolvedTsGauge) registry.MustRegister(changefeedCheckpointTsLagGauge) + registry.MustRegister(changefeedResolvedTsLagGauge) registry.MustRegister(ownershipCounter) registry.MustRegister(ownerMaintainTableNumGauge) registry.MustRegister(changefeedStatusGauge) diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index f1e6bda773c..83e9388f2e5 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -125,7 +125,7 @@ "gnetId": null, "graphTooltip": 1, "id": null, - "iteration": 1636101093950, + "iteration": 1640447119260, "links": [], "panels": [ { @@ -196,6 +196,13 @@ "intervalFactor": 1, "legendFormat": "TiKV-{{instance}}", "refId": "B" + }, + { + "expr": "(time() - process_start_time_seconds{tidb_cluster=\"$tidb_cluster\", job=\"pd\"})", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "PD-{{instance}}", + "refId": "C" } ], "thresholds": [], @@ -823,24 +830,22 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "The history of PD cluster leadership, leader node has a value that is great than 0", "fill": 1, - "fillGradient": 0, "gridPos": { "h": 7, "w": 6, "x": 18, "y": 8 }, - "hiddenSeries": false, - "id": 114, + "id": 293, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": true, - "max": true, + "max": false, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "total": false, "values": true @@ -849,10 +854,6 @@ "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -860,35 +861,23 @@ "seriesOverrides": [], "spaceLength": 10, "stack": false, - "steppedLine": false, + "steppedLine": true, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_server_etcd_health_check_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,instance))", + "expr": "pd_tso_role{tidb_cluster=\"$tidb_cluster\", dc=\"global\"} > BOOL 0.5", "format": "time_series", + "hide": false, + "interval": "30s", "intervalFactor": 1, - "legendFormat": "p999-{{instance}}", + "legendFormat": "PD-{{instance}}", "refId": "A" - }, - { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_server_etcd_health_check_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "p99-{{instance}}", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_server_etcd_health_check_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "p95-{{instance}}", - "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Etcd health check duration", + "title": "PD leader history", "tooltip": { "shared": true, "sort": 0, @@ -904,7 +893,7 @@ }, "yaxes": [ { - "format": "s", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -931,155 +920,698 @@ }, { "collapsed": true, - "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, "y": 1 }, - "id": 266, + "id": 11, "panels": [ { - "cards": { - "cardPadding": 1, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": null, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", + "columns": [ + { + "text": "Current", + "value": "current" + } + ], "datasource": "${DS_TEST-CLUSTER}", - "description": "", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, + "description": "The number of captured table of TiCDC nodes ", + "fontSize": "100%", "gridPos": { - "h": 8, - "w": 12, + "h": 5, + "w": 7, "x": 0, - "y": 2 + "y": 3 }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 262, - "legend": { - "show": true + "id": 4, + "links": [], + "pageSize": null, + "scroll": true, + "showHeader": true, + "sort": { + "col": 0, + "desc": true }, - "pluginVersion": "6.1.6", - "reverseYBuckets": false, + "styles": [ + { + "alias": "Time", + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "pattern": "Time", + "type": "date" + }, + { + "alias": "", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "decimals": 2, + "pattern": "/.*/", + "thresholds": [], + "type": "number", + "unit": "short" + } + ], "targets": [ { - "exemplar": true, - "expr": "sum(rate(ticdc_etcd_worker_tick_reactor_duration_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, capture)", - "format": "heatmap", - "interval": "1", - "intervalFactor": 2, - "legendFormat": "{{le}}", - "queryType": "randomWalk", + "expr": "sum(ticdc_processor_num_of_tables{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}", "refId": "A" } ], - "title": "EtcdWorker tick reactor duration", - "tooltip": { - "show": true, - "showHistogram": false - }, - "tooltipDecimals": null, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": null, - "format": "s", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "auto", - "yBucketNumber": null, - "yBucketSize": null + "timeFrom": null, + "timeShift": null, + "title": "Changefeed table count", + "transform": "timeseries_aggregations", + "type": "table" }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, + "columns": [ + { + "text": "Current", + "value": "current" + } + ], "datasource": "${DS_TEST-CLUSTER}", - "description": "", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, - "fill": 1, - "fillGradient": 0, + "description": "Internal resolved ts of TiCDC nodes", + "fontSize": "100%", "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 2 - }, - "hiddenSeries": false, - "id": 264, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false + "h": 10, + "w": 7, + "x": 7, + "y": 3 }, - "lines": true, - "linewidth": 1, - "nullPointMode": "null", - "options": { - "alertThreshold": true + "id": 90, + "links": [], + "pageSize": null, + "scroll": true, + "showHeader": true, + "sort": { + "col": 0, + "desc": true }, - "percentage": false, - "pluginVersion": "6.1.6", - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, + "styles": [ + { + "alias": "table", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "decimals": 2, + "mappingType": 1, + "pattern": "Metric", + "thresholds": [], + "type": "string", + "unit": "short" + }, + { + "alias": "resolved ts", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "MM-DD HH:mm:ss.SSS", + "decimals": 2, + "pattern": "Current", + "thresholds": [], + "type": "date", + "unit": "short" + } + ], "targets": [ { - "exemplar": true, - "expr": "histogram_quantile(0.95, sum(rate(ticdc_etcd_worker_tick_reactor_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", + "expr": "max(ticdc_processor_resolved_ts{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture,changefeed)", + "format": "time_series", "interval": "", - "legendFormat": "{{capture}}-95", - "queryType": "randomWalk", + "intervalFactor": 1, + "legendFormat": "{{capture}}-{{changefeed}}", "refId": "A" }, { - "exemplar": true, - "expr": "histogram_quantile(0.99, sum(rate(ticdc_etcd_worker_tick_reactor_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", - "hide": false, + "expr": "max(ticdc_processor_checkpoint_ts{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture,changefeed) > 0", + "format": "time_series", "interval": "", - "legendFormat": "{{capture}}-99", + "intervalFactor": 1, + "legendFormat": "checkpoint-{{capture}}-{{changefeed}}", "refId": "B" } ], + "timeFrom": null, + "timeShift": null, + "title": "Processor resolved ts", + "transform": "timeseries_aggregations", + "type": "table" + }, + { + "columns": [ + { + "text": "Current", + "value": "current" + } + ], + "datasource": "${DS_TEST-CLUSTER}", + "description": "Internal resolved ts of captured tables", + "fontSize": "100%", + "gridPos": { + "h": 10, + "w": 10, + "x": 14, + "y": 3 + }, + "id": 30, + "links": [], + "pageSize": null, + "scroll": true, + "showHeader": true, + "sort": { + "col": 0, + "desc": true + }, + "styles": [ + { + "alias": "table", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "decimals": 2, + "mappingType": 1, + "pattern": "Metric", + "thresholds": [], + "type": "string", + "unit": "short" + }, + { + "alias": "resolved ts", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "dateFormat": "MM-DD HH:mm:ss.SSS", + "decimals": 2, + "pattern": "Current", + "thresholds": [], + "type": "date", + "unit": "short" + } + ], + "targets": [ + { + "expr": "bottomk(10, max(ticdc_processor_table_resolved_ts{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture,changefeed,table))", + "format": "time_series", + "instant": true, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}-{{changefeed}}-{{table}}", + "refId": "A" + }, + { + "expr": "bottomk(10, max(ticdc_processor_checkpoint_ts{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture,changefeed,table) > 0)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "checkpoint-{{capture}}-{{changefeed}}", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Table resolved ts", + "transform": "timeseries_aggregations", + "type": "table" + }, + { + "columns": [ + { + "text": "Current", + "value": "current" + } + ], + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of replicated tables maintained in owner", + "fontSize": "100%", + "gridPos": { + "h": 5, + "w": 7, + "x": 0, + "y": 8 + }, + "id": 138, + "links": [], + "pageSize": null, + "scroll": true, + "showHeader": true, + "sort": { + "col": null, + "desc": false + }, + "styles": [ + { + "alias": "Time", + "align": "auto", + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "pattern": "Time", + "type": "date" + }, + { + "alias": "", + "align": "auto", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "decimals": 2, + "pattern": "/.*/", + "thresholds": [], + "type": "number", + "unit": "short" + } + ], + "targets": [ + { + "expr": "sum(ticdc_owner_maintain_table_num{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\",type=\"total\"}) by (capture)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}-total", + "refId": "A" + }, + { + "expr": "sum(ticdc_owner_maintain_table_num{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\",type=\"wip\"}) by (capture)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}-wip", + "refId": "B" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Table count maintained by owner", + "transform": "timeseries_aggregations", + "type": "table" + }, + { + "aliasColors": {}, + "bars": true, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The checkpoint ts of changefeeds.", + "fill": 0, + "gridPos": { + "h": 7, + "w": 9, + "x": 0, + "y": 13 + }, + "id": 86, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "sideWidth": null, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 2, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/approximate current time.*/", + "bars": false + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "max(pd_cluster_tso{tidb_cluster=\"$tidb_cluster\"})", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "approximate current time (s)", + "refId": "A" + }, + { + "expr": "max(ticdc_owner_checkpoint_ts{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed) > 0", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}", + "refId": "B" + }, + { + "expr": "max(ticdc_owner_resolved_ts{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed) > 0", + "format": "time_series", + "hide": true, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-resolvedts", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Changefeed checkpoint", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "max": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "dateTimeAsIso", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Request count of etcd operation per second", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 8, + "x": 9, + "y": 13 + }, + "hiddenSeries": false, + "id": 102, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(ticdc_etcd_request_count{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (capture, type)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}-{{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "PD etcd requests/s", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of errors that interrupt changefeed per minute ", + "fill": 1, + "gridPos": { + "h": 7, + "w": 7, + "x": 17, + "y": 13 + }, + "id": 82, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(delta(ticdc_processor_exit_with_error_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Exit error count/m", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The lag between changefeed checkpoint ts and the latest ts of upstream TiDB.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 20 + }, + "hiddenSeries": false, + "id": 3, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "max(ticdc_owner_checkpoint_ts_lag{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}", + "refId": "A" + }, + { + "expr": "sum(ticdc_processor_checkpoint_ts_lag{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture,changefeed)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}", + "refId": "B" + }, + { + "expr": "max(ticdc_owner_resolved_ts_lag{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-resolvedts", + "refId": "C" + }, + { + "expr": "sum(ticdc_processor_resolved_ts_lag{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture,changefeed)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}-resolvedts", + "refId": "D" + } + ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "EtcdWorker tick reactor duration", + "title": "Changefeed checkpoint lag", "tooltip": { "shared": true, "sort": 0, @@ -1095,17 +1627,15 @@ }, "yaxes": [ { - "$$hashKey": "object:2612", "format": "s", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { - "$$hashKey": "object:2613", - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -1119,88 +1649,197 @@ } }, { - "cards": { - "cardPadding": 0, - "cardRound": 0 + "aliasColors": {}, + "bars": true, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The derivative of a changefeed checkpoint, ideally it should not be 0 (means the changefeed is stopped).", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 20 }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "max": null, - "min": 1, - "mode": "spectrum" + "hiddenSeries": false, + "id": 2, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "hideZero": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "", - "fieldConfig": { - "defaults": {}, - "overrides": [] + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 0.5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(deriv(ticdc_owner_checkpoint_ts{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (changefeed) / 1000 > 0", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Changefeed checkpoint derivative", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 2, + "max": "3600", + "min": "0.001", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The status of each changefeed.\n\n0: Normal\n\n1: Error\n\n2: Failed\n\n3: Stopped\n\n4: Finished\n\n-1: Unknown", + "fill": 1, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 10 + "y": 27 }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 256, + "id": 163, "legend": { - "show": true + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": true, + "show": true, + "total": false, + "values": true }, - "pluginVersion": "6.1.6", - "reverseYBuckets": false, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 1, + "points": true, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum(rate(ticdc_etcd_worker_etcd_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", + "expr": "ticdc_owner_status{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}", + "format": "time_series", "instant": false, - "interval": "", - "intervalFactor": 2, - "legendFormat": "{{le}}", - "queryType": "randomWalk", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}", "refId": "A" } ], + "thresholds": [], "timeFrom": null, + "timeRegions": [], "timeShift": null, - "title": "EtcdWorker exec etcd txn duration", + "title": "The status of changefeeds", "tooltip": { - "show": true, - "showHistogram": true - }, - "tooltipDecimals": 1, - "type": "heatmap", - "xAxis": { - "show": true + "shared": true, + "sort": 0, + "value_type": "individual" }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 1, - "format": "s", - "logBase": 1, - "max": null, - "min": null, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, "show": true, - "splitFactor": null + "values": [] }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { "aliasColors": {}, - "bars": false, + "bars": true, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "", + "description": "Estimate the remaining time for a changefeed catch-up upstream TiDB.", "fieldConfig": { "defaults": { "unit": "s" @@ -1210,30 +1849,35 @@ "fill": 1, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 10 + "y": 27 }, "hiddenSeries": false, - "id": 258, + "id": 253, "legend": { + "alignAsTable": true, "avg": false, - "current": false, + "current": true, + "hideEmpty": true, + "hideZero": true, "max": false, "min": false, + "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, - "lines": true, + "lines": false, "linewidth": 1, + "links": [], "nullPointMode": "null", "options": { "alertThreshold": true }, + "paceLength": 10, "percentage": false, - "pluginVersion": "6.1.6", "pointradius": 2, "points": false, "renderer": "flot", @@ -1244,27 +1888,19 @@ "targets": [ { "exemplar": true, - "expr": "histogram_quantile(0.95, sum(rate(ticdc_etcd_worker_etcd_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", + "expr": "abs(max(ticdc_owner_checkpoint_ts_lag{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"} / (deriv(ticdc_owner_checkpoint_ts{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])/1000)) by (changefeed))", "format": "time_series", "interval": "", - "legendFormat": "{{capture}}-p95", - "queryType": "randomWalk", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}", "refId": "A" - }, - { - "exemplar": true, - "expr": "histogram_quantile(0.99, sum(rate(ticdc_etcd_worker_etcd_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", - "hide": false, - "interval": "", - "legendFormat": "{{lcapture}-p99}", - "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "EtcdWorker exec etcd txn duration percentile", + "title": "Changefeed checkpoint catch-up ETA", "tooltip": { "shared": true, "sort": 0, @@ -1280,16 +1916,14 @@ }, "yaxes": [ { - "$$hashKey": "object:1612", "format": "s", - "label": null, + "label": "", "logBase": 2, - "max": null, - "min": null, + "max": "604800", + "min": "1", "show": true }, { - "$$hashKey": "object:1613", "format": "short", "label": null, "logBase": 1, @@ -1305,58 +1939,64 @@ }, { "cards": { - "cardPadding": null, - "cardRound": null + "cardPadding": 0, + "cardRound": 0 }, "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolatePurples", + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", "exponent": 0.5, - "min": 0, + "max": null, + "min": 1, "mode": "spectrum" }, "dataFormat": "tsbuckets", "datasource": "${DS_TEST-CLUSTER}", - "description": "", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, + "description": "Sink write duration of changefeeds", "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 18 + "y": 34 }, "heatmap": {}, "hideZeroBuckets": true, "highlightCards": true, - "id": 254, + "id": 94, "legend": { - "show": true + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true }, - "pluginVersion": "6.1.6", + "links": [], + "repeat": null, + "repeatDirection": "h", "reverseYBuckets": false, "targets": [ { - "exemplar": true, - "expr": "sum(rate(ticdc_etcd_worker_etcd_txn_size_bytes_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, capture)", + "expr": "max(rate(ticdc_sink_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le)", "format": "heatmap", "instant": false, - "interval": "", - "intervalFactor": 1, + "intervalFactor": 2, "legendFormat": "{{le}}", - "queryType": "randomWalk", "refId": "A" } ], - "title": "EtcdWorker txn size ", + "title": "Sink write duration", "tooltip": { "show": true, "showHistogram": true }, - "tooltipDecimals": null, + "tooltipDecimals": 1, "type": "heatmap", "xAxis": { "show": true @@ -1364,15 +2004,15 @@ "xBucketNumber": null, "xBucketSize": null, "yAxis": { - "decimals": null, - "format": "decbytes", + "decimals": 1, + "format": "s", "logBase": 1, "max": null, "min": null, "show": true, "splitFactor": null }, - "yBucketBound": "auto", + "yBucketBound": "upper", "yBucketNumber": null, "yBucketSize": null }, @@ -1382,38 +2022,37 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, + "description": "Percentiles of sink write duration of changefeeds", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 18 + "y": 34 }, "hiddenSeries": false, - "id": 260, + "id": 35, "legend": { + "alignAsTable": true, "avg": false, - "current": false, + "current": true, "max": false, "min": false, + "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, + "links": [], "nullPointMode": "null", "options": { - "alertThreshold": true + "dataLinks": [] }, + "paceLength": 10, "percentage": false, - "pluginVersion": "6.1.6", "pointradius": 2, "points": false, "renderer": "flot", @@ -1423,27 +2062,32 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "histogram_quantile(0.95, sum(rate(ticdc_etcd_worker_etcd_txn_size_bytes_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", - "interval": "", - "legendFormat": "{{capture}}-p95", - "queryType": "randomWalk", + "expr": "histogram_quantile(0.95, sum(rate(ticdc_sink_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p95", "refId": "A" }, { - "exemplar": true, - "expr": "histogram_quantile(0.99, sum(rate(ticdc_etcd_worker_etcd_txn_size_bytes_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", - "hide": false, - "interval": "", - "legendFormat": "{{capture}}-p99", + "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p99", "refId": "B" + }, + { + "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p999", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "EtcdWorker txn size percentile", + "title": "Sink write duration percentile", "tooltip": { "shared": true, "sort": 0, @@ -1459,16 +2103,14 @@ }, "yaxes": [ { - "$$hashKey": "object:2055", - "format": "bytes", + "format": "s", "label": null, - "logBase": 1, + "logBase": 2, "max": null, "min": null, "show": true }, { - "$$hashKey": "object:2056", "format": "short", "label": null, "logBase": 1, @@ -1481,92 +2123,109 @@ "align": false, "alignLevel": null } - } - ], - "title": "EtcdWorker", - "type": "row" - }, - { - "collapsed": true, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 1 - }, - "id": 266, - "panels": [ + }, { - "cards": { - "cardPadding": 1, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": null, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, + "description": "The number of changed rows that are written to downstream per second", + "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 2 + "y": 41 }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 262, + "hiddenSeries": false, + "id": 34, "legend": { - "show": true + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true }, - "pluginVersion": "6.1.6", - "reverseYBuckets": false, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum(rate(ticdc_etcd_worker_tick_reactor_duration_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, capture)", - "format": "heatmap", - "interval": "1", - "intervalFactor": 2, - "legendFormat": "{{le}}", - "queryType": "randomWalk", + "expr": "sum (rate(ticdc_sink_txn_batch_size_sum{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}", "refId": "A" + }, + { + "expr": "sum (rate(ticdc_sink_txn_batch_size_sum{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (changefeed)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "total", + "refId": "B" } ], - "title": "EtcdWorker tick reactor duration", + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Sink write rows count/s", "tooltip": { - "show": true, - "showHistogram": false - }, - "tooltipDecimals": null, - "type": "heatmap", - "xAxis": { - "show": true + "shared": true, + "sort": 0, + "value_type": "individual" }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": null, - "format": "s", - "logBase": 1, - "max": null, - "min": null, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, "show": true, - "splitFactor": null - }, - "yBucketBound": "auto", - "yBucketNumber": null, - "yBucketSize": null + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { "aliasColors": {}, @@ -1574,38 +2233,37 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, + "description": "Percentiles of sink batch size", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 2 + "y": 41 }, "hiddenSeries": false, - "id": 264, + "id": 36, "legend": { + "alignAsTable": true, "avg": false, - "current": false, + "current": true, "max": false, "min": false, + "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, + "links": [], "nullPointMode": "null", "options": { - "alertThreshold": true + "dataLinks": [] }, + "paceLength": 10, "percentage": false, - "pluginVersion": "6.1.6", "pointradius": 2, "points": false, "renderer": "flot", @@ -1615,27 +2273,33 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "histogram_quantile(0.95, sum(rate(ticdc_etcd_worker_tick_reactor_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", - "interval": "", - "legendFormat": "{{capture}}-95", - "queryType": "randomWalk", + "expr": "histogram_quantile(0.90, sum(rate(ticdc_sink_txn_batch_size_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-p90", "refId": "A" }, { - "exemplar": true, - "expr": "histogram_quantile(0.99, sum(rate(ticdc_etcd_worker_tick_reactor_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", - "hide": false, - "interval": "", - "legendFormat": "{{capture}}-99", + "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_txn_batch_size_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-p99", "refId": "B" + }, + { + "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_txn_batch_size_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "{{capture}}-p999", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "EtcdWorker tick reactor duration", + "title": "Sink write batch size percentile", "tooltip": { "shared": true, "sort": 0, @@ -1651,17 +2315,15 @@ }, "yaxes": [ { - "$$hashKey": "object:2612", - "format": "s", + "format": "none", "label": null, - "logBase": 1, + "logBase": 2, "max": null, "min": null, "show": true }, { - "$$hashKey": "object:2613", - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -1680,52 +2342,53 @@ "cardRound": 0 }, "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", + "cardColor": "#FF9830", + "colorScale": "linear", "colorScheme": "interpolateSpectral", "exponent": 0.5, "max": null, - "min": 1, + "min": 0, "mode": "spectrum" }, "dataFormat": "tsbuckets", "datasource": "${DS_TEST-CLUSTER}", - "description": "", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, + "description": "Asynchronous flush sink duration of changefeeds", "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 10 + "y": 48 }, "heatmap": {}, "hideZeroBuckets": true, "highlightCards": true, - "id": 256, + "id": 93, "legend": { - "show": true + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true }, - "pluginVersion": "6.1.6", + "links": [], "reverseYBuckets": false, "targets": [ { - "exemplar": true, - "expr": "sum(rate(ticdc_etcd_worker_etcd_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le)", + "expr": "max(rate(ticdc_sink_flush_event_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le)", "format": "heatmap", "instant": false, - "interval": "", "intervalFactor": 2, "legendFormat": "{{le}}", - "queryType": "randomWalk", "refId": "A" } ], - "timeFrom": null, - "timeShift": null, - "title": "EtcdWorker exec etcd txn duration", + "title": "Flush sink duration", "tooltip": { "show": true, "showHistogram": true @@ -1756,40 +2419,37 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "", - "fieldConfig": { - "defaults": { - "unit": "s" - }, - "overrides": [] - }, + "description": "Percentiles of asynchronous flush sink duration of changefeeds", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 10 + "y": 48 }, "hiddenSeries": false, - "id": 258, + "id": 98, "legend": { + "alignAsTable": true, "avg": false, - "current": false, + "current": true, "max": false, "min": false, + "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, + "links": [], "nullPointMode": "null", "options": { - "alertThreshold": true + "dataLinks": [] }, + "paceLength": 10, "percentage": false, - "pluginVersion": "6.1.6", "pointradius": 2, "points": false, "renderer": "flot", @@ -1799,28 +2459,32 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "histogram_quantile(0.95, sum(rate(ticdc_etcd_worker_etcd_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", + "expr": "histogram_quantile(0.95, sum(rate(ticdc_sink_flush_event_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance,type))", "format": "time_series", - "interval": "", - "legendFormat": "{{capture}}-p95", - "queryType": "randomWalk", + "intervalFactor": 1, + "legendFormat": "{{instance}}-{{type}}-p95", "refId": "A" }, { - "exemplar": true, - "expr": "histogram_quantile(0.99, sum(rate(ticdc_etcd_worker_etcd_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", - "hide": false, - "interval": "", - "legendFormat": "{{capture}-p99}", + "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_flush_event_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance,type))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-{{type}}-p99", "refId": "B" + }, + { + "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_flush_event_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance,type))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-{{type}}-p999", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "EtcdWorker exec etcd txn duration percentile", + "title": "Flush sink duration percentile", "tooltip": { "shared": true, "sort": 0, @@ -1836,7 +2500,6 @@ }, "yaxes": [ { - "$$hashKey": "object:1612", "format": "s", "label": null, "logBase": 2, @@ -1845,7 +2508,6 @@ "show": true }, { - "$$hashKey": "object:1613", "format": "short", "label": null, "logBase": 1, @@ -1861,58 +2523,63 @@ }, { "cards": { - "cardPadding": null, - "cardRound": null + "cardPadding": 0, + "cardRound": 0 }, "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolatePurples", + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", "exponent": 0.5, - "min": 0, + "max": null, + "min": 1, "mode": "spectrum" }, "dataFormat": "tsbuckets", "datasource": "${DS_TEST-CLUSTER}", - "description": "", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, + "description": "The duration of detecting and waiting conflict of MySQL sink", "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 18 + "y": 55 }, "heatmap": {}, "hideZeroBuckets": true, "highlightCards": true, - "id": 254, + "id": 103, "legend": { - "show": true + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true }, - "pluginVersion": "6.1.6", + "links": [], + "repeatDirection": "h", "reverseYBuckets": false, "targets": [ { - "exemplar": true, - "expr": "sum(rate(ticdc_etcd_worker_etcd_txn_size_bytes_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, capture)", + "expr": "max(rate(ticdc_sink_conflict_detect_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le)", "format": "heatmap", "instant": false, - "interval": "", - "intervalFactor": 1, + "intervalFactor": 2, "legendFormat": "{{le}}", - "queryType": "randomWalk", "refId": "A" } ], - "title": "EtcdWorker txn size ", + "title": "MySQL sink conflict detect duration", "tooltip": { "show": true, "showHistogram": true }, - "tooltipDecimals": null, + "tooltipDecimals": 1, "type": "heatmap", "xAxis": { "show": true @@ -1920,15 +2587,15 @@ "xBucketNumber": null, "xBucketSize": null, "yAxis": { - "decimals": null, - "format": "decbytes", + "decimals": 1, + "format": "s", "logBase": 1, "max": null, "min": null, "show": true, "splitFactor": null }, - "yBucketBound": "auto", + "yBucketBound": "upper", "yBucketNumber": null, "yBucketSize": null }, @@ -1938,71 +2605,108 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, + "description": "Distribution of MySQL worker loads", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 18 + "y": 55 }, "hiddenSeries": false, - "id": 260, + "id": 95, "legend": { - "avg": false, - "current": false, - "max": false, + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, "min": false, + "rightSide": false, "show": true, + "sort": "current", + "sortDesc": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, + "links": [], "nullPointMode": "null", "options": { - "alertThreshold": true + "dataLinks": [] }, "percentage": false, - "pluginVersion": "6.1.6", "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, - "stack": false, + "stack": true, "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "histogram_quantile(0.95, sum(rate(ticdc_etcd_worker_etcd_txn_size_bytes_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", + "expr": "sum(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture,bucket)", + "format": "time_series", + "hide": true, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}-{{bucket}}", + "refId": "A" + }, + { + "expr": "count(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) >= 0)", + "format": "time_series", + "hide": true, "interval": "", - "legendFormat": "{{capture}}-p95", - "queryType": "randomWalk", - "refId": "A" + "intervalFactor": 1, + "legendFormat": "total worker", + "refId": "B" }, { - "exemplar": true, - "expr": "histogram_quantile(0.99, sum(rate(ticdc_etcd_worker_etcd_txn_size_bytes_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", + "expr": "count(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 2)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "0-2 row/s worker", + "refId": "C" + }, + { + "expr": "count(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 2 and rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 10)", + "format": "time_series", "hide": false, "interval": "", - "legendFormat": "{{capture}}-p99", - "refId": "B" + "intervalFactor": 1, + "legendFormat": "2-10 row/s worker", + "refId": "D" + }, + { + "expr": "count(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 10 and rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 100)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "10-100 row/s worker", + "refId": "E" + }, + { + "expr": "count(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 100)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": ">100 row/s worker", + "refId": "F" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "EtcdWorker txn size percentile", + "title": "MySQL sink worker load", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -2015,8 +2719,7 @@ }, "yaxes": [ { - "$$hashKey": "object:2055", - "format": "bytes", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -2024,7 +2727,6 @@ "show": true }, { - "$$hashKey": "object:2056", "format": "short", "label": null, "logBase": 1, @@ -2037,391 +2739,244 @@ "align": false, "alignLevel": null } - } - ], - "title": "EtcdWorker", - "type": "row" - }, - { - "collapsed": true, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 1 - }, - "id": 11, - "panels": [ + }, { - "columns": [ - { - "text": "Current", - "value": "current" - } - ], + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "max": null, + "min": 1, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of captured table of TiCDC nodes ", - "fontSize": "100%", + "description": "The duration of executing DDL in MySQL sink", "gridPos": { - "h": 5, - "w": 7, + "h": 7, + "w": 12, "x": 0, - "y": 2 + "y": 62 }, - "id": 4, - "links": [], - "pageSize": null, - "scroll": true, - "showHeader": true, - "sort": { - "col": 0, - "desc": true + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 165, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true }, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "date" - }, - { - "alias": "", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" - ], - "decimals": 2, - "pattern": "/.*/", - "thresholds": [], - "type": "number", - "unit": "short" - } - ], + "links": [], + "repeatDirection": "h", + "reverseYBuckets": false, "targets": [ { - "expr": "sum(ticdc_processor_num_of_tables{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{capture}}", + "expr": "max(rate(ticdc_sink_ddl_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", "refId": "A" } ], - "timeFrom": null, - "timeShift": null, - "title": "Changefeed table count", - "transform": "timeseries_aggregations", - "type": "table" - }, - { - "columns": [ - { - "text": "Current", - "value": "current" - } - ], - "datasource": "${DS_TEST-CLUSTER}", - "description": "Internal resolved ts of TiCDC nodes", - "fontSize": "100%", - "gridPos": { - "h": 10, - "w": 7, - "x": 7, - "y": 2 + "title": "MySQL sink ddl execution duration", + "tooltip": { + "show": true, + "showHistogram": true }, - "id": 90, - "links": [], - "pageSize": null, - "scroll": true, - "showHeader": true, - "sort": { - "col": 0, - "desc": true + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true }, - "styles": [ - { - "alias": "table", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" - ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "mappingType": 1, - "pattern": "Metric", - "thresholds": [], - "type": "string", - "unit": "short" - }, - { - "alias": "resolved ts", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" - ], - "dateFormat": "MM-DD HH:mm:ss.SSS", - "decimals": 2, - "pattern": "Current", - "thresholds": [], - "type": "date", - "unit": "short" - } - ], - "targets": [ - { - "expr": "max(ticdc_processor_resolved_ts{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture,changefeed)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{capture}}-{{changefeed}}", - "refId": "A" - }, - { - "expr": "max(ticdc_processor_checkpoint_ts{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture,changefeed) > 0", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "checkpoint-{{capture}}-{{changefeed}}", - "refId": "B" - } - ], - "timeFrom": null, - "timeShift": null, - "title": "Processor resolved ts", - "transform": "timeseries_aggregations", - "type": "table" + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null }, { - "columns": [ - { - "text": "Current", - "value": "current" - } - ], + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Internal resolved ts of captured tables", - "fontSize": "100%", + "description": "Percentiles of detecting and waiting conflict duration of MySQL sink", + "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 10, - "w": 10, - "x": 14, - "y": 2 + "h": 7, + "w": 12, + "x": 12, + "y": 62 }, - "id": 30, + "hiddenSeries": false, + "id": 83, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, "links": [], - "pageSize": null, - "scroll": true, - "showHeader": true, - "sort": { - "col": 0, - "desc": true + "nullPointMode": "null", + "options": { + "dataLinks": [] }, - "styles": [ - { - "alias": "table", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" - ], - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "decimals": 2, - "mappingType": 1, - "pattern": "Metric", - "thresholds": [], - "type": "string", - "unit": "short" - }, - { - "alias": "resolved ts", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" - ], - "dateFormat": "MM-DD HH:mm:ss.SSS", - "decimals": 2, - "pattern": "Current", - "thresholds": [], - "type": "date", - "unit": "short" - } - ], + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, "targets": [ { - "expr": "bottomk(10, max(ticdc_processor_table_resolved_ts{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture,changefeed,table))", + "expr": "histogram_quantile(0.95,sum(rate(ticdc_sink_conflict_detect_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", "format": "time_series", - "instant": true, "interval": "", "intervalFactor": 1, - "legendFormat": "{{capture}}-{{changefeed}}-{{table}}", + "legendFormat": "{{instance}}-p95", "refId": "A" }, { - "expr": "bottomk(10, max(ticdc_processor_checkpoint_ts{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture,changefeed,table) > 0)", + "expr": "histogram_quantile(0.99,sum(rate(ticdc_sink_conflict_detect_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "checkpoint-{{capture}}-{{changefeed}}", + "legendFormat": "{{instance}}-p99", "refId": "B" + }, + { + "expr": "histogram_quantile(0.999,sum(rate(ticdc_sink_conflict_detect_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p999", + "refId": "C" } ], + "thresholds": [], "timeFrom": null, + "timeRegions": [], "timeShift": null, - "title": "Table resolved ts", - "transform": "timeseries_aggregations", - "type": "table" - }, - { - "columns": [ - { - "text": "Current", - "value": "current" - } - ], - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of replicated tables maintained in owner", - "fontSize": "100%", - "gridPos": { - "h": 5, - "w": 7, - "x": 0, - "y": 7 + "title": "MySQL sink conflict detect duration percentile", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" }, - "id": 138, - "links": [], - "pageSize": null, - "scroll": true, - "showHeader": true, - "sort": { - "col": null, - "desc": false + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] }, - "styles": [ - { - "alias": "Time", - "align": "auto", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "date" - }, - { - "alias": "", - "align": "auto", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" - ], - "decimals": 2, - "pattern": "/.*/", - "thresholds": [], - "type": "number", - "unit": "short" - } - ], - "targets": [ + "yaxes": [ { - "expr": "sum(ticdc_owner_maintain_table_num{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\",type=\"total\"}) by (capture)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{capture}}-total", - "refId": "A" + "format": "s", + "label": null, + "logBase": 2, + "max": null, + "min": null, + "show": true }, { - "expr": "sum(ticdc_owner_maintain_table_num{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\",type=\"wip\"}) by (capture)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{capture}}-wip", - "refId": "B" + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true } ], - "timeFrom": null, - "timeShift": null, - "title": "Table count maintained by owner", - "transform": "timeseries_aggregations", - "type": "table" + "yaxis": { + "align": false, + "alignLevel": null + } }, { "aliasColors": {}, - "bars": true, - "cacheTimeout": null, + "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The checkpoint ts of changefeeds.", - "fill": 0, + "fill": 1, "gridPos": { "h": 7, - "w": 9, + "w": 12, "x": 0, - "y": 12 + "y": 69 }, - "id": 86, + "id": 149, "legend": { - "alignAsTable": true, "avg": false, - "current": true, + "current": false, "max": false, "min": false, - "rightSide": false, "show": true, - "sideWidth": null, "total": false, - "values": true + "values": false }, "lines": true, - "linewidth": 2, + "linewidth": 1, "links": [], "nullPointMode": "null", "percentage": false, "pointradius": 2, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/approximate current time.*/", - "bars": false - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "max(ticdc_owner_checkpoint_ts{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed) > 0", + "expr": "sum(rate(ticdc_processor_table_memory_consumption_sum{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[30s]) / rate(ticdc_processor_table_memory_consumption_count{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[30s])) by (capture)", "format": "time_series", - "interval": "", "intervalFactor": 1, - "legendFormat": "{{changefeed}}", + "legendFormat": "{{ capture }}", "refId": "A" - }, - { - "expr": "max(pd_cluster_tso{tidb_cluster=\"$tidb_cluster\"})", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": "approximate current time (s)", - "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Changefeed checkpoint", + "title": "Processor Memory Consumption Per Capture", "tooltip": { "shared": true, "sort": 0, @@ -2430,7 +2985,6 @@ "type": "graph", "xaxis": { "buckets": null, - "max": null, "mode": "time", "name": null, "show": true, @@ -2438,7 +2992,7 @@ }, "yaxes": [ { - "format": "dateTimeAsIso", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -2451,7 +3005,7 @@ "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -2465,26 +3019,24 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Request count of etcd operation per second", + "description": "Percentiles of executing DDL duration of MySQL sink", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, - "w": 8, - "x": 9, - "y": 12 + "w": 12, + "x": 12, + "y": 69 }, "hiddenSeries": false, - "id": 102, + "id": 166, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": true, - "hideZero": true, "max": true, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "total": false, "values": true @@ -2496,7 +3048,6 @@ "options": { "dataLinks": [] }, - "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -2507,22 +3058,38 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_etcd_request_count{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (capture, type)", + "expr": "histogram_quantile(0.95,sum(rate(ticdc_sink_ddl_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "{{capture}}-{{type}}", + "legendFormat": "{{instance}}-p95", "refId": "A" + }, + { + "expr": "histogram_quantile(0.99,sum(rate(ticdc_sink_ddl_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p99", + "refId": "B" + }, + { + "expr": "histogram_quantile(0.999,sum(rate(ticdc_sink_ddl_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{instance}}-p999", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "PD etcd requests/s", + "title": "MySQL sink ddl execution duration percentile", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -2535,9 +3102,9 @@ }, "yaxes": [ { - "format": "short", + "format": "s", "label": null, - "logBase": 1, + "logBase": 2, "max": null, "min": null, "show": true @@ -2562,25 +3129,22 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of errors that interrupt changefeed per minute ", "fill": 1, "gridPos": { - "h": 7, - "w": 7, - "x": 17, - "y": 12 + "h": 8, + "w": 12, + "x": 12, + "y": 76 }, - "id": 82, + "id": 164, "legend": { - "alignAsTable": true, "avg": false, - "current": true, + "current": false, "max": false, "min": false, - "rightSide": true, "show": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, @@ -2596,10 +3160,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(delta(ticdc_processor_exit_with_error_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", + "expr": "sum(rate(ticdc_processor_table_memory_consumption_sum{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[30s]) / rate(ticdc_processor_table_memory_consumption_count{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[30s])) by (capture, changefeed)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}", + "legendFormat": "{{ capture }}-{{ changefeed }}", "refId": "A" } ], @@ -2607,7 +3171,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Exit error count/m", + "title": "Processor Memory Consumption Per Changefeed", "tooltip": { "shared": true, "sort": 0, @@ -2623,7 +3187,7 @@ }, "yaxes": [ { - "format": "short", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -2644,30 +3208,197 @@ "alignLevel": null } }, + { + "columns": [], + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "left", + "filterable": true + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "dateTimeAsSystem" + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "capture" + }, + "properties": [ + { + "id": "custom.width", + "value": 191 + } + ] + }, + { + "matcher": { + "id": "byName", + "options": "changefeed" + }, + "properties": [ + { + "id": "custom.width", + "value": 209 + } + ] + } + ] + }, + "fontSize": "100%", + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 77 + }, + "id": 200, + "options": { + "showHeader": true, + "sortBy": [ + { + "desc": true, + "displayName": "last-snap-ts" + } + ] + }, + "pageSize": null, + "pluginVersion": "6.1.6", + "scroll": true, + "showHeader": true, + "sort": { + "col": 0, + "desc": true + }, + "styles": [ + { + "alias": "Time", + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "pattern": "Value", + "type": "date" + }, + { + "alias": "", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "decimals": 2, + "pattern": "/.*/", + "thresholds": [], + "type": "number", + "unit": "short" + } + ], + "targets": [ + { + "exemplar": true, + "expr": "ticdc_processor_schema_storage_gc_ts{changefeed=~\"$changefeed\", capture=~\"$capture\"}", + "format": "table", + "instant": true, + "interval": "", + "legendFormat": "", + "queryType": "randomWalk", + "refId": "A" + } + ], + "title": "Schema Storage GC progress", + "transform": "table", + "transformations": [ + { + "id": "filterFieldsByName", + "options": { + "include": { + "names": [ + "capture", + "changefeed", + "Value" + ] + } + } + }, + { + "id": "organize", + "options": { + "excludeByName": {}, + "indexByName": {}, + "renameByName": { + "Value #A": "last-snap-ts" + } + } + }, + { + "id": "sortBy", + "options": { + "fields": {}, + "sort": [ + { + "field": "last-snap-ts" + } + ] + } + } + ], + "type": "table" + } + ], + "title": "Changefeed", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 2 + }, + "id": 206, + "panels": [ { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The lag between changefeed checkpoint ts and the latest ts of upstream TiDB.", + "description": "The number of events that puller outputs to sorter \n per second", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 4, "w": 12, "x": 0, - "y": 19 + "y": 4 }, "hiddenSeries": false, - "id": 3, + "id": 218, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": true, - "max": false, + "max": true, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "total": false, "values": true @@ -2685,32 +3416,23 @@ "points": false, "renderer": "flot", "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "max(ticdc_owner_checkpoint_ts_lag{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}", - "refId": "A" - }, + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ { - "expr": "sum(ticdc_processor_resolved_ts_lag{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture,changefeed)", + "expr": "sum (rate(ticdc_puller_txn_collect_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (changefeed, capture, type)", "format": "time_series", - "interval": "", "intervalFactor": 1, - "legendFormat": "{{changefeed}}-{{capture}}", - "refId": "B" + "legendFormat": "{{changefeed}}-{{capture}}-{{type}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Changefeed checkpoint lag", + "title": "Puller output events/s", "tooltip": { "shared": true, "sort": 0, @@ -2726,11 +3448,11 @@ }, "yaxes": [ { - "format": "s", + "format": "none", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -2739,7 +3461,7 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -2749,30 +3471,28 @@ }, { "aliasColors": {}, - "bars": true, + "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The derivative of a changefeed checkpoint, ideally it should not be 0 (means the changefeed is stopped).", + "description": "The total number of events that puller outputs", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 4, "w": 12, "x": 12, - "y": 19 + "y": 4 }, "hiddenSeries": false, - "id": 2, + "id": 229, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": false, - "max": false, + "max": true, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "total": false, "values": true @@ -2786,7 +3506,7 @@ }, "paceLength": 10, "percentage": false, - "pointradius": 0.5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -2795,11 +3515,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(deriv(ticdc_owner_checkpoint_ts{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (changefeed) / 1000 > 0", + "expr": "sum(ticdc_puller_txn_collect_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture, type)", "format": "time_series", - "interval": "", "intervalFactor": 1, - "legendFormat": "{{changefeed}}", + "legendFormat": "{{changefeed}}-{{capture}}-{{type}}", "refId": "A" } ], @@ -2807,7 +3526,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Changefeed checkpoint derivative", + "title": "Puller output events", "tooltip": { "shared": true, "sort": 0, @@ -2823,11 +3542,11 @@ }, "yaxes": [ { - "format": "s", + "format": "none", "label": null, - "logBase": 2, - "max": "3600", - "min": "0.001", + "logBase": 1, + "max": null, + "min": null, "show": true }, { @@ -2850,21 +3569,24 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The status of each changefeed.\n\n0: Normal\n\n1: Error\n\n2: Failed\n\n3: Stopped\n\n4: Finished\n\n-1: Unknown", + "description": "The number of events that sorter outputs to puller \n per second", "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 7, + "h": 4, "w": 12, "x": 0, - "y": 26 + "y": 8 }, - "id": 163, + "hiddenSeries": false, + "id": 228, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": true, "max": true, - "min": true, + "min": false, + "rightSide": false, "show": true, "total": false, "values": true @@ -2873,9 +3595,13 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, "percentage": false, - "pointradius": 1, - "points": true, + "pointradius": 2, + "points": false, "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, @@ -2883,11 +3609,10 @@ "steppedLine": false, "targets": [ { - "expr": "ticdc_owner_status{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}", + "expr": "sum(rate(ticdc_sorter_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (changefeed, capture, type)", "format": "time_series", - "instant": false, "intervalFactor": 1, - "legendFormat": "{{changefeed}}", + "legendFormat": "{{changefeed}}-{{capture}}-{{type}}", "refId": "A" } ], @@ -2895,7 +3620,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "The status of changefeeds", + "title": "Sorter output events/s", "tooltip": { "shared": true, "sort": 0, @@ -2911,7 +3636,7 @@ }, "yaxes": [ { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -2924,7 +3649,7 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -2934,46 +3659,38 @@ }, { "aliasColors": {}, - "bars": true, + "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Estimate the remaining time for a changefeed catch-up upstream TiDB.", - "fieldConfig": { - "defaults": { - "unit": "s" - }, - "overrides": [] - }, + "description": "The total number of events that sorter outputs", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 4, "w": 12, "x": 12, - "y": 26 + "y": 8 }, "hiddenSeries": false, - "id": 253, + "id": 220, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": true, - "hideEmpty": true, - "hideZero": true, - "max": false, + "max": true, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "total": false, "values": true }, - "lines": false, + "lines": true, "linewidth": 1, "links": [], "nullPointMode": "null", "options": { - "alertThreshold": true + "dataLinks": [] }, "paceLength": 10, "percentage": false, @@ -2986,12 +3703,10 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "abs(max(ticdc_owner_checkpoint_ts_lag{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"} / (deriv(ticdc_owner_checkpoint_ts{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])/1000)) by (changefeed))", + "expr": "sum(ticdc_sorter_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture, changefeed, type)", "format": "time_series", - "interval": "", "intervalFactor": 1, - "legendFormat": "{{changefeed}}", + "legendFormat": "{{changefeed}}-{{capture}}-{{type}}", "refId": "A" } ], @@ -2999,7 +3714,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Changefeed checkpoint catch-up ETA", + "title": "Sorter output events", "tooltip": { "shared": true, "sort": 0, @@ -3015,11 +3730,11 @@ }, "yaxes": [ { - "format": "s", - "label": "", - "logBase": 2, - "max": "604800", - "min": "1", + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, "show": true }, { @@ -3036,109 +3751,30 @@ "alignLevel": null } }, - { - "cards": { - "cardPadding": 0, - "cardRound": 0 - }, - "color": { - "cardColor": "#FF9830", - "colorScale": "linear", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "max": null, - "min": 1, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "Sink write duration of changefeeds", - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 33 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 94, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "links": [], - "repeat": null, - "repeatDirection": "h", - "reverseYBuckets": false, - "targets": [ - { - "expr": "max(rate(ticdc_sink_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "title": "Sink write duration", - "tooltip": { - "show": true, - "showHistogram": true - }, - "tooltipDecimals": 1, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 1, - "format": "s", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null - }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of sink write duration of changefeeds", + "description": "The number of events that mounter outputs to sink per second", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 4, "w": 12, - "x": 12, - "y": 33 + "x": 0, + "y": 12 }, "hiddenSeries": false, - "id": 35, + "id": 219, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": true, - "max": false, + "max": true, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "total": false, "values": true @@ -3161,32 +3797,18 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_sink_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "expr": "sum(rate(ticdc_mounter_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture,changefeed)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}-p95", + "legendFormat": "{{changefeed}}-{{capture}}", "refId": "A" - }, - { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p99", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p999", - "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Sink write duration percentile", + "title": "Mounter output events/s", "tooltip": { "shared": true, "sort": 0, @@ -3202,9 +3824,9 @@ }, "yaxes": [ { - "format": "s", + "format": "none", "label": null, - "logBase": 2, + "logBase": 1, "max": null, "min": null, "show": true @@ -3215,7 +3837,7 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -3229,24 +3851,24 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of changed rows that are written to downstream per second", + "description": "The total number of events that mounter outputs", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 4, "w": 12, - "x": 0, - "y": 40 + "x": 12, + "y": 12 }, "hiddenSeries": false, - "id": 34, + "id": 224, "legend": { - "alignAsTable": true, - "avg": true, + "alignAsTable": false, + "avg": false, "current": true, "max": true, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "total": false, "values": true @@ -3269,27 +3891,18 @@ "steppedLine": false, "targets": [ { - "expr": "sum (rate(ticdc_sink_txn_batch_size_sum{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", + "expr": "sum(ticdc_mounter_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}", + "legendFormat": "{{changefeed}}-{{capture}}", "refId": "A" - }, - { - "expr": "sum (rate(ticdc_sink_txn_batch_size_sum{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (changefeed)", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": "total", - "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Sink write rows count/s", + "title": "Mounter output events", "tooltip": { "shared": true, "sort": 0, @@ -3332,24 +3945,24 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of sink batch size", + "description": "The number of events that table sorter outputs to buffer sink per second", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 4, "w": 12, - "x": 12, - "y": 40 + "x": 0, + "y": 16 }, "hiddenSeries": false, - "id": 36, + "id": 223, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": true, - "max": false, + "max": true, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "total": false, "values": true @@ -3372,33 +3985,18 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.90, sum(rate(ticdc_sink_txn_batch_size_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "expr": "sum(rate(ticdc_sink_table_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (changefeed, capture)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}-p90", + "legendFormat": "{{changefeed}}-{{capture}}", "refId": "A" - }, - { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_txn_batch_size_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-p99", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_txn_batch_size_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "{{capture}}-p999", - "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Sink write batch size percentile", + "title": "Table sink output events/s", "tooltip": { "shared": true, "sort": 0, @@ -3416,7 +4014,7 @@ { "format": "none", "label": null, - "logBase": 2, + "logBase": 1, "max": null, "min": null, "show": true @@ -3427,7 +4025,7 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -3435,107 +4033,30 @@ "alignLevel": null } }, - { - "cards": { - "cardPadding": 0, - "cardRound": 0 - }, - "color": { - "cardColor": "#FF9830", - "colorScale": "linear", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "max": null, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "Asynchronous flush sink duration of changefeeds", - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 47 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 93, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "links": [], - "reverseYBuckets": false, - "targets": [ - { - "expr": "max(rate(ticdc_sink_flush_event_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "title": "Flush sink duration", - "tooltip": { - "show": true, - "showHistogram": true - }, - "tooltipDecimals": 1, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 1, - "format": "s", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null - }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of asynchronous flush sink duration of changefeeds", + "description": "The total number of events that table sinks emit", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 4, "w": 12, "x": 12, - "y": 47 + "y": 16 }, "hiddenSeries": false, - "id": 98, + "id": 221, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": true, - "max": false, + "max": true, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "total": false, "values": true @@ -3558,32 +4079,18 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_sink_flush_event_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance,type))", + "expr": "sum(ticdc_sink_table_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}-p95", + "legendFormat": "{{changefeed}}-{{capture}}", "refId": "A" - }, - { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_sink_flush_event_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance,type))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}-p99", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_sink_flush_event_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance,type))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}-p999", - "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Flush sink duration percentile", + "title": "Table sink output events", "tooltip": { "shared": true, "sort": 0, @@ -3599,9 +4106,9 @@ }, "yaxes": [ { - "format": "s", + "format": "none", "label": null, - "logBase": 2, + "logBase": 1, "max": null, "min": null, "show": true @@ -3613,90 +4120,12 @@ "max": null, "min": null, "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "cards": { - "cardPadding": 0, - "cardRound": 0 - }, - "color": { - "cardColor": "#FF9830", - "colorScale": "linear", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "max": null, - "min": 1, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "The duration of detecting and waiting conflict of MySQL sink", - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 54 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 103, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "links": [], - "repeatDirection": "h", - "reverseYBuckets": false, - "targets": [ - { - "expr": "max(rate(ticdc_sink_conflict_detect_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "title": "MySQL sink conflict detect duration", - "tooltip": { - "show": true, - "showHistogram": true - }, - "tooltipDecimals": 1, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 1, - "format": "s", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { "aliasColors": {}, @@ -3704,27 +4133,25 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Distribution of MySQL worker loads", + "description": "The total number of events that buffer sink outputs to backend sink per second", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 4, "w": 12, - "x": 12, - "y": 54 + "x": 0, + "y": 20 }, "hiddenSeries": false, - "id": 95, + "id": 222, "legend": { - "alignAsTable": true, - "avg": true, + "alignAsTable": false, + "avg": false, "current": true, "max": true, "min": false, "rightSide": false, "show": true, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, @@ -3735,77 +4162,32 @@ "options": { "dataLinks": [] }, + "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, - "stack": true, + "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture,bucket)", + "expr": "sum(rate(ticdc_sink_buffer_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (changefeed, capture)", "format": "time_series", - "hide": true, - "interval": "", "intervalFactor": 1, - "legendFormat": "{{capture}}-{{bucket}}", + "legendFormat": "{{changefeed}}-{{capture}}", "refId": "A" - }, - { - "expr": "count(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) >= 0)", - "format": "time_series", - "hide": true, - "interval": "", - "intervalFactor": 1, - "legendFormat": "total worker", - "refId": "B" - }, - { - "expr": "count(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 2)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "0-2 row/s worker", - "refId": "C" - }, - { - "expr": "count(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 2 and rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 10)", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": "2-10 row/s worker", - "refId": "D" - }, - { - "expr": "count(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 10 and rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 100)", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": "10-100 row/s worker", - "refId": "E" - }, - { - "expr": "count(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) > 100)", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": ">100 row/s worker", - "refId": "F" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "MySQL sink worker load", + "title": "Buffer sink output events/s", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -3818,7 +4200,7 @@ }, "yaxes": [ { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -3831,7 +4213,7 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -3839,103 +4221,25 @@ "alignLevel": null } }, - { - "cards": { - "cardPadding": 0, - "cardRound": 0 - }, - "color": { - "cardColor": "#FF9830", - "colorScale": "linear", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "max": null, - "min": 1, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "The duration of executing DDL in MySQL sink", - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 61 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 165, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "links": [], - "repeatDirection": "h", - "reverseYBuckets": false, - "targets": [ - { - "expr": "max(rate(ticdc_sink_ddl_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "title": "MySQL sink ddl execution duration", - "tooltip": { - "show": true, - "showHistogram": true - }, - "tooltipDecimals": 1, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 1, - "format": "s", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null - }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of detecting and waiting conflict duration of MySQL sink", + "description": "The total number of events that buffer sink emits", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 4, "w": 12, "x": 12, - "y": 61 + "y": 20 }, "hiddenSeries": false, - "id": 83, + "id": 226, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": true, "max": true, @@ -3952,6 +4256,7 @@ "options": { "dataLinks": [] }, + "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -3962,38 +4267,21 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.95,sum(rate(ticdc_sink_conflict_detect_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", + "expr": "sum(ticdc_sink_buffer_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture)", "format": "time_series", - "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}-p95", + "legendFormat": "{{changefeed}}-{{capture}}", "refId": "A" - }, - { - "expr": "histogram_quantile(0.99,sum(rate(ticdc_sink_conflict_detect_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p99", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.999,sum(rate(ticdc_sink_conflict_detect_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p999", - "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "MySQL sink conflict detect duration percentile", + "title": "Buffer sink output events", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -4006,9 +4294,9 @@ }, "yaxes": [ { - "format": "s", + "format": "none", "label": null, - "logBase": 2, + "logBase": 1, "max": null, "min": null, "show": true @@ -4033,27 +4321,37 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of rows(events) that are processed by sink per second", "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 4, "w": 12, "x": 0, - "y": 68 + "y": 24 }, - "id": 149, + "hiddenSeries": false, + "id": 227, "legend": { + "alignAsTable": false, "avg": false, - "current": false, - "max": false, + "current": true, + "max": true, "min": false, + "rightSide": false, "show": true, + "sort": "current", + "sortDesc": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "dataLinks": [] + }, "percentage": false, "pointradius": 2, "points": false, @@ -4064,10 +4362,11 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_processor_table_memory_consumption_sum{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[30s]) / rate(ticdc_processor_table_memory_consumption_count{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[30s])) by (capture)", + "expr": "sum (rate(ticdc_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (changefeed, capture)", "format": "time_series", + "interval": "", "intervalFactor": 1, - "legendFormat": "{{ capture }}", + "legendFormat": "{{changefeed}}-{{capture}}", "refId": "A" } ], @@ -4075,10 +4374,10 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Processor Memory Consumption Per Capture", + "title": "Sink output events/s", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -4091,7 +4390,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -4104,7 +4403,7 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -4118,25 +4417,27 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of executing DDL duration of MySQL sink", + "description": "The number of rows(events) that are processed by sink", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 4, "w": 12, "x": 12, - "y": 68 + "y": 24 }, "hiddenSeries": false, - "id": 166, + "id": 225, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": true, "max": true, "min": false, "rightSide": false, "show": true, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -4157,35 +4458,19 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.95,sum(rate(ticdc_sink_ddl_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", + "expr": "sum(ticdc_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture)", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}-p95", + "legendFormat": "{{changefeed}}-{{capture}}", "refId": "A" - }, - { - "expr": "histogram_quantile(0.99,sum(rate(ticdc_sink_ddl_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p99", - "refId": "B" - }, - { - "expr": "histogram_quantile(0.999,sum(rate(ticdc_sink_ddl_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,instance))", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p999", - "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "MySQL sink ddl execution duration percentile", + "title": "Sink output events", "tooltip": { "shared": true, "sort": 2, @@ -4201,9 +4486,9 @@ }, "yaxes": [ { - "format": "s", + "format": "none", "label": null, - "logBase": 2, + "logBase": 1, "max": null, "min": null, "show": true @@ -4228,27 +4513,37 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of rows that sink flushes to downstream per second", "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 4, "w": 12, - "x": 12, - "y": 75 + "x": 0, + "y": 28 }, - "id": 164, + "hiddenSeries": false, + "id": 108, "legend": { + "alignAsTable": false, "avg": false, - "current": false, - "max": false, + "current": true, + "max": true, "min": false, + "rightSide": false, "show": true, + "sort": "current", + "sortDesc": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "dataLinks": [] + }, "percentage": false, "pointradius": 2, "points": false, @@ -4259,10 +4554,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_processor_table_memory_consumption_sum{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[30s]) / rate(ticdc_processor_table_memory_consumption_count{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[30s])) by (capture, changefeed)", + "expr": "sum(rate(ticdc_sink_total_flushed_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (changefeed, capture)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{ capture }}-{{ changefeed }}", + "legendFormat": "{{changefeed}}-{{capture}}", "refId": "A" } ], @@ -4270,10 +4565,10 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Processor Memory Consumption Per Changefeed", + "title": "Sink flush rows/s", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -4286,7 +4581,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -4294,12 +4589,12 @@ "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -4307,190 +4602,23 @@ "alignLevel": null } }, - { - "columns": [], - "datasource": "${DS_TEST-CLUSTER}", - "fieldConfig": { - "defaults": { - "color": { - "mode": "thresholds" - }, - "custom": { - "align": "left", - "filterable": true - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "green", - "value": null - }, - { - "color": "red", - "value": 80 - } - ] - }, - "unit": "dateTimeAsSystem" - }, - "overrides": [ - { - "matcher": { - "id": "byName", - "options": "capture" - }, - "properties": [ - { - "id": "custom.width", - "value": 191 - } - ] - }, - { - "matcher": { - "id": "byName", - "options": "changefeed" - }, - "properties": [ - { - "id": "custom.width", - "value": 209 - } - ] - } - ] - }, - "fontSize": "100%", - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 76 - }, - "id": 200, - "options": { - "showHeader": true, - "sortBy": [ - { - "desc": true, - "displayName": "last-snap-ts" - } - ] - }, - "pageSize": null, - "pluginVersion": "6.1.6", - "scroll": true, - "showHeader": true, - "sort": { - "col": 0, - "desc": true - }, - "styles": [ - { - "alias": "Time", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "date" - }, - { - "alias": "", - "colorMode": null, - "colors": [ - "rgba(245, 54, 54, 0.9)", - "rgba(237, 129, 40, 0.89)", - "rgba(50, 172, 45, 0.97)" - ], - "decimals": 2, - "pattern": "/.*/", - "thresholds": [], - "type": "number", - "unit": "short" - } - ], - "targets": [ - { - "exemplar": true, - "expr": "ticdc_processor_schema_storage_gc_ts{changefeed=~\"$changefeed\", capture=~\"$capture\"}", - "format": "table", - "instant": true, - "interval": "", - "legendFormat": "", - "queryType": "randomWalk", - "refId": "A" - } - ], - "title": "Schema Storage GC progress", - "transform": "table", - "transformations": [ - { - "id": "filterFieldsByName", - "options": { - "include": { - "names": [ - "capture", - "changefeed", - "Value" - ] - } - } - }, - { - "id": "organize", - "options": { - "excludeByName": {}, - "indexByName": {}, - "renameByName": { - "Value #A": "last-snap-ts" - } - } - }, - { - "id": "sortBy", - "options": { - "fields": {}, - "sort": [ - { - "field": "last-snap-ts" - } - ] - } - } - ], - "type": "table" - } - ], - "title": "Changefeed", - "type": "row" - }, - { - "collapsed": true, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 2 - }, - "id": 206, - "panels": [ { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of events that puller outputs to sorter \n per second", + "description": "The number of rows(events) that are flushed by sink", "fill": 1, "fillGradient": 0, "gridPos": { "h": 4, "w": 12, - "x": 0, - "y": 3 + "x": 12, + "y": 28 }, "hiddenSeries": false, - "id": 218, + "id": 240, "legend": { "alignAsTable": false, "avg": false, @@ -4499,6 +4627,8 @@ "min": false, "rightSide": false, "show": true, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -4509,7 +4639,6 @@ "options": { "dataLinks": [] }, - "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -4520,10 +4649,11 @@ "steppedLine": false, "targets": [ { - "expr": "sum (rate(ticdc_puller_txn_collect_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (instance, type)", + "expr": "sum(ticdc_sink_total_flushed_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed, capture)", "format": "time_series", + "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{changefeed}}-{{capture}}", "refId": "A" } ], @@ -4531,10 +4661,10 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Puller output events/s", + "title": "Sink flush rows", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -4560,38 +4690,53 @@ "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { "align": false, "alignLevel": null } - }, + } + ], + "title": "Dataflow", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 3 + }, + "id": 13, + "panels": [ { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The total number of events that puller outputs", + "description": "The number of established Eventfeed RPC between TiCDC and TiKV", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 4, - "w": 12, - "x": 12, - "y": 3 + "h": 7, + "w": 8, + "x": 0, + "y": 5 }, "hiddenSeries": false, - "id": 229, + "id": 15, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, - "max": true, + "hideEmpty": false, + "max": false, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "total": false, "values": true @@ -4608,24 +4753,52 @@ "pointradius": 2, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/.*-rpc/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_puller_txn_collect_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (instance, type)", + "expr": "sum(ticdc_kvclient_event_feed_count{tidb_cluster=\"$tidb_cluster\"}) by (instance)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{instance}}", "refId": "A" + }, + { + "expr": "sum(grpc_client_started_total{tidb_cluster=\"$tidb_cluster\", grpc_method=\"EventFeed\"}) by (instance) - sum(grpc_client_handled_total{tidb_cluster=\"$tidb_cluster\", grpc_method=\"EventFeed\"}) by (instance)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-rpc", + "refId": "B" + }, + { + "expr": "sum(grpc_client_started_total{tidb_cluster=\"$tidb_cluster\", grpc_method=\"EventFeed\"}) by (instance)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "{{instance}}-rpc-started", + "refId": "C" + }, + { + "expr": "sum(grpc_client_handled_total{tidb_cluster=\"$tidb_cluster\", grpc_method=\"EventFeed\"}) by (instance)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "{{instance}}-rpc-handled", + "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Puller output events", + "title": "Eventfeed count", "tooltip": { "shared": true, "sort": 0, @@ -4641,7 +4814,7 @@ }, "yaxes": [ { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -4668,24 +4841,25 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of events that sorter outputs to puller \n per second", + "decimals": null, + "description": "Percentiles of Eventfeed message size", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 4, - "w": 12, - "x": 0, - "y": 7 + "h": 7, + "w": 8, + "x": 8, + "y": 5 }, "hiddenSeries": false, - "id": 228, + "id": 17, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, - "max": true, + "max": false, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "total": false, "values": true @@ -4708,18 +4882,25 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_sorter_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture,changefeed,type)", + "expr": "histogram_quantile(0.999, sum(rate(ticdc_kvclient_event_size_bytes_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance, type))", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}-{{type}}", + "legendFormat": "{{instance}}-{{type}}-p999", "refId": "A" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(ticdc_kvclient_event_size_bytes_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance, type))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-{{type}}-p95", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Sorter output events/s", + "title": "Event size percentile", "tooltip": { "shared": true, "sort": 0, @@ -4735,7 +4916,7 @@ }, "yaxes": [ { - "format": "none", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -4748,7 +4929,7 @@ "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -4758,28 +4939,30 @@ }, { "aliasColors": {}, - "bars": false, + "bars": true, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The total number of events that sorter outputs", + "description": "The number of errors that interrupt Eventfeed RPC", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 4, - "w": 12, - "x": 12, - "y": 7 + "h": 7, + "w": 8, + "x": 16, + "y": 5 }, "hiddenSeries": false, - "id": 220, + "id": 28, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": true, + "hideZero": true, "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "total": false, "values": true @@ -4802,18 +4985,37 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_sorter_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (capture, changefeed, type)", + "expr": "sum(increase(ticdc_kvclient_event_feed_error_count{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (type)", "format": "time_series", + "hide": false, + "interval": "1m", "intervalFactor": 1, - "legendFormat": "{{capture}}-{{type}}", + "legendFormat": "{{type}}", "refId": "A" + }, + { + "expr": "-sum(increase(pd_schedule_operators_count{tidb_cluster=\"$tidb_cluster\", event=\"create\", type=~\".*leader\"}[1m]))", + "format": "time_series", + "hide": false, + "interval": "1m", + "intervalFactor": 1, + "legendFormat": "transfer-leader", + "refId": "B" + }, + { + "expr": "-sum(increase(pd_schedule_operators_count{tidb_cluster=\"$tidb_cluster\", event=\"create\", type=~\".*(peer|region)\"}[1m]))", + "format": "time_series", + "interval": "1m", + "intervalFactor": 1, + "legendFormat": "move-region", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Sorter output events", + "title": "Eventfeed error/m", "tooltip": { "shared": true, "sort": 0, @@ -4829,7 +5031,7 @@ }, "yaxes": [ { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -4856,21 +5058,23 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of events that mounter outputs to sink per second", + "description": "The number of KV client received events from TiKV per seconds", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 4, - "w": 12, + "h": 7, + "w": 8, "x": 0, - "y": 11 + "y": 12 }, "hiddenSeries": false, - "id": 219, + "id": 29, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": true, + "hideZero": true, "max": true, "min": false, "rightSide": false, @@ -4896,10 +5100,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_mounter_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture,changefeed)", + "expr": "sum(rate(ticdc_kvclient_pull_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (instance, type)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}", + "legendFormat": "{{instance}}-{{type}}", "refId": "A" } ], @@ -4907,7 +5111,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Mounter output events/s", + "title": "KV client receive events/s", "tooltip": { "shared": true, "sort": 0, @@ -4923,7 +5127,7 @@ }, "yaxes": [ { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -4936,7 +5140,7 @@ "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -4950,19 +5154,19 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The total number of events that mounter outputs", + "description": "The number of events that puller outputs to sorter \n per second", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 4, - "w": 12, - "x": 12, - "y": 11 + "h": 7, + "w": 8, + "x": 8, + "y": 12 }, "hiddenSeries": false, - "id": 224, + "id": 5, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, "max": true, @@ -4990,10 +5194,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_mounter_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (instance)", + "expr": "sum (rate(ticdc_puller_txn_collect_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (instance, type)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}", + "legendFormat": "{{instance}}-{{type}}", "refId": "A" } ], @@ -5001,7 +5205,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Mounter output events", + "title": "Puller output events/s", "tooltip": { "shared": true, "sort": 0, @@ -5044,21 +5248,22 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of events that table sorter outputs to buffer sink per second", + "description": "The number of events that are buffered in Processor's output channel and Mounter input channel", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 4, - "w": 12, - "x": 0, - "y": 15 + "h": 7, + "w": 8, + "x": 16, + "y": 12 }, "hiddenSeries": false, - "id": 223, + "id": 107, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, "max": true, "min": false, "rightSide": false, @@ -5084,18 +5289,25 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_sink_table_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (instance)", + "expr": "sum(ticdc_mounter_input_chan_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}", + "legendFormat": "{{capture}}-mounter input chan", "refId": "A" + }, + { + "expr": "-sum(ticdc_sink_buffer_chan_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-sink buffer chan", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Table sink output events/s", + "title": "Sink/Mounter buffer size", "tooltip": { "shared": true, "sort": 0, @@ -5111,7 +5323,7 @@ }, "yaxes": [ { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -5124,7 +5336,7 @@ "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -5132,30 +5344,107 @@ "alignLevel": null } }, + { + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "Duration of sorting unsorted events", + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 19 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 99, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(rate(ticdc_puller_entry_sorter_sort_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "Entry sorter sort duration", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The total number of events that table sinks emit", + "description": "Percentiles of sorting events duration", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 4, + "h": 7, "w": 12, "x": 12, - "y": 15 + "y": 19 }, "hiddenSeries": false, - "id": 221, + "id": 53, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "total": false, "values": true @@ -5178,18 +5467,26 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_sink_table_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (instance)", + "expr": "histogram_quantile(0.999, sum(rate(ticdc_puller_entry_sorter_sort_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}", + "legendFormat": "{{capture}}-p999", "refId": "A" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(ticdc_puller_entry_sorter_sort_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "{{capture}}-p95", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Table sink output events", + "title": "Entry sorter sort duration percentile", "tooltip": { "shared": true, "sort": 0, @@ -5205,9 +5502,9 @@ }, "yaxes": [ { - "format": "none", + "format": "s", "label": null, - "logBase": 1, + "logBase": 2, "max": null, "min": null, "show": true @@ -5227,98 +5524,80 @@ } }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", "datasource": "${DS_TEST-CLUSTER}", - "description": "The total number of events that buffer sink outputs to backend sink per second", - "fill": 1, - "fillGradient": 0, + "description": "Duration of merging sorted events", "gridPos": { - "h": 4, + "h": 7, "w": 12, "x": 0, - "y": 19 + "y": 26 }, - "hiddenSeries": false, - "id": 222, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 105, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, - "lines": true, - "linewidth": 1, "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, + "reverseYBuckets": false, "targets": [ { - "expr": "sum(rate(ticdc_sink_buffer_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (instance)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}", + "expr": "sum(rate(ticdc_puller_entry_sorter_merge_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", "refId": "A" } ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Buffer sink output events/s", + "title": "Entry sorter merge duration", "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" + "show": true, + "showHistogram": true }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, "show": true, - "values": [] + "splitFactor": null }, - "yaxes": [ - { - "format": "none", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": false - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null }, { "aliasColors": {}, @@ -5326,24 +5605,25 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The total number of events that buffer sink emits", + "description": "Percentiles of merging sorted events duration", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 4, + "h": 7, "w": 12, "x": 12, - "y": 19 + "y": 26 }, "hiddenSeries": false, - "id": 226, + "id": 106, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "total": false, "values": true @@ -5366,18 +5646,26 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_sink_buffer_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (instance)", + "expr": "histogram_quantile(0.999, sum(rate(ticdc_puller_entry_sorter_merge_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}", + "legendFormat": "{{capture}}-p999", "refId": "A" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(ticdc_puller_entry_sorter_merge_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "{{capture}}-p95", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Buffer sink output events", + "title": "Entry sorter merge duration percentile", "tooltip": { "shared": true, "sort": 0, @@ -5393,9 +5681,9 @@ }, "yaxes": [ { - "format": "none", + "format": "s", "label": null, - "logBase": 1, + "logBase": 2, "max": null, "min": null, "show": true @@ -5414,30 +5702,106 @@ "alignLevel": null } }, + { + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "Duration of unmarshal events from kv to SQL row", + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 33 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 101, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "max(rate(ticdc_mounter_unmarshal_and_mount_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "Mounter unmarshal duration", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of rows(events) that are processed by sink per second", - "fill": 1, + "description": "Percentiles of unmarshal events from kv to SQL row duration", + "fill": 0, "fillGradient": 0, "gridPos": { - "h": 4, + "h": 7, "w": 12, - "x": 0, - "y": 23 + "x": 12, + "y": 33 }, "hiddenSeries": false, - "id": 227, + "id": 55, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "sort": "current", "sortDesc": true, @@ -5451,6 +5815,7 @@ "options": { "dataLinks": [] }, + "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -5461,19 +5826,28 @@ "steppedLine": false, "targets": [ { - "expr": "sum (rate(ticdc_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (capture)", + "expr": "histogram_quantile(0.99, sum(rate(ticdc_mounter_unmarshal_and_mount_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le, capture))", "format": "time_series", - "interval": "", + "hide": false, "intervalFactor": 1, - "legendFormat": "{{capture}}", + "legendFormat": "{{capture}}-p99", "refId": "A" + }, + { + "expr": "histogram_quantile(0.999, sum(rate(ticdc_mounter_unmarshal_and_mount_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le, capture))", + "format": "time_series", + "hide": true, + "instant": false, + "intervalFactor": 1, + "legendFormat": "{{capture}}-p999", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Sink output events/s", + "title": "Mounter unmarshal duration percentile", "tooltip": { "shared": true, "sort": 2, @@ -5489,7 +5863,7 @@ }, "yaxes": [ { - "format": "none", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -5497,12 +5871,12 @@ "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -5516,27 +5890,27 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of rows(events) that are processed by sink", + "description": "The number of KV client dispatched event per second", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 4, + "h": 7, "w": 12, - "x": 12, - "y": 23 + "x": 0, + "y": 40 }, "hiddenSeries": false, - "id": 225, + "id": 31, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": true, + "hideZero": true, "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, @@ -5547,32 +5921,44 @@ "options": { "dataLinks": [] }, + "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/.*batch-resolved/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (capture, type)", + "expr": "sum(rate(ticdc_kvclient_send_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (capture, changefeed, type)", "format": "time_series", - "interval": "", "intervalFactor": 1, - "legendFormat": "{{capture}}", + "legendFormat": "{{capture}}-{{changefeed}}-{{type}}", "refId": "A" + }, + { + "expr": "sum(rate(ticdc_kvclient_batch_resolved_event_size_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture, changefeed, table)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-{{changefeed}}-batch-resolved", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Sink output events", + "title": "KV client dispatch events/s", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -5585,7 +5971,7 @@ }, "yaxes": [ { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -5607,99 +5993,80 @@ } }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of rows that sink flushes to downstream per second", - "fill": 1, - "fillGradient": 0, + "description": "The size of batch resolved ts message from TiKV", "gridPos": { - "h": 4, + "h": 7, "w": 12, - "x": 0, - "y": 27 + "x": 12, + "y": 40 }, - "hiddenSeries": false, - "id": 108, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 97, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "sort": "current", "sortDesc": true, "total": false, "values": true }, - "lines": true, - "linewidth": 1, "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, + "reverseYBuckets": false, "targets": [ { - "expr": "sum(rate(ticdc_sink_total_flushed_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}", + "expr": "sum(rate(ticdc_kvclient_batch_resolved_event_size_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", "refId": "A" } ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Sink flush rows/s", + "title": "KV client batch resolved size", "tooltip": { - "shared": true, - "sort": 2, - "value_type": "individual" + "show": true, + "showHistogram": true }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "none", + "logBase": 1, + "max": null, + "min": null, "show": true, - "values": [] + "splitFactor": null }, - "yaxes": [ - { - "format": "none", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - }, - { - "format": "none", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": false - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null }, { "aliasColors": {}, @@ -5707,27 +6074,27 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of rows(events) that are flushed by sink", + "description": "The number of regions that are being scanned", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 4, + "h": 7, "w": 12, - "x": 12, - "y": 27 + "x": 0, + "y": 47 }, "hiddenSeries": false, - "id": 240, + "id": 177, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": true, + "hideZero": true, "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, @@ -5738,6 +6105,7 @@ "options": { "dataLinks": [] }, + "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -5748,11 +6116,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_sink_total_flushed_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (capture, type)", + "expr": "sum(ticdc_kvclient_region_token{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture, changefeed, store)", "format": "time_series", - "interval": "", "intervalFactor": 1, - "legendFormat": "{{capture}}", + "legendFormat": "{{changefeed}}-{{capture}}-{{store}}", "refId": "A" } ], @@ -5760,10 +6127,10 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Sink flush rows", + "title": "KV client scanning regions", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -5776,7 +6143,7 @@ }, "yaxes": [ { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -5796,44 +6163,31 @@ "align": false, "alignLevel": null } - } - ], - "title": "Dataflow", - "type": "row" - }, - { - "collapsed": true, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 3 - }, - "id": 13, - "panels": [ + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of established Eventfeed RPC between TiCDC and TiKV", + "description": "Active stream count of each gRPC connection", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, - "w": 8, - "x": 0, - "y": 32 + "w": 12, + "x": 12, + "y": 47 }, "hiddenSeries": false, - "id": 15, + "id": 188, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "max": false, + "hideEmpty": true, + "hideZero": true, + "max": true, "min": false, "rightSide": true, "show": true, @@ -5852,52 +6206,24 @@ "pointradius": 2, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/.*-rpc/", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_kvclient_event_feed_count{tidb_cluster=\"$tidb_cluster\"}) by (instance)", + "expr": "sum(ticdc_kvclient_grpc_stream_count{tidb_cluster=\"$tidb_cluster\"}) by (store)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}", + "legendFormat": "{{store}}", "refId": "A" - }, - { - "expr": "sum(grpc_client_started_total{tidb_cluster=\"$tidb_cluster\", grpc_method=\"EventFeed\"}) by (instance) - sum(grpc_client_handled_total{tidb_cluster=\"$tidb_cluster\", grpc_method=\"EventFeed\"}) by (instance)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-rpc", - "refId": "B" - }, - { - "expr": "sum(grpc_client_started_total{tidb_cluster=\"$tidb_cluster\", grpc_method=\"EventFeed\"}) by (instance)", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "{{instance}}-rpc-started", - "refId": "C" - }, - { - "expr": "sum(grpc_client_handled_total{tidb_cluster=\"$tidb_cluster\", grpc_method=\"EventFeed\"}) by (instance)", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "{{instance}}-rpc-handled", - "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Eventfeed count", + "title": "KV client gRPC stream count", "tooltip": { "shared": true, "sort": 0, @@ -5940,23 +6266,30 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": null, - "description": "Percentiles of Eventfeed message size", + "description": "The number of regions that have not connected to TiKV", + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, - "w": 8, - "x": 8, - "y": 32 + "w": 12, + "x": 0, + "y": 54 }, "hiddenSeries": false, - "id": 17, + "id": 251, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": false, + "hideEmpty": true, + "hideZero": true, + "max": true, "min": false, "rightSide": true, "show": true, @@ -5968,7 +6301,7 @@ "links": [], "nullPointMode": "null", "options": { - "dataLinks": [] + "alertThreshold": true }, "paceLength": 10, "percentage": false, @@ -5981,25 +6314,20 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_kvclient_event_size_bytes_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance, type))", + "exemplar": true, + "expr": "sum(ticdc_kvclient_cached_region{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture, changefeed, store)", "format": "time_series", + "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}-p999", + "legendFormat": "{{changefeed}}-{{capture}}-{{store}}", "refId": "A" - }, - { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_kvclient_event_size_bytes_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance, type))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}-p95", - "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Event size percentile", + "title": "KV client cached regions", "tooltip": { "shared": true, "sort": 0, @@ -6015,7 +6343,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -6038,21 +6366,27 @@ }, { "aliasColors": {}, - "bars": true, + "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of errors that interrupt Eventfeed RPC", + "description": "Estimate the remaining time for a changefeed initialization (on a specific capture)", + "fieldConfig": { + "defaults": { + "unit": "s" + }, + "overrides": [] + }, "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, - "w": 8, - "x": 16, - "y": 32 + "w": 12, + "x": 12, + "y": 54 }, "hiddenSeries": false, - "id": 28, + "id": 252, "legend": { "alignAsTable": true, "avg": false, @@ -6071,7 +6405,7 @@ "links": [], "nullPointMode": "null", "options": { - "dataLinks": [] + "alertThreshold": true }, "paceLength": 10, "percentage": false, @@ -6084,37 +6418,20 @@ "steppedLine": false, "targets": [ { - "expr": "sum(increase(ticdc_kvclient_event_feed_error_count{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (type)", + "exemplar": true, + "expr": "abs(sum(ticdc_kvclient_cached_region{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"} / deriv(ticdc_kvclient_cached_region{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture, changefeed, store))", "format": "time_series", - "hide": false, - "interval": "1m", + "interval": "", "intervalFactor": 1, - "legendFormat": "{{type}}", + "legendFormat": "{{changefeed}}-{{capture}}-{{store}}", "refId": "A" - }, - { - "expr": "-sum(increase(pd_schedule_operators_count{tidb_cluster=\"$tidb_cluster\", event=\"create\", type=~\".*leader\"}[1m]))", - "format": "time_series", - "hide": false, - "interval": "1m", - "intervalFactor": 1, - "legendFormat": "transfer-leader", - "refId": "B" - }, - { - "expr": "-sum(increase(pd_schedule_operators_count{tidb_cluster=\"$tidb_cluster\", event=\"create\", type=~\".*(peer|region)\"}[1m]))", - "format": "time_series", - "interval": "1m", - "intervalFactor": 1, - "legendFormat": "move-region", - "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Eventfeed error/m", + "title": "Estimate remaining time for initialization", "tooltip": { "shared": true, "sort": 0, @@ -6130,9 +6447,9 @@ }, "yaxes": [ { - "format": "short", - "label": null, - "logBase": 1, + "format": "s", + "label": "", + "logBase": 2, "max": null, "min": null, "show": true @@ -6150,45 +6467,48 @@ "align": false, "alignLevel": null } - }, + } + ], + "title": "Events", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 4 + }, + "id": 130, + "panels": [ { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of KV client received events from TiKV per seconds", "fill": 1, - "fillGradient": 0, "gridPos": { - "h": 7, - "w": 8, + "h": 8, + "w": 12, "x": 0, - "y": 39 + "y": 5 }, - "hiddenSeries": false, - "id": 29, + "id": 131, "legend": { - "alignAsTable": true, "avg": false, - "current": true, - "hideEmpty": true, - "hideZero": true, - "max": true, + "current": false, + "max": false, "min": false, - "rightSide": false, "show": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -6199,10 +6519,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_kvclient_pull_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (instance, type)", + "expr": "sum(rate(ticdc_sorter_consume_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture,changefeed)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{capture}}-{{changefeed}}", "refId": "A" } ], @@ -6210,7 +6530,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "KV client receive events/s", + "title": "Unified Sorter intake rate", "tooltip": { "shared": true, "sort": 0, @@ -6253,36 +6573,27 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of events that puller outputs to sorter \n per second", "fill": 1, - "fillGradient": 0, "gridPos": { - "h": 7, - "w": 8, - "x": 8, - "y": 39 + "h": 8, + "w": 12, + "x": 12, + "y": 5 }, - "hiddenSeries": false, - "id": 5, + "id": 132, "legend": { - "alignAsTable": true, "avg": false, - "current": true, - "max": true, + "current": false, + "max": false, "min": false, - "rightSide": false, "show": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -6293,10 +6604,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum (rate(ticdc_puller_txn_collect_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (instance, type)", + "expr": "sum(rate(ticdc_sorter_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture,changefeed)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{capture}}-{{changefeed}}", "refId": "A" } ], @@ -6304,7 +6615,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Puller output events/s", + "title": "Unified Sorter event output rate", "tooltip": { "shared": true, "sort": 0, @@ -6320,7 +6631,7 @@ }, "yaxes": [ { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -6347,37 +6658,27 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of events that are buffered in Processor's output channel and Mounter input channel", "fill": 1, - "fillGradient": 0, "gridPos": { - "h": 7, - "w": 8, - "x": 16, - "y": 39 + "h": 8, + "w": 12, + "x": 0, + "y": 13 }, - "hiddenSeries": false, - "id": 107, + "id": 133, "legend": { - "alignAsTable": true, "avg": false, - "current": true, - "hideEmpty": false, - "max": true, + "current": false, + "max": false, "min": false, - "rightSide": false, "show": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -6388,25 +6689,18 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_mounter_input_chan_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "expr": "sum(ticdc_sorter_on_disk_data_size_gauge{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}) by (capture)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}-mounter input chan", + "legendFormat": "{{capture}}", "refId": "A" - }, - { - "expr": "-sum(ticdc_sink_buffer_chan_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-sink buffer chan", - "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Sink/Mounter buffer size", + "title": "Unified Sorter on disk data size", "tooltip": { "shared": true, "sort": 0, @@ -6422,11 +6716,11 @@ }, "yaxes": [ { - "format": "short", + "format": "bytes", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -6443,119 +6737,33 @@ "alignLevel": null } }, - { - "cards": { - "cardPadding": 0, - "cardRound": 0 - }, - "color": { - "cardColor": "#FF9830", - "colorScale": "linear", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "Duration of sorting unsorted events", - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 46 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 99, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "links": [], - "reverseYBuckets": false, - "targets": [ - { - "expr": "sum(rate(ticdc_puller_entry_sorter_sort_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "title": "Entry sorter sort duration", - "tooltip": { - "show": true, - "showHistogram": true - }, - "tooltipDecimals": 1, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 1, - "format": "s", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null - }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of sorting events duration", "fill": 1, - "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 12, - "y": 46 + "y": 13 }, - "hiddenSeries": false, - "id": 53, + "id": 134, "legend": { - "alignAsTable": true, "avg": false, - "current": true, - "hideEmpty": false, - "max": true, + "current": false, + "max": false, "min": false, - "rightSide": true, "show": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -6566,26 +6774,18 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_puller_entry_sorter_sort_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "expr": "sum(ticdc_sorter_in_memory_data_size_gauge{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}) by (capture)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}-p999", + "legendFormat": "{{capture}}", "refId": "A" - }, - { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_puller_entry_sorter_sort_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "{{capture}}-p95", - "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Entry sorter sort duration percentile", + "title": "Unified Sorter in-memory data size", "tooltip": { "shared": true, "sort": 0, @@ -6601,11 +6801,11 @@ }, "yaxes": [ { - "format": "s", + "format": "bytes", "label": null, - "logBase": 2, + "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -6624,61 +6824,51 @@ }, { "cards": { - "cardPadding": 0, - "cardRound": 0 + "cardPadding": null, + "cardRound": null }, "color": { - "cardColor": "#FF9830", - "colorScale": "linear", - "colorScheme": "interpolateSpectral", + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateOranges", "exponent": 0.5, - "min": 0, + "max": null, + "min": null, "mode": "spectrum" }, "dataFormat": "tsbuckets", "datasource": "${DS_TEST-CLUSTER}", - "description": "Duration of merging sorted events", "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 0, - "y": 53 + "y": 21 }, "heatmap": {}, "hideZeroBuckets": true, "highlightCards": true, - "id": 105, + "id": 135, "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true + "show": false }, "links": [], "reverseYBuckets": false, "targets": [ { - "expr": "sum(rate(ticdc_puller_entry_sorter_merge_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le)", + "expr": "sum(rate(ticdc_sorter_flush_count_histogram_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (le)", "format": "heatmap", - "instant": false, - "intervalFactor": 2, + "intervalFactor": 1, "legendFormat": "{{le}}", "refId": "A" } ], - "title": "Entry sorter merge duration", + "timeFrom": null, + "timeShift": null, + "title": "Unified Sorter flush sizes", "tooltip": { "show": true, - "showHistogram": true + "showHistogram": false }, - "tooltipDecimals": 1, "type": "heatmap", "xAxis": { "show": true @@ -6686,40 +6876,116 @@ "xBucketNumber": null, "xBucketSize": null, "yAxis": { - "decimals": 1, - "format": "s", + "decimals": null, + "format": "none", "logBase": 1, "max": null, "min": null, "show": true, "splitFactor": null }, - "yBucketBound": "upper", + "yBucketBound": "auto", "yBucketNumber": null, "yBucketSize": null }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateBlues", + "exponent": 0.5, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 21 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 136, + "legend": { + "show": false + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(rate(ticdc_sorter_merge_count_histogram_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "intervalFactor": 1, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Unified Sorter merge size", + "tooltip": { + "show": true, + "showHistogram": false + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": null, + "format": "none", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "auto", + "yBucketNumber": null, + "yBucketSize": null + } + ], + "title": "Unified Sorter", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 5 + }, + "id": 266, + "panels": [ { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of merging sorted events duration", "fill": 1, - "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 12, - "y": 53 + "x": 0, + "y": 16 }, - "hiddenSeries": false, - "id": 106, + "id": 289, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, + "hideEmpty": true, + "hideZero": true, "max": true, "min": false, "rightSide": true, @@ -6731,10 +6997,6 @@ "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -6745,26 +7007,19 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_puller_entry_sorter_merge_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-p999", - "refId": "A" - }, - { - "expr": "histogram_quantile(0.95, sum(rate(ticdc_puller_entry_sorter_merge_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le,capture))", + "expr": "etcd_debugging_mvcc_db_total_size_in_bytes{tidb_cluster=\"$tidb_cluster\", job=\"pd\"}", "format": "time_series", "hide": false, "intervalFactor": 1, - "legendFormat": "{{capture}}-p95", - "refId": "B" + "legendFormat": "{{instance}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Entry sorter merge duration percentile", + "title": "Etcd MVCC DB total size", "tooltip": { "shared": true, "sort": 0, @@ -6780,9 +7035,9 @@ }, "yaxes": [ { - "format": "s", + "format": "decbytes", "label": null, - "logBase": 2, + "logBase": 1, "max": null, "min": null, "show": true @@ -6793,7 +7048,7 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -6801,109 +7056,31 @@ "alignLevel": null } }, - { - "cards": { - "cardPadding": 0, - "cardRound": 0 - }, - "color": { - "cardColor": "#FF9830", - "colorScale": "linear", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "Duration of unmarshal events from kv to SQL row", - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 60 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 101, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "links": [], - "reverseYBuckets": false, - "targets": [ - { - "expr": "max(rate(ticdc_mounter_unmarshal_and_mount_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "title": "Mounter unmarshal duration", - "tooltip": { - "show": true, - "showHistogram": true - }, - "tooltipDecimals": 1, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 1, - "format": "s", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null - }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of unmarshal events from kv to SQL row duration", - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 60 + "y": 16 }, "hiddenSeries": false, - "id": 55, + "id": 114, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": true, "max": true, "min": false, "rightSide": true, "show": true, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, @@ -6925,31 +7102,35 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(ticdc_mounter_unmarshal_and_mount_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le, capture))", + "expr": "histogram_quantile(0.999, sum(rate(ticdc_server_etcd_health_check_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,instance))", "format": "time_series", - "hide": false, "intervalFactor": 1, - "legendFormat": "{{capture}}-p99", + "legendFormat": "p999-{{instance}}", "refId": "A" }, { - "expr": "histogram_quantile(0.999, sum(rate(ticdc_mounter_unmarshal_and_mount_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le, capture))", + "expr": "histogram_quantile(0.99, sum(rate(ticdc_server_etcd_health_check_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,instance))", "format": "time_series", - "hide": true, - "instant": false, "intervalFactor": 1, - "legendFormat": "{{capture}}-p999", + "legendFormat": "p99-{{instance}}", "refId": "B" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(ticdc_server_etcd_health_check_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "p95-{{instance}}", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Mounter unmarshal duration percentile", + "title": "Etcd health check duration", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -6970,7 +7151,7 @@ "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -6983,70 +7164,142 @@ "alignLevel": null } }, + { + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "max": null, + "min": 1, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 23 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 267, + "legend": { + "show": true + }, + "pluginVersion": "6.1.6", + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(ticdc_etcd_worker_tick_reactor_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "EtcdWorker exec etcd txn duration", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of KV client dispatched event per second", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 67 + "x": 12, + "y": 23 }, "hiddenSeries": false, - "id": 31, + "id": 264, "legend": { - "alignAsTable": true, "avg": false, - "current": true, - "hideEmpty": true, - "hideZero": true, - "max": true, + "current": false, + "max": false, "min": false, - "rightSide": true, "show": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null", "options": { - "dataLinks": [] + "alertThreshold": true }, - "paceLength": 10, "percentage": false, + "pluginVersion": "6.1.6", "pointradius": 2, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/.*batch-resolved/", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_kvclient_send_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (capture, changefeed, type)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-{{changefeed}}-{{type}}", + "exemplar": true, + "expr": "histogram_quantile(0.95, sum(rate(ticdc_etcd_worker_tick_reactor_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", + "interval": "", + "legendFormat": "{{capture}}-95", + "queryType": "randomWalk", "refId": "A" }, { - "expr": "sum(rate(ticdc_kvclient_batch_resolved_event_size_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture, changefeed, table)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-{{changefeed}}-batch-resolved", + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(ticdc_etcd_worker_tick_reactor_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", + "hide": false, + "interval": "", + "legendFormat": "{{capture}}-99", "refId": "B" } ], @@ -7054,7 +7307,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "KV client dispatch events/s", + "title": "EtcdWorker tick reactor duration", "tooltip": { "shared": true, "sort": 0, @@ -7070,7 +7323,7 @@ }, "yaxes": [ { - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -7078,7 +7331,7 @@ "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -7097,52 +7350,52 @@ "cardRound": 0 }, "color": { - "cardColor": "#FF9830", - "colorScale": "linear", + "cardColor": "#b4ff00", + "colorScale": "sqrt", "colorScheme": "interpolateSpectral", "exponent": 0.5, - "min": 0, + "max": null, + "min": 1, "mode": "spectrum" }, "dataFormat": "tsbuckets", "datasource": "${DS_TEST-CLUSTER}", - "description": "The size of batch resolved ts message from TiKV", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "gridPos": { "h": 7, "w": 12, - "x": 12, - "y": 67 + "x": 0, + "y": 30 }, "heatmap": {}, "hideZeroBuckets": true, "highlightCards": true, - "id": 97, + "id": 256, "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true + "show": true }, - "links": [], + "pluginVersion": "6.1.6", "reverseYBuckets": false, "targets": [ { - "expr": "sum(rate(ticdc_kvclient_batch_resolved_event_size_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le)", + "exemplar": true, + "expr": "sum(rate(ticdc_etcd_worker_etcd_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le)", "format": "heatmap", "instant": false, + "interval": "", "intervalFactor": 2, "legendFormat": "{{le}}", + "queryType": "randomWalk", "refId": "A" } ], - "title": "KV client batch resolved size", + "timeFrom": null, + "timeShift": null, + "title": "EtcdWorker exec etcd txn duration", "tooltip": { "show": true, "showHistogram": true @@ -7156,7 +7409,7 @@ "xBucketSize": null, "yAxis": { "decimals": 1, - "format": "none", + "format": "s", "logBase": 1, "max": null, "min": null, @@ -7173,39 +7426,40 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of regions that are being scanned", + "description": "", + "fieldConfig": { + "defaults": { + "unit": "s" + }, + "overrides": [] + }, "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 74 + "x": 12, + "y": 30 }, "hiddenSeries": false, - "id": 177, + "id": 258, "legend": { - "alignAsTable": true, "avg": false, - "current": true, - "hideEmpty": true, - "hideZero": true, - "max": true, + "current": false, + "max": false, "min": false, - "rightSide": true, "show": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null", "options": { - "dataLinks": [] + "alertThreshold": true }, - "paceLength": 10, "percentage": false, + "pluginVersion": "6.1.6", "pointradius": 2, "points": false, "renderer": "flot", @@ -7215,18 +7469,31 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_kvclient_region_token{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture, changefeed, store)", + "exemplar": true, + "expr": "histogram_quantile(0.95, sum(rate(ticdc_etcd_worker_etcd_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", "format": "time_series", + "interval": "", "intervalFactor": 1, - "legendFormat": "{{changefeed}}-{{capture}}-{{store}}", + "legendFormat": "{{capture}}-p95", + "queryType": "randomWalk", "refId": "A" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(ticdc_etcd_worker_etcd_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}-p99", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "KV client scanning regions", + "title": "EtcdWorker exec etcd txn duration percentile", "tooltip": { "shared": true, "sort": 0, @@ -7242,9 +7509,9 @@ }, "yaxes": [ { - "format": "short", + "format": "s", "label": null, - "logBase": 1, + "logBase": 2, "max": null, "min": null, "show": true @@ -7263,45 +7530,117 @@ "alignLevel": null } }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolatePurples", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 37 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 254, + "legend": { + "show": true + }, + "pluginVersion": "6.1.6", + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(ticdc_etcd_worker_etcd_txn_size_bytes_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{le}}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "title": "EtcdWorker txn size", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": null, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": null, + "format": "decbytes", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "auto", + "yBucketNumber": null, + "yBucketSize": null + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Active stream count of each gRPC connection", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 74 + "y": 37 }, "hiddenSeries": false, - "id": 188, + "id": 260, "legend": { - "alignAsTable": true, "avg": false, - "current": true, - "hideEmpty": true, - "hideZero": true, - "max": true, + "current": false, + "max": false, "min": false, - "rightSide": true, "show": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null", "options": { - "dataLinks": [] + "alertThreshold": true }, - "paceLength": 10, "percentage": false, + "pluginVersion": "6.1.6", "pointradius": 2, "points": false, "renderer": "flot", @@ -7311,18 +7650,27 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_kvclient_grpc_stream_count{tidb_cluster=\"$tidb_cluster\"}) by (store)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{store}}", + "exemplar": true, + "expr": "histogram_quantile(0.95, sum(rate(ticdc_etcd_worker_etcd_txn_size_bytes_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", + "interval": "", + "legendFormat": "{{capture}}-p95", + "queryType": "randomWalk", "refId": "A" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(ticdc_etcd_worker_etcd_txn_size_bytes_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le,capture))", + "hide": false, + "interval": "", + "legendFormat": "{{capture}}-p99", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "KV client gRPC stream count", + "title": "EtcdWorker txn size percentile", "tooltip": { "shared": true, "sort": 0, @@ -7338,7 +7686,7 @@ }, "yaxes": [ { - "format": "short", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -7365,23 +7713,18 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of regions that have not connected to TiKV", - "fieldConfig": { - "defaults": { - "links": [] - }, - "overrides": [] - }, + "description": "The time consumed of writing WAL into the persistent storage in .99", + "editable": true, + "error": false, "fill": 1, - "fillGradient": 0, + "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 81 + "y": 44 }, - "hiddenSeries": false, - "id": 251, + "id": 291, "legend": { "alignAsTable": true, "avg": false, @@ -7392,6 +7735,7 @@ "min": false, "rightSide": true, "show": true, + "sideWidth": 300, "total": false, "values": true }, @@ -7399,12 +7743,9 @@ "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "alertThreshold": true - }, "paceLength": 10, "percentage": false, - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -7413,24 +7754,25 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum(ticdc_kvclient_cached_region{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture, changefeed, store)", + "expr": "histogram_quantile(0.99, sum(rate(etcd_disk_wal_fsync_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[5m])) by (instance, le))", "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}-{{capture}}-{{store}}", - "refId": "A" + "intervalFactor": 2, + "legendFormat": "{{instance}}", + "metric": "", + "refId": "A", + "step": 4 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "KV client cached regions", + "title": "Etcd 99% WAL fsync duration", "tooltip": { + "msResolution": false, "shared": true, "sort": 0, - "value_type": "individual" + "value_type": "cumulative" }, "type": "graph", "xaxis": { @@ -7442,7 +7784,7 @@ }, "yaxes": [ { - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -7469,33 +7811,27 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Estimate the remaining time for a changefeed initialization (on a specific capture)", - "fieldConfig": { - "defaults": { - "unit": "s" - }, - "overrides": [] - }, + "description": "The time consumed of handling etcd transactions in .99", + "editable": true, + "error": false, "fill": 1, - "fillGradient": 0, + "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 81 + "y": 44 }, - "hiddenSeries": false, - "id": 252, + "id": 290, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": true, - "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, + "sideWidth": 300, "total": false, "values": true }, @@ -7503,12 +7839,9 @@ "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "alertThreshold": true - }, "paceLength": 10, "percentage": false, - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -7517,24 +7850,24 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "abs(sum(ticdc_kvclient_cached_region{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"} / deriv(ticdc_kvclient_cached_region{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture, changefeed, store))", + "expr": "histogram_quantile(0.99, sum(rate(pd_txn_handle_txns_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[5m])) by (instance, result, le))", "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}-{{capture}}-{{store}}", - "refId": "A" + "intervalFactor": 2, + "legendFormat": "{{instance}} {{result}}", + "refId": "A", + "step": 4 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Estimate remaining time for initialization", + "title": "Etcd 99% Handle transactions duration", "tooltip": { + "msResolution": false, "shared": true, "sort": 0, - "value_type": "individual" + "value_type": "cumulative" }, "type": "graph", "xaxis": { @@ -7547,8 +7880,8 @@ "yaxes": [ { "format": "s", - "label": "", - "logBase": 2, + "label": null, + "logBase": 1, "max": null, "min": null, "show": true @@ -7568,7 +7901,7 @@ } } ], - "title": "Events", + "title": "Etcd", "type": "row" }, { @@ -7577,9 +7910,9 @@ "h": 1, "w": 24, "x": 0, - "y": 4 + "y": 6 }, - "id": 130, + "id": 58, "panels": [ { "aliasColors": {}, @@ -7587,49 +7920,87 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "CPU usage of TiKV CDC component", "fill": 1, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 5 + "y": 7 }, - "id": 131, + "id": 60, "legend": { + "alignAsTable": true, "avg": false, - "current": false, - "max": false, + "current": true, + "max": true, "min": false, + "rightSide": true, "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, "links": [], "nullPointMode": "null", + "paceLength": 10, "percentage": false, - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/.*workers/", + "transform": "negative-Y", + "yaxis": 2 + }, + { + "alias": "/.*tso/", + "fill": 0, + "pointradius": 1, + "points": true + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_sorter_consume_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture,changefeed)", + "expr": "sum(rate(tikv_thread_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", name=~\"cdc_.*|cdc\"}[1m])) by (instance)", "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-{{changefeed}}", - "refId": "A" + "intervalFactor": 2, + "legendFormat": "{{instance}}-endpoint", + "refId": "A", + "step": 4 + }, + { + "expr": "sum(rate(tikv_thread_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", name=~\"cdcwkr.*\"}[1m])) by (instance)", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{instance}}-workers", + "refId": "B", + "step": 4 + }, + { + "expr": "sum(rate(tikv_thread_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", name=~\"tso\"}[1m])) by (instance)", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "{{instance}}-tso", + "refId": "C", + "step": 4 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Unified Sorter intake rate", + "title": "CDC CPU", "tooltip": { "shared": true, "sort": 0, @@ -7645,7 +8016,7 @@ }, "yaxes": [ { - "format": "short", + "format": "percentunit", "label": null, "logBase": 1, "max": null, @@ -7653,7 +8024,7 @@ "show": true }, { - "format": "short", + "format": "percent", "label": null, "logBase": 1, "max": null, @@ -7668,54 +8039,80 @@ }, { "aliasColors": {}, - "bars": false, + "bars": true, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "fill": 1, + "decimals": 1, + "description": "Outbound network traffic of TiKV CDC component", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 5 + "y": 7 }, - "id": 132, + "id": 74, "legend": { + "alignAsTable": true, "avg": false, - "current": false, - "max": false, + "current": true, + "hideEmpty": false, + "hideZero": false, + "max": true, "min": false, + "rightSide": true, "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, "total": false, - "values": false + "values": true }, - "lines": true, + "lines": false, "linewidth": 1, "links": [], "nullPointMode": "null", + "paceLength": 10, "percentage": false, - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/.*resolved_ts/", + "bars": false, + "fill": 1, + "lines": true, + "linewidth": 2, + "transform": "negative-Y", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_sorter_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture,changefeed)", + "expr": "sum(rate(tikv_cdc_grpc_message_sent_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[30s])) by (instance, type)", "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}-{{changefeed}}", - "refId": "A" + "hide": false, + "intervalFactor": 2, + "legendFormat": "{{instance}}-{{type}}", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Unified Sorter event output rate", + "title": "CDC network traffic", "tooltip": { + "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -7730,7 +8127,7 @@ }, "yaxes": [ { - "format": "short", + "format": "Bps", "label": null, "logBase": 1, "max": null, @@ -7738,7 +8135,7 @@ "show": true }, { - "format": "short", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -7748,7 +8145,7 @@ ], "yaxis": { "align": false, - "alignLevel": null + "alignLevel": 0 } }, { @@ -7757,29 +8154,38 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "The count of different kinds of gRPC message", "fill": 1, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 13 + "y": 14 }, - "id": 133, + "id": 147, "legend": { + "alignAsTable": true, "avg": false, - "current": false, - "max": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, "min": false, + "rightSide": true, "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, "links": [], "nullPointMode": "null", "percentage": false, - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -7788,18 +8194,20 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_sorter_on_disk_data_size_gauge{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}) by (capture)", + "expr": "sum(rate(tikv_grpc_msg_duration_seconds_count{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", type!=\"kv_gc\"}[1m])) by (type)", "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}", - "refId": "A" + "intervalFactor": 2, + "legendFormat": "{{type}}", + "metric": "tikv_grpc_msg_duration_seconds_bucket", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Unified Sorter on disk data size", + "title": "gRPC message count", "tooltip": { "shared": true, "sort": 0, @@ -7815,11 +8223,11 @@ }, "yaxes": [ { - "format": "bytes", + "format": "ops", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -7842,50 +8250,129 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "fill": 1, + "decimals": 1, + "description": "The memory usage per TiKV instance", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 13 + "y": 14 }, - "id": 134, + "id": 194, "legend": { + "alignAsTable": true, "avg": false, - "current": false, - "max": false, + "current": true, + "max": true, "min": false, + "rightSide": true, "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, "links": [], "nullPointMode": "null", + "paceLength": 10, "percentage": false, - "pointradius": 2, + "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/.*-cap-.*/", + "yaxis": 2 + }, + { + "alias": "/.*tikv.*/", + "pointradius": 1, + "points": true, + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_sorter_in_memory_data_size_gauge{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}) by (capture)", + "expr": "avg(process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", job=~\"tikv.*\"}) by (instance)", + "format": "time_series", + "hide": true, + "intervalFactor": 2, + "legendFormat": "tikv-{{instance}}", + "refId": "A", + "step": 10 + }, + { + "expr": "avg(process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", job=~\"cdc.*\"}) by (instance)", + "format": "time_series", + "hide": true, + "intervalFactor": 2, + "legendFormat": "cdc-{{instance}}", + "refId": "B", + "step": 10 + }, + { + "expr": "(avg(process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", job=~\"tikv.*\"}) by (instance)) - (avg(tikv_engine_block_cache_size_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", db=\"kv\"}) by(instance))", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "tikv-{{instance}}", + "refId": "C", + "step": 10 + }, + { + "expr": "sum(tikv_cdc_sink_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{capture}}", - "refId": "A" + "hide": false, + "intervalFactor": 2, + "legendFormat": "sink-{{instance}}", + "refId": "D", + "step": 10 + }, + { + "expr": "sum(tikv_cdc_old_value_cache_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "old-value-{{instance}}", + "refId": "E", + "step": 10 + }, + { + "expr": "sum(tikv_cdc_sink_memory_capacity{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "format": "time_series", + "hide": true, + "intervalFactor": 2, + "legendFormat": "sink-cap-{{instance}}", + "refId": "F", + "step": 10 + }, + { + "expr": "sum(tikv_cdc_old_value_cache_memory_quota{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "format": "time_series", + "hide": true, + "intervalFactor": 2, + "legendFormat": "old-value-cap-{{instance}}", + "refId": "G", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Unified Sorter in-memory data size", + "title": "CDC memory", "tooltip": { + "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -7908,7 +8395,7 @@ "show": true }, { - "format": "short", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -7921,150 +8408,6 @@ "alignLevel": null } }, - { - "cards": { - "cardPadding": null, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateOranges", - "exponent": 0.5, - "max": null, - "min": null, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 21 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 135, - "legend": { - "show": false - }, - "links": [], - "reverseYBuckets": false, - "targets": [ - { - "expr": "sum(rate(ticdc_sorter_flush_count_histogram_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "intervalFactor": 1, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "timeFrom": null, - "timeShift": null, - "title": "Unified Sorter flush sizes", - "tooltip": { - "show": true, - "showHistogram": false - }, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": null, - "format": "none", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "auto", - "yBucketNumber": null, - "yBucketSize": null - }, - { - "cards": { - "cardPadding": null, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateBlues", - "exponent": 0.5, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 21 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 136, - "legend": { - "show": false - }, - "links": [], - "reverseYBuckets": false, - "targets": [ - { - "expr": "sum(rate(ticdc_sorter_merge_count_histogram_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "intervalFactor": 1, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "timeFrom": null, - "timeShift": null, - "title": "Unified Sorter merge size", - "tooltip": { - "show": true, - "showHistogram": false - }, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": null, - "format": "none", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "auto", - "yBucketNumber": null, - "yBucketSize": null - } - ], - "title": "Unified Sorter", - "type": "row" - }, - { - "collapsed": true, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 5 - }, - "id": 58, - "panels": [ { "aliasColors": {}, "bars": false, @@ -8072,26 +8415,29 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "CPU usage of TiKV CDC component", - "fill": 1, + "description": "The min resolved ts of each TiKV", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, "gridPos": { "h": 7, - "w": 12, + "w": 7, "x": 0, - "y": 5 + "y": 21 }, - "id": 60, + "id": 152, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": true, + "max": false, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "sideWidth": null, - "sort": "current", - "sortDesc": true, + "sort": null, + "sortDesc": null, "total": false, "values": true }, @@ -8101,20 +8447,21 @@ "nullPointMode": "null", "paceLength": 10, "percentage": false, - "pointradius": 5, + "pointradius": 1, "points": false, "renderer": "flot", "seriesOverrides": [ { - "alias": "/.*workers/", - "transform": "negative-Y", + "alias": "/.*-ts/", + "lines": true, + "linewidth": 3, + "points": false, "yaxis": 2 }, { - "alias": "/.*tso/", - "fill": 0, - "pointradius": 1, - "points": true + "alias": "/.*-lag/", + "bars": true, + "fill": 1 } ], "spaceLength": 10, @@ -8122,37 +8469,41 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tikv_thread_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", name=~\"cdc_.*|cdc\"}[1m])) by (instance)", + "expr": "scalar(max(pd_cluster_tso{tidb_cluster=\"$tidb_cluster\"}))/1000 - avg(tikv_cdc_min_resolved_ts{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}/1000) by (instance) > 0", "format": "time_series", + "hide": false, + "instant": false, "intervalFactor": 2, - "legendFormat": "{{instance}}-endpoint", + "legendFormat": "{{instance}}-min-resolved-lag", "refId": "A", - "step": 4 + "step": 10 }, { - "expr": "sum(rate(tikv_thread_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", name=~\"cdcwkr.*\"}[1m])) by (instance)", + "expr": "max(pd_cluster_tso{tidb_cluster=\"$tidb_cluster\"})", "format": "time_series", + "hide": false, "intervalFactor": 2, - "legendFormat": "{{instance}}-workers", + "legendFormat": "current-ts", "refId": "B", - "step": 4 + "step": 10 }, { - "expr": "sum(rate(tikv_thread_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", name=~\"tso\"}[1m])) by (instance)", + "expr": "avg(tikv_cdc_min_resolved_ts{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", "format": "time_series", "hide": false, "intervalFactor": 2, - "legendFormat": "{{instance}}-tso", + "legendFormat": "{{instance}}-min-resolved-ts", "refId": "C", - "step": 4 + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "CDC CPU", + "title": "Min resolved ts", "tooltip": { + "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -8167,15 +8518,15 @@ }, "yaxes": [ { - "format": "percentunit", + "format": "s", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { - "format": "percent", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -8190,36 +8541,34 @@ }, { "aliasColors": {}, - "bars": true, + "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "Outbound network traffic of TiKV CDC component", + "description": "The ID of the min resolved region of each TiKV", "editable": true, "error": false, "fill": 0, "grid": {}, "gridPos": { "h": 7, - "w": 12, - "x": 12, - "y": 5 + "w": 5, + "x": 7, + "y": 21 }, - "id": 74, + "id": 153, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": false, "max": true, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "sideWidth": null, - "sort": "max", - "sortDesc": true, + "sort": null, + "sortDesc": null, "total": false, "values": true }, @@ -8229,30 +8578,20 @@ "nullPointMode": "null", "paceLength": 10, "percentage": false, - "pointradius": 5, - "points": false, + "pointradius": 1, + "points": true, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/.*resolved_ts/", - "bars": false, - "fill": 1, - "lines": true, - "linewidth": 2, - "transform": "negative-Y", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(rate(tikv_cdc_grpc_message_sent_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[30s])) by (instance, type)", + "expr": "avg(tikv_cdc_min_resolved_ts_region{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", "format": "time_series", "hide": false, "intervalFactor": 2, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{instance}}-min-resolved-region", "refId": "A", "step": 10 } @@ -8261,7 +8600,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "CDC network traffic", + "title": "Min resolved Region", "tooltip": { "msResolution": false, "shared": true, @@ -8278,25 +8617,25 @@ }, "yaxes": [ { - "format": "Bps", + "format": "none", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { - "format": "bytes", + "format": "none", "label": null, "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { "align": false, - "alignLevel": 0 + "alignLevel": null } }, { @@ -8306,28 +8645,28 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The count of different kinds of gRPC message", + "description": "", "fill": 1, "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 12 + "x": 12, + "y": 21 }, - "id": 147, + "id": 70, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": true, + "hideEmpty": false, "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, "sideWidth": null, - "sort": "current", - "sortDesc": true, + "sort": null, + "sortDesc": null, "total": false, "values": true }, @@ -8335,6 +8674,7 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "paceLength": 10, "percentage": false, "pointradius": 5, "points": false, @@ -8345,20 +8685,18 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tikv_grpc_msg_duration_seconds_count{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", type!=\"kv_gc\"}[1m])) by (type)", + "expr": "histogram_quantile(0.99999, sum(rate(tikv_cdc_resolved_ts_gap_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance))", "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{type}}", - "metric": "tikv_grpc_msg_duration_seconds_bucket", - "refId": "A", - "step": 10 + "intervalFactor": 1, + "legendFormat": "{{instance}}-p9999", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "gRPC message count", + "title": "Resolved ts lag duration percentile", "tooltip": { "shared": true, "sort": 0, @@ -8374,9 +8712,9 @@ }, "yaxes": [ { - "format": "ops", + "format": "s", "label": null, - "logBase": 1, + "logBase": 10, "max": null, "min": null, "show": true @@ -8395,6 +8733,85 @@ "alignLevel": null } }, + { + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "The time consumed to CDC incremental scan", + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 28 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 68, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "maxPerRow": 3, + "repeat": null, + "repeatDirection": "h", + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(rate(tikv_cdc_scan_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "Initial scan duration", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, { "aliasColors": {}, "bars": false, @@ -8402,28 +8819,27 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The memory usage per TiKV instance", - "editable": true, - "error": false, - "fill": 0, - "grid": {}, + "description": "", + "fill": 1, "gridPos": { "h": 7, - "w": 12, + "w": 6, "x": 12, - "y": 12 + "y": 28 }, - "id": 194, + "id": 72, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, + "hideZero": true, "max": true, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "sideWidth": null, - "sort": "max", + "sort": "current", "sortDesc": true, "total": false, "values": true @@ -8434,96 +8850,28 @@ "nullPointMode": "null", "paceLength": 10, "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/.*-cap-.*/", - "yaxis": 2 - }, - { - "alias": "/.*tikv.*/", - "pointradius": 1, - "points": true, - "yaxis": 2 - } - ], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "avg(process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", job=~\"tikv.*\"}) by (instance)", - "format": "time_series", - "hide": true, - "intervalFactor": 2, - "legendFormat": "tikv-{{instance}}", - "refId": "A", - "step": 10 - }, - { - "expr": "avg(process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", job=~\"cdc.*\"}) by (instance)", - "format": "time_series", - "hide": true, - "intervalFactor": 2, - "legendFormat": "cdc-{{instance}}", - "refId": "B", - "step": 10 - }, - { - "expr": "(avg(process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", job=~\"tikv.*\"}) by (instance)) - (avg(tikv_engine_block_cache_size_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", db=\"kv\"}) by(instance))", - "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "tikv-{{instance}}", - "refId": "C", - "step": 10 - }, - { - "expr": "sum(tikv_cdc_sink_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", - "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "sink-{{instance}}", - "refId": "D", - "step": 10 - }, - { - "expr": "sum(tikv_cdc_old_value_cache_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", - "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "old-value-{{instance}}", - "refId": "E", - "step": 10 - }, - { - "expr": "sum(tikv_cdc_sink_memory_capacity{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", - "format": "time_series", - "hide": true, - "intervalFactor": 2, - "legendFormat": "sink-cap-{{instance}}", - "refId": "F", - "step": 10 - }, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ { - "expr": "sum(tikv_cdc_old_value_cache_memory_quota{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "expr": "histogram_quantile(0.9999, sum(rate(tikv_cdc_scan_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance))", "format": "time_series", - "hide": true, - "intervalFactor": 2, - "legendFormat": "old-value-cap-{{instance}}", - "refId": "G", - "step": 10 + "intervalFactor": 1, + "legendFormat": "{{instance}}-p9999", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "CDC memory", + "title": "Initial scan duration percentile", "tooltip": { - "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -8538,20 +8886,20 @@ }, "yaxes": [ { - "format": "bytes", + "format": "s", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "bytes", + "format": "short", "label": null, "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -8566,29 +8914,28 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The min resolved ts of each TiKV", - "editable": true, - "error": false, - "fill": 0, - "grid": {}, + "description": "The number of incremental scan task in different status.", + "fill": 1, "gridPos": { "h": 7, - "w": 7, - "x": 0, - "y": 19 + "w": 6, + "x": 18, + "y": 28 }, - "id": 152, + "id": 140, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": false, + "hideEmpty": false, + "hideZero": true, + "max": true, "min": false, "rightSide": false, "show": true, "sideWidth": null, - "sort": null, - "sortDesc": null, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -8598,21 +8945,13 @@ "nullPointMode": "null", "paceLength": 10, "percentage": false, - "pointradius": 1, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [ { - "alias": "/.*-ts/", - "lines": true, - "linewidth": 3, - "points": false, + "alias": "/.*ongoing/", "yaxis": 2 - }, - { - "alias": "/.*-lag/", - "bars": true, - "fill": 1 } ], "spaceLength": 10, @@ -8620,41 +8959,28 @@ "steppedLine": false, "targets": [ { - "expr": "scalar(max(pd_cluster_tso{tidb_cluster=\"$tidb_cluster\"}))/1000 - avg(tikv_cdc_min_resolved_ts{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}/1000) by (instance) > 0", - "format": "time_series", - "hide": false, - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{instance}}-min-resolved-lag", - "refId": "A", - "step": 10 - }, - { - "expr": "max(pd_cluster_tso{tidb_cluster=\"$tidb_cluster\"})", + "expr": "sum(tikv_cdc_scan_tasks{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", type=\"ongoing\"}) by (type, instance)", "format": "time_series", "hide": false, - "intervalFactor": 2, - "legendFormat": "current-ts", - "refId": "B", - "step": 10 + "intervalFactor": 1, + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" }, { - "expr": "avg(tikv_cdc_min_resolved_ts{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "expr": "sum(tikv_cdc_scan_tasks{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", type=\"total\"}) by (instance) - sum(tikv_cdc_scan_tasks{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", type=~\"abort|finish\"}) by (instance)", "format": "time_series", "hide": false, - "intervalFactor": 2, - "legendFormat": "{{instance}}-min-resolved-ts", - "refId": "C", - "step": 10 + "intervalFactor": 1, + "legendFormat": "{{instance}}-pending", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Min resolved ts", + "title": "Initial scan tasks status", "tooltip": { - "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -8669,11 +8995,11 @@ }, "yaxes": [ { - "format": "s", + "format": "none", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -8697,25 +9023,25 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The ID of the min resolved region of each TiKV", + "description": "The memory usage per TiKV instance", "editable": true, "error": false, "fill": 0, "grid": {}, "gridPos": { "h": 7, - "w": 5, - "x": 7, - "y": 19 + "w": 12, + "x": 0, + "y": 35 }, - "id": 153, + "id": 78, "legend": { "alignAsTable": true, "avg": false, "current": true, "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "sideWidth": null, "sort": null, @@ -8723,14 +9049,14 @@ "total": false, "values": true }, - "lines": false, + "lines": true, "linewidth": 1, "links": [], "nullPointMode": "null", "paceLength": 10, "percentage": false, - "pointradius": 1, - "points": true, + "pointradius": 5, + "points": false, "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, @@ -8738,20 +9064,29 @@ "steppedLine": false, "targets": [ { - "expr": "avg(tikv_cdc_min_resolved_ts_region{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "expr": "avg(tikv_cdc_captured_region_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", "format": "time_series", "hide": false, "intervalFactor": 2, - "legendFormat": "{{instance}}-min-resolved-region", + "legendFormat": "tikv-{{instance}}-total", "refId": "A", "step": 10 + }, + { + "expr": "sum(tikv_cdc_region_resolve_status{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance, status)", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "tikv-{{instance}}-{{status}}", + "refId": "B", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Min resolved Region", + "title": "Captured region count", "tooltip": { "msResolution": false, "shared": true, @@ -8776,12 +9111,12 @@ "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -8796,28 +9131,31 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "", - "fill": 1, + "description": "The speed of TiKV CDC incremental scan", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, "gridPos": { "h": 7, - "w": 12, + "w": 6, "x": 12, - "y": 19 + "y": 35 }, - "id": 70, + "id": 76, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, + "hideEmpty": true, "hideZero": true, "max": true, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "sideWidth": null, - "sort": null, - "sortDesc": null, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -8836,19 +9174,22 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99999, sum(rate(tikv_cdc_resolved_ts_gap_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance))", + "expr": "sum(rate(tikv_cdc_scan_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", job=\"tikv\"}[30s])) by (instance)", "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p9999", - "refId": "A" + "hide": false, + "intervalFactor": 2, + "legendFormat": "tikv-{{instance}}", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Resolved ts lag duration percentile", + "title": "CDC scan speed", "tooltip": { + "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -8863,11 +9204,11 @@ }, "yaxes": [ { - "format": "s", + "format": "bytes", "label": null, - "logBase": 10, + "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -8879,111 +9220,35 @@ "show": true } ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "cards": { - "cardPadding": 0, - "cardRound": 0 - }, - "color": { - "cardColor": "#FF9830", - "colorScale": "linear", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "The time consumed to CDC incremental scan", - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 26 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 68, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "links": [], - "maxPerRow": 3, - "repeat": null, - "repeatDirection": "h", - "reverseYBuckets": false, - "targets": [ - { - "expr": "sum(rate(tikv_cdc_scan_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "title": "Initial scan duration", - "tooltip": { - "show": true, - "showHistogram": true - }, - "tooltipDecimals": 1, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 1, - "format": "s", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null + "yaxis": { + "align": false, + "alignLevel": null + } }, { "aliasColors": {}, - "bars": false, + "bars": true, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "", - "fill": 1, + "description": "The total bytes of TiKV CDC incremental scan", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, "gridPos": { "h": 7, "w": 6, - "x": 12, - "y": 26 + "x": 18, + "y": 35 }, - "id": 72, + "id": 139, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, + "hideEmpty": true, "hideZero": true, "max": true, "min": false, @@ -8995,7 +9260,7 @@ "total": false, "values": true }, - "lines": true, + "lines": false, "linewidth": 1, "links": [], "nullPointMode": "null", @@ -9010,19 +9275,22 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.9999, sum(rate(tikv_cdc_scan_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance))", + "expr": "sum(tikv_cdc_scan_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", job=\"tikv\"}) by (instance)", "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p9999", - "refId": "A" + "hide": false, + "intervalFactor": 2, + "legendFormat": "tikv-{{instance}}", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Initial scan duration percentile", + "title": "CDC total scan bytes", "tooltip": { + "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -9037,7 +9305,7 @@ }, "yaxes": [ { - "format": "s", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -9050,7 +9318,7 @@ "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -9065,24 +9333,24 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The number of incremental scan task in different status.", - "fill": 1, + "description": "", + "fill": 4, "gridPos": { "h": 7, - "w": 6, - "x": 18, - "y": 26 + "w": 12, + "x": 0, + "y": 42 }, - "id": 140, + "id": 143, "legend": { "alignAsTable": true, "avg": false, "current": true, "hideEmpty": false, "hideZero": true, - "max": true, - "min": false, - "rightSide": false, + "max": false, + "min": true, + "rightSide": true, "show": true, "sideWidth": null, "sort": "current", @@ -9096,12 +9364,14 @@ "nullPointMode": "null", "paceLength": 10, "percentage": false, - "pointradius": 5, - "points": false, + "pointradius": 1, + "points": true, "renderer": "flot", "seriesOverrides": [ { - "alias": "/.*ongoing/", + "alias": "/(access|miss).*/", + "fill": 0, + "points": false, "yaxis": 2 } ], @@ -9110,27 +9380,43 @@ "steppedLine": false, "targets": [ { - "expr": "sum(tikv_cdc_scan_tasks{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", type=\"ongoing\"}) by (type, instance)", + "expr": "(sum(rate(tikv_cdc_old_value_cache_access{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance) - sum(rate(tikv_cdc_old_value_cache_miss{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance)) / sum(rate(tikv_cdc_old_value_cache_access{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance)", "format": "time_series", "hide": false, "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "hit-rate-{{instance}}", "refId": "A" }, { - "expr": "sum(tikv_cdc_scan_tasks{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", type=\"total\"}) by (instance) - sum(tikv_cdc_scan_tasks{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", type=~\"abort|finish\"}) by (instance)", + "expr": "-sum(rate(tikv_cdc_old_value_cache_access{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance)", "format": "time_series", - "hide": false, + "hide": true, "intervalFactor": 1, - "legendFormat": "{{instance}}-pending", + "legendFormat": "access-{{instance}}", "refId": "B" + }, + { + "expr": "-sum(rate(tikv_cdc_old_value_cache_miss{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "miss-{{instance}}", + "refId": "C" + }, + { + "expr": "-sum(rate(tikv_cdc_old_value_cache_miss_none{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "miss-none-{{instance}}", + "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Initial scan tasks status", + "title": "Old value cache hit", "tooltip": { "shared": true, "sort": 0, @@ -9146,11 +9432,11 @@ }, "yaxes": [ { - "format": "none", + "format": "percentunit", "label": null, "logBase": 1, - "max": null, - "min": null, + "max": "1", + "min": "0", "show": true }, { @@ -9174,29 +9460,28 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The memory usage per TiKV instance", - "editable": true, - "error": false, - "fill": 0, - "grid": {}, + "description": "The total number of cache entries in the old value cache.", + "fill": 1, "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 33 + "x": 12, + "y": 42 }, - "id": 78, + "id": 145, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, + "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, "sideWidth": null, - "sort": null, - "sortDesc": null, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -9206,40 +9491,50 @@ "nullPointMode": "null", "paceLength": 10, "percentage": false, - "pointradius": 5, + "pointradius": 1, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/.*len/", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "avg(tikv_cdc_captured_region_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "expr": "sum(tikv_cdc_old_value_cache_length{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", "format": "time_series", "hide": false, - "intervalFactor": 2, - "legendFormat": "tikv-{{instance}}-total", - "refId": "A", - "step": 10 + "intervalFactor": 1, + "legendFormat": "{{instance}}-len", + "refId": "A" }, { - "expr": "sum(tikv_cdc_region_resolve_status{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance, status)", + "expr": "sum(tikv_cdc_old_value_cache_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance) / sum(tikv_cdc_old_value_cache_length{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", "format": "time_series", "hide": false, - "intervalFactor": 2, - "legendFormat": "tikv-{{instance}}-{{status}}", - "refId": "B", - "step": 10 + "intervalFactor": 1, + "legendFormat": "{{instance}}-avg entry bytes", + "refId": "B" + }, + { + "expr": "sum(tikv_cdc_old_value_cache_memory_quota{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "{{instance}}-quota", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Captured region count", + "title": "Old value cache size", "tooltip": { - "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -9254,7 +9549,7 @@ }, "yaxes": [ { - "format": "none", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -9262,11 +9557,12 @@ "show": true }, { - "format": "short", + "decimals": null, + "format": "none", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -9282,18 +9578,18 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The speed of TiKV CDC incremental scan", + "description": "", "editable": true, "error": false, "fill": 0, "grid": {}, "gridPos": { "h": 7, - "w": 6, - "x": 12, - "y": 33 + "w": 12, + "x": 0, + "y": 49 }, - "id": 76, + "id": 141, "legend": { "alignAsTable": true, "avg": false, @@ -9302,7 +9598,7 @@ "hideZero": true, "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "sideWidth": null, "sort": "current", @@ -9325,11 +9621,11 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tikv_cdc_scan_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", job=\"tikv\"}[30s])) by (instance)", + "expr": "sum(rate(tikv_cdc_old_value_scan_details{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance, cf, tag)", "format": "time_series", "hide": false, "intervalFactor": 2, - "legendFormat": "tikv-{{instance}}", + "legendFormat": "{{instance}}-{{cf}}-{{tag}}", "refId": "A", "step": 10 } @@ -9338,7 +9634,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "CDC scan speed", + "title": "Old value seek operation", "tooltip": { "msResolution": false, "shared": true, @@ -9355,7 +9651,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -9376,30 +9672,105 @@ "alignLevel": null } }, + { + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "The time consumed to get an old value (both from cache and from disk)", + "gridPos": { + "h": 7, + "w": 6, + "x": 12, + "y": 49 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 146, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "maxPerRow": 3, + "repeatDirection": "h", + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(rate(tikv_cdc_old_value_duration_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "Old value seek duration", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, { "aliasColors": {}, - "bars": true, + "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The total bytes of TiKV CDC incremental scan", - "editable": true, - "error": false, - "fill": 0, - "grid": {}, + "description": "", + "fill": 1, "gridPos": { "h": 7, "w": 6, "x": 18, - "y": 33 + "y": 49 }, - "id": 139, + "id": 142, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": true, + "hideEmpty": false, "hideZero": true, "max": true, "min": false, @@ -9411,7 +9782,7 @@ "total": false, "values": true }, - "lines": false, + "lines": true, "linewidth": 1, "links": [], "nullPointMode": "null", @@ -9426,22 +9797,33 @@ "steppedLine": false, "targets": [ { - "expr": "sum(tikv_cdc_scan_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", job=\"tikv\"}) by (instance)", + "expr": "histogram_quantile(0.99, sum(rate(tikv_cdc_old_value_duration_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance, tag))", "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "tikv-{{instance}}", - "refId": "A", - "step": 10 + "intervalFactor": 1, + "legendFormat": "{{instance}}-99%-{{tag}}", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(tikv_cdc_old_value_duration_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance, tag))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-95%-{{tag}}", + "refId": "B" + }, + { + "expr": "sum(rate(tikv_cdc_old_value_duration_sum{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance, tag) / sum(rate(tikv_cdc_old_value_duration_count{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance, tag)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-avg-{{tag}}", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "CDC total scan bytes", + "title": "Old value seek duration", "tooltip": { - "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -9456,7 +9838,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -9469,223 +9851,179 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { "align": false, "alignLevel": null } - }, + } + ], + "title": "TiKV", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 7 + }, + "id": 155, + "panels": [ { "aliasColors": {}, - "bars": false, + "bars": true, + "cacheTimeout": null, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "", - "fill": 4, + "description": "TiCDC process rss memory usage. TiCDC heap memory size in use ", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 40 + "y": 6 }, - "id": 143, + "id": 157, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, - "current": true, - "hideEmpty": false, + "current": false, + "hideEmpty": true, "hideZero": true, "max": false, - "min": true, - "rightSide": true, + "min": false, + "rightSide": false, "show": true, "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, - "values": true + "values": false }, - "lines": true, + "lines": false, "linewidth": 1, "links": [], + "maxPerRow": 3, "nullPointMode": "null", - "paceLength": 10, "percentage": false, - "pointradius": 1, - "points": true, + "pluginVersion": "6.1.6", + "pointradius": 5, + "points": false, "renderer": "flot", + "repeat": null, + "repeatDirection": "h", + "scopedVars": { + "instance": { + "selected": false, + "text": "172.16.5.33:47912", + "value": "172.16.5.33:47912" + }, + "runtime_instance": { + "selected": false, + "text": "172.16.5.37:47912", + "value": "172.16.5.37:47912" + } + }, "seriesOverrides": [ { - "alias": "/(access|miss).*/", - "fill": 0, - "points": false, - "yaxis": 2 + "alias": "alloc-from-os", + "fill": 3, + "lines": true, + "stack": false + }, + { + "alias": "gc-threshold", + "bars": false, + "color": "#C4162A", + "lines": true, + "linewidth": 2, + "stack": false + }, + { + "alias": "gc", + "bars": false, + "color": "#C4162A", + "hideTooltip": true, + "legend": false, + "pointradius": 3, + "points": true, + "stack": false } ], "spaceLength": 10, - "stack": false, + "stack": true, "steppedLine": false, "targets": [ { - "expr": "(sum(rate(tikv_cdc_old_value_cache_access{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance) - sum(rate(tikv_cdc_old_value_cache_miss{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance)) / sum(rate(tikv_cdc_old_value_cache_access{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance)", + "expr": "process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", "format": "time_series", "hide": false, "intervalFactor": 1, - "legendFormat": "hit-rate-{{instance}}", + "legendFormat": "alloc-from-os", "refId": "A" }, { - "expr": "-sum(rate(tikv_cdc_old_value_cache_access{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance)", + "expr": "go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / (1 + ticdc_server_go_gc{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / 100)", "format": "time_series", - "hide": true, + "hide": false, "intervalFactor": 1, - "legendFormat": "access-{{instance}}", - "refId": "B" + "legendFormat": "estimate-inuse", + "refId": "H" }, { - "expr": "-sum(rate(tikv_cdc_old_value_cache_miss{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance)", + "expr": "go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / (1 + ticdc_server_go_gc{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / 100)", "format": "time_series", "hide": false, "intervalFactor": 1, - "legendFormat": "miss-{{instance}}", + "legendFormat": "estimate-garbage", "refId": "C" }, { - "expr": "-sum(rate(tikv_cdc_old_value_cache_miss_none{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance)", + "expr": "go_memstats_heap_idle_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_heap_released_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_heap_inuse_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", "format": "time_series", "hide": false, - "intervalFactor": 1, - "legendFormat": "miss-none-{{instance}}", - "refId": "D" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Old value cache hit", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "percentunit", - "label": null, - "logBase": 1, - "max": "1", - "min": "0", - "show": true - }, - { - "format": "none", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The total number of cache entries in the old value cache.", - "fill": 1, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 40 - }, - "id": 145, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": false, - "hideZero": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sideWidth": null, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "paceLength": 10, - "percentage": false, - "pointradius": 1, - "points": false, - "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/.*len/", - "yaxis": 2 - } - ], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ + "intervalFactor": 1, + "legendFormat": "reserved-by-go", + "refId": "B" + }, { - "expr": "sum(tikv_cdc_old_value_cache_length{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "expr": "go_memstats_stack_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_mspan_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_mcache_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_buck_hash_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_gc_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_other_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", "format": "time_series", "hide": false, "intervalFactor": 1, - "legendFormat": "{{instance}}-len", - "refId": "A" + "legendFormat": "used-by-go", + "refId": "D" }, { - "expr": "sum(tikv_cdc_old_value_cache_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance) / sum(tikv_cdc_old_value_cache_length{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "expr": "go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", "format": "time_series", "hide": false, "intervalFactor": 1, - "legendFormat": "{{instance}}-avg entry bytes", - "refId": "B" + "legendFormat": "gc-threshold", + "refId": "E" }, { - "expr": "sum(tikv_cdc_old_value_cache_memory_quota{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "expr": "(clamp_max(idelta(go_memstats_last_gc_time_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[1m]), 1) * go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}) > 0", "format": "time_series", "hide": false, "intervalFactor": 1, - "legendFormat": "{{instance}}-quota", - "refId": "C" + "legendFormat": "gc", + "refId": "F" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Old value cache size", + "title": "Memory Usage", "tooltip": { + "msResolution": true, "shared": true, "sort": 0, "value_type": "individual" @@ -9701,20 +10039,19 @@ "yaxes": [ { "format": "bytes", - "label": null, + "label": "", "logBase": 1, "max": null, "min": "0", "show": true }, { - "decimals": null, - "format": "none", - "label": null, + "format": "short", + "label": "", "logBase": 1, "max": null, - "min": "0", - "show": true + "min": null, + "show": false } ], "yaxis": { @@ -9725,11 +10062,11 @@ { "aliasColors": {}, "bars": false, + "cacheTimeout": null, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "", + "description": "Count of live objects.", "editable": true, "error": false, "fill": 0, @@ -9737,57 +10074,69 @@ "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 47 + "x": 12, + "y": 6 }, - "id": 141, + "id": 158, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, - "current": true, + "current": false, "hideEmpty": true, "hideZero": true, - "max": true, + "max": false, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, "links": [], + "maxPerRow": 3, "nullPointMode": "null", - "paceLength": 10, "percentage": false, + "pluginVersion": "6.1.6", "pointradius": 5, "points": false, "renderer": "flot", + "repeat": null, + "repeatDirection": "h", + "scopedVars": { + "instance": { + "selected": false, + "text": "172.16.5.33:47912", + "value": "172.16.5.33:47912" + }, + "runtime_instance": { + "selected": false, + "text": "172.16.5.37:47912", + "value": "172.16.5.37:47912" + } + }, "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(rate(tikv_cdc_old_value_scan_details{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance, cf, tag)", + "expr": "go_memstats_heap_objects{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", "format": "time_series", "hide": false, - "intervalFactor": 2, - "legendFormat": "{{instance}}-{{cf}}-{{tag}}", - "refId": "A", - "step": 10 + "intervalFactor": 1, + "legendFormat": "objects", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Old value seek operation", + "title": "Estimated Live Objects", "tooltip": { - "msResolution": false, + "msResolution": true, "shared": true, "sort": 0, "value_type": "individual" @@ -9802,8 +10151,8 @@ }, "yaxes": [ { - "format": "ops", - "label": null, + "format": "short", + "label": "", "logBase": 1, "max": null, "min": "0", @@ -9811,11 +10160,11 @@ }, { "format": "short", - "label": null, + "label": "", "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -9823,149 +10172,88 @@ "alignLevel": null } }, - { - "cards": { - "cardPadding": 0, - "cardRound": 0 - }, - "color": { - "cardColor": "#FF9830", - "colorScale": "linear", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "The time consumed to get an old value (both from cache and from disk)", - "gridPos": { - "h": 7, - "w": 6, - "x": 12, - "y": 47 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 146, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "links": [], - "maxPerRow": 3, - "repeatDirection": "h", - "reverseYBuckets": false, - "targets": [ - { - "expr": "sum(rate(tikv_cdc_old_value_duration_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "title": "Old value seek duration", - "tooltip": { - "show": true, - "showHistogram": true - }, - "tooltipDecimals": 1, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 1, - "format": "s", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null - }, { "aliasColors": {}, "bars": false, + "cacheTimeout": null, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "", + "description": "TiCDC process Go garbage collection STW pause duration", + "editable": true, + "error": false, "fill": 1, + "grid": {}, "gridPos": { "h": 7, - "w": 6, - "x": 18, - "y": 47 + "w": 12, + "x": 0, + "y": 13 }, - "id": 142, + "id": 160, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": true, - "max": true, + "max": false, "min": false, "rightSide": false, "show": true, - "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", - "paceLength": 10, + "nullPointMode": "null as zero", "percentage": false, + "pluginVersion": "6.1.6", "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "scopedVars": { + "instance": { + "selected": false, + "text": "172.16.5.33:47912", + "value": "172.16.5.33:47912" + }, + "runtime_instance": { + "selected": false, + "text": "172.16.5.37:47912", + "value": "172.16.5.37:47912" + } + }, + "seriesOverrides": [ + {} + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tikv_cdc_old_value_duration_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance, tag))", + "expr": "go_gc_duration_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\", quantile=\"0\"}", "format": "time_series", + "hide": false, + "instant": false, "intervalFactor": 1, - "legendFormat": "{{instance}}-99%-{{tag}}", - "refId": "A" + "legendFormat": "min", + "refId": "A", + "step": 40 }, { - "expr": "histogram_quantile(0.95, sum(rate(tikv_cdc_old_value_duration_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance, tag))", + "expr": "go_gc_duration_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\", quantile!~\"0|1\"}", "format": "time_series", + "instant": false, "intervalFactor": 1, - "legendFormat": "{{instance}}-95%-{{tag}}", + "legendFormat": "{{quantile}}", "refId": "B" }, { - "expr": "sum(rate(tikv_cdc_old_value_duration_sum{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance, tag) / sum(rate(tikv_cdc_old_value_duration_count{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance, tag)", + "expr": "go_gc_duration_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\", quantile=\"1\"}", "format": "time_series", + "instant": false, "intervalFactor": 1, - "legendFormat": "{{instance}}-avg-{{tag}}", + "legendFormat": "max", "refId": "C" } ], @@ -9973,10 +10261,11 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Old value seek duration", + "title": "GC STW Duration (last 256 GC cycles)", "tooltip": { + "msResolution": false, "shared": true, - "sort": 0, + "sort": 1, "value_type": "individual" }, "type": "graph", @@ -9993,7 +10282,7 @@ "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -10009,66 +10298,41 @@ "align": false, "alignLevel": null } - } - ], - "title": "TiKV", - "type": "row" - }, - { - "collapsed": true, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 6 - }, - "id": 155, - "panels": [ + }, { "aliasColors": {}, - "bars": true, - "cacheTimeout": null, + "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "TiCDC process rss memory usage. TiCDC heap memory size in use ", - "editable": true, - "error": false, - "fill": 0, - "grid": {}, + "description": "The throughput of Go's memory allocator.", + "fill": 1, "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 6 + "x": 12, + "y": 13 }, - "id": 157, + "id": 161, "legend": { "alignAsTable": false, "avg": false, - "current": false, - "hideEmpty": true, - "hideZero": true, - "max": false, + "current": true, + "max": true, "min": false, "rightSide": false, "show": true, - "sideWidth": null, "total": false, - "values": false + "values": true }, - "lines": false, + "lines": true, "linewidth": 1, "links": [], - "maxPerRow": 3, "nullPointMode": "null", "percentage": false, - "pluginVersion": "6.1.6", "pointradius": 5, "points": false, "renderer": "flot", - "repeat": null, - "repeatDirection": "h", "scopedVars": { "instance": { "selected": false, @@ -10077,104 +10341,64 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.5.32:47800", - "value": "172.16.5.32:47800" + "text": "172.16.5.37:47912", + "value": "172.16.5.37:47912" } }, "seriesOverrides": [ { - "alias": "alloc-from-os", - "fill": 3, - "lines": true, - "stack": false + "alias": "sweep", + "transform": "negative-Y" }, { - "alias": "gc-threshold", - "bars": false, - "color": "#C4162A", - "lines": true, - "linewidth": 2, - "stack": false + "alias": "alloc-ops", + "yaxis": 2 }, { - "alias": "gc", - "bars": false, - "color": "#C4162A", - "hideTooltip": true, - "legend": false, - "pointradius": 3, - "points": true, - "stack": false + "alias": "swepp-ops", + "transform": "negative-Y", + "yaxis": 2 } ], "spaceLength": 10, - "stack": true, + "stack": false, "steppedLine": false, "targets": [ { - "expr": "process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", + "expr": "irate(go_memstats_alloc_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[30s])", "format": "time_series", - "hide": false, "intervalFactor": 1, - "legendFormat": "alloc-from-os", + "legendFormat": "alloc", "refId": "A" }, { - "expr": "go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / (1 + ticdc_server_go_gc{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / 100)", + "expr": "irate((go_memstats_alloc_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"})[30s:])", "format": "time_series", - "hide": false, "intervalFactor": 1, - "legendFormat": "estimate-inuse", - "refId": "H" + "legendFormat": "sweep", + "refId": "B" }, { - "expr": "go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / (1 + ticdc_server_go_gc{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / 100)", + "expr": "irate(go_memstats_mallocs_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[30s])", "format": "time_series", - "hide": false, "intervalFactor": 1, - "legendFormat": "estimate-garbage", + "legendFormat": "alloc-ops", "refId": "C" }, { - "expr": "go_memstats_heap_idle_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_heap_released_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_heap_inuse_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "reserved-by-go", - "refId": "B" - }, - { - "expr": "go_memstats_stack_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_mspan_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_mcache_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_buck_hash_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_gc_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_other_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", + "expr": "irate(go_memstats_frees_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[30s])", "format": "time_series", - "hide": false, "intervalFactor": 1, - "legendFormat": "used-by-go", + "legendFormat": "swepp-ops", "refId": "D" - }, - { - "expr": "go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "gc-threshold", - "refId": "E" - }, - { - "expr": "(clamp_max(idelta(go_memstats_last_gc_time_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[1m]), 1) * go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}) > 0", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "gc", - "refId": "F" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Memory Usage", + "title": "Allocator Throughput", "tooltip": { - "msResolution": true, "shared": true, "sort": 0, "value_type": "individual" @@ -10189,234 +10413,267 @@ }, "yaxes": [ { - "format": "bytes", - "label": "", + "format": "Bps", + "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "short", - "label": "", + "format": "ops", + "label": null, "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { - "align": false, + "align": true, "alignLevel": null } - }, + } + ], + "repeat": "runtime_instance", + "title": "Runtime $runtime_instance", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 8 + }, + "id": 187, + "panels": [ { - "aliasColors": {}, - "bars": false, - "cacheTimeout": null, - "dashLength": 10, - "dashes": false, + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", "datasource": "${DS_TEST-CLUSTER}", - "description": "Count of live objects.", - "editable": true, - "error": false, - "fill": 0, - "grid": {}, + "description": "The latency distributions of fsync called by redo writer", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "gridPos": { - "h": 7, + "h": 8, "w": 12, - "x": 12, - "y": 6 + "x": 0, + "y": 8 }, - "id": 158, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 172, "legend": { - "alignAsTable": false, - "avg": false, - "current": false, - "hideEmpty": true, - "hideZero": true, - "max": false, - "min": false, - "rightSide": false, - "show": true, - "sideWidth": null, - "total": false, - "values": false + "show": true }, - "lines": true, - "linewidth": 1, - "links": [], - "maxPerRow": 3, - "nullPointMode": "null", - "percentage": false, "pluginVersion": "6.1.6", - "pointradius": 5, - "points": false, - "renderer": "flot", - "repeat": null, - "repeatDirection": "h", - "scopedVars": { - "instance": { - "selected": false, - "text": "172.16.5.33:47912", - "value": "172.16.5.33:47912" - }, - "runtime_instance": { - "selected": false, - "text": "172.16.5.32:47800", - "value": "172.16.5.32:47800" + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "max(rate(ticdc_redo_fsync_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "queryType": "randomWalk", + "refId": "A" } + ], + "timeFrom": null, + "timeShift": null, + "title": "Redo fsync duration", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "The latency distributions of flushall called by redo writer", + "fieldConfig": { + "defaults": {}, + "overrides": [] }, - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 8 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 180, + "legend": { + "show": true + }, + "pluginVersion": "6.1.6", + "reverseYBuckets": false, "targets": [ { - "expr": "go_memstats_heap_objects{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "objects", + "exemplar": true, + "expr": "max(rate(ticdc_redo_flushall_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "queryType": "randomWalk", "refId": "A" } ], - "thresholds": [], "timeFrom": null, - "timeRegions": [], "timeShift": null, - "title": "Estimated Live Objects", + "title": "Redo flushall duration", "tooltip": { - "msResolution": true, - "shared": true, - "sort": 0, - "value_type": "individual" + "show": true, + "showHistogram": true }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, "show": true, - "values": [] + "splitFactor": null }, - "yaxes": [ - { - "format": "short", - "label": "", - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": "", - "logBase": 1, - "max": null, - "min": null, - "show": false - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null }, { "aliasColors": {}, "bars": false, - "cacheTimeout": null, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "TiCDC process Go garbage collection STW pause duration", - "editable": true, - "error": false, + "description": "The total count of rows that are processed by redo writer", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, - "grid": {}, + "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 0, - "y": 13 + "y": 16 }, - "id": 160, + "hiddenSeries": false, + "id": 170, "legend": { - "alignAsTable": false, - "avg": false, + "alignAsTable": true, + "avg": true, "current": true, - "max": false, + "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "total": false, "values": true }, "lines": true, "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, "pluginVersion": "6.1.6", - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", - "scopedVars": { - "instance": { - "selected": false, - "text": "172.16.5.33:47912", - "value": "172.16.5.33:47912" - }, - "runtime_instance": { - "selected": false, - "text": "172.16.5.32:47800", - "value": "172.16.5.32:47800" - } - }, - "seriesOverrides": [ - {} - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "go_gc_duration_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\", quantile=\"0\"}", + "exemplar": true, + "expr": "sum(rate(ticdc_redo_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", "format": "time_series", - "hide": false, - "instant": false, + "interval": "", "intervalFactor": 1, - "legendFormat": "min", - "refId": "A", - "step": 40 + "legendFormat": "{{capture}}", + "queryType": "randomWalk", + "refId": "A" }, { - "expr": "go_gc_duration_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\", quantile!~\"0|1\"}", + "exemplar": true, + "expr": "sum(rate(ticdc_redo_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (changefeed)", "format": "time_series", - "instant": false, + "hide": false, + "interval": "", "intervalFactor": 1, - "legendFormat": "{{quantile}}", + "legendFormat": "total", "refId": "B" - }, - { - "expr": "go_gc_duration_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\", quantile=\"1\"}", - "format": "time_series", - "instant": false, - "intervalFactor": 1, - "legendFormat": "max", - "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "GC STW Duration (last 256 GC cycles)", + "title": "Redo Write rows/s ", "tooltip": { - "msResolution": false, "shared": true, - "sort": 1, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -10429,11 +10686,11 @@ }, "yaxes": [ { - "format": "s", + "format": "short", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -10456,99 +10713,60 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The throughput of Go's memory allocator.", + "description": "Total number of bytes redo log written", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 12, - "y": 13 + "y": 16 }, - "id": 161, + "hiddenSeries": false, + "id": 173, "legend": { - "alignAsTable": false, "avg": false, - "current": true, - "max": true, + "current": false, + "max": false, "min": false, - "rightSide": false, "show": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, - "pointradius": 5, + "pluginVersion": "6.1.6", + "pointradius": 2, "points": false, "renderer": "flot", - "scopedVars": { - "instance": { - "selected": false, - "text": "172.16.5.33:47912", - "value": "172.16.5.33:47912" - }, - "runtime_instance": { - "selected": false, - "text": "172.16.5.32:47800", - "value": "172.16.5.32:47800" - } - }, - "seriesOverrides": [ - { - "alias": "sweep", - "transform": "negative-Y" - }, - { - "alias": "alloc-ops", - "yaxis": 2 - }, - { - "alias": "swepp-ops", - "transform": "negative-Y", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "irate(go_memstats_alloc_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[30s])", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "alloc", + "exemplar": true, + "expr": "sum(rate(ticdc_redo_write_bytes_total{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", + "interval": "", + "legendFormat": "{{capture}}", + "queryType": "randomWalk", "refId": "A" - }, - { - "expr": "irate((go_memstats_alloc_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"})[30s:])", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "sweep", - "refId": "B" - }, - { - "expr": "irate(go_memstats_mallocs_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[30s])", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "alloc-ops", - "refId": "C" - }, - { - "expr": "irate(go_memstats_frees_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[30s])", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "swepp-ops", - "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Allocator Throughput", + "title": "Redo Write bytes/s ", "tooltip": { "shared": true, "sort": 0, @@ -10564,7 +10782,7 @@ }, "yaxes": [ { - "format": "Bps", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -10572,7 +10790,7 @@ "show": true }, { - "format": "ops", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -10581,13 +10799,12 @@ } ], "yaxis": { - "align": true, + "align": false, "alignLevel": null } } ], - "repeat": "runtime_instance", - "title": "Runtime $runtime_instance", + "title": "Redo", "type": "row" } ], @@ -10794,5 +11011,5 @@ "timezone": "browser", "title": "Test-Cluster-TiCDC", "uid": "YiGL8hBZ1", - "version": 26 -} + "version": 29 +} \ No newline at end of file From 088edf514cd1bd3f800239608399151e325d821d Mon Sep 17 00:00:00 2001 From: Ling Jin <7138436+3AceShowHand@users.noreply.github.com> Date: Tue, 4 Jan 2022 14:14:35 +0800 Subject: [PATCH 36/48] This is an automated cherry-pick of #4192 Signed-off-by: ti-chi-bot --- cdc/sink/codec/avro.go | 1 + cdc/sink/codec/canal.go | 8 +++++++- cdc/sink/codec/canal_flat.go | 6 ++++++ cdc/sink/codec/canal_flat_test.go | 2 ++ cdc/sink/codec/canal_test.go | 1 + cdc/sink/codec/json_test.go | 1 + cdc/sink/codec/maxwell.go | 5 +++++ cdc/sink/codec/maxwell_test.go | 1 + 8 files changed, 24 insertions(+), 1 deletion(-) diff --git a/cdc/sink/codec/avro.go b/cdc/sink/codec/avro.go index cb6ed9e9952..01a5170dfd6 100644 --- a/cdc/sink/codec/avro.go +++ b/cdc/sink/codec/avro.go @@ -122,6 +122,7 @@ func (a *AvroEventBatchEncoder) AppendRowChangedEvent(e *model.RowChangedEvent) } mqMessage.Key = evlp + mqMessage.IncRowsCount() a.resultBuf = append(a.resultBuf, mqMessage) return EncoderNeedAsyncWrite, nil diff --git a/cdc/sink/codec/canal.go b/cdc/sink/codec/canal.go index 089103d2f8d..d2dcc5bae74 100644 --- a/cdc/sink/codec/canal.go +++ b/cdc/sink/codec/canal.go @@ -378,7 +378,8 @@ func (d *CanalEventBatchEncoder) EncodeDDLEvent(e *model.DDLEvent) (*MQMessage, // Build implements the EventBatchEncoder interface func (d *CanalEventBatchEncoder) Build() []*MQMessage { - if len(d.messages.Messages) == 0 { + rowCount := len(d.messages.Messages) + if rowCount == 0 { return nil } @@ -391,7 +392,12 @@ func (d *CanalEventBatchEncoder) Build() []*MQMessage { if err != nil { log.Panic("Error when serializing Canal packet", zap.Error(err)) } +<<<<<<< HEAD ret := NewMQMessage(ProtocolCanal, nil, value, 0, model.MqMessageTypeRow, nil, nil) +======= + ret := NewMQMessage(config.ProtocolCanal, nil, value, 0, model.MqMessageTypeRow, nil, nil) + ret.SetRowsCount(rowCount) +>>>>>>> fc70dbde8 (metrics(cdc): fix mq sink write row count metrics. (#4192)) d.messages.Reset() d.resetPacket() return []*MQMessage{ret} diff --git a/cdc/sink/codec/canal_flat.go b/cdc/sink/codec/canal_flat.go index 93579556ecd..d663d9497a1 100644 --- a/cdc/sink/codec/canal_flat.go +++ b/cdc/sink/codec/canal_flat.go @@ -237,7 +237,13 @@ func (c *CanalFlatEventBatchEncoder) Build() []*MQMessage { log.Panic("CanalFlatEventBatchEncoder", zap.Error(err)) return nil } +<<<<<<< HEAD ret[i] = NewMQMessage(ProtocolCanalJSON, nil, value, msg.tikvTs, model.MqMessageTypeRow, &msg.Schema, &msg.Table) +======= + m := NewMQMessage(config.ProtocolCanalJSON, nil, value, msg.getTikvTs(), model.MqMessageTypeRow, msg.getSchema(), msg.getTable()) + m.IncRowsCount() + ret[i] = m +>>>>>>> fc70dbde8 (metrics(cdc): fix mq sink write row count metrics. (#4192)) } c.resolvedBuf = c.resolvedBuf[0:0] return ret diff --git a/cdc/sink/codec/canal_flat_test.go b/cdc/sink/codec/canal_flat_test.go index 0d6cbb7531b..64f371fe235 100644 --- a/cdc/sink/codec/canal_flat_test.go +++ b/cdc/sink/codec/canal_flat_test.go @@ -123,6 +123,8 @@ func (s *canalFlatSuite) TestBatching(c *check.C) { c.Assert(msgs, check.HasLen, int(resolvedTs-lastResolved)) for j := range msgs { + c.Assert(msgs[j].GetRowsCount(), check.Equals, 1) + var msg canalFlatMessage err := json.Unmarshal(msgs[j].Value, &msg) c.Assert(err, check.IsNil) diff --git a/cdc/sink/codec/canal_test.go b/cdc/sink/codec/canal_test.go index bf8f0fa113d..d3755d1a2df 100644 --- a/cdc/sink/codec/canal_test.go +++ b/cdc/sink/codec/canal_test.go @@ -102,6 +102,7 @@ func (s *canalBatchSuite) TestCanalEventBatchEncoder(c *check.C) { c.Assert(res, check.HasLen, 1) c.Assert(res[0].Key, check.IsNil) c.Assert(len(res[0].Value), check.Equals, size) + c.Assert(res[0].GetRowsCount(), check.Equals, len(cs)) packet := &canal.Packet{} err := proto.Unmarshal(res[0].Value, packet) diff --git a/cdc/sink/codec/json_test.go b/cdc/sink/codec/json_test.go index 7f02c564904..d1d2646e142 100644 --- a/cdc/sink/codec/json_test.go +++ b/cdc/sink/codec/json_test.go @@ -134,6 +134,7 @@ func (s *batchSuite) testBatchCodec(c *check.C, newEncoder func() EventBatchEnco if len(cs) > 0 { res := encoder.Build() c.Assert(res, check.HasLen, 1) + c.Assert(res[0].GetRowsCount(), check.Equals, len(cs)) decoder, err := newDecoder(res[0].Key, res[0].Value) c.Assert(err, check.IsNil) checkRowDecoder(decoder, cs) diff --git a/cdc/sink/codec/maxwell.go b/cdc/sink/codec/maxwell.go index d6e54c72448..5d5467bebf2 100644 --- a/cdc/sink/codec/maxwell.go +++ b/cdc/sink/codec/maxwell.go @@ -296,7 +296,12 @@ func (d *MaxwellEventBatchEncoder) Build() []*MQMessage { return nil } +<<<<<<< HEAD ret := NewMQMessage(ProtocolMaxwell, d.keyBuf.Bytes(), d.valueBuf.Bytes(), 0, model.MqMessageTypeRow, nil, nil) +======= + ret := NewMQMessage(config.ProtocolMaxwell, d.keyBuf.Bytes(), d.valueBuf.Bytes(), 0, model.MqMessageTypeRow, nil, nil) + ret.SetRowsCount(d.batchSize) +>>>>>>> fc70dbde8 (metrics(cdc): fix mq sink write row count metrics. (#4192)) d.Reset() return []*MQMessage{ret} } diff --git a/cdc/sink/codec/maxwell_test.go b/cdc/sink/codec/maxwell_test.go index 5e6fea4a58e..1f98bedac44 100644 --- a/cdc/sink/codec/maxwell_test.go +++ b/cdc/sink/codec/maxwell_test.go @@ -54,6 +54,7 @@ func (s *maxwellbatchSuite) testmaxwellBatchCodec(c *check.C, newEncoder func() continue } c.Assert(messages, check.HasLen, 1) + c.Assert(messages[0].GetRowsCount(), check.Equals, len(cs)) c.Assert(len(messages[0].Key)+len(messages[0].Value), check.Equals, size) } From ec06b29f4179c4f7d8c51d9d38753d5a0560884f Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 10 Jan 2022 13:57:41 +0800 Subject: [PATCH 37/48] retry(dm): align with tidb latest error message (#4172) (#4254) close pingcap/tiflow#4159, close pingcap/tiflow#4246 --- dm/pkg/retry/errors.go | 1 + dm/syncer/error.go | 3 ++- dm/tests/drop_column_with_index/run.sh | 2 +- dm/tests/others_integration_1.txt | 2 ++ 4 files changed, 6 insertions(+), 2 deletions(-) diff --git a/dm/pkg/retry/errors.go b/dm/pkg/retry/errors.go index 849329262c1..af5848bbe28 100644 --- a/dm/pkg/retry/errors.go +++ b/dm/pkg/retry/errors.go @@ -28,6 +28,7 @@ var ( // UnsupportedDDLMsgs list the error messages of some unsupported DDL in TiDB. UnsupportedDDLMsgs = []string{ "can't drop column with index", + "with tidb_enable_change_multi_schema is disable", // https://github.com/pingcap/tidb/pull/29526 "unsupported add column", "unsupported modify column", "unsupported modify charset", diff --git a/dm/syncer/error.go b/dm/syncer/error.go index c1aec276d8e..f017406af14 100644 --- a/dm/syncer/error.go +++ b/dm/syncer/error.go @@ -42,7 +42,8 @@ func isDropColumnWithIndexError(err error) bool { return (mysqlErr.Number == errno.ErrUnsupportedDDLOperation || mysqlErr.Number == tmysql.ErrUnknown) && strings.Contains(mysqlErr.Message, "drop column") && (strings.Contains(mysqlErr.Message, "with index") || - strings.Contains(mysqlErr.Message, "with composite index")) + strings.Contains(mysqlErr.Message, "with composite index") || + strings.Contains(mysqlErr.Message, "with tidb_enable_change_multi_schema is disable")) } // handleSpecialDDLError handles special errors for DDL execution. diff --git a/dm/tests/drop_column_with_index/run.sh b/dm/tests/drop_column_with_index/run.sh index 8c0ed0308c0..982956d8394 100755 --- a/dm/tests/drop_column_with_index/run.sh +++ b/dm/tests/drop_column_with_index/run.sh @@ -55,7 +55,7 @@ function run() { "\"isCanceled\": true" 1 sleep 5 - check_log_not_contains "dispatch auto resume task" $WORK_DIR/worker1/log/dm-worker.log + check_log_not_contains $WORK_DIR/worker1/log/dm-worker.log "dispatch auto resume task" run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ "resume-task test" \ diff --git a/dm/tests/others_integration_1.txt b/dm/tests/others_integration_1.txt index 374b8bdeff2..170b11211b6 100644 --- a/dm/tests/others_integration_1.txt +++ b/dm/tests/others_integration_1.txt @@ -1,3 +1,5 @@ +drop_column_with_index +downstream_diff_index full_mode dm_syncer sequence_sharding_optimistic From bd21afe8eb50bfa5365fe5a6bb30eb7eb1d496ca Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 10 Jan 2022 15:13:41 +0800 Subject: [PATCH 38/48] owner(ticdc): Add bootstrap and try to fix the meta information in it (#3838) (#3865) --- cdc/capture/http_validator.go | 2 +- cdc/model/capture.go | 10 ++ cdc/model/capture_test.go | 17 +++ cdc/model/changefeed.go | 59 ++++++++- cdc/model/changefeed_test.go | 148 ++++++++++++++++++++++- cdc/owner/owner.go | 34 +++++- cdc/owner/owner_test.go | 38 ++++++ pkg/cmd/cli/cli_changefeed_create.go | 2 +- pkg/cmd/util/helper.go | 3 +- pkg/orchestrator/reactor_state.go | 2 +- pkg/version/check.go | 11 +- pkg/version/check_test.go | 43 ++++--- pkg/version/creator_version_gate.go | 61 ++++++++++ pkg/version/creator_version_gate_test.go | 78 ++++++++++++ 14 files changed, 468 insertions(+), 40 deletions(-) create mode 100644 pkg/version/creator_version_gate.go create mode 100644 pkg/version/creator_version_gate_test.go diff --git a/cdc/capture/http_validator.go b/cdc/capture/http_validator.go index 029cfd68368..fefb3807f4b 100644 --- a/cdc/capture/http_validator.go +++ b/cdc/capture/http_validator.go @@ -99,7 +99,7 @@ func verifyCreateChangefeedConfig(ctx context.Context, changefeedConfig model.Ch return nil, err } // set sortEngine and EnableOldValue - cdcClusterVer, err := version.GetTiCDCClusterVersion(captureInfos) + cdcClusterVer, err := version.GetTiCDCClusterVersion(model.ListVersionsFromCaptureInfos(captureInfos)) if err != nil { return nil, err } diff --git a/cdc/model/capture.go b/cdc/model/capture.go index aa2aa76331b..22c95e2a48e 100644 --- a/cdc/model/capture.go +++ b/cdc/model/capture.go @@ -43,3 +43,13 @@ func (c *CaptureInfo) Unmarshal(data []byte) error { return errors.Annotatef(cerror.WrapError(cerror.ErrUnmarshalFailed, err), "unmarshal data: %v", data) } + +// ListVersionsFromCaptureInfos returns the version list of the CaptureInfo list. +func ListVersionsFromCaptureInfos(captureInfos []*CaptureInfo) []string { + var captureVersions []string + for _, ci := range captureInfos { + captureVersions = append(captureVersions, ci.Version) + } + + return captureVersions +} diff --git a/cdc/model/capture_test.go b/cdc/model/capture_test.go index b1ff3e97a63..67c3f7f9366 100644 --- a/cdc/model/capture_test.go +++ b/cdc/model/capture_test.go @@ -36,3 +36,20 @@ func TestMarshalUnmarshal(t *testing.T) { require.Nil(t, err) require.Equal(t, info, decodedInfo) } + +func TestListVersionsFromCaptureInfos(t *testing.T) { + infos := []*CaptureInfo{ + { + ID: "9ff52aca-aea6-4022-8ec4-fbee3f2c7891", + AdvertiseAddr: "127.0.0.1:8300", + Version: "dev", + }, + { + ID: "9ff52aca-aea6-4022-8ec4-fbee3f2c7891", + AdvertiseAddr: "127.0.0.1:8300", + Version: "", + }, + } + + require.ElementsMatch(t, []string{"dev", ""}, ListVersionsFromCaptureInfos(infos)) +} diff --git a/cdc/model/changefeed.go b/cdc/model/changefeed.go index d1806071bc9..5584ffe8a47 100644 --- a/cdc/model/changefeed.go +++ b/cdc/model/changefeed.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/cyclic/mark" cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/version" "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" ) @@ -48,7 +49,7 @@ const ( StateError FeedState = "error" StateFailed FeedState = "failed" StateStopped FeedState = "stopped" - StateRemoved FeedState = "removed" // deprecated, will be removed in the next version + StateRemoved FeedState = "removed" StateFinished FeedState = "finished" ) @@ -226,10 +227,10 @@ func (info *ChangeFeedInfo) Clone() (*ChangeFeedInfo, error) { return cloned, err } -// VerifyAndFix verifies changefeed info and may fillin some fields. -// If a must field is not provided, return an error. -// If some necessary filed is missing but can use a default value, fillin it. -func (info *ChangeFeedInfo) VerifyAndFix() error { +// VerifyAndComplete verifies changefeed info and may fill in some fields. +// If a required field is not provided, return an error. +// If some necessary filed is missing but can use a default value, fill in it. +func (info *ChangeFeedInfo) VerifyAndComplete() error { defaultConfig := config.GetDefaultReplicaConfig() if info.Engine == "" { info.Engine = SortUnified @@ -255,6 +256,54 @@ func (info *ChangeFeedInfo) VerifyAndFix() error { return nil } +// FixIncompatible fixes incompatible changefeed meta info. +func (info *ChangeFeedInfo) FixIncompatible() { + creatorVersionGate := version.NewCreatorVersionGate(info.CreatorVersion) + if creatorVersionGate.ChangefeedStateFromAdminJob() { + log.Info("Start fixing incompatible changefeed state", zap.Any("changefeed", info)) + info.fixState() + log.Info("Fix incompatibility changefeed state completed", zap.Any("changefeed", info)) + } +} + +// fixState attempts to fix state loss from upgrading the old owner to the new owner. +func (info *ChangeFeedInfo) fixState() { + // Notice: In the old owner we used AdminJobType field to determine if the task was paused or not, + // we need to handle this field in the new owner. + // Otherwise, we will see that the old version of the task is paused and then upgraded, + // and the task is automatically resumed after the upgrade. + state := info.State + // Upgrading from an old owner, we need to deal with cases where the state is normal, + // but actually contains errors and does not match the admin job type. + if state == StateNormal { + switch info.AdminJobType { + // This corresponds to the case of failure or error. + case AdminNone, AdminResume: + if info.Error != nil { + if cerror.ChangefeedFastFailErrorCode(errors.RFCErrorCode(info.Error.Code)) { + state = StateFailed + } else { + state = StateError + } + } + case AdminStop: + state = StateStopped + case AdminFinish: + state = StateFinished + case AdminRemove: + state = StateRemoved + } + } + + if state != info.State { + log.Info("handle old owner inconsistent state", + zap.String("old state", string(info.State)), + zap.String("admin job type", info.AdminJobType.String()), + zap.String("new state", string(state))) + info.State = state + } +} + // CheckErrorHistory checks error history of a changefeed // if having error record older than GC interval, set needSave to true. // if error counts reach threshold, set canInit to false. diff --git a/cdc/model/changefeed_test.go b/cdc/model/changefeed_test.go index 1a2d9103f20..ee1f03618a8 100644 --- a/cdc/model/changefeed_test.go +++ b/cdc/model/changefeed_test.go @@ -163,7 +163,7 @@ func TestFillV1(t *testing.T) { }, cfg) } -func TestVerifyAndFix(t *testing.T) { +func TestVerifyAndComplete(t *testing.T) { t.Parallel() info := &ChangeFeedInfo{ @@ -177,7 +177,7 @@ func TestVerifyAndFix(t *testing.T) { }, } - err := info.VerifyAndFix() + err := info.VerifyAndComplete() require.Nil(t, err) require.Equal(t, SortUnified, info.Engine) @@ -189,6 +189,150 @@ func TestVerifyAndFix(t *testing.T) { require.Equal(t, marshalConfig2, marshalConfig1) } +func TestFixIncompatible(t *testing.T) { + // Test to fix incompatible states. + testCases := []struct { + info *ChangeFeedInfo + expectedState FeedState + }{ + { + info: &ChangeFeedInfo{ + AdminJobType: AdminStop, + State: StateNormal, + Error: nil, + CreatorVersion: "", + }, + expectedState: StateStopped, + }, + { + info: &ChangeFeedInfo{ + AdminJobType: AdminStop, + State: StateNormal, + Error: nil, + CreatorVersion: "4.0.14", + }, + expectedState: StateStopped, + }, + { + info: &ChangeFeedInfo{ + AdminJobType: AdminStop, + State: StateNormal, + Error: nil, + CreatorVersion: "5.0.5", + }, + expectedState: StateStopped, + }, + } + + for _, tc := range testCases { + tc.info.FixIncompatible() + require.Equal(t, tc.expectedState, tc.info.State) + } +} + +func TestFixState(t *testing.T) { + t.Parallel() + + testCases := []struct { + info *ChangeFeedInfo + expectedState FeedState + }{ + { + info: &ChangeFeedInfo{ + AdminJobType: AdminNone, + State: StateNormal, + Error: nil, + }, + expectedState: StateNormal, + }, + { + info: &ChangeFeedInfo{ + AdminJobType: AdminResume, + State: StateNormal, + Error: nil, + }, + expectedState: StateNormal, + }, + { + info: &ChangeFeedInfo{ + AdminJobType: AdminNone, + State: StateNormal, + Error: &RunningError{ + Code: string(cerror.ErrGCTTLExceeded.RFCCode()), + }, + }, + expectedState: StateFailed, + }, + { + info: &ChangeFeedInfo{ + AdminJobType: AdminResume, + State: StateNormal, + Error: &RunningError{ + Code: string(cerror.ErrGCTTLExceeded.RFCCode()), + }, + }, + expectedState: StateFailed, + }, + { + info: &ChangeFeedInfo{ + AdminJobType: AdminNone, + State: StateNormal, + Error: &RunningError{ + Code: string(cerror.ErrClusterIDMismatch.RFCCode()), + }, + }, + expectedState: StateError, + }, + { + info: &ChangeFeedInfo{ + AdminJobType: AdminResume, + State: StateNormal, + Error: &RunningError{ + Code: string(cerror.ErrClusterIDMismatch.RFCCode()), + }, + }, + expectedState: StateError, + }, + { + info: &ChangeFeedInfo{ + AdminJobType: AdminStop, + State: StateNormal, + Error: nil, + }, + expectedState: StateStopped, + }, + { + info: &ChangeFeedInfo{ + AdminJobType: AdminFinish, + State: StateNormal, + Error: nil, + }, + expectedState: StateFinished, + }, + { + info: &ChangeFeedInfo{ + AdminJobType: AdminRemove, + State: StateNormal, + Error: nil, + }, + expectedState: StateRemoved, + }, + { + info: &ChangeFeedInfo{ + AdminJobType: AdminRemove, + State: StateNormal, + Error: nil, + }, + expectedState: StateRemoved, + }, + } + + for _, tc := range testCases { + tc.info.fixState() + require.Equal(t, tc.expectedState, tc.info.State) + } +} + func TestChangeFeedInfoClone(t *testing.T) { t.Parallel() diff --git a/cdc/owner/owner.go b/cdc/owner/owner.go index 8cd40ad1a6e..af6c5c4d7ee 100644 --- a/cdc/owner/owner.go +++ b/cdc/owner/owner.go @@ -86,6 +86,10 @@ type Owner struct { logLimiter *rate.Limiter lastTickTime time.Time closed int32 + // bootstrapped specifies whether the owner has been initialized. + // This will only be done when the owner starts the first Tick. + // NOTICE: Do not use it in a method other than tick unexpectedly, as it is not a thread-safe value. + bootstrapped bool newChangefeed func(id model.ChangeFeedID, gcManager gc.Manager) *changefeed } @@ -108,6 +112,8 @@ func NewOwner4Test( pdClient pd.Client, ) *Owner { o := NewOwner(pdClient) + // Most tests do not need to test bootstrap. + o.bootstrapped = true o.newChangefeed = func(id model.ChangeFeedID, gcManager gc.Manager) *changefeed { return newChangefeed4Test(id, gcManager, newDDLPuller, newSink) } @@ -120,8 +126,15 @@ func (o *Owner) Tick(stdCtx context.Context, rawState orchestrator.ReactorState) failpoint.Return(nil, errors.New("owner run with injected error")) }) failpoint.Inject("sleep-in-owner-tick", nil) - ctx := stdCtx.(cdcContext.Context) state := rawState.(*orchestrator.GlobalReactorState) + // At the first Tick, we need to do a bootstrap operation. + // Fix incompatible or incorrect meta information. + if !o.bootstrapped { + o.Bootstrap(state) + o.bootstrapped = true + return state, nil + } + o.captures = state.Captures o.updateMetrics(state) @@ -143,6 +156,7 @@ func (o *Owner) Tick(stdCtx context.Context, rawState orchestrator.ReactorState) return nil, errors.Trace(err) } + ctx := stdCtx.(cdcContext.Context) for changefeedID, changefeedState := range state.Changefeeds { if changefeedState.Info == nil { o.cleanUpChangefeed(changefeedState) @@ -256,6 +270,24 @@ func (o *Owner) cleanUpChangefeed(state *orchestrator.ChangefeedReactorState) { } } +// Bootstrap checks if the state contains incompatible or incorrect information and tries to fix it. +func (o *Owner) Bootstrap(state *orchestrator.GlobalReactorState) { + log.Info("Start bootstrapping", zap.Any("state", state)) + fixChangefeedInfos(state) +} + +// fixChangefeedInfos attempts to fix incompatible or incorrect meta information in changefeed state. +func fixChangefeedInfos(state *orchestrator.GlobalReactorState) { + for _, changefeedState := range state.Changefeeds { + if changefeedState != nil { + changefeedState.PatchInfo(func(info *model.ChangeFeedInfo) (*model.ChangeFeedInfo, bool, error) { + info.FixIncompatible() + return info, true, nil + }) + } + } +} + func (o *Owner) updateMetrics(state *orchestrator.GlobalReactorState) { // Keep the value of prometheus expression `rate(counter)` = 1 // Please also change alert rule in ticdc.rules.yml when change the expression value. diff --git a/cdc/owner/owner_test.go b/cdc/owner/owner_test.go index fe18e2e69a9..21e55ef397d 100644 --- a/cdc/owner/owner_test.go +++ b/cdc/owner/owner_test.go @@ -187,6 +187,44 @@ func (s *ownerSuite) TestStopChangefeed(c *check.C) { c.Assert(state.Changefeeds, check.Not(check.HasKey), changefeedID) } +func (s *ownerSuite) TestFixChangefeedInfos(c *check.C) { + defer testleak.AfterTest(c)() + ctx := cdcContext.NewBackendContext4Test(false) + owner, state, tester := createOwner4Test(ctx, c) + // We need to do bootstrap. + owner.bootstrapped = false + changefeedID := "test-changefeed" + // Mismatched state and admin job. + changefeedInfo := &model.ChangeFeedInfo{ + State: model.StateNormal, + AdminJobType: model.AdminStop, + StartTs: oracle.GoTimeToTS(time.Now()), + Config: config.GetDefaultReplicaConfig(), + CreatorVersion: "4.0.14", + } + changefeedStr, err := changefeedInfo.Marshal() + c.Assert(err, check.IsNil) + cdcKey := etcd.CDCKey{ + Tp: etcd.CDCKeyTypeChangefeedInfo, + ChangefeedID: changefeedID, + } + tester.MustUpdate(cdcKey.String(), []byte(changefeedStr)) + // For the first tick, we do a bootstrap, and it tries to fix the meta information. + _, err = owner.Tick(ctx, state) + tester.MustApplyPatches() + c.Assert(err, check.IsNil) + c.Assert(owner.bootstrapped, check.IsTrue) + c.Assert(owner.changefeeds, check.Not(check.HasKey), changefeedID) + + // Start tick normally. + _, err = owner.Tick(ctx, state) + tester.MustApplyPatches() + c.Assert(err, check.IsNil) + c.Assert(owner.changefeeds, check.HasKey, changefeedID) + // The meta information is fixed correctly. + c.Assert(owner.changefeeds[changefeedID].state.Info.State, check.Equals, model.StateStopped) +} + func (s *ownerSuite) TestCheckClusterVersion(c *check.C) { defer testleak.AfterTest(c)() ctx := cdcContext.NewBackendContext4Test(false) diff --git a/pkg/cmd/cli/cli_changefeed_create.go b/pkg/cmd/cli/cli_changefeed_create.go index ddf4a04229b..9cfc2d101de 100644 --- a/pkg/cmd/cli/cli_changefeed_create.go +++ b/pkg/cmd/cli/cli_changefeed_create.go @@ -181,7 +181,7 @@ func (o *createChangefeedOptions) completeCfg(ctx context.Context, cmd *cobra.Co return err } - cdcClusterVer, err := version.GetTiCDCClusterVersion(captureInfos) + cdcClusterVer, err := version.GetTiCDCClusterVersion(model.ListVersionsFromCaptureInfos(captureInfos)) if err != nil { return errors.Trace(err) } diff --git a/pkg/cmd/util/helper.go b/pkg/cmd/util/helper.go index 94348175270..69b3aa0a861 100644 --- a/pkg/cmd/util/helper.go +++ b/pkg/cmd/util/helper.go @@ -25,6 +25,7 @@ import ( "github.com/BurntSushi/toml" "github.com/pingcap/errors" "github.com/pingcap/log" + "github.com/pingcap/tiflow/cdc/model" cmdconetxt "github.com/pingcap/tiflow/pkg/cmd/context" "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/logutil" @@ -158,7 +159,7 @@ func VerifyAndGetTiCDCClusterVersion( return version.TiCDCClusterVersion{}, err } - cdcClusterVer, err := version.GetTiCDCClusterVersion(captureInfos) + cdcClusterVer, err := version.GetTiCDCClusterVersion(model.ListVersionsFromCaptureInfos(captureInfos)) if err != nil { return version.TiCDCClusterVersion{}, err } diff --git a/pkg/orchestrator/reactor_state.go b/pkg/orchestrator/reactor_state.go index c1d253e2308..0c7ec3f4b42 100644 --- a/pkg/orchestrator/reactor_state.go +++ b/pkg/orchestrator/reactor_state.go @@ -210,7 +210,7 @@ func (s *ChangefeedReactorState) UpdateCDCKey(key *etcd.CDCKey, value []byte) er return errors.Trace(err) } if key.Tp == etcd.CDCKeyTypeChangefeedInfo { - if err := s.Info.VerifyAndFix(); err != nil { + if err := s.Info.VerifyAndComplete(); err != nil { return errors.Trace(err) } } diff --git a/pkg/version/check.go b/pkg/version/check.go index 63e9e29c2b5..1a66957341e 100644 --- a/pkg/version/check.go +++ b/pkg/version/check.go @@ -25,7 +25,6 @@ import ( "github.com/coreos/go-semver/semver" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/model" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/httputil" "github.com/pingcap/tiflow/pkg/security" @@ -223,16 +222,16 @@ func (v *TiCDCClusterVersion) ShouldEnableUnifiedSorterByDefault() bool { var TiCDCClusterVersionUnknown = TiCDCClusterVersion{} // GetTiCDCClusterVersion returns the version of ticdc cluster -func GetTiCDCClusterVersion(captureInfos []*model.CaptureInfo) (TiCDCClusterVersion, error) { - if len(captureInfos) == 0 { +func GetTiCDCClusterVersion(captureVersion []string) (TiCDCClusterVersion, error) { + if len(captureVersion) == 0 { return TiCDCClusterVersionUnknown, nil } var minVer *semver.Version - for _, captureInfo := range captureInfos { + for _, versionStr := range captureVersion { var ver *semver.Version var err error - if captureInfo.Version != "" { - ver, err = semver.NewVersion(removeVAndHash(captureInfo.Version)) + if versionStr != "" { + ver, err = semver.NewVersion(removeVAndHash(versionStr)) } else { ver = defaultTiCDCVersion } diff --git a/pkg/version/check_test.go b/pkg/version/check_test.go index bd9c54404e0..47dac43a360 100644 --- a/pkg/version/check_test.go +++ b/pkg/version/check_test.go @@ -23,7 +23,6 @@ import ( "github.com/coreos/go-semver/semver" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/tiflow/cdc/model" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" pd "github.com/tikv/pd/client" @@ -188,56 +187,56 @@ func TestReleaseSemver(t *testing.T) { func TestGetTiCDCClusterVersion(t *testing.T) { t.Parallel() testCases := []struct { - captureInfos []*model.CaptureInfo - expected TiCDCClusterVersion + captureVersions []string + expected TiCDCClusterVersion }{ { - captureInfos: []*model.CaptureInfo{}, - expected: TiCDCClusterVersionUnknown, + captureVersions: []string{}, + expected: TiCDCClusterVersionUnknown, }, { - captureInfos: []*model.CaptureInfo{ - {ID: "capture1", Version: ""}, - {ID: "capture2", Version: ""}, - {ID: "capture3", Version: ""}, + captureVersions: []string{ + "", + "", + "", }, expected: TiCDCClusterVersion{defaultTiCDCVersion}, }, { - captureInfos: []*model.CaptureInfo{ - {ID: "capture1", Version: "5.0.1"}, - {ID: "capture2", Version: "4.0.7"}, - {ID: "capture3", Version: "5.0.0-rc"}, + captureVersions: []string{ + "5.0.1", + "4.0.7", + "5.0.0-rc", }, expected: TiCDCClusterVersion{semver.New("4.0.7")}, }, { - captureInfos: []*model.CaptureInfo{ - {ID: "capture1", Version: "5.0.0-rc"}, + captureVersions: []string{ + "5.0.0-rc", }, expected: TiCDCClusterVersion{semver.New("5.0.0-rc")}, }, { - captureInfos: []*model.CaptureInfo{ - {ID: "capture1", Version: "5.0.0"}, + captureVersions: []string{ + "5.0.0", }, expected: TiCDCClusterVersion{semver.New("5.0.0")}, }, { - captureInfos: []*model.CaptureInfo{ - {ID: "capture1", Version: "4.1.0"}, + captureVersions: []string{ + "4.1.0", }, expected: TiCDCClusterVersion{semver.New("4.1.0")}, }, { - captureInfos: []*model.CaptureInfo{ - {ID: "capture1", Version: "4.0.10"}, + captureVersions: []string{ + "4.0.10", }, expected: TiCDCClusterVersion{semver.New("4.0.10")}, }, } for _, tc := range testCases { - ver, err := GetTiCDCClusterVersion(tc.captureInfos) + ver, err := GetTiCDCClusterVersion(tc.captureVersions) require.Nil(t, err) require.Equal(t, ver, tc.expected) } diff --git a/pkg/version/creator_version_gate.go b/pkg/version/creator_version_gate.go new file mode 100644 index 00000000000..30102682e06 --- /dev/null +++ b/pkg/version/creator_version_gate.go @@ -0,0 +1,61 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package version + +import ( + "github.com/coreos/go-semver/semver" +) + +// CreatorVersionGate determines the introduced version and compatibility +// of some features based on the creator's version value. +type CreatorVersionGate struct { + version string +} + +// changefeedStateFromAdminJobVersions specifies the version before +// which we use the admin job type to control the state of the changefeed. +var changefeedStateFromAdminJobVersions = []semver.Version{ + // Introduced in https://github.com/pingcap/ticdc/pull/3014. + *semver.New("4.0.16"), + // Introduced in https://github.com/pingcap/ticdc/pull/2946. + *semver.New("5.0.6"), +} + +// NewCreatorVersionGate creates the creator version gate. +func NewCreatorVersionGate(version string) *CreatorVersionGate { + return &CreatorVersionGate{ + version: version, + } +} + +// ChangefeedStateFromAdminJob determines if admin job is the state +// of changefeed based on the version of the creator. +func (f *CreatorVersionGate) ChangefeedStateFromAdminJob() bool { + // Introduced in https://github.com/pingcap/ticdc/pull/1341. + // The changefeed before it was introduced was using the old owner. + if f.version == "" { + return true + } + + creatorVersion := semver.New(removeVAndHash(f.version)) + for _, version := range changefeedStateFromAdminJobVersions { + // NOTICE: To compare against the same major version. + if creatorVersion.Major == version.Major && + creatorVersion.LessThan(version) { + return true + } + } + + return false +} diff --git a/pkg/version/creator_version_gate_test.go b/pkg/version/creator_version_gate_test.go new file mode 100644 index 00000000000..5e79859e7fa --- /dev/null +++ b/pkg/version/creator_version_gate_test.go @@ -0,0 +1,78 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package version + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestChangefeedStateFromAdminJob(t *testing.T) { + t.Parallel() + + testCases := []struct { + creatorVersion string + expected bool + }{ + { + creatorVersion: "", + expected: true, + }, + { + creatorVersion: "4.0.12", + expected: true, + }, + { + creatorVersion: "4.0.14", + expected: true, + }, + { + creatorVersion: "4.0.15", + expected: true, + }, + { + creatorVersion: "4.0.16", + }, + { + creatorVersion: "5.0.0", + expected: true, + }, + { + creatorVersion: "5.0.1", + expected: true, + }, + { + creatorVersion: "5.0.6", + expected: false, + }, + { + creatorVersion: "5.1.0", + expected: false, + }, + { + creatorVersion: "5.2.0", + expected: false, + }, + { + creatorVersion: "5.3.0", + expected: false, + }, + } + + for _, tc := range testCases { + creatorVersionGate := CreatorVersionGate{version: tc.creatorVersion} + require.Equal(t, tc.expected, creatorVersionGate.ChangefeedStateFromAdminJob()) + } +} From 72ce433490bdf7d9b6fa7a7c5e5f6130ba709081 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 10 Jan 2022 16:01:41 +0800 Subject: [PATCH 39/48] redolog: add a precleanup process when s3 enable (#3525) (#3878) --- cdc/redo/writer/file.go | 4 ++ cdc/redo/writer/file_test.go | 11 +++- cdc/redo/writer/writer.go | 53 +++++++++++++++- cdc/redo/writer/writer_test.go | 109 ++++++++++++++++++++++++++++++++- 4 files changed, 174 insertions(+), 3 deletions(-) diff --git a/cdc/redo/writer/file.go b/cdc/redo/writer/file.go index 86a636af3d1..0b4d4fb136e 100644 --- a/cdc/redo/writer/file.go +++ b/cdc/redo/writer/file.go @@ -459,6 +459,10 @@ func (w *Writer) shouldRemoved(checkPointTs uint64, f os.FileInfo) (bool, error) func (w *Writer) getShouldRemovedFiles(checkPointTs uint64) ([]os.FileInfo, error) { files, err := ioutil.ReadDir(w.cfg.Dir) if err != nil { + if os.IsNotExist(err) { + log.Warn("check removed log dir fail", zap.Error(err)) + return []os.FileInfo{}, nil + } return nil, cerror.WrapError(cerror.ErrRedoFileOp, errors.Annotatef(err, "can't read log file directory: %s", w.cfg.Dir)) } diff --git a/cdc/redo/writer/file_test.go b/cdc/redo/writer/file_test.go index e71b814d553..208e843589f 100644 --- a/cdc/redo/writer/file_test.go +++ b/cdc/redo/writer/file_test.go @@ -197,8 +197,17 @@ func TestWriterGC(t *testing.T) { require.Nil(t, err, files[0].Name()) require.EqualValues(t, 3, ts) require.Equal(t, common.DefaultRowLogFileType, fileType) - time.Sleep(time.Duration(100) * time.Millisecond) + + w1 := &Writer{ + cfg: cfg, + uint64buf: make([]byte, 8), + storage: mockStorage, + } + w1.cfg.Dir += "not-exist" + w1.running.Store(true) + err = w1.GC(111) + require.Nil(t, err) } func TestAdvanceTs(t *testing.T) { diff --git a/cdc/redo/writer/writer.go b/cdc/redo/writer/writer.go index fceb3de233b..debf2243596 100644 --- a/cdc/redo/writer/writer.go +++ b/cdc/redo/writer/writer.go @@ -175,12 +175,52 @@ func NewLogWriter(ctx context.Context, cfg *LogWriterConfig) (*LogWriter, error) if err != nil { return nil, err } + } else { + if cfg.S3Storage { + // since other process get the remove changefeed job async, may still write some logs after owner delete the log + err = logWriter.preCleanUpS3(ctx) + if err != nil { + return nil, err + } + } } logWriters[cfg.ChangeFeedID] = logWriter go logWriter.runGC(ctx) return logWriter, nil } +func (l *LogWriter) preCleanUpS3(ctx context.Context) error { + ret, err := l.storage.FileExists(ctx, l.getDeletedChangefeedMarker()) + if err != nil { + return cerror.WrapError(cerror.ErrS3StorageAPI, err) + } + if !ret { + return nil + } + + files, err := getAllFilesInS3(ctx, l) + if err != nil { + return err + } + + ff := []string{} + for _, file := range files { + if file != l.getDeletedChangefeedMarker() { + ff = append(ff, file) + } + } + err = l.deleteFilesInS3(ctx, ff) + if err != nil { + return err + } + err = l.storage.DeleteFile(ctx, l.getDeletedChangefeedMarker()) + if !isNotExistInS3(err) { + return cerror.WrapError(cerror.ErrS3StorageAPI, err) + } + + return nil +} + func (l *LogWriter) initMeta(ctx context.Context) error { select { case <-ctx.Done(): @@ -433,7 +473,18 @@ func (l *LogWriter) DeleteAllLogs(ctx context.Context) error { } // after delete logs, rm the LogWriter since it is already closed l.cleanUpLogWriter() - return nil + + // write a marker to s3, since other process get the remove changefeed job async, + // may still write some logs after owner delete the log + return l.writeDeletedMarkerToS3(ctx) +} + +func (l *LogWriter) getDeletedChangefeedMarker() string { + return fmt.Sprintf("delete_%s", l.cfg.ChangeFeedID) +} + +func (l *LogWriter) writeDeletedMarkerToS3(ctx context.Context) error { + return cerror.WrapError(cerror.ErrS3StorageAPI, l.storage.WriteFile(ctx, l.getDeletedChangefeedMarker(), []byte("D"))) } func (l *LogWriter) cleanUpLogWriter() { diff --git a/cdc/redo/writer/writer_test.go b/cdc/redo/writer/writer_test.go index 618281b5537..e32f3074fa1 100644 --- a/cdc/redo/writer/writer_test.go +++ b/cdc/redo/writer/writer_test.go @@ -18,6 +18,7 @@ import ( "fmt" "io/ioutil" "math" + "net/url" "os" "path/filepath" "strings" @@ -29,6 +30,7 @@ import ( "github.com/golang/mock/gomock" "github.com/pingcap/errors" mockstorage "github.com/pingcap/tidb/br/pkg/mock/storage" + "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/redo/common" cerror "github.com/pingcap/tiflow/pkg/errors" @@ -675,6 +677,31 @@ func TestNewLogWriter(t *testing.T) { require.Equal(t, meta.ResolvedTs, l.meta.ResolvedTs) require.Equal(t, map[int64]uint64{}, l.meta.ResolvedTsList) time.Sleep(time.Millisecond * time.Duration(math.Max(float64(defaultFlushIntervalInMs), float64(defaultGCIntervalInMs))+1)) + + origin := common.InitS3storage + defer func() { + common.InitS3storage = origin + }() + controller := gomock.NewController(t) + mockStorage := mockstorage.NewMockExternalStorage(controller) + // skip pre cleanup + mockStorage.EXPECT().FileExists(gomock.Any(), gomock.Any()).Return(false, nil) + common.InitS3storage = func(ctx context.Context, uri url.URL) (storage.ExternalStorage, error) { + return mockStorage, nil + } + cfg3 := &LogWriterConfig{ + Dir: dir, + ChangeFeedID: "test-cf112232", + CaptureID: "cp", + MaxLogSize: 10, + CreateTime: time.Date(2000, 1, 1, 1, 1, 1, 1, &time.Location{}), + FlushIntervalInMs: 5, + S3Storage: true, + } + l3, err := NewLogWriter(ctx, cfg3) + require.Nil(t, err) + err = l3.Close() + require.Nil(t, err) } func TestWriterRedoGC(t *testing.T) { @@ -750,6 +777,7 @@ func TestDeleteAllLogs(t *testing.T) { closeErr error getAllFilesInS3Err error deleteFileErr error + writeFileErr error wantErr string }{ { @@ -783,6 +811,12 @@ func TestDeleteAllLogs(t *testing.T) { args: args{enableS3: true}, deleteFileErr: awserr.New(s3.ErrCodeNoSuchKey, "no such key", nil), }, + { + name: "writerFile err", + args: args{enableS3: true}, + writeFileErr: errors.New("xx"), + wantErr: ".*xx*.", + }, } for _, tt := range tests { @@ -803,6 +837,8 @@ func TestDeleteAllLogs(t *testing.T) { mockStorage := mockstorage.NewMockExternalStorage(controller) mockStorage.EXPECT().DeleteFile(gomock.Any(), gomock.Any()).Return(tt.deleteFileErr).MaxTimes(2) + mockStorage.EXPECT().WriteFile(gomock.Any(), gomock.Any(), gomock.Any()).Return(tt.writeFileErr).MaxTimes(1) + mockWriter := &mockFileWriter{} mockWriter.On("Close").Return(tt.closeErr) cfg := &LogWriterConfig{ @@ -829,7 +865,8 @@ func TestDeleteAllLogs(t *testing.T) { require.Regexp(t, tt.wantErr, ret.Error(), tt.name) } else { require.Nil(t, ret, tt.name) - require.Equal(t, 0, len(logWriters), tt.name) + _, ok := logWriters[writer.cfg.ChangeFeedID] + require.False(t, ok, tt.name) if !tt.args.enableS3 { _, err := os.Stat(dir) require.True(t, os.IsNotExist(err), tt.name) @@ -839,3 +876,73 @@ func TestDeleteAllLogs(t *testing.T) { getAllFilesInS3 = origin } } + +func TestPreCleanUpS3(t *testing.T) { + testCases := []struct { + name string + fileExistsErr error + fileExists bool + getAllFilesInS3Err error + deleteFileErr error + wantErr string + }{ + { + name: "happy no marker", + fileExists: false, + }, + { + name: "fileExists err", + fileExistsErr: errors.New("xx"), + wantErr: ".*xx*.", + }, + { + name: "getAllFilesInS3 err", + fileExists: true, + getAllFilesInS3Err: errors.New("xx"), + wantErr: ".*xx*.", + }, + { + name: "deleteFile normal err", + fileExists: true, + deleteFileErr: errors.New("xx"), + wantErr: ".*ErrS3StorageAPI*.", + }, + { + name: "deleteFile notExist err", + fileExists: true, + deleteFileErr: awserr.New(s3.ErrCodeNoSuchKey, "no such key", nil), + }, + } + + for _, tc := range testCases { + origin := getAllFilesInS3 + getAllFilesInS3 = func(ctx context.Context, l *LogWriter) ([]string, error) { + return []string{"1", "11", "delete_test-cf"}, tc.getAllFilesInS3Err + } + controller := gomock.NewController(t) + mockStorage := mockstorage.NewMockExternalStorage(controller) + + mockStorage.EXPECT().FileExists(gomock.Any(), gomock.Any()).Return(tc.fileExists, tc.fileExistsErr) + mockStorage.EXPECT().DeleteFile(gomock.Any(), gomock.Any()).Return(tc.deleteFileErr).MaxTimes(3) + + cfg := &LogWriterConfig{ + Dir: "dir", + ChangeFeedID: "test-cf", + CaptureID: "cp", + MaxLogSize: 10, + CreateTime: time.Date(2000, 1, 1, 1, 1, 1, 1, &time.Location{}), + FlushIntervalInMs: 5, + } + writer := LogWriter{ + cfg: cfg, + storage: mockStorage, + } + ret := writer.preCleanUpS3(context.Background()) + if tc.wantErr != "" { + require.Regexp(t, tc.wantErr, ret.Error(), tc.name) + } else { + require.Nil(t, ret, tc.name) + } + getAllFilesInS3 = origin + } +} From de5de8b3ebd66d3156324ef70e72d9b322a5ebd9 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 10 Jan 2022 17:31:42 +0800 Subject: [PATCH 40/48] ddl(dm): make skipped ddl pass `SplitDDL()` (#4176) (#4227) close pingcap/tiflow#4173 --- dm/syncer/syncer.go | 33 ++++++++++++++++++++++----------- dm/tests/all_mode/run.sh | 10 +++++++++- 2 files changed, 31 insertions(+), 12 deletions(-) diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 3dfe640503b..fe463ea225b 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -2159,26 +2159,37 @@ func (s *Syncer) handleQueryEvent(ev *replication.QueryEvent, ec eventContext, o appliedDDLs: make([]string, 0), sourceTbls: make(map[string]map[string]struct{}), } - qec.p, err = event.GetParserForStatusVars(ev.StatusVars) - if err != nil { - log.L().Warn("found error when get sql_mode from binlog status_vars", zap.Error(err)) - } - stmt, err := parseOneStmt(qec) - if err != nil { - // return error if parse fail and filter fail + defer func() { + if err == nil { + return + } + // why not `skipSQLByPattern` at beginning, but at defer? + // it is in order to track every ddl except for the one that will cause error. + // if `skipSQLByPattern` at beginning, some ddl should be tracked may be skipped. needSkip, err2 := s.skipSQLByPattern(qec.originSQL) if err2 != nil { - return err2 + err = err2 + return } if !needSkip { - return err + return } - // don't return error if parse fail and filter success + // don't return error if filter success metrics.SkipBinlogDurationHistogram.WithLabelValues("query", s.cfg.Name, s.cfg.SourceID).Observe(time.Since(ec.startTime).Seconds()) ec.tctx.L().Warn("skip event", zap.String("event", "query"), zap.Stringer("query event context", qec)) *ec.lastLocation = *ec.currentLocation // before record skip location, update lastLocation - return s.recordSkipSQLsLocation(&ec) + err = s.recordSkipSQLsLocation(&ec) + }() + + qec.p, err = event.GetParserForStatusVars(ev.StatusVars) + if err != nil { + log.L().Warn("found error when get sql_mode from binlog status_vars", zap.Error(err)) + } + + stmt, err := parseOneStmt(qec) + if err != nil { + return err } if node, ok := stmt.(ast.DMLNode); ok { diff --git a/dm/tests/all_mode/run.sh b/dm/tests/all_mode/run.sh index 4eba97d890f..c3d87bcf207 100755 --- a/dm/tests/all_mode/run.sh +++ b/dm/tests/all_mode/run.sh @@ -380,6 +380,15 @@ function run() { # use sync_diff_inspector to check full dump loader check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + # check create view(should be skipped by func `skipSQLByPattern`) will not stop sync task + run_sql_source1 "create view all_mode.t1_v as select * from all_mode.t1 where id=0;" + sleep 1 + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "query-status -s $SOURCE_ID1" \ + "\"result\": true" 2 \ + "\"unit\": \"Sync\"" 1 \ + "\"stage\": \"Running\"" 2 + run_sql_source1 "SHOW SLAVE HOSTS;" check_contains 'Slave_UUID' @@ -498,7 +507,6 @@ function run() { check_log_not_contains $WORK_DIR/worker2/log/dm-worker.log "Error .* Table .* doesn't exist" # test Db not exists should be reported - run_sql_tidb "drop database all_mode" run_sql_source1 "create table all_mode.db_error (c int primary key);" run_dm_ctl_with_retry $WORK_DIR "127.0.0.1:$MASTER_PORT" \ From b55018e7b88df1471c4151d72c6077df54ac6e6c Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Tue, 11 Jan 2022 19:52:20 +0800 Subject: [PATCH 41/48] cdc/sink: remove Initialize method from the sink interface (#3682) (#3765) Co-authored-by: Ling Jin <7138436+3AceShowHand@users.noreply.github.com> --- cdc/owner/async_sink.go | 5 ----- cdc/owner/async_sink_test.go | 25 ++++--------------------- cdc/owner/changefeed.go | 5 +---- cdc/processor/pipeline/sink_test.go | 4 ---- cdc/sink/black_hole.go | 5 ----- cdc/sink/manager_test.go | 8 -------- cdc/sink/mq.go | 6 ------ cdc/sink/mysql.go | 5 ----- cdc/sink/simple_mysql_tester.go | 5 ----- cdc/sink/sink.go | 2 -- cdc/sink/table_sink.go | 5 ----- 11 files changed, 5 insertions(+), 70 deletions(-) diff --git a/cdc/owner/async_sink.go b/cdc/owner/async_sink.go index 632883e5dfd..da85c5fd134 100644 --- a/cdc/owner/async_sink.go +++ b/cdc/owner/async_sink.go @@ -38,7 +38,6 @@ const ( // The EmitCheckpointTs and EmitDDLEvent is asynchronous function for now // Other functions are still synchronization type AsyncSink interface { - Initialize(ctx cdcContext.Context, tableInfo []*model.SimpleTableInfo) error // EmitCheckpointTs emits the checkpoint Ts to downstream data source // this function will return after recording the checkpointTs specified in memory immediately // and the recorded checkpointTs will be sent and updated to downstream data source every second @@ -101,10 +100,6 @@ func newAsyncSink(ctx cdcContext.Context) (AsyncSink, error) { return asyncSink, nil } -func (s *asyncSinkImpl) Initialize(ctx cdcContext.Context, tableInfo []*model.SimpleTableInfo) error { - return s.sink.Initialize(ctx, tableInfo) -} - func (s *asyncSinkImpl) run(ctx cdcContext.Context) { defer s.wg.Done() // TODO make the tick duration configurable diff --git a/cdc/owner/async_sink_test.go b/cdc/owner/async_sink_test.go index 74709d31d74..c9bb6ed9b4a 100644 --- a/cdc/owner/async_sink_test.go +++ b/cdc/owner/async_sink_test.go @@ -36,16 +36,10 @@ type asyncSinkSuite struct{} type mockSink struct { sink.Sink - initTableInfo []*model.SimpleTableInfo - checkpointTs model.Ts - ddl *model.DDLEvent - ddlMu sync.Mutex - ddlError error -} - -func (m *mockSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error { - m.initTableInfo = tableInfo - return nil + checkpointTs model.Ts + ddl *model.DDLEvent + ddlMu sync.Mutex + ddlError error } func (m *mockSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { @@ -87,17 +81,6 @@ func newAsyncSink4Test(ctx cdcContext.Context, c *check.C) (cdcContext.Context, return ctx, sink, mockSink } -func (s *asyncSinkSuite) TestInitialize(c *check.C) { - defer testleak.AfterTest(c)() - ctx := cdcContext.NewBackendContext4Test(false) - ctx, sink, mockSink := newAsyncSink4Test(ctx, c) - defer sink.Close(ctx) - tableInfos := []*model.SimpleTableInfo{{Schema: "test"}} - err := sink.Initialize(ctx, tableInfos) - c.Assert(err, check.IsNil) - c.Assert(tableInfos, check.DeepEquals, mockSink.initTableInfo) -} - func (s *asyncSinkSuite) TestCheckpoint(c *check.C) { defer testleak.AfterTest(c)() ctx := cdcContext.NewBackendContext4Test(false) diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go index a65039fd46a..d3153d86fc8 100644 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -260,10 +260,7 @@ LOOP: if err != nil { return errors.Trace(err) } - err = c.sink.Initialize(cancelCtx, c.schema.SinkTableInfos()) - if err != nil { - return errors.Trace(err) - } + // Refer to the previous comment on why we use (checkpointTs-1). c.ddlPuller, err = c.newDDLPuller(cancelCtx, checkpointTs-1) if err != nil { diff --git a/cdc/processor/pipeline/sink_test.go b/cdc/processor/pipeline/sink_test.go index dc87961ca1e..03fbeb061ef 100644 --- a/cdc/processor/pipeline/sink_test.go +++ b/cdc/processor/pipeline/sink_test.go @@ -58,10 +58,6 @@ func (c *mockFlowController) GetConsumption() uint64 { return 0 } -func (s *mockSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error { - return nil -} - func (s *mockSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { for _, row := range rows { s.received = append(s.received, struct { diff --git a/cdc/sink/black_hole.go b/cdc/sink/black_hole.go index 051da83bf05..27f6b37df39 100644 --- a/cdc/sink/black_hole.go +++ b/cdc/sink/black_hole.go @@ -68,11 +68,6 @@ func (b *blackHoleSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) e return nil } -// Initialize is no-op for blackhole -func (b *blackHoleSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error { - return nil -} - func (b *blackHoleSink) Close(ctx context.Context) error { return nil } diff --git a/cdc/sink/manager_test.go b/cdc/sink/manager_test.go index e04d4ec875e..7e901df53de 100644 --- a/cdc/sink/manager_test.go +++ b/cdc/sink/manager_test.go @@ -48,10 +48,6 @@ func newCheckSink(c *check.C) *checkSink { } } -func (c *checkSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error { - panic("unreachable") -} - func (c *checkSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { c.rowsMu.Lock() defer c.rowsMu.Unlock() @@ -344,10 +340,6 @@ type errorSink struct { *check.C } -func (e *errorSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error { - panic("unreachable") -} - func (e *errorSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { return errors.New("error in emit row changed events") } diff --git a/cdc/sink/mq.go b/cdc/sink/mq.go index fa92bb62f75..7b00fb707b2 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -238,12 +238,6 @@ func (k *mqSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error { return errors.Trace(err) } -// Initialize registers Avro schemas for all tables -func (k *mqSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error { - // No longer need it for now - return nil -} - func (k *mqSink) Close(ctx context.Context) error { err := k.mqProducer.Close() return errors.Trace(err) diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index edfc0cbecec..704c069cb81 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -203,11 +203,6 @@ func (s *mysqlSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error return errors.Trace(err) } -// Initialize is no-op for Mysql sink -func (s *mysqlSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error { - return nil -} - func (s *mysqlSink) execDDLWithMaxRetries(ctx context.Context, ddl *model.DDLEvent) error { return retry.Do(ctx, func() error { err := s.execDDL(ctx, ddl) diff --git a/cdc/sink/simple_mysql_tester.go b/cdc/sink/simple_mysql_tester.go index 75f71639401..39b7af6b080 100644 --- a/cdc/sink/simple_mysql_tester.go +++ b/cdc/sink/simple_mysql_tester.go @@ -106,11 +106,6 @@ func newSimpleMySQLSink(ctx context.Context, sinkURI *url.URL, config *config.Re return sink, nil } -func (s *simpleMySQLSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error { - // do nothing - return nil -} - // EmitRowChangedEvents sends Row Changed Event to Sink // EmitRowChangedEvents may write rows to downstream directly; func (s *simpleMySQLSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { diff --git a/cdc/sink/sink.go b/cdc/sink/sink.go index 013aeae1035..7bc9cfd1845 100644 --- a/cdc/sink/sink.go +++ b/cdc/sink/sink.go @@ -33,8 +33,6 @@ const ( // Sink is an abstraction for anything that a changefeed may emit into. type Sink interface { - Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error - // EmitRowChangedEvents sends Row Changed Event to Sink // EmitRowChangedEvents may write rows to downstream directly; EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error diff --git a/cdc/sink/table_sink.go b/cdc/sink/table_sink.go index 3a5d1a205a6..0ac9dd016aa 100644 --- a/cdc/sink/table_sink.go +++ b/cdc/sink/table_sink.go @@ -34,11 +34,6 @@ type tableSink struct { redoManager redo.LogManager } -func (t *tableSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error { - // do nothing - return nil -} - func (t *tableSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { t.buffer = append(t.buffer, rows...) t.manager.metricsTableSinkTotalRows.Add(float64(len(rows))) From 08bd00839db50e4144af33c94b7eef717cd8c790 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Wed, 12 Jan 2022 21:53:42 +0800 Subject: [PATCH 42/48] http_api (ticdc): fix http api 'get processor' panic. (#4117) (#4123) close pingcap/tiflow#3840 --- cdc/capture/http_errors.go | 2 +- cdc/capture/http_handler.go | 2 ++ tests/integration_tests/http_api/util/test_case.py | 11 ++++++++--- 3 files changed, 11 insertions(+), 4 deletions(-) diff --git a/cdc/capture/http_errors.go b/cdc/capture/http_errors.go index ed58b71e108..79db530429d 100644 --- a/cdc/capture/http_errors.go +++ b/cdc/capture/http_errors.go @@ -25,7 +25,7 @@ var httpBadRequestError = []*errors.Error{ cerror.ErrAPIInvalidParam, cerror.ErrSinkURIInvalid, cerror.ErrStartTsBeforeGC, cerror.ErrChangeFeedNotExists, cerror.ErrTargetTsBeforeStartTs, cerror.ErrTableIneligible, cerror.ErrFilterRuleInvalid, cerror.ErrChangefeedUpdateRefused, cerror.ErrMySQLConnectionError, - cerror.ErrMySQLInvalidConfig, + cerror.ErrMySQLInvalidConfig, cerror.ErrCaptureNotExist, } // IsHTTPBadRequestError check if a error is a http bad request error diff --git a/cdc/capture/http_handler.go b/cdc/capture/http_handler.go index a13f2ffffe2..05a332f826b 100644 --- a/cdc/capture/http_handler.go +++ b/cdc/capture/http_handler.go @@ -576,6 +576,7 @@ func (h *HTTPHandler) GetProcessor(c *gin.Context) { status, exist := statuses[captureID] if !exist { _ = c.Error(cerror.ErrCaptureNotExist.GenWithStackByArgs(captureID)) + return } positions, err := statusProvider.GetTaskPositions(ctx, changefeedID) @@ -586,6 +587,7 @@ func (h *HTTPHandler) GetProcessor(c *gin.Context) { position, exist := positions[captureID] if !exist { _ = c.Error(cerror.ErrCaptureNotExist.GenWithStackByArgs(captureID)) + return } processorDetail := &model.ProcessorDetail{CheckPointTs: position.CheckPointTs, ResolvedTs: position.ResolvedTs, Error: position.Error} diff --git a/tests/integration_tests/http_api/util/test_case.py b/tests/integration_tests/http_api/util/test_case.py index 5206bc9b8ca..382ffe1ab55 100644 --- a/tests/integration_tests/http_api/util/test_case.py +++ b/tests/integration_tests/http_api/util/test_case.py @@ -243,14 +243,19 @@ def list_processor(): # must at least one table is sync will the test success def get_processor(): - url = BASE_URL0 + "/processors" - resp = rq.get(url, cert=CERT, verify=VERIFY) + base_url = BASE_URL0 + "/processors" + resp = rq.get(base_url, cert=CERT, verify=VERIFY) assert resp.status_code == rq.codes.ok data = resp.json()[0] - url = url + "/" + data["changefeed_id"] + "/" + data["capture_id"] + url = base_url + "/" + data["changefeed_id"] + "/" + data["capture_id"] resp = rq.get(url, cert=CERT, verify=VERIFY) assert resp.status_code == rq.codes.ok + # test capture_id error and cdc server no panic + url = base_url + "/" + data["changefeed_id"] + "/" + "non-exist-capture-id" + resp = rq.get(url, cert=CERT, verify=VERIFY) + assert resp.status_code == rq.codes.bad_request + print("pass test: get processors") From b34f9293df3380e8a7d3b47f1982d91d64f5feaa Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Thu, 13 Jan 2022 10:47:42 +0800 Subject: [PATCH 43/48] sink (ticdc): fix a deadlock due to checkpointTs fall back in sinkNode (#4084) (#4099) close pingcap/tiflow#4055 --- cdc/processor/pipeline/sink.go | 11 +++++- cdc/processor/pipeline/sink_test.go | 58 +++++++++++++++++++++++++++++ cdc/processor/pipeline/sorter.go | 2 +- cdc/sink/manager.go | 1 + cdc/sink/table_sink.go | 9 ++++- pkg/pipeline/test.go | 2 +- 6 files changed, 79 insertions(+), 4 deletions(-) diff --git a/cdc/processor/pipeline/sink.go b/cdc/processor/pipeline/sink.go index 5c09a6736cf..849c2d79755 100644 --- a/cdc/processor/pipeline/sink.go +++ b/cdc/processor/pipeline/sink.go @@ -142,12 +142,21 @@ func (n *sinkNode) flushSink(ctx pipeline.NodeContext, resolvedTs model.Ts) (err if err != nil { return errors.Trace(err) } + + // we must call flowController.Release immediately after we call + // FlushRowChangedEvents to prevent deadlock cause by checkpointTs + // fall back + n.flowController.Release(checkpointTs) + + // the checkpointTs may fall back in some situation such as: + // 1. This table is newly added to the processor + // 2. There is one table in the processor that has a smaller + // checkpointTs than this one if checkpointTs <= n.checkpointTs { return nil } atomic.StoreUint64(&n.checkpointTs, checkpointTs) - n.flowController.Release(checkpointTs) return nil } diff --git a/cdc/processor/pipeline/sink_test.go b/cdc/processor/pipeline/sink_test.go index 03fbeb061ef..75b12a5ba4c 100644 --- a/cdc/processor/pipeline/sink_test.go +++ b/cdc/processor/pipeline/sink_test.go @@ -555,3 +555,61 @@ func (s *outputSuite) TestSplitUpdateEventWhenDisableOldValue(c *check.C) { c.Assert(node.eventBuffer[insertEventIndex].Row.Columns, check.HasLen, 2) c.Assert(node.eventBuffer[insertEventIndex].Row.PreColumns, check.HasLen, 0) } + +type flushFlowController struct { + mockFlowController + releaseCounter int +} + +func (c *flushFlowController) Release(resolvedTs uint64) { + c.releaseCounter++ +} + +type flushSink struct { + mockSink +} + +// use to simulate the situation that resolvedTs return from sink manager +// fall back +var fallBackResolvedTs = uint64(10) + +func (s *flushSink) FlushRowChangedEvents(ctx context.Context, _ model.TableID, resolvedTs uint64) (uint64, error) { + if resolvedTs == fallBackResolvedTs { + return 0, nil + } + return resolvedTs, nil +} + +// TestFlushSinkReleaseFlowController tests sinkNode.flushSink method will always +// call flowController.Release to release the memory quota of the table to avoid +// deadlock if there is no error occur +func (s *outputSuite) TestFlushSinkReleaseFlowController(c *check.C) { + defer testleak.AfterTest(c)() + ctx := cdcContext.NewContext(context.Background(), &cdcContext.GlobalVars{}) + cfg := config.GetDefaultReplicaConfig() + cfg.EnableOldValue = false + ctx = cdcContext.WithChangefeedVars(ctx, &cdcContext.ChangefeedVars{ + ID: "changefeed-id-test-flushSink", + Info: &model.ChangeFeedInfo{ + StartTs: oracle.GoTimeToTS(time.Now()), + Config: cfg, + }, + }) + flowController := &flushFlowController{} + sink := &flushSink{} + // sNode is a sinkNode + sNode := newSinkNode(1, sink, 0, 10, flowController) + c.Assert(sNode.Init(pipeline.MockNodeContext4Test(ctx, pipeline.Message{}, nil)), check.IsNil) + sNode.barrierTs = 10 + + cctx := pipeline.MockNodeContext4Test(nil, pipeline.TickMessage(), nil) + err := sNode.flushSink(cctx, uint64(8)) + c.Assert(err, check.IsNil) + c.Assert(sNode.checkpointTs, check.Equals, uint64(8)) + c.Assert(flowController.releaseCounter, check.Equals, 1) + // resolvedTs will fall back in this call + err = sNode.flushSink(cctx, uint64(10)) + c.Assert(err, check.IsNil) + c.Assert(sNode.checkpointTs, check.Equals, uint64(8)) + c.Assert(flowController.releaseCounter, check.Equals, 2) +} diff --git a/cdc/processor/pipeline/sorter.go b/cdc/processor/pipeline/sorter.go index cfd7e60a226..57ae125f517 100644 --- a/cdc/processor/pipeline/sorter.go +++ b/cdc/processor/pipeline/sorter.go @@ -147,7 +147,7 @@ func (n *sorterNode) Init(ctx pipeline.NodeContext) error { } } // NOTE we allow the quota to be exceeded if blocking means interrupting a transaction. - // Otherwise the pipeline would deadlock. + // Otherwise, the pipeline would deadlock. err := n.flowController.Consume(commitTs, size, func() error { if lastCRTs > lastSentResolvedTs { // If we are blocking, we send a Resolved Event here to elicit a sink-flush. diff --git a/cdc/sink/manager.go b/cdc/sink/manager.go index c1bfc8e8d1f..63722cf77db 100644 --- a/cdc/sink/manager.go +++ b/cdc/sink/manager.go @@ -162,6 +162,7 @@ func (m *Manager) getCheckpointTs(tableID model.TableID) uint64 { return atomic.LoadUint64(&m.changeFeedCheckpointTs) } +// UpdateChangeFeedCheckpointTs update the changeFeedCheckpointTs every processor tick func (m *Manager) UpdateChangeFeedCheckpointTs(checkpointTs uint64) { atomic.StoreUint64(&m.changeFeedCheckpointTs, checkpointTs) if m.backendSink != nil { diff --git a/cdc/sink/table_sink.go b/cdc/sink/table_sink.go index 0ac9dd016aa..48858f525b0 100644 --- a/cdc/sink/table_sink.go +++ b/cdc/sink/table_sink.go @@ -34,6 +34,13 @@ type tableSink struct { redoManager redo.LogManager } +func (t *tableSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTableInfo) error { + // do nothing + return nil +} + +var _ Sink = (*tableSink)(nil) + func (t *tableSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { t.buffer = append(t.buffer, rows...) t.manager.metricsTableSinkTotalRows.Add(float64(len(rows))) @@ -127,7 +134,7 @@ func (t *tableSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { return nil } -// Note once the Close is called, no more events can be written to this table sink +// Close once the method is called, no more events can be written to this table sink func (t *tableSink) Close(ctx context.Context) error { return t.manager.destroyTableSink(ctx, t.tableID) } diff --git a/pkg/pipeline/test.go b/pkg/pipeline/test.go index ae9373c7fb8..0bc995d28ac 100644 --- a/pkg/pipeline/test.go +++ b/pkg/pipeline/test.go @@ -30,7 +30,7 @@ func SendMessageToNode4Test(ctx context.Context, node Node, msgs []Message, outp return Message{}, nil } -// MockNodeContext4Test creates a node context with a message and a output channel for tests. +// MockNodeContext4Test creates a node context with a message and an output channel for tests. func MockNodeContext4Test(ctx context.Context, msg Message, outputCh chan Message) NodeContext { return NewNodeContext(ctx, msg, outputCh) } From 600f3893899d85153df56646a3204400a1c96882 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Thu, 13 Jan 2022 17:53:42 +0800 Subject: [PATCH 44/48] cdc/sink: adjust kafka initialization logic (#3192) (#4162) --- cdc/sink/mq.go | 10 +- cdc/sink/mq_test.go | 12 ++ cdc/sink/producer/kafka/kafka.go | 239 +++++++++++++++++++------- cdc/sink/producer/kafka/kafka_test.go | 101 ++++++----- 4 files changed, 250 insertions(+), 112 deletions(-) diff --git a/cdc/sink/mq.go b/cdc/sink/mq.go index 7b00fb707b2..35fbbf11646 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -382,7 +382,11 @@ func newKafkaSaramaSink(ctx context.Context, sinkURI *url.URL, filter *filter.Fi topic := strings.TrimFunc(sinkURI.Path, func(r rune) bool { return r == '/' }) - producer, err := kafka.NewKafkaSaramaProducer(ctx, sinkURI.Host, topic, config, errCh) + if topic == "" { + return nil, cerror.ErrKafkaInvalidConfig.GenWithStack("no topic is specified in sink-uri") + } + + producer, err := kafka.NewKafkaSaramaProducer(ctx, topic, config, errCh) if err != nil { return nil, errors.Trace(err) } @@ -412,8 +416,8 @@ func newPulsarSink(ctx context.Context, sinkURI *url.URL, filter *filter.Filter, if s != "" { opts["max-batch-size"] = s } - // For now, it's a place holder. Avro format have to make connection to Schema Registery, - // and it may needs credential. + // For now, it's a placeholder. Avro format have to make connection to Schema Registry, + // and it may need credential. credential := &security.Credential{} sink, err := newMqSink(ctx, credential, producer, filter, replicaConfig, opts, errCh) if err != nil { diff --git a/cdc/sink/mq_test.go b/cdc/sink/mq_test.go index 4a2a6730138..3c9b2fc961c 100644 --- a/cdc/sink/mq_test.go +++ b/cdc/sink/mq_test.go @@ -62,6 +62,12 @@ func (s mqSinkSuite) TestKafkaSink(c *check.C) { c.Assert(err, check.IsNil) opts := map[string]string{} errCh := make(chan error, 1) + + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/cdc/sink/producer/kafka/SkipTopicAutoCreate", "return(true)"), check.IsNil) + defer func() { + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sink/producer/kafka/SkipTopicAutoCreate") + }() + sink, err := newKafkaSaramaSink(ctx, sinkURI, fr, replicaConfig, opts, errCh) c.Assert(err, check.IsNil) @@ -163,6 +169,12 @@ func (s mqSinkSuite) TestKafkaSinkFilter(c *check.C) { c.Assert(err, check.IsNil) opts := map[string]string{} errCh := make(chan error, 1) + + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/cdc/sink/producer/kafka/SkipTopicAutoCreate", "return(true)"), check.IsNil) + defer func() { + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sink/producer/kafka/SkipTopicAutoCreate") + }() + sink, err := newKafkaSaramaSink(ctx, sinkURI, fr, replicaConfig, opts, errCh) c.Assert(err, check.IsNil) diff --git a/cdc/sink/producer/kafka/kafka.go b/cdc/sink/producer/kafka/kafka.go index 9b9fc650992..4b4750b7a7a 100644 --- a/cdc/sink/producer/kafka/kafka.go +++ b/cdc/sink/producer/kafka/kafka.go @@ -37,11 +37,14 @@ import ( "go.uber.org/zap" ) -const defaultPartitionNum = 4 +const defaultPartitionNum = 3 -// Config stores the Kafka configuration +// Config stores user specified Kafka producer configuration type Config struct { - PartitionNum int32 + BrokerEndpoints []string + PartitionNum int32 + + // User should make sure that `replication-factor` not greater than the number of kafka brokers. ReplicationFactor int16 Version string @@ -50,8 +53,8 @@ type Config struct { ClientID string Credential *security.Credential SaslScram *security.SaslScram - // control whether to create topic and verify partition number - TopicPreProcess bool + // control whether to create topic + AutoCreate bool } // NewConfig returns a default Kafka configuration @@ -64,19 +67,24 @@ func NewConfig() *Config { Compression: "none", Credential: &security.Credential{}, SaslScram: &security.SaslScram{}, - TopicPreProcess: true, + AutoCreate: true, } } // Initialize the kafka configuration func (c *Config) Initialize(sinkURI *url.URL, replicaConfig *config.ReplicaConfig, opts map[string]string) error { - s := sinkURI.Query().Get("partition-num") + c.BrokerEndpoints = strings.Split(sinkURI.Host, ",") + params := sinkURI.Query() + s := params.Get("partition-num") if s != "" { a, err := strconv.Atoi(s) if err != nil { return err } c.PartitionNum = int32(a) + if c.PartitionNum <= 0 { + return cerror.ErrKafkaInvalidPartitionNum.GenWithStackByArgs(c.PartitionNum) + } } s = sinkURI.Query().Get("replication-factor") @@ -156,7 +164,7 @@ func (c *Config) Initialize(sinkURI *url.URL, replicaConfig *config.ReplicaConfi if err != nil { return err } - c.TopicPreProcess = autoCreate + c.AutoCreate = autoCreate } return nil @@ -379,85 +387,122 @@ func (k *kafkaSaramaProducer) run(ctx context.Context) error { } } -// kafkaTopicPreProcess gets partition number from existing topic, if topic doesn't -// exit, creates it automatically. -func kafkaTopicPreProcess(topic, address string, config *Config, cfg *sarama.Config) (int32, error) { - admin, err := sarama.NewClusterAdmin(strings.Split(address, ","), cfg) +func topicPreProcess(topic string, config *Config, saramaConfig *sarama.Config) error { + // FIXME: find a way to remove this failpoint for workload the unit test + failpoint.Inject("SkipTopicAutoCreate", func() { + failpoint.Return(nil) + }) + admin, err := sarama.NewClusterAdmin(config.BrokerEndpoints, saramaConfig) if err != nil { - return 0, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) + return cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) } defer func() { - err := admin.Close() - if err != nil { - log.Warn("close admin client failed", zap.Error(err)) + if err := admin.Close(); err != nil { + log.Warn("close kafka cluster admin failed", zap.Error(err)) } }() + topics, err := admin.ListTopics() if err != nil { - return 0, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) + return cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) } - partitionNum := config.PartitionNum - topicDetail, exist := topics[topic] - if exist { - log.Info("get partition number of topic", zap.String("topic", topic), zap.Int32("partition_num", topicDetail.NumPartitions)) - if partitionNum == 0 { - partitionNum = topicDetail.NumPartitions - } else if partitionNum < topicDetail.NumPartitions { - log.Warn("partition number assigned in sink-uri is less than that of topic", zap.Int32("topic partition num", topicDetail.NumPartitions)) - } else if partitionNum > topicDetail.NumPartitions { - return 0, cerror.ErrKafkaInvalidPartitionNum.GenWithStack( - "partition number(%d) assigned in sink-uri is more than that of topic(%d)", partitionNum, topicDetail.NumPartitions) + + info, created := topics[topic] + // once we have found the topic, no matter `auto-create-topic`, make sure user input parameters are valid. + if created { + // make sure that topic's `max.message.bytes` is not less than given `max-message-bytes` + // else the producer will send message that too large to make topic reject, then changefeed would error. + // only the default `open protocol` and `craft protocol` use `max-message-bytes`, so check this for them. + topicMaxMessageBytes, err := getTopicMaxMessageBytes(admin, info) + if err != nil { + return cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) } - } else { - if partitionNum == 0 { - partitionNum = defaultPartitionNum - log.Warn("topic not found and partition number is not specified, using default partition number", zap.String("topic", topic), zap.Int32("partition_num", partitionNum)) + if topicMaxMessageBytes < config.MaxMessageBytes { + return cerror.ErrKafkaInvalidConfig.GenWithStack( + "topic already exist, and topic's max.message.bytes(%d) less than max-message-bytes(%d)."+ + "Please make sure `max-message-bytes` not greater than topic `max.message.bytes`", + topicMaxMessageBytes, config.MaxMessageBytes) } - log.Info("create a topic", zap.String("topic", topic), - zap.Int32("partition_num", partitionNum), - zap.Int16("replication_factor", config.ReplicationFactor)) - err := admin.CreateTopic(topic, &sarama.TopicDetail{ - NumPartitions: partitionNum, - ReplicationFactor: config.ReplicationFactor, - }, false) - // TODO idenfity the cause of "Topic with this name already exists" - if err != nil && !strings.Contains(err.Error(), "already exists") { - return 0, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) + + // no need to create the topic, but we would have to log user if they found enter wrong topic name later + if config.AutoCreate { + log.Warn("topic already exist, TiCDC will not create the topic", + zap.String("topic", topic), zap.Any("detail", info)) + } + + if err := config.adjustPartitionNum(info.NumPartitions); err != nil { + return errors.Trace(err) } + + return nil + } + + if !config.AutoCreate { + return cerror.ErrKafkaInvalidConfig.GenWithStack("`auto-create-topic` is false, and topic not found") + } + + // when try to create the topic, we don't know how to set the `max.message.bytes` for the topic. + // Kafka would create the topic with broker's `message.max.bytes`, + // we have to make sure it's not greater than `max-message-bytes` + brokerMessageMaxBytes, err := getBrokerMessageMaxBytes(admin) + if err != nil { + log.Warn("TiCDC cannot find `message.max.bytes` from broker's configuration") + return errors.Trace(err) + } + + if brokerMessageMaxBytes < config.MaxMessageBytes { + return cerror.ErrKafkaInvalidConfig.GenWithStack( + "broker's message.max.bytes(%d) less than max-message-bytes(%d)"+ + "Please make sure `max-message-bytes` not greater than broker's `message.max.bytes`", + brokerMessageMaxBytes, config.MaxMessageBytes) + } + + // topic not created yet, and user does not specify the `partition-num` in the sink uri. + if config.PartitionNum == 0 { + config.PartitionNum = defaultPartitionNum + log.Warn("partition-num is not set, use the default partition count", + zap.String("topic", topic), zap.Int32("partitions", config.PartitionNum)) + } + + err = admin.CreateTopic(topic, &sarama.TopicDetail{ + NumPartitions: config.PartitionNum, + ReplicationFactor: config.ReplicationFactor, + }, false) + // TODO identify the cause of "Topic with this name already exists" + if err != nil && !strings.Contains(err.Error(), "already exists") { + return cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) } - return partitionNum, nil + log.Info("TiCDC create the topic", + zap.Int32("partition-num", config.PartitionNum), + zap.Int16("replication-factor", config.ReplicationFactor)) + + return nil } var newSaramaConfigImpl = newSaramaConfig // NewKafkaSaramaProducer creates a kafka sarama producer -func NewKafkaSaramaProducer(ctx context.Context, address string, topic string, config *Config, errCh chan error) (*kafkaSaramaProducer, error) { +func NewKafkaSaramaProducer(ctx context.Context, topic string, config *Config, errCh chan error) (*kafkaSaramaProducer, error) { log.Info("Starting kafka sarama producer ...", zap.Reflect("config", config)) cfg, err := newSaramaConfigImpl(ctx, config) if err != nil { return nil, err } - if config.PartitionNum < 0 { - return nil, cerror.ErrKafkaInvalidPartitionNum.GenWithStackByArgs(config.PartitionNum) + + if err := topicPreProcess(topic, config, cfg); err != nil { + return nil, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) } - asyncClient, err := sarama.NewAsyncProducer(strings.Split(address, ","), cfg) + + asyncClient, err := sarama.NewAsyncProducer(config.BrokerEndpoints, cfg) if err != nil { return nil, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) } - syncClient, err := sarama.NewSyncProducer(strings.Split(address, ","), cfg) + syncClient, err := sarama.NewSyncProducer(config.BrokerEndpoints, cfg) if err != nil { return nil, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) } - partitionNum := config.PartitionNum - if config.TopicPreProcess { - partitionNum, err = kafkaTopicPreProcess(topic, address, config, cfg) - if err != nil { - return nil, err - } - } - notifier := new(notify.Notifier) flushedReceiver, err := notifier.NewReceiver(50 * time.Millisecond) if err != nil { @@ -467,11 +512,11 @@ func NewKafkaSaramaProducer(ctx context.Context, address string, topic string, c asyncClient: asyncClient, syncClient: syncClient, topic: topic, - partitionNum: partitionNum, + partitionNum: config.PartitionNum, partitionOffset: make([]struct { flushed uint64 sent uint64 - }, partitionNum), + }, config.PartitionNum), flushedNotifier: notifier, flushedReceiver: flushedReceiver, closeCh: make(chan struct{}), @@ -558,6 +603,11 @@ func newSaramaConfig(ctx context.Context, c *Config) (*sarama.Config, error) { config.Producer.Return.Successes = true config.Producer.Return.Errors = true config.Producer.RequiredAcks = sarama.WaitForAll + + // Time out in five minutes(600 * 500ms). + config.Producer.Retry.Max = 600 + config.Producer.Retry.Backoff = 500 * time.Millisecond + switch strings.ToLower(strings.TrimSpace(c.Compression)) { case "none": config.Producer.Compression = sarama.CompressionNone @@ -574,10 +624,6 @@ func newSaramaConfig(ctx context.Context, c *Config) (*sarama.Config, error) { config.Producer.Compression = sarama.CompressionNone } - // Time out in five minutes(600 * 500ms). - config.Producer.Retry.Max = 600 - config.Producer.Retry.Backoff = 500 * time.Millisecond - // Time out in one minute(120 * 500ms). config.Admin.Retry.Max = 120 config.Admin.Retry.Backoff = 500 * time.Millisecond @@ -606,3 +652,72 @@ func newSaramaConfig(ctx context.Context, c *Config) (*sarama.Config, error) { return config, err } + +func getBrokerMessageMaxBytes(admin sarama.ClusterAdmin) (int, error) { + target := "message.max.bytes" + _, controllerID, err := admin.DescribeCluster() + if err != nil { + return 0, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) + } + + configEntries, err := admin.DescribeConfig(sarama.ConfigResource{ + Type: sarama.BrokerResource, + Name: strconv.Itoa(int(controllerID)), + ConfigNames: []string{target}, + }) + if err != nil { + return 0, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) + } + + if len(configEntries) == 0 || configEntries[0].Name != target { + return 0, cerror.ErrKafkaNewSaramaProducer.GenWithStack( + "since cannot find the `message.max.bytes` from the broker's configuration, " + + "ticdc decline to create the topic and changefeed to prevent potential error") + } + + result, err := strconv.Atoi(configEntries[0].Value) + if err != nil { + return 0, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) + } + + return result, nil +} + +func getTopicMaxMessageBytes(admin sarama.ClusterAdmin, info sarama.TopicDetail) (int, error) { + if a, ok := info.ConfigEntries["max.message.bytes"]; ok { + result, err := strconv.Atoi(*a) + if err != nil { + return 0, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) + } + return result, nil + } + + return getBrokerMessageMaxBytes(admin) +} + +// adjust the partition-num by the topic's partition count +func (c *Config) adjustPartitionNum(realPartitionCount int32) error { + // user does not specify the `partition-num` in the sink-uri + if c.PartitionNum == 0 { + c.PartitionNum = realPartitionCount + return nil + } + + if c.PartitionNum < realPartitionCount { + log.Warn("number of partition specified in sink-uri is less than that of the actual topic. "+ + "Some partitions will not have messages dispatched to", + zap.Int32("sink-uri partitions", c.PartitionNum), + zap.Int32("topic partitions", realPartitionCount)) + return nil + } + + // Make sure that the user-specified `partition-num` is not greater than + // the real partition count, since messages would be dispatched to different + // partitions, this could prevent potential correctness problems. + if c.PartitionNum > realPartitionCount { + return cerror.ErrKafkaInvalidPartitionNum.GenWithStack( + "the number of partition (%d) specified in sink-uri is more than that of actual topic (%d)", + c.PartitionNum, realPartitionCount) + } + return nil +} diff --git a/cdc/sink/producer/kafka/kafka_test.go b/cdc/sink/producer/kafka/kafka_test.go index 2206b96d582..8f0c1b40d33 100644 --- a/cdc/sink/producer/kafka/kafka_test.go +++ b/cdc/sink/producer/kafka/kafka_test.go @@ -17,6 +17,7 @@ import ( "context" "fmt" "net/url" + "strings" "sync" "testing" "time" @@ -24,6 +25,7 @@ import ( "github.com/Shopify/sarama" "github.com/pingcap/check" "github.com/pingcap/errors" + "github.com/pingcap/failpoint" "github.com/pingcap/tiflow/cdc/sink/codec" "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" @@ -96,6 +98,12 @@ func (s *kafkaSuite) TestInitializeConfig(c *check.C) { for k, v := range opts { c.Assert(v, check.Equals, expectedOpts[k]) } + + uri = "kafka://127.0.0.1:9092/abc?kafka-version=2.6.0&partition-num=0" + sinkURI, err = url.Parse(uri) + c.Assert(err, check.IsNil) + err = cfg.Initialize(sinkURI, replicaConfig, opts) + c.Assert(errors.Cause(err), check.ErrorMatches, ".*invalid partition num.*") } func (s *kafkaSuite) TestSaramaProducer(c *check.C) { @@ -116,7 +124,7 @@ func (s *kafkaSuite) TestSaramaProducer(c *check.C) { prodSuccess.AddTopicPartition(topic, 0, sarama.ErrNoError) prodSuccess.AddTopicPartition(topic, 1, sarama.ErrNoError) // 200 async messages and 2 sync message, Kafka flush could be in batch, - // we can set flush.maxmessages to 1 to control message count exactly. + // we can set flush.max.messages to 1 to control message count exactly. for i := 0; i < 202; i++ { leader.Returns(prodSuccess) } @@ -128,7 +136,8 @@ func (s *kafkaSuite) TestSaramaProducer(c *check.C) { // Ref: https://github.com/Shopify/sarama/blob/89707055369768913defac030c15cf08e9e57925/async_producer_test.go#L1445-L1447 config.Version = "0.9.0.0" config.PartitionNum = int32(2) - config.TopicPreProcess = false + config.AutoCreate = false + config.BrokerEndpoints = strings.Split(leader.Addr(), ",") newSaramaConfigImplBak := newSaramaConfigImpl newSaramaConfigImpl = func(ctx context.Context, config *Config) (*sarama.Config, error) { @@ -137,11 +146,13 @@ func (s *kafkaSuite) TestSaramaProducer(c *check.C) { cfg.Producer.Flush.MaxMessages = 1 return cfg, err } + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/cdc/sink/producer/kafka/SkipTopicAutoCreate", "return(true)"), check.IsNil) defer func() { newSaramaConfigImpl = newSaramaConfigImplBak + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sink/producer/kafka/SkipTopicAutoCreate") }() - producer, err := NewKafkaSaramaProducer(ctx, leader.Addr(), topic, config, errCh) + producer, err := NewKafkaSaramaProducer(ctx, topic, config, errCh) c.Assert(err, check.IsNil) c.Assert(producer.GetPartitionNum(), check.Equals, int32(2)) for i := 0; i < 100; i++ { @@ -223,6 +234,23 @@ func (s *kafkaSuite) TestSaramaProducer(c *check.C) { } } +func (s *kafkaSuite) TestAdjustPartitionNum(c *check.C) { + defer testleak.AfterTest(c)() + config := NewConfig() + err := config.adjustPartitionNum(2) + c.Assert(err, check.IsNil) + c.Assert(config.PartitionNum, check.Equals, int32(2)) + + config.PartitionNum = 1 + err = config.adjustPartitionNum(2) + c.Assert(err, check.IsNil) + c.Assert(config.PartitionNum, check.Equals, int32(1)) + + config.PartitionNum = 3 + err = config.adjustPartitionNum(2) + c.Assert(cerror.ErrKafkaInvalidPartitionNum.Equal(err), check.IsTrue) +} + func (s *kafkaSuite) TestTopicPreProcess(c *check.C) { defer testleak.AfterTest(c) topic := "unit_test_2" @@ -240,47 +268,19 @@ func (s *kafkaSuite) TestTopicPreProcess(c *check.C) { "MetadataRequest": metaResponse, "DescribeConfigsRequest": sarama.NewMockDescribeConfigsResponse(c), }) - config := NewConfig() config.PartitionNum = int32(0) + config.BrokerEndpoints = strings.Split(broker.Addr(), ",") + config.AutoCreate = false + cfg, err := newSaramaConfigImpl(ctx, config) c.Assert(err, check.IsNil) - num, err := kafkaTopicPreProcess(topic, broker.Addr(), config, cfg) - c.Assert(err, check.IsNil) - c.Assert(num, check.Equals, int32(2)) + config.BrokerEndpoints = []string{""} cfg.Metadata.Retry.Max = 1 - _, err = kafkaTopicPreProcess(topic, "", config, cfg) - c.Assert(errors.Cause(err), check.Equals, sarama.ErrOutOfBrokers) - config.PartitionNum = int32(4) - _, err = kafkaTopicPreProcess(topic, broker.Addr(), config, cfg) - c.Assert(cerror.ErrKafkaInvalidPartitionNum.Equal(err), check.IsTrue) -} - -func (s *kafkaSuite) TestTopicPreProcessCreate(c *check.C) { - defer testleak.AfterTest(c)() - topic := "unit_test_3" - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - broker := sarama.NewMockBroker(c, 1) - broker.SetHandlerByMap(map[string]sarama.MockResponse{ - "MetadataRequest": sarama.NewMockMetadataResponse(c). - SetBroker(broker.Addr(), broker.BrokerID()). - SetController(broker.BrokerID()), - "DescribeConfigsRequest": sarama.NewMockDescribeConfigsResponse(c), - "CreateTopicsRequest": sarama.NewMockCreateTopicsResponse(c), - }) - defer broker.Close() - - config := NewConfig() - config.PartitionNum = int32(0) - cfg, err := newSaramaConfigImpl(ctx, config) - c.Assert(err, check.IsNil) - num, err := kafkaTopicPreProcess(topic, broker.Addr(), config, cfg) - c.Assert(err, check.IsNil) - c.Assert(num, check.Equals, int32(4)) + err = topicPreProcess(topic, config, cfg) + c.Assert(errors.Cause(err), check.Equals, sarama.ErrOutOfBrokers) } func (s *kafkaSuite) TestNewSaramaConfig(c *check.C) { @@ -345,13 +345,12 @@ func (s *kafkaSuite) TestCreateProducerFailed(c *check.C) { errCh := make(chan error, 1) config := NewConfig() config.Version = "invalid" - _, err := NewKafkaSaramaProducer(ctx, "127.0.0.1:1111", "topic", config, errCh) + config.BrokerEndpoints = []string{"127.0.0.1:1111"} + topic := "topic" + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/cdc/sink/producer/kafka/SkipTopicAutoCreate", "return(true)"), check.IsNil) + _, err := NewKafkaSaramaProducer(ctx, topic, config, errCh) c.Assert(errors.Cause(err), check.ErrorMatches, "invalid version.*") - - config.Version = "0.8.2.0" - config.PartitionNum = int32(-1) - _, err = NewKafkaSaramaProducer(ctx, "127.0.0.1:1111", "topic", config, errCh) - c.Assert(cerror.ErrKafkaInvalidPartitionNum.Equal(err), check.IsTrue) + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sink/producer/kafka/SkipTopicAutoCreate") } func (s *kafkaSuite) TestProducerSendMessageFailed(c *check.C) { @@ -375,7 +374,10 @@ func (s *kafkaSuite) TestProducerSendMessageFailed(c *check.C) { // Ref: https://github.com/Shopify/sarama/blob/89707055369768913defac030c15cf08e9e57925/async_producer_test.go#L1445-L1447 config.Version = "0.9.0.0" config.PartitionNum = int32(2) - config.TopicPreProcess = false + config.AutoCreate = false + config.BrokerEndpoints = strings.Split(leader.Addr(), ",") + + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/cdc/sink/producer/kafka/SkipTopicAutoCreate", "return(true)"), check.IsNil) newSaramaConfigImplBak := newSaramaConfigImpl newSaramaConfigImpl = func(ctx context.Context, config *Config) (*sarama.Config, error) { @@ -391,8 +393,9 @@ func (s *kafkaSuite) TestProducerSendMessageFailed(c *check.C) { }() errCh := make(chan error, 1) - producer, err := NewKafkaSaramaProducer(ctx, leader.Addr(), topic, config, errCh) + producer, err := NewKafkaSaramaProducer(ctx, topic, config, errCh) defer func() { + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sink/producer/kafka/SkipTopicAutoCreate") err := producer.Close() c.Assert(err, check.IsNil) }() @@ -449,13 +452,17 @@ func (s *kafkaSuite) TestProducerDoubleClose(c *check.C) { // Ref: https://github.com/Shopify/sarama/blob/89707055369768913defac030c15cf08e9e57925/async_producer_test.go#L1445-L1447 config.Version = "0.9.0.0" config.PartitionNum = int32(2) - config.TopicPreProcess = false + config.AutoCreate = false + config.BrokerEndpoints = strings.Split(leader.Addr(), ",") + + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/cdc/sink/producer/kafka/SkipTopicAutoCreate", "return(true)"), check.IsNil) errCh := make(chan error, 1) - producer, err := NewKafkaSaramaProducer(ctx, leader.Addr(), topic, config, errCh) + producer, err := NewKafkaSaramaProducer(ctx, topic, config, errCh) defer func() { err := producer.Close() c.Assert(err, check.IsNil) + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/sink/producer/kafka/SkipTopicAutoCreate") }() c.Assert(err, check.IsNil) From 16f5f42291cc43172a56296483933d4c87ff11d9 Mon Sep 17 00:00:00 2001 From: 3AceShowHand Date: Thu, 13 Jan 2022 18:48:56 +0800 Subject: [PATCH 45/48] try fix conflicts. --- cdc/sink/codec/canal.go | 4 ---- cdc/sink/codec/canal_flat.go | 6 ------ cdc/sink/codec/maxwell.go | 4 ---- 3 files changed, 14 deletions(-) diff --git a/cdc/sink/codec/canal.go b/cdc/sink/codec/canal.go index d2dcc5bae74..bdd73e272fd 100644 --- a/cdc/sink/codec/canal.go +++ b/cdc/sink/codec/canal.go @@ -392,12 +392,8 @@ func (d *CanalEventBatchEncoder) Build() []*MQMessage { if err != nil { log.Panic("Error when serializing Canal packet", zap.Error(err)) } -<<<<<<< HEAD ret := NewMQMessage(ProtocolCanal, nil, value, 0, model.MqMessageTypeRow, nil, nil) -======= - ret := NewMQMessage(config.ProtocolCanal, nil, value, 0, model.MqMessageTypeRow, nil, nil) ret.SetRowsCount(rowCount) ->>>>>>> fc70dbde8 (metrics(cdc): fix mq sink write row count metrics. (#4192)) d.messages.Reset() d.resetPacket() return []*MQMessage{ret} diff --git a/cdc/sink/codec/canal_flat.go b/cdc/sink/codec/canal_flat.go index d663d9497a1..93579556ecd 100644 --- a/cdc/sink/codec/canal_flat.go +++ b/cdc/sink/codec/canal_flat.go @@ -237,13 +237,7 @@ func (c *CanalFlatEventBatchEncoder) Build() []*MQMessage { log.Panic("CanalFlatEventBatchEncoder", zap.Error(err)) return nil } -<<<<<<< HEAD ret[i] = NewMQMessage(ProtocolCanalJSON, nil, value, msg.tikvTs, model.MqMessageTypeRow, &msg.Schema, &msg.Table) -======= - m := NewMQMessage(config.ProtocolCanalJSON, nil, value, msg.getTikvTs(), model.MqMessageTypeRow, msg.getSchema(), msg.getTable()) - m.IncRowsCount() - ret[i] = m ->>>>>>> fc70dbde8 (metrics(cdc): fix mq sink write row count metrics. (#4192)) } c.resolvedBuf = c.resolvedBuf[0:0] return ret diff --git a/cdc/sink/codec/maxwell.go b/cdc/sink/codec/maxwell.go index 5d5467bebf2..0a67cb939e9 100644 --- a/cdc/sink/codec/maxwell.go +++ b/cdc/sink/codec/maxwell.go @@ -296,12 +296,8 @@ func (d *MaxwellEventBatchEncoder) Build() []*MQMessage { return nil } -<<<<<<< HEAD ret := NewMQMessage(ProtocolMaxwell, d.keyBuf.Bytes(), d.valueBuf.Bytes(), 0, model.MqMessageTypeRow, nil, nil) -======= - ret := NewMQMessage(config.ProtocolMaxwell, d.keyBuf.Bytes(), d.valueBuf.Bytes(), 0, model.MqMessageTypeRow, nil, nil) ret.SetRowsCount(d.batchSize) ->>>>>>> fc70dbde8 (metrics(cdc): fix mq sink write row count metrics. (#4192)) d.Reset() return []*MQMessage{ret} } From 62dd14017fb0ad85596e7bb1102490f400f48c7d Mon Sep 17 00:00:00 2001 From: Ling Jin <7138436+3AceShowHand@users.noreply.github.com> Date: Tue, 4 Jan 2022 14:14:35 +0800 Subject: [PATCH 46/48] This is an automated cherry-pick of #4192 Signed-off-by: ti-chi-bot --- cdc/sink/codec/avro.go | 1 + cdc/sink/codec/canal.go | 8 +++++++- cdc/sink/codec/canal_flat.go | 6 ++++++ cdc/sink/codec/canal_flat_test.go | 2 ++ cdc/sink/codec/canal_test.go | 1 + cdc/sink/codec/json_test.go | 1 + cdc/sink/codec/maxwell.go | 5 +++++ cdc/sink/codec/maxwell_test.go | 1 + 8 files changed, 24 insertions(+), 1 deletion(-) diff --git a/cdc/sink/codec/avro.go b/cdc/sink/codec/avro.go index cb6ed9e9952..01a5170dfd6 100644 --- a/cdc/sink/codec/avro.go +++ b/cdc/sink/codec/avro.go @@ -122,6 +122,7 @@ func (a *AvroEventBatchEncoder) AppendRowChangedEvent(e *model.RowChangedEvent) } mqMessage.Key = evlp + mqMessage.IncRowsCount() a.resultBuf = append(a.resultBuf, mqMessage) return EncoderNeedAsyncWrite, nil diff --git a/cdc/sink/codec/canal.go b/cdc/sink/codec/canal.go index 089103d2f8d..d2dcc5bae74 100644 --- a/cdc/sink/codec/canal.go +++ b/cdc/sink/codec/canal.go @@ -378,7 +378,8 @@ func (d *CanalEventBatchEncoder) EncodeDDLEvent(e *model.DDLEvent) (*MQMessage, // Build implements the EventBatchEncoder interface func (d *CanalEventBatchEncoder) Build() []*MQMessage { - if len(d.messages.Messages) == 0 { + rowCount := len(d.messages.Messages) + if rowCount == 0 { return nil } @@ -391,7 +392,12 @@ func (d *CanalEventBatchEncoder) Build() []*MQMessage { if err != nil { log.Panic("Error when serializing Canal packet", zap.Error(err)) } +<<<<<<< HEAD ret := NewMQMessage(ProtocolCanal, nil, value, 0, model.MqMessageTypeRow, nil, nil) +======= + ret := NewMQMessage(config.ProtocolCanal, nil, value, 0, model.MqMessageTypeRow, nil, nil) + ret.SetRowsCount(rowCount) +>>>>>>> fc70dbde8 (metrics(cdc): fix mq sink write row count metrics. (#4192)) d.messages.Reset() d.resetPacket() return []*MQMessage{ret} diff --git a/cdc/sink/codec/canal_flat.go b/cdc/sink/codec/canal_flat.go index 93579556ecd..d663d9497a1 100644 --- a/cdc/sink/codec/canal_flat.go +++ b/cdc/sink/codec/canal_flat.go @@ -237,7 +237,13 @@ func (c *CanalFlatEventBatchEncoder) Build() []*MQMessage { log.Panic("CanalFlatEventBatchEncoder", zap.Error(err)) return nil } +<<<<<<< HEAD ret[i] = NewMQMessage(ProtocolCanalJSON, nil, value, msg.tikvTs, model.MqMessageTypeRow, &msg.Schema, &msg.Table) +======= + m := NewMQMessage(config.ProtocolCanalJSON, nil, value, msg.getTikvTs(), model.MqMessageTypeRow, msg.getSchema(), msg.getTable()) + m.IncRowsCount() + ret[i] = m +>>>>>>> fc70dbde8 (metrics(cdc): fix mq sink write row count metrics. (#4192)) } c.resolvedBuf = c.resolvedBuf[0:0] return ret diff --git a/cdc/sink/codec/canal_flat_test.go b/cdc/sink/codec/canal_flat_test.go index 0d6cbb7531b..64f371fe235 100644 --- a/cdc/sink/codec/canal_flat_test.go +++ b/cdc/sink/codec/canal_flat_test.go @@ -123,6 +123,8 @@ func (s *canalFlatSuite) TestBatching(c *check.C) { c.Assert(msgs, check.HasLen, int(resolvedTs-lastResolved)) for j := range msgs { + c.Assert(msgs[j].GetRowsCount(), check.Equals, 1) + var msg canalFlatMessage err := json.Unmarshal(msgs[j].Value, &msg) c.Assert(err, check.IsNil) diff --git a/cdc/sink/codec/canal_test.go b/cdc/sink/codec/canal_test.go index bf8f0fa113d..d3755d1a2df 100644 --- a/cdc/sink/codec/canal_test.go +++ b/cdc/sink/codec/canal_test.go @@ -102,6 +102,7 @@ func (s *canalBatchSuite) TestCanalEventBatchEncoder(c *check.C) { c.Assert(res, check.HasLen, 1) c.Assert(res[0].Key, check.IsNil) c.Assert(len(res[0].Value), check.Equals, size) + c.Assert(res[0].GetRowsCount(), check.Equals, len(cs)) packet := &canal.Packet{} err := proto.Unmarshal(res[0].Value, packet) diff --git a/cdc/sink/codec/json_test.go b/cdc/sink/codec/json_test.go index 7f02c564904..d1d2646e142 100644 --- a/cdc/sink/codec/json_test.go +++ b/cdc/sink/codec/json_test.go @@ -134,6 +134,7 @@ func (s *batchSuite) testBatchCodec(c *check.C, newEncoder func() EventBatchEnco if len(cs) > 0 { res := encoder.Build() c.Assert(res, check.HasLen, 1) + c.Assert(res[0].GetRowsCount(), check.Equals, len(cs)) decoder, err := newDecoder(res[0].Key, res[0].Value) c.Assert(err, check.IsNil) checkRowDecoder(decoder, cs) diff --git a/cdc/sink/codec/maxwell.go b/cdc/sink/codec/maxwell.go index d6e54c72448..5d5467bebf2 100644 --- a/cdc/sink/codec/maxwell.go +++ b/cdc/sink/codec/maxwell.go @@ -296,7 +296,12 @@ func (d *MaxwellEventBatchEncoder) Build() []*MQMessage { return nil } +<<<<<<< HEAD ret := NewMQMessage(ProtocolMaxwell, d.keyBuf.Bytes(), d.valueBuf.Bytes(), 0, model.MqMessageTypeRow, nil, nil) +======= + ret := NewMQMessage(config.ProtocolMaxwell, d.keyBuf.Bytes(), d.valueBuf.Bytes(), 0, model.MqMessageTypeRow, nil, nil) + ret.SetRowsCount(d.batchSize) +>>>>>>> fc70dbde8 (metrics(cdc): fix mq sink write row count metrics. (#4192)) d.Reset() return []*MQMessage{ret} } diff --git a/cdc/sink/codec/maxwell_test.go b/cdc/sink/codec/maxwell_test.go index 5e6fea4a58e..1f98bedac44 100644 --- a/cdc/sink/codec/maxwell_test.go +++ b/cdc/sink/codec/maxwell_test.go @@ -54,6 +54,7 @@ func (s *maxwellbatchSuite) testmaxwellBatchCodec(c *check.C, newEncoder func() continue } c.Assert(messages, check.HasLen, 1) + c.Assert(messages[0].GetRowsCount(), check.Equals, len(cs)) c.Assert(len(messages[0].Key)+len(messages[0].Value), check.Equals, size) } From 6f742e676c7b4c9ccc093fa340918189c1bc90d2 Mon Sep 17 00:00:00 2001 From: 3AceShowHand Date: Tue, 18 Jan 2022 11:03:45 +0800 Subject: [PATCH 47/48] fix conflicts. --- cdc/sink/codec/craft.go | 2 +- cdc/sink/codec/interface.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/cdc/sink/codec/craft.go b/cdc/sink/codec/craft.go index 8cae661b434..fdf77ba1a01 100644 --- a/cdc/sink/codec/craft.go +++ b/cdc/sink/codec/craft.go @@ -47,7 +47,7 @@ func (e *CraftEventBatchEncoder) flush() { schema := headers.GetSchema(0) table := headers.GetTable(0) rowsCnt := e.rowChangedBuffer.RowsCount() - mqMessage := NewMQMessage(config.ProtocolCraft, nil, e.rowChangedBuffer.Encode(), ts, model.MqMessageTypeRow, &schema, &table) + mqMessage := NewMQMessage(ProtocolCraft, nil, e.rowChangedBuffer.Encode(), ts, model.MqMessageTypeRow, &schema, &table) mqMessage.SetRowsCount(rowsCnt) e.messageBuf = append(e.messageBuf, mqMessage) } diff --git a/cdc/sink/codec/interface.go b/cdc/sink/codec/interface.go index 3c017ae86f5..06b89fb0e44 100644 --- a/cdc/sink/codec/interface.go +++ b/cdc/sink/codec/interface.go @@ -63,7 +63,7 @@ type MQMessage struct { Schema *string // schema Table *string // table Type model.MqMessageType // type - Protocol config.Protocol // protocol + Protocol Protocol // protocol rowsCount int // rows in one MQ Message } From 7ee6dbafc29d9dc4a8a5d708db19cffb3e9f9935 Mon Sep 17 00:00:00 2001 From: 3AceShowHand Date: Tue, 18 Jan 2022 11:08:10 +0800 Subject: [PATCH 48/48] fix conflicts. --- cdc/sink/codec/maxwell.go | 8 -------- dm/tests/metrics/run.sh | 3 --- 2 files changed, 11 deletions(-) diff --git a/cdc/sink/codec/maxwell.go b/cdc/sink/codec/maxwell.go index cd8c97f1f7f..0a67cb939e9 100644 --- a/cdc/sink/codec/maxwell.go +++ b/cdc/sink/codec/maxwell.go @@ -296,16 +296,8 @@ func (d *MaxwellEventBatchEncoder) Build() []*MQMessage { return nil } -<<<<<<< HEAD ret := NewMQMessage(ProtocolMaxwell, d.keyBuf.Bytes(), d.valueBuf.Bytes(), 0, model.MqMessageTypeRow, nil, nil) -<<<<<<< HEAD ret.SetRowsCount(d.batchSize) -======= -======= - ret := NewMQMessage(config.ProtocolMaxwell, d.keyBuf.Bytes(), d.valueBuf.Bytes(), 0, model.MqMessageTypeRow, nil, nil) - ret.SetRowsCount(d.batchSize) ->>>>>>> fc70dbde8 (metrics(cdc): fix mq sink write row count metrics. (#4192)) ->>>>>>> b1b8182771865456f90a9399f504c114494c4189 d.Reset() return []*MQMessage{ret} } diff --git a/dm/tests/metrics/run.sh b/dm/tests/metrics/run.sh index 48132452fd6..9ecf992fc75 100755 --- a/dm/tests/metrics/run.sh +++ b/dm/tests/metrics/run.sh @@ -20,10 +20,7 @@ function run() { inject_points=( "github.com/pingcap/tiflow/dm/syncer/BlockDDLJob=return(1)" "github.com/pingcap/tiflow/dm/syncer/ShowLagInLog=return(1)" # test lag metric >= 1 beacuse we inject BlockDDLJob(ddl) to sleep(1) -<<<<<<< HEAD -======= "github.com/pingcap/tiflow/dm/dm/worker/PrintStatusCheckSeconds=return(1)" ->>>>>>> b1b8182771865456f90a9399f504c114494c4189 ) export GO_FAILPOINTS="$(join_string \; ${inject_points[@]})"