From d728d0271e804b6759fcca35de80564a062f766a Mon Sep 17 00:00:00 2001 From: nhsmw Date: Thu, 31 Oct 2024 12:14:09 +0800 Subject: [PATCH] metric(cdc): fix uncorrect metric WorkerBusyRatio (#11669) close pingcap/tiflow#11664 --- cdc/kv/metrics.go | 2 +- cdc/redo/common/metric.go | 26 +++++++++--------- cdc/redo/manager.go | 18 ++++++------- cdc/redo/meta_manager.go | 24 ++++++++++------- cdc/redo/writer/file/file.go | 13 +++++---- cdc/redo/writer/file/file_test.go | 30 ++++++++++----------- cdc/redo/writer/memory/file_worker.go | 9 ++++--- cdc/sink/dmlsink/cloudstorage/dml_worker.go | 15 +++++------ cdc/sink/metrics/cloudstorage/metrics.go | 6 ++--- metrics/grafana/TiCDC-Monitor-Summary.json | 16 +++++------ metrics/grafana/ticdc.json | 22 +++++++-------- 11 files changed, 91 insertions(+), 90 deletions(-) diff --git a/cdc/kv/metrics.go b/cdc/kv/metrics.go index f2f2c022553..74267b5a186 100644 --- a/cdc/kv/metrics.go +++ b/cdc/kv/metrics.go @@ -149,7 +149,7 @@ var ( Namespace: "ticdc", Subsystem: "kvclient", Name: "region_worker_busy_ratio", - Help: "Busy ratio (X ms in 1s) for region worker.", + Help: "Busy ratio for region worker.", }, []string{"namespace", "changefeed", "table", "store", "type"}) workerChannelSize = prometheus.NewGaugeVec( prometheus.GaugeOpts{ diff --git a/cdc/redo/common/metric.go b/cdc/redo/common/metric.go index 74011bab63b..b9c2389161a 100644 --- a/cdc/redo/common/metric.go +++ b/cdc/redo/common/metric.go @@ -29,7 +29,7 @@ var ( Subsystem: subsystem, Name: "write_bytes_total", Help: "Total number of bytes redo log written", - }, []string{"namespace", "changefeed"}) + }, []string{"namespace", "changefeed", "type"}) // RedoFsyncDurationHistogram records the latency distributions of fsync called by redo writer. RedoFsyncDurationHistogram = prometheus.NewHistogramVec(prometheus.HistogramOpts{ @@ -37,18 +37,18 @@ var ( Subsystem: subsystem, Name: "fsync_duration_seconds", Help: "The latency distributions of fsync called by redo writer", - Buckets: prometheus.ExponentialBuckets(0.001, 2.0, 13), - }, []string{"namespace", "changefeed"}) + Buckets: prometheus.ExponentialBuckets(0.001, 2.0, 16), + }, []string{"namespace", "changefeed", "type"}) // RedoFlushAllDurationHistogram records the latency distributions of flushAll // called by redo writer. RedoFlushAllDurationHistogram = prometheus.NewHistogramVec(prometheus.HistogramOpts{ Namespace: namespace, Subsystem: subsystem, - Name: "flushall_duration_seconds", + Name: "flush_all_duration_seconds", Help: "The latency distributions of flushall called by redo writer", - Buckets: prometheus.ExponentialBuckets(0.001, 2.0, 13), - }, []string{"namespace", "changefeed"}) + Buckets: prometheus.ExponentialBuckets(0.001, 2.0, 16), + }, []string{"namespace", "changefeed", "type"}) // RedoTotalRowsCountGauge records the total number of rows written to redo log. RedoTotalRowsCountGauge = prometheus.NewGaugeVec(prometheus.GaugeOpts{ @@ -56,7 +56,7 @@ var ( Subsystem: subsystem, Name: "total_rows_count", Help: "The total count of rows that are processed by redo writer", - }, []string{"namespace", "changefeed"}) + }, []string{"namespace", "changefeed", "type"}) // RedoWriteLogDurationHistogram records the latency distributions of writeLog. RedoWriteLogDurationHistogram = prometheus.NewHistogramVec(prometheus.HistogramOpts{ @@ -64,8 +64,8 @@ var ( Subsystem: subsystem, Name: "write_log_duration_seconds", Help: "The latency distributions of writeLog called by redoManager", - Buckets: prometheus.ExponentialBuckets(0.001, 2.0, 13), - }, []string{"namespace", "changefeed"}) + Buckets: prometheus.ExponentialBuckets(0.001, 2.0, 16), + }, []string{"namespace", "changefeed", "type"}) // RedoFlushLogDurationHistogram records the latency distributions of flushLog. RedoFlushLogDurationHistogram = prometheus.NewHistogramVec(prometheus.HistogramOpts{ @@ -73,8 +73,8 @@ var ( Subsystem: subsystem, Name: "flush_log_duration_seconds", Help: "The latency distributions of flushLog called by redoManager", - Buckets: prometheus.ExponentialBuckets(0.001, 2.0, 13), - }, []string{"namespace", "changefeed"}) + Buckets: prometheus.ExponentialBuckets(0.001, 2.0, 16), + }, []string{"namespace", "changefeed", "type"}) // RedoWorkerBusyRatio records the busy ratio of redo bgUpdateLog worker. RedoWorkerBusyRatio = prometheus.NewCounterVec( @@ -82,8 +82,8 @@ var ( Namespace: namespace, Subsystem: subsystem, Name: "worker_busy_ratio", - Help: "Busy ratio (X ms in 1s) for redo bgUpdateLog worker.", - }, []string{"namespace", "changefeed"}) + Help: "Busy ratio for redo bgUpdateLog worker.", + }, []string{"namespace", "changefeed", "type"}) ) // InitMetrics registers all metrics in this file diff --git a/cdc/redo/manager.go b/cdc/redo/manager.go index a2dd002d6a6..c490751b695 100644 --- a/cdc/redo/manager.go +++ b/cdc/redo/manager.go @@ -231,13 +231,13 @@ func newLogManager( logBuffer: chann.NewAutoDrainChann[cacheEvents](), rtsMap: spanz.SyncMap{}, metricWriteLogDuration: common.RedoWriteLogDurationHistogram. - WithLabelValues(changefeedID.Namespace, changefeedID.ID), + WithLabelValues(changefeedID.Namespace, changefeedID.ID, logType), metricFlushLogDuration: common.RedoFlushLogDurationHistogram. - WithLabelValues(changefeedID.Namespace, changefeedID.ID), + WithLabelValues(changefeedID.Namespace, changefeedID.ID, logType), metricTotalRowsCount: common.RedoTotalRowsCountGauge. - WithLabelValues(changefeedID.Namespace, changefeedID.ID), + WithLabelValues(changefeedID.Namespace, changefeedID.ID, logType), metricRedoWorkerBusyRatio: common.RedoWorkerBusyRatio. - WithLabelValues(changefeedID.Namespace, changefeedID.ID), + WithLabelValues(changefeedID.Namespace, changefeedID.ID, logType), } } @@ -521,10 +521,10 @@ func (m *logManager) bgUpdateLog(ctx context.Context, flushDuration time.Duratio logErrCh := make(chan error, 1) handleErr := func(err error) { logErrCh <- err } - overseerTicker := time.NewTicker(time.Second * 5) + overseerDuration := time.Second * 5 + overseerTicker := time.NewTicker(overseerDuration) defer overseerTicker.Stop() var workTimeSlice time.Duration - startToWork := time.Now() for { select { case <-ctx.Done(): @@ -536,10 +536,8 @@ func (m *logManager) bgUpdateLog(ctx context.Context, flushDuration time.Duratio return nil // channel closed } err = m.handleEvent(ctx, event, &workTimeSlice) - case now := <-overseerTicker.C: - busyRatio := int(workTimeSlice.Seconds() / now.Sub(startToWork).Seconds() * 1000) - m.metricRedoWorkerBusyRatio.Add(float64(busyRatio)) - startToWork = now + case <-overseerTicker.C: + m.metricRedoWorkerBusyRatio.Add(workTimeSlice.Seconds()) workTimeSlice = 0 case err = <-logErrCh: } diff --git a/cdc/redo/meta_manager.go b/cdc/redo/meta_manager.go index cb79af6ff72..078939b8411 100644 --- a/cdc/redo/meta_manager.go +++ b/cdc/redo/meta_manager.go @@ -144,7 +144,7 @@ func (m *metaManager) preStart(ctx context.Context) error { m.extStorage = extStorage m.metricFlushLogDuration = common.RedoFlushLogDurationHistogram. - WithLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID) + WithLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID, redo.RedoMetaFileType) err = m.preCleanupExtStorage(ctx) if err != nil { @@ -469,17 +469,23 @@ func (m *metaManager) flush(ctx context.Context, meta common.LogMeta) error { return nil } -// Cleanup removes all redo logs of this manager, it is called when changefeed is removed -// only owner should call this method. -func (m *metaManager) Cleanup(ctx context.Context) error { - common.RedoWriteLogDurationHistogram. - DeleteLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID) +func (m *metaManager) cleanup(logType string) { common.RedoFlushLogDurationHistogram. - DeleteLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID) + DeleteLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID, logType) + common.RedoWriteLogDurationHistogram. + DeleteLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID, logType) common.RedoTotalRowsCountGauge. - DeleteLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID) + DeleteLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID, logType) common.RedoWorkerBusyRatio. - DeleteLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID) + DeleteLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID, logType) +} + +// Cleanup removes all redo logs of this manager, it is called when changefeed is removed +// only owner should call this method. +func (m *metaManager) Cleanup(ctx context.Context) error { + m.cleanup(redo.RedoMetaFileType) + m.cleanup(redo.RedoRowLogFileType) + m.cleanup(redo.RedoDDLLogFileType) return m.deleteAllLogs(ctx) } diff --git a/cdc/redo/writer/file/file.go b/cdc/redo/writer/file/file.go index e12a5df59de..39d3b4c4ce8 100644 --- a/cdc/redo/writer/file/file.go +++ b/cdc/redo/writer/file/file.go @@ -104,11 +104,11 @@ func NewFileWriter( storage: extStorage, metricFsyncDuration: common.RedoFsyncDurationHistogram. - WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID), + WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID, cfg.LogType), metricFlushAllDuration: common.RedoFlushAllDurationHistogram. - WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID), + WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID, cfg.LogType), metricWriteBytes: common.RedoWriteBytesGauge. - WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID), + WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID, cfg.LogType), } if w.op.GetUUIDGenerator != nil { w.uuidGenerator = w.op.GetUUIDGenerator() @@ -213,11 +213,11 @@ func (w *Writer) Close() error { } common.RedoFlushAllDurationHistogram. - DeleteLabelValues(w.cfg.ChangeFeedID.Namespace, w.cfg.ChangeFeedID.ID) + DeleteLabelValues(w.cfg.ChangeFeedID.Namespace, w.cfg.ChangeFeedID.ID, w.cfg.LogType) common.RedoFsyncDurationHistogram. - DeleteLabelValues(w.cfg.ChangeFeedID.Namespace, w.cfg.ChangeFeedID.ID) + DeleteLabelValues(w.cfg.ChangeFeedID.Namespace, w.cfg.ChangeFeedID.ID, w.cfg.LogType) common.RedoWriteBytesGauge. - DeleteLabelValues(w.cfg.ChangeFeedID.Namespace, w.cfg.ChangeFeedID.ID) + DeleteLabelValues(w.cfg.ChangeFeedID.Namespace, w.cfg.ChangeFeedID.ID, w.cfg.LogType) ctx, cancel := context.WithTimeout(context.Background(), redo.CloseTimeout) defer cancel() @@ -399,7 +399,6 @@ func (w *Writer) flushAndRotateFile() error { if err != nil { return nil } - w.metricFlushAllDuration.Observe(time.Since(start).Seconds()) return err diff --git a/cdc/redo/writer/file/file_test.go b/cdc/redo/writer/file/file_test.go index ca13a775e30..68965ec24ca 100644 --- a/cdc/redo/writer/file/file_test.go +++ b/cdc/redo/writer/file/file_test.go @@ -67,11 +67,11 @@ func TestWriterWrite(t *testing.T) { uint64buf: make([]byte, 8), running: *atomic.NewBool(true), metricWriteBytes: common.RedoWriteBytesGauge. - WithLabelValues("default", "test-cf"), + WithLabelValues("default", "test-cf", redo.RedoRowLogFileType), metricFsyncDuration: common.RedoFsyncDurationHistogram. - WithLabelValues("default", "test-cf"), + WithLabelValues("default", "test-cf", redo.RedoRowLogFileType), metricFlushAllDuration: common.RedoFlushAllDurationHistogram. - WithLabelValues("default", "test-cf"), + WithLabelValues("default", "test-cf", redo.RedoRowLogFileType), uuidGenerator: uuidGen, } @@ -158,11 +158,11 @@ func TestWriterWrite(t *testing.T) { uint64buf: make([]byte, 8), running: *atomic.NewBool(true), metricWriteBytes: common.RedoWriteBytesGauge. - WithLabelValues("default", "test-cf11"), + WithLabelValues("default", "test-cf11", redo.RedoRowLogFileType), metricFsyncDuration: common.RedoFsyncDurationHistogram. - WithLabelValues("default", "test-cf11"), + WithLabelValues("default", "test-cf11", redo.RedoRowLogFileType), metricFlushAllDuration: common.RedoFlushAllDurationHistogram. - WithLabelValues("default", "test-cf11"), + WithLabelValues("default", "test-cf11", redo.RedoRowLogFileType), uuidGenerator: uuidGen, } @@ -253,11 +253,11 @@ func TestNewFileWriter(t *testing.T) { uint64buf: make([]byte, 8), storage: mockStorage, metricWriteBytes: common.RedoWriteBytesGauge. - WithLabelValues("default", "test"), + WithLabelValues("default", "test", redo.RedoRowLogFileType), metricFsyncDuration: common.RedoFsyncDurationHistogram. - WithLabelValues("default", "test"), + WithLabelValues("default", "test", redo.RedoRowLogFileType), metricFlushAllDuration: common.RedoFlushAllDurationHistogram. - WithLabelValues("default", "test"), + WithLabelValues("default", "test", redo.RedoRowLogFileType), uuidGenerator: uuidGen, } w.running.Store(true) @@ -310,11 +310,11 @@ func TestRotateFileWithFileAllocator(t *testing.T) { }, uint64buf: make([]byte, 8), metricWriteBytes: common.RedoWriteBytesGauge. - WithLabelValues("default", "test"), + WithLabelValues("default", "test", redo.RedoRowLogFileType), metricFsyncDuration: common.RedoFsyncDurationHistogram. - WithLabelValues("default", "test"), + WithLabelValues("default", "test", redo.RedoRowLogFileType), metricFlushAllDuration: common.RedoFlushAllDurationHistogram. - WithLabelValues("default", "test"), + WithLabelValues("default", "test", redo.RedoRowLogFileType), storage: mockStorage, uuidGenerator: uuidGen, } @@ -377,11 +377,11 @@ func TestRotateFileWithoutFileAllocator(t *testing.T) { }, uint64buf: make([]byte, 8), metricWriteBytes: common.RedoWriteBytesGauge. - WithLabelValues("default", "test"), + WithLabelValues("default", "test", redo.RedoDDLLogFileType), metricFsyncDuration: common.RedoFsyncDurationHistogram. - WithLabelValues("default", "test"), + WithLabelValues("default", "test", redo.RedoDDLLogFileType), metricFlushAllDuration: common.RedoFlushAllDurationHistogram. - WithLabelValues("default", "test"), + WithLabelValues("default", "test", redo.RedoDDLLogFileType), storage: mockStorage, uuidGenerator: uuidGen, } diff --git a/cdc/redo/writer/memory/file_worker.go b/cdc/redo/writer/memory/file_worker.go index b92e37e59ee..3bb398954a1 100644 --- a/cdc/redo/writer/memory/file_worker.go +++ b/cdc/redo/writer/memory/file_worker.go @@ -144,9 +144,9 @@ func newFileWorkerGroup( }, flushCh: make(chan *fileCache), metricWriteBytes: common.RedoWriteBytesGauge. - WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID), + WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID, cfg.LogType), metricFlushAllDuration: common.RedoFlushAllDurationHistogram. - WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID), + WithLabelValues(cfg.ChangeFeedID.Namespace, cfg.ChangeFeedID.ID, cfg.LogType), } } @@ -179,9 +179,9 @@ func (f *fileWorkerGroup) Run( func (f *fileWorkerGroup) close() { common.RedoFlushAllDurationHistogram. - DeleteLabelValues(f.cfg.ChangeFeedID.Namespace, f.cfg.ChangeFeedID.ID) + DeleteLabelValues(f.cfg.ChangeFeedID.Namespace, f.cfg.ChangeFeedID.ID, f.cfg.LogType) common.RedoWriteBytesGauge. - DeleteLabelValues(f.cfg.ChangeFeedID.Namespace, f.cfg.ChangeFeedID.ID) + DeleteLabelValues(f.cfg.ChangeFeedID.Namespace, f.cfg.ChangeFeedID.ID, f.cfg.LogType) } func (f *fileWorkerGroup) bgFlushFileCache(egCtx context.Context) error { @@ -191,6 +191,7 @@ func (f *fileWorkerGroup) bgFlushFileCache(egCtx context.Context) error { return errors.Trace(egCtx.Err()) case file := <-f.flushCh: start := time.Now() + if err := file.writer.Close(); err != nil { return errors.Trace(err) } diff --git a/cdc/sink/dmlsink/cloudstorage/dml_worker.go b/cdc/sink/dmlsink/cloudstorage/dml_worker.go index 731e6085803..ddb95675cec 100644 --- a/cdc/sink/dmlsink/cloudstorage/dml_worker.go +++ b/cdc/sink/dmlsink/cloudstorage/dml_worker.go @@ -130,7 +130,7 @@ func newDMLWorker( WithLabelValues(changefeedID.Namespace, changefeedID.ID), metricFlushDuration: mcloudstorage.CloudStorageFlushDurationHistogram. WithLabelValues(changefeedID.Namespace, changefeedID.ID), - metricsWorkerBusyRatio: mcloudstorage.CloudStorageWorkerBusyRatioCounter. + metricsWorkerBusyRatio: mcloudstorage.CloudStorageWorkerBusyRatio. WithLabelValues(changefeedID.Namespace, changefeedID.ID, strconv.Itoa(id)), } @@ -158,19 +158,16 @@ func (d *dmlWorker) run(ctx context.Context) error { // flushMessages flushed messages of active tables to cloud storage. // active tables are those tables that have received events after the last flush. func (d *dmlWorker) flushMessages(ctx context.Context) error { - var flushTimeSlice, totalTimeSlice time.Duration - overseerTicker := time.NewTicker(d.config.FlushInterval * 2) + var flushTimeSlice time.Duration + overseerDuration := d.config.FlushInterval * 2 + overseerTicker := time.NewTicker(overseerDuration) defer overseerTicker.Stop() - startToWork := time.Now() for { select { case <-ctx.Done(): return errors.Trace(ctx.Err()) - case now := <-overseerTicker.C: - totalTimeSlice = now.Sub(startToWork) - busyRatio := flushTimeSlice.Seconds() / totalTimeSlice.Seconds() * 1000 - d.metricsWorkerBusyRatio.Add(busyRatio) - startToWork = now + case <-overseerTicker.C: + d.metricsWorkerBusyRatio.Add(flushTimeSlice.Seconds()) flushTimeSlice = 0 case batchedTask := <-d.toBeFlushedCh: if atomic.LoadUint64(&d.isClosed) == 1 { diff --git a/cdc/sink/metrics/cloudstorage/metrics.go b/cdc/sink/metrics/cloudstorage/metrics.go index 52905b71346..74506ea7c4a 100644 --- a/cdc/sink/metrics/cloudstorage/metrics.go +++ b/cdc/sink/metrics/cloudstorage/metrics.go @@ -56,12 +56,12 @@ var ( }, []string{"namespace", "changefeed"}) // CloudStorageWorkerBusyRatio records the busy ratio of CloudStorage bgUpdateLog worker. - CloudStorageWorkerBusyRatioCounter = prometheus.NewCounterVec( + CloudStorageWorkerBusyRatio = prometheus.NewCounterVec( prometheus.CounterOpts{ Namespace: namespace, Subsystem: subsystem, Name: "cloud_storage_worker_busy_ratio", - Help: "Busy ratio (X ms in 1s) for cloud storage sink dml worker.", + Help: "Busy ratio for cloud storage sink dml worker.", }, []string{"namespace", "changefeed", "id"}) ) @@ -71,5 +71,5 @@ func InitMetrics(registry *prometheus.Registry) { registry.MustRegister(CloudStorageFileCountGauge) registry.MustRegister(CloudStorageWriteDurationHistogram) registry.MustRegister(CloudStorageFlushDurationHistogram) - registry.MustRegister(CloudStorageWorkerBusyRatioCounter) + registry.MustRegister(CloudStorageWorkerBusyRatio) } diff --git a/metrics/grafana/TiCDC-Monitor-Summary.json b/metrics/grafana/TiCDC-Monitor-Summary.json index d9d59b613a3..c2ffd96e7b5 100644 --- a/metrics/grafana/TiCDC-Monitor-Summary.json +++ b/metrics/grafana/TiCDC-Monitor-Summary.json @@ -2544,11 +2544,11 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(ticdc_redo_total_rows_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", namespace=~\"$namespace\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (instance)", + "expr": "sum(rate(ticdc_redo_total_rows_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", namespace=~\"$namespace\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (instance,type)", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}", + "legendFormat": "{{instance}}-{{type}}", "queryType": "randomWalk", "refId": "A" }, @@ -2652,9 +2652,9 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(ticdc_redo_write_bytes_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", namespace=~\"$namespace\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (instance)", + "expr": "sum(rate(ticdc_redo_write_bytes_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", namespace=~\"$namespace\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (instance,type)", "interval": "", - "legendFormat": "{{instance}}", + "legendFormat": "{{instance}}-{{type}}", "queryType": "randomWalk", "refId": "A" } @@ -2812,7 +2812,7 @@ "targets": [ { "exemplar": true, - "expr": "max(rate(ticdc_redo_flushall_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", namespace=~\"$namespace\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (le)", + "expr": "max(rate(ticdc_redo_flush_all_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", namespace=~\"$namespace\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (le)", "format": "heatmap", "interval": "", "intervalFactor": 2, @@ -2823,7 +2823,7 @@ ], "timeFrom": null, "timeShift": null, - "title": "Redo flushall duration", + "title": "Redo flush all duration", "tooltip": { "show": true, "showHistogram": true @@ -3164,5 +3164,5 @@ "timezone": "browser", "title": "Test-Cluster-TiCDC-Summary", "uid": "McUpY954z", - "version": 2 -} + "version": 3 +} \ No newline at end of file diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index aa159db19e3..e2b37d14ab0 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -9404,7 +9404,7 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(ticdc_sink_cloud_storage_worker_busy_ratio{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",namespace=~\"$namespace\",changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])/10) by (namespace,changefeed,id,instance)", + "expr": "sum(rate(ticdc_sink_cloud_storage_worker_busy_ratio{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",namespace=~\"$namespace\",changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])*100) by (namespace,changefeed,id,instance)", "hide": false, "interval": "", "legendFormat": "{{namespace}}-{{changefeed}}-{{id}}-{{instance}}", @@ -18963,7 +18963,7 @@ "targets": [ { "exemplar": true, - "expr": "max(rate(ticdc_redo_flushall_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (le)", + "expr": "max(rate(ticdc_redo_flush_all_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (le)", "format": "heatmap", "interval": "", "intervalFactor": 2, @@ -18974,7 +18974,7 @@ ], "timeFrom": null, "timeShift": null, - "title": "Redo flushall duration", + "title": "Redo flush all duration", "tooltip": { "show": true, "showHistogram": true @@ -19197,11 +19197,11 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(ticdc_redo_total_rows_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (instance)", + "expr": "sum(rate(ticdc_redo_total_rows_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (instance,type)", "format": "time_series", "interval": "", "intervalFactor": 1, - "legendFormat": "{{instance}}", + "legendFormat": "{{instance}}-{{type}}", "queryType": "randomWalk", "refId": "A" }, @@ -19305,9 +19305,9 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(ticdc_redo_write_bytes_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (instance)", + "expr": "sum(rate(ticdc_redo_write_bytes_total{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])) by (instance,type)", "interval": "", - "legendFormat": "{{instance}}", + "legendFormat": "{{instance}}-{{type}}", "queryType": "randomWalk", "refId": "A" } @@ -19403,9 +19403,9 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(ticdc_redo_worker_busy_ratio{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])/10) by (changefeed,instance)", + "expr": "sum(rate(ticdc_redo_worker_busy_ratio{k8s_cluster=\"$k8s_cluster\",tidb_cluster=\"$tidb_cluster\",changefeed=~\"$changefeed\",instance=~\"$ticdc_instance\"}[1m])*100) by (changefeed,instance,type)", "interval": "", - "legendFormat": "{{changefeed}}-{{instance}}", + "legendFormat": "{{changefeed}}-{{instance}}-{{type}}", "queryType": "randomWalk", "refId": "A" } @@ -21651,5 +21651,5 @@ "timezone": "browser", "title": "${DS_TEST-CLUSTER}-TiCDC", "uid": "YiGL8hBZ1", - "version": 61 -} + "version": 62 +} \ No newline at end of file