From def4d82d6c1736b5d314f09e4e3bc6ddc00fc388 Mon Sep 17 00:00:00 2001 From: zhaoxinyu Date: Wed, 24 Nov 2021 18:59:32 +0800 Subject: [PATCH 01/30] 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 0472edc5174..b4c68738d01 100644 --- a/cdc/sink/codec/interface.go +++ b/cdc/sink/codec/interface.go @@ -55,13 +55,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. @@ -93,13 +94,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 a16b39f0ecf..2fd69ff7e43 100644 --- a/cdc/sink/codec/json.go +++ b/cdc/sink/codec/json.go @@ -453,6 +453,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.maxKafkaMessageSize { // `len(d.messageBuf) == 1` is implied diff --git a/cdc/sink/mq.go b/cdc/sink/mq.go index a2a47f7fa66..f7fabea0b6f 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -302,8 +302,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 } @@ -312,6 +312,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 630ec41b9f8490c587a90835f1bf46a485eb10ac Mon Sep 17 00:00:00 2001 From: zhaoxinyu Date: Wed, 1 Dec 2021 18:16:22 +0800 Subject: [PATCH 02/30] 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 03a4cb75cd2..a78662a75dd 100644 --- a/cdc/sink/codec/craft.go +++ b/cdc/sink/codec/craft.go @@ -45,7 +45,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 b4c68738d01..73faad2feeb 100644 --- a/cdc/sink/codec/interface.go +++ b/cdc/sink/codec/interface.go @@ -62,7 +62,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. @@ -82,6 +82,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) } @@ -101,7 +114,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 2fd69ff7e43..8ae135a47a7 100644 --- a/cdc/sink/codec/json.go +++ b/cdc/sink/codec/json.go @@ -453,7 +453,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.maxKafkaMessageSize { // `len(d.messageBuf) == 1` is implied diff --git a/cdc/sink/mq.go b/cdc/sink/mq.go index f7fabea0b6f..5b8e838b9f9 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -312,7 +312,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 7e584b1d156bd754a1edd8323e10f7d57740e7f3 Mon Sep 17 00:00:00 2001 From: zhaoxinyu Date: Wed, 1 Dec 2021 18:33:50 +0800 Subject: [PATCH 03/30] 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 73faad2feeb..0ae248c441b 100644 --- a/cdc/sink/codec/interface.go +++ b/cdc/sink/codec/interface.go @@ -82,15 +82,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 c3c0ac54cf2cc9b0a67241231cbd1e0aaef330e6 Mon Sep 17 00:00:00 2001 From: zhaoxinyu Date: Thu, 9 Dec 2021 10:26:52 +0800 Subject: [PATCH 04/30] 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 a78662a75dd..bc03b9837ed 100644 --- a/cdc/sink/codec/craft.go +++ b/cdc/sink/codec/craft.go @@ -46,7 +46,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 0ae248c441b..ae4599ff6b7 100644 --- a/cdc/sink/codec/interface.go +++ b/cdc/sink/codec/interface.go @@ -61,7 +61,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 db3d42ed2439aaafa8575ea5dfebf257269bd824 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Fri, 10 Dec 2021 23:09:57 +0800 Subject: [PATCH 05/30] workerpool: limit the rate to output deadlock warning (#3775) (#3799) --- pkg/workerpool/pool_impl.go | 20 +++++++++++++++++--- pkg/workerpool/pool_test.go | 34 ++++++++++++++++++++++++++++++++++ 2 files changed, 51 insertions(+), 3 deletions(-) diff --git a/pkg/workerpool/pool_impl.go b/pkg/workerpool/pool_impl.go index 966e6ea35c3..d98d4ffc1a8 100644 --- a/pkg/workerpool/pool_impl.go +++ b/pkg/workerpool/pool_impl.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/ticdc/pkg/notify" "go.uber.org/zap" "golang.org/x/sync/errgroup" + "golang.org/x/time/rate" ) const ( @@ -248,6 +249,9 @@ type worker struct { isRunning int32 // notifies exits of run() stopNotifier notify.Notifier + + slowSynchronizeThreshold time.Duration + slowSynchronizeLimiter *rate.Limiter } func newWorker() *worker { @@ -255,6 +259,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), } } @@ -340,13 +347,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 dc5a2bff61a..c34378fd625 100644 --- a/pkg/workerpool/pool_test.go +++ b/pkg/workerpool/pool_test.go @@ -25,8 +25,10 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/stretchr/testify/require" "go.uber.org/zap" "golang.org/x/sync/errgroup" + "golang.org/x/time/rate" ) func TestSuite(t *testing.T) { check.TestingT(t) } @@ -425,6 +427,38 @@ func (s *workerPoolSuite) TestCancelByAddEventContext(c *check.C) { c.Assert(err, check.IsNil) } +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 eaa63a238cb1c571aade01e8241b3da12af6cb67 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Sat, 11 Dec 2021 00:27:57 +0800 Subject: [PATCH 06/30] metrics: add changefeed checkepoint catch-up ETA (#3300) (#3311) --- metrics/grafana/ticdc.json | 372 +++++++++++++++++++++++++------------ 1 file changed, 253 insertions(+), 119 deletions(-) diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index 9a039e8537d..ed21e8d6100 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -125,7 +125,7 @@ "gnetId": null, "graphTooltip": 1, "id": null, - "iteration": 1633763775536, + "iteration": 1636101093950, "links": [], "panels": [ { @@ -1821,11 +1821,17 @@ }, { "aliasColors": {}, - "bars": false, + "bars": true, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of sink write duration of changefeeds", + "description": "Estimate the remaining time for a changefeed catch-up upstream TiDB.", + "fieldConfig": { + "defaults": { + "unit": "s" + }, + "overrides": [] + }, "fill": 1, "fillGradient": 0, "gridPos": { @@ -1835,11 +1841,13 @@ "y": 26 }, "hiddenSeries": false, - "id": 35, + "id": 253, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": true, + "hideZero": true, "max": false, "min": false, "rightSide": true, @@ -1847,12 +1855,12 @@ "total": false, "values": true }, - "lines": true, + "lines": false, "linewidth": 1, "links": [], "nullPointMode": "null", "options": { - "dataLinks": [] + "alertThreshold": true }, "paceLength": 10, "percentage": false, @@ -1865,32 +1873,20 @@ "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))", + "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))", "format": "time_series", + "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}-p95", + "legendFormat": "{{changefeed}}", "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": "Changefeed checkpoint catch-up ETA", "tooltip": { "shared": true, "sort": 0, @@ -1907,10 +1903,10 @@ "yaxes": [ { "format": "s", - "label": null, + "label": "", "logBase": 2, - "max": null, - "min": null, + "max": "604800", + "min": "1", "show": true }, { @@ -2012,7 +2008,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of sink batch size", + "description": "Percentiles of sink write duration of changefeeds", "fill": 1, "fillGradient": 0, "gridPos": { @@ -2022,7 +2018,7 @@ "y": 33 }, "hiddenSeries": false, - "id": 36, + "id": 35, "legend": { "alignAsTable": true, "avg": false, @@ -2052,25 +2048,24 @@ "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": "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": "{{capture}}-p90", + "legendFormat": "{{instance}}-p95", "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))", + "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": "{{capture}}-p99", + "legendFormat": "{{instance}}-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))", + "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", - "hide": true, "intervalFactor": 1, - "legendFormat": "{{capture}}-p999", + "legendFormat": "{{instance}}-p999", "refId": "C" } ], @@ -2078,7 +2073,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Sink write batch size percentile", + "title": "Sink write duration percentile", "tooltip": { "shared": true, "sort": 0, @@ -2094,7 +2089,7 @@ }, "yaxes": [ { - "format": "none", + "format": "s", "label": null, "logBase": 2, "max": null, @@ -2224,7 +2219,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of asynchronous flush sink duration of changefeeds", + "description": "Percentiles of sink batch size", "fill": 1, "fillGradient": 0, "gridPos": { @@ -2234,7 +2229,7 @@ "y": 40 }, "hiddenSeries": false, - "id": 98, + "id": 36, "legend": { "alignAsTable": true, "avg": false, @@ -2264,24 +2259,25 @@ "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": "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": "{{instance}}-{{type}}-p95", + "legendFormat": "{{capture}}-p90", "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))", + "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": "{{instance}}-{{type}}-p99", + "legendFormat": "{{capture}}-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))", + "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": "{{instance}}-{{type}}-p999", + "legendFormat": "{{capture}}-p999", "refId": "C" } ], @@ -2289,7 +2285,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Flush sink duration percentile", + "title": "Sink write batch size percentile", "tooltip": { "shared": true, "sort": 0, @@ -2305,7 +2301,7 @@ }, "yaxes": [ { - "format": "s", + "format": "none", "label": null, "logBase": 2, "max": null, @@ -2409,7 +2405,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Distribution of MySQL worker loads", + "description": "Percentiles of asynchronous flush sink duration of changefeeds", "fill": 1, "fillGradient": 0, "gridPos": { @@ -2419,17 +2415,15 @@ "y": 47 }, "hiddenSeries": false, - "id": 95, + "id": 98, "legend": { "alignAsTable": true, - "avg": true, + "avg": false, "current": true, - "max": true, + "max": false, "min": false, - "rightSide": false, + "rightSide": true, "show": true, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, @@ -2440,77 +2434,46 @@ "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": "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", - "hide": true, - "interval": "", "intervalFactor": 1, - "legendFormat": "{{capture}}-{{bucket}}", + "legendFormat": "{{instance}}-{{type}}-p95", "refId": "A" }, { - "expr": "count(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) >= 0)", + "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", - "hide": true, - "interval": "", "intervalFactor": 1, - "legendFormat": "total worker", + "legendFormat": "{{instance}}-{{type}}-p99", "refId": "B" }, { - "expr": "count(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 2)", + "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", - "interval": "", "intervalFactor": 1, - "legendFormat": "0-2 row/s worker", + "legendFormat": "{{instance}}-{{type}}-p999", "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": "Flush sink duration percentile", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -2523,9 +2486,9 @@ }, "yaxes": [ { - "format": "short", + "format": "s", "label": null, - "logBase": 1, + "logBase": 2, "max": null, "min": null, "show": true @@ -2628,7 +2591,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of detecting and waiting conflict duration of MySQL sink", + "description": "Distribution of MySQL worker loads", "fill": 1, "fillGradient": 0, "gridPos": { @@ -2638,15 +2601,17 @@ "y": 54 }, "hiddenSeries": false, - "id": 83, + "id": 95, "legend": { "alignAsTable": true, - "avg": false, + "avg": true, "current": true, "max": true, "min": false, "rightSide": false, "show": true, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -2663,39 +2628,68 @@ "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, - "stack": false, + "stack": true, "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(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": "{{instance}}-p95", + "legendFormat": "{{capture}}-{{bucket}}", "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))", + "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": "{{instance}}-p99", + "legendFormat": "total worker", "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))", + "expr": "count(rate(ticdc_sink_bucket_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m]) <= 2)", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}-p999", + "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 conflict detect duration percentile", + "title": "MySQL sink worker load", "tooltip": { "shared": true, "sort": 2, @@ -2711,9 +2705,9 @@ }, "yaxes": [ { - "format": "s", + "format": "short", "label": null, - "logBase": 2, + "logBase": 1, "max": null, "min": null, "show": true @@ -2816,7 +2810,7 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of executing DDL duration of MySQL sink", + "description": "Percentiles of detecting and waiting conflict duration of MySQL sink", "fill": 1, "fillGradient": 0, "gridPos": { @@ -2826,7 +2820,7 @@ "y": 61 }, "hiddenSeries": false, - "id": 166, + "id": 83, "legend": { "alignAsTable": true, "avg": false, @@ -2855,7 +2849,7 @@ "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": "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", "interval": "", "intervalFactor": 1, @@ -2863,7 +2857,7 @@ "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))", + "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, @@ -2871,7 +2865,7 @@ "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))", + "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, @@ -2883,7 +2877,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "MySQL sink ddl execution duration percentile", + "title": "MySQL sink conflict detect duration percentile", "tooltip": { "shared": true, "sort": 2, @@ -3011,13 +3005,123 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "Percentiles of executing DDL duration of MySQL sink", "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, "y": 68 }, + "hiddenSeries": false, + "id": 166, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": 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, + "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))", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "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": "MySQL sink ddl execution duration percentile", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 2, + "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}", + "fill": 1, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 75 + }, "id": 164, "legend": { "avg": false, @@ -3160,7 +3264,36 @@ } ] }, + "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, @@ -3174,6 +3307,7 @@ } ], "title": "Schema Storage GC progress", + "transform": "table", "transformations": [ { "id": "filterFieldsByName", @@ -9545,5 +9679,5 @@ "timezone": "browser", "title": "Test-Cluster-TiCDC", "uid": "YiGL8hBZ1", - "version": 24 -} + "version": 25 +} \ No newline at end of file From ef9a4fe2b70b524eb4af7d51c02bfd16a214e15d Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Fri, 17 Dec 2021 18:58:37 +0800 Subject: [PATCH 07/30] pkg,cdc: do not use log package (#3902) (#3939) --- .golangci.yml | 13 +++++++++++++ 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 | 8 +++----- pkg/context/context_test.go | 2 +- pkg/etcd/etcdkey.go | 2 +- pkg/orchestrator/util/key_utils.go | 3 ++- 10 files changed, 26 insertions(+), 13 deletions(-) diff --git a/.golangci.yml b/.golangci.yml index b19a99a8dca..a6c7a6830d5 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -2,3 +2,16 @@ linters: enable: - unconvert - unparam + - depguard + +linters-settings: + 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 482637bc854..89fc7f4a4eb 100644 --- a/cdc/sink/codec/canal.go +++ b/cdc/sink/codec/canal.go @@ -15,12 +15,12 @@ package codec import ( "fmt" - "log" "strconv" "strings" "github.com/golang/protobuf/proto" // nolint:staticcheck "github.com/pingcap/errors" + "github.com/pingcap/log" mm "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" parser_types "github.com/pingcap/parser/types" 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 9449638e4a0..fe129cd907a 100644 --- a/pkg/context/context.go +++ b/pkg/context/context.go @@ -15,16 +15,14 @@ package context import ( "context" - "log" "time" - "github.com/pingcap/ticdc/pkg/pdtime" - - "github.com/pingcap/ticdc/pkg/version" - + "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/pdtime" + "github.com/pingcap/ticdc/pkg/version" tidbkv "github.com/pingcap/tidb/kv" "github.com/tikv/client-go/v2/oracle" pd "github.com/tikv/pd/client" diff --git a/pkg/context/context_test.go b/pkg/context/context_test.go index 78186e1673d..b0caee01c3f 100644 --- a/pkg/context/context_test.go +++ b/pkg/context/context_test.go @@ -136,7 +136,7 @@ func (s *contextSuite) TestThrowPanic(c *check.C) { defer testleak.AfterTest(c)() defer func() { panicMsg := recover() - c.Assert(panicMsg, check.Equals, "an error has escaped, please report a bug{error 26 0 mock error}") + c.Assert(panicMsg, check.Equals, "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 51d4c91cf0a..ca7ac9cfe15 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 3c36fc09c2ea32944b90a84c2700b7cb80163015 Mon Sep 17 00:00:00 2001 From: amyangfei Date: Mon, 20 Dec 2021 10:39:46 +0800 Subject: [PATCH 08/30] *: rename repo from pingcap/ticdc to pingcap/tiflow (#3957) --- Dockerfile | 4 +- Dockerfile.development | 4 +- Makefile | 8 +- README.md | 8 +- cdc/capture.go | 18 ++-- cdc/capture/capture.go | 20 ++--- cdc/capture/http_handler.go | 16 ++-- cdc/capture/http_validator.go | 20 ++--- cdc/capture/http_validator_test.go | 4 +- cdc/capture_test.go | 18 ++-- cdc/changefeed.go | 16 ++-- cdc/changefeed_test.go | 10 +-- cdc/entry/codec.go | 4 +- cdc/entry/codec_test.go | 2 +- cdc/entry/mounter.go | 6 +- cdc/entry/mounter_test.go | 6 +- cdc/entry/schema_storage.go | 8 +- cdc/entry/schema_storage_test.go | 6 +- 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 | 82 +++++++++---------- cdc/kv/etcd.go | 10 +-- cdc/kv/etcd_test.go | 10 +-- 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 | 18 ++-- cdc/model/capture.go | 2 +- cdc/model/capture_test.go | 2 +- cdc/model/changefeed.go | 6 +- cdc/model/changefeed_test.go | 6 +- cdc/model/http_model.go | 4 +- cdc/model/http_model_test.go | 2 +- cdc/model/kv.go | 2 +- cdc/model/kv_test.go | 4 +- cdc/model/mounter_test.go | 2 +- cdc/model/owner.go | 2 +- cdc/model/owner_test.go | 2 +- cdc/model/reactor_state.go | 8 +- cdc/model/reactor_state_test.go | 8 +- cdc/model/schema_storage_test.go | 2 +- cdc/model/sink.go | 4 +- cdc/model/sink_test.go | 2 +- cdc/model/string.go | 2 +- cdc/model/string_test.go | 2 +- cdc/owner.go | 24 +++--- 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 | 10 +-- cdc/owner/changefeed_test.go | 18 ++-- cdc/owner/ddl_puller.go | 14 ++-- cdc/owner/ddl_puller_test.go | 8 +- cdc/owner/feed_state_manager.go | 4 +- cdc/owner/feed_state_manager_test.go | 10 +-- cdc/owner/owner.go | 12 +-- cdc/owner/owner_test.go | 16 ++-- cdc/owner/scheduler.go | 4 +- cdc/owner/scheduler_test.go | 6 +- cdc/owner/schema.go | 12 +-- cdc/owner/schema_test.go | 8 +- cdc/owner_operator.go | 12 +-- cdc/owner_test.go | 22 ++--- cdc/processor.go | 24 +++--- 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 | 12 +-- cdc/processor/pipeline/sorter_test.go | 12 +-- cdc/processor/pipeline/table.go | 16 ++-- cdc/processor/processor.go | 28 +++---- cdc/processor/processor_test.go | 16 ++-- cdc/processor_test.go | 18 ++-- cdc/puller/entry_sorter.go | 6 +- cdc/puller/entry_sorter_test.go | 4 +- 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/puller/sorter.go | 2 +- cdc/puller/sorter/backend.go | 2 +- cdc/puller/sorter/backend_pool.go | 8 +- cdc/puller/sorter/backend_pool_test.go | 20 ++--- cdc/puller/sorter/file_backend.go | 4 +- cdc/puller/sorter/file_backend_test.go | 6 +- cdc/puller/sorter/heap.go | 2 +- cdc/puller/sorter/heap_sorter.go | 10 +-- cdc/puller/sorter/memory_backend.go | 2 +- cdc/puller/sorter/memory_backend_test.go | 4 +- cdc/puller/sorter/merger.go | 8 +- cdc/puller/sorter/merger_test.go | 28 +++---- cdc/puller/sorter/serde.go | 2 +- cdc/puller/sorter/sorter_test.go | 40 ++++----- cdc/puller/sorter/unified_sorter.go | 8 +- cdc/server.go | 16 ++-- cdc/server_test.go | 12 +-- cdc/sink/black_hole.go | 2 +- 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 | 4 +- 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 | 2 +- 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 | 6 +- cdc/sink/mq.go | 24 +++--- 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/txns_heap.go | 2 +- cdc/sink/txns_heap_test.go | 4 +- cdc/task.go | 4 +- cdc/task_test.go | 10 +-- chaos/Dockerfile.chaos | 4 +- cmd/cdc/main.go | 2 +- cmd/kafka-consumer/main.go | 18 ++-- demo/java/README.md | 4 +- ...5-ticdc-high-availability-new-design-cn.md | 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/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 | 18 ++-- 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_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 | 6 +- pkg/cmd/factory/factory.go | 6 +- pkg/cmd/factory/factory_impl.go | 14 ++-- 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/config_test.go | 2 +- pkg/config/sorter.go | 2 +- pkg/context/context.go | 10 +-- pkg/context/context_test.go | 6 +- pkg/cyclic/filter.go | 4 +- pkg/cyclic/filter_test.go | 6 +- pkg/cyclic/mark/mark.go | 6 +- pkg/cyclic/mark/mark_test.go | 2 +- pkg/cyclic/replication.go | 8 +- pkg/cyclic/replication_test.go | 8 +- pkg/errors/helper_test.go | 2 +- pkg/etcd/client.go | 4 +- pkg/etcd/client_test.go | 2 +- pkg/etcd/etcd_test.go | 2 +- 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 | 4 +- pkg/flags/urls.go | 2 +- pkg/flags/urls_test.go | 2 +- pkg/httputil/httputil.go | 2 +- pkg/httputil/httputil_test.go | 4 +- pkg/logutil/log.go | 4 +- pkg/logutil/log_test.go | 2 +- pkg/notify/notify.go | 2 +- pkg/notify/notify_test.go | 4 +- 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_tester.go | 4 +- pkg/orchestrator/util/key_utils_test.go | 2 +- pkg/pdtime/acquirer.go | 2 +- pkg/pdtime/acquirer_test.go | 2 +- pkg/pipeline/context.go | 2 +- pkg/pipeline/message.go | 2 +- pkg/pipeline/pipeline.go | 4 +- pkg/pipeline/pipeline_test.go | 10 +-- pkg/pipeline/runner.go | 2 +- pkg/pipeline/test.go | 2 +- pkg/quotes/quotes_test.go | 2 +- pkg/regionspan/region_range_lock_test.go | 2 +- pkg/regionspan/region_test.go | 2 +- pkg/regionspan/span.go | 2 +- pkg/regionspan/span_test.go | 2 +- pkg/retry/retry_test.go | 2 +- pkg/retry/retry_with_opt.go | 2 +- pkg/scheduler/interface.go | 2 +- pkg/scheduler/table_number.go | 2 +- pkg/scheduler/table_number_test.go | 4 +- pkg/scheduler/workload.go | 2 +- pkg/scheduler/workload_test.go | 4 +- pkg/security/credential.go | 2 +- 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/urls.go | 2 +- pkg/types/urls_test.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 | 4 +- pkg/version/version.go | 2 +- pkg/workerpool/async_pool_impl.go | 4 +- pkg/workerpool/async_pool_test.go | 2 +- pkg/workerpool/pool_impl.go | 4 +- pkg/workerpool/pool_test.go | 12 +-- .../cdc_state_checker/cdc_monitor.go | 8 +- 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 | 12 +-- tests/integration_tests/bank/case.go | 4 +- .../capture_session_done_during_task/run.sh | 4 +- .../run.sh | 2 +- tests/integration_tests/cdc/cdc.go | 4 +- .../changefeed_auto_stop/run.sh | 4 +- .../integration_tests/changefeed_error/run.sh | 14 ++-- .../changefeed_fast_fail/run.sh | 2 +- tests/integration_tests/dailytest/db.go | 2 +- tests/integration_tests/ddl_puller_lag/run.sh | 4 +- .../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 | 4 +- .../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 | 4 +- .../processor_err_chan/run.sh | 4 +- .../integration_tests/processor_panic/main.go | 4 +- .../integration_tests/processor_panic/run.sh | 4 +- .../processor_resolved_ts_fallback/run.sh | 2 +- .../processor_stop_delay/run.sh | 4 +- 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 +- 359 files changed, 1165 insertions(+), 1165 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 1a86719e97c..497c79c4d46 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 7412bf401cb..1850adedae6 100644 --- a/Makefile +++ b/Makefile @@ -2,7 +2,7 @@ .PHONY: build test check clean fmt cdc kafka_consumer coverage \ integration_test_build integration_test integration_test_mysql integration_test_kafka bank -PROJECT=ticdc +PROJECT=tiflow FAIL_ON_STDOUT := awk '{ print } END { if (NR > 0) { exit 1 } }' @@ -35,7 +35,7 @@ PACKAGES := $$($(PACKAGE_LIST)) PACKAGE_DIRECTORIES := $(PACKAGE_LIST) | sed 's|github.com/pingcap/$(PROJECT)/||' FILES := $$(find . -name '*.go' -type f | grep -vE 'vendor|kv_gen|proto') TEST_FILES := $$(find . -name '*_test.go' -type f | grep -vE 'vendor|kv_gen|integration|testing_utils') -CDC_PKG := github.com/pingcap/ticdc +CDC_PKG := github.com/pingcap/tiflow FAILPOINT_DIR := $$(for p in $(PACKAGES); do echo $${p\#"github.com/pingcap/$(PROJECT)/"}|grep -v "github.com/pingcap/$(PROJECT)"; done) FAILPOINT := bin/failpoint-ctl @@ -135,8 +135,8 @@ integration_test_build: check_failpoint_ctl ./scripts/fix_lib_zstd.sh $(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; } diff --git a/README.md b/README.md index 7841edc10c4..ba312a20260 100644 --- a/README.md +++ b/README.md @@ -1,10 +1,10 @@ # TiCDC [![Build Status](https://internal.pingcap.net/idc-jenkins/job/build_cdc_multi_branch/job/master/badge/icon)](https://internal.pingcap.net/idc-jenkins/job/build_cdc_multi_branch/job/master) -[![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) +[![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.go b/cdc/capture.go index 0015c387bb1..f5e889759df 100644 --- a/cdc/capture.go +++ b/cdc/capture.go @@ -22,16 +22,16 @@ 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/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/orchestrator" - "github.com/pingcap/ticdc/pkg/util" - "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/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/orchestrator" + "github.com/pingcap/tiflow/pkg/util" + "github.com/pingcap/tiflow/pkg/version" pd "github.com/tikv/pd/client" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/clientv3/concurrency" diff --git a/cdc/capture/capture.go b/cdc/capture/capture.go index 46e580c62e2..cc9adebedf0 100644 --- a/cdc/capture/capture.go +++ b/cdc/capture/capture.go @@ -24,17 +24,17 @@ 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/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/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 dab346af776..cb616f4e10a 100644 --- a/cdc/capture/http_handler.go +++ b/cdc/capture/http_handler.go @@ -20,15 +20,15 @@ import ( "github.com/gin-gonic/gin" "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/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/kv" + "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 8fe917f9e04..5baa2219f20 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_test.go b/cdc/capture_test.go index deb94dacbd1..8f353dde519 100644 --- a/cdc/capture_test.go +++ b/cdc/capture_test.go @@ -22,13 +22,13 @@ import ( "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/failpoint" - "github.com/pingcap/ticdc/cdc/kv" - "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/util" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/kv" + "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/util" + "github.com/pingcap/tiflow/pkg/util/testleak" pd "github.com/tikv/pd/client" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/clientv3/concurrency" @@ -113,10 +113,10 @@ func (s *captureSuite) TestCaptureSessionDoneDuringHandleTask(c *check.C) { c.Assert(err, check.IsNil) runProcessorCount := 0 - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/captureHandleTaskDelay", "sleep(500)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/captureHandleTaskDelay", "sleep(500)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/captureHandleTaskDelay") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/captureHandleTaskDelay") }() runProcessorBackup := runProcessorImpl runProcessorImpl = func( diff --git a/cdc/changefeed.go b/cdc/changefeed.go index e4add926bd9..30a0a108a93 100644 --- a/cdc/changefeed.go +++ b/cdc/changefeed.go @@ -24,15 +24,15 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/log" timodel "github.com/pingcap/parser/model" - "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/cyclic/mark" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/filter" - "github.com/pingcap/ticdc/pkg/scheduler" "github.com/pingcap/tidb/sessionctx/binloginfo" + "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/cyclic/mark" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/filter" + "github.com/pingcap/tiflow/pkg/scheduler" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/mvcc/mvccpb" "go.uber.org/zap" diff --git a/cdc/changefeed_test.go b/cdc/changefeed_test.go index 8ab24cc6cda..52c733240a6 100644 --- a/cdc/changefeed_test.go +++ b/cdc/changefeed_test.go @@ -19,11 +19,11 @@ import ( "time" "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/kv" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/util" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/kv" + "github.com/pingcap/tiflow/cdc/model" + "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/clientv3/concurrency" "go.etcd.io/etcd/embed" diff --git a/cdc/entry/codec.go b/cdc/entry/codec.go index 0aed19266c4..98a5d3157ef 100644 --- a/cdc/entry/codec.go +++ b/cdc/entry/codec.go @@ -19,13 +19,13 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/mysql" - "github.com/pingcap/ticdc/cdc/model" - cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/tidb/kv" "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/codec_test.go b/cdc/entry/codec_test.go index c961495c5a1..4b8ceffd4f7 100644 --- a/cdc/entry/codec_test.go +++ b/cdc/entry/codec_test.go @@ -17,10 +17,10 @@ import ( "testing" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/util/testleak" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tiflow/pkg/util/testleak" ) func Test(t *testing.T) { check.TestingT(t) } diff --git a/cdc/entry/mounter.go b/cdc/entry/mounter.go index 9b27761fac8..28b50011132 100644 --- a/cdc/entry/mounter.go +++ b/cdc/entry/mounter.go @@ -26,13 +26,13 @@ import ( "github.com/pingcap/log" timodel "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" - "github.com/pingcap/ticdc/cdc/model" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/util" "github.com/pingcap/tidb/kv" "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 523bf8d1015..7fc9d74464f 100644 --- a/cdc/entry/mounter_test.go +++ b/cdc/entry/mounter_test.go @@ -21,14 +21,14 @@ import ( "github.com/pingcap/check" "github.com/pingcap/log" "github.com/pingcap/parser/mysql" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/regionspan" - "github.com/pingcap/ticdc/pkg/util/testleak" ticonfig "github.com/pingcap/tidb/config" tidbkv "github.com/pingcap/tidb/kv" "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/pingcap/tiflow/pkg/util/testleak" "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 6031a1a859b..a23cb651bec 100644 --- a/cdc/entry/schema_storage.go +++ b/cdc/entry/schema_storage.go @@ -23,11 +23,11 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" timodel "github.com/pingcap/parser/model" - "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" + "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 15db66ebd82..f6032a314b3 100644 --- a/cdc/entry/schema_storage_test.go +++ b/cdc/entry/schema_storage_test.go @@ -23,9 +23,6 @@ import ( "github.com/pingcap/errors" timodel "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" - "github.com/pingcap/ticdc/cdc/kv" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/util/testleak" ticonfig "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/domain" tidbkv "github.com/pingcap/tidb/kv" @@ -35,6 +32,9 @@ import ( "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tiflow/cdc/kv" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/util/testleak" "github.com/tikv/client-go/v2/oracle" ) diff --git a/cdc/http_handler.go b/cdc/http_handler.go index 62d63ee03b9..ba7478d358e 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 a27bb14fba2..da4f75e8ac7 100644 --- a/cdc/http_router.go +++ b/cdc/http_router.go @@ -21,11 +21,11 @@ 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 4effd1b59f7..7319701e222 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/cdc/kv" - "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/util" - "github.com/pingcap/ticdc/pkg/version" + "github.com/pingcap/tiflow/cdc/capture" + "github.com/pingcap/tiflow/cdc/kv" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" + "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 7470bb069c5..722bf058ffe 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 2274b3587f1..e91008ae546 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 fb82bba833a..8ed69ce9a1b 100644 --- a/cdc/kv/client_test.go +++ b/cdc/kv/client_test.go @@ -32,17 +32,17 @@ 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/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/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" @@ -1300,10 +1300,10 @@ func (s *etcdSuite) 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) @@ -1488,10 +1488,10 @@ func (s *etcdSuite) 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 @@ -1642,10 +1642,10 @@ func (s *etcdSuite) 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{} @@ -1902,10 +1902,10 @@ func (s *etcdSuite) 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) @@ -2001,10 +2001,10 @@ func (s *etcdSuite) testEventCommitTsFallback(c *check.C, events []*cdcpb.Change 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) @@ -2149,10 +2149,10 @@ func (s *etcdSuite) 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) @@ -2648,13 +2648,13 @@ func (s *etcdSuite) 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)) @@ -2730,13 +2730,13 @@ func (s *etcdSuite) 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{} @@ -2793,16 +2793,16 @@ func (s *etcdSuite) 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{} @@ -2871,12 +2871,12 @@ func (s *etcdSuite) 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 }() @@ -3024,10 +3024,10 @@ func (s *etcdSuite) 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{} @@ -3137,10 +3137,10 @@ func (s *etcdSuite) 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/etcd.go b/cdc/kv/etcd.go index bda6a10ca76..86dd0843680 100644 --- a/cdc/kv/etcd.go +++ b/cdc/kv/etcd.go @@ -22,11 +22,11 @@ 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/etcd" - "github.com/pingcap/ticdc/pkg/retry" - "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/tiflow/cdc/model" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/retry" + "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/cdc/kv/etcd_test.go b/cdc/kv/etcd_test.go index e577c044a12..96ab729f196 100644 --- a/cdc/kv/etcd_test.go +++ b/cdc/kv/etcd_test.go @@ -21,11 +21,11 @@ import ( "time" "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - cerror "github.com/pingcap/ticdc/pkg/errors" - "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" + cerror "github.com/pingcap/tiflow/pkg/errors" + "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/clientv3/concurrency" "go.etcd.io/etcd/embed" 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 08d676e6712..67767a58808 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 etcdSuite 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 96556cba690..1e3c8fb8a47 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 6dad49a935c..a8d217996ef 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 2fde08e12d2..1d59c443302 100644 --- a/cdc/metrics.go +++ b/cdc/metrics.go @@ -14,15 +14,15 @@ 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/puller/sorter" - "github.com/pingcap/ticdc/cdc/sink" - "github.com/pingcap/ticdc/pkg/config" + "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/puller/sorter" + "github.com/pingcap/tiflow/cdc/sink" + "github.com/pingcap/tiflow/pkg/config" "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/capture_test.go b/cdc/model/capture_test.go index 688f0bf7726..f5358c301bb 100644 --- a/cdc/model/capture_test.go +++ b/cdc/model/capture_test.go @@ -15,7 +15,7 @@ package model import ( "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type captureSuite struct{} diff --git a/cdc/model/changefeed.go b/cdc/model/changefeed.go index 6ddf23fa60b..31acdd509da 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 ff7f0a3e2b4..865c614afdd 100644 --- a/cdc/model/changefeed_test.go +++ b/cdc/model/changefeed_test.go @@ -19,10 +19,10 @@ import ( "github.com/pingcap/check" "github.com/pingcap/parser/model" - "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/util/testleak" filter "github.com/pingcap/tidb-tools/pkg/table-filter" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/util/testleak" "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 643428fd538..63af3804af5 100644 --- a/cdc/model/kv_test.go +++ b/cdc/model/kv_test.go @@ -15,8 +15,8 @@ package model import ( "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 kvSuite struct{} diff --git a/cdc/model/mounter_test.go b/cdc/model/mounter_test.go index 61452d246f4..fbbd1db383f 100644 --- a/cdc/model/mounter_test.go +++ b/cdc/model/mounter_test.go @@ -18,7 +18,7 @@ import ( "sync" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type mounterSuite struct{} diff --git a/cdc/model/owner.go b/cdc/model/owner.go index 0a988d33ae1..61dd65d5fa2 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/owner_test.go b/cdc/model/owner_test.go index ea811233cc1..d960963cdfe 100644 --- a/cdc/model/owner_test.go +++ b/cdc/model/owner_test.go @@ -18,7 +18,7 @@ import ( "testing" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" ) func TestSuite(t *testing.T) { check.TestingT(t) } diff --git a/cdc/model/reactor_state.go b/cdc/model/reactor_state.go index b4e4b8668a3..05a64f145bb 100644 --- a/cdc/model/reactor_state.go +++ b/cdc/model/reactor_state.go @@ -19,10 +19,10 @@ 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" - "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" + "github.com/pingcap/tiflow/pkg/orchestrator/util" "go.uber.org/zap" ) diff --git a/cdc/model/reactor_state_test.go b/cdc/model/reactor_state_test.go index df76386c49a..a1ab46509e8 100644 --- a/cdc/model/reactor_state_test.go +++ b/cdc/model/reactor_state_test.go @@ -20,10 +20,10 @@ import ( "github.com/google/go-cmp/cmp" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/orchestrator" - "github.com/pingcap/ticdc/pkg/orchestrator/util" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/orchestrator" + "github.com/pingcap/tiflow/pkg/orchestrator/util" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type stateSuite struct{} diff --git a/cdc/model/schema_storage_test.go b/cdc/model/schema_storage_test.go index 42d675bcb95..5979546c707 100644 --- a/cdc/model/schema_storage_test.go +++ b/cdc/model/schema_storage_test.go @@ -18,7 +18,7 @@ import ( timodel "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" parser_types "github.com/pingcap/parser/types" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type schemaStorageSuite struct{} diff --git a/cdc/model/sink.go b/cdc/model/sink.go index 79a628527a3..cc0c9911ad9 100644 --- a/cdc/model/sink.go +++ b/cdc/model/sink.go @@ -20,8 +20,8 @@ import ( "github.com/pingcap/log" "github.com/pingcap/parser/model" - "github.com/pingcap/ticdc/pkg/quotes" - "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/tiflow/pkg/quotes" + "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" ) diff --git a/cdc/model/sink_test.go b/cdc/model/sink_test.go index 1e7294038c6..799b6c7e990 100644 --- a/cdc/model/sink_test.go +++ b/cdc/model/sink_test.go @@ -18,7 +18,7 @@ import ( timodel "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/types" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type columnFlagTypeSuite struct{} 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/model/string_test.go b/cdc/model/string_test.go index a6dcbb67beb..a9ad15e185c 100644 --- a/cdc/model/string_test.go +++ b/cdc/model/string_test.go @@ -15,7 +15,7 @@ package model import ( "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type stringSuite struct{} diff --git a/cdc/owner.go b/cdc/owner.go index 5d713631fbb..4bbd5ef9283 100644 --- a/cdc/owner.go +++ b/cdc/owner.go @@ -25,18 +25,18 @@ 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" - "github.com/pingcap/ticdc/cdc/sink" - "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/scheduler" - "github.com/pingcap/ticdc/pkg/txnutil/gc" - "github.com/pingcap/ticdc/pkg/util" + "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" + "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/scheduler" + "github.com/pingcap/tiflow/pkg/txnutil/gc" + "github.com/pingcap/tiflow/pkg/util" "github.com/tikv/client-go/v2/oracle" pd "github.com/tikv/pd/client" "go.etcd.io/etcd/clientv3" diff --git a/cdc/owner/async_sink.go b/cdc/owner/async_sink.go index 4e33b21f289..0cd1f7cad70 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 fb1e1aadd01..5a91821c9d1 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 dcf2540d49a..90be548eb0b 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 a4c1355e0ab..9a0d3a80ee8 100644 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -21,12 +21,12 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/log" timodel "github.com/pingcap/parser/model" - "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/txnutil/gc" - "github.com/pingcap/ticdc/pkg/util" "github.com/pingcap/tidb/sessionctx/binloginfo" + "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/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 ec255aabd0f..d6d2d0b7c84 100644 --- a/cdc/owner/changefeed_test.go +++ b/cdc/owner/changefeed_test.go @@ -21,15 +21,15 @@ import ( "github.com/pingcap/check" "github.com/pingcap/errors" timodel "github.com/pingcap/parser/model" - "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" + "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 49ff27e8243..d011c63a605 100644 --- a/cdc/owner/ddl_puller.go +++ b/cdc/owner/ddl_puller.go @@ -20,13 +20,13 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" timodel "github.com/pingcap/parser/model" - "github.com/pingcap/ticdc/cdc/entry" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/puller" - 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" + "github.com/pingcap/tiflow/cdc/entry" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/puller" + 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 da341c98104..f7ec756b63b 100644 --- a/cdc/owner/ddl_puller_test.go +++ b/cdc/owner/ddl_puller_test.go @@ -22,11 +22,11 @@ import ( "github.com/pingcap/check" "github.com/pingcap/errors" timodel "github.com/pingcap/parser/model" - "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" "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 72245ac4a84..fe84913e7b4 100644 --- a/cdc/owner/feed_state_manager.go +++ b/cdc/owner/feed_state_manager.go @@ -18,8 +18,8 @@ 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/tiflow/cdc/model" + cerrors "github.com/pingcap/tiflow/pkg/errors" "go.uber.org/zap" ) diff --git a/cdc/owner/feed_state_manager_test.go b/cdc/owner/feed_state_manager_test.go index f16cf41594d..4a9ec487745 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 30926147f68..17fb2d6a0b3 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 da18644ac7f..9230f94fa4c 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 d71725dc465..00f0771d815 100644 --- a/cdc/owner/scheduler.go +++ b/cdc/owner/scheduler.go @@ -19,8 +19,8 @@ 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/tiflow/cdc/model" + cerror "github.com/pingcap/tiflow/pkg/errors" "go.uber.org/zap" ) diff --git a/cdc/owner/scheduler_test.go b/cdc/owner/scheduler_test.go index 712a89a31e6..669ab85a8f5 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 f518cdada16..9cd467191b1 100644 --- a/cdc/owner/schema.go +++ b/cdc/owner/schema.go @@ -17,14 +17,14 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" timodel "github.com/pingcap/parser/model" - "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" + "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 f787a5c81f1..e7386c19d2a 100644 --- a/cdc/owner/schema_test.go +++ b/cdc/owner/schema_test.go @@ -19,10 +19,10 @@ import ( "github.com/pingcap/check" timodel "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" - "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" + "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_operator.go b/cdc/owner_operator.go index 8652e6d6705..42bb1879faa 100644 --- a/cdc/owner_operator.go +++ b/cdc/owner_operator.go @@ -21,12 +21,12 @@ import ( "github.com/pingcap/errors" timodel "github.com/pingcap/parser/model" - "github.com/pingcap/ticdc/cdc/entry" - "github.com/pingcap/ticdc/cdc/kv" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/puller" - "github.com/pingcap/ticdc/pkg/regionspan" - "github.com/pingcap/ticdc/pkg/util" + "github.com/pingcap/tiflow/cdc/entry" + "github.com/pingcap/tiflow/cdc/kv" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/puller" + "github.com/pingcap/tiflow/pkg/regionspan" + "github.com/pingcap/tiflow/pkg/util" pd "github.com/tikv/pd/client" "golang.org/x/sync/errgroup" ) diff --git a/cdc/owner_test.go b/cdc/owner_test.go index c51578fb58e..edd348f7f81 100644 --- a/cdc/owner_test.go +++ b/cdc/owner_test.go @@ -28,19 +28,19 @@ import ( timodel "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/types" - "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/etcd" - "github.com/pingcap/ticdc/pkg/filter" - "github.com/pingcap/ticdc/pkg/security" - "github.com/pingcap/ticdc/pkg/util" - "github.com/pingcap/ticdc/pkg/util/testleak" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/store/mockstore" + "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/etcd" + "github.com/pingcap/tiflow/pkg/filter" + "github.com/pingcap/tiflow/pkg/security" + "github.com/pingcap/tiflow/pkg/util" + "github.com/pingcap/tiflow/pkg/util/testleak" "github.com/tikv/client-go/v2/oracle" pd "github.com/tikv/pd/client" "go.etcd.io/etcd/clientv3" diff --git a/cdc/processor.go b/cdc/processor.go index 2cdcf9f8be5..d4305ea2faf 100644 --- a/cdc/processor.go +++ b/cdc/processor.go @@ -27,19 +27,19 @@ 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" - "github.com/pingcap/ticdc/cdc/puller" - psorter "github.com/pingcap/ticdc/cdc/puller/sorter" - "github.com/pingcap/ticdc/cdc/sink" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/filter" - "github.com/pingcap/ticdc/pkg/notify" - "github.com/pingcap/ticdc/pkg/regionspan" - "github.com/pingcap/ticdc/pkg/retry" - "github.com/pingcap/ticdc/pkg/util" 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/puller" + psorter "github.com/pingcap/tiflow/cdc/puller/sorter" + "github.com/pingcap/tiflow/cdc/sink" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/filter" + "github.com/pingcap/tiflow/pkg/notify" + "github.com/pingcap/tiflow/pkg/regionspan" + "github.com/pingcap/tiflow/pkg/retry" + "github.com/pingcap/tiflow/pkg/util" "github.com/tikv/client-go/v2/oracle" pd "github.com/tikv/pd/client" "go.etcd.io/etcd/clientv3" diff --git a/cdc/processor/manager.go b/cdc/processor/manager.go index 29c4f6e737d..56f8aea781a 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 33ae08d3f24..1d61816c2e6 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 f9af23c7ab1..e1d7461c0e4 100644 --- a/cdc/processor/pipeline/sorter.go +++ b/cdc/processor/pipeline/sorter.go @@ -20,12 +20,12 @@ 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/puller" - psorter "github.com/pingcap/ticdc/cdc/puller/sorter" - 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/puller" + psorter "github.com/pingcap/tiflow/cdc/puller/sorter" + 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 da700295689..a8f3f51f068 100644 --- a/cdc/processor/pipeline/sorter_test.go +++ b/cdc/processor/pipeline/sorter_test.go @@ -15,12 +15,12 @@ package pipeline import ( "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/puller/sorter" - "github.com/pingcap/ticdc/pkg/config" - cdcContext "github.com/pingcap/ticdc/pkg/context" - "github.com/pingcap/ticdc/pkg/pipeline" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/puller/sorter" + "github.com/pingcap/tiflow/pkg/config" + cdcContext "github.com/pingcap/tiflow/pkg/context" + "github.com/pingcap/tiflow/pkg/pipeline" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type sorterSuite struct{} diff --git a/cdc/processor/pipeline/table.go b/cdc/processor/pipeline/table.go index 4defb705795..09f8823a3b6 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 06fdfa2a16c..ddcca8a87da 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -24,20 +24,20 @@ 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/sink" - 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/sink" + 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 1786d99c1a8..0b2d17681b7 100644 --- a/cdc/processor/processor_test.go +++ b/cdc/processor/processor_test.go @@ -24,14 +24,14 @@ 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" - 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" + 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/processor_test.go b/cdc/processor_test.go index 93e75c036ed..5e591e89d36 100644 --- a/cdc/processor_test.go +++ b/cdc/processor_test.go @@ -17,9 +17,9 @@ import ( "bytes" "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 processorSuite struct{} @@ -54,15 +54,15 @@ import ( "sync" "time" - "github.com/pingcap/ticdc/cdc/entry" + "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/check" pd "github.com/pingcap/pd/client" - "github.com/pingcap/ticdc/cdc/kv" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/cdc/roles/storage" - "github.com/pingcap/ticdc/cdc/sink" - "github.com/pingcap/ticdc/pkg/etcd" + "github.com/pingcap/tiflow/cdc/kv" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/roles/storage" + "github.com/pingcap/tiflow/cdc/sink" + "github.com/pingcap/tiflow/pkg/etcd" ) type processorSuite struct{} diff --git a/cdc/puller/entry_sorter.go b/cdc/puller/entry_sorter.go index 8a9caa8b539..a11a8eaf0a6 100644 --- a/cdc/puller/entry_sorter.go +++ b/cdc/puller/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/puller/entry_sorter_test.go b/cdc/puller/entry_sorter_test.go index 91a64bfb300..5c074af791f 100644 --- a/cdc/puller/entry_sorter_test.go +++ b/cdc/puller/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/puller/frontier/frontier.go b/cdc/puller/frontier/frontier.go index ed1b10f94d3..6d6d493808e 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 36cf9ad1809..f436256bf45 100644 --- a/cdc/puller/mock_puller.go +++ b/cdc/puller/mock_puller.go @@ -22,13 +22,13 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/log" timodel "github.com/pingcap/parser/model" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/regionspan" "github.com/pingcap/tidb/domain" tidbkv "github.com/pingcap/tidb/kv" "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 7e6c8d3a09a..e347845a363 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 667a1ef5e1a..b9fceed79b7 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/puller/sorter.go b/cdc/puller/sorter.go index 3864cd65e43..53136e01836 100644 --- a/cdc/puller/sorter.go +++ b/cdc/puller/sorter.go @@ -16,7 +16,7 @@ package puller 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/puller/sorter/backend.go b/cdc/puller/sorter/backend.go index 4554ffc7e69..50ef7cc8ac7 100644 --- a/cdc/puller/sorter/backend.go +++ b/cdc/puller/sorter/backend.go @@ -13,7 +13,7 @@ package sorter -import "github.com/pingcap/ticdc/cdc/model" +import "github.com/pingcap/tiflow/cdc/model" type backEnd interface { reader() (backEndReader, error) diff --git a/cdc/puller/sorter/backend_pool.go b/cdc/puller/sorter/backend_pool.go index dcdfd522d93..7a3e3fc0122 100644 --- a/cdc/puller/sorter/backend_pool.go +++ b/cdc/puller/sorter/backend_pool.go @@ -27,11 +27,11 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "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" + "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/puller/sorter/backend_pool_test.go b/cdc/puller/sorter/backend_pool_test.go index 81bef2a4c90..8feae7b8918 100644 --- a/cdc/puller/sorter/backend_pool_test.go +++ b/cdc/puller/sorter/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/puller/sorter/memoryPressureInjectPoint", "return(100)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/puller/sorter/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/puller/sorter/memoryPressureInjectPoint", "return(0)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/puller/sorter/memoryPressureInjectPoint", "return(0)") c.Assert(err, check.IsNil) - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/puller/sorter/memoryUsageInjectPoint", "return(34359738368)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/puller/sorter/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/puller/sorter/memoryPressureInjectPoint", "return(0)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/puller/sorter/memoryPressureInjectPoint", "return(0)") c.Assert(err, check.IsNil) - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/puller/sorter/memoryUsageInjectPoint", "return(0)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/puller/sorter/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/puller/sorter/memoryPressureInjectPoint", "return(100)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/puller/sorter/memoryPressureInjectPoint", "return(100)") c.Assert(err, check.IsNil) - defer failpoint.Disable("github.com/pingcap/ticdc/cdc/puller/sorter/memoryPressureInjectPoint") //nolint:errcheck + defer failpoint.Disable("github.com/pingcap/tiflow/cdc/puller/sorter/memoryPressureInjectPoint") //nolint:errcheck backEndPool, err := newBackEndPool(sorterDir, "") c.Assert(err, check.IsNil) diff --git a/cdc/puller/sorter/file_backend.go b/cdc/puller/sorter/file_backend.go index 14b50679b17..11d3e4ed742 100644 --- a/cdc/puller/sorter/file_backend.go +++ b/cdc/puller/sorter/file_backend.go @@ -23,8 +23,8 @@ 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/tiflow/cdc/model" + cerrors "github.com/pingcap/tiflow/pkg/errors" "go.uber.org/zap" ) diff --git a/cdc/puller/sorter/file_backend_test.go b/cdc/puller/sorter/file_backend_test.go index c1035058446..48b2b6f303b 100644 --- a/cdc/puller/sorter/file_backend_test.go +++ b/cdc/puller/sorter/file_backend_test.go @@ -18,9 +18,9 @@ import ( "os" "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + cerrors "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type fileBackendSuite struct{} diff --git a/cdc/puller/sorter/heap.go b/cdc/puller/sorter/heap.go index 8b166322d34..358db5cb314 100644 --- a/cdc/puller/sorter/heap.go +++ b/cdc/puller/sorter/heap.go @@ -13,7 +13,7 @@ package sorter -import "github.com/pingcap/ticdc/cdc/model" +import "github.com/pingcap/tiflow/cdc/model" type sortItem struct { entry *model.PolymorphicEvent diff --git a/cdc/puller/sorter/heap_sorter.go b/cdc/puller/sorter/heap_sorter.go index 2ccbd653808..190bfaeba2c 100644 --- a/cdc/puller/sorter/heap_sorter.go +++ b/cdc/puller/sorter/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/puller/sorter/memory_backend.go b/cdc/puller/sorter/memory_backend.go index 9a0e81d1f80..8e8e76555fd 100644 --- a/cdc/puller/sorter/memory_backend.go +++ b/cdc/puller/sorter/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/puller/sorter/memory_backend_test.go b/cdc/puller/sorter/memory_backend_test.go index 86b93b3ebcf..4055739eea9 100644 --- a/cdc/puller/sorter/memory_backend_test.go +++ b/cdc/puller/sorter/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/puller/sorter/merger.go b/cdc/puller/sorter/merger.go index 11fc381b8ce..feaf343bfcb 100644 --- a/cdc/puller/sorter/merger.go +++ b/cdc/puller/sorter/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/puller/sorter/merger_test.go b/cdc/puller/sorter/merger_test.go index afc23a47dbd..e362ec6306d 100644 --- a/cdc/puller/sorter/merger_test.go +++ b/cdc/puller/sorter/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/puller/sorter/sorterDebug", "return(true)") + err := failpoint.Enable("github.com/pingcap/tiflow/cdc/puller/sorter/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/puller/sorter/sorterDebug", "return(true)") + err := failpoint.Enable("github.com/pingcap/tiflow/cdc/puller/sorter/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/puller/sorter/sorterDebug", "return(true)") + err := failpoint.Enable("github.com/pingcap/tiflow/cdc/puller/sorter/sorterDebug", "return(true)") c.Assert(err, check.IsNil) // enable the failpoint to simulate delays - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/puller/sorter/sorterMergeDelay", "sleep(5)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/puller/sorter/sorterMergeDelay", "sleep(5)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/puller/sorter/sorterMergeDelay") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/puller/sorter/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/puller/sorter/sorterDebug", "return(true)") + err := failpoint.Enable("github.com/pingcap/tiflow/cdc/puller/sorter/sorterDebug", "return(true)") c.Assert(err, check.IsNil) // enable the failpoint to simulate delays - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/puller/sorter/sorterMergeDelay", "sleep(10)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/puller/sorter/sorterMergeDelay", "sleep(10)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/puller/sorter/sorterMergeDelay") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/puller/sorter/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/puller/sorter/sorterDebug", "return(true)") + err := failpoint.Enable("github.com/pingcap/tiflow/cdc/puller/sorter/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/puller/sorter/sorterDebug", "return(true)") + err := failpoint.Enable("github.com/pingcap/tiflow/cdc/puller/sorter/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/puller/sorter/sorterDebug", "return(true)") + err := failpoint.Enable("github.com/pingcap/tiflow/cdc/puller/sorter/sorterDebug", "return(true)") c.Assert(err, check.IsNil) - defer failpoint.Disable("github.com/pingcap/ticdc/cdc/puller/sorter/sorterDebug") //nolint:errcheck + defer failpoint.Disable("github.com/pingcap/tiflow/cdc/puller/sorter/sorterDebug") //nolint:errcheck ctx, cancel := context.WithTimeout(context.TODO(), time.Second*25) defer cancel() diff --git a/cdc/puller/sorter/serde.go b/cdc/puller/sorter/serde.go index 66ec4c9971d..990a2a2f8c6 100644 --- a/cdc/puller/sorter/serde.go +++ b/cdc/puller/sorter/serde.go @@ -15,7 +15,7 @@ package sorter import ( "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/tiflow/cdc/model" ) type msgPackGenSerde struct { diff --git a/cdc/puller/sorter/sorter_test.go b/cdc/puller/sorter/sorter_test.go index 57f2ac9fd8c..9a27f83cbeb 100644 --- a/cdc/puller/sorter/sorter_test.go +++ b/cdc/puller/sorter/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/puller" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/puller" + "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 puller.EventSorter, count int) error { - err := failpoint.Enable("github.com/pingcap/ticdc/cdc/puller/sorter/sorterDebug", "return(true)") + err := failpoint.Enable("github.com/pingcap/tiflow/cdc/puller/sorter/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/puller/sorter/asyncFlushStartDelay", "sleep(100)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/puller/sorter/asyncFlushStartDelay", "sleep(100)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/puller/sorter/asyncFlushStartDelay") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/puller/sorter/asyncFlushStartDelay") }() // enable the failpoint to simulate delays - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/puller/sorter/asyncFlushInProcessDelay", "1%sleep(1)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/puller/sorter/asyncFlushInProcessDelay", "1%sleep(1)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/puller/sorter/asyncFlushInProcessDelay") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/puller/sorter/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/puller/sorter/InjectErrorBackEndAlloc", "return(true)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/puller/sorter/InjectErrorBackEndAlloc", "return(true)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/puller/sorter/InjectErrorBackEndAlloc") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/puller/sorter/InjectErrorBackEndAlloc") }() finishedCh := make(chan struct{}) @@ -379,12 +379,12 @@ func (s *sorterSuite) TestSorterIOError(c *check.C) { } UnifiedSorterCleanUp() - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/puller/sorter/InjectErrorBackEndAlloc") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/puller/sorter/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/puller/sorter/InjectErrorBackEndWrite", "return(true)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/puller/sorter/InjectErrorBackEndWrite", "return(true)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/puller/sorter/InjectErrorBackEndWrite") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/puller/sorter/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/puller/sorter/InjectHeapSorterExitDelay", "sleep(2000)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/puller/sorter/InjectHeapSorterExitDelay", "sleep(2000)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/puller/sorter/InjectHeapSorterExitDelay") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/puller/sorter/InjectHeapSorterExitDelay") }() - err = failpoint.Enable("github.com/pingcap/ticdc/cdc/puller/sorter/InjectErrorBackEndAlloc", "return(true)") + err = failpoint.Enable("github.com/pingcap/tiflow/cdc/puller/sorter/InjectErrorBackEndAlloc", "return(true)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/cdc/puller/sorter/InjectErrorBackEndAlloc") + _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/puller/sorter/InjectErrorBackEndAlloc") }() finishedCh := make(chan struct{}) diff --git a/cdc/puller/sorter/unified_sorter.go b/cdc/puller/sorter/unified_sorter.go index a8f84abc900..19dff63c1d0 100644 --- a/cdc/puller/sorter/unified_sorter.go +++ b/cdc/puller/sorter/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/cdc/server.go b/cdc/server.go index 1257319d320..6c9f2098d0f 100644 --- a/cdc/server.go +++ b/cdc/server.go @@ -24,15 +24,15 @@ 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/puller/sorter" - "github.com/pingcap/ticdc/pkg/config" - cerror "github.com/pingcap/ticdc/pkg/errors" - "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/puller/sorter" + "github.com/pingcap/tiflow/pkg/config" + cerror "github.com/pingcap/tiflow/pkg/errors" + "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 1020168c8d5..5552b802012 100644 --- a/cdc/server_test.go +++ b/cdc/server_test.go @@ -20,12 +20,12 @@ import ( "time" "github.com/pingcap/check" - "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/util" - "github.com/pingcap/ticdc/pkg/util/testleak" + "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/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/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 674124a070e..caf456a88c5 100644 --- a/cdc/sink/causality_test.go +++ b/cdc/sink/causality_test.go @@ -19,8 +19,8 @@ import ( "github.com/pingcap/check" "github.com/pingcap/parser/mysql" - "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 testCausalitySuite struct{} diff --git a/cdc/sink/cdclog/file.go b/cdc/sink/cdclog/file.go index 1e66bb47a9b..0a76d1a5fc2 100644 --- a/cdc/sink/cdclog/file.go +++ b/cdc/sink/cdclog/file.go @@ -22,10 +22,10 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" parsemodel "github.com/pingcap/parser/model" - "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/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 e0284b9e51e..2249628358f 100644 --- a/cdc/sink/cdclog/s3.go +++ b/cdc/sink/cdclog/s3.go @@ -23,11 +23,11 @@ import ( backup "github.com/pingcap/kvproto/pkg/brpb" "github.com/pingcap/log" parsemodel "github.com/pingcap/parser/model" - "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" + "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 7a7b2b3feef..bb06324c88c 100644 --- a/cdc/sink/codec/avro.go +++ b/cdc/sink/codec/avro.go @@ -27,9 +27,9 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/parser/mysql" - "github.com/pingcap/ticdc/cdc/model" - cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/tidb/types" + "github.com/pingcap/tiflow/cdc/model" + cerror "github.com/pingcap/tiflow/pkg/errors" "go.uber.org/zap" ) diff --git a/cdc/sink/codec/avro_test.go b/cdc/sink/codec/avro_test.go index 779be79c451..087b4874fd9 100644 --- a/cdc/sink/codec/avro_test.go +++ b/cdc/sink/codec/avro_test.go @@ -23,12 +23,12 @@ import ( "github.com/pingcap/log" model2 "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" - "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" "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 89fc7f4a4eb..2dba506a0d0 100644 --- a/cdc/sink/codec/canal.go +++ b/cdc/sink/codec/canal.go @@ -24,9 +24,9 @@ import ( mm "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" parser_types "github.com/pingcap/parser/types" - "github.com/pingcap/ticdc/cdc/model" - cerror "github.com/pingcap/ticdc/pkg/errors" - canal "github.com/pingcap/ticdc/proto/canal" + "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 68115283769..151ecece2fb 100644 --- a/cdc/sink/codec/canal_flat.go +++ b/cdc/sink/codec/canal_flat.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" - 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 5e623f533bf..1ca30741756 100644 --- a/cdc/sink/codec/canal_flat_test.go +++ b/cdc/sink/codec/canal_flat_test.go @@ -19,8 +19,8 @@ import ( "github.com/pingcap/check" mm "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" - "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" "golang.org/x/text/encoding/charmap" ) diff --git a/cdc/sink/codec/canal_test.go b/cdc/sink/codec/canal_test.go index a14ea518919..1ccb94f5409 100644 --- a/cdc/sink/codec/canal_test.go +++ b/cdc/sink/codec/canal_test.go @@ -20,9 +20,9 @@ import ( "github.com/pingcap/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 2c0af292cdc..76b066e11b9 100644 --- a/cdc/sink/codec/codec_test.go +++ b/cdc/sink/codec/codec_test.go @@ -20,10 +20,10 @@ import ( "github.com/pingcap/check" "github.com/pingcap/parser/mysql" - "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/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 bc03b9837ed..e0557f6ae43 100644 --- a/cdc/sink/codec/craft.go +++ b/cdc/sink/codec/craft.go @@ -18,9 +18,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 1e51ae2ede5..38712e2bb1e 100644 --- a/cdc/sink/codec/craft/decoder.go +++ b/cdc/sink/codec/craft/decoder.go @@ -21,8 +21,8 @@ import ( "github.com/pingcap/errors" pmodel "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" - "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" ) /// create string from byte slice without copying diff --git a/cdc/sink/codec/craft/encoder.go b/cdc/sink/codec/craft/encoder.go index 9a56af27ca4..b915ceb1b71 100644 --- a/cdc/sink/codec/craft/encoder.go +++ b/cdc/sink/codec/craft/encoder.go @@ -19,7 +19,7 @@ import ( "unsafe" "github.com/pingcap/parser/mysql" - "github.com/pingcap/ticdc/cdc/model" + "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 793683cc710..1a4c3f49aae 100644 --- a/cdc/sink/codec/craft_test.go +++ b/cdc/sink/codec/craft_test.go @@ -19,8 +19,8 @@ import ( "github.com/pingcap/check" "github.com/pingcap/parser/mysql" - "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 craftBatchSuite struct { diff --git a/cdc/sink/codec/interface.go b/cdc/sink/codec/interface.go index ae4599ff6b7..d1d028f92b1 100644 --- a/cdc/sink/codec/interface.go +++ b/cdc/sink/codec/interface.go @@ -19,7 +19,7 @@ import ( "time" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/tiflow/cdc/model" "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 c69b2545ded..21314cc378b 100644 --- a/cdc/sink/codec/interface_test.go +++ b/cdc/sink/codec/interface_test.go @@ -18,8 +18,8 @@ import ( timodel "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/types" - "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 codecInterfaceSuite struct { diff --git a/cdc/sink/codec/json.go b/cdc/sink/codec/json.go index 8ae135a47a7..d4e61faad6f 100644 --- a/cdc/sink/codec/json.go +++ b/cdc/sink/codec/json.go @@ -26,10 +26,10 @@ import ( "github.com/pingcap/log" timodel "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" - cerror "github.com/pingcap/ticdc/pkg/errors" + 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 497434fd662..048b2905273 100644 --- a/cdc/sink/codec/json_test.go +++ b/cdc/sink/codec/json_test.go @@ -21,8 +21,8 @@ import ( "github.com/pingcap/check" "github.com/pingcap/parser/mysql" - "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/codec/maxwell.go b/cdc/sink/codec/maxwell.go index 7ae7366cf8f..2b495e6986d 100644 --- a/cdc/sink/codec/maxwell.go +++ b/cdc/sink/codec/maxwell.go @@ -21,8 +21,8 @@ import ( "github.com/pingcap/errors" model2 "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" - "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" "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 a7106b548b8..bed095edf48 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 9e95e8eee56..967f43c58a9 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 ae80bc7df9e..56cf9a8e59e 100644 --- a/cdc/sink/manager.go +++ b/cdc/sink/manager.go @@ -23,8 +23,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" "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 354b06563ae..002b476a452 100644 --- a/cdc/sink/manager_test.go +++ b/cdc/sink/manager_test.go @@ -25,8 +25,8 @@ import ( "github.com/pingcap/errors" "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 managerSuite struct{} @@ -249,7 +249,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 5b8e838b9f9..d3142590758 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -22,18 +22,18 @@ 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/ticdc/pkg/util" + "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" + "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" "golang.org/x/sync/errgroup" ) diff --git a/cdc/sink/mq_test.go b/cdc/sink/mq_test.go index 40421fb3c30..31c43fd6f86 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{} @@ -197,7 +197,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 7a1946f3824..4d694c15d2a 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -31,21 +31,21 @@ import ( "github.com/pingcap/log" timodel "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" - "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" + "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 f59828894e5..29602302f49 100644 --- a/cdc/sink/mysql_test.go +++ b/cdc/sink/mysql_test.go @@ -33,16 +33,16 @@ import ( "github.com/pingcap/errors" timodel "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" - "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" + "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 aa77e2059df..fc1aaa374dc 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/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/task.go b/cdc/task.go index 239ccfd46b4..efd27618294 100644 --- a/cdc/task.go +++ b/cdc/task.go @@ -18,8 +18,8 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/log" - "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" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/mvcc" "go.uber.org/zap" diff --git a/cdc/task_test.go b/cdc/task_test.go index 3b48300c275..c97f2a5ff2f 100644 --- a/cdc/task_test.go +++ b/cdc/task_test.go @@ -20,10 +20,10 @@ import ( "github.com/pingcap/check" "github.com/pingcap/failpoint" - "github.com/pingcap/ticdc/cdc/kv" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/kv" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/util/testleak" "go.etcd.io/etcd/clientv3" "go.etcd.io/etcd/embed" ) @@ -159,7 +159,7 @@ func (s *taskSuite) TestWatch(c *check.C) { ch := s.w.Watch(context.Background()) // Trigger the ErrCompacted error - c.Assert(failpoint.Enable("github.com/pingcap/ticdc/cdc.restart_task_watch", "50%off"), check.IsNil) + c.Assert(failpoint.Enable("github.com/pingcap/tiflow/cdc.restart_task_watch", "50%off"), check.IsNil) // Put task changefeed-1 c.Assert(client.PutTaskStatus(s.c.Ctx(), "changefeed-1", 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 3d00ca88ac5..1518d636b65 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/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 20e0e3c74ac..20535a3eeb0 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/go.mod b/go.mod index 09f1a414b56..6cbe143d0a6 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 59ff6eae031..b397c8c69d4 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 faabd24ea1b..96016c308f8 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 9f3c896b059..08a0521dcc9 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 f9acf162f8f..567faa9da7a 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/cmd/cli/cli.go b/pkg/cmd/cli/cli.go index 28ab2bfb7d7..3102c8cfe21 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 a15c7a319fd..3e04b182fed 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 1fa75e5a1bb..2306a412d91 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" - "github.com/pingcap/ticdc/cdc/kv" - 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/tiflow/cdc/kv" + 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/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 c7e24e3b439..2a572b7f7c4 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 af6e916ab64..1e492a9a060 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/kv" - "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/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/kv" + "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/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 a183987721f..125e56a83f7 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 11ced30ee97..04d68db8392 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 692138bd861..4e63b82c08e 100644 --- a/pkg/cmd/cli/cli_changefeed_helper.go +++ b/pkg/cmd/cli/cli_changefeed_helper.go @@ -21,15 +21,15 @@ 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/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/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 a73eb7f19d6..99403a07fc4 100644 --- a/pkg/cmd/cli/cli_changefeed_helper_test.go +++ b/pkg/cmd/cli/cli_changefeed_helper_test.go @@ -19,7 +19,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 2065ba239e7..3f2558711b5 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/cdc/kv" - "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/security" + "github.com/pingcap/tiflow/cdc" + "github.com/pingcap/tiflow/cdc/kv" + "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/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 18b99253955..270cd9fa8ea 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/kv" - "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/security" + "github.com/pingcap/tiflow/cdc/kv" + "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/security" "github.com/spf13/cobra" ) diff --git a/pkg/cmd/cli/cli_changefeed_query.go b/pkg/cmd/cli/cli_changefeed_query.go index c4231e60f49..236db5f3ca3 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/kv" - "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/security" + "github.com/pingcap/tiflow/cdc/kv" + "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/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 6865bf3a7a7..70531f458a6 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/kv" - "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/security" + "github.com/pingcap/tiflow/cdc/kv" + "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/security" "github.com/spf13/cobra" ) diff --git a/pkg/cmd/cli/cli_changefeed_resume.go b/pkg/cmd/cli/cli_changefeed_resume.go index 946082d3b3e..eddf0432a02 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/kv" - "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/security" + "github.com/pingcap/tiflow/cdc" + "github.com/pingcap/tiflow/cdc/kv" + "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/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 ee0f8fba965..9d1d5e6bc4e 100644 --- a/pkg/cmd/cli/cli_changefeed_statistics.go +++ b/pkg/cmd/cli/cli_changefeed_statistics.go @@ -17,10 +17,10 @@ import ( "fmt" "time" - "github.com/pingcap/ticdc/cdc/kv" - "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/cdc/kv" + "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 1b2dd9cd6a2..e5a021c5c3a 100644 --- a/pkg/cmd/cli/cli_changefeed_update.go +++ b/pkg/cmd/cli/cli_changefeed_update.go @@ -19,12 +19,12 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/kv" - "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/kv" + "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 4046afaa617..4866b32f6b9 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 46f3f3158db..960e4eded8d 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 8f2d8289857..f55f2d1c429 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/cdc/kv" - "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/cdc/kv" + "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_query.go b/pkg/cmd/cli/cli_processor_query.go index 3fd27943bfb..e7a48564802 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/kv" - "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/tiflow/cdc/kv" + "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/spf13/cobra" ) 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 726d4d597b7..002287ab0b2 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 9b769eaf272..462fbb68f89 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 febf7a5d3b7..bcc3107b773 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/cdc/kv" - "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/cdc/kv" + "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_show_metadata.go b/pkg/cmd/cli/cli_unsafe_show_metadata.go index 31e51189597..5f6058755b7 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/cdc/kv" - "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/cmd/factory" + "github.com/pingcap/tiflow/cdc/kv" + "github.com/pingcap/tiflow/pkg/cmd/context" + "github.com/pingcap/tiflow/pkg/cmd/factory" "github.com/spf13/cobra" ) diff --git a/pkg/cmd/cmd.go b/pkg/cmd/cmd.go index 064a22a79a2..d6d844d5121 100644 --- a/pkg/cmd/cmd.go +++ b/pkg/cmd/cmd.go @@ -16,9 +16,9 @@ package cmd import ( "os" - "github.com/pingcap/ticdc/pkg/cmd/cli" - "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/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 16d02e8d7d8..857d6d92f83 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/cdc/kv" - "github.com/pingcap/ticdc/pkg/cmd/util" - "github.com/pingcap/ticdc/pkg/security" + "github.com/pingcap/tiflow/cdc/kv" + "github.com/pingcap/tiflow/pkg/cmd/util" + "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 b9f4014773e..c6e9f0c756d 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" - "github.com/pingcap/ticdc/cdc/kv" - cmdconetxt "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/security" - "github.com/pingcap/ticdc/pkg/version" + "github.com/pingcap/tiflow/cdc/kv" + cmdconetxt "github.com/pingcap/tiflow/pkg/cmd/context" + "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() (*kv.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(), @@ -160,7 +160,7 @@ func (f factoryImpl) PdClient() (pd.Client, error) { } // TODO: we need to check all pd endpoint and make sure they belong to the same cluster. - // See also: https://github.com/pingcap/ticdc/pull/2341#discussion_r673021305. + // See also: https://github.com/pingcap/tiflow/pull/2341#discussion_r673021305. err = version.CheckClusterVersion(ctx, pdClient, pdEndpoints[0], credential, true) if err != nil { return nil, err diff --git a/pkg/cmd/server/server.go b/pkg/cmd/server/server.go index 1f5288334af..3a53351da01 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/puller/sorter" - 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/puller/sorter" + 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 16a00b5e5aa..da9f79874a6 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 86a20eb9e05..4633baf813a 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" - "github.com/pingcap/ticdc/cdc/kv" - cmdconetxt "github.com/pingcap/ticdc/pkg/cmd/context" - "github.com/pingcap/ticdc/pkg/logutil" - "github.com/pingcap/ticdc/pkg/version" + "github.com/pingcap/tiflow/cdc/kv" + cmdconetxt "github.com/pingcap/tiflow/pkg/cmd/context" + "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 6a1ba817536..69e0d549e14 100644 --- a/pkg/cmd/util/helper_test.go +++ b/pkg/cmd/util/helper_test.go @@ -22,8 +22,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 84c5630f529..de146df9bd7 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 13159e0e4c9..d5475595cb1 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/config_test.go b/pkg/config/config_test.go index abb4286e980..0b388a6083f 100644 --- a/pkg/config/config_test.go +++ b/pkg/config/config_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/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 fe129cd907a..0b02665304a 100644 --- a/pkg/context/context.go +++ b/pkg/context/context.go @@ -18,12 +18,12 @@ 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/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/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 b0caee01c3f..a0ae1a782e6 100644 --- a/pkg/context/context_test.go +++ b/pkg/context/context_test.go @@ -20,9 +20,9 @@ import ( "github.com/pingcap/check" "github.com/pingcap/errors" - "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" ) func TestSuite(t *testing.T) { check.TestingT(t) } diff --git a/pkg/cyclic/filter.go b/pkg/cyclic/filter.go index 94b866e2590..d33634699f7 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 4d3bdb550d1..f20811d7ef5 100644 --- a/pkg/cyclic/filter_test.go +++ b/pkg/cyclic/filter_test.go @@ -18,9 +18,9 @@ import ( "github.com/davecgh/go-spew/spew" "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/cyclic/mark" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/cyclic/mark" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type markSuite struct{} 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/mark/mark_test.go b/pkg/cyclic/mark/mark_test.go index cdbb66c8d9e..89f75c28685 100644 --- a/pkg/cyclic/mark/mark_test.go +++ b/pkg/cyclic/mark/mark_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 markSuite struct{} 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 71d6668590e..aec74535151 100644 --- a/pkg/cyclic/replication_test.go +++ b/pkg/cyclic/replication_test.go @@ -17,10 +17,10 @@ import ( "testing" "github.com/pingcap/check" - "github.com/pingcap/ticdc/cdc/model" - "github.com/pingcap/ticdc/pkg/config" - "github.com/pingcap/ticdc/pkg/cyclic/mark" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/cyclic/mark" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type cyclicSuite struct{} diff --git a/pkg/errors/helper_test.go b/pkg/errors/helper_test.go index eb22439b47a..6f0b6ec7d29 100644 --- a/pkg/errors/helper_test.go +++ b/pkg/errors/helper_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" ) func TestSuite(t *testing.T) { 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 88d248fc409..04a81787d16 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_test.go b/pkg/etcd/etcd_test.go index c414e3a6238..0edcf44a9c1 100644 --- a/pkg/etcd/etcd_test.go +++ b/pkg/etcd/etcd_test.go @@ -20,7 +20,7 @@ import ( "time" "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 ca7ac9cfe15..ad1975a309f 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 b5b7f5843d4..32d45942175 100644 --- a/pkg/filter/filter.go +++ b/pkg/filter/filter.go @@ -15,11 +15,11 @@ package filter import ( "github.com/pingcap/parser/model" - "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/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 63cf425f074..8408413031a 100644 --- a/pkg/filter/filter_test.go +++ b/pkg/filter/filter_test.go @@ -16,8 +16,8 @@ package filter import ( "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/pingcap/check" "github.com/pingcap/parser/model" 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/flags/urls_test.go b/pkg/flags/urls_test.go index 382a10e46ab..a715bc8dfa6 100644 --- a/pkg/flags/urls_test.go +++ b/pkg/flags/urls_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) { 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 c1149b49495..12f3af46c5e 100644 --- a/pkg/httputil/httputil_test.go +++ b/pkg/httputil/httputil_test.go @@ -25,9 +25,9 @@ import ( "testing" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/security" - "github.com/pingcap/ticdc/pkg/util/testleak" "github.com/pingcap/tidb-tools/pkg/utils" + "github.com/pingcap/tiflow/pkg/security" + "github.com/pingcap/tiflow/pkg/util/testleak" ) func Test(t *testing.T) { check.TestingT(t) } 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/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 e2517988f8e..f6b3c6078be 100644 --- a/pkg/notify/notify_test.go +++ b/pkg/notify/notify_test.go @@ -19,8 +19,8 @@ import ( "time" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/util/testleak" ) func Test(t *testing.T) { 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 661f93751bf..001c3a401c2 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_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 342ca407cf9..5fc6fbc08dd 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 f232b56b027..2536c859fc3 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/acquirer_test.go b/pkg/pdtime/acquirer_test.go index 92ab915b78e..697998369d6 100644 --- a/pkg/pdtime/acquirer_test.go +++ b/pkg/pdtime/acquirer_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" "github.com/tikv/client-go/v2/oracle" pd "github.com/tikv/pd/client" ) diff --git a/pkg/pipeline/context.go b/pkg/pipeline/context.go index f3de42a1e74..49b3214adb7 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/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 5cc827c8089..4d868585dba 100644 --- a/pkg/pipeline/pipeline_test.go +++ b/pkg/pipeline/pipeline_test.go @@ -22,10 +22,10 @@ import ( "github.com/pingcap/check" "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/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/context" + cerror "github.com/pingcap/tiflow/pkg/errors" + "github.com/pingcap/tiflow/pkg/util/testleak" "go.uber.org/zap" ) @@ -500,7 +500,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 8e7858582f5..8288b8c9ed9 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 60d0ab73d02..bfc25ac69b8 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/quotes_test.go b/pkg/quotes/quotes_test.go index 82afd8bef16..ce411377553 100644 --- a/pkg/quotes/quotes_test.go +++ b/pkg/quotes/quotes_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/regionspan/region_range_lock_test.go b/pkg/regionspan/region_range_lock_test.go index 8842e765141..8ce725706a3 100644 --- a/pkg/regionspan/region_range_lock_test.go +++ b/pkg/regionspan/region_range_lock_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 regionRangeLockSuite struct{} diff --git a/pkg/regionspan/region_test.go b/pkg/regionspan/region_test.go index ca1336519f2..1784240ad95 100644 --- a/pkg/regionspan/region_test.go +++ b/pkg/regionspan/region_test.go @@ -16,7 +16,7 @@ package regionspan import ( "github.com/pingcap/check" "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type regionSuite struct{} 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/regionspan/span_test.go b/pkg/regionspan/span_test.go index 46940795816..215ba7819d1 100644 --- a/pkg/regionspan/span_test.go +++ b/pkg/regionspan/span_test.go @@ -17,8 +17,8 @@ import ( "testing" "github.com/pingcap/check" - "github.com/pingcap/ticdc/pkg/util/testleak" "github.com/pingcap/tidb/tablecodec" + "github.com/pingcap/tiflow/pkg/util/testleak" ) type spanSuite struct{} diff --git a/pkg/retry/retry_test.go b/pkg/retry/retry_test.go index 96a8e72cd3e..375cc1380c4 100644 --- a/pkg/retry/retry_test.go +++ b/pkg/retry/retry_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" ) func Test(t *testing.T) { check.TestingT(t) } 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/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 94ce5bc3364..f790a26cae7 100644 --- a/pkg/scheduler/table_number_test.go +++ b/pkg/scheduler/table_number_test.go @@ -16,8 +16,8 @@ package scheduler import ( "fmt" - "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" "github.com/pingcap/check" ) 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 791d89b5b3c..59f3794ab15 100644 --- a/pkg/scheduler/workload_test.go +++ b/pkg/scheduler/workload_test.go @@ -17,8 +17,8 @@ import ( "fmt" "testing" - "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" "github.com/pingcap/check" ) 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/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 314631c3868..c415ebf6b17 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/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/types/urls_test.go b/pkg/types/urls_test.go index 0adbc8c86e5..6fe6bd1c326 100644 --- a/pkg/types/urls_test.go +++ b/pkg/types/urls_test.go @@ -18,7 +18,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) { 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 4b734e88de3..5ce00bbd531 100644 --- a/pkg/util/fileutil.go +++ b/pkg/util/fileutil.go @@ -23,8 +23,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 e498107d962..00e54ac297c 100644 --- a/pkg/util/fileutil_test.go +++ b/pkg/util/fileutil_test.go @@ -22,8 +22,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{} @@ -98,8 +98,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 2fed765dbc6..31c780ceaea 100644 --- a/pkg/util/testleak/leaktest.go +++ b/pkg/util/testleak/leaktest.go @@ -52,11 +52,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/puller/sorter.newBackEndPool", - "github.com/pingcap/ticdc/cdc/puller/sorter.(*heapSorter).flush", + "github.com/pingcap/tiflow/cdc/puller/sorter.newBackEndPool", + "github.com/pingcap/tiflow/cdc/puller/sorter.(*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 632e9810860..726c08ee7e5 100644 --- a/pkg/util/tz.go +++ b/pkg/util/tz.go @@ -19,7 +19,7 @@ import ( "strings" "time" - cerror "github.com/pingcap/ticdc/pkg/errors" + 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 72b416781ab..afe786e7f46 100644 --- a/pkg/version/check.go +++ b/pkg/version/check.go @@ -22,14 +22,14 @@ import ( "regexp" "strings" - "github.com/pingcap/ticdc/cdc/model" + "github.com/pingcap/tiflow/cdc/model" "github.com/coreos/go-semver/semver" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" - cerror "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/httputil" - "github.com/pingcap/ticdc/pkg/security" + 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 0515d1e6567..b4b76e0179d 100644 --- a/pkg/version/check_test.go +++ b/pkg/version/check_test.go @@ -24,8 +24,8 @@ import ( "github.com/coreos/go-semver/semver" "github.com/pingcap/check" "github.com/pingcap/kvproto/pkg/metapb" - "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" pd "github.com/tikv/pd/client" "github.com/tikv/pd/pkg/tempurl" ) 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/async_pool_test.go b/pkg/workerpool/async_pool_test.go index d1e2107c607..0161b4e2ffc 100644 --- a/pkg/workerpool/async_pool_test.go +++ b/pkg/workerpool/async_pool_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/check" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" "golang.org/x/sync/errgroup" ) diff --git a/pkg/workerpool/pool_impl.go b/pkg/workerpool/pool_impl.go index d98d4ffc1a8..97af0aaec33 100644 --- a/pkg/workerpool/pool_impl.go +++ b/pkg/workerpool/pool_impl.go @@ -23,8 +23,8 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" - 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 c34378fd625..729c20d7a31 100644 --- a/pkg/workerpool/pool_test.go +++ b/pkg/workerpool/pool_test.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/log" - "github.com/pingcap/ticdc/pkg/util/testleak" + "github.com/pingcap/tiflow/pkg/util/testleak" "github.com/stretchr/testify/require" "go.uber.org/zap" "golang.org/x/sync/errgroup" @@ -207,10 +207,10 @@ func (s *workerPoolSuite) TestCancelHandle(c *check.C) { } } - 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)") c.Assert(err, check.IsNil) defer func() { - _ = failpoint.Disable("github.com/pingcap/ticdc/pkg/workerpool/addEventDelayPoint") + _ = failpoint.Disable("github.com/pingcap/tiflow/pkg/workerpool/addEventDelayPoint") }() handle.Unregister() @@ -240,10 +240,10 @@ func (s *workerPoolSuite) TestCancelTimer(c *check.C) { 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)") c.Assert(err, check.IsNil) 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 { @@ -460,7 +460,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 ee536bd0328..655770b96e6 100644 --- a/testing_utils/cdc_state_checker/cdc_monitor.go +++ b/testing_utils/cdc_state_checker/cdc_monitor.go @@ -17,14 +17,14 @@ import ( "context" "time" - "github.com/pingcap/ticdc/pkg/security" + "github.com/pingcap/tiflow/pkg/security" "github.com/pingcap/log" "github.com/pingcap/errors" - "github.com/pingcap/ticdc/cdc/kv" - "github.com/pingcap/ticdc/pkg/etcd" - "github.com/pingcap/ticdc/pkg/orchestrator" + "github.com/pingcap/tiflow/cdc/kv" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/orchestrator" "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 b1408f91d19..5f639e3a103 100644 --- a/testing_utils/cdc_state_checker/main.go +++ b/testing_utils/cdc_state_checker/main.go @@ -18,7 +18,7 @@ import ( "flag" "strings" - "github.com/pingcap/ticdc/pkg/security" + "github.com/pingcap/tiflow/pkg/security" "github.com/pingcap/log" "go.uber.org/zap" 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 ffc371a5c35..9cdbf1f8ac6 100644 --- a/testing_utils/cdc_state_checker/state.go +++ b/testing_utils/cdc_state_checker/state.go @@ -21,10 +21,10 @@ 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/pkg/orchestrator" - "github.com/pingcap/ticdc/pkg/orchestrator/util" + "github.com/pingcap/tiflow/cdc/kv" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/pkg/orchestrator" + "github.com/pingcap/tiflow/pkg/orchestrator/util" ) type cdcReactorState struct { diff --git a/testing_utils/many_sorters_test/many_sorters.go b/testing_utils/many_sorters_test/many_sorters.go index d3dc5dd807b..04eedddc0bf 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/puller" - pullerSorter "github.com/pingcap/ticdc/cdc/puller/sorter" - "github.com/pingcap/ticdc/pkg/config" - cerrors "github.com/pingcap/ticdc/pkg/errors" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/puller" + pullerSorter "github.com/pingcap/tiflow/cdc/puller/sorter" + "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/puller/sorter/sorterDebug", "return(true)") + err := failpoint.Enable("github.com/pingcap/tiflow/cdc/puller/sorter/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 ef9a0aacac6..abf1a58548d 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/puller" - pullerSorter "github.com/pingcap/ticdc/cdc/puller/sorter" - "github.com/pingcap/ticdc/pkg/config" + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/puller" + pullerSorter "github.com/pingcap/tiflow/cdc/puller/sorter" + "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/puller/sorter/sorterDebug", "return(true)") + err := failpoint.Enable("github.com/pingcap/tiflow/cdc/puller/sorter/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 290c7f3a8fe..6303e24063f 100755 --- a/tests/integration_tests/_utils/run_cdc_cli_tso_query +++ b/tests/integration_tests/_utils/run_cdc_cli_tso_query @@ -11,7 +11,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 f9d5f652eec..bf2ce93e67c 100755 --- a/tests/integration_tests/availability/owner.sh +++ b/tests/integration_tests/availability/owner.sh @@ -159,8 +159,8 @@ 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-campaign-compacted-error=1*return(true)' # old owner - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/capture/capture-campaign-compacted-error=1*return(true)' # new owner + # export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/capture-campaign-compacted-error=1*return(true)' # old owner + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/capture/capture-campaign-compacted-error=1*return(true)' # new owner # start a capture server run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix test_owner_retryable_error.server1 @@ -172,8 +172,8 @@ function test_owner_retryable_error() { echo "owner pid:" $owner_pid echo "owner id" $owner_id - # export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/owner-run-with-error=1*return(true);github.com/pingcap/ticdc/cdc/capture-resign-failed=1*return(true)' # old owner - 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)' # new owner + # export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner-run-with-error=1*return(true);github.com/pingcap/tiflow/cdc/capture-resign-failed=1*return(true)' # old owner + 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)' # new owner # run another server run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix test_owner_retryable_error.server2 --addr "127.0.0.1:8301" @@ -198,8 +198,8 @@ 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/sleep-before-watch-capture=1*sleep(6000)' # old owner - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/owner/sleep-in-owner-tick=1*sleep(6000)' # new owner + # export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/sleep-before-watch-capture=1*sleep(6000)' # old owner + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/sleep-in-owner-tick=1*sleep(6000)' # new owner # 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 96c73e98a8d..91c2ad133e5 100644 --- a/tests/integration_tests/capture_session_done_during_task/run.sh +++ b/tests/integration_tests/capture_session_done_during_task/run.sh @@ -28,8 +28,8 @@ 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 --pd=http://$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/captureHandleTaskDelay=sleep(2000)' # old processor - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/processor/processorManagerHandleNewChangefeedDelay=sleep(2000)' # new processor + # export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/captureHandleTaskDelay=sleep(2000)' # old processor + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/processor/processorManagerHandleNewChangefeedDelay=sleep(2000)' # new processor 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 0d93ccc9128..bc2f8e589ac 100755 --- a/tests/integration_tests/changefeed_auto_stop/run.sh +++ b/tests/integration_tests/changefeed_auto_stop/run.sh @@ -44,8 +44,8 @@ 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/ProcessorSyncResolvedError=1*return(true);github.com/pingcap/ticdc/cdc/ProcessorUpdatePositionDelaying=return(true)' # old processor - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/processor/pipeline/ProcessorSyncResolvedError=1*return(true);github.com/pingcap/ticdc/cdc/processor/ProcessorUpdatePositionDelaying=sleep(1000)' # new processor + # export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/ProcessorSyncResolvedError=1*return(true);github.com/pingcap/tiflow/cdc/ProcessorUpdatePositionDelaying=return(true)' # old processor + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/processor/pipeline/ProcessorSyncResolvedError=1*return(true);github.com/pingcap/tiflow/cdc/processor/ProcessorUpdatePositionDelaying=sleep(1000)' # new processor 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 3136497279d..3696a514d7e 100755 --- a/tests/integration_tests/changefeed_error/run.sh +++ b/tests/integration_tests/changefeed_error/run.sh @@ -114,8 +114,8 @@ function run() { start_ts=$(run_cdc_cli tso query --pd=http://$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/NewChangefeedNoRetryError=1*return(true)' # old owner - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/owner/NewChangefeedNoRetryError=1*return(true)' # new owner + # export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/NewChangefeedNoRetryError=1*return(true)' # old owner + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/NewChangefeedNoRetryError=1*return(true)' # new owner run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY capture_pid=$(ps -C $CDC_BINARY -o pid= | awk '{print $1}') @@ -139,8 +139,8 @@ 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/NewChangefeedRetryError=return(true)' # old owner - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/owner/NewChangefeedRetryError=return(true)' # new owner + # export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/NewChangefeedRetryError=return(true)' # old owner + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/NewChangefeedRetryError=return(true)' # new owner 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 @@ -153,8 +153,8 @@ function run() { cleanup_process $CDC_BINARY # owner DDL error case - # export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/InjectChangefeedDDLError=return(true)' # old owner - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/owner/InjectChangefeedDDLError=return(true)' # new owner + # export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/InjectChangefeedDDLError=return(true)' # old owner + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/InjectChangefeedDDLError=return(true)' # new owner 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 @@ -166,7 +166,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)' # new owner + export GO_FAILPOINTS='github.com/pingcap/tiflow/pkg/txnutil/gc/InjectActualGCSafePoint=return(9223372036854775807)' # new owner 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/dailytest/db.go b/tests/integration_tests/dailytest/db.go index 76f4f90c7be..db7c5f7968c 100644 --- a/tests/integration_tests/dailytest/db.go +++ b/tests/integration_tests/dailytest/db.go @@ -24,7 +24,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/parser/mysql" - "github.com/pingcap/ticdc/tests/integration_tests/util" + "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 c94609eb6f5..2d4a2d1732e 100644 --- a/tests/integration_tests/ddl_puller_lag/run.sh +++ b/tests/integration_tests/ddl_puller_lag/run.sh @@ -21,8 +21,8 @@ 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/processorDDLResolved=1*sleep(180000)' # old processor - run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --failpoint 'github.com/pingcap/ticdc/cdc/processor/processorDDLResolved=1*sleep(180000)' # new processor + # run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --failpoint 'github.com/pingcap/tiflow/cdc/processorDDLResolved=1*sleep(180000)' # old processor + run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --failpoint 'github.com/pingcap/tiflow/cdc/processor/processorDDLResolved=1*sleep(180000)' # new processor 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 ea2ab6987a5..37348de0ade 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)' # new owner + export GO_FAILPOINTS='github.com/pingcap/tiflow/pkg/txnutil/gc/InjectGcSafepointUpdateInterval=return(500)' # new owner 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 eaac26f6b26..cc5ff1ebb74 100755 --- a/tests/integration_tests/kill_owner_with_ddl/run.sh +++ b/tests/integration_tests/kill_owner_with_ddl/run.sh @@ -56,8 +56,8 @@ 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/ownerFlushIntervalInject=return(0)' # old owner - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/sink/MySQLSinkExecDDLDelay=return(true);github.com/pingcap/ticdc/cdc/capture/ownerFlushIntervalInject=return(10)' # new owner + # export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/sink/MySQLSinkExecDDLDelay=return(true);github.com/pingcap/tiflow/cdc/ownerFlushIntervalInject=return(0)' # old owner + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/sink/MySQLSinkExecDDLDelay=return(true);github.com/pingcap/tiflow/cdc/capture/ownerFlushIntervalInject=return(10)' # new owner 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 0321660fe2e..e43b2edd87b 100644 --- a/tests/integration_tests/move_table/main.go +++ b/tests/integration_tests/move_table/main.go @@ -26,9 +26,9 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" - "github.com/pingcap/ticdc/cdc/kv" - cerrors "github.com/pingcap/ticdc/pkg/errors" - "github.com/pingcap/ticdc/pkg/retry" + "github.com/pingcap/tiflow/cdc/kv" + 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 ab8cf878862..d04cc6290b3 100644 --- a/tests/integration_tests/owner_remove_table_error/run.sh +++ b/tests/integration_tests/owner_remove_table_error/run.sh @@ -25,8 +25,8 @@ 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/OwnerRemoveTableError=1*return(true)' # old owner - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/owner/OwnerRemoveTableError=1*return(true)' # new owner + # export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/OwnerRemoveTableError=1*return(true)' # old owner + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/OwnerRemoveTableError=1*return(true)' # new owner 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 5d8ea183c9d..b8cdb2c6716 100644 --- a/tests/integration_tests/processor_err_chan/run.sh +++ b/tests/integration_tests/processor_err_chan/run.sh @@ -47,8 +47,8 @@ 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/ProcessorAddTableError=1*return(true)' # old processor - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/processor/pipeline/ProcessorAddTableError=1*return(true)' # new processor + # export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/ProcessorAddTableError=1*return(true)' # old processor + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/processor/pipeline/ProcessorAddTableError=1*return(true)' # new processor 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 9023448fc47..b7b37b60dc3 100644 --- a/tests/integration_tests/processor_panic/run.sh +++ b/tests/integration_tests/processor_panic/run.sh @@ -19,10 +19,10 @@ function prepare() { start_ts=$(run_cdc_cli tso query --pd=http://$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/ProcessorSyncResolvedPreEmit=return(true)' # old processor + # --failpoint 'github.com/pingcap/tiflow/cdc/ProcessorSyncResolvedPreEmit=return(true)' # old processor 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)' # new processor + --failpoint 'github.com/pingcap/tiflow/cdc/processor/pipeline/ProcessorSyncResolvedPreEmit=return(true)' # new processor 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 4b7706c047a..b7289286bfc 100644 --- a/tests/integration_tests/processor_stop_delay/run.sh +++ b/tests/integration_tests/processor_stop_delay/run.sh @@ -20,8 +20,8 @@ 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/processorStopDelay=1*sleep(10000)' # old processor - export GO_FAILPOINTS='github.com/pingcap/ticdc/cdc/processor/processorStopDelay=1*sleep(10000)' # new processor + # export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/processorStopDelay=1*sleep(10000)' # old processor + export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/processor/processorStopDelay=1*sleep(10000)' # new processor 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 db4dff8b24f..fe7e9b26dd6 100644 --- a/tests/integration_tests/resolve_lock/main.go +++ b/tests/integration_tests/resolve_lock/main.go @@ -32,10 +32,10 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/log" "github.com/pingcap/parser/model" - "github.com/pingcap/ticdc/tests/integration_tests/util" "github.com/pingcap/tidb/kv" "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 0e8d1f76fde..4e6638b3ccc 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 --pd=http://$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 13d133e6bf2..26d4aba3176 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 +./tools/bin/errdoc-gen --source . --module github.com/pingcap/tiflow --output errors.toml diff -q errors.toml /tmp/errors.toml.before From d93f48e52d44810903a1b13f00a493961e535e58 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 20 Dec 2021 12:27:46 +0800 Subject: [PATCH 09/30] kvclient(ticdc): fix kvclient takes too long time to recover (#3612) (#3662) --- cdc/kv/client.go | 37 +++++++++---------------------------- cdc/kv/client_test.go | 41 ++++++++++++++++++----------------------- cdc/kv/region_worker.go | 7 +++---- 3 files changed, 30 insertions(+), 55 deletions(-) diff --git a/cdc/kv/client.go b/cdc/kv/client.go index e91008ae546..ccba12e638b 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. @@ -1059,10 +1035,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) @@ -1160,6 +1138,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 8ed69ce9a1b..e61310ec110 100644 --- a/cdc/kv/client_test.go +++ b/cdc/kv/client_test.go @@ -48,6 +48,7 @@ import ( "github.com/tikv/client-go/v2/tikv" "go.uber.org/zap" "google.golang.org/grpc" + "google.golang.org/grpc/keepalive" ) func Test(t *testing.T) { @@ -278,7 +279,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() { @@ -1594,10 +1606,7 @@ func (s *etcdSuite) 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() @@ -1652,7 +1661,8 @@ func (s *etcdSuite) 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() @@ -2031,7 +2041,6 @@ func (s *etcdSuite) testEventCommitTsFallback(c *check.C, events []*cdcpb.Change ch1 <- event } clientWg.Wait() - cancel() } @@ -2514,22 +2523,6 @@ func (s *etcdSuite) TestOutOfRegionRangeEvent(c *check.C) { cancel() } -func (s *clientSuite) TestSingleRegionInfoClone(c *check.C) { - defer testleak.AfterTest(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 *etcdSuite) TestResolveLockNoCandidate(c *check.C) { @@ -2851,6 +2844,7 @@ func (s *etcdSuite) 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)) @@ -2899,6 +2893,7 @@ func (s *etcdSuite) 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 cff3be90d5844da7b9b58b23c8a17e679a45bcdd Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 20 Dec 2021 13:05:46 +0800 Subject: [PATCH 10/30] tz (ticdc): fix timezone error (#3887) (#3910) --- 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 726c08ee7e5..77d03e17306 100644 --- a/pkg/util/tz.go +++ b/pkg/util/tz.go @@ -14,11 +14,11 @@ package util import ( - "os" "path/filepath" "strings" "time" + "github.com/pingcap/tidb/util/timeutil" cerror "github.com/pingcap/tiflow/pkg/errors" ) @@ -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 276e74c204ccb93c93fdedfcda42b7d862d8cc5c Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 20 Dec 2021 13:35:45 +0800 Subject: [PATCH 11/30] http_*: add log for http api and refine the err handle logic (#2997) (#3306) --- cdc/capture/http_errors.go | 51 ++++++++++++ cdc/capture/http_errors_test.go | 33 ++++++++ cdc/capture/http_handler.go | 142 ++++++++++---------------------- 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 +- 10 files changed, 197 insertions(+), 120 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 cb616f4e10a..c1446b7c551 100644 --- a/cdc/capture/http_handler.go +++ b/cdc/capture/http_handler.go @@ -74,7 +74,7 @@ func (h *HTTPHandler) ListChangefeed(c *gin.Context) { // get all changefeed status statuses, err := h.capture.etcdClient.GetAllChangeFeedStatus(ctx) if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } @@ -128,38 +128,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 := h.capture.etcdClient.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 := h.capture.etcdClient.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 := h.capture.etcdClient.GetAllTaskStatus(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 } @@ -185,7 +172,7 @@ func (h *HTTPHandler) GetChangefeed(c *gin.Context) { TaskStatus: taskStatus, } - c.JSON(http.StatusOK, changefeedDetail) + c.IndentedJSON(http.StatusOK, changefeedDetail) } // CreateChangefeed creates a changefeed @@ -206,29 +193,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 } @@ -255,18 +238,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 && check if the etcdClient work well _, _, err := h.capture.etcdClient.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 } @@ -301,18 +279,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 && check if the etcdClient work well _, _, err := h.capture.etcdClient.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 } @@ -354,21 +327,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 := h.capture.etcdClient.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 } @@ -376,19 +344,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 } @@ -413,18 +381,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 && check if the etcdClient work well _, _, err := h.capture.etcdClient.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 } @@ -460,18 +423,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 := h.capture.etcdClient.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 } @@ -502,18 +460,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 := h.capture.etcdClient.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 } @@ -523,12 +476,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 } @@ -576,33 +529,25 @@ 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 } _, status, err := h.capture.etcdClient.GetTaskStatus(ctx, changefeedID, captureID) 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, err := h.capture.etcdClient.GetTaskPosition(ctx, changefeedID, captureID) 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 } @@ -612,7 +557,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 @@ -628,7 +573,7 @@ func (h *HTTPHandler) ListProcessor(c *gin.Context) { ctx := c.Request.Context() infos, err := h.capture.etcdClient.GetProcessors(ctx) if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } resps := make([]*model.ProcessorCommonInfo, len(infos)) @@ -658,7 +603,7 @@ func (h *HTTPHandler) ListCapture(c *gin.Context) { ownerID, err := h.capture.etcdClient.GetOwnerID(c, kv.CaptureOwnerKey) if err != nil { - c.IndentedJSON(http.StatusInternalServerError, model.NewHTTPError(err)) + _ = c.Error(err) return } @@ -728,14 +673,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)) @@ -747,7 +691,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) @@ -764,13 +708,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 } @@ -792,7 +736,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 } @@ -810,7 +754,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 5baa2219f20..b1048ae3e79 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) err = sink.Validate(ctx, info.SinkURI, info.Config, info.Opts) diff --git a/cdc/http_router.go b/cdc/http_router.go index da4f75e8ac7..db8d9b60e16 100644 --- a/cdc/http_router.go +++ b/cdc/http_router.go @@ -21,22 +21,28 @@ 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 = ioutil.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)) @@ -95,7 +101,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) @@ -118,3 +124,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 7319701e222..0605deed574 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 4d694c15d2a..f24a2e4bd75 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -331,7 +331,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 { @@ -435,13 +435,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 } @@ -509,8 +508,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 { @@ -518,8 +516,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 } @@ -1357,12 +1354,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 } @@ -1402,8 +1399,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) @@ -1412,11 +1408,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://" From 47f28b0a818c691985e05d02dbbd0789af0e824e Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 20 Dec 2021 15:03:47 +0800 Subject: [PATCH 12/30] ticdc/alert: add no owner alert rule (#3809) (#3831) --- metrics/alertmanager/ticdc.rules.yml | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/metrics/alertmanager/ticdc.rules.yml b/metrics/alertmanager/ticdc.rules.yml index 5290867501d..af3cd9481f8 100644 --- a/metrics/alertmanager/ticdc.rules.yml +++ b/metrics/alertmanager/ticdc.rules.yml @@ -13,6 +13,18 @@ groups: value: '{{ $value }}' summary: cdc cluster has multiple owners + - alert: cdc_no_owner + expr: sum(rate(ticdc_owner_ownership_counter[30s])) < 0.5 + for: 10m + labels: + env: ENV_LABELS_ENV + level: warning + expr: sum(rate(ticdc_owner_ownership_counter[30s])) < 0.5 + annotations: + description: 'cluster: ENV_LABELS_ENV, instance: {{ $labels.instance }}, values: {{ $value }}' + value: '{{ $value }}' + summary: cdc cluster has no owner for more than 10 minutes + - alert: cdc_checkpoint_high_delay expr: ticdc_processor_checkpoint_ts_lag > 600 for: 1m From bb8b7c14b1992b15fad2c38d05ae956b306044b9 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 20 Dec 2021 18:09:46 +0800 Subject: [PATCH 13/30] etcd_worker: batch etcd patch (#3277) (#3393) --- cdc/capture.go | 2 +- cdc/capture/capture.go | 3 +- cdc/metrics.go | 2 + cdc/task_test.go | 4 +- errors.toml | 10 + go.mod | 1 + metrics/grafana/ticdc.json | 1117 ++++++++++++++++- pkg/errors/errors.go | 8 +- pkg/etcd/client.go | 79 +- pkg/etcd/client_test.go | 128 ++ 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 + .../cdc_state_checker/cdc_monitor.go | 2 +- 18 files changed, 1707 insertions(+), 71 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.go b/cdc/capture.go index f5e889759df..db16d3abe4d 100644 --- a/cdc/capture.go +++ b/cdc/capture.go @@ -162,7 +162,7 @@ func (c *Capture) Run(ctx context.Context) (err error) { return errors.Trace(err) } log.Info("start to listen processor task...") - if err := etcdWorker.Run(ctx, c.session, 200*time.Millisecond); err != nil { + if err := etcdWorker.Run(ctx, c.session, 200*time.Millisecond, c.info.AdvertiseAddr); 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/capture/capture.go b/cdc/capture/capture.go index cc9adebedf0..4d58deed471 100644 --- a/cdc/capture/capture.go +++ b/cdc/capture/capture.go @@ -281,7 +281,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 1d59c443302..37cc38e011d 100644 --- a/cdc/metrics.go +++ b/cdc/metrics.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/tiflow/cdc/puller/sorter" "github.com/pingcap/tiflow/cdc/sink" "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/orchestrator" "github.com/prometheus/client_golang/prometheus" ) @@ -37,6 +38,7 @@ func init() { sink.InitMetrics(registry) entry.InitMetrics(registry) sorter.InitMetrics(registry) + orchestrator.InitMetrics(registry) if config.NewReplicaImpl { processor.InitMetrics(registry) tablepipeline.InitMetrics(registry) diff --git a/cdc/task_test.go b/cdc/task_test.go index c97f2a5ff2f..4499e9e533f 100644 --- a/cdc/task_test.go +++ b/cdc/task_test.go @@ -156,7 +156,9 @@ func (s *taskSuite) TestWatch(c *check.C) { } // Watch with a normal context - ch := s.w.Watch(context.Background()) + ctx, cancel = context.WithCancel(context.Background()) + defer cancel() + ch := s.w.Watch(ctx) // Trigger the ErrCompacted error c.Assert(failpoint.Enable("github.com/pingcap/tiflow/cdc.restart_task_watch", "50%off"), check.IsNil) diff --git a/errors.toml b/errors.toml index b2c4981e156..689c375357a 100755 --- a/errors.toml +++ b/errors.toml @@ -206,6 +206,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 6cbe143d0a6..a41faa6afb7 100644 --- a/go.mod +++ b/go.mod @@ -8,6 +8,7 @@ require ( github.com/Shopify/sarama v1.27.2 github.com/alecthomas/template v0.0.0-20190718012654-fb15b899a751 github.com/apache/pulsar-client-go v0.1.1 + 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/chzyer/readline v0.0.0-20180603132655-2972be24d48e diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index ed21e8d6100..4fe6eed8ddb 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": { @@ -9679,5 +10792,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 64d583ac919..46c963ff7ee 100644 --- a/pkg/errors/errors.go +++ b/pkg/errors/errors.go @@ -201,9 +201,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..7769f09a3b0 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,70 @@ 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))) + } + } + } + + ticker.Reset(etcdRequestProgressDuration) + 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/etcd/client_test.go b/pkg/etcd/client_test.go index 04a81787d16..56973e1f777 100644 --- a/pkg/etcd/client_test.go +++ b/pkg/etcd/client_test.go @@ -17,6 +17,7 @@ import ( "context" "time" + "github.com/benbjohnson/clock" "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/tiflow/pkg/util/testleak" @@ -45,6 +46,23 @@ func (m *mockClient) Put(ctx context.Context, key, val string, opts ...clientv3. return nil, errors.New("mock error") } +type mockWatcher struct { + clientv3.Watcher + watchCh chan clientv3.WatchResponse + resetCount *int + requestCount *int +} + +func (m mockWatcher) Watch(ctx context.Context, key string, opts ...clientv3.OpOption) clientv3.WatchChan { + *m.resetCount++ + return m.watchCh +} + +func (m mockWatcher) RequestProgress(ctx context.Context) error { + *m.requestCount++ + return nil +} + func (s *clientSuite) TestRetry(c *check.C) { defer testleak.AfterTest(c)() originValue := maxTries @@ -91,3 +109,113 @@ func (s *etcdSuite) TestDelegateLease(c *check.C) { c.Assert(err, check.IsNil) c.Assert(ttlResp.TTL, check.Equals, int64(-1)) } + +// test no data lost when WatchCh blocked +func (s *etcdSuite) TestWatchChBlocked(c *check.C) { + defer testleak.AfterTest(c)() + defer s.TearDownTest(c) + cli := clientv3.NewCtxClient(context.TODO()) + resetCount := 0 + requestCount := 0 + watchCh := make(chan clientv3.WatchResponse, 1) + watcher := mockWatcher{watchCh: watchCh, resetCount: &resetCount, requestCount: &requestCount} + cli.Watcher = watcher + + sentRes := []clientv3.WatchResponse{ + {CompactRevision: 1}, + {CompactRevision: 2}, + {CompactRevision: 3}, + {CompactRevision: 4}, + {CompactRevision: 5}, + {CompactRevision: 6}, + } + + go func() { + for _, r := range sentRes { + watchCh <- r + } + }() + + mockClock := clock.NewMock() + watchCli := Wrap(cli, nil) + watchCli.clock = mockClock + + key := "testWatchChBlocked" + outCh := make(chan clientv3.WatchResponse, 6) + revision := int64(1) + ctx, cancel := context.WithTimeout(context.Background(), time.Second*2) + defer cancel() + + go func() { + watchCli.WatchWithChan(ctx, outCh, key, clientv3.WithPrefix(), clientv3.WithRev(revision)) + }() + receivedRes := make([]clientv3.WatchResponse, 0) + // wait for WatchWithChan set up + r := <-outCh + receivedRes = append(receivedRes, r) + // move time forward + mockClock.Add(time.Second * 30) + + for r := range outCh { + receivedRes = append(receivedRes, r) + } + + c.Check(sentRes, check.DeepEquals, receivedRes) + // make sure watchCh has been reset since timeout + c.Assert(*watcher.resetCount > 1, check.IsTrue) + // make sure RequestProgress has been call since timeout + c.Assert(*watcher.requestCount > 1, check.IsTrue) + // make sure etcdRequestProgressDuration is less than etcdWatchChTimeoutDuration + c.Assert(etcdRequestProgressDuration, check.Less, etcdWatchChTimeoutDuration) +} + +// test no data lost when OutCh blocked +func (s *etcdSuite) TestOutChBlocked(c *check.C) { + defer testleak.AfterTest(c)() + defer s.TearDownTest(c) + + cli := clientv3.NewCtxClient(context.TODO()) + resetCount := 0 + requestCount := 0 + watchCh := make(chan clientv3.WatchResponse, 1) + watcher := mockWatcher{watchCh: watchCh, resetCount: &resetCount, requestCount: &requestCount} + cli.Watcher = watcher + + mockClock := clock.NewMock() + watchCli := Wrap(cli, nil) + watchCli.clock = mockClock + + sentRes := []clientv3.WatchResponse{ + {CompactRevision: 1}, + {CompactRevision: 2}, + {CompactRevision: 3}, + } + + go func() { + for _, r := range sentRes { + watchCh <- r + } + }() + + key := "testOutChBlocked" + outCh := make(chan clientv3.WatchResponse, 1) + revision := int64(1) + + ctx, cancel := context.WithTimeout(context.Background(), time.Second*2) + defer cancel() + go func() { + watchCli.WatchWithChan(ctx, outCh, key, clientv3.WithPrefix(), clientv3.WithRev(revision)) + }() + receivedRes := make([]clientv3.WatchResponse, 0) + // wait for WatchWithChan set up + r := <-outCh + receivedRes = append(receivedRes, r) + // move time forward + mockClock.Add(time.Second * 30) + + for r := range outCh { + receivedRes = append(receivedRes, r) + } + + c.Check(sentRes, check.DeepEquals, receivedRes) +} 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..0fa2243815e 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/tiflow/pkg/orchestrator/InjectProgressRequestAfterCommit", "10%return(true)") + defer func() { + _ = failpoint.Disable("github.com/pingcap/tiflow/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 001c3a401c2..9b21a02200a 100644 --- a/pkg/orchestrator/etcd_worker_test.go +++ b/pkg/orchestrator/etcd_worker_test.go @@ -268,7 +268,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")) }) } @@ -353,7 +353,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) @@ -438,7 +438,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) @@ -507,7 +507,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) @@ -586,7 +586,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) @@ -653,7 +653,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) @@ -734,7 +734,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) }() @@ -749,7 +749,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/testing_utils/cdc_state_checker/cdc_monitor.go b/testing_utils/cdc_state_checker/cdc_monitor.go index 655770b96e6..c628c4e67a0 100644 --- a/testing_utils/cdc_state_checker/cdc_monitor.go +++ b/testing_utils/cdc_state_checker/cdc_monitor.go @@ -92,7 +92,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 4915083b1e41bd54b45ef90e6d2af0c054e75c9b Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Wed, 22 Dec 2021 11:35:47 +0800 Subject: [PATCH 14/30] ticdc/owner: Fix ddl special comment syntax error (#3845) (#3977) --- 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 9a0d3a80ee8..83ced2291b4 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/parser" + "github.com/pingcap/parser/format" timodel "github.com/pingcap/parser/model" - "github.com/pingcap/tidb/sessionctx/binloginfo" "github.com/pingcap/tiflow/cdc/model" cdcContext "github.com/pingcap/tiflow/pkg/context" cerror "github.com/pingcap/tiflow/pkg/errors" @@ -422,7 +424,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 job.BinlogInfo.TableInfo != nil && c.schema.IsIneligibleTableID(job.BinlogInfo.TableInfo.ID) { @@ -480,3 +486,27 @@ func (c *changefeed) updateStatus(currentTs int64, barrierTs model.Ts) { func (c *changefeed) Close() { c.releaseResources() } + +// addSpecialComment translate tidb feature to comment +func addSpecialComment(ddlQuery string) (string, error) { + stms, _, err := parser.New().Parse(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 d6d2d0b7c84..861f3c42849 100644 --- a/cdc/owner/changefeed_test.go +++ b/cdc/owner/changefeed_test.go @@ -271,7 +271,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 @@ -286,7 +286,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 @@ -353,3 +353,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 93020bbf823e02e8273ac384a8a3e1d4b213abef Mon Sep 17 00:00:00 2001 From: dongmen <20351731+asddongmen@users.noreply.github.com> Date: Thu, 23 Dec 2021 09:11:48 +0800 Subject: [PATCH 15/30] Clean old owner and old processor in release 5.2 branch (#4019) --- cdc/capture.go | 337 ---- cdc/capture_test.go | 175 -- cdc/changefeed.go | 1034 ---------- cdc/changefeed_test.go | 130 -- cdc/metrics.go | 12 +- cdc/metrics_owner.go | 62 - cdc/metrics_processor.go | 98 - cdc/owner.go | 1734 ----------------- cdc/owner_operator.go | 120 -- cdc/owner_test.go | 1478 -------------- cdc/processor.go | 1292 ------------ cdc/processor_test.go | 344 ---- cdc/server.go | 20 +- cdc/sink/mysql.go | 7 - cdc/task.go | 234 --- cdc/task_test.go | 286 --- pkg/config/config.go | 3 - pkg/util/testleak/fake.go | 1 + pkg/util/testleak/leaktest.go | 1 + tests/integration_tests/availability/owner.sh | 9 +- .../capture_session_done_during_task/run.sh | 3 +- .../changefeed_auto_stop/run.sh | 3 +- .../integration_tests/changefeed_error/run.sh | 11 +- tests/integration_tests/ddl_puller_lag/run.sh | 3 +- tests/integration_tests/gc_safepoint/run.sh | 2 +- .../kill_owner_with_ddl/run.sh | 3 +- .../owner_remove_table_error/run.sh | 3 +- .../processor_err_chan/run.sh | 3 +- .../integration_tests/processor_panic/run.sh | 5 +- .../processor_stop_delay/run.sh | 3 +- 30 files changed, 22 insertions(+), 7394 deletions(-) delete mode 100644 cdc/capture.go delete mode 100644 cdc/capture_test.go delete mode 100644 cdc/changefeed.go delete mode 100644 cdc/changefeed_test.go delete mode 100644 cdc/metrics_owner.go delete mode 100644 cdc/metrics_processor.go delete mode 100644 cdc/owner.go delete mode 100644 cdc/owner_operator.go delete mode 100644 cdc/owner_test.go delete mode 100644 cdc/processor.go delete mode 100644 cdc/processor_test.go delete mode 100644 cdc/task.go delete mode 100644 cdc/task_test.go diff --git a/cdc/capture.go b/cdc/capture.go deleted file mode 100644 index db16d3abe4d..00000000000 --- a/cdc/capture.go +++ /dev/null @@ -1,337 +0,0 @@ -// Copyright 2020 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 cdc - -import ( - "context" - "sync" - "time" - - "github.com/google/uuid" - "github.com/pingcap/errors" - "github.com/pingcap/failpoint" - "github.com/pingcap/log" - tidbkv "github.com/pingcap/tidb/kv" - "github.com/pingcap/tiflow/cdc/kv" - "github.com/pingcap/tiflow/cdc/model" - "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/orchestrator" - "github.com/pingcap/tiflow/pkg/util" - "github.com/pingcap/tiflow/pkg/version" - pd "github.com/tikv/pd/client" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/concurrency" - "go.etcd.io/etcd/mvcc" - "go.etcd.io/etcd/pkg/logutil" - "go.uber.org/zap" - "go.uber.org/zap/zapcore" - "google.golang.org/grpc" - "google.golang.org/grpc/backoff" -) - -// Capture represents a Capture server, it monitors the changefeed information in etcd and schedules Task on it. -type Capture struct { - etcdClient kv.CDCEtcdClient - pdCli pd.Client - kvStorage tidbkv.Storage - grpcPool kv.GrpcPool - - processorManager *processor.Manager - - processors map[string]*oldProcessor - procLock sync.Mutex - - info *model.CaptureInfo - - // session keeps alive between the capture and etcd - session *concurrency.Session - election *concurrency.Election - - closed chan struct{} -} - -// NewCapture returns a new Capture instance -func NewCapture( - stdCtx context.Context, - pdEndpoints []string, - pdCli pd.Client, - kvStorage tidbkv.Storage, -) (c *Capture, err error) { - conf := config.GetGlobalServerConfig() - credential := conf.Security - tlsConfig, err := credential.ToTLSConfig() - if err != nil { - return nil, errors.Trace(err) - } - grpcTLSOption, err := credential.ToGRPCDialOption() - if err != nil { - return nil, errors.Trace(err) - } - logConfig := logutil.DefaultZapLoggerConfig - logConfig.Level = zap.NewAtomicLevelAt(zapcore.ErrorLevel) - etcdCli, err := clientv3.New(clientv3.Config{ - Endpoints: pdEndpoints, - TLS: tlsConfig, - Context: stdCtx, - LogConfig: &logConfig, - DialTimeout: 5 * time.Second, - DialOptions: []grpc.DialOption{ - grpcTLSOption, - grpc.WithBlock(), - grpc.WithConnectParams(grpc.ConnectParams{ - Backoff: backoff.Config{ - BaseDelay: time.Second, - Multiplier: 1.1, - Jitter: 0.1, - MaxDelay: 3 * time.Second, - }, - MinConnectTimeout: 3 * time.Second, - }), - }, - }) - if err != nil { - return nil, errors.Annotate(cerror.WrapError(cerror.ErrNewCaptureFailed, err), "new etcd client") - } - sess, err := concurrency.NewSession(etcdCli, - concurrency.WithTTL(conf.CaptureSessionTTL)) - if err != nil { - return nil, errors.Annotate(cerror.WrapError(cerror.ErrNewCaptureFailed, err), "create capture session") - } - elec := concurrency.NewElection(sess, kv.CaptureOwnerKey) - cli := kv.NewCDCEtcdClient(stdCtx, etcdCli) - id := uuid.New().String() - info := &model.CaptureInfo{ - ID: id, - AdvertiseAddr: conf.AdvertiseAddr, - Version: version.ReleaseVersion, - } - processorManager := processor.NewManager() - grpcPool := kv.NewGrpcPoolImpl(stdCtx, credential) - log.Info("creating capture", zap.String("capture-id", id), util.ZapFieldCapture(stdCtx)) - - c = &Capture{ - processors: make(map[string]*oldProcessor), - etcdClient: cli, - grpcPool: grpcPool, - session: sess, - election: elec, - info: info, - pdCli: pdCli, - kvStorage: kvStorage, - processorManager: processorManager, - closed: make(chan struct{}), - } - - return -} - -// Run runs the Capture mainloop -func (c *Capture) Run(ctx context.Context) (err error) { - ctx, cancel := context.WithCancel(ctx) - // TODO: we'd better to add some wait mechanism to ensure no routine is blocked - defer cancel() - defer close(c.closed) - - ctx = cdcContext.NewContext(ctx, &cdcContext.GlobalVars{ - PDClient: c.pdCli, - KVStorage: c.kvStorage, - CaptureInfo: c.info, - }) - err = c.register(ctx) - if err != nil { - return errors.Trace(err) - } - if config.NewReplicaImpl { - sessionCli := c.session.Client() - etcdWorker, err := orchestrator.NewEtcdWorker(kv.NewCDCEtcdClient(ctx, sessionCli).Client, kv.EtcdKeyBase, c.processorManager, model.NewGlobalState()) - if err != nil { - return errors.Trace(err) - } - log.Info("start to listen processor task...") - if err := etcdWorker.Run(ctx, c.session, 200*time.Millisecond, c.info.AdvertiseAddr); 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 - // 1/3 of session ttl that `session.Done` can't be triggered even - // the lease is already revoked. - if cerror.ErrEtcdSessionDone.Equal(err) { - log.Warn("session is disconnected", zap.Error(err)) - return cerror.ErrCaptureSuicide.GenWithStackByArgs() - } - lease, inErr := c.etcdClient.Client.TimeToLive(ctx, c.session.Lease()) - if inErr != nil { - return cerror.WrapError(cerror.ErrPDEtcdAPIError, inErr) - } - if lease.TTL == int64(-1) { - log.Warn("handle task event failed because session is disconnected", zap.Error(err)) - return cerror.ErrCaptureSuicide.GenWithStackByArgs() - } - return errors.Trace(err) - } - } else { - defer c.grpcPool.Close() - go func() { - c.grpcPool.RecycleConn(ctx) - }() - taskWatcher := NewTaskWatcher(c, &TaskWatcherConfig{ - Prefix: kv.TaskStatusKeyPrefix + "/" + c.info.ID, - ChannelSize: 128, - }) - log.Info("waiting for tasks", zap.String("capture-id", c.info.ID)) - var ev *TaskEvent - wch := taskWatcher.Watch(ctx) - for { - // Return error when the session is done unexpectedly, it means the - // server does not send heartbeats in time, or network interrupted - // In this case, the state of the capture is undermined, the task may - // have or have not been rebalanced, the owner may be or not be held, - // so we must cancel context to let all sub routines exit. - select { - case <-c.session.Done(): - if ctx.Err() != context.Canceled { - log.Info("capture session done, capture suicide itself", zap.String("capture-id", c.info.ID)) - return cerror.ErrCaptureSuicide.GenWithStackByArgs() - } - case ev = <-wch: - if ev == nil { - return nil - } - if ev.Err != nil { - return errors.Trace(ev.Err) - } - failpoint.Inject("captureHandleTaskDelay", nil) - if err := c.handleTaskEvent(ctx, ev); 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 - // 1/3 of session ttl that `session.Done` can't be triggered even - // the lease is already revoked. - lease, inErr := c.etcdClient.Client.TimeToLive(ctx, c.session.Lease()) - if inErr != nil { - return cerror.WrapError(cerror.ErrPDEtcdAPIError, inErr) - } - if lease.TTL == int64(-1) { - log.Warn("handle task event failed because session is disconnected", zap.Error(err)) - return cerror.ErrCaptureSuicide.GenWithStackByArgs() - } - return errors.Trace(err) - } - } - } - } - return nil -} - -// Campaign to be an owner -func (c *Capture) Campaign(ctx context.Context) error { - failpoint.Inject("capture-campaign-compacted-error", func() { - failpoint.Return(errors.Trace(mvcc.ErrCompacted)) - }) - return cerror.WrapError(cerror.ErrCaptureCampaignOwner, c.election.Campaign(ctx, c.info.ID)) -} - -// Resign lets a owner start a new election. -func (c *Capture) Resign(ctx context.Context) error { - failpoint.Inject("capture-resign-failed", func() { - failpoint.Return(errors.New("capture resign failed")) - }) - return cerror.WrapError(cerror.ErrCaptureResignOwner, c.election.Resign(ctx)) -} - -// Cleanup cleans all dynamic resources -func (c *Capture) Cleanup() { - c.procLock.Lock() - defer c.procLock.Unlock() - - for _, processor := range c.processors { - processor.wait() - } -} - -// Close closes the capture by unregistering it from etcd -func (c *Capture) Close(ctx context.Context) error { - if config.NewReplicaImpl { - c.processorManager.AsyncClose() - select { - case <-c.closed: - case <-ctx.Done(): - } - } else { - if c.grpcPool != nil { - c.grpcPool.Close() - } - } - return errors.Trace(c.etcdClient.DeleteCaptureInfo(ctx, c.info.ID)) -} - -func (c *Capture) handleTaskEvent(ctx context.Context, ev *TaskEvent) error { - task := ev.Task - if ev.Op == TaskOpCreate { - if _, ok := c.processors[task.ChangeFeedID]; !ok { - p, err := c.assignTask(ctx, task) - if err != nil { - return err - } - c.processors[task.ChangeFeedID] = p - } - } else if ev.Op == TaskOpDelete { - if p, ok := c.processors[task.ChangeFeedID]; ok { - if err := p.stop(ctx); err != nil { - return errors.Trace(err) - } - delete(c.processors, task.ChangeFeedID) - } - } - return nil -} - -func (c *Capture) assignTask(ctx context.Context, task *Task) (*oldProcessor, error) { - cf, err := c.etcdClient.GetChangeFeedInfo(ctx, task.ChangeFeedID) - if err != nil { - log.Error("get change feed info failed", - zap.String("changefeed", task.ChangeFeedID), - zap.String("capture-id", c.info.ID), - util.ZapFieldCapture(ctx), - zap.Error(err)) - return nil, err - } - err = cf.VerifyAndFix() - if err != nil { - return nil, err - } - log.Info("run processor", - zap.String("capture-id", c.info.ID), util.ZapFieldCapture(ctx), - zap.String("changefeed", task.ChangeFeedID)) - conf := config.GetGlobalServerConfig() - p, err := runProcessorImpl( - ctx, c.pdCli, c.grpcPool, c.session, *cf, task.ChangeFeedID, *c.info, task.CheckpointTS, time.Duration(conf.ProcessorFlushInterval)) - if err != nil { - log.Error("run processor failed", - zap.String("changefeed", task.ChangeFeedID), - zap.String("capture-id", c.info.ID), - util.ZapFieldCapture(ctx), - zap.Error(err)) - return nil, err - } - return p, nil -} - -// register registers the capture information in etcd -func (c *Capture) register(ctx context.Context) error { - err := c.etcdClient.PutCaptureInfo(ctx, c.info, c.session.Lease()) - return cerror.WrapError(cerror.ErrCaptureRegister, err) -} diff --git a/cdc/capture_test.go b/cdc/capture_test.go deleted file mode 100644 index 8f353dde519..00000000000 --- a/cdc/capture_test.go +++ /dev/null @@ -1,175 +0,0 @@ -// Copyright 2020 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 cdc - -import ( - "context" - "net/url" - "sync" - "time" - - "github.com/pingcap/check" - "github.com/pingcap/errors" - "github.com/pingcap/failpoint" - "github.com/pingcap/tiflow/cdc/kv" - "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/util" - "github.com/pingcap/tiflow/pkg/util/testleak" - pd "github.com/tikv/pd/client" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/concurrency" - "go.etcd.io/etcd/embed" - "golang.org/x/sync/errgroup" -) - -type captureSuite struct { - e *embed.Etcd - clientURL *url.URL - client kv.CDCEtcdClient - ctx context.Context - cancel context.CancelFunc - errg *errgroup.Group -} - -var _ = check.Suite(&captureSuite{}) - -func (s *captureSuite) SetUpTest(c *check.C) { - dir := c.MkDir() - var err error - s.clientURL, s.e, err = etcd.SetupEmbedEtcd(dir) - c.Assert(err, check.IsNil) - client, err := clientv3.New(clientv3.Config{ - Endpoints: []string{s.clientURL.String()}, - DialTimeout: 3 * time.Second, - }) - c.Assert(err, check.IsNil) - s.client = kv.NewCDCEtcdClient(context.Background(), client) - s.ctx, s.cancel = context.WithCancel(context.Background()) - s.errg = util.HandleErrWithErrGroup(s.ctx, s.e.Err(), func(e error) { c.Log(e) }) -} - -func (s *captureSuite) TearDownTest(c *check.C) { - s.e.Close() - s.cancel() - err := s.errg.Wait() - if err != nil { - c.Errorf("Error group error: %s", err) - } - s.client.Close() //nolint:errcheck -} - -func (s *captureSuite) TestCaptureSuicide(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - capture, err := NewCapture(ctx, []string{s.clientURL.String()}, nil, nil) - c.Assert(err, check.IsNil) - - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - err := capture.Run(ctx) - c.Assert(cerror.ErrCaptureSuicide.Equal(err), check.IsTrue) - }() - // ttl is 5s, wait 1s to ensure `capture.Run` starts - time.Sleep(time.Second) - _, err = s.client.Client.Revoke(ctx, capture.session.Lease()) - c.Assert(err, check.IsNil) - wg.Wait() - - err = capture.etcdClient.Close() - if err != nil { - c.Assert(errors.Cause(err), check.Equals, context.Canceled) - } -} - -func (s *captureSuite) TestCaptureSessionDoneDuringHandleTask(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - if config.NewReplicaImpl { - c.Skip("this case is designed for old processor") - } - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - capture, err := NewCapture(ctx, []string{s.clientURL.String()}, nil, nil) - c.Assert(err, check.IsNil) - - runProcessorCount := 0 - err = failpoint.Enable("github.com/pingcap/tiflow/cdc/captureHandleTaskDelay", "sleep(500)") - c.Assert(err, check.IsNil) - defer func() { - _ = failpoint.Disable("github.com/pingcap/tiflow/cdc/captureHandleTaskDelay") - }() - runProcessorBackup := runProcessorImpl - runProcessorImpl = func( - ctx context.Context, _ pd.Client, grpcPool kv.GrpcPool, - session *concurrency.Session, info model.ChangeFeedInfo, changefeedID string, - captureInfo model.CaptureInfo, checkpointTs uint64, flushCheckpointInterval time.Duration, - ) (*oldProcessor, error) { - runProcessorCount++ - etcdCli := kv.NewCDCEtcdClient(ctx, session.Client()) - _, _, err := etcdCli.GetTaskStatus(ctx, changefeedID, captureInfo.ID) - return nil, err - } - defer func() { - runProcessorImpl = runProcessorBackup - }() - - // The test simulates the following procedure - // 1. owner: dispatches new task to a capture - // 2. capture: detects the task, and starts to handle task - // 3. capture: during the task handling, capture session is disconnected - // 4. owner: observes the capture session disconnected and cleanup the task status of this capture - // 5. capture: queries task status failed when handling task - // 6. capture: checks session ttl, finds session disconnected and returns ErrCaptureSuicide to restart itself - // the event sequence must be kept, especially for 2->3->4->5 - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - err := capture.Run(ctx) - // check step-6 - c.Assert(cerror.ErrCaptureSuicide.Equal(err), check.IsTrue) - // check step-5 runs - c.Assert(runProcessorCount, check.Equals, 1) - }() - changefeedID := "test-changefeed" - err = s.client.SaveChangeFeedInfo(ctx, &model.ChangeFeedInfo{Config: config.GetDefaultReplicaConfig()}, changefeedID) - c.Assert(err, check.IsNil) - // step-1 - err = s.client.PutTaskStatus(ctx, changefeedID, capture.info.ID, &model.TaskStatus{}) - c.Assert(err, check.IsNil) - // sleep 100ms to ensure step-2 happens, the failpoint injected delay will ensure step-4 is after step-3 - time.Sleep(time.Millisecond * 100) - - // step-3 - _, err = s.client.Client.Revoke(ctx, capture.session.Lease()) - c.Assert(err, check.IsNil) - err = s.client.DeleteTaskStatus(ctx, changefeedID, capture.info.ID) - c.Assert(err, check.IsNil) - - wg.Wait() - - err = capture.etcdClient.Close() - if err != nil { - c.Assert(errors.Cause(err), check.Equals, context.Canceled) - } -} diff --git a/cdc/changefeed.go b/cdc/changefeed.go deleted file mode 100644 index 30a0a108a93..00000000000 --- a/cdc/changefeed.go +++ /dev/null @@ -1,1034 +0,0 @@ -// Copyright 2020 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 cdc - -import ( - "context" - "fmt" - "math" - "sync" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/failpoint" - "github.com/pingcap/log" - timodel "github.com/pingcap/parser/model" - "github.com/pingcap/tidb/sessionctx/binloginfo" - "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/cyclic/mark" - cerror "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/filter" - "github.com/pingcap/tiflow/pkg/scheduler" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/mvcc/mvccpb" - "go.uber.org/zap" -) - -type tableIDMap = map[model.TableID]struct{} - -// OwnerDDLHandler defines the ddl handler for Owner -// which can pull ddl jobs and execute ddl jobs -type OwnerDDLHandler interface { - // PullDDL pulls the ddl jobs and returns resolvedTs of DDL Puller and job list. - PullDDL() (resolvedTs uint64, jobs []*timodel.Job, err error) - - // Close cancels the executing of OwnerDDLHandler and releases resource - Close() error -} - -// ChangeFeedRWriter defines the Reader and Writer for changeFeed -type ChangeFeedRWriter interface { - - // GetChangeFeeds returns kv revision and a map mapping from changefeedID to changefeed detail mvccpb.KeyValue - GetChangeFeeds(ctx context.Context) (int64, map[string]*mvccpb.KeyValue, error) - - // GetAllTaskStatus queries all task status of a changefeed, and returns a map - // mapping from captureID to TaskStatus - GetAllTaskStatus(ctx context.Context, changefeedID string) (model.ProcessorsInfos, error) - - // RemoveAllTaskStatus removes all task status of a changefeed - RemoveAllTaskStatus(ctx context.Context, changefeedID string) error - - // GetAllTaskPositions queries all task positions of a changefeed, and returns a map - // mapping from captureID to TaskPositions - GetAllTaskPositions(ctx context.Context, changefeedID string) (map[string]*model.TaskPosition, error) - - // RemoveAllTaskPositions removes all task partitions of a changefeed - RemoveAllTaskPositions(ctx context.Context, changefeedID string) error - - // GetChangeFeedStatus queries the checkpointTs and resovledTs of a given changefeed - GetChangeFeedStatus(ctx context.Context, id string) (*model.ChangeFeedStatus, int64, error) - - // PutAllChangeFeedStatus the changefeed info to storage such as etcd. - PutAllChangeFeedStatus(ctx context.Context, infos map[model.ChangeFeedID]*model.ChangeFeedStatus) error - - // LeaseGuardRemoveAllTaskStatus wraps RemoveAllTaskStatus with a context restricted by lease TTL. - LeaseGuardRemoveAllTaskStatus(ctx context.Context, changefeedID string, leaseID clientv3.LeaseID) error - - // LeaseGuardRemoveAllTaskPositions wraps RemoveAllTaskPositions with a context restricted by lease TTL. - LeaseGuardRemoveAllTaskPositions(ctx context.Context, changefeedID string, leaseID clientv3.LeaseID) error - - // LeaseGuardPutAllChangeFeedStatus wraps PutAllChangeFeedStatus with a context restricted by lease TTL. - LeaseGuardPutAllChangeFeedStatus(ctx context.Context, infos map[model.ChangeFeedID]*model.ChangeFeedStatus, leaseID clientv3.LeaseID) error -} - -type changeFeed struct { - id string - info *model.ChangeFeedInfo - status *model.ChangeFeedStatus - // The latest checkpointTs already applied to Etcd. - // We need to check this field to ensure visibility to the processors, - // if the operation assumes the progress of the global checkpoint. - appliedCheckpointTs uint64 - - schema *entry.SingleSchemaSnapshot - ddlState model.ChangeFeedDDLState - targetTs uint64 - ddlTs uint64 - syncpointMutex sync.Mutex - updateResolvedTs bool - startTimer chan bool - syncpointStore sink.SyncpointStore - syncCancel context.CancelFunc - taskStatus model.ProcessorsInfos - taskPositions map[model.CaptureID]*model.TaskPosition - filter *filter.Filter - sink sink.Sink - scheduler scheduler.Scheduler - - cyclicEnabled bool - - ddlHandler OwnerDDLHandler - ddlResolvedTs uint64 - ddlJobHistory []*timodel.Job - ddlExecutedTs uint64 - - schemas map[model.SchemaID]tableIDMap - tables map[model.TableID]model.TableName - // value of partitions is the slice of partitions ID. - partitions map[model.TableID][]int64 - orphanTables map[model.TableID]model.Ts - toCleanTables map[model.TableID]model.Ts - moveTableJobs map[model.TableID]*model.MoveTableJob - manualMoveCommands []*model.MoveTableJob - rebalanceNextTick bool - - lastRebalanceTime time.Time - - etcdCli kv.CDCEtcdClient - leaseID clientv3.LeaseID - - // context cancel function for all internal goroutines - cancel context.CancelFunc -} - -// String implements fmt.Stringer interface. -func (c *changeFeed) String() string { - format := "{\n ID: %s\n info: %+v\n status: %+v\n State: %v\n ProcessorInfos: %+v\n tables: %+v\n orphanTables: %+v\n toCleanTables: %v\n ddlResolvedTs: %d\n ddlJobHistory: %+v\n}\n\n" - s := fmt.Sprintf(format, - c.id, c.info, c.status, c.ddlState, c.taskStatus, c.tables, - c.orphanTables, c.toCleanTables, c.ddlResolvedTs, c.ddlJobHistory) - - if len(c.ddlJobHistory) > 0 { - job := c.ddlJobHistory[0] - s += fmt.Sprintf("next to exec job: %s query: %s\n\n", job, job.Query) - } - - return s -} - -func (c *changeFeed) updateProcessorInfos(processInfos model.ProcessorsInfos, positions map[string]*model.TaskPosition) { - c.taskStatus = processInfos - c.taskPositions = positions -} - -func (c *changeFeed) addSchema(schemaID model.SchemaID) { - if _, ok := c.schemas[schemaID]; ok { - log.Warn("add schema already exists", zap.Int64("schemaID", schemaID)) - return - } - c.schemas[schemaID] = make(map[model.TableID]struct{}) -} - -func (c *changeFeed) dropSchema(schemaID model.SchemaID, targetTs model.Ts) { - if schema, ok := c.schemas[schemaID]; ok { - for tid := range schema { - c.removeTable(schemaID, tid, targetTs) - } - } - delete(c.schemas, schemaID) -} - -func (c *changeFeed) addTable(tblInfo *model.TableInfo, targetTs model.Ts) { - if c.filter.ShouldIgnoreTable(tblInfo.TableName.Schema, tblInfo.TableName.Table) { - return - } - if c.cyclicEnabled && mark.IsMarkTable(tblInfo.TableName.Schema, tblInfo.TableName.Table) { - return - } - - if _, ok := c.tables[tblInfo.ID]; ok { - log.Warn("add table already exists", zap.Int64("tableID", tblInfo.ID), zap.Stringer("table", tblInfo.TableName)) - return - } - - if !tblInfo.IsEligible(c.info.Config.ForceReplicate) { - log.Warn("skip ineligible table", zap.Int64("tid", tblInfo.ID), zap.Stringer("table", tblInfo.TableName)) - return - } - - if _, ok := c.schemas[tblInfo.SchemaID]; !ok { - c.schemas[tblInfo.SchemaID] = make(tableIDMap) - } - c.schemas[tblInfo.SchemaID][tblInfo.ID] = struct{}{} - c.tables[tblInfo.ID] = tblInfo.TableName - if pi := tblInfo.GetPartitionInfo(); pi != nil { - delete(c.partitions, tblInfo.ID) - for _, partition := range pi.Definitions { - c.partitions[tblInfo.ID] = append(c.partitions[tblInfo.ID], partition.ID) - c.orphanTables[partition.ID] = targetTs - } - } else { - c.orphanTables[tblInfo.ID] = targetTs - } -} - -func (c *changeFeed) removeTable(sid model.SchemaID, tid model.TableID, targetTs model.Ts) { - if _, ok := c.schemas[sid]; ok { - delete(c.schemas[sid], tid) - } - delete(c.tables, tid) - - removeFunc := func(id int64) { - if _, ok := c.orphanTables[id]; ok { - delete(c.orphanTables, id) - } else { - c.toCleanTables[id] = targetTs - } - } - - if pids, ok := c.partitions[tid]; ok { - for _, id := range pids { - removeFunc(id) - } - delete(c.partitions, tid) - } else { - removeFunc(tid) - } -} - -func (c *changeFeed) updatePartition(tblInfo *timodel.TableInfo, startTs uint64) { - tid := tblInfo.ID - partitionsID, ok := c.partitions[tid] - if !ok || len(partitionsID) == 0 { - return - } - oldIDs := make(map[int64]struct{}, len(partitionsID)) - for _, pid := range partitionsID { - oldIDs[pid] = struct{}{} - } - - pi := tblInfo.GetPartitionInfo() - if pi == nil { - return - } - newPartitionIDs := make([]int64, 0, len(pi.Definitions)) - for _, partition := range pi.Definitions { - pid := partition.ID - _, ok := c.orphanTables[pid] - if !ok { - // new partition. - c.orphanTables[pid] = startTs - } - delete(oldIDs, partition.ID) - newPartitionIDs = append(newPartitionIDs, partition.ID) - } - // update the table partition IDs. - c.partitions[tid] = newPartitionIDs - - // drop partition. - for pid := range oldIDs { - if _, ok := c.orphanTables[pid]; ok { - delete(c.orphanTables, pid) - } else { - c.toCleanTables[pid] = startTs - } - } -} - -func (c *changeFeed) tryBalance(ctx context.Context, captures map[string]*model.CaptureInfo, rebalanceNow bool, - manualMoveCommands []*model.MoveTableJob) error { - err := c.balanceOrphanTables(ctx, captures) - if err != nil { - return errors.Trace(err) - } - c.manualMoveCommands = append(c.manualMoveCommands, manualMoveCommands...) - if rebalanceNow { - c.rebalanceNextTick = true - } - c.handleManualMoveTableJobs(captures) - err = c.rebalanceTables(ctx, captures) - if err != nil { - return errors.Trace(err) - } - err = c.handleMoveTableJobs(ctx, captures) - return errors.Trace(err) -} - -func findTaskStatusWithTable(infos model.ProcessorsInfos, tableID model.TableID) (captureID model.CaptureID, info *model.TaskStatus, ok bool) { - for cid, info := range infos { - for tid := range info.Tables { - if tid == tableID { - return cid, info, true - } - } - } - return "", nil, false -} - -func (c *changeFeed) balanceOrphanTables(ctx context.Context, captures map[model.CaptureID]*model.CaptureInfo) error { - if len(captures) == 0 { - return nil - } - - // Do NOT rebalance orphan tables before checkpoint ts has advanced to FinishTs of a DDL - if c.appliedCheckpointTs != c.status.CheckpointTs { - return nil - } - - captureIDs := make(map[model.CaptureID]struct{}, len(captures)) - cleanedTables := make(map[model.TableID]struct{}) - addedTables := make(map[model.TableID]struct{}) - updateFuncs := make(map[model.CaptureID][]kv.UpdateTaskStatusFunc) - for cid := range captures { - captureIDs[cid] = struct{}{} - } - c.scheduler.AlignCapture(captureIDs) - - for id, targetTs := range c.toCleanTables { - captureID, _, ok := findTaskStatusWithTable(c.taskStatus, id) - if !ok { - log.Warn("ignore clean table id", zap.Int64("id", id)) - delete(c.toCleanTables, id) - continue - } - - id := id - targetTs := targetTs - updateFuncs[captureID] = append(updateFuncs[captureID], func(_ int64, status *model.TaskStatus) (bool, error) { - status.RemoveTable(id, targetTs, false /*isMoveTable*/) - return true, nil - }) - cleanedTables[id] = struct{}{} - } - - operations := c.scheduler.DistributeTables(c.orphanTables) - for captureID, operation := range operations { - schemaSnapshot := c.schema - for tableID, op := range operation { - var orphanMarkTableID model.TableID - tableName, found := schemaSnapshot.GetTableNameByID(tableID) - if !found { - log.Warn("balance orphan tables delay, table not found", - zap.String("changefeed", c.id), - zap.Int64("tableID", tableID)) - continue - } - if c.cyclicEnabled { - markTableSchameName, markTableTableName := mark.GetMarkTableName(tableName.Schema, tableName.Table) - orphanMarkTableID, found = schemaSnapshot.GetTableIDByName(markTableSchameName, markTableTableName) - if !found { - // Mark table is not created yet, skip and wait. - log.Info("balance orphan tables delay, wait mark table", - zap.String("changefeed", c.id), - zap.Int64("tableID", tableID), - zap.String("markTableName", markTableTableName)) - continue - } - } - info := &model.TableReplicaInfo{ - StartTs: op.BoundaryTs, - MarkTableID: orphanMarkTableID, - } - tableID := tableID - op := op - updateFuncs[captureID] = append(updateFuncs[captureID], func(_ int64, status *model.TaskStatus) (bool, error) { - status.AddTable(tableID, info, op.BoundaryTs) - return true, nil - }) - addedTables[tableID] = struct{}{} - } - } - - for captureID, funcs := range updateFuncs { - newStatus, _, err := c.etcdCli.LeaseGuardAtomicPutTaskStatus(ctx, c.id, captureID, c.leaseID, funcs...) - if err != nil { - return errors.Trace(err) - } - c.taskStatus[captureID] = newStatus.Clone() - log.Info("dispatch table success", zap.String("capture-id", captureID), zap.Stringer("status", newStatus)) - failpoint.Inject("OwnerRemoveTableError", func() { - if len(cleanedTables) > 0 { - failpoint.Return(errors.New("failpoint injected error")) - } - }) - } - - for tableID := range cleanedTables { - delete(c.toCleanTables, tableID) - } - for tableID := range addedTables { - delete(c.orphanTables, tableID) - } - - return nil -} - -func (c *changeFeed) updateTaskStatus(ctx context.Context, taskStatus map[model.CaptureID]*model.TaskStatus) error { - for captureID, status := range taskStatus { - newStatus, _, err := c.etcdCli.LeaseGuardAtomicPutTaskStatus( - ctx, c.id, captureID, c.leaseID, - func(modRevision int64, taskStatus *model.TaskStatus) (bool, error) { - if taskStatus.SomeOperationsUnapplied() { - log.Error("unexpected task status, there are operations unapplied in this status", zap.Any("status", taskStatus)) - return false, cerror.ErrWaitHandleOperationTimeout.GenWithStackByArgs() - } - taskStatus.Tables = status.Tables - taskStatus.Operation = status.Operation - return true, nil - }) - if err != nil { - return errors.Trace(err) - } - c.taskStatus[captureID] = newStatus.Clone() - log.Info("dispatch table success", zap.String("capture-id", captureID), zap.Stringer("status", status)) - } - return nil -} - -func (c *changeFeed) handleManualMoveTableJobs(captures map[model.CaptureID]*model.CaptureInfo) { - if len(captures) == 0 { - return - } - if len(c.moveTableJobs) > 0 { - return - } - for len(c.manualMoveCommands) > 0 { - moveJob := c.manualMoveCommands[0] - if _, exist := c.moveTableJobs[moveJob.TableID]; exist { - break - } - c.manualMoveCommands = c.manualMoveCommands[1:] - moveJob.From = "" - for captureID, taskStatus := range c.taskStatus { - if _, exist := taskStatus.Tables[moveJob.TableID]; exist { - moveJob.From = captureID - break - } - } - if moveJob.From == "" { - log.Warn("invalid manual move job, the table is not found", zap.Reflect("job", moveJob)) - continue - } - if moveJob.To == moveJob.From { - log.Warn("invalid manual move job, the table is already exists in the target capture", zap.Reflect("job", moveJob)) - continue - } - if _, exist := captures[moveJob.To]; !exist { - log.Warn("invalid manual move job, the target capture is not found", zap.Reflect("job", moveJob)) - continue - } - if c.moveTableJobs == nil { - c.moveTableJobs = make(map[model.TableID]*model.MoveTableJob) - } - c.moveTableJobs[moveJob.TableID] = moveJob - log.Info("received the manual move table job", zap.Reflect("job", moveJob)) - } -} - -func (c *changeFeed) rebalanceTables(ctx context.Context, captures map[model.CaptureID]*model.CaptureInfo) error { - if len(captures) == 0 { - return nil - } - if len(c.moveTableJobs) != 0 { - return nil - } - for _, status := range c.taskStatus { - if status.SomeOperationsUnapplied() { - return nil - } - } - timeToRebalance := time.Since(c.lastRebalanceTime) > time.Duration(c.info.Config.Scheduler.PollingTime)*time.Minute - timeToRebalance = timeToRebalance && c.info.Config.Scheduler.PollingTime > 0 - - if !c.rebalanceNextTick && !timeToRebalance { - return nil - } - c.lastRebalanceTime = time.Now() - c.rebalanceNextTick = false - - captureIDs := make(map[model.CaptureID]struct{}, len(captures)) - for cid := range captures { - captureIDs[cid] = struct{}{} - workloads, err := c.etcdCli.GetTaskWorkload(ctx, c.id, cid) - if err != nil { - return errors.Trace(err) - } - c.scheduler.ResetWorkloads(cid, workloads) - } - c.scheduler.AlignCapture(captureIDs) - - _, moveTableJobs := c.scheduler.CalRebalanceOperates(0) - log.Info("rebalance operations", zap.Reflect("moveTableJobs", moveTableJobs)) - c.moveTableJobs = moveTableJobs - return nil -} - -func (c *changeFeed) handleMoveTableJobs(ctx context.Context, captures map[model.CaptureID]*model.CaptureInfo) error { - if len(captures) == 0 { - return nil - } - if len(c.moveTableJobs) == 0 { - return nil - } - for _, status := range c.taskStatus { - if status.SomeOperationsUnapplied() { - return nil - } - } - newTaskStatus := make(map[model.CaptureID]*model.TaskStatus, len(captures)) - cloneStatus := func(captureID model.CaptureID) (*model.TaskStatus, bool) { - status, exist := newTaskStatus[captureID] - if !exist { - taskStatus := c.taskStatus[captureID] - if taskStatus == nil { - if _, exist := captures[captureID]; !exist { - return nil, false - } - status = new(model.TaskStatus) - } else { - status = taskStatus.Clone() - } - newTaskStatus[captureID] = status - } - return status, true - } - for tableID, job := range c.moveTableJobs { - switch job.Status { - case model.MoveTableStatusNone: - // delete table from original capture - status, exist := cloneStatus(job.From) - if !exist { - delete(c.moveTableJobs, tableID) - log.Warn("ignored the move job, the source capture is not found", zap.Reflect("job", job)) - continue - } - // To ensure that the replication pipeline stops exactly at the boundary TS, - // The boundary TS specified by Remove Table Operation MUST greater or equal to the checkpoint TS of this table. - // So the global resolved TS is a reasonable values. - replicaInfo, exist := status.RemoveTable(tableID, c.status.ResolvedTs, true /*isMoveTable*/) - if !exist { - delete(c.moveTableJobs, tableID) - log.Warn("ignored the move job, the table is not exist in the source capture", zap.Reflect("job", job)) - continue - } - replicaInfo.StartTs = c.status.ResolvedTs - job.TableReplicaInfo = replicaInfo - job.Status = model.MoveTableStatusDeleted - log.Info("handle the move job, remove table from the source capture", zap.Reflect("job", job)) - case model.MoveTableStatusDeleted: - // Do NOT dispatch tables before checkpoint ts has been flushed to Etcd. - if c.appliedCheckpointTs != c.status.CheckpointTs { - log.Debug("handle the move job, waiting for checkpoint ts to be uploaded", - zap.Uint64("applied-checkpoint-ts", c.appliedCheckpointTs), - zap.Uint64("latest-checkpoint-ts", c.status.CheckpointTs)) - continue - } - - // add table to target capture - status, exist := cloneStatus(job.To) - replicaInfo := job.TableReplicaInfo.Clone() - if replicaInfo.StartTs < c.status.CheckpointTs { - replicaInfo.StartTs = c.status.CheckpointTs - } - if !exist { - // the target capture is not exist, add table to orphanTables. - c.orphanTables[tableID] = replicaInfo.StartTs - delete(c.moveTableJobs, tableID) - log.Warn("the target capture is not exist, sent the table to orphanTables", zap.Reflect("job", job)) - continue - } - status.AddTable(tableID, replicaInfo, replicaInfo.StartTs) - job.Status = model.MoveTableStatusFinished - delete(c.moveTableJobs, tableID) - log.Info("handle the move job, add table to the target capture", zap.Reflect("job", job)) - } - } - err := c.updateTaskStatus(ctx, newTaskStatus) - return errors.Trace(err) -} - -func (c *changeFeed) applyJob(job *timodel.Job) (skip bool, err error) { - schemaID := job.SchemaID - if job.BinlogInfo != nil && job.BinlogInfo.TableInfo != nil && c.schema.IsIneligibleTableID(job.BinlogInfo.TableInfo.ID) { - tableID := job.BinlogInfo.TableInfo.ID - if _, exist := c.tables[tableID]; exist { - c.removeTable(schemaID, tableID, job.BinlogInfo.FinishedTS) - } - return true, nil - } - - err = func() error { - // case table id set may change - switch job.Type { - case timodel.ActionCreateSchema: - c.addSchema(schemaID) - case timodel.ActionDropSchema: - c.dropSchema(schemaID, job.BinlogInfo.FinishedTS) - case timodel.ActionCreateTable, timodel.ActionRecoverTable: - addID := job.BinlogInfo.TableInfo.ID - table, exist := c.schema.TableByID(addID) - if !exist { - return cerror.ErrSnapshotTableNotFound.GenWithStackByArgs(addID) - } - c.addTable(table, job.BinlogInfo.FinishedTS) - case timodel.ActionDropTable: - dropID := job.TableID - c.removeTable(schemaID, dropID, job.BinlogInfo.FinishedTS) - case timodel.ActionRenameTable: - tableName, exist := c.schema.GetTableNameByID(job.TableID) - if !exist { - return cerror.ErrSnapshotTableNotFound.GenWithStackByArgs(job.TableID) - } - // no id change just update name - c.tables[job.TableID] = tableName - case timodel.ActionTruncateTable: - dropID := job.TableID - c.removeTable(schemaID, dropID, job.BinlogInfo.FinishedTS) - - addID := job.BinlogInfo.TableInfo.ID - table, exist := c.schema.TableByID(addID) - if !exist { - return cerror.ErrSnapshotTableNotFound.GenWithStackByArgs(addID) - } - c.addTable(table, job.BinlogInfo.FinishedTS) - case timodel.ActionTruncateTablePartition, timodel.ActionAddTablePartition, timodel.ActionDropTablePartition: - c.updatePartition(job.BinlogInfo.TableInfo, job.BinlogInfo.FinishedTS) - } - return nil - }() - if err != nil { - log.Error("failed to applyJob, start to print debug info", zap.Error(err)) - c.schema.PrintStatus(log.Error) - } - return false, err -} - -// handleDDL check if we can change the status to be `ChangeFeedExecDDL` and execute the DDL asynchronously -// if the status is in ChangeFeedWaitToExecDDL. -// After executing the DDL successfully, the status will be changed to be ChangeFeedSyncDML. -func (c *changeFeed) handleDDL(ctx context.Context) error { - if c.ddlState != model.ChangeFeedWaitToExecDDL { - return nil - } - - if len(c.ddlJobHistory) == 0 { - log.Panic("ddl job history can not be empty in changefeed when should to execute DDL") - } - todoDDLJob := c.ddlJobHistory[0] - - // Check if all the checkpointTs of capture are achieving global resolvedTs(which is equal to todoDDLJob.FinishedTS) - if len(c.taskStatus) > len(c.taskPositions) { - return nil - } - - if c.appliedCheckpointTs < todoDDLJob.BinlogInfo.FinishedTS-1 { - log.Debug("wait checkpoint ts", - zap.Uint64("checkpoint ts", c.status.CheckpointTs), - zap.Uint64("applied checkpoint ts", c.appliedCheckpointTs), - zap.Uint64("finish ts", todoDDLJob.BinlogInfo.FinishedTS), - zap.String("ddl query", todoDDLJob.Query)) - return nil - } - - if c.appliedCheckpointTs >= todoDDLJob.BinlogInfo.FinishedTS { - log.Panic("applied checkpoint ts is larger than DDL finish ts", - zap.Uint64("applied checkpoint ts", c.appliedCheckpointTs), - zap.Uint64("finish ts", todoDDLJob.BinlogInfo.FinishedTS)) - } - - log.Info("apply job", zap.Stringer("job", todoDDLJob), - zap.String("schema", todoDDLJob.SchemaName), - zap.String("query", todoDDLJob.Query), - zap.Uint64("start-ts", todoDDLJob.StartTS), - zap.Uint64("ts", todoDDLJob.BinlogInfo.FinishedTS)) - - ddlEvent := new(model.DDLEvent) - preTableInfo, err := c.schema.PreTableInfo(todoDDLJob) - if err != nil { - return errors.Trace(err) - } - err = c.schema.HandleDDL(todoDDLJob) - if err != nil { - return errors.Trace(err) - } - err = c.schema.FillSchemaName(todoDDLJob) - if err != nil { - return errors.Trace(err) - } - - ddlEvent.FromJob(todoDDLJob, preTableInfo) - - // Execute DDL Job asynchronously - c.ddlState = model.ChangeFeedExecDDL - - // TODO consider some newly added DDL types such as `ActionCreateSequence` - skip, err := c.applyJob(todoDDLJob) - if err != nil { - return errors.Trace(err) - } - if skip { - log.Info("ddl job ignored", zap.String("changefeed", c.id), zap.Reflect("job", todoDDLJob)) - c.ddlJobHistory = c.ddlJobHistory[1:] - c.ddlExecutedTs = todoDDLJob.BinlogInfo.FinishedTS - c.ddlState = model.ChangeFeedSyncDML - return nil - } - - executed := false - if !c.cyclicEnabled || c.info.Config.Cyclic.SyncDDL { - failpoint.Inject("InjectChangefeedDDLError", func() { - failpoint.Return(cerror.ErrExecDDLFailed.GenWithStackByArgs()) - }) - - ddlEvent.Query = binloginfo.AddSpecialComment(ddlEvent.Query) - log.Debug("DDL processed to make special features mysql-compatible", zap.String("query", ddlEvent.Query)) - err = c.sink.EmitDDLEvent(ctx, ddlEvent) - // If DDL executing failed, pause the changefeed and print log, rather - // than return an error and break the running of this owner. - if err != nil { - if cerror.ErrDDLEventIgnored.NotEqual(err) { - c.ddlState = model.ChangeFeedDDLExecuteFailed - log.Error("Execute DDL failed", - zap.String("ChangeFeedID", c.id), - zap.Error(err), - zap.Reflect("ddlJob", todoDDLJob)) - return cerror.ErrExecDDLFailed.GenWithStackByArgs() - } - } else { - executed = true - } - } - if executed { - log.Info("Execute DDL succeeded", zap.String("changefeed", c.id), zap.Reflect("ddlJob", todoDDLJob)) - } else { - log.Info("Execute DDL ignored", zap.String("changefeed", c.id), zap.Reflect("ddlJob", todoDDLJob)) - } - - c.ddlJobHistory = c.ddlJobHistory[1:] - c.ddlExecutedTs = todoDDLJob.BinlogInfo.FinishedTS - c.ddlState = model.ChangeFeedSyncDML - return nil -} - -// handleSyncPoint record every syncpoint to downstream if the syncpoint feature is enable -func (c *changeFeed) handleSyncPoint(ctx context.Context) error { - // sync-point on - if c.info.SyncPointEnabled { - c.syncpointMutex.Lock() - defer c.syncpointMutex.Unlock() - // ticker and ddl can trigger syncpoint record at the same time, only record once - syncpointRecorded := false - // ResolvedTs == CheckpointTs means a syncpoint reached; - // !c.updateResolvedTs means the syncpoint is setted by ticker; - // c.ddlTs == 0 means no DDL wait to exec and we can sink the syncpoint record securely ( c.ddlTs != 0 means some DDL should be sink to downstream and this syncpoint is fake ). - if c.status.ResolvedTs == c.status.CheckpointTs && !c.updateResolvedTs { - log.Info("sync point reached by ticker", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs), zap.Uint64("ddlExecutedTs", c.ddlExecutedTs)) - c.updateResolvedTs = true - err := c.syncpointStore.SinkSyncpoint(ctx, c.id, c.status.CheckpointTs) - if err != nil { - log.Error("syncpoint sink fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Error(err)) - return err - } - syncpointRecorded = true - } - - if c.status.ResolvedTs == 0 { - c.updateResolvedTs = true - } - - // ResolvedTs == CheckpointTs means a syncpoint reached; - // ResolvedTs == ddlTs means the syncpoint is setted by DDL; - // ddlTs <= ddlExecutedTs means the DDL has been execed. - if c.status.ResolvedTs == c.status.CheckpointTs && c.status.ResolvedTs == c.ddlTs && c.ddlTs <= c.ddlExecutedTs { - log.Info("sync point reached by ddl", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs), zap.Uint64("ddlExecutedTs", c.ddlExecutedTs)) - if !syncpointRecorded { - err := c.syncpointStore.SinkSyncpoint(ctx, c.id, c.status.CheckpointTs) - if err != nil { - log.Error("syncpoint sink fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Error(err)) - return err - } - } - c.ddlTs = 0 - } - } - return nil -} - -// calcResolvedTs update every changefeed's resolve ts and checkpoint ts. -func (c *changeFeed) calcResolvedTs(ctx context.Context) error { - if c.ddlState != model.ChangeFeedSyncDML && c.ddlState != model.ChangeFeedWaitToExecDDL { - log.Debug("skip update resolved ts", zap.String("ddlState", c.ddlState.String())) - return nil - } - - minResolvedTs := c.targetTs - minCheckpointTs := c.targetTs - - // prevMinResolvedTs and prevMinCheckpointTs are used for debug - prevMinResolvedTs := c.targetTs - prevMinCheckpointTs := c.targetTs - checkUpdateTs := func() { - if prevMinCheckpointTs != minCheckpointTs { - log.L().WithOptions(zap.AddCallerSkip(1)).Debug("min checkpoint updated", - zap.Uint64("prevMinCheckpointTs", prevMinCheckpointTs), - zap.Uint64("minCheckpointTs", minCheckpointTs)) - prevMinCheckpointTs = minCheckpointTs - } - if prevMinResolvedTs != minResolvedTs { - log.L().WithOptions(zap.AddCallerSkip(1)).Debug("min resolved updated", - zap.Uint64("prevMinResolvedTs", prevMinResolvedTs), - zap.Uint64("minResolvedTs", minResolvedTs)) - prevMinResolvedTs = minResolvedTs - } - } - - if len(c.taskPositions) < len(c.taskStatus) { - log.Debug("skip update resolved ts", - zap.Int("taskPositions", len(c.taskPositions)), - zap.Int("taskStatus", len(c.taskStatus))) - return nil - } - if len(c.taskPositions) == 0 { - minCheckpointTs = c.status.ResolvedTs - } else { - // calc the min of all resolvedTs in captures - for _, position := range c.taskPositions { - if minResolvedTs > position.ResolvedTs { - minResolvedTs = position.ResolvedTs - } - - if minCheckpointTs > position.CheckPointTs { - minCheckpointTs = position.CheckPointTs - } - } - } - prevMinCheckpointTs = minCheckpointTs - prevMinResolvedTs = minResolvedTs - - for captureID, status := range c.taskStatus { - appliedTs := status.AppliedTs() - if minCheckpointTs > appliedTs { - minCheckpointTs = appliedTs - } - if minResolvedTs > appliedTs { - minResolvedTs = appliedTs - } - if appliedTs != math.MaxUint64 { - log.Debug("some operation is still unapplied", - zap.String("capture-id", captureID), - zap.Uint64("appliedTs", appliedTs), - zap.Stringer("status", status)) - } - } - checkUpdateTs() - - for _, startTs := range c.orphanTables { - if minCheckpointTs > startTs { - minCheckpointTs = startTs - } - if minResolvedTs > startTs { - minResolvedTs = startTs - } - } - checkUpdateTs() - - for _, targetTs := range c.toCleanTables { - if minCheckpointTs > targetTs { - minCheckpointTs = targetTs - } - if minResolvedTs > targetTs { - minResolvedTs = targetTs - } - } - checkUpdateTs() - - // if minResolvedTs is greater than ddlResolvedTs, - // it means that ddlJobHistory in memory is not intact, - // there are some ddl jobs which finishedTs is smaller than minResolvedTs we don't know. - // so we need to call `pullDDLJob`, update the ddlJobHistory and ddlResolvedTs. - if minResolvedTs > c.ddlResolvedTs { - if err := c.pullDDLJob(); err != nil { - return errors.Trace(err) - } - - if minResolvedTs > c.ddlResolvedTs { - minResolvedTs = c.ddlResolvedTs - } - } - checkUpdateTs() - - // if minResolvedTs is greater than the finishedTS of ddl job which is not executed, - // we need to execute this ddl job - for len(c.ddlJobHistory) > 0 && c.ddlJobHistory[0].BinlogInfo.FinishedTS <= c.ddlExecutedTs { - c.ddlJobHistory = c.ddlJobHistory[1:] - } - if len(c.ddlJobHistory) > 0 && minResolvedTs >= c.ddlJobHistory[0].BinlogInfo.FinishedTS { - minResolvedTs = c.ddlJobHistory[0].BinlogInfo.FinishedTS - c.ddlState = model.ChangeFeedWaitToExecDDL - c.ddlTs = minResolvedTs - } - - if len(c.ddlJobHistory) > 0 && minCheckpointTs >= c.ddlJobHistory[0].BinlogInfo.FinishedTS { - minCheckpointTs = c.ddlJobHistory[0].BinlogInfo.FinishedTS - 1 - } - - // if downstream sink is the MQ sink, the MQ sink do not promise that checkpoint is less than globalResolvedTs - if minCheckpointTs > minResolvedTs { - minCheckpointTs = minResolvedTs - } - checkUpdateTs() - - var tsUpdated bool - - // syncpoint on - if c.info.SyncPointEnabled { - c.syncpointMutex.Lock() - if c.updateResolvedTs && minResolvedTs > c.status.ResolvedTs { - c.status.ResolvedTs = minResolvedTs - tsUpdated = true - } - c.syncpointMutex.Unlock() - } else if minResolvedTs > c.status.ResolvedTs { - c.status.ResolvedTs = minResolvedTs - tsUpdated = true - } - - if minCheckpointTs > c.status.CheckpointTs { - c.status.CheckpointTs = minCheckpointTs - // when the `c.ddlState` is `model.ChangeFeedWaitToExecDDL`, - // some DDL is waiting to executed, we can't ensure whether the DDL has been executed. - // so we can't emit checkpoint to sink - if c.ddlState != model.ChangeFeedWaitToExecDDL { - failpoint.Inject("InjectEmitCheckpointTsError", func() { - failpoint.Return(cerror.ErrEmitCheckpointTsFailed.GenWithStackByArgs()) - }) - err := c.sink.EmitCheckpointTs(ctx, minCheckpointTs) - if err != nil { - return errors.Trace(err) - } - } - tsUpdated = true - } - checkUpdateTs() - - if tsUpdated { - log.Debug("update changefeed", zap.String("id", c.id), - zap.Uint64("checkpoint ts", c.status.CheckpointTs), - zap.Uint64("resolved ts", c.status.ResolvedTs)) - } - return nil -} - -func (c *changeFeed) pullDDLJob() error { - ddlResolvedTs, ddlJobs, err := c.ddlHandler.PullDDL() - if err != nil { - return errors.Trace(err) - } - c.ddlResolvedTs = ddlResolvedTs - for _, ddl := range ddlJobs { - if c.filter.ShouldDiscardDDL(ddl.Type) { - log.Info("discard the ddl job", zap.Int64("jobID", ddl.ID), zap.String("query", ddl.Query)) - continue - } - c.ddlJobHistory = append(c.ddlJobHistory, ddl) - } - return nil -} - -// startSyncPeriod start a timer for every changefeed to create sync point by time -func (c *changeFeed) startSyncPeriod(ctx context.Context, interval time.Duration) { - log.Debug("sync ticker start", zap.Duration("sync-interval", interval)) - go func(ctx context.Context) { - ticker := time.NewTicker(interval) - for { - select { - case <-ctx.Done(): - return - case <-ticker.C: - c.syncpointMutex.Lock() - c.updateResolvedTs = false - c.syncpointMutex.Unlock() - } - } - }(ctx) -} - -func (c *changeFeed) stopSyncPointTicker() { - if c.syncCancel != nil { - c.syncCancel() - c.syncCancel = nil - } -} - -func (c *changeFeed) startSyncPointTicker(ctx context.Context, interval time.Duration) { - var syncCtx context.Context - syncCtx, c.syncCancel = context.WithCancel(ctx) - c.startSyncPeriod(syncCtx, interval) -} - -func (c *changeFeed) Close() { - if c.ddlHandler != nil { - err := c.ddlHandler.Close() - if err != nil && errors.Cause(err) != context.Canceled { - log.Warn("failed to close ddl handler", zap.Error(err)) - } - } - - ctx, cancel := context.WithCancel(context.Background()) - cancel() - if c.sink != nil { - // pass a canceled context is enough, since the Close of backend sink - // here doesn't use context actually. - err := c.sink.Close(ctx) - if err != nil && errors.Cause(err) != context.Canceled { - log.Warn("failed to close owner sink", zap.Error(err)) - } - } - - if c.syncpointStore != nil { - err := c.syncpointStore.Close() - if err != nil && errors.Cause(err) != context.Canceled { - log.Warn("failed to close owner sink", zap.Error(err)) - } - } - - if c.cancel != nil { - c.cancel() - } - log.Info("changefeed closed", zap.String("id", c.id)) -} diff --git a/cdc/changefeed_test.go b/cdc/changefeed_test.go deleted file mode 100644 index 52c733240a6..00000000000 --- a/cdc/changefeed_test.go +++ /dev/null @@ -1,130 +0,0 @@ -// 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 cdc - -import ( - "context" - "net/url" - "time" - - "github.com/pingcap/check" - "github.com/pingcap/tiflow/cdc/kv" - "github.com/pingcap/tiflow/cdc/model" - "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/clientv3/concurrency" - "go.etcd.io/etcd/embed" - "golang.org/x/sync/errgroup" -) - -type changefeedSuite struct { - e *embed.Etcd - clientURL *url.URL - client kv.CDCEtcdClient - sess *concurrency.Session - ctx context.Context - cancel context.CancelFunc - errg *errgroup.Group -} - -var _ = check.Suite(&changefeedSuite{}) - -func (s *changefeedSuite) SetUpTest(c *check.C) { - dir := c.MkDir() - var err error - s.clientURL, s.e, err = etcd.SetupEmbedEtcd(dir) - c.Assert(err, check.IsNil) - client, err := clientv3.New(clientv3.Config{ - Endpoints: []string{s.clientURL.String()}, - DialTimeout: 3 * time.Second, - }) - c.Assert(err, check.IsNil) - sess, err := concurrency.NewSession(client) - c.Assert(err, check.IsNil) - s.sess = sess - s.client = kv.NewCDCEtcdClient(context.Background(), client) - s.ctx, s.cancel = context.WithCancel(context.Background()) - s.errg = util.HandleErrWithErrGroup(s.ctx, s.e.Err(), func(e error) { c.Log(e) }) -} - -func (s *changefeedSuite) TearDownTest(c *check.C) { - s.e.Close() - s.cancel() - err := s.errg.Wait() - if err != nil { - c.Errorf("Error group error: %s", err) - } - s.client.Close() //nolint:errcheck -} - -func (s *changefeedSuite) TestHandleMoveTableJobs(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - changefeed := new(changeFeed) - captureID1 := "capture1" - captureID2 := "capture2" - changefeed.id = "changefeed-test" - changefeed.leaseID = s.sess.Lease() - changefeed.etcdCli = s.client - changefeed.status = new(model.ChangeFeedStatus) - changefeed.orphanTables = map[model.TableID]model.Ts{} - captures := make(map[model.CaptureID]*model.CaptureInfo) - captures[captureID1] = &model.CaptureInfo{ - ID: captureID1, - } - captures[captureID2] = &model.CaptureInfo{ - ID: captureID2, - } - changefeed.taskStatus = make(map[model.CaptureID]*model.TaskStatus) - changefeed.taskStatus[captureID1] = &model.TaskStatus{Tables: map[model.TableID]*model.TableReplicaInfo{1: {}}} - changefeed.taskStatus[captureID2] = &model.TaskStatus{Tables: map[model.TableID]*model.TableReplicaInfo{}} - changefeed.moveTableJobs = make(map[model.TableID]*model.MoveTableJob) - changefeed.moveTableJobs[1] = &model.MoveTableJob{ - TableID: 1, - From: captureID1, - To: captureID2, - TableReplicaInfo: new(model.TableReplicaInfo), - } - err := changefeed.handleMoveTableJobs(s.ctx, captures) - c.Assert(err, check.IsNil) - taskStatuses, err := s.client.GetAllTaskStatus(s.ctx, changefeed.id) - c.Assert(err, check.IsNil) - taskStatuses[captureID1].ModRevision = 0 - c.Assert(taskStatuses, check.DeepEquals, model.ProcessorsInfos{captureID1: { - Tables: map[model.TableID]*model.TableReplicaInfo{}, - Operation: map[model.TableID]*model.TableOperation{1: { - Delete: true, - Flag: model.OperFlagMoveTable, - }}, - }}) - - // finish operation - err = s.client.PutTaskStatus(s.ctx, changefeed.id, captureID1, &model.TaskStatus{ - Tables: map[model.TableID]*model.TableReplicaInfo{}, - Operation: map[model.TableID]*model.TableOperation{}, - }) - c.Assert(err, check.IsNil) - delete(changefeed.taskStatus[captureID1].Operation, 1) - - // capture2 offline - delete(captures, captureID2) - delete(changefeed.taskStatus, captureID2) - - err = changefeed.handleMoveTableJobs(s.ctx, captures) - c.Assert(err, check.IsNil) - c.Assert(changefeed.orphanTables, check.HasKey, model.TableID(1)) - c.Assert(changefeed.moveTableJobs, check.HasLen, 0) -} diff --git a/cdc/metrics.go b/cdc/metrics.go index 37cc38e011d..657b0d348d1 100644 --- a/cdc/metrics.go +++ b/cdc/metrics.go @@ -22,7 +22,6 @@ import ( "github.com/pingcap/tiflow/cdc/puller" "github.com/pingcap/tiflow/cdc/puller/sorter" "github.com/pingcap/tiflow/cdc/sink" - "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/orchestrator" "github.com/prometheus/client_golang/prometheus" ) @@ -39,13 +38,8 @@ func init() { entry.InitMetrics(registry) sorter.InitMetrics(registry) orchestrator.InitMetrics(registry) - if config.NewReplicaImpl { - processor.InitMetrics(registry) - tablepipeline.InitMetrics(registry) - owner.InitMetrics(registry) - } else { - initProcessorMetrics(registry) - initOwnerMetrics(registry) - } + processor.InitMetrics(registry) + tablepipeline.InitMetrics(registry) + owner.InitMetrics(registry) initServerMetrics(registry) } diff --git a/cdc/metrics_owner.go b/cdc/metrics_owner.go deleted file mode 100644 index 2807ecff7c1..00000000000 --- a/cdc/metrics_owner.go +++ /dev/null @@ -1,62 +0,0 @@ -// Copyright 2020 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 cdc - -import "github.com/prometheus/client_golang/prometheus" - -var ( - changefeedCheckpointTsGauge = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "owner", - Name: "checkpoint_ts", - Help: "checkpoint ts of changefeeds", - }, []string{"changefeed"}) - changefeedCheckpointTsLagGauge = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "owner", - Name: "checkpoint_ts_lag", - Help: "checkpoint ts lag of changefeeds", - }, []string{"changefeed"}) - ownershipCounter = prometheus.NewCounter( - prometheus.CounterOpts{ - Namespace: "ticdc", - Subsystem: "owner", - Name: "ownership_counter", - Help: "The counter of ownership increases every 5 seconds on a owner capture", - }) - ownerMaintainTableNumGauge = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "owner", - Name: "maintain_table_num", - Help: "number of replicated tables maintained in owner", - }, []string{"changefeed", "capture", "type"}) -) - -const ( - // total tables that have been dispatched to a single processor - maintainTableTypeTotal string = "total" - // tables that are dispatched to a processor and have not been finished yet - maintainTableTypeWip string = "wip" -) - -// initOwnerMetrics registers all metrics used in owner -func initOwnerMetrics(registry *prometheus.Registry) { - registry.MustRegister(changefeedCheckpointTsGauge) - registry.MustRegister(changefeedCheckpointTsLagGauge) - registry.MustRegister(ownershipCounter) - registry.MustRegister(ownerMaintainTableNumGauge) -} diff --git a/cdc/metrics_processor.go b/cdc/metrics_processor.go deleted file mode 100644 index 4065327521c..00000000000 --- a/cdc/metrics_processor.go +++ /dev/null @@ -1,98 +0,0 @@ -// Copyright 2020 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 cdc - -import ( - "github.com/prometheus/client_golang/prometheus" -) - -var ( - resolvedTsGauge = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "processor", - Name: "resolved_ts", - Help: "local resolved ts of processor", - }, []string{"changefeed", "capture"}) - resolvedTsLagGauge = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "processor", - Name: "resolved_ts_lag", - Help: "local resolved ts lag of processor", - }, []string{"changefeed", "capture"}) - tableResolvedTsGauge = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "processor", - Name: "table_resolved_ts", - Help: "local resolved ts of processor", - }, []string{"changefeed", "capture"}) - checkpointTsGauge = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "processor", - Name: "checkpoint_ts", - Help: "global checkpoint ts of processor", - }, []string{"changefeed", "capture"}) - checkpointTsLagGauge = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "processor", - Name: "checkpoint_ts_lag", - Help: "global checkpoint ts lag of processor", - }, []string{"changefeed", "capture"}) - syncTableNumGauge = prometheus.NewGaugeVec( - prometheus.GaugeOpts{ - Namespace: "ticdc", - Subsystem: "processor", - Name: "num_of_tables", - Help: "number of synchronized table of processor", - }, []string{"changefeed", "capture"}) - txnCounter = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Namespace: "ticdc", - Subsystem: "processor", - Name: "txn_count", - Help: "txn count received/executed by this processor", - }, []string{"type", "changefeed", "capture"}) - updateInfoDuration = prometheus.NewHistogramVec( - prometheus.HistogramOpts{ - Namespace: "ticdc", - Subsystem: "processor", - Name: "update_info_duration_seconds", - Help: "The time it took to update sub changefeed info.", - Buckets: prometheus.ExponentialBuckets(0.001 /* 1 ms */, 2, 18), - }, []string{"capture"}) - processorErrorCounter = prometheus.NewCounterVec( - prometheus.CounterOpts{ - Namespace: "ticdc", - Subsystem: "processor", - Name: "exit_with_error_count", - Help: "counter for processor exits with error", - }, []string{"changefeed", "capture"}) -) - -// initProcessorMetrics registers all metrics used in processor -func initProcessorMetrics(registry *prometheus.Registry) { - registry.MustRegister(resolvedTsGauge) - registry.MustRegister(resolvedTsLagGauge) - registry.MustRegister(tableResolvedTsGauge) - registry.MustRegister(checkpointTsGauge) - registry.MustRegister(checkpointTsLagGauge) - registry.MustRegister(syncTableNumGauge) - registry.MustRegister(txnCounter) - registry.MustRegister(updateInfoDuration) - registry.MustRegister(processorErrorCounter) -} diff --git a/cdc/owner.go b/cdc/owner.go deleted file mode 100644 index 4bbd5ef9283..00000000000 --- a/cdc/owner.go +++ /dev/null @@ -1,1734 +0,0 @@ -// Copyright 2020 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 cdc - -import ( - "context" - "fmt" - "io" - "math" - "sync" - "sync/atomic" - "time" - - "github.com/pingcap/errors" - "github.com/pingcap/failpoint" - "github.com/pingcap/log" - "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" - "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/scheduler" - "github.com/pingcap/tiflow/pkg/txnutil/gc" - "github.com/pingcap/tiflow/pkg/util" - "github.com/tikv/client-go/v2/oracle" - pd "github.com/tikv/pd/client" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/concurrency" - "go.etcd.io/etcd/mvcc" - "go.uber.org/zap" - "golang.org/x/time/rate" -) - -type ownership struct { - lastTickTime time.Time - tickTime time.Duration -} - -func newOwnership(tickTime time.Duration) ownership { - minTickTime := 5 * time.Second - if tickTime > minTickTime { - log.Panic("ownership counter must be incearsed every 5 seconds") - } - return ownership{ - tickTime: minTickTime, - } -} - -func (o *ownership) inc() { - now := time.Now() - if now.Sub(o.lastTickTime) > o.tickTime { - // Keep the value of promtheus expression `rate(counter)` = 1 - // Please also change alert rule in ticdc.rules.yml when change the expression value. - ownershipCounter.Add(float64(o.tickTime / time.Second)) - o.lastTickTime = now - } -} - -type minGCSafePointCacheEntry struct { - ts model.Ts - lastUpdated time.Time -} - -func (o *Owner) getMinGCSafePointCache(ctx context.Context) model.Ts { - if time.Now().After(o.minGCSafePointCache.lastUpdated.Add(MinGCSafePointCacheUpdateInterval)) { - physicalTs, logicalTs, err := o.pdClient.GetTS(ctx) - if err != nil { - log.Warn("Fail to update minGCSafePointCache.", zap.Error(err)) - return o.minGCSafePointCache.ts - } - o.minGCSafePointCache.ts = oracle.ComposeTS(physicalTs-(o.gcTTL*1000), logicalTs) - - // o.pdGCSafePoint pd is the smallest gcSafePoint across all services. - // If tikv_gc_life_time > gcTTL, means that tikv_gc_safe_point < o.minGCSafePointCache.ts here. - // It also means that pd.pdGCSafePoint < o.minGCSafePointCache.ts here, we should use its value as the min value. - // This ensures that when tikv_gc_life_time > gcTTL , cdc will not advance the gcSafePoint. - if o.pdGCSafePoint < o.minGCSafePointCache.ts { - o.minGCSafePointCache.ts = o.pdGCSafePoint - } - - o.minGCSafePointCache.lastUpdated = time.Now() - } - return o.minGCSafePointCache.ts -} - -// Owner manages the cdc cluster -type Owner struct { - done chan struct{} - session *concurrency.Session - changeFeeds map[model.ChangeFeedID]*changeFeed - // failInitFeeds record changefeeds that meet error during initialization - failInitFeeds map[model.ChangeFeedID]struct{} - // stoppedFeeds record changefeeds that meet running error - stoppedFeeds map[model.ChangeFeedID]*model.ChangeFeedStatus - rebalanceTigger map[model.ChangeFeedID]bool - rebalanceForAllChangefeed bool - manualScheduleCommand map[model.ChangeFeedID][]*model.MoveTableJob - rebalanceMu sync.Mutex - - cfRWriter ChangeFeedRWriter - - l sync.RWMutex - - pdEndpoints []string - grpcPool kv.GrpcPool - pdClient pd.Client - etcdClient kv.CDCEtcdClient - - captureLoaded int32 - captures map[model.CaptureID]*model.CaptureInfo - - adminJobs []model.AdminJob - adminJobsLock sync.Mutex - - stepDown func(ctx context.Context) error - - // gcTTL is the ttl of cdc gc safepoint ttl. - gcTTL int64 - // last update gc safepoint time. zero time means has not updated or cleared - gcSafepointLastUpdate time.Time - // stores the ts obtained from PD and is updated every MinGCSafePointCacheUpdateInterval. - minGCSafePointCache minGCSafePointCacheEntry - // stores the actual gcSafePoint stored in pd - pdGCSafePoint model.Ts - // record last time that flushes all changefeeds' replication status - lastFlushChangefeeds time.Time - flushChangefeedInterval time.Duration - feedChangeNotifier *notify.Notifier -} - -const ( - // CDCServiceSafePointID is the ID of CDC service in pd.UpdateServiceGCSafePoint. - CDCServiceSafePointID = "ticdc" - // GCSafepointUpdateInterval is the minimual interval that CDC can update gc safepoint - GCSafepointUpdateInterval = 2 * time.Second - // MinGCSafePointCacheUpdateInterval is the interval that update minGCSafePointCache - MinGCSafePointCacheUpdateInterval = time.Second * 2 -) - -// NewOwner creates a new Owner instance -func NewOwner( - ctx context.Context, - pdClient pd.Client, - grpcPool kv.GrpcPool, - sess *concurrency.Session, - gcTTL int64, - flushChangefeedInterval time.Duration, -) (*Owner, error) { - cli := kv.NewCDCEtcdClient(ctx, sess.Client()) - endpoints := sess.Client().Endpoints() - - failpoint.Inject("ownerFlushIntervalInject", func(val failpoint.Value) { - flushChangefeedInterval = time.Millisecond * time.Duration(val.(int)) - }) - - owner := &Owner{ - done: make(chan struct{}), - session: sess, - pdClient: pdClient, - grpcPool: grpcPool, - changeFeeds: make(map[model.ChangeFeedID]*changeFeed), - failInitFeeds: make(map[model.ChangeFeedID]struct{}), - stoppedFeeds: make(map[model.ChangeFeedID]*model.ChangeFeedStatus), - captures: make(map[model.CaptureID]*model.CaptureInfo), - rebalanceTigger: make(map[model.ChangeFeedID]bool), - manualScheduleCommand: make(map[model.ChangeFeedID][]*model.MoveTableJob), - pdEndpoints: endpoints, - cfRWriter: cli, - etcdClient: cli, - gcTTL: gcTTL, - flushChangefeedInterval: flushChangefeedInterval, - feedChangeNotifier: new(notify.Notifier), - } - - return owner, nil -} - -func (o *Owner) addCapture(_ context.Context, info *model.CaptureInfo) { - o.l.Lock() - o.captures[info.ID] = info - o.l.Unlock() - o.rebalanceMu.Lock() - o.rebalanceForAllChangefeed = true - o.rebalanceMu.Unlock() -} - -// When a table is moved from one capture to another, the workflow is as follows -// 1. Owner deletes the table from the original capture (we call it capture-1), -// and adds an table operation record in the task status -// 2. The processor in capture-1 reads the operation record, and waits the table -// checkpoint ts reaches the boundary ts in operation, which often equals to -// the global resovled ts, larger the current checkpoint ts of this table. -// 3. After table checkpoint ts reaches boundary ts, capture-1 marks the table -// operation as finished. -// 4. Owner reads the finished mark and re-dispatches this table to another capture. -// -// When capture-1 crashes between step-2 and step-3, this function should be -// called to let owner re dispatch the table. Besides owner could also crash at -// the same time, in that case this function should also be called. In addtition, -// this function only handles move table job: 1) the add table job persists both -// table replicaInfo and operation, we can recover enough information from table -// replicaInfo; 2) if a table is deleted from a capture and that capture crashes, -// we just ignore this table. -func (o *Owner) rebuildTableFromOperations(cf *changeFeed, taskStatus *model.TaskStatus, startTs uint64) { - for tableID, op := range taskStatus.Operation { - if op.Delete && op.Flag&model.OperFlagMoveTable > 0 { - cf.orphanTables[tableID] = startTs - if job, ok := cf.moveTableJobs[tableID]; ok { - log.Info("remove outdated move table job", zap.Reflect("job", job), zap.Uint64("start-ts", startTs)) - delete(cf.moveTableJobs, tableID) - } - } - } -} - -func (o *Owner) removeCapture(ctx context.Context, info *model.CaptureInfo) { - o.l.Lock() - defer o.l.Unlock() - - delete(o.captures, info.ID) - - for _, feed := range o.changeFeeds { - task, ok := feed.taskStatus[info.ID] - if !ok { - log.Warn("task status not found", zap.String("capture-id", info.ID), zap.String("changefeed", feed.id)) - continue - } - var startTs uint64 - pos, ok := feed.taskPositions[info.ID] - if ok { - startTs = pos.CheckPointTs - } else { - log.Warn("task position not found, fallback to use changefeed checkpointts", - zap.String("capture-id", info.ID), zap.String("changefeed", feed.id)) - // maybe the processor hasn't added table yet, fallback to use the - // global checkpoint ts as the start ts of the table. - startTs = feed.status.CheckpointTs - } - - for tableID, replicaInfo := range task.Tables { - feed.orphanTables[tableID] = startTs - if startTs < replicaInfo.StartTs { - log.Warn("table startTs not consistent", - zap.Uint64("table-start-ts", replicaInfo.StartTs), - zap.Uint64("checkpoint-ts", startTs), - zap.Reflect("status", feed.status)) - feed.orphanTables[tableID] = replicaInfo.StartTs - } - } - - o.rebuildTableFromOperations(feed, task, startTs) - - if err := o.etcdClient.LeaseGuardDeleteTaskStatus(ctx, feed.id, info.ID, o.session.Lease()); err != nil { - log.Warn("failed to delete task status", - zap.String("capture-id", info.ID), zap.String("changefeed", feed.id), zap.Error(err)) - } - if err := o.etcdClient.LeaseGuardDeleteTaskPosition(ctx, feed.id, info.ID, o.session.Lease()); err != nil { - log.Warn("failed to delete task position", - zap.String("capture-id", info.ID), zap.String("changefeed", feed.id), zap.Error(err)) - } - if err := o.etcdClient.LeaseGuardDeleteTaskWorkload(ctx, feed.id, info.ID, o.session.Lease()); err != nil { - log.Warn("failed to delete task workload", - zap.String("capture-id", info.ID), zap.String("changefeed", feed.id), zap.Error(err)) - } - ownerMaintainTableNumGauge.DeleteLabelValues(feed.id, info.AdvertiseAddr, maintainTableTypeTotal) - ownerMaintainTableNumGauge.DeleteLabelValues(feed.id, info.AdvertiseAddr, maintainTableTypeWip) - } -} - -func (o *Owner) addOrphanTable(cid model.CaptureID, tableID model.TableID, startTs model.Ts) { - if cf, ok := o.changeFeeds[cid]; ok { - cf.orphanTables[tableID] = startTs - } else { - log.Warn("changefeed not found", zap.String("changefeed", cid)) - } -} - -func (o *Owner) newChangeFeed( - ctx context.Context, - id model.ChangeFeedID, - processorsInfos model.ProcessorsInfos, - taskPositions map[string]*model.TaskPosition, - info *model.ChangeFeedInfo, - checkpointTs uint64) (cf *changeFeed, resultErr error) { - log.Info("Find new changefeed", zap.Stringer("info", info), - zap.String("changefeed", id), zap.Uint64("checkpoint ts", checkpointTs)) - if info.Config.CheckGCSafePoint { - ensureTTL := int64(10 * 60) - err := gc.EnsureChangefeedStartTsSafety( - ctx, o.pdClient, id, ensureTTL, checkpointTs) - if err != nil { - return nil, errors.Trace(err) - } - } - failpoint.Inject("NewChangefeedNoRetryError", func() { - failpoint.Return(nil, cerror.ErrStartTsBeforeGC.GenWithStackByArgs(checkpointTs-300, checkpointTs)) - }) - - failpoint.Inject("NewChangefeedRetryError", func() { - failpoint.Return(nil, errors.New("failpoint injected retriable error")) - }) - - kvStore, err := util.KVStorageFromCtx(ctx) - if err != nil { - return nil, errors.Trace(err) - } - meta, err := kv.GetSnapshotMeta(kvStore, checkpointTs) - if err != nil { - return nil, errors.Trace(err) - } - schemaSnap, err := entry.NewSingleSchemaSnapshotFromMeta(meta, checkpointTs, info.Config.ForceReplicate) - if err != nil { - return nil, errors.Trace(err) - } - - filter, err := filter.NewFilter(info.Config) - if err != nil { - return nil, errors.Trace(err) - } - - ddlHandler := newDDLHandler(o.pdClient, o.grpcPool, kvStore, checkpointTs) - defer func() { - if resultErr != nil { - ddlHandler.Close() - } - }() - - existingTables := make(map[model.TableID]model.Ts) - for captureID, taskStatus := range processorsInfos { - var checkpointTs uint64 - if pos, exist := taskPositions[captureID]; exist { - checkpointTs = pos.CheckPointTs - } - for tableID, replicaInfo := range taskStatus.Tables { - if replicaInfo.StartTs > checkpointTs { - checkpointTs = replicaInfo.StartTs - } - existingTables[tableID] = checkpointTs - } - } - - ctx, cancel := context.WithCancel(ctx) - defer func() { - if resultErr != nil { - cancel() - } - }() - schemas := make(map[model.SchemaID]tableIDMap) - tables := make(map[model.TableID]model.TableName) - partitions := make(map[model.TableID][]int64) - orphanTables := make(map[model.TableID]model.Ts) - sinkTableInfo := make([]*model.SimpleTableInfo, len(schemaSnap.CloneTables())) - j := 0 - for tid, table := range schemaSnap.CloneTables() { - j++ - if filter.ShouldIgnoreTable(table.Schema, table.Table) { - continue - } - if info.Config.Cyclic.IsEnabled() && mark.IsMarkTable(table.Schema, table.Table) { - // skip the mark table if cyclic is enabled - continue - } - - tables[tid] = table - schema, ok := schemaSnap.SchemaByTableID(tid) - if !ok { - log.Warn("schema not found for table", zap.Int64("tid", tid)) - } else { - sid := schema.ID - if _, ok := schemas[sid]; !ok { - schemas[sid] = make(tableIDMap) - } - schemas[sid][tid] = struct{}{} - } - tblInfo, ok := schemaSnap.TableByID(tid) - if !ok { - log.Warn("table not found for table ID", zap.Int64("tid", tid)) - continue - } - if !tblInfo.IsEligible(info.Config.ForceReplicate) { - log.Warn("skip ineligible table", zap.Int64("tid", tid), zap.Stringer("table", table)) - continue - } - // `existingTables` are tables dispatched to a processor, however the - // capture that this processor belongs to could have crashed or exited. - // So we check this before task dispatching, but after the update of - // changefeed schema information. - if ts, ok := existingTables[tid]; ok { - log.Info("ignore known table", zap.Int64("tid", tid), zap.Stringer("table", table), zap.Uint64("ts", ts)) - continue - } - if pi := tblInfo.GetPartitionInfo(); pi != nil { - delete(partitions, tid) - for _, partition := range pi.Definitions { - id := partition.ID - partitions[tid] = append(partitions[tid], id) - if ts, ok := existingTables[id]; ok { - log.Info("ignore known table partition", zap.Int64("tid", tid), zap.Int64("partitionID", id), zap.Stringer("table", table), zap.Uint64("ts", ts)) - continue - } - orphanTables[id] = checkpointTs - } - } else { - orphanTables[tid] = checkpointTs - } - - sinkTableInfo[j-1] = new(model.SimpleTableInfo) - sinkTableInfo[j-1].TableID = tid - sinkTableInfo[j-1].ColumnInfo = make([]*model.ColumnInfo, len(tblInfo.Cols())) - sinkTableInfo[j-1].Schema = table.Schema - sinkTableInfo[j-1].Table = table.Table - - for i, colInfo := range tblInfo.Cols() { - sinkTableInfo[j-1].ColumnInfo[i] = new(model.ColumnInfo) - sinkTableInfo[j-1].ColumnInfo[i].FromTiColumnInfo(colInfo) - } - - } - errCh := make(chan error, 1) - - primarySink, err := sink.NewSink(ctx, id, info.SinkURI, filter, info.Config, info.Opts, errCh) - if err != nil { - return nil, errors.Trace(err) - } - defer func() { - if resultErr != nil && primarySink != nil { - // The Close of backend sink here doesn't use context, it is ok to pass - // a canceled context here. - primarySink.Close(ctx) - } - }() - go func() { - var err error - select { - case <-ctx.Done(): - case err = <-errCh: - cancel() - } - if err != nil && errors.Cause(err) != context.Canceled { - log.Error("error on running changefeed", zap.Error(err), zap.String("changefeed", id)) - } else { - log.Info("changefeed exited", zap.String("changfeed", id)) - } - }() - - err = primarySink.Initialize(ctx, sinkTableInfo) - if err != nil { - log.Error("error on running owner", zap.Error(err)) - } - - var syncpointStore sink.SyncpointStore - if info.SyncPointEnabled { - syncpointStore, err = sink.NewSyncpointStore(ctx, id, info.SinkURI) - if err != nil { - return nil, errors.Trace(err) - } - } - - cf = &changeFeed{ - info: info, - id: id, - ddlHandler: ddlHandler, - schema: schemaSnap, - schemas: schemas, - tables: tables, - partitions: partitions, - orphanTables: orphanTables, - toCleanTables: make(map[model.TableID]model.Ts), - status: &model.ChangeFeedStatus{ - ResolvedTs: 0, - CheckpointTs: checkpointTs, - }, - appliedCheckpointTs: checkpointTs, - scheduler: scheduler.NewScheduler(info.Config.Scheduler.Tp), - ddlState: model.ChangeFeedSyncDML, - ddlExecutedTs: checkpointTs, - targetTs: info.GetTargetTs(), - ddlTs: 0, - updateResolvedTs: true, - startTimer: make(chan bool), - syncpointStore: syncpointStore, - syncCancel: nil, - taskStatus: processorsInfos, - taskPositions: taskPositions, - etcdCli: o.etcdClient, - leaseID: o.session.Lease(), - filter: filter, - sink: primarySink, - cyclicEnabled: info.Config.Cyclic.IsEnabled(), - lastRebalanceTime: time.Now(), - cancel: cancel, - } - return cf, nil -} - -// This is a compatibility hack between v4.0.0 and v4.0.1 -// This function will try to decode the task status, if that throw a unmarshal error, -// it will remove the invalid task status -func (o *Owner) checkAndCleanTasksInfo(ctx context.Context) error { - _, details, err := o.cfRWriter.GetChangeFeeds(ctx) - if err != nil { - return err - } - cleaned := false - for changefeedID := range details { - _, err := o.cfRWriter.GetAllTaskStatus(ctx, changefeedID) - if err != nil { - if cerror.ErrDecodeFailed.NotEqual(err) { - return errors.Trace(err) - } - err := o.cfRWriter.LeaseGuardRemoveAllTaskStatus(ctx, changefeedID, o.session.Lease()) - if err != nil { - return errors.Trace(err) - } - cleaned = true - } - } - if cleaned { - log.Warn("the task status is outdated, clean them") - } - return nil -} - -func (o *Owner) loadChangeFeeds(ctx context.Context) error { - _, details, err := o.cfRWriter.GetChangeFeeds(ctx) - if err != nil { - return err - } - errorFeeds := make(map[model.ChangeFeedID]*model.RunningError) - for changeFeedID, cfInfoRawValue := range details { - taskStatus, err := o.cfRWriter.GetAllTaskStatus(ctx, changeFeedID) - if err != nil { - return err - } - taskPositions, err := o.cfRWriter.GetAllTaskPositions(ctx, changeFeedID) - if err != nil { - return err - } - if cf, exist := o.changeFeeds[changeFeedID]; exist { - cf.updateProcessorInfos(taskStatus, taskPositions) - for _, pos := range taskPositions { - // TODO: only record error of one capture, - // is it necessary to record all captures' error - if pos.Error != nil { - errorFeeds[changeFeedID] = pos.Error - break - } - } - continue - } - - // we find a new changefeed, init changefeed here. - cfInfo := &model.ChangeFeedInfo{} - err = cfInfo.Unmarshal(cfInfoRawValue.Value) - if err != nil { - return err - } - if cfInfo.State == model.StateFailed { - if _, ok := o.failInitFeeds[changeFeedID]; ok { - continue - } - log.Warn("changefeed is not in normal state", zap.String("changefeed", changeFeedID)) - o.failInitFeeds[changeFeedID] = struct{}{} - continue - } - if _, ok := o.failInitFeeds[changeFeedID]; ok { - log.Info("changefeed recovered from failure", zap.String("changefeed", changeFeedID)) - delete(o.failInitFeeds, changeFeedID) - } - needSave, canInit := cfInfo.CheckErrorHistory() - if needSave { - err := o.etcdClient.LeaseGuardSaveChangeFeedInfo(ctx, cfInfo, changeFeedID, o.session.Lease()) - if err != nil { - return err - } - } - if !canInit { - // avoid too many logs here - if time.Now().Unix()%60 == 0 { - log.Warn("changefeed fails reach rate limit, try to initialize it later", zap.Int64s("history", cfInfo.ErrorHis)) - } - continue - } - err = cfInfo.VerifyAndFix() - if err != nil { - return err - } - - status, _, err := o.cfRWriter.GetChangeFeedStatus(ctx, changeFeedID) - if err != nil && cerror.ErrChangeFeedNotExists.NotEqual(err) { - return err - } - if status != nil && status.AdminJobType.IsStopState() { - if status.AdminJobType == model.AdminStop { - if _, ok := o.stoppedFeeds[changeFeedID]; !ok { - o.stoppedFeeds[changeFeedID] = status - } - } - continue - } - - // remaining task status means some processors are not exited, wait until - // all these statuses cleaned. If the capture of pending processor loses - // etcd session, the cleanUpStaleTasks will clean these statuses later. - allMetadataCleaned := true - allTaskStatus, err := o.etcdClient.GetAllTaskStatus(ctx, changeFeedID) - if err != nil { - return err - } - for _, taskStatus := range allTaskStatus { - if taskStatus.AdminJobType == model.AdminStop || taskStatus.AdminJobType == model.AdminRemove { - log.Info("stale task status is not deleted, wait metadata cleaned to create new changefeed", - zap.Reflect("task status", taskStatus), zap.String("changefeed", changeFeedID)) - allMetadataCleaned = false - break - } - } - if !allMetadataCleaned { - continue - } - - checkpointTs := cfInfo.GetCheckpointTs(status) - - newCf, err := o.newChangeFeed(ctx, changeFeedID, taskStatus, taskPositions, cfInfo, checkpointTs) - if err != nil { - cfInfo.Error = &model.RunningError{ - Addr: util.CaptureAddrFromCtx(ctx), - Code: "CDC-owner-1001", - Message: err.Error(), - } - cfInfo.ErrorHis = append(cfInfo.ErrorHis, time.Now().UnixNano()/1e6) - - if cerror.ChangefeedFastFailError(err) { - log.Error("create changefeed with fast fail error, mark changefeed as failed", - zap.Error(err), zap.String("changefeed", changeFeedID)) - cfInfo.State = model.StateFailed - err := o.etcdClient.LeaseGuardSaveChangeFeedInfo(ctx, cfInfo, changeFeedID, o.session.Lease()) - if err != nil { - return err - } - continue - } - - err2 := o.etcdClient.LeaseGuardSaveChangeFeedInfo(ctx, cfInfo, changeFeedID, o.session.Lease()) - if err2 != nil { - return err2 - } - // changefeed error has been recorded in etcd, log error here and - // don't need to return an error. - log.Warn("create changefeed failed, retry later", - zap.String("changefeed", changeFeedID), zap.Error(err)) - continue - } - - if newCf.info.SyncPointEnabled { - log.Info("syncpoint is on, creating the sync table") - // create the sync table - err := newCf.syncpointStore.CreateSynctable(ctx) - if err != nil { - return err - } - newCf.startSyncPointTicker(ctx, newCf.info.SyncPointInterval) - } else { - log.Info("syncpoint is off") - } - - o.changeFeeds[changeFeedID] = newCf - delete(o.stoppedFeeds, changeFeedID) - } - o.adminJobsLock.Lock() - for cfID, err := range errorFeeds { - job := model.AdminJob{ - CfID: cfID, - Type: model.AdminStop, - Error: err, - } - o.adminJobs = append(o.adminJobs, job) - } - o.adminJobsLock.Unlock() - return nil -} - -func (o *Owner) balanceTables(ctx context.Context) error { - rebalanceForAllChangefeed := false - o.rebalanceMu.Lock() - if o.rebalanceForAllChangefeed { - rebalanceForAllChangefeed = true - o.rebalanceForAllChangefeed = false - } - o.rebalanceMu.Unlock() - for id, changefeed := range o.changeFeeds { - rebalanceNow := false - var scheduleCommands []*model.MoveTableJob - o.rebalanceMu.Lock() - if r, exist := o.rebalanceTigger[id]; exist { - rebalanceNow = r - delete(o.rebalanceTigger, id) - } - if rebalanceForAllChangefeed { - rebalanceNow = true - } - if c, exist := o.manualScheduleCommand[id]; exist { - scheduleCommands = c - delete(o.manualScheduleCommand, id) - } - o.rebalanceMu.Unlock() - err := changefeed.tryBalance(ctx, o.captures, rebalanceNow, scheduleCommands) - if err != nil { - return errors.Trace(err) - } - } - return nil -} - -func (o *Owner) flushChangeFeedInfos(ctx context.Context) error { - // no running or stopped changefeed, clear gc safepoint. - if len(o.changeFeeds) == 0 && len(o.stoppedFeeds) == 0 { - if !o.gcSafepointLastUpdate.IsZero() { - log.Info("clean service safe point", zap.String("service-id", CDCServiceSafePointID)) - _, err := o.pdClient.UpdateServiceGCSafePoint(ctx, CDCServiceSafePointID, 0, 0) - if err != nil { - log.Warn("failed to update service safe point", zap.Error(err)) - } else { - o.gcSafepointLastUpdate = time.Time{} - } - } - return nil - } - - staleChangeFeeds := make(map[model.ChangeFeedID]*model.ChangeFeedStatus, len(o.changeFeeds)) - gcSafePoint := uint64(math.MaxUint64) - - // get the lower bound of gcSafePoint - minGCSafePoint := o.getMinGCSafePointCache(ctx) - - if len(o.changeFeeds) > 0 { - snapshot := make(map[model.ChangeFeedID]*model.ChangeFeedStatus, len(o.changeFeeds)) - for id, changefeed := range o.changeFeeds { - snapshot[id] = changefeed.status - if changefeed.status.CheckpointTs < gcSafePoint { - gcSafePoint = changefeed.status.CheckpointTs - } - // 1. If changefeed's appliedCheckpoinTs <= minGCSafePoint, it means this changefeed is stagnant. - // They are collected into this map, and then handleStaleChangeFeed() is called to deal with these stagnant changefeed. - // A changefeed will not enter the map twice, because in run(), - // handleAdminJob() will always be executed before flushChangeFeedInfos(), - // ensuring that the previous changefeed in staleChangeFeeds has been stopped and removed from o.changeFeeds. - // 2. We need the `<=` check here is because when a changefeed is stagnant, its checkpointTs will be updated to pd, - // and it would be the minimum gcSafePoint across all services. - // So as described above(line 92) minGCSafePoint = gcSafePoint = CheckpointTs would happens. - // In this case, if we check `<` here , this changefeed will not be put into staleChangeFeeds, and its checkpoints will be updated to pd again and again. - // This will cause the cdc's gcSafePoint never advance. - // If we check `<=` here, when we encounter the changefeed again, we will put it into staleChangeFeeds. - if changefeed.status.CheckpointTs <= minGCSafePoint { - staleChangeFeeds[id] = changefeed.status - } - - phyTs := oracle.ExtractPhysical(changefeed.status.CheckpointTs) - changefeedCheckpointTsGauge.WithLabelValues(id).Set(float64(phyTs)) - // It is more accurate to get tso from PD, but in most cases we have - // deployed NTP service, a little bias is acceptable here. - changefeedCheckpointTsLagGauge.WithLabelValues(id).Set(float64(oracle.GetPhysical(time.Now())-phyTs) / 1e3) - } - if time.Since(o.lastFlushChangefeeds) > o.flushChangefeedInterval { - err := o.cfRWriter.LeaseGuardPutAllChangeFeedStatus(ctx, snapshot, o.session.Lease()) - if err != nil { - return errors.Trace(err) - } - for id, changefeedStatus := range snapshot { - o.changeFeeds[id].appliedCheckpointTs = changefeedStatus.CheckpointTs - } - o.lastFlushChangefeeds = time.Now() - } - } - - for _, status := range o.stoppedFeeds { - // If a stopped changefeed's CheckpoinTs <= minGCSafePoint, means this changefeed is stagnant. - // It should never be resumed. This part of the logic is in newChangeFeed() - // So here we can skip it. - if status.CheckpointTs <= minGCSafePoint { - continue - } - - if status.CheckpointTs < gcSafePoint { - gcSafePoint = status.CheckpointTs - } - } - - // handle stagnant changefeed collected above - err := o.handleStaleChangeFeed(ctx, staleChangeFeeds, minGCSafePoint) - if err != nil { - log.Warn("failed to handleStaleChangeFeed ", zap.Error(err)) - } - - if time.Since(o.gcSafepointLastUpdate) > GCSafepointUpdateInterval { - actual, err := o.pdClient.UpdateServiceGCSafePoint(ctx, CDCServiceSafePointID, o.gcTTL, gcSafePoint) - if err != nil { - sinceLastUpdate := time.Since(o.gcSafepointLastUpdate) - log.Warn("failed to update service safe point", zap.Error(err), - zap.Duration("since-last-update", sinceLastUpdate)) - // We do not throw an error unless updating GC safepoint has been failing for more than gcTTL. - if sinceLastUpdate >= time.Second*time.Duration(o.gcTTL) { - return cerror.ErrUpdateServiceSafepointFailed.Wrap(err) - } - } else { - o.pdGCSafePoint = actual - o.gcSafepointLastUpdate = time.Now() - } - - failpoint.Inject("InjectActualGCSafePoint", func(val failpoint.Value) { - actual = uint64(val.(int)) - }) - - if actual > gcSafePoint { - // UpdateServiceGCSafePoint has failed. - log.Warn("updating an outdated service safe point", zap.Uint64("checkpoint-ts", gcSafePoint), zap.Uint64("actual-safepoint", actual)) - - for cfID, cf := range o.changeFeeds { - if cf.status.CheckpointTs < actual { - runningError := &model.RunningError{ - Addr: util.CaptureAddrFromCtx(ctx), - Code: "CDC-owner-1001", - Message: cerror.ErrServiceSafepointLost.GenWithStackByArgs(actual).Error(), - } - - err := o.EnqueueJob(model.AdminJob{ - CfID: cfID, - Type: model.AdminStop, - Error: runningError, - }) - if err != nil { - return errors.Trace(err) - } - } - } - } - } - return nil -} - -// calcResolvedTs call calcResolvedTs of every changefeeds -func (o *Owner) calcResolvedTs(ctx context.Context) error { - for id, cf := range o.changeFeeds { - if err := cf.calcResolvedTs(ctx); err != nil { - log.Error("fail to calculate checkpoint ts, so it will be stopped", zap.String("changefeed", cf.id), zap.Error(err)) - // error may cause by sink.EmitCheckpointTs`, just stop the changefeed at the moment - // todo: make the method mentioned above more robust. - var code string - if rfcCode, ok := cerror.RFCCode(err); ok { - code = string(rfcCode) - } else { - code = string(cerror.ErrOwnerUnknown.RFCCode()) - } - - job := model.AdminJob{ - CfID: id, - Type: model.AdminStop, - Error: &model.RunningError{ - Addr: util.CaptureAddrFromCtx(ctx), - Code: code, - Message: err.Error(), - }, - } - - if err := o.EnqueueJob(job); err != nil { - return errors.Trace(err) - } - } - } - return nil -} - -// handleDDL call handleDDL of every changefeeds -func (o *Owner) handleDDL(ctx context.Context) error { - for _, cf := range o.changeFeeds { - err := cf.handleDDL(ctx) - if err != nil { - var code string - if terror, ok := err.(*errors.Error); ok { - code = string(terror.RFCCode()) - } else { - code = string(cerror.ErrExecDDLFailed.RFCCode()) - } - err = o.EnqueueJob(model.AdminJob{ - CfID: cf.id, - Type: model.AdminStop, - Error: &model.RunningError{ - Addr: util.CaptureAddrFromCtx(ctx), - Code: code, - Message: err.Error(), - }, - }) - if err != nil { - return errors.Trace(err) - } - } - } - return nil -} - -// handleSyncPoint call handleSyncPoint of every changefeeds -func (o *Owner) handleSyncPoint(ctx context.Context) error { - for _, cf := range o.changeFeeds { - if err := cf.handleSyncPoint(ctx); err != nil { - return errors.Trace(err) - } - } - return nil -} - -// dispatchJob dispatches job to processors -// Note job type in this function contains pause, remove and finish -func (o *Owner) dispatchJob(ctx context.Context, job model.AdminJob) error { - cf, ok := o.changeFeeds[job.CfID] - if !ok { - return cerror.ErrOwnerChangefeedNotFound.GenWithStackByArgs(job.CfID) - } - for captureID := range cf.taskStatus { - newStatus, _, err := cf.etcdCli.LeaseGuardAtomicPutTaskStatus( - ctx, cf.id, captureID, o.session.Lease(), - func(modRevision int64, taskStatus *model.TaskStatus) (bool, error) { - taskStatus.AdminJobType = job.Type - return true, nil - }, - ) - if err != nil { - return errors.Trace(err) - } - cf.taskStatus[captureID] = newStatus.Clone() - } - // record admin job in changefeed status - cf.status.AdminJobType = job.Type - infos := map[model.ChangeFeedID]*model.ChangeFeedStatus{job.CfID: cf.status} - err := o.cfRWriter.LeaseGuardPutAllChangeFeedStatus(ctx, infos, o.session.Lease()) - if err != nil { - return errors.Trace(err) - } - cf.Close() - // Only need to process stoppedFeeds with `AdminStop` command here. - // For `AdminResume`, we remove stopped feed in changefeed initialization phase. - // For `AdminRemove`, we need to update stoppedFeeds when removing a stopped changefeed. - if job.Type == model.AdminStop { - log.Debug("put changefeed into stoppedFeeds queue", zap.String("changefeed", job.CfID)) - o.stoppedFeeds[job.CfID] = cf.status - } - for captureID := range cf.taskStatus { - capture, ok := o.captures[captureID] - if !ok { - log.Warn("capture not found", zap.String("capture-id", captureID)) - continue - } - ownerMaintainTableNumGauge.DeleteLabelValues(cf.id, capture.AdvertiseAddr, maintainTableTypeTotal) - ownerMaintainTableNumGauge.DeleteLabelValues(cf.id, capture.AdvertiseAddr, maintainTableTypeWip) - } - delete(o.changeFeeds, job.CfID) - return nil -} - -func (o *Owner) collectChangefeedInfo(ctx context.Context, cid model.ChangeFeedID) ( - cf *changeFeed, - status *model.ChangeFeedStatus, - feedState model.FeedState, - err error, -) { - var ok bool - cf, ok = o.changeFeeds[cid] - if ok { - return cf, cf.status, cf.info.State, nil - } - feedState = model.StateNormal - - var cfInfo *model.ChangeFeedInfo - cfInfo, err = o.etcdClient.GetChangeFeedInfo(ctx, cid) - if err != nil && cerror.ErrChangeFeedNotExists.NotEqual(err) { - return - } - - status, _, err = o.etcdClient.GetChangeFeedStatus(ctx, cid) - if err != nil { - if cerror.ErrChangeFeedNotExists.Equal(err) { - // Only changefeed info exists and error field is not nil means - // the changefeed has met error, mark it as failed. - if cfInfo != nil && cfInfo.Error != nil { - feedState = model.StateFailed - } - } - return - } - switch status.AdminJobType { - case model.AdminNone, model.AdminResume: - if cfInfo != nil && cfInfo.Error != nil { - feedState = model.StateFailed - } - case model.AdminStop: - feedState = model.StateStopped - case model.AdminRemove: - feedState = model.StateRemoved - case model.AdminFinish: - feedState = model.StateFinished - } - return -} - -func (o *Owner) checkClusterHealth(_ context.Context) error { - // check whether a changefeed has finished by comparing checkpoint-ts and target-ts - for _, cf := range o.changeFeeds { - if cf.status.CheckpointTs == cf.info.GetTargetTs() { - log.Info("changefeed replication finished", zap.String("changefeed", cf.id), zap.Uint64("checkpointTs", cf.status.CheckpointTs)) - err := o.EnqueueJob(model.AdminJob{ - CfID: cf.id, - Type: model.AdminFinish, - }) - if err != nil { - return err - } - } - } - for _, cf := range o.changeFeeds { - for captureID, pinfo := range cf.taskStatus { - capture, ok := o.captures[captureID] - if !ok { - log.Warn("capture not found", zap.String("capture-id", captureID)) - continue - } - ownerMaintainTableNumGauge.WithLabelValues(cf.id, capture.AdvertiseAddr, maintainTableTypeTotal).Set(float64(len(pinfo.Tables))) - ownerMaintainTableNumGauge.WithLabelValues(cf.id, capture.AdvertiseAddr, maintainTableTypeWip).Set(float64(len(pinfo.Operation))) - } - } - // TODO: check processor normal exited - return nil -} - -func (o *Owner) handleAdminJob(ctx context.Context) error { - removeIdx := 0 - o.adminJobsLock.Lock() - defer func() { - o.adminJobs = o.adminJobs[removeIdx:] - o.adminJobsLock.Unlock() - }() - for i, job := range o.adminJobs { - log.Info("handle admin job", zap.String("changefeed", job.CfID), zap.Stringer("type", job.Type)) - removeIdx = i + 1 - - cf, status, feedState, err := o.collectChangefeedInfo(ctx, job.CfID) - if err != nil { - if cerror.ErrChangeFeedNotExists.NotEqual(err) { - return err - } - if feedState == model.StateFailed && job.Type == model.AdminRemove { - // changefeed in failed state, but changefeed status has not - // been created yet. Try to remove changefeed info only. - err := o.etcdClient.LeaseGuardDeleteChangeFeedInfo(ctx, job.CfID, o.session.Lease()) - if err != nil { - return errors.Trace(err) - } - } else { - log.Warn("invalid admin job, changefeed status not found", zap.String("changefeed", job.CfID)) - } - continue - } - switch job.Type { - case model.AdminStop: - switch feedState { - case model.StateStopped: - log.Info("changefeed has been stopped, pause command will do nothing") - continue - case model.StateRemoved: - log.Info("changefeed has been removed, pause command will do nothing") - continue - case model.StateFinished: - log.Info("changefeed has finished, pause command will do nothing") - continue - } - if cf == nil { - log.Warn("invalid admin job, changefeed not found", zap.String("changefeed", job.CfID)) - continue - } - - cf.info.AdminJobType = model.AdminStop - cf.info.Error = job.Error - if job.Error != nil { - cf.info.ErrorHis = append(cf.info.ErrorHis, time.Now().UnixNano()/1e6) - } - - err := o.etcdClient.LeaseGuardSaveChangeFeedInfo(ctx, cf.info, job.CfID, o.session.Lease()) - if err != nil { - return errors.Trace(err) - } - err = o.dispatchJob(ctx, job) - if err != nil { - return errors.Trace(err) - } - cf.stopSyncPointTicker() - case model.AdminRemove, model.AdminFinish: - if cf != nil { - cf.stopSyncPointTicker() - err := o.dispatchJob(ctx, job) - if err != nil { - return errors.Trace(err) - } - } else { - switch feedState { - case model.StateRemoved, model.StateFinished: - // remove a removed or finished changefeed - if job.Opts != nil && job.Opts.ForceRemove { - err := o.etcdClient.LeaseGuardRemoveChangeFeedStatus(ctx, job.CfID, o.session.Lease()) - if err != nil { - return errors.Trace(err) - } - } else { - log.Info("changefeed has been removed or finished, remove command will do nothing") - } - continue - case model.StateStopped, model.StateFailed: - // remove a paused or failed changefeed - status.AdminJobType = model.AdminRemove - err = o.etcdClient.LeaseGuardPutChangeFeedStatus(ctx, job.CfID, status, o.session.Lease()) - if err != nil { - return errors.Trace(err) - } - delete(o.stoppedFeeds, job.CfID) - default: - return cerror.ErrChangefeedAbnormalState.GenWithStackByArgs(feedState, status) - } - } - // remove changefeed info - err := o.etcdClient.DeleteChangeFeedInfo(ctx, job.CfID) - if err != nil { - return errors.Trace(err) - } - if job.Opts != nil && job.Opts.ForceRemove { - // if `ForceRemove` is enabled, remove all information related to this changefeed - err := o.etcdClient.LeaseGuardRemoveChangeFeedStatus(ctx, job.CfID, o.session.Lease()) - if err != nil { - return errors.Trace(err) - } - } else { - // set ttl to changefeed status - err = o.etcdClient.SetChangeFeedStatusTTL(ctx, job.CfID, 24*3600 /*24 hours*/) - if err != nil { - return errors.Trace(err) - } - } - case model.AdminResume: - // resume changefeed must read checkpoint from ChangeFeedStatus - if cerror.ErrChangeFeedNotExists.Equal(err) { - log.Warn("invalid admin job, changefeed not found", zap.String("changefeed", job.CfID)) - continue - } - if feedState == model.StateRemoved || feedState == model.StateFinished { - log.Info("changefeed has been removed or finished, cannot be resumed anymore") - continue - } - cfInfo, err := o.etcdClient.GetChangeFeedInfo(ctx, job.CfID) - if err != nil { - return errors.Trace(err) - } - - // set admin job in changefeed status to tell owner resume changefeed - status.AdminJobType = model.AdminResume - err = o.etcdClient.LeaseGuardPutChangeFeedStatus(ctx, job.CfID, status, o.session.Lease()) - if err != nil { - return errors.Trace(err) - } - - // set admin job in changefeed cfInfo to trigger each capture's changefeed list watch event - cfInfo.AdminJobType = model.AdminResume - // clear last running error - cfInfo.State = model.StateNormal - cfInfo.Error = nil - err = o.etcdClient.LeaseGuardSaveChangeFeedInfo(ctx, cfInfo, job.CfID, o.session.Lease()) - if err != nil { - return errors.Trace(err) - } - if config.NewReplicaImpl { - // remove all positions because the old positions may be include an error - err = o.etcdClient.RemoveAllTaskPositions(ctx, job.CfID) - if err != nil { - return errors.Trace(err) - } - } - } - // TODO: we need a better admin job workflow. Supposing uses create - // multiple admin jobs to a specific changefeed at the same time, such - // as pause -> resume -> pause, should the one job handler waits for - // the previous job finished? However it is difficult to distinguish - // whether a job is totally finished in some cases, for example when - // resuming a changefeed, seems we should mark the job finished if all - // processors have started. Currently the owner only processes one - // admin job in each tick loop as a workaround. - break - } - return nil -} - -func (o *Owner) throne(ctx context.Context) error { - // Start a routine to keep watching on the liveness of - // captures. - o.startCaptureWatcher(ctx) - return nil -} - -// Close stops a running owner -func (o *Owner) Close(ctx context.Context, stepDown func(ctx context.Context) error) { - // stepDown is called after exiting the main loop by the owner, it is useful - // to clean up some resource, like dropping the leader key. - o.stepDown = stepDown - - // Close and Run should be in separated goroutines - // A channel is used here to synchronize the steps. - - // Single the Run function to exit - select { - case o.done <- struct{}{}: - case <-ctx.Done(): - } - - // Wait until it exited - select { - case <-o.done: - case <-ctx.Done(): - } -} - -// Run the owner -// TODO avoid this tick style, this means we get `tickTime` latency here. -func (o *Owner) Run(ctx context.Context, tickTime time.Duration) error { - failpoint.Inject("owner-run-with-error", func() { - failpoint.Return(errors.New("owner run with injected error")) - }) - - ctx, cancel := context.WithCancel(ctx) - defer cancel() - - go func() { - if err := o.watchCampaignKey(ctx); err != nil { - cancel() - } - }() - - if err := o.throne(ctx); err != nil { - return err - } - - ctx1, cancel1 := context.WithCancel(ctx) - defer cancel1() - feedChangeReceiver, err := o.feedChangeNotifier.NewReceiver(tickTime) - if err != nil { - return err - } - defer feedChangeReceiver.Stop() - o.watchFeedChange(ctx1) - - ownership := newOwnership(tickTime) -loop: - for { - select { - case <-o.done: - close(o.done) - break loop - case <-ctx.Done(): - // FIXME: cancel the context doesn't ensure all resources are destructed, is it reasonable? - // Anyway we just break loop here to ensure the following destruction. - err = ctx.Err() - break loop - case <-feedChangeReceiver.C: - ownership.inc() - } - - err = o.run(ctx) - if err != nil { - switch errors.Cause(err) { - case context.DeadlineExceeded: - // context timeout means the o.run doesn't finish in a safe owner - // lease cycle, it is safe to retry. If the lease is revoked, - // another run loop will detect it. - continue loop - case context.Canceled: - default: - log.Error("owner exited with error", zap.Error(err)) - } - break loop - } - } - for _, cf := range o.changeFeeds { - cf.Close() - changefeedCheckpointTsGauge.DeleteLabelValues(cf.id) - changefeedCheckpointTsLagGauge.DeleteLabelValues(cf.id) - } - if o.stepDown != nil { - if err := o.stepDown(ctx); err != nil { - return err - } - } - - return err -} - -// watchCampaignKey watches the aliveness of campaign owner key in etcd -func (o *Owner) watchCampaignKey(ctx context.Context) error { - key := fmt.Sprintf("%s/%x", kv.CaptureOwnerKey, o.session.Lease()) -restart: - resp, err := o.etcdClient.Client.Get(ctx, key) - if err != nil { - return cerror.WrapError(cerror.ErrPDEtcdAPIError, err) - } - if resp.Count == 0 { - return cerror.ErrOwnerCampaignKeyDeleted.GenWithStackByArgs() - } - // watch the key change from the next revision relatived to the current - wch := o.etcdClient.Client.Watch(ctx, key, clientv3.WithRev(resp.Header.Revision+1)) - for resp := range wch { - err := resp.Err() - if err != nil { - if err != mvcc.ErrCompacted { - log.Error("watch owner campaign key failed, restart the watcher", zap.Error(err)) - } - goto restart - } - for _, ev := range resp.Events { - if ev.Type == clientv3.EventTypeDelete { - log.Warn("owner campaign key deleted", zap.String("key", key)) - return cerror.ErrOwnerCampaignKeyDeleted.GenWithStackByArgs() - } - } - } - return nil -} - -func (o *Owner) watchFeedChange(ctx context.Context) { - go func() { - for { - select { - case <-ctx.Done(): - return - default: - } - cctx, cancel := context.WithCancel(ctx) - wch := o.etcdClient.Client.Watch(cctx, kv.TaskPositionKeyPrefix, clientv3.WithFilterDelete(), clientv3.WithPrefix()) - - for resp := range wch { - if resp.Err() != nil { - log.Error("position watcher restarted with error", zap.Error(resp.Err())) - break - } - - // TODO: because the main loop has many serial steps, it is hard to do a partial update without change - // majority logical. For now just to wakeup the main loop ASAP to reduce latency, the efficiency of etcd - // operations should be resolved in future release. - - o.feedChangeNotifier.Notify() - } - cancel() - } - }() -} - -func (o *Owner) run(ctx context.Context) error { - // captureLoaded == 0 means capture information is not built, owner can't - // run normal jobs now. - if atomic.LoadInt32(&o.captureLoaded) == int32(0) { - return nil - } - - o.l.Lock() - defer o.l.Unlock() - - var err error - - err = o.cleanUpStaleTasks(ctx) - if err != nil { - return errors.Trace(err) - } - - err = o.loadChangeFeeds(ctx) - if err != nil { - return errors.Trace(err) - } - - err = o.balanceTables(ctx) - if err != nil { - return errors.Trace(err) - } - - err = o.handleDDL(ctx) - if err != nil { - return errors.Trace(err) - } - - err = o.handleSyncPoint(ctx) - if err != nil { - return errors.Trace(err) - } - - err = o.handleAdminJob(ctx) - if err != nil { - return errors.Trace(err) - } - - err = o.calcResolvedTs(ctx) - if err != nil { - return errors.Trace(err) - } - - // It is better for flushChangeFeedInfos to follow calcResolvedTs immediately, - // because operations such as handleDDL and rebalancing rely on proper progress of the checkpoint in Etcd. - err = o.flushChangeFeedInfos(ctx) - if err != nil { - return errors.Trace(err) - } - - err = o.checkClusterHealth(ctx) - if err != nil { - return errors.Trace(err) - } - - return nil -} - -// EnqueueJob adds an admin job -func (o *Owner) EnqueueJob(job model.AdminJob) error { - switch job.Type { - case model.AdminResume, model.AdminRemove, model.AdminStop, model.AdminFinish: - default: - return cerror.ErrInvalidAdminJobType.GenWithStackByArgs(job.Type) - } - o.adminJobsLock.Lock() - o.adminJobs = append(o.adminJobs, job) - o.adminJobsLock.Unlock() - return nil -} - -// TriggerRebalance triggers the rebalance in the specified changefeed -func (o *Owner) TriggerRebalance(changefeedID model.ChangeFeedID) { - o.rebalanceMu.Lock() - defer o.rebalanceMu.Unlock() - o.rebalanceTigger[changefeedID] = true - // TODO(leoppro) throw an error if the changefeed is not exist -} - -// ManualSchedule moves the table from a capture to another capture -func (o *Owner) ManualSchedule(changefeedID model.ChangeFeedID, to model.CaptureID, tableID model.TableID) { - o.rebalanceMu.Lock() - defer o.rebalanceMu.Unlock() - o.manualScheduleCommand[changefeedID] = append(o.manualScheduleCommand[changefeedID], &model.MoveTableJob{ - To: to, - TableID: tableID, - }) -} - -func (o *Owner) writeDebugInfo(w io.Writer) { - fmt.Fprintf(w, "** active changefeeds **:\n") - for _, info := range o.changeFeeds { - fmt.Fprintf(w, "%s\n", info) - } - fmt.Fprintf(w, "** stopped changefeeds **:\n") - for _, feedStatus := range o.stoppedFeeds { - fmt.Fprintf(w, "%+v\n", *feedStatus) - } - fmt.Fprintf(w, "\n** captures **:\n") - for _, capture := range o.captures { - fmt.Fprintf(w, "%+v\n", *capture) - } -} - -// cleanUpStaleTasks cleans up the task status which does not associated -// with an active processor. This function is not thread safe. -// -// When a new owner is elected, it does not know the events occurs before, like -// processor deletion. In this case, the new owner should check if the task -// status is stale because of the processor deletion. -func (o *Owner) cleanUpStaleTasks(ctx context.Context) error { - _, changefeeds, err := o.etcdClient.GetChangeFeeds(ctx) - if err != nil { - return errors.Trace(err) - } - for changeFeedID := range changefeeds { - statuses, err := o.etcdClient.GetAllTaskStatus(ctx, changeFeedID) - if err != nil { - return errors.Trace(err) - } - positions, err := o.etcdClient.GetAllTaskPositions(ctx, changeFeedID) - if err != nil { - return errors.Trace(err) - } - workloads, err := o.etcdClient.GetAllTaskWorkloads(ctx, changeFeedID) - if err != nil { - return errors.Trace(err) - } - // in most cases statuses and positions have the same keys, or positions - // are more than statuses, as we always delete task status first. - captureIDs := make(map[string]struct{}, len(statuses)) - for captureID := range statuses { - captureIDs[captureID] = struct{}{} - } - for captureID := range positions { - captureIDs[captureID] = struct{}{} - } - for captureID := range workloads { - captureIDs[captureID] = struct{}{} - } - - log.Debug("cleanUpStaleTasks", - zap.Reflect("statuses", statuses), - zap.Reflect("positions", positions), - zap.Reflect("workloads", workloads)) - - for captureID := range captureIDs { - if _, ok := o.captures[captureID]; !ok { - status, ok1 := statuses[captureID] - if ok1 { - pos, taskPosFound := positions[captureID] - if !taskPosFound { - log.Warn("task position not found, fallback to use original start ts", - zap.String("capture", captureID), - zap.String("changefeed", changeFeedID), - zap.Reflect("task status", status), - ) - } - for tableID, replicaInfo := range status.Tables { - startTs := replicaInfo.StartTs - if taskPosFound { - if startTs < pos.CheckPointTs { - startTs = pos.CheckPointTs - } - } - o.addOrphanTable(changeFeedID, tableID, startTs) - } - if cf, ok := o.changeFeeds[changeFeedID]; ok { - o.rebuildTableFromOperations(cf, status, cf.status.CheckpointTs) - } - } - - if err := o.etcdClient.LeaseGuardDeleteTaskStatus(ctx, changeFeedID, captureID, o.session.Lease()); err != nil { - return errors.Trace(err) - } - if err := o.etcdClient.LeaseGuardDeleteTaskPosition(ctx, changeFeedID, captureID, o.session.Lease()); err != nil { - return errors.Trace(err) - } - if err := o.etcdClient.LeaseGuardDeleteTaskWorkload(ctx, changeFeedID, captureID, o.session.Lease()); err != nil { - return errors.Trace(err) - } - log.Info("cleanup stale task", zap.String("capture-id", captureID), zap.String("changefeed", changeFeedID)) - } - } - } - return nil -} - -func (o *Owner) watchCapture(ctx context.Context) error { - ctx = clientv3.WithRequireLeader(ctx) - - failpoint.Inject("sleep-before-watch-capture", nil) - - // When an owner just starts, changefeed information is not updated at once. - // Supposing a crashed capture should be removed now, the owner will miss deleting - // task status and task position if changefeed information is not loaded. - // If the task positions and status decode failed, remove them. - if err := o.checkAndCleanTasksInfo(ctx); err != nil { - return errors.Trace(err) - } - o.l.Lock() - if err := o.loadChangeFeeds(ctx); err != nil { - o.l.Unlock() - return errors.Trace(err) - } - o.l.Unlock() - - rev, captureList, err := o.etcdClient.GetCaptures(ctx) - if err != nil { - return errors.Trace(err) - } - captures := make(map[model.CaptureID]*model.CaptureInfo) - for _, c := range captureList { - captures[c.ID] = c - } - // before watching, rebuild events according to - // the existed captures. This is necessary because - // the etcd events may be compacted. - if err := o.rebuildCaptureEvents(ctx, captures); err != nil { - return errors.Trace(err) - } - - log.Info("monitoring captures", - zap.String("key", kv.CaptureInfoKeyPrefix), - zap.Int64("rev", rev)) - ch := o.etcdClient.Client.Watch(ctx, kv.CaptureInfoKeyPrefix, - clientv3.WithPrefix(), - clientv3.WithRev(rev+1), - clientv3.WithPrevKV()) - - for resp := range ch { - err := resp.Err() - failpoint.Inject("restart-capture-watch", func() { - err = mvcc.ErrCompacted - }) - if err != nil { - return cerror.WrapError(cerror.ErrOwnerEtcdWatch, resp.Err()) - } - for _, ev := range resp.Events { - c := &model.CaptureInfo{} - switch ev.Type { - case clientv3.EventTypeDelete: - if err := c.Unmarshal(ev.PrevKv.Value); err != nil { - return errors.Trace(err) - } - log.Info("delete capture", - zap.String("capture-id", c.ID), - zap.String("capture", c.AdvertiseAddr)) - o.removeCapture(ctx, c) - case clientv3.EventTypePut: - if !ev.IsCreate() { - continue - } - if err := c.Unmarshal(ev.Kv.Value); err != nil { - return errors.Trace(err) - } - log.Info("add capture", - zap.String("capture-id", c.ID), - zap.String("capture", c.AdvertiseAddr)) - o.addCapture(ctx, c) - } - } - } - return nil -} - -func (o *Owner) rebuildCaptureEvents(ctx context.Context, captures map[model.CaptureID]*model.CaptureInfo) error { - for _, c := range captures { - o.addCapture(ctx, c) - } - for _, c := range o.captures { - if _, ok := captures[c.ID]; !ok { - o.removeCapture(ctx, c) - } - } - // captureLoaded is used to check whether the owner can execute cleanup stale tasks job. - // Because at the very beginning of a new owner, it doesn't have capture information in - // memory, cleanup stale tasks could have a false positive (where positive means owner - // should cleanup the stale task of a specific capture). After the first time of capture - // rebuild, even the etcd compaction and watch capture is rerun, we don't need to check - // captureLoaded anymore because existing tasks must belong to a capture which is still - // maintained in owner's memory. - atomic.StoreInt32(&o.captureLoaded, 1) - - // clean up stale tasks each time before watch capture event starts, - // for two reasons: - // 1. when a new owner is elected, it must clean up stale task status and positions. - // 2. when error happens in owner's capture event watch, the owner just resets - // the watch loop, with the following two steps: - // 1) load all captures from PD, having a revision for data - // 2) start a new watch from revision in step1 - // the step-2 may meet an error such as ErrCompacted, and we will continue - // from step-1, however other capture may crash just after step-2 returns - // and before step-1 starts, the longer time gap between step-2 to step-1, - // missing a crashed capture is more likely to happen. - o.l.Lock() - defer o.l.Unlock() - return errors.Trace(o.cleanUpStaleTasks(ctx)) -} - -func (o *Owner) startCaptureWatcher(ctx context.Context) { - log.Info("start to watch captures") - go func() { - rl := rate.NewLimiter(0.05, 2) - for { - err := rl.Wait(ctx) - if err != nil { - if errors.Cause(err) == context.Canceled { - return - } - log.Error("capture watcher wait limit token error", zap.Error(err)) - return - } - if err := o.watchCapture(ctx); err != nil { - // When the watching routine returns, the error must not - // be nil, it may be caused by a temporary error or a context - // error(ctx.Err()) - if ctx.Err() != nil { - if errors.Cause(ctx.Err()) != context.Canceled { - // The context error indicates the termination of the owner - log.Error("watch capture failed", zap.Error(ctx.Err())) - } else { - log.Info("watch capture exited") - } - return - } - log.Warn("watch capture returned", zap.Error(err)) - // Otherwise, a temporary error occurred(ErrCompact), - // restart the watching routine. - } - } - }() -} - -// handle the StaleChangeFeed -// By setting the AdminJob type to AdminStop and the Error code to indicate that the changefeed is stagnant. -func (o *Owner) handleStaleChangeFeed(ctx context.Context, staleChangeFeeds map[model.ChangeFeedID]*model.ChangeFeedStatus, minGCSafePoint uint64) error { - for id, status := range staleChangeFeeds { - err := cerror.ErrSnapshotLostByGC.GenWithStackByArgs(status.CheckpointTs, minGCSafePoint) - log.Warn("changefeed checkpoint is lagging too much, so it will be stopped.", zap.String("changefeed", id), zap.Error(err)) - runningError := &model.RunningError{ - Addr: util.CaptureAddrFromCtx(ctx), - Code: string(cerror.ErrSnapshotLostByGC.RFCCode()), // changefeed is stagnant - Message: err.Error(), - } - - err = o.EnqueueJob(model.AdminJob{ - CfID: id, - Type: model.AdminStop, - Error: runningError, - }) - if err != nil { - return errors.Trace(err) - } - delete(staleChangeFeeds, id) - } - return nil -} diff --git a/cdc/owner_operator.go b/cdc/owner_operator.go deleted file mode 100644 index 42bb1879faa..00000000000 --- a/cdc/owner_operator.go +++ /dev/null @@ -1,120 +0,0 @@ -// Copyright 2020 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 cdc - -import ( - "context" - "sync" - - tidbkv "github.com/pingcap/tidb/kv" - - "github.com/pingcap/errors" - timodel "github.com/pingcap/parser/model" - "github.com/pingcap/tiflow/cdc/entry" - "github.com/pingcap/tiflow/cdc/kv" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/puller" - "github.com/pingcap/tiflow/pkg/regionspan" - "github.com/pingcap/tiflow/pkg/util" - pd "github.com/tikv/pd/client" - "golang.org/x/sync/errgroup" -) - -// TODO: add tests -type ddlHandler struct { - puller puller.Puller - resolvedTS uint64 - ddlJobs []*timodel.Job - - mu sync.Mutex - wg *errgroup.Group - cancel func() -} - -func newDDLHandler(pdCli pd.Client, grpcPool kv.GrpcPool, kvStorage tidbkv.Storage, checkpointTS uint64) *ddlHandler { - // TODO: context should be passed from outter caller - ctx, cancel := context.WithCancel(context.Background()) - plr := puller.NewPuller(ctx, pdCli, grpcPool, kvStorage, checkpointTS, []regionspan.Span{regionspan.GetDDLSpan(), regionspan.GetAddIndexDDLSpan()}, false) - h := &ddlHandler{ - puller: plr, - cancel: cancel, - } - // Set it up so that one failed goroutine cancels all others sharing the same ctx - errg, ctx := errgroup.WithContext(ctx) - ctx = util.PutTableInfoInCtx(ctx, -1, "") - - // FIXME: user of ddlHandler can't know error happen. - errg.Go(func() error { - return plr.Run(ctx) - }) - - rawDDLCh := puller.SortOutput(ctx, plr.Output()) - - errg.Go(func() error { - for { - select { - case <-ctx.Done(): - return ctx.Err() - case e := <-rawDDLCh: - if e == nil { - continue - } - err := h.receiveDDL(e) - if err != nil { - return errors.Trace(err) - } - } - } - }) - h.wg = errg - return h -} - -func (h *ddlHandler) receiveDDL(rawDDL *model.RawKVEntry) error { - if rawDDL.OpType == model.OpTypeResolved { - h.mu.Lock() - h.resolvedTS = rawDDL.CRTs - h.mu.Unlock() - return nil - } - job, err := entry.UnmarshalDDL(rawDDL) - if err != nil { - return errors.Trace(err) - } - if job == nil { - return nil - } - - h.mu.Lock() - defer h.mu.Unlock() - h.ddlJobs = append(h.ddlJobs, job) - return nil -} - -var _ OwnerDDLHandler = &ddlHandler{} - -// PullDDL implements `roles.OwnerDDLHandler` interface. -func (h *ddlHandler) PullDDL() (uint64, []*timodel.Job, error) { - h.mu.Lock() - defer h.mu.Unlock() - result := h.ddlJobs - h.ddlJobs = nil - return h.resolvedTS, result, nil -} - -func (h *ddlHandler) Close() error { - h.cancel() - err := h.wg.Wait() - return errors.Trace(err) -} diff --git a/cdc/owner_test.go b/cdc/owner_test.go deleted file mode 100644 index edd348f7f81..00000000000 --- a/cdc/owner_test.go +++ /dev/null @@ -1,1478 +0,0 @@ -// Copyright 2020 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 cdc - -import ( - "bytes" - "context" - "fmt" - "net/url" - "sync" - "sync/atomic" - "time" - - "github.com/google/uuid" - "github.com/pingcap/check" - "github.com/pingcap/errors" - timodel "github.com/pingcap/parser/model" - "github.com/pingcap/parser/mysql" - "github.com/pingcap/parser/types" - "github.com/pingcap/tidb/meta" - "github.com/pingcap/tidb/store/mockstore" - "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/etcd" - "github.com/pingcap/tiflow/pkg/filter" - "github.com/pingcap/tiflow/pkg/security" - "github.com/pingcap/tiflow/pkg/util" - "github.com/pingcap/tiflow/pkg/util/testleak" - "github.com/tikv/client-go/v2/oracle" - pd "github.com/tikv/pd/client" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/concurrency" - "go.etcd.io/etcd/embed" - "golang.org/x/sync/errgroup" -) - -const TiKVGCLifeTime = 10 * 60 * time.Second // 10 min - -type ownerSuite struct { - e *embed.Etcd - clientURL *url.URL - client kv.CDCEtcdClient - ctx context.Context - cancel context.CancelFunc - errg *errgroup.Group -} - -var _ = check.Suite(&ownerSuite{}) - -func (s *ownerSuite) SetUpTest(c *check.C) { - dir := c.MkDir() - var err error - s.clientURL, s.e, err = etcd.SetupEmbedEtcd(dir) - c.Assert(err, check.IsNil) - client, err := clientv3.New(clientv3.Config{ - Endpoints: []string{s.clientURL.String()}, - DialTimeout: 3 * time.Second, - }) - c.Assert(err, check.IsNil) - s.client = kv.NewCDCEtcdClient(context.TODO(), client) - s.ctx, s.cancel = context.WithCancel(context.Background()) - s.errg = util.HandleErrWithErrGroup(s.ctx, s.e.Err(), func(e error) { c.Log(e) }) -} - -func (s *ownerSuite) TearDownTest(c *check.C) { - s.e.Close() - s.cancel() - err := s.errg.Wait() - if err != nil { - c.Errorf("Error group error: %s", err) - } - s.client.Close() //nolint:errcheck -} - -type mockPDClient struct { - pd.Client - invokeCounter int - mockSafePointLost bool - mockPDFailure bool - mockTiKVGCLifeTime bool -} - -func (m *mockPDClient) GetTS(ctx context.Context) (int64, int64, error) { - if m.mockPDFailure { - return 0, 0, errors.New("injected PD failure") - } - if m.mockSafePointLost { - return 0, 0, nil - } - return oracle.GetPhysical(time.Now()), 0, nil -} - -func (m *mockPDClient) UpdateServiceGCSafePoint(ctx context.Context, serviceID string, ttl int64, safePoint uint64) (uint64, error) { - m.invokeCounter++ - - if m.mockSafePointLost { - return 1000, nil - } - if m.mockPDFailure { - return 0, errors.New("injected PD failure") - } - if m.mockTiKVGCLifeTime { - Ts := oracle.GoTimeToTS(time.Now().Add(-TiKVGCLifeTime)) - return Ts, nil - } - return safePoint, nil -} - -type mockSink struct { - sink.Sink - checkpointTs model.Ts - - checkpointMu sync.Mutex - checkpointError error -} - -func (m *mockSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { - m.checkpointMu.Lock() - defer m.checkpointMu.Unlock() - atomic.StoreUint64(&m.checkpointTs, ts) - return m.checkpointError -} - -func (m *mockSink) Close(ctx context.Context) error { - return nil -} - -func (m *mockSink) Barrier(ctx context.Context) error { - return nil -} - -// Test whether the owner can tolerate sink caused error, it won't be killed. -// also set the specific changefeed to stop -func (s *ownerSuite) TestOwnerCalcResolvedTs(c *check.C) { - defer testleak.AfterTest(c)() - mockPDCli := &mockPDClient{} - - sink := &mockSink{checkpointError: cerror.ErrKafkaSendMessage} - changeFeeds := map[model.ChangeFeedID]*changeFeed{ - "test_change_feed_1": { - info: &model.ChangeFeedInfo{State: model.StateNormal}, - etcdCli: s.client, - status: &model.ChangeFeedStatus{ - CheckpointTs: 0, - }, - targetTs: 2000, - ddlResolvedTs: 2000, - ddlState: model.ChangeFeedSyncDML, - taskStatus: model.ProcessorsInfos{ - "capture_1": {}, - "capture_2": {}, - }, - taskPositions: map[string]*model.TaskPosition{ - "capture_1": { - CheckPointTs: 2333, - ResolvedTs: 2333, - }, - "capture_2": { - CheckPointTs: 2333, - ResolvedTs: 2333, - }, - }, - sink: sink, - }, - } - - session, err := concurrency.NewSession(s.client.Client.Unwrap(), - concurrency.WithTTL(config.GetDefaultServerConfig().CaptureSessionTTL)) - c.Assert(err, check.IsNil) - mockOwner := Owner{ - session: session, - pdClient: mockPDCli, - etcdClient: s.client, - lastFlushChangefeeds: time.Now(), - flushChangefeedInterval: 1 * time.Hour, - changeFeeds: changeFeeds, - cfRWriter: s.client, - stoppedFeeds: make(map[model.ChangeFeedID]*model.ChangeFeedStatus), - minGCSafePointCache: minGCSafePointCacheEntry{}, - } - - err = mockOwner.calcResolvedTs(s.ctx) - c.Assert(err, check.IsNil) - - err = mockOwner.handleAdminJob(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockOwner.stoppedFeeds["test_change_feed_1"], check.NotNil) - - err = mockOwner.flushChangeFeedInfos(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockPDCli.invokeCounter, check.Equals, 1) - - s.TearDownTest(c) -} - -func (s *ownerSuite) TestOwnerFlushChangeFeedInfos(c *check.C) { - defer testleak.AfterTest(c)() - session, err := concurrency.NewSession(s.client.Client.Unwrap(), - concurrency.WithTTL(config.GetDefaultServerConfig().CaptureSessionTTL)) - c.Assert(err, check.IsNil) - mockPDCli := &mockPDClient{} - mockOwner := Owner{ - session: session, - etcdClient: s.client, - pdClient: mockPDCli, - gcSafepointLastUpdate: time.Now(), - } - - err = mockOwner.flushChangeFeedInfos(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockPDCli.invokeCounter, check.Equals, 1) - s.TearDownTest(c) -} - -func (s *ownerSuite) TestOwnerFlushChangeFeedInfosFailed(c *check.C) { - defer testleak.AfterTest(c)() - mockPDCli := &mockPDClient{ - mockPDFailure: true, - } - - changeFeeds := map[model.ChangeFeedID]*changeFeed{ - "test_change_feed_1": { - info: &model.ChangeFeedInfo{State: model.StateNormal}, - status: &model.ChangeFeedStatus{ - CheckpointTs: 100, - }, - targetTs: 2000, - ddlState: model.ChangeFeedSyncDML, - taskStatus: model.ProcessorsInfos{ - "capture_1": {}, - "capture_2": {}, - }, - taskPositions: map[string]*model.TaskPosition{ - "capture_1": {}, - "capture_2": {}, - }, - }, - } - - session, err := concurrency.NewSession(s.client.Client.Unwrap(), - concurrency.WithTTL(config.GetDefaultServerConfig().CaptureSessionTTL)) - c.Assert(err, check.IsNil) - mockOwner := Owner{ - session: session, - pdClient: mockPDCli, - etcdClient: s.client, - lastFlushChangefeeds: time.Now(), - flushChangefeedInterval: 1 * time.Hour, - gcSafepointLastUpdate: time.Now(), - gcTTL: 6, // 6 seconds - changeFeeds: changeFeeds, - } - - time.Sleep(3 * time.Second) - err = mockOwner.flushChangeFeedInfos(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockPDCli.invokeCounter, check.Equals, 1) - - time.Sleep(6 * time.Second) - err = mockOwner.flushChangeFeedInfos(s.ctx) - c.Assert(err, check.ErrorMatches, ".*CDC:ErrUpdateServiceSafepointFailed.*") - c.Assert(mockPDCli.invokeCounter, check.Equals, 2) - - s.TearDownTest(c) -} - -// Test whether it is possible to successfully create a changefeed -// with startTs less than currentTs - gcTTL when tikv_gc_life_time is greater than gc-ttl -func (s *ownerSuite) TestTiKVGCLifeTimeLargeThanGCTTL(c *check.C) { - defer testleak.AfterTest(c) - mockPDCli := &mockPDClient{} - mockPDCli.mockTiKVGCLifeTime = true - - changeFeeds := map[model.ChangeFeedID]*changeFeed{ - "test_change_feed_1": { - info: &model.ChangeFeedInfo{State: model.StateNormal}, - etcdCli: s.client, - status: &model.ChangeFeedStatus{ - CheckpointTs: oracle.GoTimeToTS(time.Now().Add(-6 * time.Second)), - }, - targetTs: 2000, - ddlState: model.ChangeFeedSyncDML, - taskStatus: model.ProcessorsInfos{ - "capture_1": {}, - "capture_2": {}, - }, - taskPositions: map[string]*model.TaskPosition{ - "capture_1": {}, - "capture_2": {}, - }, - }, - } - - session, err := concurrency.NewSession(s.client.Client.Unwrap(), - concurrency.WithTTL(config.GetDefaultServerConfig().CaptureSessionTTL)) - c.Assert(err, check.IsNil) - - mockOwner := Owner{ - session: session, - pdClient: mockPDCli, - etcdClient: s.client, - lastFlushChangefeeds: time.Now(), - flushChangefeedInterval: 1 * time.Hour, - // gcSafepointLastUpdate: time.Now(), - gcTTL: 6, // 6 seconds - changeFeeds: changeFeeds, - cfRWriter: s.client, - stoppedFeeds: make(map[model.ChangeFeedID]*model.ChangeFeedStatus), - minGCSafePointCache: minGCSafePointCacheEntry{}, - } - - err = mockOwner.flushChangeFeedInfos(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockPDCli.invokeCounter, check.Equals, 1) - - err = mockOwner.handleAdminJob(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockOwner.stoppedFeeds["test_change_feed_1"], check.IsNil) - c.Assert(mockOwner.changeFeeds["test_change_feed_1"].info.State, check.Equals, model.StateNormal) - - time.Sleep(7 * time.Second) // wait for gcTTL time pass - err = mockOwner.flushChangeFeedInfos(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockPDCli.invokeCounter, check.Equals, 2) - - err = mockOwner.handleAdminJob(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockOwner.stoppedFeeds["test_change_feed_1"], check.IsNil) - - s.TearDownTest(c) -} - -// Test whether the owner handles the stagnant task correctly, so that it can't block the update of gcSafePoint. -// If a changefeed is put into the stop queue due to stagnation, it can no longer affect the update of gcSafePoint. -// So we just need to test whether the stagnant changefeed is put into the stop queue. -func (s *ownerSuite) TestOwnerHandleStaleChangeFeed(c *check.C) { - defer testleak.AfterTest(c)() - mockPDCli := &mockPDClient{} - changeFeeds := map[model.ChangeFeedID]*changeFeed{ - "test_change_feed_1": { - info: &model.ChangeFeedInfo{State: model.StateNormal}, - etcdCli: s.client, - status: &model.ChangeFeedStatus{ - CheckpointTs: 1000, - }, - targetTs: 2000, - ddlState: model.ChangeFeedSyncDML, - taskStatus: model.ProcessorsInfos{ - "capture_1": {}, - "capture_2": {}, - }, - taskPositions: map[string]*model.TaskPosition{ - "capture_1": {}, - "capture_2": {}, - }, - }, - "test_change_feed_2": { - info: &model.ChangeFeedInfo{State: model.StateNormal}, - etcdCli: s.client, - status: &model.ChangeFeedStatus{ - CheckpointTs: oracle.GoTimeToTS(time.Now()), - }, - targetTs: 2000, - ddlState: model.ChangeFeedSyncDML, - taskStatus: model.ProcessorsInfos{ - "capture_1": {}, - "capture_2": {}, - }, - taskPositions: map[string]*model.TaskPosition{ - "capture_1": {}, - "capture_2": {}, - }, - }, - } - - session, err := concurrency.NewSession(s.client.Client.Unwrap(), - concurrency.WithTTL(config.GetDefaultServerConfig().CaptureSessionTTL)) - c.Assert(err, check.IsNil) - - mockOwner := Owner{ - session: session, - pdClient: mockPDCli, - etcdClient: s.client, - lastFlushChangefeeds: time.Now(), - flushChangefeedInterval: 1 * time.Hour, - gcSafepointLastUpdate: time.Now().Add(-4 * time.Second), - gcTTL: 6, // 6 seconds - changeFeeds: changeFeeds, - cfRWriter: s.client, - stoppedFeeds: make(map[model.ChangeFeedID]*model.ChangeFeedStatus), - minGCSafePointCache: minGCSafePointCacheEntry{}, - } - - err = mockOwner.flushChangeFeedInfos(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockPDCli.invokeCounter, check.Equals, 1) - err = mockOwner.handleAdminJob(s.ctx) - c.Assert(err, check.IsNil) - - time.Sleep(2 * time.Second) - err = mockOwner.flushChangeFeedInfos(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockPDCli.invokeCounter, check.Equals, 2) - err = mockOwner.handleAdminJob(s.ctx) - c.Assert(err, check.IsNil) - - c.Assert(mockOwner.stoppedFeeds["test_change_feed_1"], check.NotNil) - c.Assert(mockOwner.changeFeeds["test_change_feed_2"].info.State, check.Equals, model.StateNormal) - - time.Sleep(6 * time.Second) - err = mockOwner.flushChangeFeedInfos(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockPDCli.invokeCounter, check.Equals, 3) - err = mockOwner.handleAdminJob(s.ctx) - c.Assert(err, check.IsNil) - - time.Sleep(2 * time.Second) - err = mockOwner.flushChangeFeedInfos(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockPDCli.invokeCounter, check.Equals, 4) - err = mockOwner.handleAdminJob(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockOwner.stoppedFeeds["test_change_feed_2"], check.NotNil) - - s.TearDownTest(c) -} - -func (s *ownerSuite) TestOwnerUploadGCSafePointOutdated(c *check.C) { - defer testleak.AfterTest(c)() - mockPDCli := &mockPDClient{ - mockSafePointLost: true, - } - changeFeeds := map[model.ChangeFeedID]*changeFeed{ - "test_change_feed_1": { - info: &model.ChangeFeedInfo{State: model.StateNormal}, - etcdCli: s.client, - status: &model.ChangeFeedStatus{ - CheckpointTs: 100, - }, - targetTs: 2000, - ddlState: model.ChangeFeedSyncDML, - taskStatus: model.ProcessorsInfos{ - "capture_1": {}, - "capture_2": {}, - }, - taskPositions: map[string]*model.TaskPosition{ - "capture_1": {}, - "capture_2": {}, - }, - }, - "test_change_feed_2": { - info: &model.ChangeFeedInfo{State: model.StateNormal}, - etcdCli: s.client, - status: &model.ChangeFeedStatus{ - CheckpointTs: 1100, - }, - targetTs: 2000, - ddlState: model.ChangeFeedSyncDML, - taskStatus: model.ProcessorsInfos{ - "capture_1": {}, - "capture_2": {}, - }, - taskPositions: map[string]*model.TaskPosition{ - "capture_1": {}, - "capture_2": {}, - }, - }, - } - - session, err := concurrency.NewSession(s.client.Client.Unwrap(), - concurrency.WithTTL(config.GetDefaultServerConfig().CaptureSessionTTL)) - c.Assert(err, check.IsNil) - - mockOwner := Owner{ - pdClient: mockPDCli, - session: session, - etcdClient: s.client, - lastFlushChangefeeds: time.Now(), - flushChangefeedInterval: 1 * time.Hour, - changeFeeds: changeFeeds, - cfRWriter: s.client, - stoppedFeeds: make(map[model.ChangeFeedID]*model.ChangeFeedStatus), - minGCSafePointCache: minGCSafePointCacheEntry{}, - } - - err = mockOwner.flushChangeFeedInfos(s.ctx) - c.Assert(err, check.IsNil) - c.Assert(mockPDCli.invokeCounter, check.Equals, 1) - - err = mockOwner.handleAdminJob(s.ctx) - c.Assert(err, check.IsNil) - - c.Assert(mockOwner.stoppedFeeds["test_change_feed_1"], check.NotNil) - c.Assert(changeFeeds["test_change_feed_2"].info.State, check.Equals, model.StateNormal) - s.TearDownTest(c) -} - -/* -type handlerForPrueDMLTest struct { - mu sync.RWMutex - index int - resolvedTs1 []uint64 - resolvedTs2 []uint64 - expectResolvedTs []uint64 - c *check.C - cancel func() -} - -func (h *handlerForPrueDMLTest) PullDDL() (resolvedTs uint64, ddl []*model.DDL, err error) { - return uint64(math.MaxUint64), nil, nil -} - -func (h *handlerForPrueDMLTest) ExecDDL(context.Context, string, map[string]string, model.SingleTableTxn) error { - panic("unreachable") -} - -func (h *handlerForPrueDMLTest) Close() error { - return nil -} - -var _ ChangeFeedRWriter = &handlerForPrueDMLTest{} - -func (h *handlerForPrueDMLTest) GetChangeFeeds(ctx context.Context) (int64, map[string]*mvccpb.KeyValue, error) { - h.mu.RLock() - defer h.mu.RUnlock() - cfInfo := &model.ChangeFeedInfo{ - TargetTs: 100, - } - cfInfoJSON, err := cfInfo.Marshal() - h.c.Assert(err, check.IsNil) - rawKV := &mvccpb.KeyValue{ - Value: []byte(cfInfoJSON), - } - return 0, map[model.ChangeFeedID]*mvccpb.KeyValue{ - "test_change_feed": rawKV, - }, nil -} - -func (h *handlerForPrueDMLTest) GetAllTaskStatus(ctx context.Context, changefeedID string) (model.ProcessorsInfos, error) { - if changefeedID != "test_change_feed" { - return nil, cerror.ErrTaskStatusNotExists.GenWithStackByArgs("test_change_feed) - } - h.mu.RLock() - defer h.mu.RUnlock() - h.index++ - return model.ProcessorsInfos{ - "capture_1": {}, - "capture_2": {}, - }, nil -} - -func (h *handlerForPrueDMLTest) GetAllTaskPositions(ctx context.Context, changefeedID string) (map[string]*model.TaskPosition, error) { - if changefeedID != "test_change_feed" { - return nil, cerror.ErrTaskStatusNotExists.GenWithStackByArgs("test_change_feed) - } - h.mu.RLock() - defer h.mu.RUnlock() - h.index++ - return map[string]*model.TaskPosition{ - "capture_1": { - ResolvedTs: h.resolvedTs1[h.index], - }, - "capture_2": { - ResolvedTs: h.resolvedTs2[h.index], - }, - }, nil -} - -func (h *handlerForPrueDMLTest) GetChangeFeedStatus(ctx context.Context, id string) (*model.ChangeFeedStatus, error) { - return nil, cerror.ErrChangeFeedNotExists.GenWithStackByArgs(id) -} - -func (h *handlerForPrueDMLTest) PutAllChangeFeedStatus(ctx context.Context, infos map[model.ChangeFeedID]*model.ChangeFeedStatus) error { - h.mu.Lock() - defer h.mu.Unlock() - info, exist := infos["test_change_feed"] - h.c.Assert(exist, check.IsTrue) - h.c.Assert(info.ResolvedTs, check.Equals, h.expectResolvedTs[h.index]) - // h.c.Assert(info.State, check.Equals, model.ChangeFeedSyncDML) - if h.index >= len(h.expectResolvedTs)-1 { - log.Info("cancel") - h.cancel() - } - return nil -} - -func (s *ownerSuite) TestPureDML(c *check.C) { - defer testleak.AfterTest(c)() - ctx, cancel := context.WithCancel(context.Background()) - handler := &handlerForPrueDMLTest{ - index: -1, - resolvedTs1: []uint64{10, 22, 64, 92, 99, 120}, - resolvedTs2: []uint64{8, 36, 53, 88, 103, 108}, - expectResolvedTs: []uint64{8, 22, 53, 88, 99, 100}, - cancel: cancel, - c: c, - } - - tables := map[uint64]model.TableName{1: {Schema: "any"}} - - changeFeeds := map[model.ChangeFeedID]*changeFeed{ - "test_change_feed": { - tables: tables, - status: &model.ChangeFeedStatus{}, - targetTs: 100, - ddlState: model.ChangeFeedSyncDML, - taskStatus: model.ProcessorsInfos{ - "capture_1": {}, - "capture_2": {}, - }, - taskPositions: map[string]*model.TaskPosition{ - "capture_1": {}, - "capture_2": {}, - }, - ddlHandler: handler, - }, - } - - manager := roles.NewMockManager(uuid.New().String(), cancel) - err := manager.CampaignOwner(ctx) - c.Assert(err, check.IsNil) - owner := &ownerImpl{ - cancelWatchCapture: cancel, - changeFeeds: changeFeeds, - cfRWriter: handler, - etcdClient: s.client, - manager: manager, - } - s.owner = owner - err = owner.Run(ctx, 50*time.Millisecond) - c.Assert(err.Error(), check.Equals, "context canceled") -} - -type handlerForDDLTest struct { - mu sync.RWMutex - - ddlIndex int - ddls []*model.DDL - ddlResolvedTs []uint64 - - ddlExpectIndex int - - dmlIndex int - resolvedTs1 []uint64 - resolvedTs2 []uint64 - currentGlobalResolvedTs uint64 - - dmlExpectIndex int - expectResolvedTs []uint64 - expectStatus []model.ChangeFeedDDLState - - c *check.C - cancel func() -} - -func (h *handlerForDDLTest) PullDDL() (resolvedTs uint64, jobs []*model.DDL, err error) { - h.mu.RLock() - defer h.mu.RUnlock() - if h.ddlIndex < len(h.ddls)-1 { - h.ddlIndex++ - } - return h.ddlResolvedTs[h.ddlIndex], []*model.DDL{h.ddls[h.ddlIndex]}, nil -} - -func (h *handlerForDDLTest) ExecDDL(ctx context.Context, sinkURI string, _ map[string]string, txn model.SingleTableTxn) error { - h.mu.Lock() - defer h.mu.Unlock() - h.ddlExpectIndex++ - h.c.Assert(txn.DDL, check.DeepEquals, h.ddls[h.ddlExpectIndex]) - h.c.Assert(txn.DDL.Job.BinlogInfo.FinishedTS, check.Equals, h.currentGlobalResolvedTs) - return nil -} - -func (h *handlerForDDLTest) Close() error { - return nil -} - -func (h *handlerForDDLTest) GetChangeFeeds(ctx context.Context) (int64, map[string]*mvccpb.KeyValue, error) { - h.mu.RLock() - defer h.mu.RUnlock() - cfInfo := &model.ChangeFeedInfo{ - TargetTs: 100, - } - cfInfoJSON, err := cfInfo.Marshal() - h.c.Assert(err, check.IsNil) - rawKV := &mvccpb.KeyValue{ - Value: []byte(cfInfoJSON), - } - return 0, map[model.ChangeFeedID]*mvccpb.KeyValue{ - "test_change_feed": rawKV, - }, nil -} - -func (h *handlerForDDLTest) GetAllTaskStatus(ctx context.Context, changefeedID string) (model.ProcessorsInfos, error) { - if changefeedID != "test_change_feed" { - return nil, cerror.ErrTaskStatusNotExists.GenWithStackByArgs("test_change_feed") - } - h.mu.RLock() - defer h.mu.RUnlock() - if h.dmlIndex < len(h.resolvedTs1)-1 { - h.dmlIndex++ - } - return model.ProcessorsInfos{ - "capture_1": {}, - "capture_2": {}, - }, nil -} - -func (h *handlerForDDLTest) GetAllTaskPositions(ctx context.Context, changefeedID string) (map[string]*model.TaskPosition, error) { - if changefeedID != "test_change_feed" { - return nil, cerror.ErrTaskStatusNotExists.GenWithStackByArgs("test_change_feed") - } - h.mu.RLock() - defer h.mu.RUnlock() - if h.dmlIndex < len(h.resolvedTs1)-1 { - h.dmlIndex++ - } - return map[string]*model.TaskPosition{ - "capture_1": { - ResolvedTs: h.resolvedTs1[h.dmlIndex], - CheckPointTs: h.currentGlobalResolvedTs, - }, - "capture_2": { - ResolvedTs: h.resolvedTs2[h.dmlIndex], - CheckPointTs: h.currentGlobalResolvedTs, - }, - }, nil -} - -func (h *handlerForDDLTest) GetChangeFeedStatus(ctx context.Context, id string) (*model.ChangeFeedStatus, error) { - return nil, cerror.ErrChangeFeedNotExists.GenWithStackByArgs(id) -} - -func (h *handlerForDDLTest) PutAllChangeFeedStatus(ctx context.Context, infos map[model.ChangeFeedID]*model.ChangeFeedStatus) error { - h.mu.Lock() - defer h.mu.Unlock() - h.dmlExpectIndex++ - info, exist := infos["test_change_feed"] - h.c.Assert(exist, check.IsTrue) - h.currentGlobalResolvedTs = info.ResolvedTs - h.c.Assert(info.ResolvedTs, check.Equals, h.expectResolvedTs[h.dmlExpectIndex]) - // h.c.Assert(info.State, check.Equals, h.expectStatus[h.dmlExpectIndex]) - if h.dmlExpectIndex >= len(h.expectResolvedTs)-1 { - log.Info("cancel") - h.cancel() - } - return nil -} - -func (s *ownerSuite) TestDDL(c *check.C) { - defer testleak.AfterTest(c)() - ctx, cancel := context.WithCancel(context.Background()) - - handler := &handlerForDDLTest{ - ddlIndex: -1, - ddlResolvedTs: []uint64{5, 8, 49, 91, 113}, - ddls: []*model.DDL{ - {Job: &timodel.Job{ - ID: 1, - BinlogInfo: &timodel.HistoryInfo{ - FinishedTS: 3, - }, - }}, - {Job: &timodel.Job{ - ID: 2, - BinlogInfo: &timodel.HistoryInfo{ - FinishedTS: 7, - }, - }}, - {Job: &timodel.Job{ - ID: 3, - BinlogInfo: &timodel.HistoryInfo{ - FinishedTS: 11, - }, - }}, - {Job: &timodel.Job{ - ID: 4, - BinlogInfo: &timodel.HistoryInfo{ - FinishedTS: 89, - }, - }}, - {Job: &timodel.Job{ - ID: 5, - BinlogInfo: &timodel.HistoryInfo{ - FinishedTS: 111, - }, - }}, - }, - - ddlExpectIndex: -1, - - dmlIndex: -1, - resolvedTs1: []uint64{10, 22, 64, 92, 99, 120}, - resolvedTs2: []uint64{8, 36, 53, 88, 103, 108}, - currentGlobalResolvedTs: 0, - - dmlExpectIndex: -1, - expectResolvedTs: []uint64{ - 3, 3, - 7, 7, - 11, 11, - 89, 89, - 100}, - expectStatus: []model.ChangeFeedDDLState{ - model.ChangeFeedWaitToExecDDL, model.ChangeFeedExecDDL, - model.ChangeFeedWaitToExecDDL, model.ChangeFeedExecDDL, - model.ChangeFeedWaitToExecDDL, model.ChangeFeedExecDDL, - model.ChangeFeedWaitToExecDDL, model.ChangeFeedExecDDL, - model.ChangeFeedSyncDML}, - - cancel: cancel, - c: c, - } - - tables := map[uint64]model.TableName{1: {Schema: "any"}} - - filter, err := newTxnFilter(&model.ReplicaConfig{}) - c.Assert(err, check.IsNil) - changeFeeds := map[model.ChangeFeedID]*changeFeed{ - "test_change_feed": { - tables: tables, - info: &model.ChangeFeedInfo{}, - status: &model.ChangeFeedStatus{}, - targetTs: 100, - ddlState: model.ChangeFeedSyncDML, - taskStatus: model.ProcessorsInfos{ - "capture_1": {}, - "capture_2": {}, - }, - taskPositions: map[string]*model.TaskPosition{ - "capture_1": {}, - "capture_2": {}, - }, - ddlHandler: handler, - filter: filter, - }, - } - - manager := roles.NewMockManager(uuid.New().String(), cancel) - err = manager.CampaignOwner(ctx) - c.Assert(err, check.IsNil) - owner := &ownerImpl{ - cancelWatchCapture: cancel, - changeFeeds: changeFeeds, - - // ddlHandler: handler, - etcdClient: s.client, - cfRWriter: handler, - manager: manager, - } - s.owner = owner - err = owner.Run(ctx, 50*time.Millisecond) - c.Assert(errors.Cause(err), check.DeepEquals, context.Canceled) -} -*/ -var cdcGCSafePointTTL4Test = int64(24 * 60 * 60) - -func (s *ownerSuite) TestHandleAdmin(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - cfID := "test_handle_admin" - - ctx, cancel0 := context.WithCancel(context.Background()) - defer cancel0() - cctx, cancel := context.WithCancel(ctx) - errg, _ := errgroup.WithContext(cctx) - - replicaConf := config.GetDefaultReplicaConfig() - f, err := filter.NewFilter(replicaConf) - c.Assert(err, check.IsNil) - - sampleCF := &changeFeed{ - id: cfID, - info: &model.ChangeFeedInfo{}, - status: &model.ChangeFeedStatus{}, - ddlState: model.ChangeFeedSyncDML, - taskStatus: model.ProcessorsInfos{ - "capture_1": {}, - "capture_2": {}, - }, - taskPositions: map[string]*model.TaskPosition{ - "capture_1": {ResolvedTs: 10001}, - "capture_2": {}, - }, - ddlHandler: &ddlHandler{ - cancel: cancel, - wg: errg, - }, - cancel: cancel, - } - errCh := make(chan error, 1) - sink, err := sink.NewSink(ctx, cfID, "blackhole://", f, replicaConf, map[string]string{}, errCh) - c.Assert(err, check.IsNil) - defer sink.Close(cctx) //nolint:errcheck - sampleCF.sink = sink - - capture, err := NewCapture(ctx, []string{s.clientURL.String()}, nil, nil) - c.Assert(err, check.IsNil) - err = capture.Campaign(ctx) - c.Assert(err, check.IsNil) - - grpcPool := kv.NewGrpcPoolImpl(ctx, &security.Credential{}) - defer grpcPool.Close() - owner, err := NewOwner(ctx, nil, grpcPool, capture.session, cdcGCSafePointTTL4Test, time.Millisecond*200) - c.Assert(err, check.IsNil) - - sampleCF.etcdCli = owner.etcdClient - owner.changeFeeds = map[model.ChangeFeedID]*changeFeed{cfID: sampleCF} - for cid, pinfo := range sampleCF.taskPositions { - key := kv.GetEtcdKeyTaskStatus(cfID, cid) - pinfoStr, err := pinfo.Marshal() - c.Assert(err, check.IsNil) - _, err = s.client.Client.Put(ctx, key, pinfoStr) - c.Assert(err, check.IsNil) - } - err = owner.etcdClient.PutChangeFeedStatus(ctx, cfID, &model.ChangeFeedStatus{}) - c.Assert(err, check.IsNil) - err = owner.etcdClient.SaveChangeFeedInfo(ctx, sampleCF.info, cfID) - c.Assert(err, check.IsNil) - checkAdminJobLen := func(length int) { - owner.adminJobsLock.Lock() - c.Assert(owner.adminJobs, check.HasLen, length) - owner.adminJobsLock.Unlock() - } - - c.Assert(owner.EnqueueJob(model.AdminJob{CfID: cfID, Type: model.AdminStop}), check.IsNil) - checkAdminJobLen(1) - c.Assert(owner.handleAdminJob(ctx), check.IsNil) - checkAdminJobLen(0) - c.Assert(len(owner.changeFeeds), check.Equals, 0) - // check changefeed info is set admin job - info, err := owner.etcdClient.GetChangeFeedInfo(ctx, cfID) - c.Assert(err, check.IsNil) - c.Assert(info.AdminJobType, check.Equals, model.AdminStop) - // check processor is set admin job - for cid := range sampleCF.taskPositions { - _, subInfo, err := owner.etcdClient.GetTaskStatus(ctx, cfID, cid) - c.Assert(err, check.IsNil) - c.Assert(subInfo.AdminJobType, check.Equals, model.AdminStop) - } - // check changefeed status is set admin job - st, _, err := owner.etcdClient.GetChangeFeedStatus(ctx, cfID) - c.Assert(err, check.IsNil) - c.Assert(st.AdminJobType, check.Equals, model.AdminStop) - // check changefeed context is canceled - select { - case <-cctx.Done(): - default: - c.Fatal("changefeed context is expected canceled") - } - - cctx, cancel = context.WithCancel(ctx) - sampleCF.cancel = cancel - - c.Assert(owner.EnqueueJob(model.AdminJob{CfID: cfID, Type: model.AdminResume}), check.IsNil) - c.Assert(owner.handleAdminJob(ctx), check.IsNil) - checkAdminJobLen(0) - // check changefeed info is set admin job - info, err = owner.etcdClient.GetChangeFeedInfo(ctx, cfID) - c.Assert(err, check.IsNil) - c.Assert(info.AdminJobType, check.Equals, model.AdminResume) - // check changefeed status is set admin job - st, _, err = owner.etcdClient.GetChangeFeedStatus(ctx, cfID) - c.Assert(err, check.IsNil) - c.Assert(st.AdminJobType, check.Equals, model.AdminResume) - - owner.changeFeeds[cfID] = sampleCF - c.Assert(owner.EnqueueJob(model.AdminJob{CfID: cfID, Type: model.AdminRemove}), check.IsNil) - c.Assert(owner.handleAdminJob(ctx), check.IsNil) - checkAdminJobLen(0) - c.Assert(len(owner.changeFeeds), check.Equals, 0) - // check changefeed info is deleted - _, err = owner.etcdClient.GetChangeFeedInfo(ctx, cfID) - c.Assert(cerror.ErrChangeFeedNotExists.Equal(err), check.IsTrue) - // check processor is set admin job - for cid := range sampleCF.taskPositions { - _, subInfo, err := owner.etcdClient.GetTaskStatus(ctx, cfID, cid) - c.Assert(err, check.IsNil) - c.Assert(subInfo.AdminJobType, check.Equals, model.AdminRemove) - } - // check changefeed status is set admin job - st, _, err = owner.etcdClient.GetChangeFeedStatus(ctx, cfID) - c.Assert(err, check.IsNil) - c.Assert(st.AdminJobType, check.Equals, model.AdminRemove) - // check changefeed context is canceled - select { - case <-cctx.Done(): - default: - c.Fatal("changefeed context is expected canceled") - } - owner.etcdClient.Close() //nolint:errcheck -} - -func (s *ownerSuite) TestChangefeedApplyDDLJob(c *check.C) { - defer testleak.AfterTest(c)() - var ( - jobs = []*timodel.Job{ - { - ID: 1, - SchemaID: 1, - Type: timodel.ActionCreateSchema, - State: timodel.JobStateSynced, - Query: "create database test", - BinlogInfo: &timodel.HistoryInfo{ - SchemaVersion: 1, - DBInfo: &timodel.DBInfo{ - ID: 1, - Name: timodel.NewCIStr("test"), - }, - }, - }, - { - ID: 2, - SchemaID: 1, - Type: timodel.ActionCreateTable, - State: timodel.JobStateSynced, - Query: "create table t1 (id int primary key)", - BinlogInfo: &timodel.HistoryInfo{ - SchemaVersion: 2, - DBInfo: &timodel.DBInfo{ - ID: 1, - Name: timodel.NewCIStr("test"), - }, - TableInfo: &timodel.TableInfo{ - ID: 47, - Name: timodel.NewCIStr("t1"), - PKIsHandle: true, - Columns: []*timodel.ColumnInfo{ - {ID: 1, FieldType: types.FieldType{Flag: mysql.PriKeyFlag}, State: timodel.StatePublic}, - }, - }, - }, - }, - { - ID: 2, - SchemaID: 1, - Type: timodel.ActionCreateTable, - State: timodel.JobStateSynced, - Query: "create table t2 (id int primary key)", - BinlogInfo: &timodel.HistoryInfo{ - SchemaVersion: 2, - DBInfo: &timodel.DBInfo{ - ID: 1, - Name: timodel.NewCIStr("test"), - }, - TableInfo: &timodel.TableInfo{ - ID: 49, - Name: timodel.NewCIStr("t2"), - PKIsHandle: true, - Columns: []*timodel.ColumnInfo{ - {ID: 1, FieldType: types.FieldType{Flag: mysql.PriKeyFlag}, State: timodel.StatePublic}, - }, - }, - }, - }, - { - ID: 2, - SchemaID: 1, - TableID: 49, - Type: timodel.ActionDropTable, - State: timodel.JobStateSynced, - Query: "drop table t2", - BinlogInfo: &timodel.HistoryInfo{ - SchemaVersion: 3, - DBInfo: &timodel.DBInfo{ - ID: 1, - Name: timodel.NewCIStr("test"), - }, - TableInfo: &timodel.TableInfo{ - ID: 49, - Name: timodel.NewCIStr("t2"), - }, - }, - }, - { - ID: 2, - SchemaID: 1, - TableID: 47, - Type: timodel.ActionTruncateTable, - State: timodel.JobStateSynced, - Query: "truncate table t1", - BinlogInfo: &timodel.HistoryInfo{ - SchemaVersion: 4, - DBInfo: &timodel.DBInfo{ - ID: 1, - Name: timodel.NewCIStr("test"), - }, - TableInfo: &timodel.TableInfo{ - ID: 51, - Name: timodel.NewCIStr("t1"), - PKIsHandle: true, - Columns: []*timodel.ColumnInfo{ - {ID: 1, FieldType: types.FieldType{Flag: mysql.PriKeyFlag}, State: timodel.StatePublic}, - }, - }, - }, - }, - { - ID: 2, - SchemaID: 1, - TableID: 51, - Type: timodel.ActionDropTable, - State: timodel.JobStateSynced, - Query: "drop table t1", - BinlogInfo: &timodel.HistoryInfo{ - SchemaVersion: 5, - DBInfo: &timodel.DBInfo{ - ID: 1, - Name: timodel.NewCIStr("test"), - }, - TableInfo: &timodel.TableInfo{ - ID: 51, - Name: timodel.NewCIStr("t1"), - }, - }, - }, - { - ID: 2, - SchemaID: 1, - Type: timodel.ActionDropSchema, - State: timodel.JobStateSynced, - Query: "drop database test", - BinlogInfo: &timodel.HistoryInfo{ - SchemaVersion: 6, - DBInfo: &timodel.DBInfo{ - ID: 1, - Name: timodel.NewCIStr("test"), - }, - }, - }, - } - - expectSchemas = []map[int64]tableIDMap{ - {1: make(tableIDMap)}, - {1: {47: struct{}{}}}, - {1: {47: struct{}{}, 49: struct{}{}}}, - {1: {47: struct{}{}}}, - {1: {51: struct{}{}}}, - {1: make(tableIDMap)}, - {}, - } - - expectTables = []map[int64]model.TableName{ - {}, - {47: {Schema: "test", Table: "t1"}}, - {47: {Schema: "test", Table: "t1"}, 49: {Schema: "test", Table: "t2"}}, - {47: {Schema: "test", Table: "t1"}}, - {51: {Schema: "test", Table: "t1"}}, - {}, - {}, - } - ) - f, err := filter.NewFilter(config.GetDefaultReplicaConfig()) - c.Assert(err, check.IsNil) - - store, err := mockstore.NewMockStore() - c.Assert(err, check.IsNil) - defer func() { - _ = store.Close() - }() - - txn, err := store.Begin() - c.Assert(err, check.IsNil) - defer func() { - _ = txn.Rollback() - }() - t := meta.NewMeta(txn) - - schemaSnap, err := entry.NewSingleSchemaSnapshotFromMeta(t, 0, false) - c.Assert(err, check.IsNil) - - cf := &changeFeed{ - schema: schemaSnap, - schemas: make(map[model.SchemaID]tableIDMap), - tables: make(map[model.TableID]model.TableName), - partitions: make(map[model.TableID][]int64), - orphanTables: make(map[model.TableID]model.Ts), - toCleanTables: make(map[model.TableID]model.Ts), - filter: f, - info: &model.ChangeFeedInfo{Config: config.GetDefaultReplicaConfig()}, - } - for i, job := range jobs { - err = cf.schema.HandleDDL(job) - c.Assert(err, check.IsNil) - err = cf.schema.FillSchemaName(job) - c.Assert(err, check.IsNil) - _, err = cf.applyJob(job) - c.Assert(err, check.IsNil) - c.Assert(cf.schemas, check.DeepEquals, expectSchemas[i]) - c.Assert(cf.tables, check.DeepEquals, expectTables[i]) - } - s.TearDownTest(c) -} - -func (s *ownerSuite) TestWatchCampaignKey(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - capture, err := NewCapture(ctx, []string{s.clientURL.String()}, nil, nil) - c.Assert(err, check.IsNil) - err = capture.Campaign(ctx) - c.Assert(err, check.IsNil) - - grpcPool := kv.NewGrpcPoolImpl(ctx, &security.Credential{}) - defer grpcPool.Close() - ctx1, cancel1 := context.WithCancel(ctx) - owner, err := NewOwner(ctx1, nil, grpcPool, capture.session, - cdcGCSafePointTTL4Test, time.Millisecond*200) - c.Assert(err, check.IsNil) - - // check campaign key deleted can be detected - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - err := owner.watchCampaignKey(ctx1) - c.Assert(cerror.ErrOwnerCampaignKeyDeleted.Equal(err), check.IsTrue) - cancel1() - }() - // ensure the watch loop has started - time.Sleep(time.Millisecond * 100) - etcdCli := owner.etcdClient.Client.Unwrap() - key := fmt.Sprintf("%s/%x", kv.CaptureOwnerKey, owner.session.Lease()) - _, err = etcdCli.Delete(ctx, key) - c.Assert(err, check.IsNil) - wg.Wait() - - // check key is deleted before watch loop starts - ctx1, cancel1 = context.WithCancel(ctx) - err = owner.watchCampaignKey(ctx1) - c.Assert(cerror.ErrOwnerCampaignKeyDeleted.Equal(err), check.IsTrue) - - // check the watch routine can be canceled - err = capture.Campaign(ctx) - c.Assert(err, check.IsNil) - wg.Add(1) - go func() { - defer wg.Done() - err := owner.watchCampaignKey(ctx1) - c.Assert(err, check.IsNil) - }() - // ensure the watch loop has started - time.Sleep(time.Millisecond * 100) - cancel1() - wg.Wait() - - err = capture.etcdClient.Close() - c.Assert(err, check.IsNil) -} - -func (s *ownerSuite) TestCleanUpStaleTasks(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - addr := "127.0.0.1:12034" - ctx = util.PutCaptureAddrInCtx(ctx, addr) - capture, err := NewCapture(ctx, []string{s.clientURL.String()}, nil, nil) - c.Assert(err, check.IsNil) - err = s.client.PutCaptureInfo(ctx, capture.info, capture.session.Lease()) - c.Assert(err, check.IsNil) - - changefeed := "changefeed-name" - invalidCapture := uuid.New().String() - for _, captureID := range []string{capture.info.ID, invalidCapture} { - taskStatus := &model.TaskStatus{} - if captureID == invalidCapture { - taskStatus.Tables = map[model.TableID]*model.TableReplicaInfo{ - 51: {StartTs: 110}, - } - } - err = s.client.PutTaskStatus(ctx, changefeed, captureID, taskStatus) - c.Assert(err, check.IsNil) - _, err = s.client.PutTaskPositionOnChange(ctx, changefeed, captureID, &model.TaskPosition{CheckPointTs: 100, ResolvedTs: 120}) - c.Assert(err, check.IsNil) - err = s.client.PutTaskWorkload(ctx, changefeed, captureID, &model.TaskWorkload{}) - c.Assert(err, check.IsNil) - } - err = s.client.SaveChangeFeedInfo(ctx, &model.ChangeFeedInfo{}, changefeed) - c.Assert(err, check.IsNil) - - _, captureList, err := s.client.GetCaptures(ctx) - c.Assert(err, check.IsNil) - captures := make(map[model.CaptureID]*model.CaptureInfo) - for _, c := range captureList { - captures[c.ID] = c - } - grpcPool := kv.NewGrpcPoolImpl(ctx, &security.Credential{}) - defer grpcPool.Close() - owner, err := NewOwner(ctx, nil, grpcPool, capture.session, - cdcGCSafePointTTL4Test, time.Millisecond*200) - c.Assert(err, check.IsNil) - // It is better to update changefeed information by `loadChangeFeeds`, however - // `loadChangeFeeds` is too overweight, just mock enough information here. - owner.changeFeeds = map[model.ChangeFeedID]*changeFeed{ - changefeed: { - id: changefeed, - orphanTables: make(map[model.TableID]model.Ts), - status: &model.ChangeFeedStatus{ - CheckpointTs: 100, - }, - }, - } - - // capture information is not built, owner.run does nothing - err = owner.run(ctx) - c.Assert(err, check.IsNil) - statuses, err := s.client.GetAllTaskStatus(ctx, changefeed) - c.Assert(err, check.IsNil) - // stale tasks are not cleaned up, since `cleanUpStaleTasks` does not run - c.Assert(len(statuses), check.Equals, 2) - c.Assert(len(owner.captures), check.Equals, 0) - - err = owner.rebuildCaptureEvents(ctx, captures) - c.Assert(err, check.IsNil) - c.Assert(len(owner.captures), check.Equals, 1) - c.Assert(owner.captures, check.HasKey, capture.info.ID) - c.Assert(owner.changeFeeds[changefeed].orphanTables, check.DeepEquals, map[model.TableID]model.Ts{51: 110}) - c.Assert(atomic.LoadInt32(&owner.captureLoaded), check.Equals, int32(1)) - // check stale tasks are cleaned up - statuses, err = s.client.GetAllTaskStatus(ctx, changefeed) - c.Assert(err, check.IsNil) - c.Assert(len(statuses), check.Equals, 1) - c.Assert(statuses, check.HasKey, capture.info.ID) - positions, err := s.client.GetAllTaskPositions(ctx, changefeed) - c.Assert(err, check.IsNil) - c.Assert(len(positions), check.Equals, 1) - c.Assert(positions, check.HasKey, capture.info.ID) - workloads, err := s.client.GetAllTaskWorkloads(ctx, changefeed) - c.Assert(err, check.IsNil) - c.Assert(len(workloads), check.Equals, 1) - c.Assert(workloads, check.HasKey, capture.info.ID) - - err = capture.etcdClient.Close() - c.Assert(err, check.IsNil) -} - -func (s *ownerSuite) TestWatchFeedChange(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - addr := "127.0.0.1:12034" - ctx = util.PutCaptureAddrInCtx(ctx, addr) - capture, err := NewCapture(ctx, []string{s.clientURL.String()}, nil, nil) - c.Assert(err, check.IsNil) - grpcPool := kv.NewGrpcPoolImpl(ctx, &security.Credential{}) - defer grpcPool.Close() - owner, err := NewOwner(ctx, nil, grpcPool, capture.session, - cdcGCSafePointTTL4Test, time.Millisecond*200) - c.Assert(err, check.IsNil) - - var ( - wg sync.WaitGroup - updateCount = 0 - recvChangeCount = 0 - ) - ctx1, cancel1 := context.WithCancel(ctx) - wg.Add(1) - go func() { - defer wg.Done() - changefeedID := "test-changefeed" - pos := &model.TaskPosition{CheckPointTs: 100, ResolvedTs: 102} - for { - select { - case <-ctx1.Done(): - return - default: - } - pos.ResolvedTs++ - pos.CheckPointTs++ - updated, err := capture.etcdClient.PutTaskPositionOnChange(ctx1, changefeedID, capture.info.ID, pos) - if errors.Cause(err) == context.Canceled { - return - } - c.Assert(err, check.IsNil) - c.Assert(updated, check.IsTrue) - updateCount++ - // sleep to avoid other goroutine starvation - time.Sleep(time.Millisecond) - } - }() - - feedChangeReceiver, err := owner.feedChangeNotifier.NewReceiver(ownerRunInterval) - c.Assert(err, check.IsNil) - defer feedChangeReceiver.Stop() - owner.watchFeedChange(ctx) - wg.Add(1) - go func() { - defer func() { - // there could be one message remaining in notification receiver, try to consume it - select { - case <-feedChangeReceiver.C: - default: - } - wg.Done() - }() - for { - select { - case <-ctx1.Done(): - return - case <-feedChangeReceiver.C: - recvChangeCount++ - // sleep to simulate some owner work - time.Sleep(time.Millisecond * 50) - } - } - }() - - time.Sleep(time.Second * 2) - // use cancel1 to avoid cancel the watchFeedChange - cancel1() - wg.Wait() - c.Assert(recvChangeCount, check.Greater, 0) - c.Assert(recvChangeCount, check.Less, updateCount) - select { - case <-feedChangeReceiver.C: - c.Error("should not receive message from feed change chan any more") - default: - } - - err = capture.etcdClient.Close() - if err != nil { - c.Assert(errors.Cause(err), check.Equals, context.Canceled) - } -} - -func (s *ownerSuite) TestWriteDebugInfo(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - owner := &Owner{ - changeFeeds: map[model.ChangeFeedID]*changeFeed{ - "test": { - id: "test", - info: &model.ChangeFeedInfo{ - SinkURI: "blackhole://", - Config: config.GetDefaultReplicaConfig(), - }, - status: &model.ChangeFeedStatus{ - ResolvedTs: 120, - CheckpointTs: 100, - }, - }, - }, - stoppedFeeds: map[model.ChangeFeedID]*model.ChangeFeedStatus{ - "test-2": { - ResolvedTs: 120, - CheckpointTs: 100, - }, - }, - captures: map[model.CaptureID]*model.CaptureInfo{ - "capture-1": { - ID: "capture-1", - AdvertiseAddr: "127.0.0.1:8301", - }, - }, - } - var buf bytes.Buffer - owner.writeDebugInfo(&buf) - c.Assert(buf.String(), check.Matches, `[\s\S]*active changefeeds[\s\S]*stopped changefeeds[\s\S]*captures[\s\S]*`) -} diff --git a/cdc/processor.go b/cdc/processor.go deleted file mode 100644 index d4305ea2faf..00000000000 --- a/cdc/processor.go +++ /dev/null @@ -1,1292 +0,0 @@ -// Copyright 2020 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 cdc - -import ( - "context" - "fmt" - "io" - "strconv" - "sync" - "sync/atomic" - "time" - - "github.com/cenkalti/backoff" - "github.com/google/uuid" - "github.com/pingcap/errors" - "github.com/pingcap/failpoint" - "github.com/pingcap/log" - 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/puller" - psorter "github.com/pingcap/tiflow/cdc/puller/sorter" - "github.com/pingcap/tiflow/cdc/sink" - cerror "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/filter" - "github.com/pingcap/tiflow/pkg/notify" - "github.com/pingcap/tiflow/pkg/regionspan" - "github.com/pingcap/tiflow/pkg/retry" - "github.com/pingcap/tiflow/pkg/util" - "github.com/tikv/client-go/v2/oracle" - pd "github.com/tikv/pd/client" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/clientv3/concurrency" - "go.etcd.io/etcd/mvcc" - "go.uber.org/zap" - "golang.org/x/sync/errgroup" -) - -const ( - defaultSyncResolvedBatch = 1024 - - schemaStorageGCLag = time.Minute * 20 - - maxTries = 3 -) - -type oldProcessor struct { - id string - captureInfo model.CaptureInfo - changefeedID string - changefeed model.ChangeFeedInfo - stopped int32 - - pdCli pd.Client - etcdCli kv.CDCEtcdClient - grpcPool kv.GrpcPool - session *concurrency.Session - - sinkManager *sink.Manager - - globalResolvedTs uint64 - localResolvedTs uint64 - checkpointTs uint64 - globalCheckpointTs uint64 - appliedLocalCheckpointTs uint64 - flushCheckpointInterval time.Duration - - ddlPuller puller.Puller - ddlPullerCancel context.CancelFunc - schemaStorage entry.SchemaStorage - - mounter entry.Mounter - - stateMu sync.Mutex - status *model.TaskStatus - position *model.TaskPosition - tables map[int64]*tableInfo - markTableIDs map[int64]struct{} - statusModRevision int64 - - globalResolvedTsNotifier *notify.Notifier - localResolvedNotifier *notify.Notifier - localResolvedReceiver *notify.Receiver - localCheckpointTsNotifier *notify.Notifier - localCheckpointTsReceiver *notify.Receiver - - wg *errgroup.Group - errCh chan<- error - opDoneCh chan int64 -} - -type tableInfo struct { - id int64 - name string // quoted schema and table, used in metircs only - resolvedTs uint64 - checkpointTs uint64 - - markTableID int64 - mResolvedTs uint64 - mCheckpointTs uint64 - workload model.WorkloadInfo - cancel context.CancelFunc -} - -func (t *tableInfo) loadResolvedTs() uint64 { - tableRts := atomic.LoadUint64(&t.resolvedTs) - if t.markTableID != 0 { - mTableRts := atomic.LoadUint64(&t.mResolvedTs) - if mTableRts < tableRts { - return mTableRts - } - } - return tableRts -} - -func (t *tableInfo) loadCheckpointTs() uint64 { - tableCkpt := atomic.LoadUint64(&t.checkpointTs) - if t.markTableID != 0 { - mTableCkpt := atomic.LoadUint64(&t.mCheckpointTs) - if mTableCkpt < tableCkpt { - return mTableCkpt - } - } - return tableCkpt -} - -// newProcessor creates and returns a processor for the specified change feed -func newProcessor( - ctx context.Context, - pdCli pd.Client, - grpcPool kv.GrpcPool, - session *concurrency.Session, - changefeed model.ChangeFeedInfo, - sinkManager *sink.Manager, - changefeedID string, - captureInfo model.CaptureInfo, - checkpointTs uint64, - errCh chan error, - flushCheckpointInterval time.Duration, -) (*oldProcessor, error) { - etcdCli := session.Client() - cdcEtcdCli := kv.NewCDCEtcdClient(ctx, etcdCli) - - log.Info("start processor with startts", - zap.Uint64("startts", checkpointTs), util.ZapFieldChangefeed(ctx)) - kvStorage, err := util.KVStorageFromCtx(ctx) - if err != nil { - return nil, errors.Trace(err) - } - ddlspans := []regionspan.Span{regionspan.GetDDLSpan(), regionspan.GetAddIndexDDLSpan()} - ddlPuller := puller.NewPuller(ctx, pdCli, grpcPool, kvStorage, checkpointTs, ddlspans, false) - filter, err := filter.NewFilter(changefeed.Config) - if err != nil { - return nil, errors.Trace(err) - } - schemaStorage, err := createSchemaStorage(kvStorage, checkpointTs, filter, changefeed.Config.ForceReplicate) - if err != nil { - return nil, errors.Trace(err) - } - - localResolvedNotifier := new(notify.Notifier) - localCheckpointTsNotifier := new(notify.Notifier) - globalResolvedTsNotifier := new(notify.Notifier) - localResolvedReceiver, err := localResolvedNotifier.NewReceiver(50 * time.Millisecond) - if err != nil { - return nil, err - } - localCheckpointTsReceiver, err := localCheckpointTsNotifier.NewReceiver(50 * time.Millisecond) - if err != nil { - return nil, err - } - - p := &oldProcessor{ - id: uuid.New().String(), - captureInfo: captureInfo, - changefeedID: changefeedID, - changefeed: changefeed, - pdCli: pdCli, - grpcPool: grpcPool, - etcdCli: cdcEtcdCli, - session: session, - sinkManager: sinkManager, - ddlPuller: ddlPuller, - mounter: entry.NewMounter(schemaStorage, changefeed.Config.Mounter.WorkerNum, changefeed.Config.EnableOldValue), - schemaStorage: schemaStorage, - errCh: errCh, - - flushCheckpointInterval: flushCheckpointInterval, - - position: &model.TaskPosition{CheckPointTs: checkpointTs}, - - globalResolvedTsNotifier: globalResolvedTsNotifier, - localResolvedNotifier: localResolvedNotifier, - localResolvedReceiver: localResolvedReceiver, - - checkpointTs: checkpointTs, - localCheckpointTsNotifier: localCheckpointTsNotifier, - localCheckpointTsReceiver: localCheckpointTsReceiver, - - tables: make(map[int64]*tableInfo), - markTableIDs: make(map[int64]struct{}), - - opDoneCh: make(chan int64, 256), - } - modRevision, status, err := p.etcdCli.GetTaskStatus(ctx, p.changefeedID, p.captureInfo.ID) - if err != nil { - return nil, errors.Trace(err) - } - p.status = status - p.statusModRevision = modRevision - - info, _, err := p.etcdCli.GetChangeFeedStatus(ctx, p.changefeedID) - if err != nil && cerror.ErrChangeFeedNotExists.NotEqual(err) { - return nil, errors.Trace(err) - } - - if err == nil { - p.globalCheckpointTs = info.CheckpointTs - } - - for tableID, replicaInfo := range p.status.Tables { - p.addTable(ctx, tableID, replicaInfo) - } - return p, nil -} - -func (p *oldProcessor) Run(ctx context.Context) { - wg, cctx := errgroup.WithContext(ctx) - p.wg = wg - ddlPullerCtx, ddlPullerCancel := - context.WithCancel(util.PutTableInfoInCtx(cctx, 0, "ticdc-processor-ddl")) - p.ddlPullerCancel = ddlPullerCancel - - wg.Go(func() error { - return p.positionWorker(cctx) - }) - - wg.Go(func() error { - return p.globalStatusWorker(cctx) - }) - - wg.Go(func() error { - return p.ddlPuller.Run(ddlPullerCtx) - }) - - wg.Go(func() error { - return p.ddlPullWorker(cctx) - }) - - wg.Go(func() error { - return p.mounter.Run(cctx) - }) - - wg.Go(func() error { - return p.workloadWorker(cctx) - }) - - go func() { - if err := wg.Wait(); err != nil { - p.sendError(err) - } - }() -} - -// wait blocks until all routines in processor are returned -func (p *oldProcessor) wait() { - err := p.wg.Wait() - if err != nil && errors.Cause(err) != context.Canceled { - log.Error("processor wait error", - zap.String("capture-id", p.captureInfo.ID), - zap.String("capture", p.captureInfo.AdvertiseAddr), - zap.String("changefeed", p.changefeedID), - zap.Error(err), - ) - } -} - -func (p *oldProcessor) writeDebugInfo(w io.Writer) { - fmt.Fprintf(w, "changefeedID:\n\t%s\ninfo:\n\t%s\nstatus:\n\t%+v\nposition:\n\t%s\n", - p.changefeedID, p.changefeed.String(), p.status, p.position.String()) - - fmt.Fprintf(w, "tables:\n") - p.stateMu.Lock() - for _, table := range p.tables { - fmt.Fprintf(w, "\ttable id: %d, resolveTS: %d\n", table.id, table.loadResolvedTs()) - } - p.stateMu.Unlock() -} - -// localResolvedWorker do the flowing works. -// 1, update resolve ts by scanning all table's resolve ts. -// 2, update checkpoint ts by consuming entry from p.executedTxns. -// 3, sync TaskStatus between in memory and storage. -// 4, check admin command in TaskStatus and apply corresponding command -func (p *oldProcessor) positionWorker(ctx context.Context) error { - lastFlushTime := time.Now() - retryFlushTaskStatusAndPosition := func() error { - t0Update := time.Now() - err := retry.Do(ctx, func() error { - inErr := p.flushTaskStatusAndPosition(ctx) - if inErr != nil { - if errors.Cause(inErr) != context.Canceled { - logError := log.Error - errField := zap.Error(inErr) - if cerror.ErrAdminStopProcessor.Equal(inErr) { - logError = log.Warn - errField = zap.String("error", inErr.Error()) - } - logError("update info failed", util.ZapFieldChangefeed(ctx), errField) - } - if p.isStopped() || cerror.ErrAdminStopProcessor.Equal(inErr) { - return cerror.ErrAdminStopProcessor.FastGenByArgs() - } - } - return inErr - }, retry.WithBackoffBaseDelay(500), retry.WithMaxTries(maxTries), retry.WithIsRetryableErr(isRetryable)) - updateInfoDuration. - WithLabelValues(p.captureInfo.AdvertiseAddr). - Observe(time.Since(t0Update).Seconds()) - if err != nil { - return errors.Annotate(err, "failed to update info") - } - return nil - } - - defer func() { - p.localResolvedReceiver.Stop() - p.localCheckpointTsReceiver.Stop() - - if !p.isStopped() { - err := retryFlushTaskStatusAndPosition() - if err != nil && errors.Cause(err) != context.Canceled { - log.Warn("failed to update info before exit", util.ZapFieldChangefeed(ctx), zap.Error(err)) - } - } - - log.Info("Local resolved worker exited", util.ZapFieldChangefeed(ctx)) - }() - - resolvedTsGauge := resolvedTsGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - metricResolvedTsLagGauge := resolvedTsLagGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - checkpointTsGauge := checkpointTsGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - metricCheckpointTsLagGauge := checkpointTsLagGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - for { - select { - case <-ctx.Done(): - return ctx.Err() - case <-p.localResolvedReceiver.C: - minResolvedTs := p.ddlPuller.GetResolvedTs() - p.stateMu.Lock() - for _, table := range p.tables { - ts := table.loadResolvedTs() - - if ts < minResolvedTs { - minResolvedTs = ts - } - } - p.stateMu.Unlock() - atomic.StoreUint64(&p.localResolvedTs, minResolvedTs) - - phyTs := oracle.ExtractPhysical(minResolvedTs) - // It is more accurate to get tso from PD, but in most cases we have - // deployed NTP service, a little bias is acceptable here. - metricResolvedTsLagGauge.Set(float64(oracle.GetPhysical(time.Now())-phyTs) / 1e3) - resolvedTsGauge.Set(float64(phyTs)) - - if p.position.ResolvedTs < minResolvedTs { - p.position.ResolvedTs = minResolvedTs - if err := retryFlushTaskStatusAndPosition(); err != nil { - return errors.Trace(err) - } - } - case <-p.localCheckpointTsReceiver.C: - checkpointTs := atomic.LoadUint64(&p.globalResolvedTs) - p.stateMu.Lock() - for _, table := range p.tables { - ts := table.loadCheckpointTs() - if ts < checkpointTs { - checkpointTs = ts - } - } - p.stateMu.Unlock() - if checkpointTs == 0 { - log.Debug("0 is not a valid checkpointTs", util.ZapFieldChangefeed(ctx)) - continue - } - atomic.StoreUint64(&p.checkpointTs, checkpointTs) - phyTs := oracle.ExtractPhysical(checkpointTs) - // It is more accurate to get tso from PD, but in most cases we have - // deployed NTP service, a little bias is acceptable here. - metricCheckpointTsLagGauge.Set(float64(oracle.GetPhysical(time.Now())-phyTs) / 1e3) - - if time.Since(lastFlushTime) < p.flushCheckpointInterval { - continue - } - - p.position.CheckPointTs = checkpointTs - checkpointTsGauge.Set(float64(phyTs)) - if err := retryFlushTaskStatusAndPosition(); err != nil { - return errors.Trace(err) - } - atomic.StoreUint64(&p.appliedLocalCheckpointTs, checkpointTs) - lastFlushTime = time.Now() - } - } -} - -func isRetryable(err error) bool { - return cerror.IsRetryableError(err) && cerror.ErrAdminStopProcessor.NotEqual(err) -} - -func (p *oldProcessor) ddlPullWorker(ctx context.Context) error { - ddlRawKVCh := puller.SortOutput(ctx, p.ddlPuller.Output()) - var ddlRawKV *model.RawKVEntry - for { - select { - case <-ctx.Done(): - return errors.Trace(ctx.Err()) - case ddlRawKV = <-ddlRawKVCh: - } - if ddlRawKV == nil { - continue - } - failpoint.Inject("processorDDLResolved", func() {}) - if ddlRawKV.OpType == model.OpTypeResolved { - p.schemaStorage.AdvanceResolvedTs(ddlRawKV.CRTs) - p.localResolvedNotifier.Notify() - } - job, err := entry.UnmarshalDDL(ddlRawKV) - if err != nil { - return errors.Trace(err) - } - if job == nil { - continue - } - if err := p.schemaStorage.HandleDDLJob(job); err != nil { - return errors.Trace(err) - } - } -} - -func (p *oldProcessor) workloadWorker(ctx context.Context) error { - t := time.NewTicker(10 * time.Second) - err := p.etcdCli.PutTaskWorkload(ctx, p.changefeedID, p.captureInfo.ID, nil) - if err != nil { - return errors.Trace(err) - } - for { - select { - case <-ctx.Done(): - return errors.Trace(ctx.Err()) - case <-t.C: - } - if p.isStopped() { - continue - } - p.stateMu.Lock() - workload := make(model.TaskWorkload, len(p.tables)) - for _, table := range p.tables { - workload[table.id] = table.workload - } - p.stateMu.Unlock() - err := p.etcdCli.PutTaskWorkload(ctx, p.changefeedID, p.captureInfo.ID, &workload) - if err != nil { - return errors.Trace(err) - } - } -} - -func (p *oldProcessor) flushTaskPosition(ctx context.Context) error { - failpoint.Inject("ProcessorUpdatePositionDelaying", func() { - time.Sleep(1 * time.Second) - }) - if p.isStopped() { - return cerror.ErrAdminStopProcessor.GenWithStackByArgs() - } - // p.position.Count = p.sink.Count() - updated, err := p.etcdCli.PutTaskPositionOnChange(ctx, p.changefeedID, p.captureInfo.ID, p.position) - if err != nil { - if errors.Cause(err) != context.Canceled { - log.Error("failed to flush task position", util.ZapFieldChangefeed(ctx), zap.Error(err)) - return errors.Trace(err) - } - } - if updated { - log.Debug("flushed task position", util.ZapFieldChangefeed(ctx), zap.Stringer("position", p.position)) - } - return nil -} - -// First try to synchronize task status from etcd. -// If local cached task status is outdated (caused by new table scheduling), -// update it to latest value, and force update task position, since add new -// tables may cause checkpoint ts fallback in processor. -func (p *oldProcessor) flushTaskStatusAndPosition(ctx context.Context) error { - if p.isStopped() { - return cerror.ErrAdminStopProcessor.GenWithStackByArgs() - } - var tablesToRemove []model.TableID - newTaskStatus, newModRevision, err := p.etcdCli.AtomicPutTaskStatus(ctx, p.changefeedID, p.captureInfo.ID, - func(modRevision int64, taskStatus *model.TaskStatus) (bool, error) { - // if the task status is not changed and not operation to handle - // we need not to change the task status - if p.statusModRevision == modRevision && !taskStatus.SomeOperationsUnapplied() { - return false, nil - } - // task will be stopped in capture task handler, do nothing - if taskStatus.AdminJobType.IsStopState() { - return false, backoff.Permanent(cerror.ErrAdminStopProcessor.GenWithStackByArgs()) - } - toRemove, err := p.handleTables(ctx, taskStatus) - tablesToRemove = append(tablesToRemove, toRemove...) - if err != nil { - return false, backoff.Permanent(errors.Trace(err)) - } - // processor reads latest task status from etcd, analyzes operation - // field and processes table add or delete. If operation is unapplied - // but stays unchanged after processor handling tables, it means no - // status is changed and we don't need to flush task status neigher. - if !taskStatus.Dirty { - return false, nil - } - err = p.flushTaskPosition(ctx) - return true, err - }) - if err != nil { - // not need to check error - //nolint:errcheck - p.flushTaskPosition(ctx) - return errors.Trace(err) - } - for _, tableID := range tablesToRemove { - p.removeTable(tableID) - } - // newModRevision == 0 means status is not updated - if newModRevision > 0 { - p.statusModRevision = newModRevision - p.status = newTaskStatus - } - syncTableNumGauge. - WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr). - Set(float64(len(p.status.Tables))) - - return p.flushTaskPosition(ctx) -} - -func (p *oldProcessor) removeTable(tableID int64) { - p.stateMu.Lock() - defer p.stateMu.Unlock() - - log.Debug("remove table", zap.String("changefeed", p.changefeedID), zap.Int64("id", tableID)) - - table, ok := p.tables[tableID] - if !ok { - log.Warn("table not found", zap.String("changefeed", p.changefeedID), zap.Int64("tableID", tableID)) - return - } - - table.cancel() - delete(p.tables, tableID) - if table.markTableID != 0 { - delete(p.markTableIDs, table.markTableID) - } - tableResolvedTsGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - syncTableNumGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr).Dec() -} - -// handleTables handles table scheduler on this processor, add or remove table puller -func (p *oldProcessor) handleTables(ctx context.Context, status *model.TaskStatus) (tablesToRemove []model.TableID, err error) { - for tableID, opt := range status.Operation { - if opt.TableProcessed() { - continue - } - if opt.Delete { - if opt.BoundaryTs <= p.position.CheckPointTs { - if opt.BoundaryTs != p.position.CheckPointTs { - log.Warn("the replication progresses beyond the BoundaryTs and duplicate data may be received by downstream", - zap.Uint64("local resolved TS", p.position.ResolvedTs), zap.Any("opt", opt)) - } - table, exist := p.tables[tableID] - if !exist { - log.Warn("table which will be deleted is not found", - util.ZapFieldChangefeed(ctx), zap.Int64("tableID", tableID)) - opt.Done = true - opt.Status = model.OperFinished - status.Dirty = true - continue - } - table.cancel() - checkpointTs := table.loadCheckpointTs() - log.Debug("stop table", zap.Int64("tableID", tableID), - util.ZapFieldChangefeed(ctx), - zap.Any("opt", opt), - zap.Uint64("checkpointTs", checkpointTs)) - opt.BoundaryTs = checkpointTs - tablesToRemove = append(tablesToRemove, tableID) - opt.Done = true - opt.Status = model.OperFinished - status.Dirty = true - tableResolvedTsGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - } - } else { - replicaInfo, exist := status.Tables[tableID] - if !exist { - return tablesToRemove, cerror.ErrProcessorTableNotFound.GenWithStack("replicaInfo of table(%d)", tableID) - } - if p.changefeed.Config.Cyclic.IsEnabled() && replicaInfo.MarkTableID == 0 { - return tablesToRemove, cerror.ErrProcessorTableNotFound.GenWithStack("normal table(%d) and mark table not match ", tableID) - } - p.addTable(ctx, tableID, replicaInfo) - opt.Status = model.OperProcessed - status.Dirty = true - } - } - - for { - select { - case <-ctx.Done(): - return nil, ctx.Err() - case tableID := <-p.opDoneCh: - log.Debug("Operation done signal received", - util.ZapFieldChangefeed(ctx), - zap.Int64("tableID", tableID), - zap.Reflect("operation", status.Operation[tableID])) - if status.Operation[tableID] == nil { - log.Debug("TableID does not exist, probably a mark table, ignore", - util.ZapFieldChangefeed(ctx), zap.Int64("tableID", tableID)) - continue - } - status.Operation[tableID].Done = true - status.Operation[tableID].Status = model.OperFinished - status.Dirty = true - default: - goto done - } - } -done: - if !status.SomeOperationsUnapplied() { - status.Operation = nil - // status.Dirty must be true when status changes from `unapplied` to `applied`, - // setting status.Dirty = true is not **must** here. - status.Dirty = true - } - return tablesToRemove, nil -} - -// globalStatusWorker read global resolve ts from changefeed level info and forward `tableInputChans` regularly. -func (p *oldProcessor) globalStatusWorker(ctx context.Context) error { - log.Info("Global status worker started", util.ZapFieldChangefeed(ctx)) - - var ( - changefeedStatus *model.ChangeFeedStatus - statusRev int64 - lastCheckPointTs uint64 - lastResolvedTs uint64 - watchKey = kv.GetEtcdKeyJob(p.changefeedID) - ) - - updateStatus := func(changefeedStatus *model.ChangeFeedStatus) { - atomic.StoreUint64(&p.globalCheckpointTs, changefeedStatus.CheckpointTs) - if lastResolvedTs == changefeedStatus.ResolvedTs && - lastCheckPointTs == changefeedStatus.CheckpointTs { - return - } - if lastCheckPointTs < changefeedStatus.CheckpointTs { - // Delay GC to accommodate pullers starting from a startTs that's too small - // TODO fix startTs problem and remove GC delay, or use other mechanism that prevents the problem deterministically - gcTime := oracle.GetTimeFromTS(changefeedStatus.CheckpointTs).Add(-schemaStorageGCLag) - gcTs := oracle.ComposeTS(gcTime.Unix(), 0) - p.schemaStorage.DoGC(gcTs) - lastCheckPointTs = changefeedStatus.CheckpointTs - } - if lastResolvedTs < changefeedStatus.ResolvedTs { - lastResolvedTs = changefeedStatus.ResolvedTs - atomic.StoreUint64(&p.globalResolvedTs, lastResolvedTs) - log.Debug("Update globalResolvedTs", - zap.Uint64("globalResolvedTs", lastResolvedTs), util.ZapFieldChangefeed(ctx)) - p.globalResolvedTsNotifier.Notify() - } - } - - retryCfg := backoff.WithMaxRetries( - backoff.WithContext( - backoff.NewExponentialBackOff(), ctx), - 5, - ) - for { - select { - case <-ctx.Done(): - log.Info("Global resolved worker exited", util.ZapFieldChangefeed(ctx)) - return ctx.Err() - default: - } - - err := backoff.Retry(func() error { - var err error - changefeedStatus, statusRev, err = p.etcdCli.GetChangeFeedStatus(ctx, p.changefeedID) - if err != nil { - if errors.Cause(err) == context.Canceled { - return backoff.Permanent(err) - } - log.Error("Global resolved worker: read global resolved ts failed", - util.ZapFieldChangefeed(ctx), zap.Error(err)) - } - return err - }, retryCfg) - if err != nil { - return errors.Trace(err) - } - - updateStatus(changefeedStatus) - - ch := p.etcdCli.Client.Watch(ctx, watchKey, clientv3.WithRev(statusRev+1), clientv3.WithFilterDelete()) - for resp := range ch { - if resp.Err() == mvcc.ErrCompacted { - break - } - if resp.Err() != nil { - return cerror.WrapError(cerror.ErrProcessorEtcdWatch, err) - } - for _, ev := range resp.Events { - var status model.ChangeFeedStatus - if err := status.Unmarshal(ev.Kv.Value); err != nil { - return err - } - updateStatus(&status) - } - } - } -} - -func createSchemaStorage( - kvStorage tidbkv.Storage, - checkpointTs uint64, - filter *filter.Filter, - forceReplicate bool, -) (entry.SchemaStorage, error) { - meta, err := kv.GetSnapshotMeta(kvStorage, checkpointTs) - if err != nil { - return nil, errors.Trace(err) - } - return entry.NewSchemaStorage(meta, checkpointTs, filter, forceReplicate) -} - -func (p *oldProcessor) addTable(ctx context.Context, tableID int64, replicaInfo *model.TableReplicaInfo) { - p.stateMu.Lock() - defer p.stateMu.Unlock() - - var tableName string - - err := retry.Do(ctx, func() error { - if name, ok := p.schemaStorage.GetLastSnapshot().GetTableNameByID(tableID); ok { - tableName = name.QuoteString() - return nil - } - return errors.Errorf("failed to get table name, fallback to use table id: %d", tableID) - }, retry.WithBackoffBaseDelay(5), retry.WithMaxTries(maxTries), retry.WithIsRetryableErr(cerror.IsRetryableError)) - if err != nil { - log.Warn("get table name for metric", util.ZapFieldChangefeed(ctx), zap.String("error", err.Error())) - tableName = strconv.Itoa(int(tableID)) - } - - if _, ok := p.tables[tableID]; ok { - log.Warn("Ignore existing table", util.ZapFieldChangefeed(ctx), zap.Int64("ID", tableID)) - return - } - - globalCheckpointTs := atomic.LoadUint64(&p.globalCheckpointTs) - - if replicaInfo.StartTs < globalCheckpointTs { - // use Warn instead of Panic in case that p.globalCheckpointTs has not been initialized. - // The cdc_state_checker will catch a real inconsistency in integration tests. - log.Warn("addTable: startTs < checkpoint", - util.ZapFieldChangefeed(ctx), - zap.Int64("tableID", tableID), - zap.Uint64("checkpoint", globalCheckpointTs), - zap.Uint64("startTs", replicaInfo.StartTs)) - } - - globalResolvedTs := atomic.LoadUint64(&p.globalResolvedTs) - log.Debug("Add table", zap.Int64("tableID", tableID), - util.ZapFieldChangefeed(ctx), - zap.String("name", tableName), - zap.Any("replicaInfo", replicaInfo), - zap.Uint64("globalResolvedTs", globalResolvedTs)) - - ctx = util.PutTableInfoInCtx(ctx, tableID, tableName) - ctx, cancel := context.WithCancel(ctx) - table := &tableInfo{ - id: tableID, - name: tableName, - resolvedTs: replicaInfo.StartTs, - } - // TODO(leoppro) calculate the workload of this table - // We temporarily set the value to constant 1 - table.workload = model.WorkloadInfo{Workload: 1} - - startPuller := func(tableID model.TableID, pResolvedTs *uint64, pCheckpointTs *uint64) sink.Sink { - // start table puller - span := regionspan.GetTableSpan(tableID) - kvStorage, err := util.KVStorageFromCtx(ctx) - if err != nil { - p.sendError(err) - return nil - } - // NOTICE: always pull the old value internally - // See also: TODO(hi-rustin): add issue link here. - plr := puller.NewPuller(ctx, p.pdCli, p.grpcPool, kvStorage, - replicaInfo.StartTs, []regionspan.Span{span}, - true) - go func() { - err := plr.Run(ctx) - if errors.Cause(err) != context.Canceled { - p.sendError(err) - } - }() - - var sorter puller.EventSorter - switch p.changefeed.Engine { - case model.SortInMemory: - sorter = puller.NewEntrySorter() - case model.SortUnified, model.SortInFile /* `file` becomes an alias of `unified` for backward compatibility */ : - if p.changefeed.Engine == model.SortInFile { - log.Warn("File sorter is obsolete. Please revise your changefeed settings and use unified sorter", - util.ZapFieldChangefeed(ctx)) - } - err := psorter.UnifiedSorterCheckDir(p.changefeed.SortDir) - if err != nil { - p.sendError(errors.Trace(err)) - return nil - } - sorter, err = psorter.NewUnifiedSorter(p.changefeed.SortDir, p.changefeedID, tableName, tableID, util.CaptureAddrFromCtx(ctx)) - if err != nil { - p.sendError(errors.Trace(err)) - return nil - } - default: - p.sendError(cerror.ErrUnknownSortEngine.GenWithStackByArgs(p.changefeed.Engine)) - return nil - } - failpoint.Inject("ProcessorAddTableError", func() { - p.sendError(errors.New("processor add table injected error")) - failpoint.Return(nil) - }) - go func() { - err := sorter.Run(ctx) - if errors.Cause(err) != context.Canceled { - p.sendError(err) - } - }() - - go func() { - p.pullerConsume(ctx, plr, sorter) - }() - - tableSink := p.sinkManager.CreateTableSink(tableID, replicaInfo.StartTs) - go func() { - p.sorterConsume(ctx, tableID, sorter, pResolvedTs, pCheckpointTs, replicaInfo, tableSink) - }() - return tableSink - } - var tableSink, mTableSink sink.Sink - if p.changefeed.Config.Cyclic.IsEnabled() && replicaInfo.MarkTableID != 0 { - mTableID := replicaInfo.MarkTableID - // we should to make sure a mark table is only listened once. - if _, exist := p.markTableIDs[mTableID]; !exist { - p.markTableIDs[mTableID] = struct{}{} - table.markTableID = mTableID - table.mResolvedTs = replicaInfo.StartTs - - mTableSink = startPuller(mTableID, &table.mResolvedTs, &table.mCheckpointTs) - } - } - - p.tables[tableID] = table - if p.position.CheckPointTs > replicaInfo.StartTs { - p.position.CheckPointTs = replicaInfo.StartTs - } - if p.position.ResolvedTs > replicaInfo.StartTs { - p.position.ResolvedTs = replicaInfo.StartTs - } - - atomic.StoreUint64(&p.localResolvedTs, p.position.ResolvedTs) - tableSink = startPuller(tableID, &table.resolvedTs, &table.checkpointTs) - table.cancel = func() { - cancel() - if tableSink != nil { - tableSink.Close(ctx) - } - if mTableSink != nil { - mTableSink.Close(ctx) - } - } - syncTableNumGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr).Inc() -} - -const maxLagWithCheckpointTs = (30 * 1000) << 18 // 30s - -// sorterConsume receives sorted PolymorphicEvent from sorter of each table and -// sends to processor's output chan -func (p *oldProcessor) sorterConsume( - ctx context.Context, - tableID int64, - sorter puller.EventSorter, - pResolvedTs *uint64, - pCheckpointTs *uint64, - replicaInfo *model.TableReplicaInfo, - sink sink.Sink, -) { - var lastResolvedTs, lastCheckPointTs uint64 - opDone := false - resolvedGauge := tableResolvedTsGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - checkDoneTicker := time.NewTicker(1 * time.Second) - checkDone := func() { - localResolvedTs := atomic.LoadUint64(&p.localResolvedTs) - globalResolvedTs := atomic.LoadUint64(&p.globalResolvedTs) - tableCheckPointTs := atomic.LoadUint64(pCheckpointTs) - localCheckpoint := atomic.LoadUint64(&p.appliedLocalCheckpointTs) - - if !opDone && lastResolvedTs >= localResolvedTs && localResolvedTs >= globalResolvedTs && - tableCheckPointTs >= localCheckpoint { - - log.Debug("localResolvedTs >= globalResolvedTs, sending operation done signal", - zap.Uint64("localResolvedTs", localResolvedTs), zap.Uint64("globalResolvedTs", globalResolvedTs), - zap.Int64("tableID", tableID), util.ZapFieldChangefeed(ctx)) - - opDone = true - checkDoneTicker.Stop() - select { - case <-ctx.Done(): - if errors.Cause(ctx.Err()) != context.Canceled { - p.sendError(ctx.Err()) - } - return - case p.opDoneCh <- tableID: - } - } - if !opDone { - log.Debug("addTable not done", - util.ZapFieldChangefeed(ctx), - zap.Uint64("tableResolvedTs", lastResolvedTs), - zap.Uint64("localResolvedTs", localResolvedTs), - zap.Uint64("globalResolvedTs", globalResolvedTs), - zap.Uint64("tableCheckpointTs", tableCheckPointTs), - zap.Uint64("localCheckpointTs", localCheckpoint), - zap.Int64("tableID", tableID)) - } - } - - events := make([]*model.PolymorphicEvent, 0, defaultSyncResolvedBatch) - rows := make([]*model.RowChangedEvent, 0, defaultSyncResolvedBatch) - - flushRowChangedEvents := func() error { - for _, ev := range events { - err := ev.WaitPrepare(ctx) - if err != nil { - return errors.Trace(err) - } - if ev.Row == nil { - continue - } - rows = append(rows, ev.Row) - } - failpoint.Inject("ProcessorSyncResolvedPreEmit", func() { - log.Info("Prepare to panic for ProcessorSyncResolvedPreEmit") - time.Sleep(10 * time.Second) - panic("ProcessorSyncResolvedPreEmit") - }) - err := sink.EmitRowChangedEvents(ctx, rows...) - if err != nil { - return errors.Trace(err) - } - events = events[:0] - rows = rows[:0] - return nil - } - - processRowChangedEvent := func(row *model.PolymorphicEvent) error { - events = append(events, row) - - if len(events) >= defaultSyncResolvedBatch { - err := flushRowChangedEvents() - if err != nil { - return errors.Trace(err) - } - } - return nil - } - - globalResolvedTsReceiver, err := p.globalResolvedTsNotifier.NewReceiver(500 * time.Millisecond) - if err != nil { - if errors.Cause(err) != context.Canceled { - p.errCh <- errors.Trace(err) - } - return - } - defer globalResolvedTsReceiver.Stop() - - sendResolvedTs2Sink := func() error { - localResolvedTs := atomic.LoadUint64(&p.localResolvedTs) - globalResolvedTs := atomic.LoadUint64(&p.globalResolvedTs) - var minTs uint64 - if localResolvedTs < globalResolvedTs { - minTs = localResolvedTs - log.Warn("the local resolved ts is less than the global resolved ts", - zap.Uint64("localResolvedTs", localResolvedTs), zap.Uint64("globalResolvedTs", globalResolvedTs)) - } else { - minTs = globalResolvedTs - } - if minTs == 0 { - return nil - } - - checkpointTs, err := sink.FlushRowChangedEvents(ctx, minTs) - if err != nil { - if errors.Cause(err) != context.Canceled { - p.sendError(errors.Trace(err)) - } - return err - } - lastCheckPointTs = checkpointTs - - if checkpointTs < replicaInfo.StartTs { - checkpointTs = replicaInfo.StartTs - } - - if checkpointTs != 0 { - atomic.StoreUint64(pCheckpointTs, checkpointTs) - p.localCheckpointTsNotifier.Notify() - } - return nil - } - for { - select { - case <-ctx.Done(): - if errors.Cause(ctx.Err()) != context.Canceled { - p.sendError(ctx.Err()) - } - return - case pEvent := <-sorter.Output(): - if pEvent == nil { - continue - } - - for lastResolvedTs > maxLagWithCheckpointTs+lastCheckPointTs { - log.Debug("the lag between local checkpoint Ts and local resolved Ts is too lang", - zap.Uint64("resolvedTs", lastResolvedTs), zap.Uint64("lastCheckPointTs", lastCheckPointTs), - zap.Int64("tableID", tableID), util.ZapFieldChangefeed(ctx)) - select { - case <-ctx.Done(): - if ctx.Err() != context.Canceled { - p.sendError(errors.Trace(ctx.Err())) - } - return - case <-globalResolvedTsReceiver.C: - if err := sendResolvedTs2Sink(); err != nil { - // error is already sent to processor, so we can just ignore it - return - } - case <-checkDoneTicker.C: - if !opDone { - checkDone() - } - } - } - - pEvent.SetUpFinishedChan() - select { - case <-ctx.Done(): - if errors.Cause(ctx.Err()) != context.Canceled { - p.sendError(ctx.Err()) - } - return - case p.mounter.Input() <- pEvent: - } - - if pEvent.RawKV != nil && pEvent.RawKV.OpType == model.OpTypeResolved { - if pEvent.CRTs == 0 { - continue - } - err := flushRowChangedEvents() - if err != nil { - if errors.Cause(err) != context.Canceled { - p.errCh <- errors.Trace(err) - } - return - } - atomic.StoreUint64(pResolvedTs, pEvent.CRTs) - lastResolvedTs = pEvent.CRTs - p.localResolvedNotifier.Notify() - resolvedGauge.Set(float64(oracle.ExtractPhysical(pEvent.CRTs))) - if !opDone { - checkDone() - } - continue - } - if pEvent.CRTs <= lastResolvedTs || pEvent.CRTs < replicaInfo.StartTs { - log.Panic("The CRTs of event is not expected, please report a bug", - util.ZapFieldChangefeed(ctx), - zap.String("model", "sorter"), - zap.Uint64("resolvedTs", lastResolvedTs), - zap.Int64("tableID", tableID), - zap.Any("replicaInfo", replicaInfo), - zap.Any("row", pEvent)) - } - failpoint.Inject("ProcessorSyncResolvedError", func() { - p.errCh <- errors.New("processor sync resolved injected error") - failpoint.Return() - }) - err := processRowChangedEvent(pEvent) - if err != nil { - if errors.Cause(err) != context.Canceled { - p.sendError(ctx.Err()) - } - return - } - case <-globalResolvedTsReceiver.C: - if err := sendResolvedTs2Sink(); err != nil { - // error is already sent to processor, so we can just ignore it - return - } - case <-checkDoneTicker.C: - if !opDone { - checkDone() - } - } - } -} - -// pullerConsume receives RawKVEntry from a given puller and sends to sorter -// for data sorting and mounter for data encode -func (p *oldProcessor) pullerConsume( - ctx context.Context, - plr puller.Puller, - sorter puller.EventSorter, -) { - for { - select { - case <-ctx.Done(): - if errors.Cause(ctx.Err()) != context.Canceled { - p.sendError(ctx.Err()) - } - return - case rawKV := <-plr.Output(): - if rawKV == nil { - continue - } - pEvent := model.NewPolymorphicEvent(rawKV) - sorter.AddEntry(ctx, pEvent) - } - } -} - -func (p *oldProcessor) stop(ctx context.Context) error { - log.Info("stop processor", zap.String("id", p.id), zap.String("capture", p.captureInfo.AdvertiseAddr), zap.String("changefeed", p.changefeedID)) - p.stateMu.Lock() - for _, tbl := range p.tables { - tbl.cancel() - tableResolvedTsGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) - } - p.ddlPullerCancel() - // mark tables share the same context with its original table, don't need to cancel - p.stateMu.Unlock() - p.globalResolvedTsNotifier.Close() - p.localCheckpointTsNotifier.Close() - p.localResolvedNotifier.Close() - failpoint.Inject("processorStopDelay", nil) - atomic.StoreInt32(&p.stopped, 1) - syncTableNumGauge.WithLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr).Set(0) - if err := p.etcdCli.DeleteTaskPosition(ctx, p.changefeedID, p.captureInfo.ID); err != nil { - return err - } - if err := p.etcdCli.DeleteTaskStatus(ctx, p.changefeedID, p.captureInfo.ID); err != nil { - return err - } - if err := p.etcdCli.DeleteTaskWorkload(ctx, p.changefeedID, p.captureInfo.ID); err != nil { - return err - } - return p.sinkManager.Close(ctx) -} - -func (p *oldProcessor) isStopped() bool { - return atomic.LoadInt32(&p.stopped) == 1 -} - -var runProcessorImpl = runProcessor - -// runProcessor creates a new processor then starts it. -func runProcessor( - ctx context.Context, - pdCli pd.Client, - grpcPool kv.GrpcPool, - session *concurrency.Session, - info model.ChangeFeedInfo, - changefeedID string, - captureInfo model.CaptureInfo, - checkpointTs uint64, - flushCheckpointInterval time.Duration, -) (*oldProcessor, error) { - opts := make(map[string]string, len(info.Opts)+2) - for k, v := range info.Opts { - opts[k] = v - } - opts[sink.OptChangefeedID] = changefeedID - opts[sink.OptCaptureAddr] = captureInfo.AdvertiseAddr - ctx = util.PutChangefeedIDInCtx(ctx, changefeedID) - filter, err := filter.NewFilter(info.Config) - if err != nil { - return nil, errors.Trace(err) - } - ctx, cancel := context.WithCancel(ctx) - // processor only receives one error from the channel, all producers to this - // channel must use the non-blocking way to send error. - errCh := make(chan error, 1) - s, err := sink.NewSink(ctx, changefeedID, info.SinkURI, filter, info.Config, opts, errCh) - if err != nil { - cancel() - return nil, errors.Trace(err) - } - sinkManager := sink.NewManager(ctx, s, errCh, checkpointTs, captureInfo.AdvertiseAddr, changefeedID) - processor, err := newProcessor(ctx, pdCli, grpcPool, session, info, sinkManager, - changefeedID, captureInfo, checkpointTs, errCh, flushCheckpointInterval) - if err != nil { - cancel() - return nil, err - } - log.Info("start to run processor", zap.String("changefeed", changefeedID), zap.String("processor", processor.id)) - - processorErrorCounter.WithLabelValues(changefeedID, captureInfo.AdvertiseAddr).Add(0) - processor.Run(ctx) - - go func() { - err := <-errCh - cancel() - processor.wait() - cause := errors.Cause(err) - if cause != nil && cause != context.Canceled && cerror.ErrAdminStopProcessor.NotEqual(cause) { - processorErrorCounter.WithLabelValues(changefeedID, captureInfo.AdvertiseAddr).Inc() - log.Error("error on running processor", - util.ZapFieldCapture(ctx), - zap.String("changefeed", changefeedID), - zap.String("processor", processor.id), - zap.Error(err)) - // record error information in etcd - var code string - if terror, ok := err.(*errors.Error); ok { - code = string(terror.RFCCode()) - } else { - code = string(cerror.ErrProcessorUnknown.RFCCode()) - } - processor.position.Error = &model.RunningError{ - Addr: captureInfo.AdvertiseAddr, - Code: code, - Message: err.Error(), - } - timeoutCtx, timeoutCancel := context.WithTimeout(context.Background(), 5*time.Second) - _, err = processor.etcdCli.PutTaskPositionOnChange(timeoutCtx, processor.changefeedID, processor.captureInfo.ID, processor.position) - if err != nil { - log.Warn("upload processor error failed", util.ZapFieldChangefeed(ctx), zap.Error(err)) - } - timeoutCancel() - } else { - log.Info("processor exited", - util.ZapFieldCapture(ctx), - zap.String("changefeed", changefeedID), - zap.String("processor", processor.id)) - } - }() - - return processor, nil -} - -func (p *oldProcessor) sendError(err error) { - select { - case p.errCh <- err: - default: - log.Error("processor receives redundant error", zap.Error(err)) - } -} diff --git a/cdc/processor_test.go b/cdc/processor_test.go deleted file mode 100644 index 5e591e89d36..00000000000 --- a/cdc/processor_test.go +++ /dev/null @@ -1,344 +0,0 @@ -// Copyright 2020 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 cdc - -import ( - "bytes" - - "github.com/pingcap/check" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/config" - "github.com/pingcap/tiflow/pkg/util/testleak" -) - -type processorSuite struct{} - -var _ = check.Suite(&processorSuite{}) - -func (s *processorSuite) TestWriteDebugInfo(c *check.C) { - defer testleak.AfterTest(c)() - p := &oldProcessor{ - changefeedID: "test", - changefeed: model.ChangeFeedInfo{ - SinkURI: "blackhole://", - Config: config.GetDefaultReplicaConfig(), - }, - tables: map[int64]*tableInfo{ - 1: { - id: 47, - name: "test.t1", - resolvedTs: 100, - }, - }, - } - var buf bytes.Buffer - p.writeDebugInfo(&buf) - c.Assert(buf.String(), check.Matches, `changefeedID[\s\S]*info[\s\S]*tables[\s\S]*`) -} - -/* -import ( - "context" - "sort" - "sync" - "time" - - "github.com/pingcap/tiflow/cdc/entry" - - "github.com/pingcap/check" - pd "github.com/pingcap/pd/client" - "github.com/pingcap/tiflow/cdc/kv" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/roles/storage" - "github.com/pingcap/tiflow/cdc/sink" - "github.com/pingcap/tiflow/pkg/etcd" -) - -type processorSuite struct{} - -type mockTsRWriter struct { - l sync.Mutex - globalResolvedTs uint64 - - memInfo *model.TaskStatus - storageInfo *model.TaskStatus -} - -var _ storage.ProcessorTsRWriter = &mockTsRWriter{} - -// ReadGlobalResolvedTs implement ProcessorTsRWriter interface. -func (s *mockTsRWriter) ReadGlobalResolvedTs(ctx context.Context) (uint64, error) { - s.l.Lock() - defer s.l.Unlock() - return s.globalResolvedTs, nil -} - -func (s *mockTsRWriter) WritePosition(ctx context.Context, taskPosition *model.TaskPosition) error { - return nil -} - -// GetTaskStatus implement ProcessorTsRWriter interface. -func (s *mockTsRWriter) GetTaskStatus() *model.TaskStatus { - return s.memInfo -} - -// WriteInfoIntoStorage implement ProcessorTsRWriter interface. -func (s *mockTsRWriter) WriteInfoIntoStorage(ctx context.Context) error { - s.storageInfo = s.memInfo.Clone() - return nil -} - -// UpdateInfo implement ProcessorTsRWriter interface. -func (s *mockTsRWriter) UpdateInfo(ctx context.Context) (bool, error) { - s.memInfo = s.storageInfo - s.storageInfo = s.memInfo.Clone() - - return true, nil -} - -func (s *mockTsRWriter) SetGlobalResolvedTs(ts uint64) { - s.l.Lock() - defer s.l.Unlock() - s.globalResolvedTs = ts -} - -// mockMounter pretend to decode a RawTxn by returning a Txn of the same Ts -type mockMounter struct{} - -func (m mockMounter) Mount(rawTxn model.RawTxn) (model.Txn, error) { - return model.Txn{Ts: rawTxn.Ts}, nil -} - -// mockSinker append all received Txns for validation -type mockSinker struct { - sink.Sink - synced []model.Txn - mu sync.Mutex -} - -func (m *mockSinker) Emit(ctx context.Context, txns ...model.Txn) error { - m.mu.Lock() - defer m.mu.Unlock() - m.synced = append(m.synced, txns...) - return nil -} - -var _ = check.Suite(&processorSuite{}) - -type processorTestCase struct { - rawTxnTs [][]uint64 - globalResolvedTs []uint64 - expect [][]uint64 -} - -func (p *processorSuite) TestProcessor(c *check.C) { - defer testleak.AfterTest(c)() - c.Skip("can't create mock puller") - cases := &processorTestCase{ - rawTxnTs: [][]uint64{ - {1, 4, 7, 9, 12, 14, 16, 20}, - {2, 4, 8, 13, 24}, - }, - globalResolvedTs: []uint64{14, 15, 19}, - expect: [][]uint64{ - {1, 2, 4, 4, 7, 8, 9, 12, 13, 14}, - {}, - {16}, - }, - } - runCase(c, cases) -} - -func runCase(c *check.C, cases *processorTestCase) { - origFSchema := fCreateSchema - fCreateSchema = func(pdEndpoints []string) (*entry.Storage, error) { - return nil, nil - } - origFNewPD := fNewPDCli - fNewPDCli = func(pdAddrs []string, security pd.SecurityOption, opts ...pd.ClientOption) (pd.Client, error) { - return nil, nil - } - origFNewTsRw := fNewTsRWriter - fNewTsRWriter = func(cli kv.CDCEtcdClient, changefeedID, captureID string) (storage.ProcessorTsRWriter, error) { - return &mockTsRWriter{}, nil - } - origFNewMounter := fNewMounter - fNewMounter = func(schema *entry.Storage) mounter { - return mockMounter{} - } - origFNewSink := fNewMySQLSink - sinker := &mockSinker{} - fNewMySQLSink = func(sinkURI string, infoGetter sink.TableInfoGetter, opts map[string]string) (sink.Sink, error) { - return sinker, nil - } - defer func() { - fCreateSchema = origFSchema - fNewPDCli = origFNewPD - fNewTsRWriter = origFNewTsRw - fNewMounter = origFNewMounter - fNewMySQLSink = origFNewSink - }() - - dir := c.MkDir() - etcdURL, etcd, err := etcd.SetupEmbedEtcd(dir) - c.Assert(err, check.IsNil) - defer etcd.Close() - - ctx, cancel := context.WithCancel(context.Background()) - p, err := NewProcessor(ctx, []string{etcdURL.String()}, model.ChangeFeedInfo{}, "", "", 0) - c.Assert(err, check.IsNil) - errCh := make(chan error, 1) - p.Run(ctx, errCh) - - for i, rawTxnTs := range cases.rawTxnTs { - p.addTable(ctx, int64(i), 0) - - table := p.tables[int64(i)] - input := table.inputTxn - - go func(rawTxnTs []uint64) { - for _, txnTs := range rawTxnTs { - input <- model.RawTxn{CRTs: txnTs} - } - }(rawTxnTs) - } - - for i, globalResolvedTs := range cases.globalResolvedTs { - // hack to simulate owner to update global resolved ts - p.getTsRwriter().(*mockTsRWriter).SetGlobalResolvedTs(globalResolvedTs) - // waiting for processor push to resolvedTs - for { - sinker.mu.Lock() - needBreak := len(sinker.synced) == len(cases.expect[i]) - sinker.mu.Unlock() - if needBreak { - break - } - time.Sleep(10 * time.Millisecond) - } - - sinker.mu.Lock() - syncedTs := make([]uint64, 0, len(sinker.synced)) - for _, s := range sinker.synced { - syncedTs = append(syncedTs, s.Ts) - } - sort.Slice(syncedTs, func(i, j int) bool { - return syncedTs[i] < syncedTs[j] - }) - c.Assert(syncedTs, check.DeepEquals, cases.expect[i]) - sinker.synced = sinker.synced[:0] - sinker.mu.Unlock() - } - cancel() -} - -func (p *processorSuite) TestDiffProcessTableInfos(c *check.C) { - defer testleak.AfterTest(c)() - infos := make([]*model.ProcessTableInfo, 0, 3) - for i := uint64(0); i < uint64(3); i++ { - infos = append(infos, &model.ProcessTableInfo{ID: i, StartTs: 10 * i}) - } - var ( - emptyInfo = make([]*model.ProcessTableInfo, 0) - cases = []struct { - oldInfo []*model.ProcessTableInfo - newInfo []*model.ProcessTableInfo - removed []*model.ProcessTableInfo - added []*model.ProcessTableInfo - }{ - {emptyInfo, emptyInfo, nil, nil}, - {[]*model.ProcessTableInfo{infos[0]}, []*model.ProcessTableInfo{infos[0]}, nil, nil}, - {emptyInfo, []*model.ProcessTableInfo{infos[0]}, nil, []*model.ProcessTableInfo{infos[0]}}, - {[]*model.ProcessTableInfo{infos[0]}, emptyInfo, []*model.ProcessTableInfo{infos[0]}, nil}, - {[]*model.ProcessTableInfo{infos[0]}, []*model.ProcessTableInfo{infos[1]}, []*model.ProcessTableInfo{infos[0]}, []*model.ProcessTableInfo{infos[1]}}, - {[]*model.ProcessTableInfo{infos[1], infos[0]}, []*model.ProcessTableInfo{infos[2], infos[1]}, []*model.ProcessTableInfo{infos[0]}, []*model.ProcessTableInfo{infos[2]}}, - {[]*model.ProcessTableInfo{infos[1]}, []*model.ProcessTableInfo{infos[0]}, []*model.ProcessTableInfo{infos[1]}, []*model.ProcessTableInfo{infos[0]}}, - } - ) - - for _, tc := range cases { - removed, added := diffProcessTableInfos(tc.oldInfo, tc.newInfo) - c.Assert(removed, check.DeepEquals, tc.removed) - c.Assert(added, check.DeepEquals, tc.added) - } -} - -type txnChannelSuite struct{} - -var _ = check.Suite(&txnChannelSuite{}) - -func (s *txnChannelSuite) TestShouldForwardTxnsByTs(c *check.C) { - defer testleak.AfterTest(c)() - input := make(chan model.RawTxn, 5) - var lastTs uint64 - callback := func(ts uint64) { - lastTs = ts - } - tc := newTxnChannel(input, 5, callback) - for _, ts := range []uint64{1, 2, 4, 6} { - select { - case input <- model.RawTxn{CRTs: ts}: - case <-time.After(time.Second): - c.Fatal("Timeout sending to input") - } - } - close(input) - - output := make(chan model.RawTxn, 5) - - assertCorrectOutput := func(expected []uint64) { - for _, ts := range expected { - c.Logf("Checking %d", ts) - select { - case e := <-output: - c.Assert(e.Ts, check.Equals, ts) - case <-time.After(time.Second): - c.Fatal("Timeout reading output") - } - } - - select { - case <-output: - c.Fatal("Output should be empty now") - default: - } - } - - tc.Forward(context.Background(), 3, output) - // Assert that all txns with ts not greater than 3 is sent to output - assertCorrectOutput([]uint64{1, 2}) - tc.Forward(context.Background(), 10, output) - // Assert that all txns with ts not greater than 10 is sent to output - assertCorrectOutput([]uint64{4, 6}) - c.Assert(lastTs, check.Equals, uint64(6)) -} - -func (s *txnChannelSuite) TestShouldBeCancellable(c *check.C) { - defer testleak.AfterTest(c)() - input := make(chan model.RawTxn, 5) - tc := newTxnChannel(input, 5, func(ts uint64) {}) - ctx, cancel := context.WithCancel(context.Background()) - stopped := make(chan struct{}) - go func() { - tc.Forward(ctx, 1, make(chan model.RawTxn)) - close(stopped) - }() - cancel() - select { - case <-stopped: - case <-time.After(time.Second): - c.Fatal("Not stopped in time after cancelled") - } -} -*/ diff --git a/cdc/server.go b/cdc/server.go index 6c9f2098d0f..4bbd6a89f47 100644 --- a/cdc/server.go +++ b/cdc/server.go @@ -45,8 +45,7 @@ import ( ) const ( - ownerRunInterval = time.Millisecond * 500 - defaultDataDir = "/tmp/cdc_data" + defaultDataDir = "/tmp/cdc_data" // dataDirThreshold is used to warn if the free space of the specified data-dir is lower than it, unit is GB dataDirThreshold = 500 ) @@ -285,23 +284,6 @@ func (s *Server) setUpDataDir(ctx context.Context) error { return nil } - // s.etcdClient maybe nil if NewReplicaImpl is not set to true - // todo: remove this after NewReplicaImpl set to true in a specific branch, and use server.etcdClient instead. - cli := s.etcdClient - if cli == nil { - client, err := clientv3.New(clientv3.Config{ - Endpoints: s.pdEndpoints, - Context: ctx, - DialTimeout: 5 * time.Second, - }) - if err != nil { - return err - } - etcdClient := kv.NewCDCEtcdClient(ctx, client) - cli = &etcdClient - defer cli.Close() - } - // data-dir will be decided by exist changefeed for backward compatibility allInfo, err := s.etcdClient.GetAllChangeFeedInfo(ctx) if err != nil { diff --git a/cdc/sink/mysql.go b/cdc/sink/mysql.go index f24a2e4bd75..fee490bea30 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -168,13 +168,6 @@ func (s *mysqlSink) flushRowChangedEvents(ctx context.Context, receiver *notify. s.txnCache.UpdateCheckpoint(resolvedTs) continue } - - if !config.NewReplicaImpl && s.cyclic != nil { - // Filter rows if it is origined from downstream. - skippedRowCount := cyclic.FilterAndReduceTxns( - resolvedTxnsMap, s.cyclic.FilterReplicaID(), s.cyclic.ReplicaID()) - s.statistics.SubRowsCount(skippedRowCount) - } s.dispatchAndExecTxns(ctx, resolvedTxnsMap) for _, worker := range s.workers { atomic.StoreUint64(&worker.checkpointTs, resolvedTs) diff --git a/cdc/task.go b/cdc/task.go deleted file mode 100644 index efd27618294..00000000000 --- a/cdc/task.go +++ /dev/null @@ -1,234 +0,0 @@ -// Copyright 2020 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 cdc - -import ( - "context" - - "github.com/pingcap/failpoint" - "github.com/pingcap/log" - "github.com/pingcap/tiflow/cdc/model" - cerror "github.com/pingcap/tiflow/pkg/errors" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/mvcc" - "go.uber.org/zap" -) - -// TaskEventOp is the operation of a task -type TaskEventOp string - -// Task Event Operatrions -const ( - TaskOpCreate TaskEventOp = "create" - TaskOpDelete TaskEventOp = "delete" -) - -// Task is dispatched by the owner -type Task struct { - ChangeFeedID string - CheckpointTS uint64 -} - -// TaskEvent represents a task is created or deleted -type TaskEvent struct { - Op TaskEventOp - Task *Task - Err error -} - -// TaskWatcher watches on new tasks -type TaskWatcher struct { - capture *Capture - cfg *TaskWatcherConfig - - events map[string]*TaskEvent -} - -// TaskWatcherConfig configures a watcher -type TaskWatcherConfig struct { - Prefix string - ChannelSize int64 -} - -// NewTaskWatcher returns a TaskWatcher -func NewTaskWatcher(c *Capture, cfg *TaskWatcherConfig) *TaskWatcher { - return &TaskWatcher{capture: c, cfg: cfg} -} - -// Watch on the new tasks, a channel is returned -func (w *TaskWatcher) Watch(ctx context.Context) <-chan *TaskEvent { - c := make(chan *TaskEvent, w.cfg.ChannelSize) - go w.watch(ctx, c) - return c -} - -func (w *TaskWatcher) watch(ctx context.Context, c chan *TaskEvent) { - etcd := w.capture.etcdClient.Client - - // Leader is required in this context to prevent read outdated data - // from a stale leader - ctx = clientv3.WithRequireLeader(ctx) - - // Send a task event to the channel, checks ctx.Done() to avoid blocking - send := func(ctx context.Context, ev *TaskEvent) error { - select { - case <-ctx.Done(): - close(c) - return ctx.Err() - case c <- ev: - } - return nil - } -restart: - // Load all the existed tasks - events := make(map[string]*TaskEvent) - resp, err := etcd.Get(ctx, w.cfg.Prefix, clientv3.WithPrefix()) - if err != nil { - _ = send(ctx, &TaskEvent{Err: err}) - return - } - for _, kv := range resp.Kvs { - ev, err := w.parseTaskEvent(ctx, kv.Key, kv.Value) - if err != nil { - log.Warn("parse task event failed", - zap.String("capture-id", w.capture.info.ID), - zap.Error(err)) - continue - } - events[ev.Task.ChangeFeedID] = ev - } - - // Rebuild the missed events - // When an error is occured during watch, the watch routine is restarted, - // in that case, some events maybe missed. Rebuild the events by comparing - // the new task list with the last successfully recorded tasks. - events = w.rebuildTaskEvents(events) - for _, ev := range events { - if err := send(ctx, ev); err != nil { - return - } - } - - wch := etcd.Watch(ctx, w.cfg.Prefix, - clientv3.WithPrefix(), - clientv3.WithPrevKV(), - clientv3.WithRev(resp.Header.Revision+1)) - for wresp := range wch { - err := wresp.Err() - failpoint.Inject("restart-task-watch", func() { - err = mvcc.ErrCompacted - }) - if err != nil { - goto restart - } - for _, ev := range wresp.Events { - if ev.Type == clientv3.EventTypePut { - ev, err := w.parseTaskEvent(ctx, ev.Kv.Key, ev.Kv.Value) - if err != nil { - log.Warn("parse task event failed", - zap.String("capture-id", w.capture.info.ID), - zap.Error(err)) - continue - } - w.events[ev.Task.ChangeFeedID] = ev - if err := send(ctx, ev); err != nil { - return - } - } else if ev.Type == clientv3.EventTypeDelete { - task, err := w.parseTask(ctx, ev.PrevKv.Key) - if err != nil { - log.Warn("parse task failed", - zap.String("capture-id", w.capture.info.ID), - zap.Error(err)) - continue - } - delete(w.events, task.ChangeFeedID) - if err := send(ctx, &TaskEvent{Op: TaskOpDelete, Task: task}); err != nil { - return - } - } - } - } - close(c) -} - -func (w *TaskWatcher) parseTask(ctx context.Context, - key []byte) (*Task, error) { - if len(key) <= len(w.cfg.Prefix) { - return nil, cerror.ErrInvalidTaskKey.GenWithStackByArgs(string(key)) - } - changeFeedID := string(key[len(w.cfg.Prefix)+1:]) - cf, err := w.capture.etcdClient.GetChangeFeedInfo(ctx, changeFeedID) - if err != nil { - return nil, err - } - status, _, err := w.capture.etcdClient.GetChangeFeedStatus(ctx, changeFeedID) - if err != nil && cerror.ErrChangeFeedNotExists.NotEqual(err) { - return nil, err - } - checkpointTs := cf.GetCheckpointTs(status) - return &Task{ChangeFeedID: changeFeedID, CheckpointTS: checkpointTs}, nil -} - -func (w *TaskWatcher) parseTaskEvent(ctx context.Context, key, val []byte) (*TaskEvent, error) { - task, err := w.parseTask(ctx, key) - if err != nil { - log.Warn("parse task failed", - zap.String("capture-id", w.capture.info.ID), - zap.Error(err)) - return nil, err - } - - taskStatus := &model.TaskStatus{} - if err := taskStatus.Unmarshal(val); err != nil { - log.Warn("unmarshal task status failed", - zap.String("capture-id", w.capture.info.ID), - zap.Error(err)) - return nil, err - } - var op TaskEventOp - switch taskStatus.AdminJobType { - case model.AdminNone, model.AdminResume: - op = TaskOpCreate - case model.AdminStop, model.AdminRemove, model.AdminFinish: - op = TaskOpDelete - } - return &TaskEvent{Op: op, Task: task}, nil -} - -func (w *TaskWatcher) rebuildTaskEvents(latest map[string]*TaskEvent) map[string]*TaskEvent { - events := make(map[string]*TaskEvent) - outdated := w.events - for id, ev := range outdated { - // Check if the task still exists - if nev, ok := latest[id]; ok { - if ev.Op != nev.Op { - events[id] = nev - } - } else if ev.Op != TaskOpDelete { - events[id] = &TaskEvent{Op: TaskOpDelete, Task: ev.Task} - } - } - - for id, ev := range latest { - if _, ok := outdated[id]; !ok { - events[id] = ev - } - } - - // Update to the latest tasks - w.events = events - - return events -} diff --git a/cdc/task_test.go b/cdc/task_test.go deleted file mode 100644 index 4499e9e533f..00000000000 --- a/cdc/task_test.go +++ /dev/null @@ -1,286 +0,0 @@ -// Copyright 2020 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 cdc - -import ( - "context" - "math" - "time" - - "github.com/pingcap/check" - "github.com/pingcap/failpoint" - "github.com/pingcap/tiflow/cdc/kv" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/etcd" - "github.com/pingcap/tiflow/pkg/util/testleak" - "go.etcd.io/etcd/clientv3" - "go.etcd.io/etcd/embed" -) - -type taskSuite struct { - s *embed.Etcd - c *clientv3.Client - w *TaskWatcher - endpoints []string -} - -var _ = check.Suite(&taskSuite{}) - -func (s *taskSuite) SetUpTest(c *check.C) { - dir := c.MkDir() - url, etcd, err := etcd.SetupEmbedEtcd(dir) - c.Assert(err, check.IsNil) - - endpoints := []string{url.String()} - client, err := clientv3.New(clientv3.Config{ - Endpoints: endpoints, - }) - c.Assert(err, check.IsNil) - - // Create a task watcher - capture := &Capture{ - etcdClient: kv.NewCDCEtcdClient(context.TODO(), client), - processors: make(map[string]*oldProcessor), - info: &model.CaptureInfo{ID: "task-suite-capture", AdvertiseAddr: "task-suite-addr"}, - } - c.Assert(capture, check.NotNil) - watcher := NewTaskWatcher(capture, &TaskWatcherConfig{ - Prefix: kv.TaskStatusKeyPrefix + "/" + capture.info.ID, - }) - c.Assert(watcher, check.NotNil) - - s.s = etcd - s.c = client - s.w = watcher - s.endpoints = endpoints -} - -func (s *taskSuite) TearDownTest(c *check.C) { - s.s.Close() - s.c.Close() -} - -func (s *taskSuite) setupFeedInfo(c *check.C, changeFeedID string) { - client := kv.NewCDCEtcdClient(context.TODO(), s.c) - // Create the change feed - c.Assert(client.SaveChangeFeedInfo(s.c.Ctx(), &model.ChangeFeedInfo{ - SinkURI: "mysql://fake", - StartTs: 0, - TargetTs: math.MaxUint64, - CreateTime: time.Now(), - }, changeFeedID), check.IsNil) - - // Fake the change feed status - c.Assert(client.PutChangeFeedStatus(s.c.Ctx(), changeFeedID, - &model.ChangeFeedStatus{ - ResolvedTs: 1, - CheckpointTs: 1, - }), check.IsNil) -} - -func (s *taskSuite) teardownFeedInfo(c *check.C, changeFeedID string) { - etcd := s.c - // Delete change feed info - resp, err := etcd.Delete(s.c.Ctx(), kv.GetEtcdKeyChangeFeedInfo(changeFeedID), clientv3.WithPrefix()) - c.Assert(err, check.IsNil) - c.Assert(resp, check.NotNil) - - // Delete change feed status(job status) - resp, err = etcd.Delete(s.c.Ctx(), kv.GetEtcdKeyJob(changeFeedID), clientv3.WithPrefix()) - c.Assert(err, check.IsNil) - c.Assert(resp, check.NotNil) -} - -func (s *taskSuite) TestParseTask(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - changeFeedID := "task-suite-changefeed" - s.setupFeedInfo(c, changeFeedID) - defer s.teardownFeedInfo(c, changeFeedID) - - tests := []struct { - Desc string - Key []byte - Expected *Task - }{ - {"nil task key", nil, nil}, - {"short task key", []byte("test"), nil}, - { - "normal task key", - []byte(kv.GetEtcdKeyTaskStatus(changeFeedID, s.w.capture.info.ID)), - &Task{changeFeedID, 1}, - }, - } - for _, t := range tests { - c.Log("testing ", t.Desc) - task, err := s.w.parseTask(ctx, t.Key) - if t.Expected == nil { - c.Assert(err, check.NotNil) - c.Assert(task, check.IsNil) - } else { - c.Assert(task, check.DeepEquals, t.Expected) - } - } -} - -func (s *taskSuite) TestWatch(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - client := kv.NewCDCEtcdClient(ctx, s.c) - defer client.Close() //nolint:errcheck - - s.setupFeedInfo(c, "changefeed-1") - defer s.teardownFeedInfo(c, "changefeed-1") - - // Watch with a canceled context - failedCtx, cancel := context.WithCancel(context.Background()) - cancel() - ev := <-s.w.Watch(failedCtx) - if ev != nil { - c.Assert(ev.Err, check.NotNil) - } - - // Watch with a normal context - ctx, cancel = context.WithCancel(context.Background()) - defer cancel() - ch := s.w.Watch(ctx) - - // Trigger the ErrCompacted error - c.Assert(failpoint.Enable("github.com/pingcap/tiflow/cdc.restart_task_watch", "50%off"), check.IsNil) - - // Put task changefeed-1 - c.Assert(client.PutTaskStatus(s.c.Ctx(), "changefeed-1", - s.w.capture.info.ID, - &model.TaskStatus{}), check.IsNil) - ev = <-ch - c.Assert(len(ch), check.Equals, 0) - c.Assert(ev, check.NotNil) - c.Assert(ev.Err, check.IsNil) - c.Assert(ev.Op, check.Equals, TaskOpCreate) - c.Assert(ev.Task.ChangeFeedID, check.Equals, "changefeed-1") - c.Assert(ev.Task.CheckpointTS, check.Equals, uint64(1)) - - // Stop the task changefeed-1 - c.Assert(client.PutTaskStatus(s.c.Ctx(), "changefeed-1", - s.w.capture.info.ID, - &model.TaskStatus{AdminJobType: model.AdminStop}), check.IsNil) - ev = <-ch - c.Assert(len(ch), check.Equals, 0) - c.Assert(ev, check.NotNil) - c.Assert(ev.Err, check.IsNil) - c.Assert(ev.Op, check.Equals, TaskOpDelete) - c.Assert(ev.Task.ChangeFeedID, check.Equals, "changefeed-1") - c.Assert(ev.Task.CheckpointTS, check.Equals, uint64(1)) - - // Resume the task changefeed-1 - c.Assert(client.PutTaskStatus(s.c.Ctx(), "changefeed-1", - s.w.capture.info.ID, - &model.TaskStatus{AdminJobType: model.AdminResume}), check.IsNil) - ev = <-ch - c.Assert(len(ch), check.Equals, 0) - c.Assert(ev, check.NotNil) - c.Assert(ev.Err, check.IsNil) - c.Assert(ev.Op, check.Equals, TaskOpCreate) - c.Assert(ev.Task.ChangeFeedID, check.Equals, "changefeed-1") - c.Assert(ev.Task.CheckpointTS, check.Equals, uint64(1)) - - // Delete the task changefeed-1 - c.Assert(client.DeleteTaskStatus(ctx, "changefeed-1", - s.w.capture.info.ID), check.IsNil) - ev = <-ch - c.Assert(len(ch), check.Equals, 0) - c.Assert(ev, check.NotNil) - c.Assert(ev.Err, check.IsNil) - c.Assert(ev.Op, check.Equals, TaskOpDelete) - c.Assert(ev.Task.ChangeFeedID, check.Equals, "changefeed-1") - c.Assert(ev.Task.CheckpointTS, check.Equals, uint64(1)) - - // Put task changefeed-2 which does not exist - c.Assert(client.PutTaskStatus(s.c.Ctx(), "changefeed-2", - s.w.capture.info.ID, - &model.TaskStatus{}), check.IsNil) - c.Assert(len(ch), check.Equals, 0) -} - -func (s *taskSuite) TestRebuildTaskEvents(c *check.C) { - defer testleak.AfterTest(c)() - defer s.TearDownTest(c) - type T map[string]*TaskEvent - tests := []struct { - desc string - outdated T - latest T - expected T - }{ - { - desc: "nil outdated", - outdated: nil, - latest: T{"changefeed-1": &TaskEvent{TaskOpCreate, &Task{"changeed-1", 0}, nil}}, - expected: T{"changefeed-1": &TaskEvent{TaskOpCreate, &Task{"changeed-1", 0}, nil}}, - }, - { - desc: "empty outdated", - outdated: nil, - latest: T{"changefeed-1": &TaskEvent{TaskOpCreate, &Task{"changeed-1", 0}, nil}}, - expected: T{"changefeed-1": &TaskEvent{TaskOpCreate, &Task{"changeed-1", 0}, nil}}, - }, - { - desc: "need to be updated", - outdated: T{"changefeed-1": &TaskEvent{TaskOpCreate, &Task{"changeed-1", 0}, nil}}, - latest: T{"changefeed-1": &TaskEvent{TaskOpDelete, &Task{"changeed-1", 0}, nil}}, - expected: T{"changefeed-1": &TaskEvent{TaskOpDelete, &Task{"changeed-1", 0}, nil}}, - }, - { - desc: "miss some events", - outdated: T{"changefeed-1": &TaskEvent{TaskOpCreate, &Task{"changeed-1", 0}, nil}}, - latest: T{ - "changefeed-1": &TaskEvent{TaskOpDelete, &Task{"changeed-1", 0}, nil}, - "changefeed-2": &TaskEvent{TaskOpCreate, &Task{"changefeed-2", 0}, nil}, - }, - expected: T{ - "changefeed-1": &TaskEvent{TaskOpDelete, &Task{"changeed-1", 0}, nil}, - "changefeed-2": &TaskEvent{TaskOpCreate, &Task{"changefeed-2", 0}, nil}, - }, - }, - { - desc: "left some events", - outdated: T{ - "changefeed-1": &TaskEvent{TaskOpDelete, &Task{"changeed-1", 0}, nil}, - "changefeed-2": &TaskEvent{TaskOpCreate, &Task{"changefeed-2", 0}, nil}, - }, - latest: T{"changefeed-1": &TaskEvent{TaskOpCreate, &Task{"changeed-1", 0}, nil}}, - expected: T{ - "changefeed-1": &TaskEvent{TaskOpCreate, &Task{"changeed-1", 0}, nil}, - "changefeed-2": &TaskEvent{TaskOpDelete, &Task{"changefeed-2", 0}, nil}, - }, - }, - } - - for _, t := range tests { - c.Log("RUN CASE: ", t.desc) - s.w.events = t.outdated - got := s.w.rebuildTaskEvents(t.latest) - c.Assert(len(got), check.Equals, len(t.expected)) - for k, v := range got { - e := t.expected[k] - c.Assert(v.Err, check.IsNil) - c.Assert(v.Op, check.Equals, e.Op) - c.Assert(v.Task, check.DeepEquals, e.Task) - } - } -} diff --git a/pkg/config/config.go b/pkg/config/config.go index d5475595cb1..01b64ada2d4 100644 --- a/pkg/config/config.go +++ b/pkg/config/config.go @@ -30,9 +30,6 @@ import ( ) const ( - // NewReplicaImpl is true if we using new processor - // new owner should be also switched on after it implemented - NewReplicaImpl = true // DefaultSortDir is the default value of sort-dir, it will be s sub directory of data-dir. DefaultSortDir = "/tmp/sorter" ) diff --git a/pkg/util/testleak/fake.go b/pkg/util/testleak/fake.go index 46b1b990671..05d1a0a695b 100644 --- a/pkg/util/testleak/fake.go +++ b/pkg/util/testleak/fake.go @@ -10,6 +10,7 @@ // 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. +//go:build !leak // +build !leak package testleak diff --git a/pkg/util/testleak/leaktest.go b/pkg/util/testleak/leaktest.go index 31c780ceaea..0c940915490 100644 --- a/pkg/util/testleak/leaktest.go +++ b/pkg/util/testleak/leaktest.go @@ -14,6 +14,7 @@ // 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. +//go:build leak // +build leak package testleak diff --git a/tests/integration_tests/availability/owner.sh b/tests/integration_tests/availability/owner.sh index bf2ce93e67c..d8975521862 100755 --- a/tests/integration_tests/availability/owner.sh +++ b/tests/integration_tests/availability/owner.sh @@ -159,8 +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/tiflow/cdc/capture-campaign-compacted-error=1*return(true)' # old owner - export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/capture/capture-campaign-compacted-error=1*return(true)' # new owner + 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 @@ -172,8 +171,7 @@ function test_owner_retryable_error() { echo "owner pid:" $owner_pid echo "owner id" $owner_id - # export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner-run-with-error=1*return(true);github.com/pingcap/tiflow/cdc/capture-resign-failed=1*return(true)' # old owner - 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)' # new owner + 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" @@ -198,8 +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/tiflow/cdc/sleep-before-watch-capture=1*sleep(6000)' # old owner - export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/sleep-in-owner-tick=1*sleep(6000)' # new owner + 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/capture_session_done_during_task/run.sh b/tests/integration_tests/capture_session_done_during_task/run.sh index 91c2ad133e5..412b1cc120b 100644 --- a/tests/integration_tests/capture_session_done_during_task/run.sh +++ b/tests/integration_tests/capture_session_done_during_task/run.sh @@ -28,8 +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 --pd=http://$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/tiflow/cdc/captureHandleTaskDelay=sleep(2000)' # old processor - export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/processor/processorManagerHandleNewChangefeedDelay=sleep(2000)' # new processor + 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/changefeed_auto_stop/run.sh b/tests/integration_tests/changefeed_auto_stop/run.sh index bc2f8e589ac..19ce910a00b 100755 --- a/tests/integration_tests/changefeed_auto_stop/run.sh +++ b/tests/integration_tests/changefeed_auto_stop/run.sh @@ -44,8 +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/tiflow/cdc/ProcessorSyncResolvedError=1*return(true);github.com/pingcap/tiflow/cdc/ProcessorUpdatePositionDelaying=return(true)' # old processor - export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/processor/pipeline/ProcessorSyncResolvedError=1*return(true);github.com/pingcap/tiflow/cdc/processor/ProcessorUpdatePositionDelaying=sleep(1000)' # new processor + 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 3696a514d7e..ab0ce8b70e2 100755 --- a/tests/integration_tests/changefeed_error/run.sh +++ b/tests/integration_tests/changefeed_error/run.sh @@ -114,8 +114,7 @@ function run() { start_ts=$(run_cdc_cli tso query --pd=http://$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/tiflow/cdc/NewChangefeedNoRetryError=1*return(true)' # old owner - export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/NewChangefeedNoRetryError=1*return(true)' # new owner + 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}') @@ -139,8 +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/tiflow/cdc/NewChangefeedRetryError=return(true)' # old owner - export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/NewChangefeedRetryError=return(true)' # new owner + 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 @@ -153,8 +151,7 @@ function run() { cleanup_process $CDC_BINARY # owner DDL error case - # export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/InjectChangefeedDDLError=return(true)' # old owner - export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/InjectChangefeedDDLError=return(true)' # new owner + 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 @@ -166,7 +163,7 @@ function run() { cleanup_process $CDC_BINARY # updating GC safepoint failure case - export GO_FAILPOINTS='github.com/pingcap/tiflow/pkg/txnutil/gc/InjectActualGCSafePoint=return(9223372036854775807)' # new owner + 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/ddl_puller_lag/run.sh b/tests/integration_tests/ddl_puller_lag/run.sh index 2d4a2d1732e..96d74bc337b 100644 --- a/tests/integration_tests/ddl_puller_lag/run.sh +++ b/tests/integration_tests/ddl_puller_lag/run.sh @@ -21,8 +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/tiflow/cdc/processorDDLResolved=1*sleep(180000)' # old processor - run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --failpoint 'github.com/pingcap/tiflow/cdc/processor/processorDDLResolved=1*sleep(180000)' # new processor + 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/gc_safepoint/run.sh b/tests/integration_tests/gc_safepoint/run.sh index 37348de0ade..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/tiflow/pkg/txnutil/gc/InjectGcSafepointUpdateInterval=return(500)' # new owner + 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/kill_owner_with_ddl/run.sh b/tests/integration_tests/kill_owner_with_ddl/run.sh index cc5ff1ebb74..b13f8dbc8fc 100755 --- a/tests/integration_tests/kill_owner_with_ddl/run.sh +++ b/tests/integration_tests/kill_owner_with_ddl/run.sh @@ -56,8 +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/tiflow/cdc/sink/MySQLSinkExecDDLDelay=return(true);github.com/pingcap/tiflow/cdc/ownerFlushIntervalInject=return(0)' # old owner - export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/sink/MySQLSinkExecDDLDelay=return(true);github.com/pingcap/tiflow/cdc/capture/ownerFlushIntervalInject=return(10)' # new owner + 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/owner_remove_table_error/run.sh b/tests/integration_tests/owner_remove_table_error/run.sh index d04cc6290b3..5ba75eb1ef1 100644 --- a/tests/integration_tests/owner_remove_table_error/run.sh +++ b/tests/integration_tests/owner_remove_table_error/run.sh @@ -25,8 +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/tiflow/cdc/OwnerRemoveTableError=1*return(true)' # old owner - export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/owner/OwnerRemoveTableError=1*return(true)' # new owner + 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 b8cdb2c6716..7e9ca97176c 100644 --- a/tests/integration_tests/processor_err_chan/run.sh +++ b/tests/integration_tests/processor_err_chan/run.sh @@ -47,8 +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/tiflow/cdc/ProcessorAddTableError=1*return(true)' # old processor - export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/processor/pipeline/ProcessorAddTableError=1*return(true)' # new processor + 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/run.sh b/tests/integration_tests/processor_panic/run.sh index b7b37b60dc3..5ae9522d3f9 100644 --- a/tests/integration_tests/processor_panic/run.sh +++ b/tests/integration_tests/processor_panic/run.sh @@ -18,11 +18,8 @@ function prepare() { # record tso before we create tables to skip the system table DDLs start_ts=$(run_cdc_cli tso query --pd=http://$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/tiflow/cdc/ProcessorSyncResolvedPreEmit=return(true)' # old processor - run_cdc_server --workdir $WORK_DIR --binary $CDC_BINARY --logsuffix 1 --addr 127.0.0.1:8300 --restart true \ - --failpoint 'github.com/pingcap/tiflow/cdc/processor/pipeline/ProcessorSyncResolvedPreEmit=return(true)' # new processor + --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_stop_delay/run.sh b/tests/integration_tests/processor_stop_delay/run.sh index b7289286bfc..89d23b963fb 100644 --- a/tests/integration_tests/processor_stop_delay/run.sh +++ b/tests/integration_tests/processor_stop_delay/run.sh @@ -20,8 +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/tiflow/cdc/processorStopDelay=1*sleep(10000)' # old processor - export GO_FAILPOINTS='github.com/pingcap/tiflow/cdc/processor/processorStopDelay=1*sleep(10000)' # new processor + 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}') From 2ac75e491f688329a3b043df5a003b1857f7b1e3 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Fri, 24 Dec 2021 11:33:47 +0800 Subject: [PATCH 16/30] tests(ticdc): set up the sync diff output directory correctly (#3725) (#3745) --- 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 +- 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_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 +- 46 files changed, 48 insertions(+), 50 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/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_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 6b0924046f7..b15b6342c28 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 149f3f4532ebc2bac77f2c3d935a30df2b94980e Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Fri, 24 Dec 2021 14:25:47 +0800 Subject: [PATCH 17/30] owner,scheduler(cdc): fix nil pointer panic in owner scheduler (#2980) (#4007) (#4015) --- 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 00f0771d815..7f47c61ff9f 100644 --- a/cdc/owner/scheduler.go +++ b/cdc/owner/scheduler.go @@ -300,6 +300,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 669ab85a8f5..701e1838efb 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" ) @@ -84,8 +85,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 28614b34f8baf449929b14763788a99353d63529 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Fri, 24 Dec 2021 15:51:47 +0800 Subject: [PATCH 18/30] config(ticdc): Fix old value configuration check for maxwell protocol (#3747) (#3782) --- 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 d1d028f92b1..5ffe27767c1 100644 --- a/cdc/sink/codec/interface.go +++ b/cdc/sink/codec/interface.go @@ -192,6 +192,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") + } +} + // NewEventBatchEncoder returns a function of creating an EventBatchEncoder by protocol. func NewEventBatchEncoder(p Protocol) func() EventBatchEncoder { switch p { diff --git a/cdc/sink/mq.go b/cdc/sink/mq.go index d3142590758..83b09b5ff88 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" @@ -107,9 +108,10 @@ func newMqSink( avroEncoder.SetTimeZone(util.TimezoneFromCtx(ctx)) return avroEncoder } - } else 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")) + } else 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()))) } // pre-flight verification of encoder parameters diff --git a/pkg/cmd/cli/cli_changefeed_create.go b/pkg/cmd/cli/cli_changefeed_create.go index 1e492a9a060..d88af5cce33 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 6476b3701926a083583b4d5c7801d60000f3ed66 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Fri, 24 Dec 2021 19:33:54 +0800 Subject: [PATCH 19/30] ticdc/processor: Fix backoff base delay misconfiguration (#3992) (#4027) --- 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 ddcca8a87da..fd50fa844a2 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -700,7 +700,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 7e39a2fbe1bdeccb8a0a1ac00dd98980da040f29 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Thu, 30 Dec 2021 10:11:50 +0800 Subject: [PATCH 20/30] sink(ticdc): cherry pick sink bug fix to release 5.2 (#4083) (#4119) --- 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_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 | 137 +++-- cdc/sink/manager_test.go | 65 ++- cdc/sink/mq.go | 15 +- cdc/sink/mq_test.go | 10 +- cdc/sink/mysql.go | 86 ++- cdc/sink/mysql_test.go | 715 +++++++---------------- cdc/sink/mysql_worker_test.go | 362 ++++++++++++ cdc/sink/simple_mysql_tester.go | 4 +- cdc/sink/sink.go | 4 +- cmd/kafka-consumer/main.go | 68 ++- tests/integration_tests/sink_hang/run.sh | 4 +- 23 files changed, 984 insertions(+), 727 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 5a91821c9d1..833c9bfb603 100644 --- a/cdc/owner/async_sink_test.go +++ b/cdc/owner/async_sink_test.go @@ -66,7 +66,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 09f8823a3b6..296051b37f1 100644 --- a/cdc/processor/pipeline/table.go +++ b/cdc/processor/pipeline/table.go @@ -194,7 +194,7 @@ func NewTablePipeline(ctx cdcContext.Context, if cyclicEnabled { p.AppendNode(ctx, "cyclic", newCyclicMarkNode(replicaInfo.MarkTableID)) } - tablePipeline.sinkNode = newSinkNode(sink, replicaInfo.StartTs, targetTs, flowController) + tablePipeline.sinkNode = newSinkNode(tableID, sink, replicaInfo.StartTs, targetTs, flowController) p.AppendNode(ctx, "sink", tablePipeline.sinkNode) tablePipeline.p = p return tablePipeline diff --git a/cdc/processor/processor.go b/cdc/processor/processor.go index fd50fa844a2..b20dc9121d2 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -163,6 +163,8 @@ func (p *processor) tick(ctx cdcContext.Context, state *model.ChangefeedReactorS 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 0b2d17681b7..f03cc0b17b5 100644 --- a/cdc/processor/processor_test.go +++ b/cdc/processor/processor_test.go @@ -27,6 +27,7 @@ import ( "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/sink" cdcContext "github.com/pingcap/tiflow/pkg/context" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/etcd" @@ -47,6 +48,7 @@ func newProcessor4Test( ) *processor { p := newProcessor(ctx) p.lazyInit = func(ctx cdcContext.Context) error { return nil } + p.sinkManager = &sink.Manager{} p.createTablePipeline = createTablePipeline p.schemaStorage = &mockSchemaStorage{c: c} return p 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_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 0a76d1a5fc2..1e0df2b2242 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 2249628358f..dc5a6cf2833 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 56cf9a8e59e..6fae483b2d0 100644 --- a/cdc/sink/manager.go +++ b/cdc/sink/manager.go @@ -35,10 +35,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 @@ -58,7 +59,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,26 +88,29 @@ 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 { - emittedTs := tableSink.getEmittedTs() - if minTs > emittedTs { - minTs = emittedTs + for _, tblSink := range m.tableSinks { + resolvedTs := tblSink.getResolvedTs() + if minTs > resolvedTs { + minTs = resolvedTs } } 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(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) getCheckpointTs() uint64 { - return atomic.LoadUint64(&m.checkpointTs) +func (m *Manager) UpdateChangeFeedCheckpointTs(checkpointTs uint64) { + atomic.StoreUint64(&m.changeFeedCheckpointTs, checkpointTs) + if m.backendSink != nil { + m.backendSink.UpdateChangeFeedCheckpointTs(checkpointTs) + } } type tableSink struct { @@ -176,7 +193,10 @@ func (t *tableSink) EmitDDLEvent(ctx context.Context, ddl *model.DDLEvent) error return nil } -func (t *tableSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64) (uint64, 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, tableID model.TableID, resolvedTs uint64) (uint64, error) { // Log abnormal checkpoint that is large than resolved ts. logAbnormalCheckpoint := func(ckpt uint64) { if ckpt > resolvedTs { @@ -192,7 +212,7 @@ func (t *tableSink) FlushRowChangedEvents(ctx context.Context, resolvedTs uint64 }) if i == 0 { atomic.StoreUint64(&t.emittedTs, resolvedTs) - ckpt, err := t.manager.flushBackendSink(ctx) + ckpt, err := t.manager.flushBackendSink(ctx, tableID) if err != nil { return ckpt, err } @@ -204,10 +224,10 @@ 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.manager.flushBackendSink(ctx) + ckpt, err := t.manager.flushBackendSink(ctx, tableID) if err != nil { return ckpt, err } @@ -229,8 +249,15 @@ func (t *tableSink) Close(ctx context.Context) error { return t.manager.destroyTableSink(ctx, t.tableID) } +// getResolvedTs returns resolved ts, which means all events before resolved ts +// have been sent to sink manager +func (t *tableSink) getResolvedTs() uint64 { + ts := atomic.LoadUint64(&t.emittedTs) + return ts +} + // 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 } @@ -241,11 +268,12 @@ type drawbackMsg struct { 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( @@ -254,14 +282,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 @@ -293,8 +321,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 { @@ -321,14 +350,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()) @@ -358,11 +388,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 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.checkpointTs), nil + 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/manager_test.go b/cdc/sink/manager_test.go index 002b476a452..b9ba6a545f2 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) } @@ -232,7 +243,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) @@ -242,7 +253,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) @@ -253,7 +264,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 @@ -293,11 +304,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) } @@ -308,9 +319,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) } @@ -343,7 +354,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") } @@ -355,7 +366,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 } @@ -372,7 +383,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 83b09b5ff88..d9633399982 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -52,7 +52,7 @@ type mqSink struct { resolvedTs uint64 } partitionResolvedTs []uint64 - checkpointTs uint64 + tableCheckpointTs map[model.TableID]uint64 resolvedNotifier *notify.Notifier resolvedReceiver *notify.Receiver @@ -143,6 +143,7 @@ func newMqSink( partitionNum: partitionNum, partitionInput: partitionInput, partitionResolvedTs: make([]uint64, partitionNum), + tableCheckpointTs: make(map[model.TableID]uint64), resolvedNotifier: notifier, resolvedReceiver: resolvedReceiver, @@ -185,9 +186,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++ { @@ -220,9 +221,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 { @@ -275,7 +276,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 forcedlly. return nil diff --git a/cdc/sink/mq_test.go b/cdc/sink/mq_test.go index 31c43fd6f86..c60e3cb229d 100644 --- a/cdc/sink/mq_test.go +++ b/cdc/sink/mq_test.go @@ -72,10 +72,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, @@ -83,11 +85,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 fee490bea30..15e8125887b 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 *filter.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 @@ -120,13 +119,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 @@ -136,13 +133,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 } @@ -159,20 +150,18 @@ 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 } 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) } } @@ -724,7 +713,6 @@ type mysqlSinkWorker struct { execDMLs func(context.Context, []*model.RowChangedEvent, uint64, int) error metricBucketSize prometheus.Counter receiver *notify.Receiver - checkpointTs uint64 closedCh chan struct{} } @@ -767,10 +755,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. @@ -808,7 +795,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 @@ -838,7 +824,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 { @@ -875,7 +860,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() @@ -884,15 +869,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 } @@ -905,15 +898,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 29602302f49..3cd3bcb89a4 100644 --- a/cdc/sink/mysql_test.go +++ b/cdc/sink/mysql_test.go @@ -27,9 +27,7 @@ import ( "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" timodel "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -40,21 +38,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 +55,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 +90,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 +143,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 +183,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 +267,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 +306,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 +318,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,19 +423,18 @@ 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) + require.Equal(t, tc.expectSQLs, sqls) + require.Equal(t, tc.expectArgs, args) } } -func (s MySQLSinkSuite) TestSinkParamsClone(c *check.C) { - defer testleak.AfterTest(c)() +func TestSinkParamsClone(t *testing.T) { param1 := defaultParams.Clone() param2 := param1.Clone() param2.changefeedID = "123" param2.batchReplaceEnabled = false param2.maxTxnRow = 1 - c.Assert(param1, check.DeepEquals, &sinkParams{ + require.Equal(t, &sinkParams{ workerCount: defaultWorkerCount, maxTxnRow: defaultMaxTxnRow, tidbTxnMode: defaultTiDBTxnMode, @@ -754,8 +444,8 @@ func (s MySQLSinkSuite) TestSinkParamsClone(c *check.C) { writeTimeout: defaultWriteTimeout, dialTimeout: defaultDialTimeout, safeMode: defaultSafeMode, - }) - c.Assert(param2, check.DeepEquals, &sinkParams{ + }, param1) + require.Equal(t, &sinkParams{ changefeedID: "123", workerCount: defaultWorkerCount, maxTxnRow: 1, @@ -766,22 +456,20 @@ func (s MySQLSinkSuite) TestSinkParamsClone(c *check.C) { writeTimeout: defaultWriteTimeout, dialTimeout: defaultDialTimeout, safeMode: defaultSafeMode, - }) + }, param2) } -func (s MySQLSinkSuite) TestConfigureSinkURI(c *check.C) { - defer testleak.AfterTest(c)() - +func TestConfigureSinkURI(t *testing.T) { testDefaultParams := func() { db, err := mockTestDB() - c.Assert(err, check.IsNil) + require.Nil(t, err) defer db.Close() dsn, err := dmysql.ParseDSN("root:123456@tcp(127.0.0.1:4000)/") - c.Assert(err, check.IsNil) + require.Nil(t, err) params := defaultParams.Clone() dsnStr, err := configureSinkURI(context.TODO(), dsn, params, db) - c.Assert(err, check.IsNil) + require.Nil(t, err) expectedParams := []string{ "tidb_txn_mode=optimistic", "readTimeout=2m", @@ -789,45 +477,45 @@ func (s MySQLSinkSuite) TestConfigureSinkURI(c *check.C) { "allow_auto_random_explicit_insert=1", } for _, param := range expectedParams { - c.Assert(strings.Contains(dsnStr, param), check.IsTrue) + require.True(t, strings.Contains(dsnStr, param)) } - c.Assert(strings.Contains(dsnStr, "time_zone"), check.IsFalse) + require.False(t, strings.Contains(dsnStr, "time_zone")) } testTimezoneParam := func() { db, err := mockTestDB() - c.Assert(err, check.IsNil) + require.Nil(t, err) defer db.Close() dsn, err := dmysql.ParseDSN("root:123456@tcp(127.0.0.1:4000)/") - c.Assert(err, check.IsNil) + require.Nil(t, err) 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) + require.Nil(t, err) + require.True(t, strings.Contains(dsnStr, "time_zone=%22UTC%22")) } testTimeoutParams := func() { db, err := mockTestDB() - c.Assert(err, check.IsNil) + require.Nil(t, err) defer db.Close() dsn, err := dmysql.ParseDSN("root:123456@tcp(127.0.0.1:4000)/") - c.Assert(err, check.IsNil) + require.Nil(t, err) uri, err := url.Parse("mysql://127.0.0.1:3306/?read-timeout=4m&write-timeout=5m&timeout=3m") - c.Assert(err, check.IsNil) + require.Nil(t, err) params, err := parseSinkURI(context.TODO(), uri, map[string]string{}) - c.Assert(err, check.IsNil) + require.Nil(t, err) dsnStr, err := configureSinkURI(context.TODO(), dsn, params, db) - c.Assert(err, check.IsNil) + require.Nil(t, err) expectedParams := []string{ "readTimeout=4m", "writeTimeout=5m", "timeout=3m", } for _, param := range expectedParams { - c.Assert(strings.Contains(dsnStr, param), check.IsTrue) + require.True(t, strings.Contains(dsnStr, param)) } } @@ -836,8 +524,7 @@ func (s MySQLSinkSuite) TestConfigureSinkURI(c *check.C) { testTimeoutParams() } -func (s MySQLSinkSuite) TestParseSinkURI(c *check.C) { - defer testleak.AfterTest(c)() +func TestParseSinkURI(t *testing.T) { expected := defaultParams.Clone() expected.workerCount = 64 expected.maxTxnRow = 20 @@ -856,14 +543,13 @@ func (s MySQLSinkSuite) TestParseSinkURI(c *check.C) { OptCaptureAddr: expected.captureAddr, } uri, err := url.Parse(uriStr) - c.Assert(err, check.IsNil) + require.Nil(t, err) params, err := parseSinkURI(context.TODO(), uri, opts) - c.Assert(err, check.IsNil) - c.Assert(params, check.DeepEquals, expected) + require.Nil(t, err) + require.Equal(t, expected, params) } -func (s MySQLSinkSuite) TestParseSinkURITimezone(c *check.C) { - defer testleak.AfterTest(c)() +func TestParseSinkURITimezone(t *testing.T) { 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", @@ -878,15 +564,14 @@ func (s MySQLSinkSuite) TestParseSinkURITimezone(c *check.C) { opts := map[string]string{} for i, uriStr := range uris { uri, err := url.Parse(uriStr) - c.Assert(err, check.IsNil) + require.Nil(t, err) params, err := parseSinkURI(ctx, uri, opts) - c.Assert(err, check.IsNil) - c.Assert(params.timezone, check.Equals, expected[i]) + require.Nil(t, err) + require.Equal(t, expected[i], params.timezone) } } -func (s MySQLSinkSuite) TestParseSinkURIBadQueryString(c *check.C) { - defer testleak.AfterTest(c)() +func TestParseSinkURIBadQueryString(t *testing.T) { uris := []string{ "", "postgre://127.0.0.1:3306", @@ -904,39 +589,15 @@ func (s MySQLSinkSuite) TestParseSinkURIBadQueryString(c *check.C) { for _, uriStr := range uris { if uriStr != "" { uri, err = url.Parse(uriStr) - c.Assert(err, check.IsNil) + require.Nil(t, err) } else { uri = nil } _, err = parseSinkURI(ctx, uri, opts) - c.Assert(err, check.NotNil) + require.NotNil(t, err) } } -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 +615,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 +627,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 +649,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 +658,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 +677,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 +699,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 +718,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 +744,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 +773,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 +829,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 +888,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 +912,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 +954,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 +978,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 +1020,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 +1046,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 +1069,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 +1099,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 +1131,18 @@ 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 TestNewMySQLSink(t *testing.T) { dbIndex := 0 mockGetDBConn := func(ctx context.Context, dsnStr string) (*sql.DB, error) { defer func() { @@ -1504,13 +1151,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 +1171,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 +1190,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 +1209,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 4bfd5abf7ae..3eec6494158 100644 --- a/cdc/sink/simple_mysql_tester.go +++ b/cdc/sink/simple_mysql_tester.go @@ -185,7 +185,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)) @@ -215,7 +215,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 fc1aaa374dc..c1cf6ecda3c 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/cmd/kafka-consumer/main.go b/cmd/kafka-consumer/main.go index 1518d636b65..e1d954572a0 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.NewSink(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/tests/integration_tests/sink_hang/run.sh b/tests/integration_tests/sink_hang/run.sh index 83d78c9c050..a75689e101d 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/tiflow/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 6228225f72bea5d82b12894f20c785056824ccb2 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Thu, 30 Dec 2021 20:39:50 +0800 Subject: [PATCH 21/30] metrics(ticdc): add resolved ts and add changefeed to dataflow (#4038) (#4103) --- cdc/owner/changefeed.go | 21 +- cdc/owner/metrics.go | 18 +- metrics/grafana/ticdc.json | 5749 ++++++++++++++++++------------------ 3 files changed, 2835 insertions(+), 2953 deletions(-) diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go index 83ced2291b4..413554d1f73 100644 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -65,6 +65,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) @@ -270,6 +272,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 } @@ -291,10 +296,17 @@ func (c *changefeed) releaseResources() { 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 } @@ -477,10 +489,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() { 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 4fe6eed8ddb..8227a03aa83 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,393 @@ }, { "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": "EtcdWorker tick reactor duration", + "title": "Changefeed checkpoint", "tooltip": { "shared": true, "sort": 0, @@ -1088,6 +1315,7 @@ "type": "graph", "xaxis": { "buckets": null, + "max": null, "mode": "time", "name": null, "show": true, @@ -1095,8 +1323,7 @@ }, "yaxes": [ { - "$$hashKey": "object:2612", - "format": "s", + "format": "dateTimeAsIso", "label": null, "logBase": 1, "max": null, @@ -1104,13 +1331,12 @@ "show": true }, { - "$$hashKey": "object:2613", - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -1118,122 +1344,45 @@ "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": [] - }, + "description": "Request count of etcd operation per second", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 10 + "h": 7, + "w": 8, + "x": 9, + "y": 13 }, "hiddenSeries": false, - "id": 258, + "id": 102, "legend": { + "alignAsTable": true, "avg": false, - "current": false, - "max": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, "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", @@ -1243,28 +1392,19 @@ "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": "sum(rate(ticdc_etcd_request_count{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (capture, type)", "format": "time_series", "interval": "", - "legendFormat": "{{capture}}-p95", - "queryType": "randomWalk", + "intervalFactor": 1, + "legendFormat": "{{capture}}-{{type}}", "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": "PD etcd requests/s", "tooltip": { "shared": true, "sort": 0, @@ -1280,16 +1420,14 @@ }, "yaxes": [ { - "$$hashKey": "object:1612", - "format": "s", + "format": "short", "label": null, - "logBase": 2, + "logBase": 1, "max": null, "min": null, "show": true }, { - "$$hashKey": "object:1613", "format": "short", "label": null, "logBase": 1, @@ -1303,117 +1441,37 @@ "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": [] - }, + "description": "The number of errors that interrupt changefeed per minute ", "fill": 1, - "fillGradient": 0, "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 18 + "h": 7, + "w": 7, + "x": 17, + "y": 13 }, - "hiddenSeries": false, - "id": 260, + "id": 82, "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 - }, "percentage": false, - "pluginVersion": "6.1.6", "pointradius": 2, "points": false, "renderer": "flot", @@ -1423,27 +1481,18 @@ "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": "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" - }, - { - "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", + "title": "Exit error count/m", "tooltip": { "shared": true, "sort": 0, @@ -1459,8 +1508,7 @@ }, "yaxes": [ { - "$$hashKey": "object:2055", - "format": "bytes", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -1468,7 +1516,6 @@ "show": true }, { - "$$hashKey": "object:2056", "format": "short", "label": null, "logBase": 1, @@ -1481,131 +1528,44 @@ "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": [] - }, + "description": "The lag between changefeed checkpoint ts and the latest ts of upstream TiDB.", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, - "x": 12, - "y": 2 + "x": 0, + "y": 20 }, "hiddenSeries": false, - "id": 264, + "id": 3, "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 +1575,43 @@ "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))", + "expr": "max(ticdc_owner_checkpoint_ts_lag{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed)", + "format": "time_series", "interval": "", - "legendFormat": "{{capture}}-95", - "queryType": "randomWalk", + "intervalFactor": 1, + "legendFormat": "{{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": "sum(ticdc_processor_checkpoint_ts_lag{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture,changefeed)", + "format": "time_series", "interval": "", - "legendFormat": "{{capture}}-99", + "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, @@ -1651,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, @@ -1674,123 +1648,46 @@ "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, + "bars": true, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "", - "fieldConfig": { - "defaults": { - "unit": "s" - }, - "overrides": [] - }, + "description": "The derivative of a changefeed checkpoint, ideally it should not be 0 (means the changefeed is stopped).", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 10 + "y": 20 }, "hiddenSeries": false, - "id": 258, + "id": 2, "legend": { + "alignAsTable": true, "avg": false, - "current": false, + "current": true, + "hideEmpty": false, + "hideZero": false, "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, + "pointradius": 0.5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -1799,28 +1696,19 @@ "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": "sum(deriv(ticdc_owner_checkpoint_ts{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (changefeed) / 1000 > 0", "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": "{{capture}-p99}", - "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "EtcdWorker exec etcd txn duration percentile", + "title": "Changefeed checkpoint derivative", "tooltip": { "shared": true, "sort": 0, @@ -1836,16 +1724,14 @@ }, "yaxes": [ { - "$$hashKey": "object:1612", "format": "s", "label": null, "logBase": 2, - "max": null, - "min": null, + "max": "3600", + "min": "0.001", "show": true }, { - "$$hashKey": "object:1613", "format": "short", "label": null, "logBase": 1, @@ -1860,118 +1746,37 @@ } }, { - "cards": { - "cardPadding": null, - "cardRound": null - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolatePurples", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, "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": [] - }, + "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, - "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, - "x": 12, - "y": 18 + "x": 0, + "y": 27 }, - "hiddenSeries": false, - "id": 260, + "id": 163, "legend": { + "alignAsTable": true, "avg": false, - "current": false, - "max": false, - "min": false, + "current": true, + "max": true, + "min": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, + "links": [], "nullPointMode": "null", - "options": { - "alertThreshold": true - }, "percentage": false, - "pluginVersion": "6.1.6", - "pointradius": 2, - "points": false, + "pointradius": 1, + "points": true, "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, @@ -1979,27 +1784,19 @@ "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": "ticdc_owner_status{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}", + "format": "time_series", + "instant": false, + "intervalFactor": 1, + "legendFormat": "{{changefeed}}", "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", + "title": "The status of changefeeds", "tooltip": { "shared": true, "sort": 0, @@ -2015,8 +1812,7 @@ }, "yaxes": [ { - "$$hashKey": "object:2055", - "format": "bytes", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -2024,7 +1820,6 @@ "show": true }, { - "$$hashKey": "object:2056", "format": "short", "label": null, "logBase": 1, @@ -2037,391 +1832,262 @@ "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" - } - ], - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of captured table of TiCDC nodes ", - "fontSize": "100%", - "gridPos": { - "h": 5, - "w": 7, - "x": 0, - "y": 2 - }, - "id": 4, - "links": [], - "pageSize": null, - "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": [ - { - "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" - } - ], - "timeFrom": null, - "timeShift": null, - "title": "Changefeed table count", - "transform": "timeseries_aggregations", - "type": "table" }, { - "columns": [ - { - "text": "Current", - "value": "current" - } - ], + "aliasColors": {}, + "bars": true, + "dashLength": 10, + "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Internal resolved ts of TiCDC nodes", - "fontSize": "100%", - "gridPos": { - "h": 10, - "w": 7, - "x": 7, - "y": 2 - }, - "id": 90, - "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": "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" + "description": "Estimate the remaining time for a changefeed catch-up upstream TiDB.", + "fieldConfig": { + "defaults": { + "unit": "s" }, - { - "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" - }, - { - "columns": [ - { - "text": "Current", - "value": "current" - } - ], - "datasource": "${DS_TEST-CLUSTER}", - "description": "Internal resolved ts of captured tables", - "fontSize": "100%", + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 10, - "w": 10, - "x": 14, - "y": 2 + "h": 7, + "w": 12, + "x": 12, + "y": 27 }, - "id": 30, + "hiddenSeries": false, + "id": 253, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": false, + "linewidth": 1, "links": [], - "pageSize": null, - "scroll": true, - "showHeader": true, - "sort": { - "col": 0, - "desc": true + "nullPointMode": "null", + "options": { + "alertThreshold": 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" - } - ], + "paceLength": 10, + "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))", + "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))", "format": "time_series", - "instant": true, "interval": "", "intervalFactor": 1, - "legendFormat": "{{capture}}-{{changefeed}}-{{table}}", + "legendFormat": "{{changefeed}}", "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" } ], + "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": "Changefeed checkpoint catch-up ETA", + "tooltip": { + "shared": true, + "sort": 0, + "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": [ + "yaxes": [ { - "alias": "Time", - "align": "auto", - "dateFormat": "YYYY-MM-DD HH:mm:ss", - "pattern": "Time", - "type": "date" + "format": "s", + "label": "", + "logBase": 2, + "max": "604800", + "min": "1", + "show": true }, { - "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" + "format": "short", + "label": null, + "logBase": 1, + "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": "Sink write duration of changefeeds", + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 34 + }, + "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": "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", + "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" - }, - { - "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" + "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": true, - "cacheTimeout": null, + "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The checkpoint ts of changefeeds.", - "fill": 0, + "description": "Percentiles of sink write duration of changefeeds", + "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, - "w": 9, - "x": 0, - "y": 12 + "w": 12, + "x": 12, + "y": 34 }, - "id": 86, + "hiddenSeries": false, + "id": 35, "legend": { "alignAsTable": true, "avg": false, "current": true, "max": false, "min": false, - "rightSide": false, + "rightSide": true, "show": true, - "sideWidth": null, "total": false, "values": true }, "lines": true, - "linewidth": 2, + "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, "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": "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", - "interval": "", "intervalFactor": 1, - "legendFormat": "{{changefeed}}", + "legendFormat": "{{instance}}-p95", "refId": "A" }, { - "expr": "max(pd_cluster_tso{tidb_cluster=\"$tidb_cluster\"})", + "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", - "hide": false, - "interval": "", "intervalFactor": 1, - "legendFormat": "approximate current time (s)", + "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": "Changefeed checkpoint", + "title": "Sink write duration percentile", "tooltip": { "shared": true, "sort": 0, @@ -2430,7 +2096,6 @@ "type": "graph", "xaxis": { "buckets": null, - "max": null, "mode": "time", "name": null, "show": true, @@ -2438,9 +2103,9 @@ }, "yaxes": [ { - "format": "dateTimeAsIso", + "format": "s", "label": null, - "logBase": 1, + "logBase": 2, "max": null, "min": null, "show": true @@ -2451,7 +2116,7 @@ "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -2465,23 +2130,21 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Request count of etcd operation per second", + "description": "The number of changed rows that are written to downstream per second", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, - "w": 8, - "x": 9, - "y": 12 + "w": 12, + "x": 0, + "y": 41 }, "hiddenSeries": false, - "id": 102, + "id": 34, "legend": { "alignAsTable": true, - "avg": false, + "avg": true, "current": true, - "hideEmpty": true, - "hideZero": true, "max": true, "min": false, "rightSide": true, @@ -2507,19 +2170,27 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_etcd_request_count{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (capture, type)", + "expr": "sum (rate(ticdc_sink_txn_batch_size_sum{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", "format": "time_series", - "interval": "", "intervalFactor": 1, - "legendFormat": "{{capture}}-{{type}}", + "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" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "PD etcd requests/s", + "title": "Sink write rows count/s", "tooltip": { "shared": true, "sort": 0, @@ -2535,7 +2206,7 @@ }, "yaxes": [ { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -2562,15 +2233,17 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of errors that interrupt changefeed per minute ", + "description": "Percentiles of sink batch size", "fill": 1, + "fillGradient": 0, "gridPos": { "h": 7, - "w": 7, - "x": 17, - "y": 12 + "w": 12, + "x": 12, + "y": 41 }, - "id": 82, + "hiddenSeries": false, + "id": 36, "legend": { "alignAsTable": true, "avg": false, @@ -2586,6 +2259,10 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -2596,18 +2273,33 @@ "steppedLine": false, "targets": [ { - "expr": "sum(delta(ticdc_processor_exit_with_error_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", + "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}}", + "legendFormat": "{{capture}}-p90", "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": "Exit error count/m", + "title": "Sink write batch size percentile", "tooltip": { "shared": true, "sort": 0, @@ -2623,9 +2315,9 @@ }, "yaxes": [ { - "format": "short", + "format": "none", "label": null, - "logBase": 1, + "logBase": 2, "max": null, "min": null, "show": true @@ -2644,23 +2336,100 @@ "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": 48 + }, + "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": "The lag between changefeed checkpoint ts and the latest ts of upstream TiDB.", + "description": "Percentiles of asynchronous flush sink duration of changefeeds", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 19 + "x": 12, + "y": 48 }, "hiddenSeries": false, - "id": 3, + "id": 98, "legend": { "alignAsTable": true, "avg": false, @@ -2690,27 +2459,32 @@ "steppedLine": false, "targets": [ { - "expr": "max(ticdc_owner_checkpoint_ts_lag{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed)", + "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": "", "intervalFactor": 1, - "legendFormat": "{{changefeed}}", + "legendFormat": "{{instance}}-{{type}}-p95", "refId": "A" }, { - "expr": "sum(ticdc_processor_checkpoint_ts_lag{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture,changefeed)", + "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", - "interval": "", "intervalFactor": 1, - "legendFormat": "{{changefeed}}-{{capture}}", + "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": "Changefeed checkpoint lag", + "title": "Flush sink duration percentile", "tooltip": { "shared": true, "sort": 0, @@ -2728,9 +2502,9 @@ { "format": "s", "label": null, - "logBase": 1, + "logBase": 2, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -2747,33 +2521,111 @@ "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": 55 + }, + "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 + }, { "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": "Distribution of MySQL worker loads", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 19 + "y": 55 }, "hiddenSeries": false, - "id": 2, + "id": 95, "legend": { "alignAsTable": true, - "avg": false, + "avg": true, "current": true, - "hideEmpty": false, - "hideZero": false, - "max": false, + "max": true, "min": false, - "rightSide": true, + "rightSide": false, "show": true, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -2784,33 +2636,77 @@ "options": { "dataLinks": [] }, - "paceLength": 10, "percentage": false, - "pointradius": 0.5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, - "stack": false, + "stack": true, "steppedLine": false, "targets": [ { - "expr": "sum(deriv(ticdc_owner_checkpoint_ts{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (changefeed) / 1000 > 0", + "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": "{{changefeed}}", + "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": "", + "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": "Changefeed checkpoint derivative", + "title": "MySQL sink worker load", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -2823,11 +2719,11 @@ }, "yaxes": [ { - "format": "s", + "format": "short", "label": null, - "logBase": 2, - "max": "3600", - "min": "0.001", + "logBase": 1, + "max": null, + "min": null, "show": true }, { @@ -2845,137 +2741,118 @@ } }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, + "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 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, + "description": "The duration of executing DDL in MySQL sink", "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 26 + "y": 62 }, - "id": 163, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 165, "legend": { "alignAsTable": true, "avg": false, "current": true, "max": true, - "min": true, + "min": false, + "rightSide": true, "show": true, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, - "lines": true, - "linewidth": 1, "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 1, - "points": true, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, + "repeatDirection": "h", + "reverseYBuckets": false, "targets": [ { - "expr": "ticdc_owner_status{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}", - "format": "time_series", + "expr": "max(rate(ticdc_sink_ddl_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", "instant": false, - "intervalFactor": 1, - "legendFormat": "{{changefeed}}", + "intervalFactor": 2, + "legendFormat": "{{le}}", "refId": "A" } ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "The status of changefeeds", + "title": "MySQL sink ddl execution 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": "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 - } + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null }, { "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": "Percentiles of detecting and waiting conflict duration of MySQL sink", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 26 + "y": 62 }, "hiddenSeries": false, - "id": 253, + "id": 83, "legend": { "alignAsTable": true, "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, "pointradius": 2, "points": false, @@ -2986,23 +2863,38 @@ "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": "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", "interval": "", "intervalFactor": 1, - "legendFormat": "{{changefeed}}", + "legendFormat": "{{instance}}-p95", "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": "Changefeed checkpoint catch-up ETA", + "title": "MySQL sink conflict detect duration percentile", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -3016,10 +2908,10 @@ "yaxes": [ { "format": "s", - "label": "", + "label": null, "logBase": 2, - "max": "604800", - "min": "1", + "max": null, + "min": null, "show": true }, { @@ -3037,83 +2929,89 @@ } }, { - "cards": { - "cardPadding": 0, - "cardRound": 0 - }, - "color": { - "cardColor": "#FF9830", - "colorScale": "linear", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "max": null, - "min": 1, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Sink write duration of changefeeds", + "fill": 1, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 33 + "y": 69 }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 94, + "id": 149, "legend": { - "alignAsTable": true, "avg": false, - "current": true, - "max": true, + "current": false, + "max": false, "min": false, - "rightSide": true, "show": true, - "sort": "current", - "sortDesc": true, "total": false, - "values": true + "values": false }, + "lines": true, + "linewidth": 1, "links": [], - "repeat": null, - "repeatDirection": "h", - "reverseYBuckets": false, + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": 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}}", + "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", + "intervalFactor": 1, + "legendFormat": "{{ capture }}", "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, + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Processor Memory Consumption Per Capture", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, "show": true, - "splitFactor": null + "values": [] }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null + "yaxes": [ + { + "format": "bytes", + "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": {}, @@ -3121,24 +3019,24 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of sink write duration of changefeeds", + "description": "Percentiles of executing DDL duration of MySQL sink", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 33 + "y": 69 }, "hiddenSeries": false, - "id": 35, + "id": 166, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": false, + "max": true, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "total": false, "values": true @@ -3150,7 +3048,6 @@ "options": { "dataLinks": [] }, - "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -3161,22 +3058,25 @@ "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": "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": "{{instance}}-p95", "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))", + "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_txn_exec_duration_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (le,instance))", + "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" @@ -3186,10 +3086,10 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Sink write duration percentile", + "title": "MySQL sink ddl execution duration percentile", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -3229,36 +3129,27 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of changed rows that are written to downstream per second", "fill": 1, - "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, - "x": 0, - "y": 40 + "x": 12, + "y": 76 }, - "hiddenSeries": false, - "id": 34, + "id": 164, "legend": { - "alignAsTable": true, - "avg": true, - "current": true, - "max": true, + "avg": false, + "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, @@ -3269,27 +3160,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(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" - }, - { - "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": "Processor Memory Consumption Per Changefeed", "tooltip": { "shared": true, "sort": 0, @@ -3305,7 +3187,7 @@ }, "yaxes": [ { - "format": "none", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -3326,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": "Percentiles of sink batch size", + "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": 12, - "y": 40 + "x": 0, + "y": 4 }, "hiddenSeries": false, - "id": 36, + "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 @@ -3372,33 +3421,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_puller_txn_collect_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (changefeed, capture, type)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}-p90", + "legendFormat": "{{changefeed}}-{{capture}}-{{type}}", "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": "Puller output events/s", "tooltip": { "shared": true, "sort": 0, @@ -3416,7 +3450,7 @@ { "format": "none", "label": null, - "logBase": 2, + "logBase": 1, "max": null, "min": null, "show": true @@ -3427,7 +3461,7 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -3435,107 +3469,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 puller outputs", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, + "h": 4, "w": 12, "x": 12, - "y": 47 + "y": 4 }, "hiddenSeries": false, - "id": 98, + "id": 229, "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 +3515,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_puller_txn_collect_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture, type)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}-p95", + "legendFormat": "{{changefeed}}-{{capture}}-{{type}}", "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": "Puller output events", "tooltip": { "shared": true, "sort": 0, @@ -3592,111 +3535,33 @@ "type": "graph", "xaxis": { "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "s", - "label": null, - "logBase": 2, - "max": null, - "min": null, - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "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, + "mode": "time", + "name": null, "show": true, - "splitFactor": null + "values": [] }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": 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": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { "aliasColors": {}, @@ -3704,27 +3569,25 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Distribution of MySQL worker loads", + "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": 12, - "y": 54 + "x": 0, + "y": 8 }, "hiddenSeries": false, - "id": 95, + "id": 228, "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 +3598,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_sorter_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (changefeed, capture, type)", "format": "time_series", - "hide": true, - "interval": "", "intervalFactor": 1, - "legendFormat": "{{capture}}-{{bucket}}", + "legendFormat": "{{changefeed}}-{{capture}}-{{type}}", "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": "Sorter output events/s", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -3818,7 +3636,7 @@ }, "yaxes": [ { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -3831,7 +3649,7 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -3840,82 +3658,98 @@ } }, { - "cards": { - "cardPadding": 0, - "cardRound": 0 - }, - "color": { - "cardColor": "#FF9830", - "colorScale": "linear", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "max": null, - "min": 1, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The duration of executing DDL in MySQL sink", + "description": "The total number of events that sorter outputs", + "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 7, + "h": 4, "w": 12, - "x": 0, - "y": 61 + "x": 12, + "y": 8 }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 165, + "hiddenSeries": false, + "id": 220, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": true, "max": true, "min": false, - "rightSide": true, + "rightSide": false, "show": true, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, + "lines": true, + "linewidth": 1, "links": [], - "repeatDirection": "h", - "reverseYBuckets": false, + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": 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}}", + "expr": "sum(ticdc_sorter_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture, changefeed, type)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}-{{type}}", "refId": "A" } ], - "title": "MySQL sink ddl execution duration", + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Sorter output events", "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": "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": {}, @@ -3923,19 +3757,19 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of detecting and waiting conflict duration of MySQL sink", + "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": 61 + "x": 0, + "y": 12 }, "hiddenSeries": false, - "id": 83, + "id": 219, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": true, "max": true, @@ -3952,6 +3786,7 @@ "options": { "dataLinks": [] }, + "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -3962,38 +3797,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(rate(ticdc_mounter_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture,changefeed)", "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": "Mounter output events/s", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -4006,9 +3824,9 @@ }, "yaxes": [ { - "format": "s", + "format": "none", "label": null, - "logBase": 2, + "logBase": 1, "max": null, "min": null, "show": true @@ -4019,7 +3837,7 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -4033,27 +3851,36 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "The total number of events that mounter outputs", "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 4, "w": 12, - "x": 0, - "y": 68 + "x": 12, + "y": 12 }, - "id": 149, + "hiddenSeries": false, + "id": 224, "legend": { + "alignAsTable": false, "avg": false, - "current": false, - "max": false, + "current": true, + "max": true, "min": false, + "rightSide": false, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -4064,10 +3891,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)", + "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" } ], @@ -4075,7 +3902,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Processor Memory Consumption Per Capture", + "title": "Mounter output events", "tooltip": { "shared": true, "sort": 0, @@ -4091,7 +3918,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -4118,19 +3945,19 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Percentiles of executing DDL duration of MySQL sink", + "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": 68 + "x": 0, + "y": 16 }, "hiddenSeries": false, - "id": 166, + "id": 223, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": true, "max": true, @@ -4147,6 +3974,7 @@ "options": { "dataLinks": [] }, + "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -4157,38 +3985,21 @@ "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(rate(ticdc_sink_table_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) 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": "Table sink output events/s", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -4201,9 +4012,9 @@ }, "yaxes": [ { - "format": "s", + "format": "none", "label": null, - "logBase": 2, + "logBase": 1, "max": null, "min": null, "show": true @@ -4214,7 +4025,7 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -4228,27 +4039,36 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "The total number of events that table sinks emit", "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 4, "w": 12, "x": 12, - "y": 75 + "y": 16 }, - "id": 164, + "hiddenSeries": false, + "id": 221, "legend": { + "alignAsTable": false, "avg": false, - "current": false, - "max": false, + "current": true, + "max": true, "min": false, + "rightSide": false, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, "links": [], "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -4259,10 +4079,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(ticdc_sink_table_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{ capture }}-{{ changefeed }}", + "legendFormat": "{{changefeed}}-{{capture}}", "refId": "A" } ], @@ -4270,7 +4090,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Processor Memory Consumption Per Changefeed", + "title": "Table sink output events", "tooltip": { "shared": true, "sort": 0, @@ -4282,213 +4102,48 @@ "mode": "time", "name": null, "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "bytes", - "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 - } - }, - { - "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 - } - ] - } - ] - }, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 69 - }, - "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" - ] - } - } - }, + "values": [] + }, + "yaxes": [ { - "id": "organize", - "options": { - "excludeByName": {}, - "indexByName": {}, - "renameByName": { - "Value #A": "last-snap-ts" - } - } + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true }, { - "id": "sortBy", - "options": { - "fields": {}, - "sort": [ - { - "field": "last-snap-ts" - } - ] - } + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true } ], - "type": "table" - } - ], - "title": "Changefeed", - "type": "row" - }, - { - "collapsed": true, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 2 - }, - "id": 206, - "panels": [ + "yaxis": { + "align": false, + "alignLevel": null + } + }, { "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 total number of events that buffer sink outputs to backend sink per second", "fill": 1, "fillGradient": 0, "gridPos": { "h": 4, "w": 12, "x": 0, - "y": 3 + "y": 20 }, "hiddenSeries": false, - "id": 218, + "id": 222, "legend": { "alignAsTable": false, "avg": false, @@ -4518,10 +4173,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_sink_buffer_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (changefeed, capture)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{changefeed}}-{{capture}}", "refId": "A" } ], @@ -4529,7 +4184,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Puller output events/s", + "title": "Buffer sink output events/s", "tooltip": { "shared": true, "sort": 0, @@ -4572,17 +4227,17 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The total number of events that puller outputs", + "description": "The total number of events that buffer sink emits", "fill": 1, "fillGradient": 0, "gridPos": { "h": 4, "w": 12, "x": 12, - "y": 3 + "y": 20 }, "hiddenSeries": false, - "id": 229, + "id": 226, "legend": { "alignAsTable": false, "avg": false, @@ -4612,10 +4267,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_puller_txn_collect_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (instance, type)", + "expr": "sum(ticdc_sink_buffer_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{changefeed}}-{{capture}}", "refId": "A" } ], @@ -4623,7 +4278,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Puller output events", + "title": "Buffer sink output events", "tooltip": { "shared": true, "sort": 0, @@ -4666,17 +4321,17 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of events that sorter outputs to puller \n per second", + "description": "The number of rows(events) that are processed by sink per second", "fill": 1, "fillGradient": 0, "gridPos": { "h": 4, "w": 12, "x": 0, - "y": 7 + "y": 24 }, "hiddenSeries": false, - "id": 228, + "id": 227, "legend": { "alignAsTable": false, "avg": false, @@ -4685,6 +4340,8 @@ "min": false, "rightSide": false, "show": true, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -4695,7 +4352,6 @@ "options": { "dataLinks": [] }, - "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -4706,10 +4362,11 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_sorter_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture,changefeed,type)", + "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}}-{{type}}", + "legendFormat": "{{changefeed}}-{{capture}}", "refId": "A" } ], @@ -4717,10 +4374,10 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Sorter output events/s", + "title": "Sink output events/s", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -4760,17 +4417,17 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The total number of events that sorter outputs", + "description": "The number of rows(events) that are processed by sink", "fill": 1, "fillGradient": 0, "gridPos": { "h": 4, "w": 12, "x": 12, - "y": 7 + "y": 24 }, "hiddenSeries": false, - "id": 220, + "id": 225, "legend": { "alignAsTable": false, "avg": false, @@ -4779,6 +4436,8 @@ "min": false, "rightSide": false, "show": true, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -4789,7 +4448,6 @@ "options": { "dataLinks": [] }, - "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -4800,10 +4458,11 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_sorter_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (capture, changefeed, type)", + "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": "{{capture}}-{{type}}", + "legendFormat": "{{changefeed}}-{{capture}}", "refId": "A" } ], @@ -4811,10 +4470,10 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Sorter output events", + "title": "Sink output events", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -4854,17 +4513,17 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of events that mounter outputs to sink per second", + "description": "The number of rows that sink flushes to downstream per second", "fill": 1, "fillGradient": 0, "gridPos": { "h": 4, "w": 12, "x": 0, - "y": 11 + "y": 28 }, "hiddenSeries": false, - "id": 219, + "id": 108, "legend": { "alignAsTable": false, "avg": false, @@ -4873,6 +4532,8 @@ "min": false, "rightSide": false, "show": true, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -4883,7 +4544,6 @@ "options": { "dataLinks": [] }, - "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -4894,10 +4554,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_sink_total_flushed_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (changefeed, capture)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}", + "legendFormat": "{{changefeed}}-{{capture}}", "refId": "A" } ], @@ -4905,10 +4565,10 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Mounter output events/s", + "title": "Sink flush rows/s", "tooltip": { "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -4929,7 +4589,7 @@ "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -4948,17 +4608,17 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The total number of events that mounter outputs", + "description": "The number of rows(events) that are flushed by sink", "fill": 1, "fillGradient": 0, "gridPos": { "h": 4, "w": 12, "x": 12, - "y": 11 + "y": 28 }, "hiddenSeries": false, - "id": 224, + "id": 240, "legend": { "alignAsTable": false, "avg": false, @@ -4967,6 +4627,8 @@ "min": false, "rightSide": false, "show": true, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -4977,7 +4639,6 @@ "options": { "dataLinks": [] }, - "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -4988,18 +4649,156 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_mounter_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (instance)", + "expr": "sum(ticdc_sink_total_flushed_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed, capture)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Sink flush rows", + "tooltip": { + "shared": true, + "sort": 2, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "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 + } + } + ], + "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", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 8, + "x": 0, + "y": 5 + }, + "hiddenSeries": false, + "id": 15, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "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": [ + { + "alias": "/.*-rpc/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_kvclient_event_feed_count{tidb_cluster=\"$tidb_cluster\"}) by (instance)", + "format": "time_series", + "intervalFactor": 1, + "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}}", - "refId": "A" + "legendFormat": "{{instance}}-rpc-handled", + "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Mounter output events", + "title": "Eventfeed count", "tooltip": { "shared": true, "sort": 0, @@ -5015,7 +4814,7 @@ }, "yaxes": [ { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -5042,24 +4841,25 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of events that table sorter outputs to buffer sink per second", + "decimals": null, + "description": "Percentiles of Eventfeed message size", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 4, - "w": 12, - "x": 0, - "y": 15 + "h": 7, + "w": 8, + "x": 8, + "y": 5 }, "hiddenSeries": false, - "id": 223, + "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 @@ -5082,18 +4882,25 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_sink_table_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (instance)", + "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": "{{instance}}", + "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": "Table sink output events/s", + "title": "Event size percentile", "tooltip": { "shared": true, "sort": 0, @@ -5109,7 +4916,7 @@ }, "yaxes": [ { - "format": "none", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -5122,7 +4929,7 @@ "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -5132,28 +4939,30 @@ }, { "aliasColors": {}, - "bars": false, + "bars": true, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The total number of events that table sinks emit", + "description": "The number of errors that interrupt Eventfeed RPC", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 4, - "w": 12, - "x": 12, - "y": 15 + "h": 7, + "w": 8, + "x": 16, + "y": 5 }, "hiddenSeries": false, - "id": 221, + "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 @@ -5176,18 +4985,37 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_sink_table_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (instance)", + "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": "{{instance}}", + "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": "Table sink output events", + "title": "Eventfeed error/m", "tooltip": { "shared": true, "sort": 0, @@ -5203,7 +5031,7 @@ }, "yaxes": [ { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -5230,21 +5058,23 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The total number of events that buffer sink outputs to backend 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": 19 + "y": 12 }, "hiddenSeries": false, - "id": 222, + "id": 29, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": true, + "hideZero": true, "max": true, "min": false, "rightSide": false, @@ -5270,10 +5100,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_sink_buffer_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (instance)", + "expr": "sum(rate(ticdc_kvclient_pull_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (instance, type)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}", + "legendFormat": "{{instance}}-{{type}}", "refId": "A" } ], @@ -5281,7 +5111,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Buffer sink output events/s", + "title": "KV client receive events/s", "tooltip": { "shared": true, "sort": 0, @@ -5297,7 +5127,7 @@ }, "yaxes": [ { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -5310,7 +5140,7 @@ "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -5324,19 +5154,19 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The total number of events that buffer sink emits", + "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": 19 + "h": 7, + "w": 8, + "x": 8, + "y": 12 }, "hiddenSeries": false, - "id": 226, + "id": 5, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, "max": true, @@ -5364,10 +5194,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_sink_buffer_sink_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" } ], @@ -5375,7 +5205,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Buffer sink output events", + "title": "Puller output events/s", "tooltip": { "shared": true, "sort": 0, @@ -5418,27 +5248,26 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of rows(events) that are processed by 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": 23 + "h": 7, + "w": 8, + "x": 16, + "y": 12 }, "hiddenSeries": false, - "id": 227, + "id": 107, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, "max": true, "min": false, "rightSide": false, "show": true, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, @@ -5449,6 +5278,7 @@ "options": { "dataLinks": [] }, + "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -5459,22 +5289,28 @@ "steppedLine": false, "targets": [ { - "expr": "sum (rate(ticdc_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (capture)", + "expr": "sum(ticdc_mounter_input_chan_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", "format": "time_series", - "interval": "", "intervalFactor": 1, - "legendFormat": "{{capture}}", + "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": "Sink output events/s", + "title": "Sink/Mounter buffer size", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -5487,7 +5323,7 @@ }, "yaxes": [ { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -5500,7 +5336,7 @@ "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -5508,33 +5344,108 @@ "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 number of rows(events) that are processed by sink", + "description": "Percentiles of sorting events duration", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 4, + "h": 7, "w": 12, "x": 12, - "y": 23 + "y": 19 }, "hiddenSeries": false, - "id": 225, + "id": 53, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, @@ -5545,6 +5456,7 @@ "options": { "dataLinks": [] }, + "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -5555,22 +5467,29 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (capture, type)", + "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", - "interval": "", "intervalFactor": 1, - "legendFormat": "{{capture}}", + "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": "Sink output events", + "title": "Entry sorter sort duration percentile", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -5583,9 +5502,9 @@ }, "yaxes": [ { - "format": "none", + "format": "s", "label": null, - "logBase": 1, + "logBase": 2, "max": null, "min": null, "show": true @@ -5604,33 +5523,108 @@ "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 merging sorted events", + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 26 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 105, + "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_merge_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "Entry sorter merge 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 that sink flushes to downstream per second", + "description": "Percentiles of merging sorted events duration", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 4, + "h": 7, "w": 12, - "x": 0, - "y": 27 + "x": 12, + "y": 26 }, "hiddenSeries": false, - "id": 108, + "id": 106, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, @@ -5641,6 +5635,7 @@ "options": { "dataLinks": [] }, + "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -5651,21 +5646,29 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_sink_total_flushed_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", + "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}}", + "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": "Sink flush rows/s", + "title": "Entry sorter merge duration percentile", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -5678,20 +5681,20 @@ }, "yaxes": [ { - "format": "none", + "format": "s", "label": null, - "logBase": 1, + "logBase": 2, "max": null, "min": null, "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -5699,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 flushed by sink", - "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": 12, - "y": 27 + "y": 33 }, "hiddenSeries": false, - "id": 240, + "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, @@ -5736,6 +5815,7 @@ "options": { "dataLinks": [] }, + "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -5746,19 +5826,28 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_sink_total_flushed_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (capture, type)", + "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 flush rows", + "title": "Mounter unmarshal duration percentile", "tooltip": { "shared": true, "sort": 2, @@ -5774,7 +5863,7 @@ }, "yaxes": [ { - "format": "none", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -5782,7 +5871,7 @@ "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -5794,44 +5883,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": "The number of KV client dispatched event per second", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, - "w": 8, + "w": 12, "x": 0, - "y": 32 + "y": 40 }, "hiddenSeries": false, - "id": 15, + "id": 31, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "max": false, + "hideEmpty": true, + "hideZero": true, + "max": true, "min": false, "rightSide": true, "show": true, @@ -5852,7 +5928,7 @@ "renderer": "flot", "seriesOverrides": [ { - "alias": "/.*-rpc/", + "alias": "/.*batch-resolved/", "yaxis": 2 } ], @@ -5861,41 +5937,25 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_kvclient_event_feed_count{tidb_cluster=\"$tidb_cluster\"}) by (instance)", + "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": "{{instance}}", + "legendFormat": "{{capture}}-{{changefeed}}-{{type}}", "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)", + "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": "{{instance}}-rpc", + "legendFormat": "{{capture}}-{{changefeed}}-batch-resolved", "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 dispatch events/s", "tooltip": { "shared": true, "sort": 0, @@ -5932,29 +5992,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": "The size of batch resolved ts message from TiKV", + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 40 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 97, + "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_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" + } + ], + "title": "KV client batch resolved size", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "none", + "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}", - "decimals": null, - "description": "Percentiles of Eventfeed message size", + "description": "The number of regions that are being scanned", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, - "w": 8, - "x": 8, - "y": 32 + "w": 12, + "x": 0, + "y": 47 }, "hiddenSeries": false, - "id": 17, + "id": 177, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": false, + "hideEmpty": true, + "hideZero": true, + "max": true, "min": false, "rightSide": true, "show": true, @@ -5979,25 +6116,18 @@ "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))", + "expr": "sum(ticdc_kvclient_region_token{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture, changefeed, store)", "format": "time_series", "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 scanning regions", "tooltip": { "shared": true, "sort": 0, @@ -6013,7 +6143,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -6036,21 +6166,21 @@ }, { "aliasColors": {}, - "bars": true, + "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of errors that interrupt Eventfeed RPC", + "description": "Active stream count of each gRPC connection", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, - "w": 8, - "x": 16, - "y": 32 + "w": 12, + "x": 12, + "y": 47 }, "hiddenSeries": false, - "id": 28, + "id": 188, "legend": { "alignAsTable": true, "avg": false, @@ -6082,37 +6212,18 @@ "steppedLine": false, "targets": [ { - "expr": "sum(increase(ticdc_kvclient_event_feed_error_count{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (type)", + "expr": "sum(ticdc_kvclient_grpc_stream_count{tidb_cluster=\"$tidb_cluster\"}) by (store)", "format": "time_series", - "hide": false, - "interval": "1m", "intervalFactor": 1, - "legendFormat": "{{type}}", + "legendFormat": "{{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": "KV client gRPC stream count", "tooltip": { "shared": true, "sort": 0, @@ -6155,17 +6266,23 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of KV client received events from TiKV per seconds", + "description": "The number of regions that have not connected to TiKV", + "fieldConfig": { + "defaults": { + "links": [] + }, + "overrides": [] + }, "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, - "w": 8, + "w": 12, "x": 0, - "y": 39 + "y": 54 }, "hiddenSeries": false, - "id": 29, + "id": 251, "legend": { "alignAsTable": true, "avg": false, @@ -6174,7 +6291,7 @@ "hideZero": true, "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "total": false, "values": true @@ -6184,7 +6301,7 @@ "links": [], "nullPointMode": "null", "options": { - "dataLinks": [] + "alertThreshold": true }, "paceLength": 10, "percentage": false, @@ -6197,10 +6314,12 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_kvclient_pull_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (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}}", + "legendFormat": "{{changefeed}}-{{capture}}-{{store}}", "refId": "A" } ], @@ -6208,7 +6327,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "KV client receive events/s", + "title": "KV client cached regions", "tooltip": { "shared": true, "sort": 0, @@ -6251,24 +6370,32 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of events that puller outputs to sorter \n per second", + "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": 8, - "y": 39 + "w": 12, + "x": 12, + "y": 54 }, "hiddenSeries": false, - "id": 5, + "id": 252, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": true, + "hideZero": true, "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "total": false, "values": true @@ -6278,7 +6405,7 @@ "links": [], "nullPointMode": "null", "options": { - "dataLinks": [] + "alertThreshold": true }, "paceLength": 10, "percentage": false, @@ -6291,10 +6418,12 @@ "steppedLine": false, "targets": [ { - "expr": "sum (rate(ticdc_puller_txn_collect_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (instance, 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", + "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{changefeed}}-{{capture}}-{{store}}", "refId": "A" } ], @@ -6302,7 +6431,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Puller output events/s", + "title": "Estimate remaining time for initialization", "tooltip": { "shared": true, "sort": 0, @@ -6318,9 +6447,9 @@ }, "yaxes": [ { - "format": "none", - "label": null, - "logBase": 1, + "format": "s", + "label": "", + "logBase": 2, "max": null, "min": null, "show": true @@ -6338,44 +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 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": 5 }, - "hiddenSeries": false, - "id": 107, + "id": 131, "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, @@ -6386,25 +6519,18 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_mounter_input_chan_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "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": "{{capture}}-mounter input chan", + "legendFormat": "{{capture}}-{{changefeed}}", "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 intake rate", "tooltip": { "shared": true, "sort": 0, @@ -6441,119 +6567,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": 5 }, - "hiddenSeries": false, - "id": 53, + "id": 132, "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, @@ -6564,26 +6604,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(rate(ticdc_sorter_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture,changefeed)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}-p999", + "legendFormat": "{{capture}}-{{changefeed}}", "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 event output rate", "tooltip": { "shared": true, "sort": 0, @@ -6599,9 +6631,9 @@ }, "yaxes": [ { - "format": "s", + "format": "short", "label": null, - "logBase": 2, + "logBase": 1, "max": null, "min": null, "show": true @@ -6620,119 +6652,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 merging sorted events", - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 53 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 105, - "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_merge_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "title": "Entry sorter merge 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 merging sorted events duration", "fill": 1, - "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, - "x": 12, - "y": 53 + "x": 0, + "y": 13 }, - "hiddenSeries": false, - "id": 106, + "id": 133, "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, @@ -6743,26 +6689,18 @@ "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))", + "expr": "sum(ticdc_sorter_on_disk_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_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": "Entry sorter merge duration percentile", + "title": "Unified Sorter on disk data size", "tooltip": { "shared": true, "sort": 0, @@ -6778,11 +6716,11 @@ }, "yaxes": [ { - "format": "s", + "format": "bytes", "label": null, - "logBase": 2, + "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -6799,120 +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 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, - "fillGradient": 0, + "fill": 1, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 12, - "y": 60 + "y": 13 }, - "hiddenSeries": false, - "id": 55, + "id": 134, "legend": { - "alignAsTable": true, "avg": false, - "current": true, - "max": true, + "current": false, + "max": false, "min": false, - "rightSide": true, "show": true, - "sort": "current", - "sortDesc": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -6923,31 +6774,21 @@ "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": "sum(ticdc_sorter_in_memory_data_size_gauge{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}) by (capture)", "format": "time_series", - "hide": false, "intervalFactor": 1, - "legendFormat": "{{capture}}-p99", + "legendFormat": "{{capture}}", "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": "Mounter unmarshal duration percentile", + "title": "Unified Sorter in-memory data size", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -6960,15 +6801,15 @@ }, "yaxes": [ { - "format": "s", + "format": "bytes", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -6982,170 +6823,116 @@ } }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, + "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}", - "description": "The number of KV client dispatched event per second", - "fill": 1, - "fillGradient": 0, "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 0, - "y": 67 + "y": 21 }, - "hiddenSeries": false, - "id": 31, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 135, "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "hideEmpty": true, - "hideZero": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "total": false, - "values": true + "show": false }, - "lines": true, - "linewidth": 1, "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/.*batch-resolved/", - "yaxis": 2 - } - ], - "spaceLength": 10, - "stack": false, - "steppedLine": false, + "reverseYBuckets": false, "targets": [ { - "expr": "sum(rate(ticdc_kvclient_send_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (capture, changefeed, type)", - "format": "time_series", + "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": "{{capture}}-{{changefeed}}-{{type}}", + "legendFormat": "{{le}}", "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": "KV client dispatch events/s", + "title": "Unified Sorter flush sizes", "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" + "show": true, + "showHistogram": false }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": null, + "format": "none", + "logBase": 1, + "max": null, + "min": null, "show": true, - "values": [] + "splitFactor": 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 - } + "yBucketBound": "auto", + "yBucketNumber": null, + "yBucketSize": null }, { "cards": { - "cardPadding": 0, - "cardRound": 0 + "cardPadding": null, + "cardRound": null }, "color": { - "cardColor": "#FF9830", - "colorScale": "linear", - "colorScheme": "interpolateSpectral", + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateBlues", "exponent": 0.5, - "min": 0, "mode": "spectrum" }, "dataFormat": "tsbuckets", "datasource": "${DS_TEST-CLUSTER}", - "description": "The size of batch resolved ts message from TiKV", "gridPos": { - "h": 7, + "h": 8, "w": 12, "x": 12, - "y": 67 + "y": 21 }, "heatmap": {}, "hideZeroBuckets": true, "highlightCards": true, - "id": 97, + "id": 136, "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_kvclient_batch_resolved_event_size_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le)", + "expr": "sum(rate(ticdc_sorter_merge_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": "KV client batch resolved size", + "timeFrom": null, + "timeShift": null, + "title": "Unified Sorter merge size", "tooltip": { "show": true, - "showHistogram": true + "showHistogram": false }, - "tooltipDecimals": 1, "type": "heatmap", "xAxis": { "show": true @@ -7153,7 +6940,7 @@ "xBucketNumber": null, "xBucketSize": null, "yAxis": { - "decimals": 1, + "decimals": null, "format": "none", "logBase": 1, "max": null, @@ -7161,27 +6948,38 @@ "show": true, "splitFactor": null }, - "yBucketBound": "upper", + "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": "The number of regions that are being scanned", "fill": 1, - "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 74 + "y": 16 }, - "hiddenSeries": false, - "id": 177, + "id": 289, "legend": { "alignAsTable": true, "avg": false, @@ -7199,10 +6997,6 @@ "linewidth": 1, "links": [], "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -7213,10 +7007,11 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_kvclient_region_token{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture, changefeed, store)", + "expr": "etcd_debugging_mvcc_db_total_size_in_bytes{tidb_cluster=\"$tidb_cluster\", job=\"pd\"}", "format": "time_series", + "hide": false, "intervalFactor": 1, - "legendFormat": "{{changefeed}}-{{capture}}-{{store}}", + "legendFormat": "{{instance}}", "refId": "A" } ], @@ -7224,7 +7019,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "KV client scanning regions", + "title": "Etcd MVCC DB total size", "tooltip": { "shared": true, "sort": 0, @@ -7240,7 +7035,7 @@ }, "yaxes": [ { - "format": "short", + "format": "decbytes", "label": null, "logBase": 1, "max": null, @@ -7253,7 +7048,7 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -7267,23 +7062,21 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Active stream count of each gRPC connection", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 74 + "y": 16 }, "hiddenSeries": false, - "id": 188, + "id": 114, "legend": { "alignAsTable": true, "avg": false, "current": true, "hideEmpty": true, - "hideZero": true, "max": true, "min": false, "rightSide": true, @@ -7309,18 +7102,32 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_kvclient_grpc_stream_count{tidb_cluster=\"$tidb_cluster\"}) by (store)", + "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", "intervalFactor": 1, - "legendFormat": "{{store}}", + "legendFormat": "p999-{{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": "KV client gRPC stream count", + "title": "Etcd health check duration", "tooltip": { "shared": true, "sort": 0, @@ -7336,7 +7143,7 @@ }, "yaxes": [ { - "format": "short", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -7358,108 +7165,80 @@ } }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, + "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": "The number of regions that have not connected to TiKV", + "description": "", "fieldConfig": { - "defaults": { - "links": [] - }, + "defaults": {}, "overrides": [] }, - "fill": 1, - "fillGradient": 0, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 81 + "y": 23 }, - "hiddenSeries": false, - "id": 251, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 267, "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": { - "alertThreshold": true + "show": true }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, + "pluginVersion": "6.1.6", + "reverseYBuckets": false, "targets": [ { "exemplar": true, - "expr": "sum(ticdc_kvclient_cached_region{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture, changefeed, store)", - "format": "time_series", + "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": 1, - "legendFormat": "{{changefeed}}-{{capture}}-{{store}}", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "queryType": "randomWalk", "refId": "A" } ], - "thresholds": [], "timeFrom": null, - "timeRegions": [], "timeShift": null, - "title": "KV client cached regions", + "title": "EtcdWorker exec etcd txn 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": "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 - } + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null }, { "aliasColors": {}, @@ -7467,11 +7246,9 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Estimate the remaining time for a changefeed initialization (on a specific capture)", + "description": "", "fieldConfig": { - "defaults": { - "unit": "s" - }, + "defaults": {}, "overrides": [] }, "fill": 1, @@ -7480,32 +7257,27 @@ "h": 7, "w": 12, "x": 12, - "y": 81 + "y": 23 }, "hiddenSeries": false, - "id": 252, + "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": { "alertThreshold": true }, - "paceLength": 10, "percentage": false, + "pluginVersion": "6.1.6", "pointradius": 2, "points": false, "renderer": "flot", @@ -7516,19 +7288,26 @@ "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))", - "format": "time_series", + "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": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}-{{capture}}-{{store}}", + "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": "Estimate remaining time for initialization", + "title": "EtcdWorker tick reactor duration", "tooltip": { "shared": true, "sort": 0, @@ -7545,14 +7324,14 @@ "yaxes": [ { "format": "s", - "label": "", - "logBase": 2, + "label": null, + "logBase": 1, "max": null, "min": null, "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -7564,35 +7343,106 @@ "align": false, "alignLevel": null } - } - ], - "title": "Events", - "type": "row" - }, - { - "collapsed": true, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 4 - }, - "id": 130, - "panels": [ + }, + { + "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": 30 + }, + "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, + "h": 7, "w": 12, - "x": 0, - "y": 5 + "x": 12, + "y": 30 }, - "id": 131, + "hiddenSeries": false, + "id": 258, "legend": { "avg": false, "current": false, @@ -7604,9 +7454,12 @@ }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "6.1.6", "pointradius": 2, "points": false, "renderer": "flot", @@ -7616,18 +7469,31 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_sorter_consume_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture,changefeed)", + "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": "{{capture}}-{{changefeed}}", + "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": "Unified Sorter intake rate", + "title": "EtcdWorker exec etcd txn duration percentile", "tooltip": { "shared": true, "sort": 0, @@ -7643,9 +7509,9 @@ }, "yaxes": [ { - "format": "short", + "format": "s", "label": null, - "logBase": 1, + "logBase": 2, "max": null, "min": null, "show": true @@ -7664,20 +7530,100 @@ "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": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 5 + "y": 37 }, - "id": 132, + "hiddenSeries": false, + "id": 260, "legend": { "avg": false, "current": false, @@ -7689,9 +7635,12 @@ }, "lines": true, "linewidth": 1, - "links": [], "nullPointMode": "null", + "options": { + "alertThreshold": true + }, "percentage": false, + "pluginVersion": "6.1.6", "pointradius": 2, "points": false, "renderer": "flot", @@ -7701,18 +7650,27 @@ "steppedLine": false, "targets": [ { - "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": "{{capture}}-{{changefeed}}", + "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": "Unified Sorter event output rate", + "title": "EtcdWorker txn size percentile", "tooltip": { "shared": true, "sort": 0, @@ -7728,7 +7686,7 @@ }, "yaxes": [ { - "format": "short", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -7755,29 +7713,39 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "The time consumed of writing WAL into the persistent storage in .99", + "editable": true, + "error": false, "fill": 1, + "grid": {}, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 13 + "y": 44 }, - "id": 133, + "id": 291, "legend": { + "alignAsTable": true, "avg": false, - "current": false, - "max": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, "min": false, + "rightSide": true, "show": true, + "sideWidth": 300, "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": [], @@ -7786,22 +7754,25 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_sorter_on_disk_data_size_gauge{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}) by (capture)", + "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", - "intervalFactor": 1, - "legendFormat": "{{capture}}", - "refId": "A" + "intervalFactor": 2, + "legendFormat": "{{instance}}", + "metric": "", + "refId": "A", + "step": 4 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Unified Sorter on disk data size", + "title": "Etcd 99% WAL fsync duration", "tooltip": { + "msResolution": false, "shared": true, "sort": 0, - "value_type": "individual" + "value_type": "cumulative" }, "type": "graph", "xaxis": { @@ -7813,11 +7784,11 @@ }, "yaxes": [ { - "format": "bytes", + "format": "s", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -7840,29 +7811,37 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", + "description": "The time consumed of handling etcd transactions in .99", + "editable": true, + "error": false, "fill": 1, + "grid": {}, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 13 + "y": 44 }, - "id": 134, + "id": 290, "legend": { + "alignAsTable": true, "avg": false, - "current": false, - "max": false, + "current": true, + "max": true, "min": false, + "rightSide": true, "show": true, + "sideWidth": 300, "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": [], @@ -7871,22 +7850,24 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_sorter_in_memory_data_size_gauge{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}) by (capture)", + "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", - "intervalFactor": 1, - "legendFormat": "{{capture}}", - "refId": "A" + "intervalFactor": 2, + "legendFormat": "{{instance}} {{result}}", + "refId": "A", + "step": 4 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Unified Sorter in-memory data size", + "title": "Etcd 99% Handle transactions duration", "tooltip": { + "msResolution": false, "shared": true, "sort": 0, - "value_type": "individual" + "value_type": "cumulative" }, "type": "graph", "xaxis": { @@ -7898,11 +7879,11 @@ }, "yaxes": [ { - "format": "bytes", + "format": "s", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -7918,139 +7899,9 @@ "align": false, "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", + "title": "Etcd", "type": "row" }, { @@ -8059,7 +7910,7 @@ "h": 1, "w": 24, "x": 0, - "y": 5 + "y": 6 }, "id": 58, "panels": [ @@ -8076,7 +7927,7 @@ "h": 7, "w": 12, "x": 0, - "y": 5 + "y": 7 }, "id": 60, "legend": { @@ -8202,7 +8053,7 @@ "h": 7, "w": 12, "x": 12, - "y": 5 + "y": 7 }, "id": 74, "legend": { @@ -8310,7 +8161,7 @@ "h": 7, "w": 12, "x": 0, - "y": 12 + "y": 14 }, "id": 147, "legend": { @@ -8409,7 +8260,7 @@ "h": 7, "w": 12, "x": 12, - "y": 12 + "y": 14 }, "id": 194, "legend": { @@ -8573,7 +8424,7 @@ "h": 7, "w": 7, "x": 0, - "y": 19 + "y": 21 }, "id": 152, "legend": { @@ -8704,7 +8555,7 @@ "h": 7, "w": 5, "x": 7, - "y": 19 + "y": 21 }, "id": 153, "legend": { @@ -8800,7 +8651,7 @@ "h": 7, "w": 12, "x": 12, - "y": 19 + "y": 21 }, "id": 70, "legend": { @@ -8902,7 +8753,7 @@ "h": 7, "w": 12, "x": 0, - "y": 26 + "y": 28 }, "heatmap": {}, "hideZeroBuckets": true, @@ -8974,7 +8825,7 @@ "h": 7, "w": 6, "x": 12, - "y": 26 + "y": 28 }, "id": 72, "legend": { @@ -9069,7 +8920,7 @@ "h": 7, "w": 6, "x": 18, - "y": 26 + "y": 28 }, "id": 140, "legend": { @@ -9181,7 +9032,7 @@ "h": 7, "w": 12, "x": 0, - "y": 33 + "y": 35 }, "id": 78, "legend": { @@ -9289,7 +9140,7 @@ "h": 7, "w": 6, "x": 12, - "y": 33 + "y": 35 }, "id": 76, "legend": { @@ -9390,7 +9241,7 @@ "h": 7, "w": 6, "x": 18, - "y": 33 + "y": 35 }, "id": 139, "legend": { @@ -9488,7 +9339,7 @@ "h": 7, "w": 12, "x": 0, - "y": 40 + "y": 42 }, "id": 143, "legend": { @@ -9615,7 +9466,7 @@ "h": 7, "w": 12, "x": 12, - "y": 40 + "y": 42 }, "id": 145, "legend": { @@ -9736,7 +9587,7 @@ "h": 7, "w": 12, "x": 0, - "y": 47 + "y": 49 }, "id": 141, "legend": { @@ -9841,7 +9692,7 @@ "h": 7, "w": 6, "x": 12, - "y": 47 + "y": 49 }, "heatmap": {}, "hideZeroBuckets": true, @@ -9912,7 +9763,7 @@ "h": 7, "w": 6, "x": 18, - "y": 47 + "y": 49 }, "id": 142, "legend": { @@ -10018,7 +9869,7 @@ "h": 1, "w": 24, "x": 0, - "y": 6 + "y": 7 }, "id": 155, "panels": [ @@ -10075,8 +9926,8 @@ }, "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": [ @@ -10261,8 +10112,8 @@ }, "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": [], @@ -10368,8 +10219,8 @@ }, "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": [ @@ -10490,8 +10341,8 @@ }, "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": [ @@ -10792,5 +10643,5 @@ "timezone": "browser", "title": "Test-Cluster-TiCDC", "uid": "YiGL8hBZ1", - "version": 26 -} + "version": 29 +} \ No newline at end of file From 75957ec6af4c1348229bc8d80c464a520a7d9057 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Thu, 30 Dec 2021 21:11:51 +0800 Subject: [PATCH 22/30] kv(ticdc): reduce eventfeed rate limited log (#4072) (#4110) 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 ccba12e638b..c4f5a3190aa 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 @@ -1081,14 +1104,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 */) } }() @@ -1100,9 +1117,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 e61310ec110..fd5094637f3 100644 --- a/cdc/kv/client_test.go +++ b/cdc/kv/client_test.go @@ -43,6 +43,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" @@ -3497,3 +3498,19 @@ func (s *etcdSuite) 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 9f84f95640a82894dbe6884b706b588e5e89c489 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 23/30] 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 bb06324c88c..d4b3cb7f0c1 100644 --- a/cdc/sink/codec/avro.go +++ b/cdc/sink/codec/avro.go @@ -120,6 +120,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 2dba506a0d0..01b54162704 100644 --- a/cdc/sink/codec/canal.go +++ b/cdc/sink/codec/canal.go @@ -377,7 +377,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 } @@ -390,7 +391,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 151ecece2fb..52587fb0214 100644 --- a/cdc/sink/codec/canal_flat.go +++ b/cdc/sink/codec/canal_flat.go @@ -218,7 +218,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 1ca30741756..6813a231983 100644 --- a/cdc/sink/codec/canal_flat_test.go +++ b/cdc/sink/codec/canal_flat_test.go @@ -124,6 +124,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 1ccb94f5409..f3fcf52f62a 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 048b2905273..f23d074c258 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 2b495e6986d..c59d48676db 100644 --- a/cdc/sink/codec/maxwell.go +++ b/cdc/sink/codec/maxwell.go @@ -277,7 +277,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 4ca5c72dc94f2a8a3fe98aaf9a55c738f767f464 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Mon, 10 Jan 2022 15:11:42 +0800 Subject: [PATCH 24/30] http_api (ticdc): fix http api 'get processor' panic. (#4117) (#4122) close pingcap/tiflow#3840 --- cdc/capture/http_errors.go | 3 ++- cdc/sink/manager.go | 5 +---- tests/integration_tests/http_api/util/test_case.py | 11 ++++++++--- 3 files changed, 11 insertions(+), 8 deletions(-) diff --git a/cdc/capture/http_errors.go b/cdc/capture/http_errors.go index ed58b71e108..91aa41f6d87 100644 --- a/cdc/capture/http_errors.go +++ b/cdc/capture/http_errors.go @@ -25,7 +25,8 @@ 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, cerror.ErrTaskStatusNotExists, + cerror.ErrTaskPositionNotExists, } // IsHTTPBadRequestError check if a error is a http bad request error diff --git a/cdc/sink/manager.go b/cdc/sink/manager.go index 6fae483b2d0..1697a0b8cba 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 { @@ -235,10 +236,6 @@ func (t *tableSink) FlushRowChangedEvents(ctx context.Context, tableID model.Tab return ckpt, err } -func (t *tableSink) getEmittedTs() uint64 { - return atomic.LoadUint64(&t.emittedTs) -} - func (t *tableSink) EmitCheckpointTs(ctx context.Context, ts uint64) error { // the table sink doesn't receive the checkpoint event return nil 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 03a292e953ea1d90d8771676214c2b43475e51d0 Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Wed, 12 Jan 2022 11:12:03 +0800 Subject: [PATCH 25/30] cdc/sink: adjust kafka initialization logic (#3192) (#3568) * This is an automated cherry-pick of #3192 Signed-off-by: ti-chi-bot * fix conflicts. * This is an automated cherry-pick of #3682 Signed-off-by: ti-chi-bot * fix import. * fix failpoint path. * try to fix initialize. * remove table_sink. * remove initialization. * remove owner. * fix mq. Co-authored-by: Ling Jin <7138436+3AceShowHand@users.noreply.github.com> Co-authored-by: 3AceShowHand --- 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 | 16 +- cdc/sink/mq_test.go | 12 ++ cdc/sink/mysql.go | 5 - cdc/sink/producer/kafka/kafka.go | 236 +++++++++++++++++++------- cdc/sink/producer/kafka/kafka_test.go | 101 ++++++----- cdc/sink/simple_mysql_tester.go | 5 - cdc/sink/sink.go | 2 - 13 files changed, 252 insertions(+), 177 deletions(-) diff --git a/cdc/owner/async_sink.go b/cdc/owner/async_sink.go index 0cd1f7cad70..c2dc5459e0f 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 833c9bfb603..795cd066267 100644 --- a/cdc/owner/async_sink_test.go +++ b/cdc/owner/async_sink_test.go @@ -37,16 +37,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 { @@ -88,17 +82,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 413554d1f73..cf8b3182b01 100644 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -254,10 +254,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 b9ba6a545f2..efc18c0f398 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() @@ -342,10 +338,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 d9633399982..41c583ccb1b 100644 --- a/cdc/sink/mq.go +++ b/cdc/sink/mq.go @@ -265,12 +265,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) @@ -415,7 +409,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) } @@ -445,8 +443,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 c60e3cb229d..2f2ad09fdd6 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) @@ -161,6 +167,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/mysql.go b/cdc/sink/mysql.go index 15e8125887b..382e79db466 100644 --- a/cdc/sink/mysql.go +++ b/cdc/sink/mysql.go @@ -185,11 +185,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/producer/kafka/kafka.go b/cdc/sink/producer/kafka/kafka.go index 9b9fc650992..eab55b678bf 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") } - return partitionNum, nil + // 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` for the default open protocol. + 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) + } + + 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,8 @@ 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 + 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 +621,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 +649,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) diff --git a/cdc/sink/simple_mysql_tester.go b/cdc/sink/simple_mysql_tester.go index 3eec6494158..d8ef81f9f1f 100644 --- a/cdc/sink/simple_mysql_tester.go +++ b/cdc/sink/simple_mysql_tester.go @@ -104,11 +104,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 c1cf6ecda3c..1dd36d4477b 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 From 9a997487583cb066d01755971695d03661de375f Mon Sep 17 00:00:00 2001 From: Ti Chi Robot Date: Wed, 12 Jan 2022 16:59:42 +0800 Subject: [PATCH 26/30] sink (ticdc): fix a deadlock due to checkpointTs fall back in sinkNode (#4084) (#4098) close pingcap/tiflow#4055 --- cdc/processor/pipeline/sink.go | 11 +++++- cdc/processor/pipeline/sink_test.go | 58 +++++++++++++++++++++++++++++ pkg/pipeline/test.go | 2 +- 3 files changed, 69 insertions(+), 2 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/pkg/pipeline/test.go b/pkg/pipeline/test.go index bfc25ac69b8..e0d3cd7f68c 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 f65fa378bf6a60a251992902c9a55cbd9eff8fe1 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 27/30] 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 bb06324c88c..d4b3cb7f0c1 100644 --- a/cdc/sink/codec/avro.go +++ b/cdc/sink/codec/avro.go @@ -120,6 +120,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 2dba506a0d0..01b54162704 100644 --- a/cdc/sink/codec/canal.go +++ b/cdc/sink/codec/canal.go @@ -377,7 +377,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 } @@ -390,7 +391,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 151ecece2fb..52587fb0214 100644 --- a/cdc/sink/codec/canal_flat.go +++ b/cdc/sink/codec/canal_flat.go @@ -218,7 +218,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 1ca30741756..6813a231983 100644 --- a/cdc/sink/codec/canal_flat_test.go +++ b/cdc/sink/codec/canal_flat_test.go @@ -124,6 +124,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 1ccb94f5409..f3fcf52f62a 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 048b2905273..f23d074c258 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 2b495e6986d..c59d48676db 100644 --- a/cdc/sink/codec/maxwell.go +++ b/cdc/sink/codec/maxwell.go @@ -277,7 +277,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 c6b02f9d02c55ed499ab0a30aac402516948f7b0 Mon Sep 17 00:00:00 2001 From: 3AceShowHand Date: Tue, 18 Jan 2022 10:41:10 +0800 Subject: [PATCH 28/30] fix conflicts. --- cdc/sink/codec/canal.go | 4 ---- cdc/sink/codec/canal_flat.go | 6 +----- cdc/sink/codec/maxwell.go | 4 ---- 3 files changed, 1 insertion(+), 13 deletions(-) diff --git a/cdc/sink/codec/canal.go b/cdc/sink/codec/canal.go index 01b54162704..be3e7c00ed5 100644 --- a/cdc/sink/codec/canal.go +++ b/cdc/sink/codec/canal.go @@ -391,12 +391,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 52587fb0214..a6c709a90bd 100644 --- a/cdc/sink/codec/canal_flat.go +++ b/cdc/sink/codec/canal_flat.go @@ -218,13 +218,9 @@ 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 := NewMQMessage(ProtocolCanalJSON, nil, value, msg.tikvTs, model.MqMessageTypeRow, &msg.Schema, &msg.Table) 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 c59d48676db..0951a528aea 100644 --- a/cdc/sink/codec/maxwell.go +++ b/cdc/sink/codec/maxwell.go @@ -277,12 +277,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 2ede76879c61ce56a7e4445cb118e7d75a51769f Mon Sep 17 00:00:00 2001 From: 3AceShowHand Date: Tue, 18 Jan 2022 10:46:56 +0800 Subject: [PATCH 29/30] fix conflicts. --- cdc/sink/codec/craft.go | 4 ---- cdc/sink/codec/craft/model.go | 4 ---- cdc/sink/codec/interface.go | 4 ---- cdc/sink/codec/maxwell.go | 1 + pkg/cmd/util/helper.go | 3 --- pkg/version/check.go | 5 ----- pkg/version/check_test.go | 4 ---- 7 files changed, 1 insertion(+), 24 deletions(-) diff --git a/cdc/sink/codec/craft.go b/cdc/sink/codec/craft.go index 1374a997098..270f669d6c6 100644 --- a/cdc/sink/codec/craft.go +++ b/cdc/sink/codec/craft.go @@ -46,11 +46,7 @@ func (e *CraftEventBatchEncoder) flush() { schema := headers.GetSchema(0) table := headers.GetTable(0) rowsCnt := e.rowChangedBuffer.RowsCount() -<<<<<<< HEAD - 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) ->>>>>>> 342afe7fa8f2b6bbfd752f5040a759b3916595b0 mqMessage.SetRowsCount(rowsCnt) e.messageBuf = append(e.messageBuf, mqMessage) } diff --git a/cdc/sink/codec/craft/model.go b/cdc/sink/codec/craft/model.go index 19621caf7b3..823f96e642a 100644 --- a/cdc/sink/codec/craft/model.go +++ b/cdc/sink/codec/craft/model.go @@ -498,11 +498,7 @@ func (b *RowChangedEventBuffer) Size() int { return b.estimatedSize } -<<<<<<< HEAD -// Number of rows batched in this buffer -======= // RowsCount return Number of rows batched in this buffer ->>>>>>> 342afe7fa8f2b6bbfd752f5040a759b3916595b0 func (b *RowChangedEventBuffer) RowsCount() int { return b.eventsCount } diff --git a/cdc/sink/codec/interface.go b/cdc/sink/codec/interface.go index 44a284b45ef..33a3457dcad 100644 --- a/cdc/sink/codec/interface.go +++ b/cdc/sink/codec/interface.go @@ -61,11 +61,7 @@ type MQMessage struct { Schema *string // schema Table *string // table Type model.MqMessageType // type -<<<<<<< HEAD - Protocol config.Protocol // protocol -======= Protocol Protocol // protocol ->>>>>>> 342afe7fa8f2b6bbfd752f5040a759b3916595b0 rowsCount int // rows in one MQ Message } diff --git a/cdc/sink/codec/maxwell.go b/cdc/sink/codec/maxwell.go index 0db48ab7f7a..0951a528aea 100644 --- a/cdc/sink/codec/maxwell.go +++ b/cdc/sink/codec/maxwell.go @@ -276,6 +276,7 @@ func (d *MaxwellEventBatchEncoder) Build() []*MQMessage { if d.batchSize == 0 { return nil } + ret := NewMQMessage(ProtocolMaxwell, d.keyBuf.Bytes(), d.valueBuf.Bytes(), 0, model.MqMessageTypeRow, nil, nil) ret.SetRowsCount(d.batchSize) d.Reset() diff --git a/pkg/cmd/util/helper.go b/pkg/cmd/util/helper.go index d780eede075..aee989ca3f8 100644 --- a/pkg/cmd/util/helper.go +++ b/pkg/cmd/util/helper.go @@ -26,10 +26,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/kv" -<<<<<<< HEAD -======= "github.com/pingcap/tiflow/cdc/model" ->>>>>>> 342afe7fa8f2b6bbfd752f5040a759b3916595b0 cmdconetxt "github.com/pingcap/tiflow/pkg/cmd/context" "github.com/pingcap/tiflow/pkg/logutil" "github.com/pingcap/tiflow/pkg/version" diff --git a/pkg/version/check.go b/pkg/version/check.go index 2004a3b948e..692fa7771b3 100644 --- a/pkg/version/check.go +++ b/pkg/version/check.go @@ -22,11 +22,6 @@ import ( "regexp" "strings" -<<<<<<< HEAD - "github.com/pingcap/tiflow/cdc/model" - -======= ->>>>>>> 342afe7fa8f2b6bbfd752f5040a759b3916595b0 "github.com/coreos/go-semver/semver" "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/log" diff --git a/pkg/version/check_test.go b/pkg/version/check_test.go index 0c7fe5df228..688eaf5723e 100644 --- a/pkg/version/check_test.go +++ b/pkg/version/check_test.go @@ -24,10 +24,6 @@ import ( "github.com/coreos/go-semver/semver" "github.com/pingcap/check" "github.com/pingcap/kvproto/pkg/metapb" -<<<<<<< HEAD - "github.com/pingcap/tiflow/cdc/model" -======= ->>>>>>> 342afe7fa8f2b6bbfd752f5040a759b3916595b0 "github.com/pingcap/tiflow/pkg/util/testleak" pd "github.com/tikv/pd/client" "github.com/tikv/pd/pkg/tempurl" From 098a9c52652cff16dc637735495a5b4f75aaa4b3 Mon Sep 17 00:00:00 2001 From: 3AceShowHand Date: Tue, 18 Jan 2022 10:49:07 +0800 Subject: [PATCH 30/30] fix conflicts. --- cdc/model/changefeed.go | 3 --- 1 file changed, 3 deletions(-) diff --git a/cdc/model/changefeed.go b/cdc/model/changefeed.go index 0cbca03ae10..e93a3463ae2 100644 --- a/cdc/model/changefeed.go +++ b/cdc/model/changefeed.go @@ -25,10 +25,7 @@ import ( "github.com/pingcap/tiflow/pkg/config" "github.com/pingcap/tiflow/pkg/cyclic/mark" cerror "github.com/pingcap/tiflow/pkg/errors" -<<<<<<< HEAD -======= "github.com/pingcap/tiflow/pkg/version" ->>>>>>> 342afe7fa8f2b6bbfd752f5040a759b3916595b0 "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" )