diff --git a/cdc/entry/metrics.go b/cdc/entry/metrics.go index c818754b807..cacd8308399 100644 --- a/cdc/entry/metrics.go +++ b/cdc/entry/metrics.go @@ -33,10 +33,18 @@ var ( Help: "Bucketed histogram of processing time (s) of unmarshal and mount in mounter.", Buckets: prometheus.ExponentialBuckets(0.000001, 10, 10), }, []string{"capture", "changefeed"}) + totalRowsCountGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "mounter", + Name: "total_rows_count", + Help: "The total count of rows that are processed by mounter", + }, []string{"capture", "changefeed"}) ) // InitMetrics registers all metrics in this file func InitMetrics(registry *prometheus.Registry) { registry.MustRegister(mounterInputChanSizeGauge) registry.MustRegister(mountDuration) + registry.MustRegister(totalRowsCountGauge) } diff --git a/cdc/entry/mounter.go b/cdc/entry/mounter.go index c65bf6e3fad..9b27761fac8 100644 --- a/cdc/entry/mounter.go +++ b/cdc/entry/mounter.go @@ -116,6 +116,11 @@ func (m *mounterImpl) codecWorker(ctx context.Context, index int) error { captureAddr := util.CaptureAddrFromCtx(ctx) changefeedID := util.ChangefeedIDFromCtx(ctx) metricMountDuration := mountDuration.WithLabelValues(captureAddr, changefeedID) + metricTotalRows := totalRowsCountGauge.WithLabelValues(captureAddr, changefeedID) + defer func() { + mountDuration.DeleteLabelValues(captureAddr, changefeedID) + totalRowsCountGauge.DeleteLabelValues(captureAddr, changefeedID) + }() for { var pEvent *model.PolymorphicEvent @@ -138,6 +143,7 @@ func (m *mounterImpl) codecWorker(ctx context.Context, index int) error { pEvent.RawKV.OldValue = nil pEvent.PrepareFinished() metricMountDuration.Observe(time.Since(startTime).Seconds()) + metricTotalRows.Inc() } } diff --git a/cdc/processor.go b/cdc/processor.go index bcbc4dfa59d..2cdcf9f8be5 100644 --- a/cdc/processor.go +++ b/cdc/processor.go @@ -1230,7 +1230,7 @@ func runProcessor( cancel() return nil, errors.Trace(err) } - sinkManager := sink.NewManager(ctx, s, errCh, checkpointTs) + 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 { diff --git a/cdc/processor/pipeline/sink.go b/cdc/processor/pipeline/sink.go index 4ab16524c74..30570519dcc 100644 --- a/cdc/processor/pipeline/sink.go +++ b/cdc/processor/pipeline/sink.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/ticdc/cdc/sink" cerror "github.com/pingcap/ticdc/pkg/errors" "github.com/pingcap/ticdc/pkg/pipeline" + "go.uber.org/zap" ) const ( @@ -150,6 +151,7 @@ func (n *sinkNode) flushSink(ctx pipeline.NodeContext, resolvedTs model.Ts) (err func (n *sinkNode) emitEvent(ctx pipeline.NodeContext, event *model.PolymorphicEvent) error { if event == nil || event.Row == nil { + log.Warn("skip emit empty rows", zap.Any("event", event)) return nil } diff --git a/cdc/processor/processor.go b/cdc/processor/processor.go index dea5ccd81c0..fc3ae3a1971 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -286,7 +286,8 @@ func (p *processor) lazyInitImpl(ctx cdcContext.Context) error { return errors.Trace(err) } checkpointTs := p.changefeed.Info.GetCheckpointTs(p.changefeed.Status) - p.sinkManager = sink.NewManager(stdCtx, s, errCh, checkpointTs) + captureAddr := ctx.GlobalVars().CaptureInfo.AdvertiseAddr + p.sinkManager = sink.NewManager(stdCtx, s, errCh, checkpointTs, captureAddr, p.changefeedID) p.initialized = true log.Info("run processor", cdcContext.ZapFieldCapture(ctx), cdcContext.ZapFieldChangefeed(ctx)) return nil diff --git a/cdc/sink/manager.go b/cdc/sink/manager.go index 3221ca34a55..bb5d0024bde 100644 --- a/cdc/sink/manager.go +++ b/cdc/sink/manager.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/log" "github.com/pingcap/ticdc/cdc/model" "github.com/pingcap/ticdc/pkg/util" + "github.com/prometheus/client_golang/prometheus" "go.uber.org/zap" ) @@ -42,16 +43,26 @@ type Manager struct { flushMu sync.Mutex drawbackChan chan drawbackMsg + + captureAddr string + changefeedID model.ChangeFeedID + metricsTableSinkTotalRows prometheus.Counter } // NewManager creates a new Sink manager -func NewManager(ctx context.Context, backendSink Sink, errCh chan error, checkpointTs model.Ts) *Manager { +func NewManager( + ctx context.Context, backendSink Sink, errCh chan error, checkpointTs model.Ts, + captureAddr string, changefeedID model.ChangeFeedID, +) *Manager { drawbackChan := make(chan drawbackMsg, 16) return &Manager{ - backendSink: newBufferSink(ctx, backendSink, errCh, checkpointTs, drawbackChan), - checkpointTs: checkpointTs, - tableSinks: make(map[model.TableID]*tableSink), - drawbackChan: drawbackChan, + backendSink: newBufferSink(ctx, backendSink, errCh, checkpointTs, drawbackChan), + checkpointTs: checkpointTs, + tableSinks: make(map[model.TableID]*tableSink), + drawbackChan: drawbackChan, + captureAddr: captureAddr, + changefeedID: changefeedID, + metricsTableSinkTotalRows: tableSinkTotalRowsCountCounter.WithLabelValues(captureAddr, changefeedID), } } @@ -74,6 +85,7 @@ 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) } @@ -142,6 +154,7 @@ func (t *tableSink) Initialize(ctx context.Context, tableInfo []*model.SimpleTab func (t *tableSink) EmitRowChangedEvents(ctx context.Context, rows ...*model.RowChangedEvent) error { t.buffer = append(t.buffer, rows...) + t.manager.metricsTableSinkTotalRows.Add(float64(len(rows))) return nil } @@ -227,6 +240,13 @@ func (b *bufferSink) run(ctx context.Context, errCh chan error) { metricFlushDuration := flushRowChangedDuration.WithLabelValues(advertiseAddr, changefeedID, "Flush") metricEmitRowDuration := flushRowChangedDuration.WithLabelValues(advertiseAddr, changefeedID, "EmitRow") metricBufferSize := bufferChanSizeGauge.WithLabelValues(advertiseAddr, changefeedID) + metricTotalRows := bufferSinkTotalRowsCountCounter.WithLabelValues(advertiseAddr, changefeedID) + defer func() { + flushRowChangedDuration.DeleteLabelValues(advertiseAddr, changefeedID, "Flush") + flushRowChangedDuration.DeleteLabelValues(advertiseAddr, changefeedID, "EmitRow") + bufferChanSizeGauge.DeleteLabelValues(advertiseAddr, changefeedID) + bufferSinkTotalRowsCountCounter.DeleteLabelValues(advertiseAddr, changefeedID) + }() for { select { case <-ctx.Done(): @@ -247,6 +267,7 @@ func (b *bufferSink) run(ctx context.Context, errCh chan error) { i := sort.Search(len(rows), func(i int) bool { return rows[i].CommitTs > resolvedTs }) + metricTotalRows.Add(float64(i)) start := time.Now() err := b.Sink.EmitRowChangedEvents(ctx, rows[:i]...) diff --git a/cdc/sink/manager_test.go b/cdc/sink/manager_test.go index 890fe69eb89..c1b95fcf936 100644 --- a/cdc/sink/manager_test.go +++ b/cdc/sink/manager_test.go @@ -91,7 +91,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, &checkSink{C: c}, errCh, 0, "", "") defer manager.Close(ctx) goroutineNum := 10 rowNum := 100 @@ -146,7 +146,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, &checkSink{C: c}, errCh, 0, "", "") defer manager.Close(ctx) goroutineNum := 200 var wg sync.WaitGroup @@ -231,7 +231,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, &checkSink{C: c}, errCh, 0, "", "") defer manager.Close(ctx) tableID := int64(49) @@ -284,7 +284,7 @@ func (s *managerSuite) TestManagerError(c *check.C) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() errCh := make(chan error, 16) - manager := NewManager(ctx, &errorSink{C: c}, errCh, 0) + manager := NewManager(ctx, &errorSink{C: c}, errCh, 0, "", "") defer manager.Close(ctx) sink := manager.CreateTableSink(1, 0) err := sink.EmitRowChangedEvents(ctx, &model.RowChangedEvent{ diff --git a/cdc/sink/metrics.go b/cdc/sink/metrics.go index 5b1176ddf91..7d7380f6ad1 100644 --- a/cdc/sink/metrics.go +++ b/cdc/sink/metrics.go @@ -69,14 +69,14 @@ var ( Namespace: "ticdc", Subsystem: "sink", Name: "total_rows_count", - Help: "totla count of rows", + Help: "The total count of rows that are processed by sink", }, []string{"capture", "changefeed"}) totalFlushedRowsCountGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: "ticdc", Subsystem: "sink", Name: "total_flushed_rows_count", - Help: "totla count of flushed rows", + Help: "The total count of rows that are flushed by sink", }, []string{"capture", "changefeed"}) flushRowChangedDuration = prometheus.NewHistogramVec( prometheus.HistogramOpts{ @@ -93,6 +93,22 @@ var ( Name: "buffer_chan_size", Help: "size of row changed event buffer channel in sink manager", }, []string{"capture", "changefeed"}) + + tableSinkTotalRowsCountCounter = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "table_sink_total_rows_count", + Help: "The total count of rows that are processed by table sink", + }, []string{"capture", "changefeed"}) + + bufferSinkTotalRowsCountCounter = prometheus.NewCounterVec( + prometheus.CounterOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "buffer_sink_total_rows_count", + Help: "The total count of rows that are processed by buffer sink", + }, []string{"capture", "changefeed"}) ) // InitMetrics registers all metrics in this file @@ -107,4 +123,6 @@ func InitMetrics(registry *prometheus.Registry) { registry.MustRegister(totalFlushedRowsCountGauge) registry.MustRegister(flushRowChangedDuration) registry.MustRegister(bufferChanSizeGauge) + registry.MustRegister(tableSinkTotalRowsCountCounter) + registry.MustRegister(bufferSinkTotalRowsCountCounter) } diff --git a/cdc/sink/statistics.go b/cdc/sink/statistics.go index 2c0114bbd95..b09a8c988c1 100644 --- a/cdc/sink/statistics.go +++ b/cdc/sink/statistics.go @@ -44,11 +44,16 @@ func NewStatistics(ctx context.Context, name string, opts map[string]string) *St statistics.metricExecErrCnt = executionErrorCounter.WithLabelValues(statistics.captureAddr, statistics.changefeedID) // Flush metrics in background for better accuracy and efficiency. + captureAddr, changefeedID := statistics.captureAddr, statistics.changefeedID ticker := time.NewTicker(flushMetricsInterval) - metricTotalRows := totalRowsCountGauge.WithLabelValues(statistics.captureAddr, statistics.changefeedID) - metricTotalFlushedRows := totalFlushedRowsCountGauge.WithLabelValues(statistics.captureAddr, statistics.changefeedID) go func() { defer ticker.Stop() + metricTotalRows := totalRowsCountGauge.WithLabelValues(captureAddr, changefeedID) + metricTotalFlushedRows := totalFlushedRowsCountGauge.WithLabelValues(captureAddr, changefeedID) + defer func() { + totalRowsCountGauge.DeleteLabelValues(captureAddr, changefeedID) + totalFlushedRowsCountGauge.DeleteLabelValues(captureAddr, changefeedID) + }() for { select { case <-ctx.Done(): diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index 9973fe0a4b5..edadcd6dd73 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -125,7 +125,7 @@ "gnetId": null, "graphTooltip": 1, "id": null, - "iteration": 1630305196418, + "iteration": 1631019842531, "links": [], "panels": [ { @@ -187,14 +187,14 @@ "expr": "(time() - process_start_time_seconds{tidb_cluster=\"$tidb_cluster\", job=\"ticdc\"})", "format": "time_series", "intervalFactor": 1, - "legendFormat": "TiCDC - {{instance}}", + "legendFormat": "TiCDC-{{instance}}", "refId": "A" }, { "expr": "(time() - process_start_time_seconds{tidb_cluster=\"$tidb_cluster\", job=\"tikv\"})", "format": "time_series", "intervalFactor": 1, - "legendFormat": "TiKV - {{instance}}", + "legendFormat": "TiKV-{{instance}}", "refId": "B" } ], @@ -1052,7 +1052,7 @@ "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "{{capture}}--{{changefeed}}", + "legendFormat": "{{capture}}-{{changefeed}}", "refId": "A" }, { @@ -1060,7 +1060,7 @@ "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "checkpoint-{{capture}}--{{changefeed}}", + "legendFormat": "checkpoint-{{capture}}-{{changefeed}}", "refId": "B" } ], @@ -1130,7 +1130,7 @@ ], "targets": [ { - "expr": "max(ticdc_processor_table_resolved_ts{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture,changefeed,table)", + "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": "", @@ -1139,11 +1139,11 @@ "refId": "A" }, { - "expr": "max(ticdc_processor_checkpoint_ts{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture,changefeed,table) > 0", + "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-{{changefeed}}", + "legendFormat": "checkpoint-{{capture}}-{{changefeed}}", "refId": "B" } ], @@ -1569,6 +1569,14 @@ "intervalFactor": 1, "legendFormat": "{{changefeed}}", "refId": "A" + }, + { + "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}}", + "refId": "B" } ], "thresholds": [], @@ -1614,11 +1622,11 @@ }, { "aliasColors": {}, - "bars": false, + "bars": true, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The lag between internal resolved ts and the latest ts of upstream TiDB.", + "description": "The derivative of a changefeed checkpoint, ideally it should not be 0 (means the changefeed is stopped).", "fill": 1, "fillGradient": 0, "gridPos": { @@ -1651,7 +1659,7 @@ }, "paceLength": 10, "percentage": false, - "pointradius": 2, + "pointradius": 0.5, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -1660,11 +1668,11 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_processor_resolved_ts_lag{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture,changefeed)", + "expr": "sum(deriv(ticdc_owner_checkpoint_ts{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (changefeed) / 1000 > 0", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "{{capture}}--{{changefeed}}", + "legendFormat": "{{changefeed}}", "refId": "A" } ], @@ -1672,7 +1680,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Processor resolved ts lag", + "title": "Changefeed checkpoint derivative", "tooltip": { "shared": true, "sort": 0, @@ -1690,9 +1698,9 @@ { "format": "s", "label": null, - "logBase": 1, - "max": null, - "min": null, + "logBase": 2, + "max": "3600", + "min": "0.001", "show": true }, { @@ -3023,7 +3031,7 @@ "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 }} - {{ changefeed }}", + "legendFormat": "{{ capture }}-{{ changefeed }}", "refId": "A" } ], @@ -3080,7 +3088,7 @@ "x": 0, "y": 2 }, - "id": 13, + "id": 206, "panels": [ { "aliasColors": {}, @@ -3088,25 +3096,24 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of established Eventfeed RPC between TiCDC and TiKV", + "description": "The number of events that puller outputs to sorter \n per second", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, - "w": 8, + "h": 4, + "w": 12, "x": 0, "y": 3 }, "hiddenSeries": false, - "id": 15, + "id": 218, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": true, - "hideEmpty": false, - "max": false, + "max": true, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "total": false, "values": true @@ -3123,52 +3130,24 @@ "pointradius": 2, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/.*-rpc/", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_kvclient_event_feed_count{tidb_cluster=\"$tidb_cluster\"}) by (instance)", + "expr": "sum (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" - }, - { - "expr": "sum(grpc_client_started_total{tidb_cluster=\"$tidb_cluster\", grpc_method=\"EventFeed\"}) by (instance) - sum(grpc_client_handled_total{tidb_cluster=\"$tidb_cluster\", grpc_method=\"EventFeed\"}) by (instance)", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-rpc", - "refId": "B" - }, - { - "expr": "sum(grpc_client_started_total{tidb_cluster=\"$tidb_cluster\", grpc_method=\"EventFeed\"}) by (instance)", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "{{instance}}-rpc-started", - "refId": "C" - }, - { - "expr": "sum(grpc_client_handled_total{tidb_cluster=\"$tidb_cluster\", grpc_method=\"EventFeed\"}) by (instance)", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "{{instance}}-rpc-handled", - "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Eventfeed count", + "title": "Puller output events/s", "tooltip": { "shared": true, "sort": 0, @@ -3184,7 +3163,7 @@ }, "yaxes": [ { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -3197,7 +3176,7 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -3211,25 +3190,24 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": null, - "description": "Percentiles of Eventfeed message size", + "description": "The total number of events that puller outputs", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, - "w": 8, - "x": 8, + "h": 4, + "w": 12, + "x": 12, "y": 3 }, "hiddenSeries": false, - "id": 17, + "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 @@ -3252,25 +3230,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_puller_txn_collect_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (instance, type)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}-p999", + "legendFormat": "{{instance}}-{{type}}", "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": "Puller output events", "tooltip": { "shared": true, "sort": 0, @@ -3286,7 +3257,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -3309,30 +3280,28 @@ }, { "aliasColors": {}, - "bars": true, + "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of errors that interrupt Eventfeed RPC", + "description": "The number of events that sorter outputs to puller \n per second", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, - "w": 8, - "x": 16, - "y": 3 + "h": 4, + "w": 12, + "x": 0, + "y": 7 }, "hiddenSeries": false, - "id": 28, + "id": 228, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": true, - "hideEmpty": true, - "hideZero": true, "max": true, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "total": false, "values": true @@ -3355,37 +3324,112 @@ "steppedLine": false, "targets": [ { - "expr": "sum(increase(ticdc_kvclient_event_feed_error_count{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (type)", + "expr": "sum(rate(ticdc_sorter_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture,changefeed,type)", "format": "time_series", - "hide": false, - "interval": "1m", "intervalFactor": 1, - "legendFormat": "{{type}}", + "legendFormat": "{{capture}}-{{type}}", "refId": "A" - }, + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Sorter output events/s", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ { - "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" + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true }, { - "expr": "-sum(increase(pd_schedule_operators_count{tidb_cluster=\"$tidb_cluster\", event=\"create\", type=~\".*(peer|region)\"}[1m]))", + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The total number of events that sorter outputs", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 4, + "w": 12, + "x": 12, + "y": 7 + }, + "hiddenSeries": false, + "id": 220, + "legend": { + "alignAsTable": false, + "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": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sorter_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (capture, changefeed, type)", "format": "time_series", - "interval": "1m", "intervalFactor": 1, - "legendFormat": "move-region", - "refId": "C" + "legendFormat": "{{capture}}-{{type}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Eventfeed error/m", + "title": "Sorter output events", "tooltip": { "shared": true, "sort": 0, @@ -3401,7 +3445,7 @@ }, "yaxes": [ { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -3428,23 +3472,21 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of KV client received events from TiKV per seconds", + "description": "The number of events that mounter outputs to sink per second", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, - "w": 8, + "h": 4, + "w": 12, "x": 0, - "y": 10 + "y": 11 }, "hiddenSeries": false, - "id": 29, + "id": 219, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": true, - "hideEmpty": true, - "hideZero": true, "max": true, "min": false, "rightSide": false, @@ -3470,10 +3512,10 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_kvclient_pull_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (instance, type)", + "expr": "sum(rate(ticdc_mounter_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture,changefeed)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{capture}}", "refId": "A" } ], @@ -3481,7 +3523,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "KV client receive events/s", + "title": "Mounter output events/s", "tooltip": { "shared": true, "sort": 0, @@ -3497,7 +3539,7 @@ }, "yaxes": [ { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -3510,7 +3552,7 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -3524,19 +3566,19 @@ "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 mounter outputs", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 7, - "w": 8, - "x": 8, - "y": 10 + "h": 4, + "w": 12, + "x": 12, + "y": 11 }, "hiddenSeries": false, - "id": 5, + "id": 224, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": true, "max": true, @@ -3564,18 +3606,914 @@ "steppedLine": false, "targets": [ { - "expr": "sum (rate(ticdc_puller_txn_collect_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (instance, type)", + "expr": "sum(ticdc_mounter_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (instance)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Mounter output events", + "tooltip": { + "shared": true, + "sort": 0, + "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 + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of events that table sorter outputs to buffer sink per second", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 4, + "w": 12, + "x": 0, + "y": 15 + }, + "hiddenSeries": false, + "id": 223, + "legend": { + "alignAsTable": false, + "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": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(ticdc_sink_table_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (instance)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Table sink output events/s", + "tooltip": { + "shared": true, + "sort": 0, + "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": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The total number of events that table sinks emit", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 4, + "w": 12, + "x": 12, + "y": 15 + }, + "hiddenSeries": false, + "id": 221, + "legend": { + "alignAsTable": false, + "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": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_table_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (instance)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Table sink output events", + "tooltip": { + "shared": true, + "sort": 0, + "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 + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "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": 19 + }, + "hiddenSeries": false, + "id": 222, + "legend": { + "alignAsTable": false, + "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": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(ticdc_sink_buffer_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (instance)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Buffer sink output events/s", + "tooltip": { + "shared": true, + "sort": 0, + "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": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The total number of events that buffer sink emits", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 4, + "w": 12, + "x": 12, + "y": 19 + }, + "hiddenSeries": false, + "id": 226, + "legend": { + "alignAsTable": false, + "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": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_buffer_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (instance)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Buffer sink output events", + "tooltip": { + "shared": true, + "sort": 0, + "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 + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of rows(events) that are processed by sink per second", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 4, + "w": 12, + "x": 0, + "y": 23 + }, + "hiddenSeries": false, + "id": 227, + "legend": { + "alignAsTable": false, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum (rate(ticdc_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (capture)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Sink output events/s", + "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": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of rows(events) that are processed by sink", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 4, + "w": 12, + "x": 12, + "y": 23 + }, + "hiddenSeries": false, + "id": 225, + "legend": { + "alignAsTable": false, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (capture, type)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{capture}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Sink output events", + "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 + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of rows that sink flushes to downstream per second", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 4, + "w": 12, + "x": 0, + "y": 27 + }, + "hiddenSeries": false, + "id": 108, + "legend": { + "alignAsTable": false, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(ticdc_sink_total_flushed_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Sink flush rows/s", + "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": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of rows(events) that are flushed by sink", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 4, + "w": 12, + "x": 12, + "y": 27 + }, + "hiddenSeries": false, + "id": 240, + "legend": { + "alignAsTable": false, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_total_flushed_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (capture, type)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{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": 32 + }, + "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}} - {{type}}", - "refId": "A" + "legendFormat": "{{instance}}-rpc-handled", + "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Puller output events/s", + "title": "Eventfeed count", "tooltip": { "shared": true, "sort": 0, @@ -3591,7 +4529,7 @@ }, "yaxes": [ { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -3618,27 +4556,26 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of rows that sink flushes to downstream per second", + "decimals": null, + "description": "Percentiles of Eventfeed message size", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, "w": 8, - "x": 16, - "y": 10 + "x": 8, + "y": 32 }, "hiddenSeries": false, - "id": 108, + "id": 17, "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 }, @@ -3649,6 +4586,7 @@ "options": { "dataLinks": [] }, + "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -3659,21 +4597,28 @@ "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_kvclient_event_size_bytes_bucket{tidb_cluster=\"$tidb_cluster\"}[1m])) by (le, instance, type))", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}", + "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": "Sink flush rows/s", + "title": "Event size percentile", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -3686,7 +4631,7 @@ }, "yaxes": [ { - "format": "short", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -3709,29 +4654,30 @@ }, { "aliasColors": {}, - "bars": false, + "bars": true, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of events that are buffered in Puller's memory buffer and output channel", + "description": "The number of errors that interrupt Eventfeed RPC", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, - "w": 6, - "x": 0, - "y": 17 + "w": 8, + "x": 16, + "y": 32 }, "hiddenSeries": false, - "id": 43, + "id": 28, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, + "hideEmpty": true, + "hideZero": true, "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "total": false, "values": true @@ -3748,30 +4694,35 @@ "pointradius": 2, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/.*chan.*/", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(rate(ticdc_puller_output_chan_size_sum{tidb_cluster=\\\"$tidb_cluster\\\", changefeed=~\\\"$changefeed\\\",capture=~\\\"$capture\\\"}[30s]) / rate(ticdc_puller_output_chan_size_count{tidb_cluster=\\\"$tidb_cluster\\\", changefeed=~\\\"$changefeed\\\",capture=~\\\"$capture\\\"}[30s])) by (capture)", + "expr": "sum(increase(ticdc_kvclient_event_feed_error_count{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (type)", "format": "time_series", - "hide": true, + "hide": false, + "interval": "1m", "intervalFactor": 1, - "legendFormat": "{{capture}} - output chan", + "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(rate(ticdc_puller_event_chan_size_sum{tidb_cluster=\\\"$tidb_cluster\\\", changefeed=~\\\"$changefeed\\\",capture=~\\\"$capture\\\"}[30s]) / rate(ticdc_puller_event_chan_size_count{tidb_cluster=\\\"$tidb_cluster\\\", changefeed=~\\\"$changefeed\\\",capture=~\\\"$capture\\\"}[30s])) by (capture)", + "expr": "-sum(increase(pd_schedule_operators_count{tidb_cluster=\"$tidb_cluster\", event=\"create\", type=~\".*(peer|region)\"}[1m]))", "format": "time_series", - "hide": true, + "interval": "1m", "intervalFactor": 1, - "legendFormat": "{{capture}} - event chan", + "legendFormat": "move-region", "refId": "C" } ], @@ -3779,7 +4730,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Puller buffer size", + "title": "Eventfeed error/m", "tooltip": { "shared": true, "sort": 0, @@ -3822,22 +4773,23 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of events that are buffered in Sorter's unsorted events buffer and output channel", + "description": "The number of KV client received events from TiKV per seconds", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, - "w": 6, - "x": 6, - "y": 17 + "w": 8, + "x": 0, + "y": 39 }, "hiddenSeries": false, - "id": 51, + "id": 29, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, + "hideEmpty": true, + "hideZero": true, "max": true, "min": false, "rightSide": false, @@ -3857,49 +4809,24 @@ "pointradius": 2, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/.*resolvedts/", - "yaxis": 2 - }, - { - "alias": "/.*chan/", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_puller_entry_sorter_unsorted_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "expr": "sum(rate(ticdc_kvclient_pull_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (instance, type)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{capture}}-unsorted events", + "legendFormat": "{{instance}}-{{type}}", "refId": "A" - }, - { - "expr": "-sum(ticdc_puller_entry_sorter_resolved_chan_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "{{capture}}-resolvedts", - "refId": "B" - }, - { - "expr": "-sum(ticdc_puller_entry_sorter_output_chan_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "{{capture}}-ouput chan", - "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Entry sorter buffer size", + "title": "KV client receive events/s", "tooltip": { "shared": true, "sort": 0, @@ -3942,22 +4869,21 @@ "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", + "description": "The number of events that puller outputs to sorter \n per second", "fill": 1, "fillGradient": 0, "gridPos": { "h": 7, - "w": 6, - "x": 12, - "y": 17 + "w": 8, + "x": 8, + "y": 39 }, "hiddenSeries": false, - "id": 107, + "id": 5, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, "max": true, "min": false, "rightSide": false, @@ -3977,36 +4903,24 @@ "pointradius": 2, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/.*processor.*/", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_mounter_input_chan_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture)", + "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": "{{capture}}-mounter input chan", + "legendFormat": "{{instance}}-{{type}}", "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": "Puller output events/s", "tooltip": { "shared": true, "sort": 0, @@ -4022,7 +4936,7 @@ }, "yaxes": [ { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -4049,27 +4963,26 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of rows(events) that are buffered in Sink's pending flush rows buffer", + "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": 6, - "x": 18, - "y": 17 + "w": 8, + "x": 16, + "y": 39 }, "hiddenSeries": false, - "id": 96, + "id": 107, "legend": { "alignAsTable": true, - "avg": true, + "avg": false, "current": true, + "hideEmpty": false, "max": true, "min": false, "rightSide": false, "show": true, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, @@ -4080,6 +4993,7 @@ "options": { "dataLinks": [] }, + "paceLength": 10, "percentage": false, "pointradius": 2, "points": false, @@ -4090,22 +5004,28 @@ "steppedLine": false, "targets": [ { - "expr": "sum(ticdc_sink_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) by (capture) - sum(ticdc_sink_total_flushed_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}) 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 rows buffer size", + "title": "Sink/Mounter buffer size", "tooltip": { "shared": true, - "sort": 2, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -4159,7 +5079,7 @@ "h": 7, "w": 12, "x": 0, - "y": 24 + "y": 46 }, "heatmap": {}, "hideZeroBuckets": true, @@ -4228,7 +5148,7 @@ "h": 7, "w": 12, "x": 12, - "y": 24 + "y": 46 }, "hiddenSeries": false, "id": 53, @@ -4338,7 +5258,7 @@ "h": 7, "w": 12, "x": 0, - "y": 31 + "y": 53 }, "heatmap": {}, "hideZeroBuckets": true, @@ -4407,7 +5327,7 @@ "h": 7, "w": 12, "x": 12, - "y": 31 + "y": 53 }, "hiddenSeries": false, "id": 106, @@ -4517,7 +5437,7 @@ "h": 7, "w": 12, "x": 0, - "y": 38 + "y": 60 }, "heatmap": {}, "hideZeroBuckets": true, @@ -4586,7 +5506,7 @@ "h": 7, "w": 12, "x": 12, - "y": 38 + "y": 60 }, "hiddenSeries": false, "id": 55, @@ -4692,7 +5612,7 @@ "h": 7, "w": 12, "x": 0, - "y": 45 + "y": 67 }, "hiddenSeries": false, "id": 31, @@ -4807,7 +5727,7 @@ "h": 7, "w": 12, "x": 12, - "y": 45 + "y": 67 }, "heatmap": {}, "hideZeroBuckets": true, @@ -4876,7 +5796,7 @@ "h": 7, "w": 12, "x": 0, - "y": 52 + "y": 74 }, "hiddenSeries": false, "id": 177, @@ -4972,7 +5892,7 @@ "h": 7, "w": 12, "x": 12, - "y": 52 + "y": 74 }, "hiddenSeries": false, "id": 188, @@ -5065,7 +5985,7 @@ "h": 1, "w": 24, "x": 0, - "y": 3 + "y": 4 }, "id": 130, "panels": [ @@ -5080,7 +6000,7 @@ "h": 8, "w": 12, "x": 0, - "y": 4 + "y": 5 }, "id": 131, "legend": { @@ -5165,7 +6085,7 @@ "h": 8, "w": 12, "x": 12, - "y": 4 + "y": 5 }, "id": 132, "legend": { @@ -5250,7 +6170,7 @@ "h": 8, "w": 12, "x": 0, - "y": 12 + "y": 13 }, "id": 133, "legend": { @@ -5335,7 +6255,7 @@ "h": 8, "w": 12, "x": 12, - "y": 12 + "y": 13 }, "id": 134, "legend": { @@ -5429,7 +6349,7 @@ "h": 8, "w": 12, "x": 0, - "y": 20 + "y": 21 }, "heatmap": {}, "hideZeroBuckets": true, @@ -5493,7 +6413,7 @@ "h": 8, "w": 12, "x": 12, - "y": 20 + "y": 21 }, "heatmap": {}, "hideZeroBuckets": true, @@ -5549,7 +6469,7 @@ "h": 1, "w": 24, "x": 0, - "y": 4 + "y": 5 }, "id": 58, "panels": [ @@ -5560,6 +6480,7 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, + "description": "CPU usage of TiKV CDC component", "fill": 1, "gridPos": { "h": 7, @@ -5591,7 +6512,19 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/.*workers/", + "transform": "negative-Y", + "yaxis": 2 + }, + { + "alias": "/.*tso/", + "fill": 0, + "pointradius": 1, + "points": true + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, @@ -5600,16 +6533,33 @@ "expr": "sum(rate(tikv_thread_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", name=~\"cdc_.*|cdc\"}[1m])) by (instance)", "format": "time_series", "intervalFactor": 2, - "legendFormat": "{{instance}}", + "legendFormat": "{{instance}}-endpoint", "refId": "A", "step": 4 + }, + { + "expr": "sum(rate(tikv_thread_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", name=~\"cdcwkr.*\"}[1m])) by (instance)", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{instance}}-workers", + "refId": "B", + "step": 4 + }, + { + "expr": "sum(rate(tikv_thread_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", name=~\"tso\"}[1m])) by (instance)", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "{{instance}}-tso", + "refId": "C", + "step": 4 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "CDC endpoint CPU", + "title": "CDC CPU", "tooltip": { "shared": true, "sort": 0, @@ -5633,7 +6583,7 @@ "show": true }, { - "format": "short", + "format": "percent", "label": null, "logBase": 1, "max": null, @@ -5648,34 +6598,40 @@ }, { "aliasColors": {}, - "bars": false, + "bars": true, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "fill": 1, + "description": "Outbound network traffic of TiKV CDC component", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, "y": 5 }, - "id": 62, + "id": 74, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, + "hideZero": false, "max": true, "min": false, "rightSide": true, "show": true, "sideWidth": null, - "sort": "current", + "sort": "max", "sortDesc": true, "total": false, "values": true }, - "lines": true, + "lines": false, "linewidth": 1, "links": [], "nullPointMode": "null", @@ -5686,7 +6642,12 @@ "renderer": "flot", "seriesOverrides": [ { - "alias": "/.*tso/", + "alias": "/.*resolved_ts/", + "bars": false, + "fill": 1, + "lines": true, + "linewidth": 2, + "transform": "negative-Y", "yaxis": 2 } ], @@ -5695,28 +6656,22 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tikv_thread_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", name=~\"cdcwkr.*\"}[1m])) by (instance)", + "expr": "sum(rate(tikv_cdc_grpc_message_sent_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[30s])) by (instance, type)", "format": "time_series", + "hide": false, "intervalFactor": 2, - "legendFormat": "{{instance}} - worker", + "legendFormat": "{{instance}}-{{type}}", "refId": "A", - "step": 4 - }, - { - "expr": "sum(rate(tikv_thread_cpu_seconds_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", name=~\"tso\"}[1m])) by (instance)", - "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{instance}} - tso", - "refId": "B", - "step": 4 + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "CDC worker CPU", + "title": "CDC network traffic", "tooltip": { + "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -5731,7 +6686,7 @@ }, "yaxes": [ { - "format": "percentunit", + "format": "Bps", "label": null, "logBase": 1, "max": null, @@ -5739,7 +6694,7 @@ "show": true }, { - "format": "percentunit", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -5749,7 +6704,7 @@ ], "yaxis": { "align": false, - "alignLevel": null + "alignLevel": 0 } }, { @@ -5866,7 +6821,7 @@ "x": 12, "y": 12 }, - "id": 74, + "id": 194, "legend": { "alignAsTable": true, "avg": false, @@ -6567,7 +7522,7 @@ "format": "time_series", "hide": false, "intervalFactor": 1, - "legendFormat": "{{instance}} - {{type}}", + "legendFormat": "{{instance}}-{{type}}", "refId": "A" }, { @@ -6575,7 +7530,7 @@ "format": "time_series", "hide": false, "intervalFactor": 1, - "legendFormat": "{{instance}} - pending", + "legendFormat": "{{instance}}-pending", "refId": "B" } ], @@ -7113,7 +8068,7 @@ "format": "time_series", "hide": false, "intervalFactor": 1, - "legendFormat": "{{instance}} - len", + "legendFormat": "{{instance}}-len", "refId": "A" }, { @@ -7121,7 +8076,7 @@ "format": "time_series", "hide": false, "intervalFactor": 1, - "legendFormat": "{{instance}} - avg entry bytes", + "legendFormat": "{{instance}}-avg entry bytes", "refId": "B" }, { @@ -7129,7 +8084,7 @@ "format": "time_series", "hide": false, "intervalFactor": 1, - "legendFormat": "{{instance}} - quota", + "legendFormat": "{{instance}}-quota", "refId": "C" } ], @@ -7229,7 +8184,7 @@ "format": "time_series", "hide": false, "intervalFactor": 2, - "legendFormat": "{{instance}}- {{cf}} - {{tag}}", + "legendFormat": "{{instance}}-{{cf}}-{{tag}}", "refId": "A", "step": 10 } @@ -7404,21 +8359,21 @@ "expr": "histogram_quantile(0.99, sum(rate(tikv_cdc_old_value_duration_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance, tag))", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}} - 99% - {{tag}}", + "legendFormat": "{{instance}}-99%-{{tag}}", "refId": "A" }, { "expr": "histogram_quantile(0.95, sum(rate(tikv_cdc_old_value_duration_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance, tag))", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}} - 95% - {{tag}}", + "legendFormat": "{{instance}}-95%-{{tag}}", "refId": "B" }, { "expr": "sum(rate(tikv_cdc_old_value_duration_sum{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance, tag) / sum(rate(tikv_cdc_old_value_duration_count{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance, tag)", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}} - avg - {{tag}}", + "legendFormat": "{{instance}}-avg-{{tag}}", "refId": "C" } ], @@ -7473,7 +8428,7 @@ "h": 1, "w": 24, "x": 0, - "y": 5 + "y": 6 }, "id": 155, "panels": [ @@ -7530,8 +8485,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.5.33:47912", - "value": "172.16.5.33:47912" + "text": "172.16.5.32:47800", + "value": "172.16.5.32:47800" } }, "seriesOverrides": [ @@ -7716,8 +8671,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.5.33:47912", - "value": "172.16.5.33:47912" + "text": "172.16.5.32:47800", + "value": "172.16.5.32:47800" } }, "seriesOverrides": [], @@ -7823,8 +8778,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.5.33:47912", - "value": "172.16.5.33:47912" + "text": "172.16.5.32:47800", + "value": "172.16.5.32:47800" } }, "seriesOverrides": [ @@ -7945,8 +8900,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.5.33:47912", - "value": "172.16.5.33:47912" + "text": "172.16.5.32:47800", + "value": "172.16.5.32:47800" } }, "seriesOverrides": [ @@ -8055,7 +9010,7 @@ "current": {}, "datasource": "${DS_TEST-CLUSTER}", "definition": "", - "hide": 2, + "hide": 0, "includeAll": false, "label": "tidb_cluster", "multi": false, @@ -8247,5 +9202,5 @@ "timezone": "browser", "title": "Test-Cluster-TiCDC", "uid": "YiGL8hBZ1", - "version": 21 -} + "version": 22 +} \ No newline at end of file