diff --git a/cdc/metrics.go b/cdc/metrics.go index 04d23521a1a..0f3e85a7dc7 100644 --- a/cdc/metrics.go +++ b/cdc/metrics.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tiflow/cdc/puller" redowriter "github.com/pingcap/tiflow/cdc/redo/writer" "github.com/pingcap/tiflow/cdc/sink" + "github.com/pingcap/tiflow/cdc/sink/producer/kafka" "github.com/pingcap/tiflow/cdc/sorter/memory" "github.com/pingcap/tiflow/cdc/sorter/unified" "github.com/pingcap/tiflow/pkg/actor" @@ -51,4 +52,5 @@ func init() { memory.InitMetrics(registry) unified.InitMetrics(registry) redowriter.InitMetrics(registry) + kafka.InitMetrics(registry) } diff --git a/cdc/owner/ddl_sink.go b/cdc/owner/ddl_sink.go index 22ed26e7b16..d4c17256f5b 100644 --- a/cdc/owner/ddl_sink.go +++ b/cdc/owner/ddl_sink.go @@ -27,6 +27,7 @@ import ( cdcContext "github.com/pingcap/tiflow/pkg/context" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/filter" + "github.com/pingcap/tiflow/pkg/util" "go.uber.org/zap" ) @@ -90,7 +91,9 @@ func ddlSinkInitializer(ctx cdcContext.Context, a *ddlSinkImpl, id model.ChangeF return errors.Trace(err) } - s, err := sink.New(ctx, id, info.SinkURI, filter, info.Config, info.Opts, a.errCh) + stdCtx := util.PutChangefeedIDInCtx(ctx, id) + stdCtx = util.PutRoleInCtx(stdCtx, util.RoleOwner) + s, err := sink.New(stdCtx, id, info.SinkURI, filter, info.Config, info.Opts, a.errCh) if err != nil { return errors.Trace(err) } @@ -99,13 +102,13 @@ func ddlSinkInitializer(ctx cdcContext.Context, a *ddlSinkImpl, id model.ChangeF if !info.SyncPointEnabled { return nil } - syncPointStore, err := sink.NewSyncpointStore(ctx, id, info.SinkURI) + syncPointStore, err := sink.NewSyncpointStore(stdCtx, id, info.SinkURI) if err != nil { return errors.Trace(err) } a.syncPointStore = syncPointStore - if err := a.syncPointStore.CreateSynctable(ctx); err != nil { + if err := a.syncPointStore.CreateSynctable(stdCtx); err != nil { return errors.Trace(err) } return nil @@ -121,11 +124,13 @@ func (s *ddlSinkImpl) run(ctx cdcContext.Context, id model.ChangeFeedID, info *m start := time.Now() if err := s.sinkInitHandler(ctx, s, id, info); err != nil { - log.Warn("ddl sink initialize failed", zap.Duration("elapsed", time.Since(start))) + log.Warn("ddl sink initialize failed", + zap.Duration("duration", time.Since(start))) ctx.Throw(err) return } - log.Info("ddl sink initialized, start processing...", zap.Duration("elapsed", time.Since(start))) + log.Info("ddl sink initialized, start processing...", + zap.Duration("duration", time.Since(start))) // TODO make the tick duration configurable ticker := time.NewTicker(time.Second) @@ -163,7 +168,7 @@ func (s *ddlSinkImpl) run(ctx cdcContext.Context, id model.ChangeFeedID, info *m } // If DDL executing failed, and the error can not be ignored, throw an error and pause the changefeed log.Error("Execute DDL failed", - zap.String("ChangeFeedID", ctx.ChangefeedVars().ID), + zap.String("changefeed", ctx.ChangefeedVars().ID), zap.Error(err), zap.Reflect("ddl", ddl)) ctx.Throw(errors.Trace(err)) diff --git a/cdc/sink/producer/kafka/kafka.go b/cdc/sink/producer/kafka/kafka.go index 69bd3a37b41..cc57e9e8ef1 100644 --- a/cdc/sink/producer/kafka/kafka.go +++ b/cdc/sink/producer/kafka/kafka.go @@ -38,7 +38,13 @@ import ( "go.uber.org/zap" ) -const defaultPartitionNum = 3 +const ( + // defaultPartitionNum specifies the default number of partitions when we create the topic. + defaultPartitionNum = 3 + + // flushMetricsInterval specifies the interval of refresh sarama metrics. + flushMetricsInterval = 5 * time.Second +) // Config stores user specified Kafka producer configuration type Config struct { @@ -231,8 +237,11 @@ type kafkaSaramaProducer struct { closeCh chan struct{} // atomic flag indicating whether the producer is closing closing kafkaProducerClosingFlag - role util.Role - id model.ChangeFeedID + + role util.Role + id model.ChangeFeedID + + metricsMonitor *saramaMetricsMonitor } type kafkaProducerClosingFlag = int32 @@ -415,6 +424,8 @@ func (k *kafkaSaramaProducer) Close() error { log.Info("sync client closed", zap.Duration("duration", time.Since(start)), zap.String("changefeed", k.id), zap.Any("role", k.role)) } + + k.metricsMonitor.Cleanup() return nil } @@ -425,12 +436,17 @@ func (k *kafkaSaramaProducer) run(ctx context.Context) error { zap.String("changefeed", k.id), zap.Any("role", k.role)) k.stop() }() + + ticker := time.NewTicker(flushMetricsInterval) + defer ticker.Stop() for { select { case <-ctx.Done(): return ctx.Err() case <-k.closeCh: return nil + case <-ticker.C: + k.metricsMonitor.CollectMetrics() case err := <-k.failpointCh: log.Warn("receive from failpoint chan", zap.Error(err), zap.String("changefeed", k.id), zap.Any("role", k.role)) @@ -603,6 +619,12 @@ func NewKafkaSaramaProducer(ctx context.Context, topic string, config *Config, o closeCh: make(chan struct{}), failpointCh: make(chan error, 1), closing: kafkaProducerRunning, + + id: changefeedID, + role: role, + + metricsMonitor: NewSaramaMetricsMonitor(cfg.MetricRegistry, + util.CaptureAddrFromCtx(ctx), changefeedID), } go func() { if err := k.run(ctx); err != nil && errors.Cause(err) != context.Canceled { @@ -611,10 +633,12 @@ func NewKafkaSaramaProducer(ctx context.Context, topic string, config *Config, o return case errCh <- err: default: - log.Error("error channel is full", zap.Error(err)) + log.Error("error channel is full", zap.Error(err), + zap.String("changefeed", k.id), zap.Any("role", role)) } } }() + return k, nil } diff --git a/cdc/sink/producer/kafka/metrics.go b/cdc/sink/producer/kafka/metrics.go new file mode 100644 index 00000000000..600e2eb3940 --- /dev/null +++ b/cdc/sink/producer/kafka/metrics.go @@ -0,0 +1,118 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package kafka + +import ( + "github.com/prometheus/client_golang/prometheus" + "github.com/rcrowley/go-metrics" +) + +var ( + // batch-size + batchSizeGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_batch_size", + Help: "the number of bytes sent per partition per request for all topics", + }, []string{"capture", "changefeed"}) + + // record-send-rate + recordSendRateGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_record_send_rate", + Help: "Records/second sent to all topics", + }, []string{"capture", "changefeed"}) + + // records-per-request + recordPerRequestGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_records_per_request", + Help: "the number of records sent per request for all topics", + }, []string{"capture", "changefeed"}) + + // compression-ratio + compressionRatioGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_compression_ratio", + Help: "the compression ratio times 100 of record batches for all topics", + }, []string{"capture", "changefeed"}) +) + +// InitMetrics registers all metrics in this file +func InitMetrics(registry *prometheus.Registry) { + registry.MustRegister(batchSizeGauge) + registry.MustRegister(recordSendRateGauge) + registry.MustRegister(recordPerRequestGauge) + registry.MustRegister(compressionRatioGauge) +} + +// sarama metrics names, see https://pkg.go.dev/github.com/Shopify/sarama#pkg-overview +const ( + batchSizeMetricName = "batch-size" + recordSendRateMetricName = "record-send-rate" + recordPerRequestMetricName = "records-per-request" + compressionRatioMetricName = "compression-ratio" +) + +type saramaMetricsMonitor struct { + captureAddr string + changefeedID string + + registry metrics.Registry +} + +// CollectMetrics collect all monitored metrics +func (sm *saramaMetricsMonitor) CollectMetrics() { + batchSizeMetric := sm.registry.Get(batchSizeMetricName) + if histogram, ok := batchSizeMetric.(metrics.Histogram); ok { + batchSizeGauge.WithLabelValues(sm.captureAddr, sm.changefeedID).Set(histogram.Mean()) + } + + recordSendRateMetric := sm.registry.Get(recordSendRateMetricName) + if meter, ok := recordSendRateMetric.(metrics.Meter); ok { + recordSendRateGauge.WithLabelValues(sm.captureAddr, sm.changefeedID).Set(meter.Rate1()) + } + + recordPerRequestMetric := sm.registry.Get(recordPerRequestMetricName) + if histogram, ok := recordPerRequestMetric.(metrics.Histogram); ok { + recordPerRequestGauge.WithLabelValues(sm.captureAddr, sm.changefeedID).Set(histogram.Mean()) + } + + compressionRatioMetric := sm.registry.Get(compressionRatioMetricName) + if histogram, ok := compressionRatioMetric.(metrics.Histogram); ok { + compressionRatioGauge.WithLabelValues(sm.captureAddr, sm.changefeedID).Set(histogram.Mean()) + } +} + +func NewSaramaMetricsMonitor(registry metrics.Registry, captureAddr, changefeedID string) *saramaMetricsMonitor { + return &saramaMetricsMonitor{ + captureAddr: captureAddr, + changefeedID: changefeedID, + registry: registry, + } +} + +func (sm *saramaMetricsMonitor) Cleanup() { + batchSizeGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID) + recordSendRateGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID) + recordPerRequestGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID) + compressionRatioGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID) +} diff --git a/go.mod b/go.mod index 8a632604be4..c99aff996ee 100644 --- a/go.mod +++ b/go.mod @@ -62,6 +62,7 @@ require ( github.com/prometheus/client_model v0.2.0 github.com/r3labs/diff v1.1.0 github.com/rakyll/statik v0.1.7 + github.com/rcrowley/go-metrics v0.0.0-20200313005456-10cdbea86bc0 github.com/shopspring/decimal v1.3.0 github.com/soheilhy/cmux v0.1.5 github.com/spf13/cobra v1.2.1 diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index 1d350f3470b..64a9bfd5035 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -124,8 +124,8 @@ "editable": true, "gnetId": null, "graphTooltip": 1, - "id": null, - "iteration": 1640447119260, + "id": 33, + "iteration": 1644481309661, "links": [], "panels": [ { @@ -942,7 +942,7 @@ "h": 5, "w": 7, "x": 0, - "y": 3 + "y": 2 }, "id": 4, "links": [], @@ -1005,7 +1005,7 @@ "h": 10, "w": 7, "x": 7, - "y": 3 + "y": 2 }, "id": 90, "links": [], @@ -1285,7 +1285,7 @@ "h": 5, "w": 7, "x": 0, - "y": 8 + "y": 7 }, "id": 138, "links": [], @@ -1357,7 +1357,7 @@ "h": 7, "w": 9, "x": 0, - "y": 13 + "y": 12 }, "id": 86, "legend": { @@ -1472,7 +1472,7 @@ "h": 7, "w": 8, "x": 9, - "y": 13 + "y": 12 }, "hiddenSeries": false, "id": 102, @@ -1568,7 +1568,7 @@ "h": 7, "w": 7, "x": 17, - "y": 13 + "y": 12 }, "id": 82, "legend": { @@ -1657,7 +1657,7 @@ "h": 7, "w": 12, "x": 0, - "y": 20 + "y": 19 }, "hiddenSeries": false, "id": 3, @@ -1776,7 +1776,7 @@ "h": 7, "w": 12, "x": 12, - "y": 20 + "y": 19 }, "hiddenSeries": false, "id": 2, @@ -1872,7 +1872,7 @@ "h": 7, "w": 12, "x": 0, - "y": 27 + "y": 26 }, "id": 163, "legend": { @@ -1967,7 +1967,7 @@ "h": 7, "w": 12, "x": 12, - "y": 27 + "y": 26 }, "hiddenSeries": false, "id": 253, @@ -2073,7 +2073,7 @@ "h": 7, "w": 12, "x": 0, - "y": 34 + "y": 33 }, "heatmap": {}, "hideZeroBuckets": true, @@ -2144,7 +2144,7 @@ "h": 7, "w": 12, "x": 12, - "y": 34 + "y": 33 }, "hiddenSeries": false, "id": 35, @@ -2252,7 +2252,7 @@ "h": 7, "w": 12, "x": 0, - "y": 41 + "y": 40 }, "hiddenSeries": false, "id": 34, @@ -2355,7 +2355,7 @@ "h": 7, "w": 12, "x": 12, - "y": 41 + "y": 40 }, "hiddenSeries": false, "id": 36, @@ -2472,7 +2472,7 @@ "h": 7, "w": 12, "x": 0, - "y": 48 + "y": 47 }, "heatmap": {}, "hideZeroBuckets": true, @@ -2541,7 +2541,7 @@ "h": 7, "w": 12, "x": 12, - "y": 48 + "y": 47 }, "hiddenSeries": false, "id": 98, @@ -2657,7 +2657,7 @@ "h": 7, "w": 12, "x": 0, - "y": 55 + "y": 54 }, "heatmap": {}, "hideZeroBuckets": true, @@ -2727,7 +2727,7 @@ "h": 7, "w": 12, "x": 12, - "y": 55 + "y": 54 }, "hiddenSeries": false, "id": 95, @@ -2876,7 +2876,7 @@ "h": 7, "w": 12, "x": 0, - "y": 62 + "y": 61 }, "heatmap": {}, "hideZeroBuckets": true, @@ -2946,7 +2946,7 @@ "h": 7, "w": 12, "x": 12, - "y": 62 + "y": 61 }, "hiddenSeries": false, "id": 83, @@ -3054,7 +3054,7 @@ "h": 7, "w": 12, "x": 0, - "y": 69 + "y": 68 }, "id": 149, "legend": { @@ -3141,7 +3141,7 @@ "h": 7, "w": 12, "x": 12, - "y": 69 + "y": 68 }, "hiddenSeries": false, "id": 166, @@ -3238,91 +3238,6 @@ "alignLevel": null } }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "fill": 1, - "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 76 - }, - "id": 164, - "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, - "show": true, - "total": false, - "values": false - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "sum(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 }}", - "refId": "A" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "Processor Memory Consumption Per Changefeed", - "tooltip": { - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "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 - } - }, { "columns": [], "datasource": "${DS_TEST-CLUSTER}", @@ -3383,7 +3298,7 @@ "h": 8, "w": 12, "x": 0, - "y": 77 + "y": 75 }, "id": 200, "options": { @@ -3475,6 +3390,91 @@ } ], "type": "table" + }, + { + "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, + "current": false, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(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 }}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Processor Memory Consumption Per Changefeed", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "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 + } } ], "title": "Changefeed", @@ -6182,7 +6182,21 @@ "yBucketBound": "upper", "yBucketNumber": null, "yBucketSize": null - }, + } + ], + "title": "DB", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 4 + }, + "id": 13, + "panels": [ { "aliasColors": {}, "bars": false, @@ -7595,20 +7609,19 @@ }, { "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))", + "expr": "abs(sum(ticdc_kvclient_cached_region{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"} / deriv(ticdc_kvclient_cached_region{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture, changefeed, store))", "format": "time_series", - "hide": false, "interval": "", "intervalFactor": 1, - "legendFormat": "{{capture}}-p99", - "refId": "B" + "legendFormat": "{{changefeed}}-{{capture}}-{{store}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "EtcdWorker exec etcd txn duration percentile", + "title": "Estimate remaining time for initialization", "tooltip": { "shared": true, "sort": 0, @@ -7625,7 +7638,7 @@ "yaxes": [ { "format": "s", - "label": null, + "label": "", "logBase": 2, "max": null, "min": null, @@ -7644,101 +7657,35 @@ "align": false, "alignLevel": null } - }, + } + ], + "title": "Events", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 5 + }, + "id": 130, + "panels": [ { - "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": 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": 7, + "h": 8, "w": 12, - "x": 12, - "y": 37 + "x": 0, + "y": 5 }, - "hiddenSeries": false, - "id": 260, + "id": 131, "legend": { "avg": false, "current": false, @@ -7750,12 +7697,9 @@ }, "lines": true, "linewidth": 1, + "links": [], "nullPointMode": "null", - "options": { - "alertThreshold": true - }, "percentage": false, - "pluginVersion": "6.1.6", "pointradius": 2, "points": false, "renderer": "flot", @@ -7765,27 +7709,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(rate(ticdc_sorter_consume_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture,changefeed)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{capture}}-{{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": "Unified Sorter intake rate", "tooltip": { "shared": true, "sort": 0, @@ -7801,7 +7736,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -7828,39 +7763,29 @@ "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": 7, + "h": 8, "w": 12, - "x": 0, - "y": 44 + "x": 12, + "y": 5 }, - "id": 291, + "id": 132, "legend": { - "alignAsTable": true, "avg": false, - "current": true, - "hideEmpty": true, - "hideZero": true, - "max": true, + "current": false, + "max": false, "min": false, - "rightSide": true, "show": true, - "sideWidth": 300, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, "links": [], "nullPointMode": "null", - "paceLength": 10, "percentage": false, - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -7869,25 +7794,22 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(etcd_disk_wal_fsync_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[5m])) by (instance, le))", + "expr": "sum(rate(ticdc_sorter_event_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (capture,changefeed)", "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{instance}}", - "metric": "", - "refId": "A", - "step": 4 + "intervalFactor": 1, + "legendFormat": "{{capture}}-{{changefeed}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Etcd 99% WAL fsync duration", + "title": "Unified Sorter event output rate", "tooltip": { - "msResolution": false, "shared": true, "sort": 0, - "value_type": "cumulative" + "value_type": "individual" }, "type": "graph", "xaxis": { @@ -7899,7 +7821,7 @@ }, "yaxes": [ { - "format": "s", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -7926,37 +7848,29 @@ "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": 7, + "h": 8, "w": 12, - "x": 12, - "y": 44 + "x": 0, + "y": 13 }, - "id": 290, + "id": 133, "legend": { - "alignAsTable": true, "avg": false, - "current": true, - "max": true, + "current": false, + "max": false, "min": false, - "rightSide": true, "show": true, - "sideWidth": 300, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, "links": [], "nullPointMode": "null", - "paceLength": 10, "percentage": false, - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", "seriesOverrides": [], @@ -7965,24 +7879,22 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(pd_txn_handle_txns_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\"}[5m])) by (instance, result, le))", + "expr": "sum(ticdc_sorter_on_disk_data_size_gauge{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}) by (capture)", "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{instance}} {{result}}", - "refId": "A", - "step": 4 + "intervalFactor": 1, + "legendFormat": "{{capture}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Etcd 99% Handle transactions duration", + "title": "Unified Sorter on disk data size", "tooltip": { - "msResolution": false, "shared": true, "sort": 0, - "value_type": "cumulative" + "value_type": "individual" }, "type": "graph", "xaxis": { @@ -7994,11 +7906,11 @@ }, "yaxes": [ { - "format": "s", + "format": "bytes", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -8014,109 +7926,2520 @@ "align": false, "alignLevel": null } - } - ], - "title": "Etcd", - "type": "row" - }, - { - "collapsed": true, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 6 - }, - "id": 58, - "panels": [ + }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "CPU usage of TiKV CDC component", "fill": 1, "gridPos": { - "h": 7, + "h": 8, "w": 12, - "x": 0, - "y": 7 + "x": 12, + "y": 13 }, - "id": 60, + "id": 134, "legend": { - "alignAsTable": true, "avg": false, - "current": true, - "max": true, + "current": false, + "max": false, "min": false, - "rightSide": true, "show": true, - "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, "links": [], "nullPointMode": "null", - "paceLength": 10, "percentage": false, - "pointradius": 5, + "pointradius": 2, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/.*workers/", - "transform": "negative-Y", - "yaxis": 2 - }, - { - "alias": "/.*tso/", - "fill": 0, - "pointradius": 1, - "points": true - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "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}}-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)", + "expr": "sum(ticdc_sorter_in_memory_data_size_gauge{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}) by (capture)", "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)", + "intervalFactor": 1, + "legendFormat": "{{capture}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Unified Sorter in-memory data size", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "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": "interpolateOranges", + "exponent": 0.5, + "max": null, + "min": null, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 21 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 135, + "legend": { + "show": false + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(rate(ticdc_sorter_flush_count_histogram_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "intervalFactor": 1, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Unified Sorter flush sizes", + "tooltip": { + "show": true, + "showHistogram": false + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": null, + "format": "none", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "auto", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "cards": { + "cardPadding": null, + "cardRound": null + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateBlues", + "exponent": 0.5, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 21 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 136, + "legend": { + "show": false + }, + "links": [], + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(rate(ticdc_sorter_merge_count_histogram_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "intervalFactor": 1, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Unified Sorter merge size", + "tooltip": { + "show": true, + "showHistogram": false + }, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": null, + "format": "none", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "auto", + "yBucketNumber": null, + "yBucketSize": null + } + ], + "title": "Unified Sorter", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 6 + }, + "id": 266, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fill": 1, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 16 + }, + "id": 289, + "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", + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "etcd_debugging_mvcc_db_total_size_in_bytes{tidb_cluster=\"$tidb_cluster\", job=\"pd\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Etcd MVCC DB total size", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "decbytes", + "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}", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 16 + }, + "hiddenSeries": false, + "id": 114, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "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": "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": "Etcd health check duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "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": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "max": null, + "min": 1, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 23 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 267, + "legend": { + "show": true + }, + "pluginVersion": "6.1.6", + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(ticdc_etcd_worker_tick_reactor_duration_bucket{tidb_cluster=\"$tidb_cluster\", capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "EtcdWorker exec etcd txn duration", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 23 + }, + "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": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "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": 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": 7, + "w": 12, + "x": 12, + "y": 30 + }, + "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": "", + "intervalFactor": 1, + "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": "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": [ + { + "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": 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": 7, + "w": 12, + "x": 12, + "y": 37 + }, + "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": [ + { + "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": {}, + "bars": false, + "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": 7, + "w": 12, + "x": 0, + "y": 44 + }, + "id": 291, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": 300, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "paceLength": 10, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "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": 2, + "legendFormat": "{{instance}}", + "metric": "", + "refId": "A", + "step": 4 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Etcd 99% WAL fsync duration", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "cumulative" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "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 time consumed of handling etcd transactions in .99", + "editable": true, + "error": false, + "fill": 1, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 44 + }, + "id": 290, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": 300, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "paceLength": 10, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "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": 2, + "legendFormat": "{{instance}} {{result}}", + "refId": "A", + "step": 4 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Etcd 99% Handle transactions duration", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "cumulative" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "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": "Etcd", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 7 + }, + "id": 58, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "CPU usage of TiKV CDC component", + "fill": 1, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 7 + }, + "id": 60, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "paceLength": 10, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/.*workers/", + "transform": "negative-Y", + "yaxis": 2 + }, + { + "alias": "/.*tso/", + "fill": 0, + "pointradius": 1, + "points": true + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(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}}-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 CPU", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "percentunit", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "percent", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": true, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "Outbound network traffic of TiKV CDC component", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 7 + }, + "id": 74, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "hideZero": false, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": false, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "paceLength": 10, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/.*resolved_ts/", + "bars": false, + "fill": 1, + "lines": true, + "linewidth": 2, + "transform": "negative-Y", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(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}}-{{type}}", + "refId": "A", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "CDC network traffic", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "Bps", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "The count of different kinds of gRPC message", + "fill": 1, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 14 + }, + "id": 147, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(tikv_grpc_msg_duration_seconds_count{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", type!=\"kv_gc\"}[1m])) by (type)", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}", + "metric": "tikv_grpc_msg_duration_seconds_bucket", + "refId": "A", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "gRPC message count", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "ops", + "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}", + "decimals": 1, + "description": "The memory usage per TiKV instance", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 14 + }, + "id": 194, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "paceLength": 10, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/.*-cap-.*/", + "yaxis": 2 + }, + { + "alias": "/.*tikv.*/", + "pointradius": 1, + "points": true, + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "avg(process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", job=~\"tikv.*\"}) by (instance)", + "format": "time_series", + "hide": true, + "intervalFactor": 2, + "legendFormat": "tikv-{{instance}}", + "refId": "A", + "step": 10 + }, + { + "expr": "avg(process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", job=~\"cdc.*\"}) by (instance)", + "format": "time_series", + "hide": true, + "intervalFactor": 2, + "legendFormat": "cdc-{{instance}}", + "refId": "B", + "step": 10 + }, + { + "expr": "(avg(process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", job=~\"tikv.*\"}) by (instance)) - (avg(tikv_engine_block_cache_size_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", db=\"kv\"}) by(instance))", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "tikv-{{instance}}", + "refId": "C", + "step": 10 + }, + { + "expr": "sum(tikv_cdc_sink_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "sink-{{instance}}", + "refId": "D", + "step": 10 + }, + { + "expr": "sum(tikv_cdc_old_value_cache_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "old-value-{{instance}}", + "refId": "E", + "step": 10 + }, + { + "expr": "sum(tikv_cdc_sink_memory_capacity{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "format": "time_series", + "hide": true, + "intervalFactor": 2, + "legendFormat": "sink-cap-{{instance}}", + "refId": "F", + "step": 10 + }, + { + "expr": "sum(tikv_cdc_old_value_cache_memory_quota{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "format": "time_series", + "hide": true, + "intervalFactor": 2, + "legendFormat": "old-value-cap-{{instance}}", + "refId": "G", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "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": [ + { + "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": 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": 7, + "w": 12, + "x": 12, + "y": 37 + }, + "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": [ + { + "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": {}, + "bars": false, + "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": 7, + "w": 12, + "x": 0, + "y": 44 + }, + "id": 291, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": 300, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "paceLength": 10, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "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": 2, + "legendFormat": "{{instance}}", + "metric": "", + "refId": "A", + "step": 4 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Etcd 99% WAL fsync duration", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "cumulative" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "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 time consumed of handling etcd transactions in .99", + "editable": true, + "error": false, + "fill": 1, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 44 + }, + "id": 290, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": 300, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "paceLength": 10, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "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": 2, + "legendFormat": "{{instance}} {{result}}", + "refId": "A", + "step": 4 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Etcd 99% Handle transactions duration", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "cumulative" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "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": "Etcd", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 6 + }, + "id": 58, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "CPU usage of TiKV CDC component", + "fill": 1, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 7 + }, + "id": 60, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "paceLength": 10, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/.*workers/", + "transform": "negative-Y", + "yaxis": 2 + }, + { + "alias": "/.*tso/", + "fill": 0, + "pointradius": 1, + "points": true + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(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}}-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 CPU", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "percentunit", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "percent", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": true, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "Outbound network traffic of TiKV CDC component", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 7 + }, + "id": 74, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "hideZero": false, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": false, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "paceLength": 10, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/.*resolved_ts/", + "bars": false, + "fill": 1, + "lines": true, + "linewidth": 2, + "transform": "negative-Y", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(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}}-{{type}}", + "refId": "A", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "CDC network traffic", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "Bps", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": 0 + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "The count of different kinds of gRPC message", + "fill": 1, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 14 + }, + "id": 147, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(tikv_grpc_msg_duration_seconds_count{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", type!=\"kv_gc\"}[1m])) by (type)", + "format": "time_series", + "intervalFactor": 2, + "legendFormat": "{{type}}", + "metric": "tikv_grpc_msg_duration_seconds_bucket", + "refId": "A", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "gRPC message count", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "ops", + "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}", + "decimals": 1, + "description": "The memory usage per TiKV instance", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 14 + }, + "id": 194, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "max", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "paceLength": 10, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/.*-cap-.*/", + "yaxis": 2 + }, + { + "alias": "/.*tikv.*/", + "pointradius": 1, + "points": true, + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "avg(process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", job=~\"tikv.*\"}) by (instance)", + "format": "time_series", + "hide": true, + "intervalFactor": 2, + "legendFormat": "tikv-{{instance}}", + "refId": "A", + "step": 10 + }, + { + "expr": "avg(process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", job=~\"cdc.*\"}) by (instance)", + "format": "time_series", + "hide": true, + "intervalFactor": 2, + "legendFormat": "cdc-{{instance}}", + "refId": "B", + "step": 10 + }, + { + "expr": "(avg(process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", job=~\"tikv.*\"}) by (instance)) - (avg(tikv_engine_block_cache_size_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", db=\"kv\"}) by(instance))", "format": "time_series", "hide": false, "intervalFactor": 2, - "legendFormat": "{{instance}}-tso", + "legendFormat": "tikv-{{instance}}", "refId": "C", - "step": 4 + "step": 10 + }, + { + "expr": "sum(tikv_cdc_sink_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "sink-{{instance}}", + "refId": "D", + "step": 10 + }, + { + "expr": "sum(tikv_cdc_old_value_cache_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "old-value-{{instance}}", + "refId": "E", + "step": 10 + }, + { + "expr": "sum(tikv_cdc_sink_memory_capacity{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "format": "time_series", + "hide": true, + "intervalFactor": 2, + "legendFormat": "sink-cap-{{instance}}", + "refId": "F", + "step": 10 + }, + { + "expr": "sum(tikv_cdc_old_value_cache_memory_quota{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "format": "time_series", + "hide": true, + "intervalFactor": 2, + "legendFormat": "old-value-cap-{{instance}}", + "refId": "G", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "CDC CPU", + "title": "CDC memory", "tooltip": { + "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -8131,15 +10454,146 @@ }, "yaxes": [ { - "format": "percentunit", + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "bytes", "label": null, "logBase": 1, "max": null, "min": null, "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "The min resolved ts of each TiKV", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 7, + "x": 0, + "y": 21 + }, + "id": 152, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "sideWidth": null, + "sort": null, + "sortDesc": null, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "paceLength": 10, + "percentage": false, + "pointradius": 1, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/.*-ts/", + "lines": true, + "linewidth": 3, + "points": false, + "yaxis": 2 }, { - "format": "percent", + "alias": "/.*-lag/", + "bars": true, + "fill": 1 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "scalar(max(pd_cluster_tso{tidb_cluster=\"$tidb_cluster\"}))/1000 - avg(tikv_cdc_min_resolved_ts{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}/1000) by (instance) > 0", + "format": "time_series", + "hide": false, + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{instance}}-min-resolved-lag", + "refId": "A", + "step": 10 + }, + { + "expr": "max(pd_cluster_tso{tidb_cluster=\"$tidb_cluster\"})", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "current-ts", + "refId": "B", + "step": 10 + }, + { + "expr": "avg(tikv_cdc_min_resolved_ts{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "{{instance}}-min-resolved-ts", + "refId": "C", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Min resolved ts", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "none", "label": null, "logBase": 1, "max": null, @@ -8154,36 +10608,34 @@ }, { "aliasColors": {}, - "bars": true, + "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "Outbound network traffic of TiKV CDC component", + "description": "The ID of the min resolved region of each TiKV", "editable": true, "error": false, "fill": 0, "grid": {}, "gridPos": { "h": 7, - "w": 12, - "x": 12, - "y": 7 + "w": 5, + "x": 7, + "y": 21 }, - "id": 74, + "id": 153, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": false, "max": true, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "sideWidth": null, - "sort": "max", - "sortDesc": true, + "sort": null, + "sortDesc": null, "total": false, "values": true }, @@ -8193,30 +10645,20 @@ "nullPointMode": "null", "paceLength": 10, "percentage": false, - "pointradius": 5, - "points": false, + "pointradius": 1, + "points": true, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/.*resolved_ts/", - "bars": false, - "fill": 1, - "lines": true, - "linewidth": 2, - "transform": "negative-Y", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(rate(tikv_cdc_grpc_message_sent_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[30s])) by (instance, type)", + "expr": "avg(tikv_cdc_min_resolved_ts_region{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", "format": "time_series", "hide": false, "intervalFactor": 2, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{instance}}-min-resolved-region", "refId": "A", "step": 10 } @@ -8225,7 +10667,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "CDC network traffic", + "title": "Min resolved Region", "tooltip": { "msResolution": false, "shared": true, @@ -8242,25 +10684,25 @@ }, "yaxes": [ { - "format": "Bps", + "format": "none", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { - "format": "bytes", + "format": "none", "label": null, "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { "align": false, - "alignLevel": 0 + "alignLevel": null } }, { @@ -8270,28 +10712,28 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The count of different kinds of gRPC message", + "description": "", "fill": 1, "gridPos": { "h": 7, "w": 12, - "x": 0, - "y": 14 + "x": 12, + "y": 21 }, - "id": 147, + "id": 70, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": true, + "hideEmpty": false, "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, "sideWidth": null, - "sort": "current", - "sortDesc": true, + "sort": null, + "sortDesc": null, "total": false, "values": true }, @@ -8299,6 +10741,7 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "paceLength": 10, "percentage": false, "pointradius": 5, "points": false, @@ -8309,20 +10752,18 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tikv_grpc_msg_duration_seconds_count{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", type!=\"kv_gc\"}[1m])) by (type)", + "expr": "histogram_quantile(0.99999, sum(rate(tikv_cdc_resolved_ts_gap_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance))", "format": "time_series", - "intervalFactor": 2, - "legendFormat": "{{type}}", - "metric": "tikv_grpc_msg_duration_seconds_bucket", - "refId": "A", - "step": 10 + "intervalFactor": 1, + "legendFormat": "{{instance}}-p9999", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "gRPC message count", + "title": "Resolved ts lag duration percentile", "tooltip": { "shared": true, "sort": 0, @@ -8338,9 +10779,9 @@ }, "yaxes": [ { - "format": "ops", + "format": "s", "label": null, - "logBase": 1, + "logBase": 10, "max": null, "min": null, "show": true @@ -8359,6 +10800,85 @@ "alignLevel": null } }, + { + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "The time consumed to CDC incremental scan", + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 28 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 68, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "links": [], + "maxPerRow": 3, + "repeat": null, + "repeatDirection": "h", + "reverseYBuckets": false, + "targets": [ + { + "expr": "sum(rate(tikv_cdc_scan_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, + "intervalFactor": 2, + "legendFormat": "{{le}}", + "refId": "A" + } + ], + "title": "Initial scan duration", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, { "aliasColors": {}, "bars": false, @@ -8366,128 +10886,59 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The memory usage per TiKV instance", - "editable": true, - "error": false, - "fill": 0, - "grid": {}, + "description": "", + "fill": 1, "gridPos": { "h": 7, - "w": 12, + "w": 6, "x": 12, - "y": 14 + "y": 28 }, - "id": 194, + "id": 72, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, + "hideZero": true, "max": true, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "sideWidth": null, - "sort": "max", + "sort": "current", "sortDesc": true, "total": false, "values": true }, "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "paceLength": 10, - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/.*-cap-.*/", - "yaxis": 2 - }, - { - "alias": "/.*tikv.*/", - "pointradius": 1, - "points": true, - "yaxis": 2 - } - ], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "avg(process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", job=~\"tikv.*\"}) by (instance)", - "format": "time_series", - "hide": true, - "intervalFactor": 2, - "legendFormat": "tikv-{{instance}}", - "refId": "A", - "step": 10 - }, - { - "expr": "avg(process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", job=~\"cdc.*\"}) by (instance)", - "format": "time_series", - "hide": true, - "intervalFactor": 2, - "legendFormat": "cdc-{{instance}}", - "refId": "B", - "step": 10 - }, - { - "expr": "(avg(process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", job=~\"tikv.*\"}) by (instance)) - (avg(tikv_engine_block_cache_size_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", db=\"kv\"}) by(instance))", - "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "tikv-{{instance}}", - "refId": "C", - "step": 10 - }, - { - "expr": "sum(tikv_cdc_sink_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", - "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "sink-{{instance}}", - "refId": "D", - "step": 10 - }, - { - "expr": "sum(tikv_cdc_old_value_cache_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", - "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "old-value-{{instance}}", - "refId": "E", - "step": 10 - }, - { - "expr": "sum(tikv_cdc_sink_memory_capacity{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", - "format": "time_series", - "hide": true, - "intervalFactor": 2, - "legendFormat": "sink-cap-{{instance}}", - "refId": "F", - "step": 10 - }, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "paceLength": 10, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ { - "expr": "sum(tikv_cdc_old_value_cache_memory_quota{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "expr": "histogram_quantile(0.9999, sum(rate(tikv_cdc_scan_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance))", "format": "time_series", - "hide": true, - "intervalFactor": 2, - "legendFormat": "old-value-cap-{{instance}}", - "refId": "G", - "step": 10 + "intervalFactor": 1, + "legendFormat": "{{instance}}-p9999", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "CDC memory", + "title": "Initial scan duration percentile", "tooltip": { - "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -8502,20 +10953,20 @@ }, "yaxes": [ { - "format": "bytes", + "format": "s", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { - "format": "bytes", + "format": "short", "label": null, "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -8530,29 +10981,28 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The min resolved ts of each TiKV", - "editable": true, - "error": false, - "fill": 0, - "grid": {}, + "description": "The number of incremental scan task in different status.", + "fill": 1, "gridPos": { "h": 7, - "w": 7, - "x": 0, - "y": 21 + "w": 6, + "x": 18, + "y": 28 }, - "id": 152, + "id": 140, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": false, + "hideEmpty": false, + "hideZero": true, + "max": true, "min": false, "rightSide": false, "show": true, "sideWidth": null, - "sort": null, - "sortDesc": null, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -8562,21 +11012,13 @@ "nullPointMode": "null", "paceLength": 10, "percentage": false, - "pointradius": 1, + "pointradius": 5, "points": false, "renderer": "flot", "seriesOverrides": [ { - "alias": "/.*-ts/", - "lines": true, - "linewidth": 3, - "points": false, + "alias": "/.*ongoing/", "yaxis": 2 - }, - { - "alias": "/.*-lag/", - "bars": true, - "fill": 1 } ], "spaceLength": 10, @@ -8584,41 +11026,28 @@ "steppedLine": false, "targets": [ { - "expr": "scalar(max(pd_cluster_tso{tidb_cluster=\"$tidb_cluster\"}))/1000 - avg(tikv_cdc_min_resolved_ts{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}/1000) by (instance) > 0", - "format": "time_series", - "hide": false, - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{instance}}-min-resolved-lag", - "refId": "A", - "step": 10 - }, - { - "expr": "max(pd_cluster_tso{tidb_cluster=\"$tidb_cluster\"})", + "expr": "sum(tikv_cdc_scan_tasks{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", type=\"ongoing\"}) by (type, instance)", "format": "time_series", "hide": false, - "intervalFactor": 2, - "legendFormat": "current-ts", - "refId": "B", - "step": 10 + "intervalFactor": 1, + "legendFormat": "{{instance}}-{{type}}", + "refId": "A" }, { - "expr": "avg(tikv_cdc_min_resolved_ts{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "expr": "sum(tikv_cdc_scan_tasks{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", type=\"total\"}) by (instance) - sum(tikv_cdc_scan_tasks{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", type=~\"abort|finish\"}) by (instance)", "format": "time_series", "hide": false, - "intervalFactor": 2, - "legendFormat": "{{instance}}-min-resolved-ts", - "refId": "C", - "step": 10 + "intervalFactor": 1, + "legendFormat": "{{instance}}-pending", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Min resolved ts", + "title": "Initial scan tasks status", "tooltip": { - "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -8633,11 +11062,11 @@ }, "yaxes": [ { - "format": "s", + "format": "none", "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -8661,25 +11090,25 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The ID of the min resolved region of each TiKV", + "description": "The memory usage per TiKV instance", "editable": true, "error": false, "fill": 0, "grid": {}, "gridPos": { "h": 7, - "w": 5, - "x": 7, - "y": 21 + "w": 12, + "x": 0, + "y": 35 }, - "id": 153, + "id": 78, "legend": { "alignAsTable": true, "avg": false, "current": true, "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "sideWidth": null, "sort": null, @@ -8687,14 +11116,14 @@ "total": false, "values": true }, - "lines": false, + "lines": true, "linewidth": 1, "links": [], "nullPointMode": "null", "paceLength": 10, "percentage": false, - "pointradius": 1, - "points": true, + "pointradius": 5, + "points": false, "renderer": "flot", "seriesOverrides": [], "spaceLength": 10, @@ -8702,20 +11131,29 @@ "steppedLine": false, "targets": [ { - "expr": "avg(tikv_cdc_min_resolved_ts_region{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "expr": "avg(tikv_cdc_captured_region_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", "format": "time_series", "hide": false, "intervalFactor": 2, - "legendFormat": "{{instance}}-min-resolved-region", + "legendFormat": "tikv-{{instance}}-total", "refId": "A", "step": 10 + }, + { + "expr": "sum(tikv_cdc_region_resolve_status{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance, status)", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "tikv-{{instance}}-{{status}}", + "refId": "B", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Min resolved Region", + "title": "Captured region count", "tooltip": { "msResolution": false, "shared": true, @@ -8740,19 +11178,33 @@ "show": true }, { - "format": "none", + "format": "short", "label": null, "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { "align": false, "alignLevel": null } - }, + } + ], + "title": "TiKV", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 8 + }, + "id": 294, + "panels": [ { "aliasColors": {}, "bars": false, @@ -8760,28 +11212,31 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "", - "fill": 1, + "description": "The speed of TiKV CDC incremental scan", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, "gridPos": { "h": 7, - "w": 12, + "w": 6, "x": 12, - "y": 21 + "y": 35 }, - "id": 70, + "id": 76, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, + "hideEmpty": true, "hideZero": true, "max": true, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "sideWidth": null, - "sort": null, - "sortDesc": null, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -8800,19 +11255,22 @@ "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99999, sum(rate(tikv_cdc_resolved_ts_gap_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance))", + "expr": "sum(rate(tikv_cdc_scan_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", job=\"tikv\"}[30s])) by (instance)", "format": "time_series", - "intervalFactor": 1, - "legendFormat": "{{instance}}-p9999", - "refId": "A" + "hide": false, + "intervalFactor": 2, + "legendFormat": "tikv-{{instance}}", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Resolved ts lag duration percentile", + "title": "CDC scan speed", "tooltip": { + "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -8827,11 +11285,11 @@ }, "yaxes": [ { - "format": "s", + "format": "bytes", "label": null, - "logBase": 10, + "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { @@ -8849,83 +11307,105 @@ } }, { - "cards": { - "cardPadding": 0, - "cardRound": 0 - }, - "color": { - "cardColor": "#FF9830", - "colorScale": "linear", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", + "aliasColors": {}, + "bars": true, + "dashLength": 10, + "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The time consumed to CDC incremental scan", + "decimals": 1, + "description": "The total bytes of TiKV CDC incremental scan", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, "gridPos": { "h": 7, - "w": 12, - "x": 0, - "y": 28 + "w": 6, + "x": 18, + "y": 35 }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 68, + "id": 139, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": true, + "hideZero": true, "max": true, "min": false, - "rightSide": true, + "rightSide": false, "show": true, + "sideWidth": null, "sort": "current", "sortDesc": true, "total": false, "values": true }, + "lines": false, + "linewidth": 1, "links": [], - "maxPerRow": 3, - "repeat": null, - "repeatDirection": "h", - "reverseYBuckets": false, + "nullPointMode": "null", + "paceLength": 10, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, "targets": [ - { - "expr": "sum(rate(tikv_cdc_scan_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, + { + "expr": "sum(tikv_cdc_scan_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", job=\"tikv\"}) by (instance)", + "format": "time_series", + "hide": false, "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" + "legendFormat": "tikv-{{instance}}", + "refId": "A", + "step": 10 } ], - "title": "Initial scan duration", + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "CDC total scan bytes", "tooltip": { - "show": true, - "showHistogram": true - }, - "tooltipDecimals": 1, - "type": "heatmap", - "xAxis": { - "show": true + "msResolution": false, + "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": "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": {}, @@ -8935,23 +11415,23 @@ "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, "description": "", - "fill": 1, + "fill": 4, "gridPos": { "h": 7, - "w": 6, - "x": 12, - "y": 28 + "w": 12, + "x": 0, + "y": 42 }, - "id": 72, + "id": 143, "legend": { "alignAsTable": true, "avg": false, "current": true, "hideEmpty": false, "hideZero": true, - "max": true, - "min": false, - "rightSide": false, + "max": false, + "min": true, + "rightSide": true, "show": true, "sideWidth": null, "sort": "current", @@ -8965,27 +11445,59 @@ "nullPointMode": "null", "paceLength": 10, "percentage": false, - "pointradius": 5, - "points": false, + "pointradius": 1, + "points": true, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/(access|miss).*/", + "fill": 0, + "points": false, + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.9999, sum(rate(tikv_cdc_scan_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance))", + "expr": "(sum(rate(tikv_cdc_old_value_cache_access{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance) - sum(rate(tikv_cdc_old_value_cache_miss{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance)) / sum(rate(tikv_cdc_old_value_cache_access{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance)", "format": "time_series", + "hide": false, "intervalFactor": 1, - "legendFormat": "{{instance}}-p9999", + "legendFormat": "hit-rate-{{instance}}", "refId": "A" + }, + { + "expr": "-sum(rate(tikv_cdc_old_value_cache_access{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance)", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "access-{{instance}}", + "refId": "B" + }, + { + "expr": "-sum(rate(tikv_cdc_old_value_cache_miss{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "miss-{{instance}}", + "refId": "C" + }, + { + "expr": "-sum(rate(tikv_cdc_old_value_cache_miss_none{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "miss-none-{{instance}}", + "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Initial scan duration percentile", + "title": "Old value cache hit", "tooltip": { "shared": true, "sort": 0, @@ -9001,20 +11513,20 @@ }, "yaxes": [ { - "format": "s", + "format": "percentunit", "label": null, "logBase": 1, - "max": null, - "min": null, + "max": "1", + "min": "0", "show": true }, { - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { @@ -9029,15 +11541,15 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The number of incremental scan task in different status.", + "description": "The total number of cache entries in the old value cache.", "fill": 1, "gridPos": { "h": 7, - "w": 6, - "x": 18, - "y": 28 + "w": 12, + "x": 12, + "y": 42 }, - "id": 140, + "id": 145, "legend": { "alignAsTable": true, "avg": false, @@ -9046,7 +11558,7 @@ "hideZero": true, "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "sideWidth": null, "sort": "current", @@ -9060,12 +11572,12 @@ "nullPointMode": "null", "paceLength": 10, "percentage": false, - "pointradius": 5, + "pointradius": 1, "points": false, "renderer": "flot", "seriesOverrides": [ { - "alias": "/.*ongoing/", + "alias": "/.*len/", "yaxis": 2 } ], @@ -9074,27 +11586,35 @@ "steppedLine": false, "targets": [ { - "expr": "sum(tikv_cdc_scan_tasks{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", type=\"ongoing\"}) by (type, instance)", + "expr": "sum(tikv_cdc_old_value_cache_length{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", "format": "time_series", "hide": false, "intervalFactor": 1, - "legendFormat": "{{instance}}-{{type}}", + "legendFormat": "{{instance}}-len", "refId": "A" }, { - "expr": "sum(tikv_cdc_scan_tasks{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", type=\"total\"}) by (instance) - sum(tikv_cdc_scan_tasks{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", type=~\"abort|finish\"}) by (instance)", + "expr": "sum(tikv_cdc_old_value_cache_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance) / sum(tikv_cdc_old_value_cache_length{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", "format": "time_series", "hide": false, "intervalFactor": 1, - "legendFormat": "{{instance}}-pending", + "legendFormat": "{{instance}}-avg entry bytes", "refId": "B" + }, + { + "expr": "sum(tikv_cdc_old_value_cache_memory_quota{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "{{instance}}-quota", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Initial scan tasks status", + "title": "Old value cache size", "tooltip": { "shared": true, "sort": 0, @@ -9110,19 +11630,20 @@ }, "yaxes": [ { - "format": "none", + "format": "bytes", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { + "decimals": null, "format": "none", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true } ], @@ -9138,7 +11659,7 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The memory usage per TiKV instance", + "description": "", "editable": true, "error": false, "fill": 0, @@ -9147,20 +11668,22 @@ "h": 7, "w": 12, "x": 0, - "y": 35 + "y": 49 }, - "id": 78, + "id": 141, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": true, + "hideZero": true, "max": true, "min": false, "rightSide": true, "show": true, "sideWidth": null, - "sort": null, - "sortDesc": null, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, @@ -9179,29 +11702,20 @@ "steppedLine": false, "targets": [ { - "expr": "avg(tikv_cdc_captured_region_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "expr": "sum(rate(tikv_cdc_old_value_scan_details{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance, cf, tag)", "format": "time_series", "hide": false, "intervalFactor": 2, - "legendFormat": "tikv-{{instance}}-total", + "legendFormat": "{{instance}}-{{cf}}-{{tag}}", "refId": "A", "step": 10 - }, - { - "expr": "sum(tikv_cdc_region_resolve_status{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance, status)", - "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "tikv-{{instance}}-{{status}}", - "refId": "B", - "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Captured region count", + "title": "Old value seek operation", "tooltip": { "msResolution": false, "shared": true, @@ -9218,7 +11732,7 @@ }, "yaxes": [ { - "format": "none", + "format": "ops", "label": null, "logBase": 1, "max": null, @@ -9240,130 +11754,104 @@ } }, { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The speed of TiKV CDC incremental scan", - "editable": true, - "error": false, - "fill": 0, - "grid": {}, + "description": "The time consumed to get an old value (both from cache and from disk)", "gridPos": { "h": 7, "w": 6, "x": 12, - "y": 35 + "y": 49 }, - "id": 76, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 146, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": true, - "hideZero": true, "max": true, "min": false, - "rightSide": false, + "rightSide": true, "show": true, - "sideWidth": null, "sort": "current", "sortDesc": true, "total": false, "values": true }, - "lines": true, - "linewidth": 1, "links": [], - "nullPointMode": "null", - "paceLength": 10, - "percentage": false, - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, + "maxPerRow": 3, + "repeatDirection": "h", + "reverseYBuckets": false, "targets": [ { - "expr": "sum(rate(tikv_cdc_scan_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", job=\"tikv\"}[30s])) by (instance)", - "format": "time_series", - "hide": false, + "expr": "sum(rate(tikv_cdc_old_value_duration_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le)", + "format": "heatmap", + "instant": false, "intervalFactor": 2, - "legendFormat": "tikv-{{instance}}", - "refId": "A", - "step": 10 + "legendFormat": "{{le}}", + "refId": "A" } ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "CDC scan speed", + "title": "Old value seek duration", "tooltip": { - "msResolution": false, - "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": "bytes", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "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}", "decimals": 1, - "description": "The total bytes of TiKV CDC incremental scan", - "editable": true, - "error": false, - "fill": 0, - "grid": {}, + "description": "", + "fill": 1, "gridPos": { "h": 7, "w": 6, "x": 18, - "y": 35 + "y": 49 }, - "id": 139, + "id": 142, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": true, + "hideEmpty": false, "hideZero": true, "max": true, "min": false, @@ -9375,7 +11863,7 @@ "total": false, "values": true }, - "lines": false, + "lines": true, "linewidth": 1, "links": [], "nullPointMode": "null", @@ -9390,22 +11878,33 @@ "steppedLine": false, "targets": [ { - "expr": "sum(tikv_cdc_scan_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", job=\"tikv\"}) by (instance)", + "expr": "histogram_quantile(0.99, sum(rate(tikv_cdc_old_value_duration_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance, tag))", "format": "time_series", - "hide": false, - "intervalFactor": 2, - "legendFormat": "tikv-{{instance}}", - "refId": "A", - "step": 10 + "intervalFactor": 1, + "legendFormat": "{{instance}}-99%-{{tag}}", + "refId": "A" + }, + { + "expr": "histogram_quantile(0.95, sum(rate(tikv_cdc_old_value_duration_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance, tag))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-95%-{{tag}}", + "refId": "B" + }, + { + "expr": "sum(rate(tikv_cdc_old_value_duration_sum{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance, tag) / sum(rate(tikv_cdc_old_value_duration_count{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance, tag)", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-avg-{{tag}}", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "CDC total scan bytes", + "title": "Old value seek duration", "tooltip": { - "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -9420,7 +11919,7 @@ }, "yaxes": [ { - "format": "bytes", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -9433,106 +11932,179 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { "align": false, "alignLevel": null } - }, + } + ], + "title": "TiKV", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 9 + }, + "id": 155, + "panels": [ { "aliasColors": {}, - "bars": false, + "bars": true, + "cacheTimeout": null, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "", - "fill": 4, + "description": "TiCDC process rss memory usage. TiCDC heap memory size in use ", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 42 + "y": 6 }, - "id": 143, + "id": 157, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, - "current": true, - "hideEmpty": false, + "current": false, + "hideEmpty": true, "hideZero": true, "max": false, - "min": true, - "rightSide": true, + "min": false, + "rightSide": false, "show": true, "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, - "values": true + "values": false }, - "lines": true, + "lines": false, "linewidth": 1, "links": [], + "maxPerRow": 3, "nullPointMode": "null", - "paceLength": 10, "percentage": false, - "pointradius": 1, - "points": true, + "pluginVersion": "6.1.6", + "pointradius": 5, + "points": false, "renderer": "flot", + "repeat": null, + "repeatDirection": "h", + "scopedVars": { + "instance": { + "selected": false, + "text": "172.16.5.33:47912", + "value": "172.16.5.33:47912" + }, + "runtime_instance": { + "selected": false, + "text": "172.16.6.47:8400", + "value": "172.16.6.47:8400" + } + }, "seriesOverrides": [ { - "alias": "/(access|miss).*/", - "fill": 0, - "points": false, - "yaxis": 2 + "alias": "alloc-from-os", + "fill": 3, + "lines": true, + "stack": false + }, + { + "alias": "gc-threshold", + "bars": false, + "color": "#C4162A", + "lines": true, + "linewidth": 2, + "stack": false + }, + { + "alias": "gc", + "bars": false, + "color": "#C4162A", + "hideTooltip": true, + "legend": false, + "pointradius": 3, + "points": true, + "stack": false } ], "spaceLength": 10, - "stack": false, + "stack": true, "steppedLine": false, "targets": [ { - "expr": "(sum(rate(tikv_cdc_old_value_cache_access{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance) - sum(rate(tikv_cdc_old_value_cache_miss{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance)) / sum(rate(tikv_cdc_old_value_cache_access{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance)", + "expr": "process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", "format": "time_series", "hide": false, "intervalFactor": 1, - "legendFormat": "hit-rate-{{instance}}", + "legendFormat": "alloc-from-os", "refId": "A" }, { - "expr": "-sum(rate(tikv_cdc_old_value_cache_access{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance)", + "expr": "go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / (1 + ticdc_server_go_gc{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / 100)", "format": "time_series", - "hide": true, + "hide": false, "intervalFactor": 1, - "legendFormat": "access-{{instance}}", - "refId": "B" + "legendFormat": "estimate-inuse", + "refId": "H" }, { - "expr": "-sum(rate(tikv_cdc_old_value_cache_miss{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance)", + "expr": "go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / (1 + ticdc_server_go_gc{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / 100)", "format": "time_series", "hide": false, "intervalFactor": 1, - "legendFormat": "miss-{{instance}}", + "legendFormat": "estimate-garbage", "refId": "C" }, { - "expr": "-sum(rate(tikv_cdc_old_value_cache_miss_none{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance)", + "expr": "go_memstats_heap_idle_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_heap_released_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_heap_inuse_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", "format": "time_series", "hide": false, "intervalFactor": 1, - "legendFormat": "miss-none-{{instance}}", + "legendFormat": "reserved-by-go", + "refId": "B" + }, + { + "expr": "go_memstats_stack_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_mspan_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_mcache_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_buck_hash_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_gc_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_other_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "used-by-go", "refId": "D" + }, + { + "expr": "go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "gc-threshold", + "refId": "E" + }, + { + "expr": "(clamp_max(idelta(go_memstats_last_gc_time_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[1m]), 1) * go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}) > 0", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "gc", + "refId": "F" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Old value cache hit", + "title": "Memory Usage", "tooltip": { + "msResolution": true, "shared": true, "sort": 0, "value_type": "individual" @@ -9547,20 +12119,20 @@ }, "yaxes": [ { - "format": "percentunit", - "label": null, + "format": "bytes", + "label": "", "logBase": 1, - "max": "1", + "max": null, "min": "0", "show": true }, { - "format": "none", - "label": null, + "format": "short", + "label": "", "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -9571,85 +12143,81 @@ { "aliasColors": {}, "bars": false, + "cacheTimeout": null, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The total number of cache entries in the old value cache.", - "fill": 1, + "description": "Count of live objects.", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 12, - "y": 42 + "y": 6 }, - "id": 145, + "id": 158, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, - "current": true, - "hideEmpty": false, + "current": false, + "hideEmpty": true, "hideZero": true, - "max": true, + "max": false, "min": false, - "rightSide": true, + "rightSide": false, "show": true, "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, - "values": true + "values": false }, "lines": true, "linewidth": 1, "links": [], + "maxPerRow": 3, "nullPointMode": "null", - "paceLength": 10, "percentage": false, - "pointradius": 1, + "pluginVersion": "6.1.6", + "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/.*len/", - "yaxis": 2 + "repeat": null, + "repeatDirection": "h", + "scopedVars": { + "instance": { + "selected": false, + "text": "172.16.5.33:47912", + "value": "172.16.5.33:47912" + }, + "runtime_instance": { + "selected": false, + "text": "172.16.6.47:8400", + "value": "172.16.6.47:8400" } - ], + }, + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(tikv_cdc_old_value_cache_length{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "expr": "go_memstats_heap_objects{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", "format": "time_series", "hide": false, "intervalFactor": 1, - "legendFormat": "{{instance}}-len", + "legendFormat": "objects", "refId": "A" - }, - { - "expr": "sum(tikv_cdc_old_value_cache_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance) / sum(tikv_cdc_old_value_cache_length{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "{{instance}}-avg entry bytes", - "refId": "B" - }, - { - "expr": "sum(tikv_cdc_old_value_cache_memory_quota{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "{{instance}}-quota", - "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Old value cache size", + "title": "Estimated Live Objects", "tooltip": { + "msResolution": true, "shared": true, "sort": 0, "value_type": "individual" @@ -9664,21 +12232,20 @@ }, "yaxes": [ { - "format": "bytes", - "label": null, + "format": "short", + "label": "", "logBase": 1, "max": null, "min": "0", "show": true }, { - "decimals": null, - "format": "none", - "label": null, + "format": "short", + "label": "", "logBase": 1, "max": null, - "min": "0", - "show": true + "min": null, + "show": false } ], "yaxis": { @@ -9689,71 +12256,97 @@ { "aliasColors": {}, "bars": false, + "cacheTimeout": null, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "", + "description": "TiCDC process Go garbage collection STW pause duration", "editable": true, "error": false, - "fill": 0, + "fill": 1, "grid": {}, "gridPos": { "h": 7, "w": 12, "x": 0, - "y": 49 + "y": 13 }, - "id": 141, + "id": 160, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": true, - "hideEmpty": true, - "hideZero": true, - "max": true, + "max": false, "min": false, - "rightSide": true, + "rightSide": false, "show": true, - "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", - "paceLength": 10, + "nullPointMode": "null as zero", "percentage": false, + "pluginVersion": "6.1.6", "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "scopedVars": { + "instance": { + "selected": false, + "text": "172.16.5.33:47912", + "value": "172.16.5.33:47912" + }, + "runtime_instance": { + "selected": false, + "text": "172.16.6.47:8400", + "value": "172.16.6.47:8400" + } + }, + "seriesOverrides": [ + {} + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "sum(rate(tikv_cdc_old_value_scan_details{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (instance, cf, tag)", + "expr": "go_gc_duration_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\", quantile=\"0\"}", "format": "time_series", "hide": false, - "intervalFactor": 2, - "legendFormat": "{{instance}}-{{cf}}-{{tag}}", + "instant": false, + "intervalFactor": 1, + "legendFormat": "min", "refId": "A", - "step": 10 + "step": 40 + }, + { + "expr": "go_gc_duration_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\", quantile!~\"0|1\"}", + "format": "time_series", + "instant": false, + "intervalFactor": 1, + "legendFormat": "{{quantile}}", + "refId": "B" + }, + { + "expr": "go_gc_duration_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\", quantile=\"1\"}", + "format": "time_series", + "instant": false, + "intervalFactor": 1, + "legendFormat": "max", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Old value seek operation", + "title": "GC STW Duration (last 256 GC cycles)", "tooltip": { "msResolution": false, "shared": true, - "sort": 0, + "sort": 1, "value_type": "individual" }, "type": "graph", @@ -9766,7 +12359,7 @@ }, "yaxes": [ { - "format": "ops", + "format": "s", "label": null, "logBase": 1, "max": null, @@ -9779,7 +12372,7 @@ "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -9787,113 +12380,29 @@ "alignLevel": null } }, - { - "cards": { - "cardPadding": 0, - "cardRound": 0 - }, - "color": { - "cardColor": "#FF9830", - "colorScale": "linear", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", - "datasource": "${DS_TEST-CLUSTER}", - "description": "The time consumed to get an old value (both from cache and from disk)", - "gridPos": { - "h": 7, - "w": 6, - "x": 12, - "y": 49 - }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 146, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": true, - "min": false, - "rightSide": true, - "show": true, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "links": [], - "maxPerRow": 3, - "repeatDirection": "h", - "reverseYBuckets": false, - "targets": [ - { - "expr": "sum(rate(tikv_cdc_old_value_duration_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le)", - "format": "heatmap", - "instant": false, - "intervalFactor": 2, - "legendFormat": "{{le}}", - "refId": "A" - } - ], - "title": "Old value seek duration", - "tooltip": { - "show": true, - "showHistogram": true - }, - "tooltipDecimals": 1, - "type": "heatmap", - "xAxis": { - "show": true - }, - "xBucketNumber": null, - "xBucketSize": null, - "yAxis": { - "decimals": 1, - "format": "s", - "logBase": 1, - "max": null, - "min": null, - "show": true, - "splitFactor": null - }, - "yBucketBound": "upper", - "yBucketNumber": null, - "yBucketSize": null - }, { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "", + "description": "The throughput of Go's memory allocator.", "fill": 1, "gridPos": { "h": 7, - "w": 6, - "x": 18, - "y": 49 + "w": 12, + "x": 12, + "y": 13 }, - "id": 142, + "id": 161, "legend": { - "alignAsTable": true, + "alignAsTable": false, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": true, "max": true, "min": false, "rightSide": false, "show": true, - "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, @@ -9901,43 +12410,75 @@ "linewidth": 1, "links": [], "nullPointMode": "null", - "paceLength": 10, "percentage": false, "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "scopedVars": { + "instance": { + "selected": false, + "text": "172.16.5.33:47912", + "value": "172.16.5.33:47912" + }, + "runtime_instance": { + "selected": false, + "text": "172.16.6.47:8400", + "value": "172.16.6.47:8400" + } + }, + "seriesOverrides": [ + { + "alias": "sweep", + "transform": "negative-Y" + }, + { + "alias": "alloc-ops", + "yaxis": 2 + }, + { + "alias": "swepp-ops", + "transform": "negative-Y", + "yaxis": 2 + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tikv_cdc_old_value_duration_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance, tag))", + "expr": "irate(go_memstats_alloc_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[30s])", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}-99%-{{tag}}", + "legendFormat": "alloc", "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))", + "expr": "irate((go_memstats_alloc_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"})[30s:])", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}-95%-{{tag}}", + "legendFormat": "sweep", "refId": "B" }, { - "expr": "sum(rate(tikv_cdc_old_value_duration_sum{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance, tag) / sum(rate(tikv_cdc_old_value_duration_count{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance, tag)", + "expr": "irate(go_memstats_mallocs_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[30s])", "format": "time_series", "intervalFactor": 1, - "legendFormat": "{{instance}}-avg-{{tag}}", + "legendFormat": "alloc-ops", "refId": "C" + }, + { + "expr": "irate(go_memstats_frees_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[30s])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "swepp-ops", + "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Old value seek duration", + "title": "Allocator Throughput", "tooltip": { "shared": true, "sort": 0, @@ -9953,7 +12494,7 @@ }, "yaxes": [ { - "format": "s", + "format": "Bps", "label": null, "logBase": 1, "max": null, @@ -9961,21 +12502,29 @@ "show": true }, { - "format": "short", + "format": "ops", "label": null, "logBase": 1, "max": null, "min": null, - "show": false + "show": true } ], "yaxis": { - "align": false, + "align": true, "alignLevel": null } } ], - "title": "TiKV", + "repeat": "runtime_instance", + "scopedVars": { + "runtime_instance": { + "selected": false, + "text": "172.16.6.47:8400", + "value": "172.16.6.47:8400" + } + }, + "title": "Runtime $runtime_instance", "type": "row" }, { @@ -9984,9 +12533,9 @@ "h": 1, "w": 24, "x": 0, - "y": 7 + "y": 10 }, - "id": 155, + "id": 411, "panels": [ { "aliasColors": {}, @@ -10006,7 +12555,7 @@ "x": 0, "y": 6 }, - "id": 157, + "id": 412, "legend": { "alignAsTable": false, "avg": false, @@ -10033,6 +12582,9 @@ "renderer": "flot", "repeat": null, "repeatDirection": "h", + "repeatIteration": 1644481309661, + "repeatPanelId": 157, + "repeatedByRow": true, "scopedVars": { "instance": { "selected": false, @@ -10041,8 +12593,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.5.37:47912", - "value": "172.16.5.37:47912" + "text": "172.16.6.47:8401", + "value": "172.16.6.47:8401" } }, "seriesOverrides": [ @@ -10192,7 +12744,7 @@ "x": 12, "y": 6 }, - "id": 158, + "id": 413, "legend": { "alignAsTable": false, "avg": false, @@ -10219,6 +12771,9 @@ "renderer": "flot", "repeat": null, "repeatDirection": "h", + "repeatIteration": 1644481309661, + "repeatPanelId": 158, + "repeatedByRow": true, "scopedVars": { "instance": { "selected": false, @@ -10227,8 +12782,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.5.37:47912", - "value": "172.16.5.37:47912" + "text": "172.16.6.47:8401", + "value": "172.16.6.47:8401" } }, "seriesOverrides": [], @@ -10305,7 +12860,7 @@ "x": 0, "y": 13 }, - "id": 160, + "id": 414, "legend": { "alignAsTable": false, "avg": false, @@ -10326,6 +12881,9 @@ "pointradius": 5, "points": false, "renderer": "flot", + "repeatIteration": 1644481309661, + "repeatPanelId": 160, + "repeatedByRow": true, "scopedVars": { "instance": { "selected": false, @@ -10334,8 +12892,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.5.37:47912", - "value": "172.16.5.37:47912" + "text": "172.16.6.47:8401", + "value": "172.16.6.47:8401" } }, "seriesOverrides": [ @@ -10428,7 +12986,7 @@ "x": 12, "y": 13 }, - "id": 161, + "id": 415, "legend": { "alignAsTable": false, "avg": false, @@ -10448,6 +13006,9 @@ "pointradius": 5, "points": false, "renderer": "flot", + "repeatIteration": 1644481309661, + "repeatPanelId": 161, + "repeatedByRow": true, "scopedVars": { "instance": { "selected": false, @@ -10456,8 +13017,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.5.37:47912", - "value": "172.16.5.37:47912" + "text": "172.16.6.47:8401", + "value": "172.16.6.47:8401" } }, "seriesOverrides": [ @@ -10550,7 +13111,16 @@ } } ], - "repeat": "runtime_instance", + "repeat": null, + "repeatIteration": 1644481309661, + "repeatPanelId": 155, + "scopedVars": { + "runtime_instance": { + "selected": false, + "text": "172.16.6.47:8401", + "value": "172.16.6.47:8401" + } + }, "title": "Runtime $runtime_instance", "type": "row" }, @@ -10560,7 +13130,7 @@ "h": 1, "w": 24, "x": 0, - "y": 8 + "y": 11 }, "id": 187, "panels": [ @@ -10924,165 +13494,222 @@ }, { "collapsed": true, - "datasource": null, "gridPos": { "h": 1, "w": 24, "x": 0, - "y": 261 + "y": 12 }, - "id": 187, + "id": 402, "panels": [], - "title": "Redo", + "title": "Kafka Sink", "type": "row" }, { - "cards": { - "cardPadding": 0, - "cardRound": 0 - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The latency distributions of fsync called by redo writer", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, + "description": "The number of bytes sent per partition per request for all topics", + "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 262 + "y": 13 }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 172, + "hiddenSeries": false, + "id": 410, "legend": { - "show": true + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] }, - "pluginVersion": "6.1.6", - "reverseYBuckets": false, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "max(rate(ticdc_redo_fsync_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", + "expr": "max(ticdc_sink_kafka_producer_batch_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed)", + "format": "time_series", "interval": "", - "intervalFactor": 2, - "legendFormat": "{{le}}", - "queryType": "randomWalk", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}", "refId": "A" + }, + { + "expr": "sum(ticdc_sink_kafka_producer_batch_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture,changefeed)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}", + "refId": "B" } ], + "thresholds": [], "timeFrom": null, + "timeRegions": [], "timeShift": null, - "title": "Redo fsync duration", + "title": "batch size", "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": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { - "cards": { - "cardPadding": 0, - "cardRound": 0 - }, - "color": { - "cardColor": "#b4ff00", - "colorScale": "sqrt", - "colorScheme": "interpolateSpectral", - "exponent": 0.5, - "min": 0, - "mode": "spectrum" - }, - "dataFormat": "tsbuckets", + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The latency distributions of flushall called by redo writer", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, + "description": "The number of records sent per request for all topics", + "fill": 1, + "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 262 + "y": 13 }, - "heatmap": {}, - "hideZeroBuckets": true, - "highlightCards": true, - "id": 180, + "hiddenSeries": false, + "id": 408, "legend": { - "show": true + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] }, - "pluginVersion": "6.1.6", - "reverseYBuckets": false, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "max(rate(ticdc_redo_flushall_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le)", - "format": "heatmap", + "expr": "max(ticdc_sink_kafka_producer_record_send_rate{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed)", + "format": "time_series", "interval": "", - "intervalFactor": 2, - "legendFormat": "{{le}}", - "queryType": "randomWalk", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}", "refId": "A" + }, + { + "expr": "sum(ticdc_sink_kafka_producer_record_send_rate{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture,changefeed)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}", + "refId": "B" } ], + "thresholds": [], "timeFrom": null, + "timeRegions": [], "timeShift": null, - "title": "Redo flushall duration", + "title": "records send rate", "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": "wps", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } }, { "aliasColors": {}, @@ -11090,27 +13717,23 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The total count of rows that are processed by redo writer", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, + "description": "The number of records sent per request for all topics", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 0, - "y": 270 + "y": 20 }, "hiddenSeries": false, - "id": 170, + "id": 409, "legend": { "alignAsTable": true, "avg": true, "current": true, "max": true, - "min": false, + "min": true, "rightSide": true, "show": true, "total": false, @@ -11118,12 +13741,13 @@ }, "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", @@ -11133,19 +13757,19 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum(rate(ticdc_redo_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", + "expr": "max(ticdc_sink_kafka_producer_records_per_request{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed)", + "format": "time_series", "interval": "", - "legendFormat": "{{capture}}", - "queryType": "randomWalk", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}", "refId": "A" }, { - "exemplar": true, - "expr": "sum (rate(ticdc_redo_total_rows_count{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}[1m])) by (changefeed)", - "hide": false, + "expr": "sum(ticdc_sink_kafka_producer_records_per_request{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture,changefeed)", + "format": "time_series", "interval": "", - "legendFormat": "total", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}", "refId": "B" } ], @@ -11153,7 +13777,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Redo Write rows/s ", + "title": "records per request", "tooltip": { "shared": true, "sort": 0, @@ -11169,16 +13793,14 @@ }, "yaxes": [ { - "$$hashKey": "object:640", - "format": "short", + "format": "none", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { - "$$hashKey": "object:641", "format": "short", "label": null, "logBase": 1, @@ -11198,38 +13820,37 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "Total number of bytes redo log written", - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, + "description": "the compression ratio times 100 of record batches for all topics", "fill": 1, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 7, "w": 12, "x": 12, - "y": 270 + "y": 20 }, "hiddenSeries": false, - "id": 173, + "id": 416, "legend": { - "avg": false, - "current": false, - "max": false, - "min": false, + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "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", @@ -11239,19 +13860,27 @@ "steppedLine": false, "targets": [ { - "exemplar": true, - "expr": "sum(rate(ticdc_redo_write_bytes_total{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (capture)", + "expr": "max(ticdc_sink_kafka_producer_compression_ratio{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed)", + "format": "time_series", "interval": "", - "legendFormat": "{{capture}}", - "queryType": "randomWalk", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}", "refId": "A" + }, + { + "expr": "sum(ticdc_sink_kafka_producer_compression_ratio{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture,changefeed)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}", + "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Redo Write bytes/s ", + "title": "compression ratio", "tooltip": { "shared": true, "sort": 0, @@ -11267,22 +13896,20 @@ }, "yaxes": [ { - "$$hashKey": "object:587", - "format": "bytes", + "format": "percent", "label": null, "logBase": 1, "max": null, - "min": null, + "min": "0", "show": true }, { - "$$hashKey": "object:588", "format": "short", "label": null, "logBase": 1, "max": null, "min": null, - "show": true + "show": false } ], "yaxis": { @@ -11299,7 +13926,11 @@ "list": [ { "allValue": null, - "current": {}, + "current": { + "isNone": true, + "text": "None", + "value": "" + }, "datasource": "${DS_TEST-CLUSTER}", "definition": "", "hide": 0, @@ -11321,7 +13952,10 @@ }, { "allValue": ".*", - "current": {}, + "current": { + "text": "All", + "value": "$__all" + }, "datasource": "${DS_TEST-CLUSTER}", "definition": "label_values(ticdc_processor_resolved_ts{tidb_cluster=\"$tidb_cluster\"}, changefeed)", "hide": 0, @@ -11343,7 +13977,10 @@ }, { "allValue": ".*", - "current": {}, + "current": { + "text": "All", + "value": "$__all" + }, "datasource": "${DS_TEST-CLUSTER}", "definition": "label_values(process_start_time_seconds{tidb_cluster=\"$tidb_cluster\", job=\"ticdc\"}, instance)", "hide": 0, @@ -11365,7 +14002,10 @@ }, { "allValue": ".*", - "current": {}, + "current": { + "text": "All", + "value": "$__all" + }, "datasource": "${DS_TEST-CLUSTER}", "definition": "label_values(tikv_engine_size_bytes{tidb_cluster=\"$tidb_cluster\"}, instance)", "hide": 0, @@ -11440,7 +14080,10 @@ }, { "allValue": "", - "current": {}, + "current": { + "text": "All", + "value": "$__all" + }, "datasource": "${DS_TEST-CLUSTER}", "definition": "label_values(process_start_time_seconds{tidb_cluster=\"$tidb_cluster\", job=\"ticdc\"}, instance)", "hide": 0, @@ -11494,5 +14137,5 @@ "timezone": "browser", "title": "Test-Cluster-TiCDC", "uid": "YiGL8hBZ1", - "version": 32 -} \ No newline at end of file + "version": 33 +}