From f87b6576d70acf278e5d32df009b66ebee5f0cd4 Mon Sep 17 00:00:00 2001 From: Ling Jin <7138436+3AceShowHand@users.noreply.github.com> Date: Tue, 15 Feb 2022 13:35:39 +0800 Subject: [PATCH] This is an automated cherry-pick of #4517 Signed-off-by: ti-chi-bot --- cdc/sink/producer/kafka/kafka.go | 86 + cdc/sink/producer/kafka/metrics.go | 311 ++ metrics/grafana/ticdc.json | 4629 +++++++++++++++++++++++++++- 3 files changed, 4915 insertions(+), 111 deletions(-) create mode 100644 cdc/sink/producer/kafka/metrics.go diff --git a/cdc/sink/producer/kafka/kafka.go b/cdc/sink/producer/kafka/kafka.go index 308b39b6133..88d63113c5f 100644 --- a/cdc/sink/producer/kafka/kafka.go +++ b/cdc/sink/producer/kafka/kafka.go @@ -174,10 +174,20 @@ type kafkaSaramaProducer struct { // clientLock is used to protect concurrent access of asyncClient and syncClient. // Since we don't close these two clients (which have an input chan) from the // sender routine, data race or send on closed chan could happen. +<<<<<<< HEAD clientLock sync.RWMutex asyncClient sarama.AsyncProducer syncClient sarama.SyncProducer // producersReleased records whether asyncClient and syncClient have been closed properly +======= + clientLock sync.RWMutex + admin kafka.ClusterAdminClient + client sarama.Client + asyncProducer sarama.AsyncProducer + syncProducer sarama.SyncProducer + + // producersReleased records whether asyncProducer and syncProducer have been closed properly +>>>>>>> 28fe713de (cdc/sink: kafka sink integrate broker level metrics (#4517)) producersReleased bool topic string partitionNum int32 @@ -357,6 +367,21 @@ func (k *kafkaSaramaProducer) Close() error { } else { log.Info("sync client closed", zap.Duration("duration", time.Since(start))) } +<<<<<<< HEAD +======= + + start = time.Now() + if err := k.admin.Close(); err != nil { + log.Warn("close kafka cluster admin with error", zap.Error(err), + zap.Duration("duration", time.Since(start)), + zap.String("changefeed", k.id), zap.Any("role", k.role)) + } else { + log.Info("kafka cluster admin closed", zap.Duration("duration", time.Since(start)), + zap.String("changefeed", k.id), zap.Any("role", k.role)) + } + + k.metricsMonitor.Cleanup() +>>>>>>> 28fe713de (cdc/sink: kafka sink integrate broker level metrics (#4517)) return nil } @@ -402,9 +427,70 @@ func topicPreProcess(topic string, config *Config, saramaConfig *sarama.Config) if err != nil { return cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) } +<<<<<<< HEAD defer func() { if err := admin.Close(); err != nil { log.Warn("close kafka cluster admin failed", zap.Error(err)) +======= + + if err := validateAndCreateTopic(admin, topic, config, cfg, opts); err != nil { + return nil, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) + } + + client, err := sarama.NewClient(config.BrokerEndpoints, cfg) + if err != nil { + return nil, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) + } + + asyncProducer, err := sarama.NewAsyncProducerFromClient(client) + if err != nil { + return nil, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) + } + + syncProducer, err := sarama.NewSyncProducerFromClient(client) + if err != nil { + return nil, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) + } + + notifier := new(notify.Notifier) + flushedReceiver, err := notifier.NewReceiver(50 * time.Millisecond) + if err != nil { + return nil, err + } + k := &kafkaSaramaProducer{ + admin: admin, + client: client, + asyncProducer: asyncProducer, + syncProducer: syncProducer, + topic: topic, + partitionNum: config.PartitionNum, + partitionOffset: make([]struct { + flushed uint64 + sent uint64 + }, config.PartitionNum), + flushedNotifier: notifier, + flushedReceiver: flushedReceiver, + closeCh: make(chan struct{}), + failpointCh: make(chan error, 1), + closing: kafkaProducerRunning, + + id: changefeedID, + role: role, + + metricsMonitor: NewSaramaMetricsMonitor(cfg.MetricRegistry, + util.CaptureAddrFromCtx(ctx), changefeedID, admin), + } + go func() { + if err := k.run(ctx); err != nil && errors.Cause(err) != context.Canceled { + select { + case <-ctx.Done(): + return + case errCh <- err: + default: + log.Error("error channel is full", zap.Error(err), + zap.String("changefeed", k.id), zap.Any("role", role)) + } +>>>>>>> 28fe713de (cdc/sink: kafka sink integrate broker level metrics (#4517)) } }() diff --git a/cdc/sink/producer/kafka/metrics.go b/cdc/sink/producer/kafka/metrics.go new file mode 100644 index 00000000000..844850938cc --- /dev/null +++ b/cdc/sink/producer/kafka/metrics.go @@ -0,0 +1,311 @@ +// 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 ( + "strconv" + + "github.com/pingcap/log" + "github.com/pingcap/tiflow/pkg/kafka" + "github.com/prometheus/client_golang/prometheus" + "github.com/rcrowley/go-metrics" + "go.uber.org/zap" +) + +var ( + // Histogram update by the `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"}) + + // meter mark by total records count + 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 + // histogram update by all records count. + 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"}) + + // histogram update by `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"}) + + // metrics for outgoing events + // meter mark for each request's size in bytes + outgoingByteRateGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_outgoing_byte_rate", + Help: "Bytes/second written off all brokers", + }, []string{"capture", "changefeed", "broker"}) + + // meter mark by 1 for each request + requestRateGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_request_rate", + Help: "Requests/second sent to all brokers", + }, []string{"capture", "changefeed", "broker"}) + + // meter mark for each request's size in bytes + requestSizeGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_request_size", + Help: "the request size in bytes for all brokers", + }, []string{"capture", "changefeed", "broker"}) + + // histogram update for each received response, requestLatency := time.Since(response.requestTime) + requestLatencyInMsGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_request_latency", + Help: "the request latency in ms for all brokers", + }, []string{"capture", "changefeed", "broker"}) + + // counter inc by 1 once a request send, dec by 1 for a response received. + requestsInFlightGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_in_flight_requests", + Help: "the current number of in-flight requests awaiting a response for all brokers", + }, []string{"capture", "changefeed", "broker"}) + + // metrics for incoming events + // meter mark for each received response's size in bytes + incomingByteRateGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_incoming_byte_rate", + Help: "Bytes/second read off all brokers", + }, []string{"capture", "changefeed", "broker"}) + + // meter mark by 1 once a response received. + responseRateGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_response_rate", + Help: "Responses/second received from all brokers", + }, []string{"capture", "changefeed", "broker"}) + + // meter mark by each read response size + responseSizeGauge = prometheus.NewGaugeVec( + prometheus.GaugeOpts{ + Namespace: "ticdc", + Subsystem: "sink", + Name: "kafka_producer_response_size", + Help: "the response size in bytes for all brokers", + }, []string{"capture", "changefeed", "broker"}) +) + +// InitMetrics registers all metrics in this file +func InitMetrics(registry *prometheus.Registry) { + registry.MustRegister(batchSizeGauge) + registry.MustRegister(recordSendRateGauge) + registry.MustRegister(recordPerRequestGauge) + registry.MustRegister(compressionRatioGauge) + + registry.MustRegister(incomingByteRateGauge) + registry.MustRegister(outgoingByteRateGauge) + registry.MustRegister(requestSizeGauge) + registry.MustRegister(requestRateGauge) + registry.MustRegister(requestLatencyInMsGauge) + registry.MustRegister(requestsInFlightGauge) + registry.MustRegister(responseSizeGauge) + registry.MustRegister(responseRateGauge) +} + +// sarama metrics names, see https://pkg.go.dev/github.com/Shopify/sarama#pkg-overview +const ( + // metrics at producer level. + batchSizeMetricName = "batch-size" + recordSendRateMetricName = "record-send-rate" + recordPerRequestMetricName = "records-per-request" + compressionRatioMetricName = "compression-ratio" + + // metrics at broker level. + incomingByteRateMetricNamePrefix = "incoming-byte-rate-for-broker-" + outgoingByteRateMetricNamePrefix = "outgoing-byte-rate-for-broker-" + requestRateMetricNamePrefix = "request-rate-for-broker-" + requestSizeMetricNamePrefix = "request-size-for-broker-" + requestLatencyInMsMetricNamePrefix = "request-latency-in-ms-for-broker-" + requestsInFlightMetricNamePrefix = "requests-in-flight-for-broker-" + responseRateMetricNamePrefix = "response-rate-for-broker-" + responseSizeMetricNamePrefix = "response-size-for-broker-" +) + +type saramaMetricsMonitor struct { + captureAddr string + changefeedID string + + registry metrics.Registry + admin kafka.ClusterAdminClient +} + +// CollectMetrics collect all monitored metrics +func (sm *saramaMetricsMonitor) CollectMetrics() { + sm.collectProducerMetrics() + if err := sm.collectBrokerMetrics(); err != nil { + log.Warn("collect broker metrics failed", zap.Error(err)) + } +} + +func (sm *saramaMetricsMonitor) collectProducerMetrics() { + batchSizeMetric := sm.registry.Get(batchSizeMetricName) + if histogram, ok := batchSizeMetric.(metrics.Histogram); ok { + batchSizeGauge.WithLabelValues(sm.captureAddr, sm.changefeedID).Set(histogram.Snapshot().Mean()) + } + + recordSendRateMetric := sm.registry.Get(recordSendRateMetricName) + if meter, ok := recordSendRateMetric.(metrics.Meter); ok { + recordSendRateGauge.WithLabelValues(sm.captureAddr, sm.changefeedID).Set(meter.Snapshot().Rate1()) + } + + recordPerRequestMetric := sm.registry.Get(recordPerRequestMetricName) + if histogram, ok := recordPerRequestMetric.(metrics.Histogram); ok { + recordPerRequestGauge.WithLabelValues(sm.captureAddr, sm.changefeedID).Set(histogram.Snapshot().Mean()) + } + + compressionRatioMetric := sm.registry.Get(compressionRatioMetricName) + if histogram, ok := compressionRatioMetric.(metrics.Histogram); ok { + compressionRatioGauge.WithLabelValues(sm.captureAddr, sm.changefeedID).Set(histogram.Snapshot().Mean()) + } +} + +func getBrokerMetricName(prefix, brokerID string) string { + return prefix + brokerID +} + +func (sm *saramaMetricsMonitor) collectBrokerMetrics() error { + brokers, _, err := sm.admin.DescribeCluster() + if err != nil { + return err + } + + for _, b := range brokers { + brokerID := strconv.Itoa(int(b.ID())) + + incomingByteRateMetric := sm.registry.Get(getBrokerMetricName(incomingByteRateMetricNamePrefix, brokerID)) + if meter, ok := incomingByteRateMetric.(metrics.Meter); ok { + incomingByteRateGauge.WithLabelValues(sm.captureAddr, sm.changefeedID, brokerID).Set(meter.Snapshot().Rate1()) + } + + outgoingByteRateMetric := sm.registry.Get(getBrokerMetricName(outgoingByteRateMetricNamePrefix, brokerID)) + if meter, ok := outgoingByteRateMetric.(metrics.Meter); ok { + outgoingByteRateGauge.WithLabelValues(sm.captureAddr, sm.changefeedID, brokerID).Set(meter.Snapshot().Rate1()) + } + + requestRateMetric := sm.registry.Get(getBrokerMetricName(requestRateMetricNamePrefix, brokerID)) + if meter, ok := requestRateMetric.(metrics.Meter); ok { + requestRateGauge.WithLabelValues(sm.captureAddr, sm.changefeedID, brokerID).Set(meter.Snapshot().Rate1()) + } + + requestSizeMetric := sm.registry.Get(getBrokerMetricName(requestSizeMetricNamePrefix, brokerID)) + if histogram, ok := requestSizeMetric.(metrics.Histogram); ok { + requestSizeGauge.WithLabelValues(sm.captureAddr, sm.changefeedID, brokerID).Set(histogram.Snapshot().Mean()) + } + + requestLatencyMetric := sm.registry.Get(getBrokerMetricName(requestLatencyInMsMetricNamePrefix, brokerID)) + if histogram, ok := requestLatencyMetric.(metrics.Histogram); ok { + requestLatencyInMsGauge.WithLabelValues(sm.captureAddr, sm.changefeedID, brokerID).Set(histogram.Snapshot().Mean()) + } + + requestsInFlightMetric := sm.registry.Get(getBrokerMetricName(requestsInFlightMetricNamePrefix, brokerID)) + if counter, ok := requestsInFlightMetric.(metrics.Counter); ok { + requestsInFlightGauge.WithLabelValues(sm.captureAddr, sm.changefeedID, brokerID).Set(float64(counter.Snapshot().Count())) + } + + responseRateMetric := sm.registry.Get(getBrokerMetricName(responseRateMetricNamePrefix, brokerID)) + if meter, ok := responseRateMetric.(metrics.Meter); ok { + responseRateGauge.WithLabelValues(sm.captureAddr, sm.changefeedID, brokerID).Set(meter.Snapshot().Rate1()) + } + + responseSizeMetric := sm.registry.Get(getBrokerMetricName(responseSizeMetricNamePrefix, brokerID)) + if histogram, ok := responseSizeMetric.(metrics.Histogram); ok { + responseSizeGauge.WithLabelValues(sm.captureAddr, sm.changefeedID, brokerID).Set(histogram.Snapshot().Mean()) + } + } + return nil +} + +func NewSaramaMetricsMonitor(registry metrics.Registry, captureAddr, changefeedID string, admin kafka.ClusterAdminClient) *saramaMetricsMonitor { + return &saramaMetricsMonitor{ + captureAddr: captureAddr, + changefeedID: changefeedID, + registry: registry, + admin: admin, + } +} + +func (sm *saramaMetricsMonitor) Cleanup() { + sm.cleanUpProducerMetrics() + if err := sm.cleanUpBrokerMetrics(); err != nil { + log.Warn("clean up broker metrics failed", zap.Error(err)) + } +} + +func (sm *saramaMetricsMonitor) cleanUpProducerMetrics() { + batchSizeGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID) + recordSendRateGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID) + recordPerRequestGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID) + compressionRatioGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID) +} + +func (sm *saramaMetricsMonitor) cleanUpBrokerMetrics() error { + brokers, _, err := sm.admin.DescribeCluster() + if err != nil { + return err + } + + for _, b := range brokers { + brokerID := strconv.Itoa(int(b.ID())) + + incomingByteRateGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID, brokerID) + outgoingByteRateGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID, brokerID) + requestRateGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID, brokerID) + requestSizeGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID, brokerID) + requestLatencyInMsGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID, brokerID) + requestsInFlightGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID, brokerID) + responseRateGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID, brokerID) + responseSizeGauge.DeleteLabelValues(sm.captureAddr, sm.changefeedID, brokerID) + } + return nil +} diff --git a/metrics/grafana/ticdc.json b/metrics/grafana/ticdc.json index 6391d229c44..408746ea41d 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -124,8 +124,13 @@ "editable": true, "gnetId": null, "graphTooltip": 1, +<<<<<<< HEAD "id": null, "iteration": 1640447119260, +======= + "id": 25, + "iteration": 1644826138916, +>>>>>>> 28fe713de (cdc/sink: kafka sink integrate broker level metrics (#4517)) "links": [], "panels": [ { @@ -3503,7 +3508,7 @@ "h": 4, "w": 12, "x": 0, - "y": 4 + "y": 84 }, "hiddenSeries": false, "id": 218, @@ -3597,7 +3602,7 @@ "h": 4, "w": 12, "x": 12, - "y": 4 + "y": 84 }, "hiddenSeries": false, "id": 229, @@ -3691,7 +3696,7 @@ "h": 4, "w": 12, "x": 0, - "y": 8 + "y": 88 }, "hiddenSeries": false, "id": 228, @@ -3785,7 +3790,7 @@ "h": 4, "w": 12, "x": 12, - "y": 8 + "y": 88 }, "hiddenSeries": false, "id": 220, @@ -3879,7 +3884,7 @@ "h": 4, "w": 12, "x": 0, - "y": 12 + "y": 92 }, "hiddenSeries": false, "id": 219, @@ -3973,7 +3978,7 @@ "h": 4, "w": 12, "x": 12, - "y": 12 + "y": 92 }, "hiddenSeries": false, "id": 224, @@ -4067,7 +4072,7 @@ "h": 4, "w": 12, "x": 0, - "y": 16 + "y": 96 }, "hiddenSeries": false, "id": 223, @@ -4161,7 +4166,7 @@ "h": 4, "w": 12, "x": 12, - "y": 16 + "y": 96 }, "hiddenSeries": false, "id": 221, @@ -4255,7 +4260,7 @@ "h": 4, "w": 12, "x": 0, - "y": 20 + "y": 100 }, "hiddenSeries": false, "id": 222, @@ -4349,7 +4354,7 @@ "h": 4, "w": 12, "x": 12, - "y": 20 + "y": 100 }, "hiddenSeries": false, "id": 226, @@ -4443,7 +4448,7 @@ "h": 4, "w": 12, "x": 0, - "y": 24 + "y": 104 }, "hiddenSeries": false, "id": 227, @@ -4539,7 +4544,7 @@ "h": 4, "w": 12, "x": 12, - "y": 24 + "y": 104 }, "hiddenSeries": false, "id": 225, @@ -4635,7 +4640,7 @@ "h": 4, "w": 12, "x": 0, - "y": 28 + "y": 108 }, "hiddenSeries": false, "id": 108, @@ -4730,7 +4735,7 @@ "h": 4, "w": 12, "x": 12, - "y": 28 + "y": 108 }, "hiddenSeries": false, "id": 240, @@ -10312,75 +10317,118 @@ "current": true, "max": false, "min": false, +<<<<<<< HEAD + "rightSide": 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, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "", + "fill": 1, + "gridPos": { + "h": 7, + "w": 6, + "x": 12, + "y": 28 + }, + "id": 72, + "legend": { + "alignAsTable": true, + "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 }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", + "paceLength": 10, "percentage": false, - "pluginVersion": "6.1.6", "pointradius": 5, "points": false, "renderer": "flot", - "scopedVars": { - "instance": { - "selected": false, - "text": "172.16.5.33:47912", - "value": "172.16.5.33:47912" - }, - "runtime_instance": { - "selected": false, - "text": "172.16.5.37:47912", - "value": "172.16.5.37:47912" - } - }, - "seriesOverrides": [ - {} - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "go_gc_duration_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\", quantile=\"0\"}", - "format": "time_series", - "hide": false, - "instant": false, - "intervalFactor": 1, - "legendFormat": "min", - "refId": "A", - "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\"}", + "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", - "instant": false, "intervalFactor": 1, - "legendFormat": "max", - "refId": "C" + "legendFormat": "{{instance}}-p9999", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "GC STW Duration (last 256 GC cycles)", + "title": "Initial scan duration percentile", "tooltip": { - "msResolution": false, "shared": true, - "sort": 1, + "sort": 0, "value_type": "individual" }, "type": "graph", @@ -10397,7 +10445,7 @@ "label": null, "logBase": 1, "max": null, - "min": "0", + "min": null, "show": true }, { @@ -10420,23 +10468,29 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The throughput of Go's memory allocator.", + "decimals": 1, + "description": "The number of incremental scan task in different status.", "fill": 1, "gridPos": { "h": 7, - "w": 12, - "x": 12, - "y": 13 + "w": 6, + "x": 18, + "y": 28 }, - "id": 161, + "id": 140, "legend": { - "alignAsTable": false, + "alignAsTable": true, "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 }, @@ -10444,34 +10498,14 @@ "linewidth": 1, "links": [], "nullPointMode": "null", + "paceLength": 10, "percentage": false, "pointradius": 5, "points": false, "renderer": "flot", - "scopedVars": { - "instance": { - "selected": false, - "text": "172.16.5.33:47912", - "value": "172.16.5.33:47912" - }, - "runtime_instance": { - "selected": false, - "text": "172.16.5.37:47912", - "value": "172.16.5.37:47912" - } - }, "seriesOverrides": [ { - "alias": "sweep", - "transform": "negative-Y" - }, - { - "alias": "alloc-ops", - "yaxis": 2 - }, - { - "alias": "swepp-ops", - "transform": "negative-Y", + "alias": "/.*ongoing/", "yaxis": 2 } ], @@ -10480,39 +10514,27 @@ "steppedLine": false, "targets": [ { - "expr": "irate(go_memstats_alloc_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[30s])", + "expr": "sum(tikv_cdc_scan_tasks{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", type=\"ongoing\"}) by (type, instance)", "format": "time_series", + "hide": false, "intervalFactor": 1, - "legendFormat": "alloc", + "legendFormat": "{{instance}}-{{type}}", "refId": "A" }, { - "expr": "irate((go_memstats_alloc_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"})[30s:])", + "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": 1, - "legendFormat": "sweep", + "legendFormat": "{{instance}}-pending", "refId": "B" - }, - { - "expr": "irate(go_memstats_mallocs_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[30s])", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "alloc-ops", - "refId": "C" - }, - { - "expr": "irate(go_memstats_frees_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[30s])", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "swepp-ops", - "refId": "D" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Allocator Throughput", + "title": "Initial scan tasks status", "tooltip": { "shared": true, "sort": 0, @@ -10528,7 +10550,7 @@ }, "yaxes": [ { - "format": "Bps", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -10536,7 +10558,7 @@ "show": true }, { - "format": "ops", + "format": "none", "label": null, "logBase": 1, "max": null, @@ -10545,13 +10567,4394 @@ } ], "yaxis": { - "align": true, + "align": false, "alignLevel": null } - } - ], - "repeat": "runtime_instance", - "title": "Runtime $runtime_instance", + }, + { + "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": 0, + "y": 35 + }, + "id": 78, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": null, + "sortDesc": null, + "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": "avg(tikv_cdc_captured_region_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}) by (instance)", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "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": "Captured region count", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "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 speed of TiKV CDC incremental scan", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 6, + "x": 12, + "y": 35 + }, + "id": 76, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "paceLength": 10, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": 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, + "intervalFactor": 2, + "legendFormat": "tikv-{{instance}}", + "refId": "A", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "CDC scan speed", + "tooltip": { + "msResolution": false, + "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 + } + }, + { + "aliasColors": {}, + "bars": true, + "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": {}, + "gridPos": { + "h": 7, + "w": 6, + "x": 18, + "y": 35 + }, + "id": 139, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": true, + "hideZero": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": false, + "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_scan_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\", job=\"tikv\"}) by (instance)", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "tikv-{{instance}}", + "refId": "A", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "CDC total scan bytes", + "tooltip": { + "msResolution": false, + "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}", + "decimals": 1, + "description": "", + "fill": 4, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 42 + }, + "id": 143, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "hideZero": true, + "max": false, + "min": true, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "paceLength": 10, + "percentage": false, + "pointradius": 1, + "points": true, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/(access|miss).*/", + "fill": 0, + "points": false, + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "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)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "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": "Old value cache hit", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "percentunit", + "label": null, + "logBase": 1, + "max": "1", + "min": "0", + "show": true + }, + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "The total number of cache entries in the old value cache.", + "fill": 1, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 42 + }, + "id": 145, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "hideZero": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "paceLength": 10, + "percentage": false, + "pointradius": 1, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/.*len/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "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}}-len", + "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", + "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 + }, + { + "decimals": null, + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 49 + }, + "id": 141, + "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", + "paceLength": 10, + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "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)", + "format": "time_series", + "hide": false, + "intervalFactor": 2, + "legendFormat": "{{instance}}-{{cf}}-{{tag}}", + "refId": "A", + "step": 10 + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Old value seek operation", + "tooltip": { + "msResolution": false, + "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": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#FF9830", + "colorScale": "linear", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "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": "", + "fill": 1, + "gridPos": { + "h": 7, + "w": 6, + "x": 18, + "y": 49 + }, + "id": 142, + "legend": { + "alignAsTable": true, + "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 + }, + "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(tikv_cdc_old_value_duration_bucket{tidb_cluster=\"$tidb_cluster\", instance=~\"$tikv_instance\"}[1m])) by (le, instance, tag))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "{{instance}}-99%-{{tag}}", + "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": "Old value seek 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": false + } + ], + "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, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 4, + "x": 0, + "y": 8 + }, + "hiddenSeries": false, + "id": 321, + "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": "7.5.7", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum without (from) (rate(ticdc_message_server_message_count{instance=~\"$capture\"}[30s]))", + "interval": "", + "legendFormat": "{{instance}}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Message Receive Rate", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 4, + "x": 4, + "y": 8 + }, + "hiddenSeries": false, + "id": 323, + "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": "7.5.7", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum without (to) (rate(ticdc_message_client_message_count{instance=~\"$capture\"}[30s]))", + "interval": "", + "legendFormat": "{{instance}}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Message Send Rate", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 4, + "x": 8, + "y": 8 + }, + "hiddenSeries": false, + "id": 352, + "legend": { + "avg": false, + "current": false, + "hideEmpty": true, + "hideZero": true, + "max": false, + "min": false, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.7", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.5, sum(rate(ticdc_message_server_message_batch_size_bucket[30s])) by (le))", + "interval": "", + "legendFormat": "p50", + "queryType": "randomWalk", + "refId": "A" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.95, sum(rate(ticdc_message_server_message_batch_size_bucket[30s])) by (le))", + "hide": false, + "interval": "", + "legendFormat": "p90", + "refId": "B" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(ticdc_message_server_message_batch_size_bucket[30s])) by (le))", + "hide": false, + "interval": "", + "legendFormat": "p99", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Message Batch Size", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "transformations": [], + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": { + "unit": "short" + }, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 12, + "y": 8 + }, + "hiddenSeries": false, + "id": 354, + "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": "7.5.7", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.5, sum(rate(ticdc_message_server_message_batch_bytes_bucket [30s])) by (le))", + "hide": false, + "interval": "", + "legendFormat": "p50", + "refId": "C" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.95, sum(rate(ticdc_message_server_message_batch_bytes_bucket [30s])) by (le))", + "interval": "", + "legendFormat": "p95", + "queryType": "randomWalk", + "refId": "A" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(ticdc_message_server_message_batch_bytes_bucket [30s])) by (le))", + "hide": false, + "interval": "", + "legendFormat": "p99", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Receive message batch bytes percentiles", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 6, + "x": 18, + "y": 8 + }, + "hiddenSeries": false, + "id": 356, + "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": "7.5.7", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "histogram_quantile(0.5, sum(rate(ticdc_message_server_message_bytes_bucket [30s])) by (le))", + "interval": "", + "legendFormat": "p50", + "queryType": "randomWalk", + "refId": "A" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.95, sum(rate(ticdc_message_server_message_bytes_bucket [30s])) by (le))", + "hide": false, + "interval": "", + "legendFormat": "p95", + "refId": "B" + }, + { + "exemplar": true, + "expr": "histogram_quantile(0.99, sum(rate(ticdc_message_server_message_bytes_bucket [30s])) by (le))", + "hide": false, + "interval": "", + "legendFormat": "p99", + "refId": "C" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Receive Message Bytes Percentile", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "columns": [], + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": { + "color": { + "mode": "thresholds" + }, + "custom": { + "align": "left", + "displayMode": "color-background", + "filterable": false + }, + "mappings": [], + "noValue": "0", + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "rgb(255, 255, 255)", + "value": null + }, + { + "color": "dark-blue", + "value": 1 + }, + { + "color": "dark-red", + "value": 2 + } + ] + } + }, + "overrides": [ + { + "matcher": { + "id": "byName", + "options": "dest\\src" + }, + "properties": [ + { + "id": "custom.width", + "value": 137 + } + ] + } + ] + }, + "fontSize": "100%", + "gridPos": { + "h": 8, + "w": 20, + "x": 0, + "y": 15 + }, + "id": 350, + "options": { + "showHeader": true, + "sortBy": [] + }, + "pageSize": null, + "pluginVersion": "7.5.7", + "scroll": true, + "showHeader": true, + "sort": { + "col": 0, + "desc": true + }, + "styles": [ + { + "alias": "Time", + "dateFormat": "YYYY-MM-DD HH:mm:ss", + "pattern": "Time", + "type": "date" + }, + { + "alias": "", + "colorMode": null, + "colors": [ + "rgba(245, 54, 54, 0.9)", + "rgba(237, 129, 40, 0.89)", + "rgba(50, 172, 45, 0.97)" + ], + "decimals": 2, + "pattern": "/.*/", + "thresholds": [], + "type": "number", + "unit": "short" + } + ], + "targets": [ + { + "exemplar": true, + "expr": "ticdc_message_server_cur_stream_count", + "format": "time_series", + "instant": true, + "interval": "", + "legendFormat": "", + "queryType": "randomWalk", + "refId": "A" + } + ], + "title": "Stream Count Between Nodes", + "transform": "timeseries_to_columns", + "transformations": [ + { + "id": "labelsToFields", + "options": { + "valueLabel": "from" + } + }, + { + "id": "organize", + "options": { + "excludeByName": { + "Time": true, + "instance": false, + "job": true + }, + "indexByName": {}, + "renameByName": { + "instance": "dest\\src" + } + } + } + ], + "type": "table" + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 4, + "x": 20, + "y": 15 + }, + "hiddenSeries": false, + "id": 358, + "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": "7.5.7", + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(ticdc_message_server_cur_stream_count)", + "interval": "", + "legendFormat": "actual", + "queryType": "randomWalk", + "refId": "A" + }, + { + "exemplar": true, + "expr": "2 * count(process_start_time_seconds{tidb_cluster=\"$tidb_cluster\", job=\"ticdc\"}) - 1", + "hide": false, + "interval": "", + "legendFormat": "expected", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Total Stream Count", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "title": "Peer Messages", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 9 + }, + "id": 155, + "panels": [ + { + "aliasColors": {}, + "bars": true, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiCDC process rss memory usage. TiCDC heap memory size in use ", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 6 + }, + "id": 157, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "hideEmpty": true, + "hideZero": true, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": false, + "linewidth": 1, + "links": [], + "maxPerRow": 3, + "nullPointMode": "null", + "percentage": false, + "pluginVersion": "6.1.6", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": "h", + "scopedVars": { + "instance": { + "selected": false, + "text": "172.16.5.33:47912", + "value": "172.16.5.33:47912" + }, + "runtime_instance": { + "selected": false, + "text": "172.16.6.111:8300", + "value": "172.16.6.111:8300" + } + }, + "seriesOverrides": [ + { + "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": true, + "steppedLine": false, + "targets": [ + { + "expr": "process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "alloc-from-os", + "refId": "A" + }, + { + "expr": "go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / (1 + ticdc_server_go_gc{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / 100)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "estimate-inuse", + "refId": "H" + }, + { + "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": "estimate-garbage", + "refId": "C" + }, + { + "expr": "go_memstats_heap_idle_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_heap_released_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_heap_inuse_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "reserved-by-go", + "refId": "B" + }, + { + "expr": "go_memstats_stack_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_mspan_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_mcache_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_buck_hash_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_gc_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_other_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", + "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": "Memory Usage", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": "", + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": "", + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Count of live objects.", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 6 + }, + "id": 158, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "hideEmpty": true, + "hideZero": true, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxPerRow": 3, + "nullPointMode": "null", + "percentage": false, + "pluginVersion": "6.1.6", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": "h", + "scopedVars": { + "instance": { + "selected": false, + "text": "172.16.5.33:47912", + "value": "172.16.5.33:47912" + }, + "runtime_instance": { + "selected": false, + "text": "172.16.6.111:8300", + "value": "172.16.6.111:8300" + } + }, + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "go_memstats_heap_objects{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "objects", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Estimated Live Objects", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": "", + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": "", + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiCDC process Go garbage collection STW pause duration", + "editable": true, + "error": false, + "fill": 1, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 13 + }, + "id": 160, + "legend": { + "alignAsTable": false, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "percentage": false, + "pluginVersion": "6.1.6", + "pointradius": 5, + "points": false, + "renderer": "flot", + "scopedVars": { + "instance": { + "selected": false, + "text": "172.16.5.33:47912", + "value": "172.16.5.33:47912" + }, + "runtime_instance": { + "selected": false, + "text": "172.16.6.111:8300", + "value": "172.16.6.111:8300" + } + }, + "seriesOverrides": [ + {} + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "go_gc_duration_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\", quantile=\"0\"}", + "format": "time_series", + "hide": false, + "instant": false, + "intervalFactor": 1, + "legendFormat": "min", + "refId": "A", + "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": "GC STW Duration (last 256 GC cycles)", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 1, + "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": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The throughput of Go's memory allocator.", + "fill": 1, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 13 + }, + "id": 161, + "legend": { + "alignAsTable": false, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "scopedVars": { + "instance": { + "selected": false, + "text": "172.16.5.33:47912", + "value": "172.16.5.33:47912" + }, + "runtime_instance": { + "selected": false, + "text": "172.16.6.111:8300", + "value": "172.16.6.111:8300" + } + }, + "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": "irate(go_memstats_alloc_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[30s])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "alloc", + "refId": "A" + }, + { + "expr": "irate((go_memstats_alloc_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"})[30s:])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "sweep", + "refId": "B" + }, + { + "expr": "irate(go_memstats_mallocs_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[30s])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "alloc-ops", + "refId": "C" + }, + { + "expr": "irate(go_memstats_frees_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[30s])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "swepp-ops", + "refId": "D" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Allocator Throughput", + "tooltip": { + "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": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": true, + "alignLevel": null + } + } + ], + "repeat": "runtime_instance", + "scopedVars": { + "runtime_instance": { + "selected": false, + "text": "172.16.6.111:8300", + "value": "172.16.6.111:8300" + } + }, + "title": "Runtime $runtime_instance", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 10 + }, + "id": 426, + "panels": [ + { + "aliasColors": {}, + "bars": true, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiCDC process rss memory usage. TiCDC heap memory size in use ", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 6 + }, + "id": 427, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "hideEmpty": true, + "hideZero": true, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": false, + "linewidth": 1, + "links": [], + "maxPerRow": 3, + "nullPointMode": "null", + "percentage": false, + "pluginVersion": "6.1.6", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": "h", + "repeatIteration": 1644826138916, + "repeatPanelId": 157, + "repeatedByRow": true, + "scopedVars": { + "instance": { + "selected": false, + "text": "172.16.5.33:47912", + "value": "172.16.5.33:47912" + }, + "runtime_instance": { + "selected": false, + "text": "172.16.7.51:8300", + "value": "172.16.7.51:8300" + } + }, + "seriesOverrides": [ + { + "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": true, + "steppedLine": false, + "targets": [ + { + "expr": "process_resident_memory_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "alloc-from-os", + "refId": "A" + }, + { + "expr": "go_memstats_next_gc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / (1 + ticdc_server_go_gc{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} / 100)", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "estimate-inuse", + "refId": "H" + }, + { + "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": "estimate-garbage", + "refId": "C" + }, + { + "expr": "go_memstats_heap_idle_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_heap_released_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_heap_inuse_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "reserved-by-go", + "refId": "B" + }, + { + "expr": "go_memstats_stack_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_mspan_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_mcache_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_buck_hash_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_gc_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} + go_memstats_other_sys_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", + "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": "Memory Usage", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "bytes", + "label": "", + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": "", + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Count of live objects.", + "editable": true, + "error": false, + "fill": 0, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 6 + }, + "id": 428, + "legend": { + "alignAsTable": false, + "avg": false, + "current": false, + "hideEmpty": true, + "hideZero": true, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "sideWidth": null, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "maxPerRow": 3, + "nullPointMode": "null", + "percentage": false, + "pluginVersion": "6.1.6", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeat": null, + "repeatDirection": "h", + "repeatIteration": 1644826138916, + "repeatPanelId": 158, + "repeatedByRow": true, + "scopedVars": { + "instance": { + "selected": false, + "text": "172.16.5.33:47912", + "value": "172.16.5.33:47912" + }, + "runtime_instance": { + "selected": false, + "text": "172.16.7.51:8300", + "value": "172.16.7.51:8300" + } + }, + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "go_memstats_heap_objects{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}", + "format": "time_series", + "hide": false, + "intervalFactor": 1, + "legendFormat": "objects", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Estimated Live Objects", + "tooltip": { + "msResolution": true, + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "short", + "label": "", + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": "", + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "cacheTimeout": null, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "TiCDC process Go garbage collection STW pause duration", + "editable": true, + "error": false, + "fill": 1, + "grid": {}, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 13 + }, + "id": 429, + "legend": { + "alignAsTable": false, + "avg": false, + "current": true, + "max": false, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "percentage": false, + "pluginVersion": "6.1.6", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeatIteration": 1644826138916, + "repeatPanelId": 160, + "repeatedByRow": true, + "scopedVars": { + "instance": { + "selected": false, + "text": "172.16.5.33:47912", + "value": "172.16.5.33:47912" + }, + "runtime_instance": { + "selected": false, + "text": "172.16.7.51:8300", + "value": "172.16.7.51:8300" + } + }, + "seriesOverrides": [ + {} + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "go_gc_duration_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\", quantile=\"0\"}", + "format": "time_series", + "hide": false, + "instant": false, + "intervalFactor": 1, + "legendFormat": "min", + "refId": "A", + "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": "GC STW Duration (last 256 GC cycles)", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 1, + "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": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The throughput of Go's memory allocator.", + "fill": 1, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 13 + }, + "id": 430, + "legend": { + "alignAsTable": false, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeatIteration": 1644826138916, + "repeatPanelId": 161, + "repeatedByRow": true, + "scopedVars": { + "instance": { + "selected": false, + "text": "172.16.5.33:47912", + "value": "172.16.5.33:47912" + }, + "runtime_instance": { + "selected": false, + "text": "172.16.7.51:8300", + "value": "172.16.7.51:8300" + } + }, + "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": "irate(go_memstats_alloc_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[30s])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "alloc", + "refId": "A" + }, + { + "expr": "irate((go_memstats_alloc_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"})[30s:])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "sweep", + "refId": "B" + }, + { + "expr": "irate(go_memstats_mallocs_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[30s])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "alloc-ops", + "refId": "C" + }, + { + "expr": "irate(go_memstats_frees_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[30s])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "swepp-ops", + "refId": "D" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Allocator Throughput", + "tooltip": { + "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": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": true, + "alignLevel": null + } + } + ], + "repeat": null, + "repeatIteration": 1644826138916, + "repeatPanelId": 155, + "scopedVars": { + "runtime_instance": { + "selected": false, + "text": "172.16.7.51:8300", + "value": "172.16.7.51:8300" + } + }, + "title": "Runtime $runtime_instance", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 11 + }, + "id": 187, + "panels": [ + { + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "The latency distributions of fsync called by redo writer", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 8 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 172, + "legend": { + "show": true + }, + "pluginVersion": "6.1.6", + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "max(rate(ticdc_redo_fsync_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Redo fsync duration", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "cards": { + "cardPadding": 0, + "cardRound": 0 + }, + "color": { + "cardColor": "#b4ff00", + "colorScale": "sqrt", + "colorScheme": "interpolateSpectral", + "exponent": 0.5, + "min": 0, + "mode": "spectrum" + }, + "dataFormat": "tsbuckets", + "datasource": "${DS_TEST-CLUSTER}", + "description": "The latency distributions of flushall called by redo writer", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 12, + "y": 8 + }, + "heatmap": {}, + "hideZeroBuckets": true, + "highlightCards": true, + "id": 180, + "legend": { + "show": true + }, + "pluginVersion": "6.1.6", + "reverseYBuckets": false, + "targets": [ + { + "exemplar": true, + "expr": "max(rate(ticdc_redo_flushall_duration_seconds_bucket{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\",capture=~\"$capture\"}[1m])) by (le)", + "format": "heatmap", + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{le}}", + "queryType": "randomWalk", + "refId": "A" + } + ], + "timeFrom": null, + "timeShift": null, + "title": "Redo flushall duration", + "tooltip": { + "show": true, + "showHistogram": true + }, + "tooltipDecimals": 1, + "type": "heatmap", + "xAxis": { + "show": true + }, + "xBucketNumber": null, + "xBucketSize": null, + "yAxis": { + "decimals": 1, + "format": "s", + "logBase": 1, + "max": null, + "min": null, + "show": true, + "splitFactor": null + }, + "yBucketBound": "upper", + "yBucketNumber": null, + "yBucketSize": null + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The total count of rows that are processed by redo writer", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 12, + "x": 0, + "y": 16 + }, + "hiddenSeries": false, + "id": 170, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": false, + "rightSide": true, +>>>>>>> 28fe713de (cdc/sink: kafka sink integrate broker level metrics (#4517)) + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "percentage": false, + "pluginVersion": "6.1.6", + "pointradius": 5, + "points": false, + "renderer": "flot", + "scopedVars": { + "instance": { + "selected": false, + "text": "172.16.5.33:47912", + "value": "172.16.5.33:47912" + }, + "runtime_instance": { + "selected": false, + "text": "172.16.5.37:47912", + "value": "172.16.5.37:47912" + } + }, + "seriesOverrides": [ + {} + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "go_gc_duration_seconds{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\", quantile=\"0\"}", + "format": "time_series", + "hide": false, + "instant": false, + "intervalFactor": 1, + "legendFormat": "min", + "refId": "A", + "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": "GC STW Duration (last 256 GC cycles)", + "tooltip": { + "msResolution": false, + "shared": true, + "sort": 1, + "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": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The throughput of Go's memory allocator.", + "fill": 1, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 13 + }, + "id": 161, + "legend": { + "alignAsTable": false, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "percentage": false, + "pointradius": 5, + "points": false, + "renderer": "flot", + "scopedVars": { + "instance": { + "selected": false, + "text": "172.16.5.33:47912", + "value": "172.16.5.33:47912" + }, + "runtime_instance": { + "selected": false, + "text": "172.16.5.37:47912", + "value": "172.16.5.37:47912" + } + }, + "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": "irate(go_memstats_alloc_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[30s])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "alloc", + "refId": "A" + }, + { + "expr": "irate((go_memstats_alloc_bytes_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"} - go_memstats_heap_alloc_bytes{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"})[30s:])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "sweep", + "refId": "B" + }, + { + "expr": "irate(go_memstats_mallocs_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[30s])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "alloc-ops", + "refId": "C" + }, + { + "expr": "irate(go_memstats_frees_total{tidb_cluster=\"$tidb_cluster\", instance=~\"$runtime_instance\"}[30s])", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "swepp-ops", + "refId": "D" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Allocator Throughput", + "tooltip": { + "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": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": true, + "alignLevel": null + } + } + ], +<<<<<<< HEAD + "repeat": "runtime_instance", + "title": "Runtime $runtime_instance", +======= + "title": "Redo", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 12 + }, + "id": 402, + "panels": [ + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of bytes send for all topics.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 122 + }, + "hiddenSeries": false, + "id": 410, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_kafka_producer_batch_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "batch 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": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Records/second sent to all topics\n\nvalue = one-minute moving average of record send rate", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 122 + }, + "hiddenSeries": false, + "id": 408, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_kafka_producer_record_send_rate{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "records send rate", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The number of records sent per request for all topics\n\nvalue = records per request histogram's mean", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 129 + }, + "hiddenSeries": false, + "id": 409, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_kafka_producer_records_per_request{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "records per request", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "the compression ratio times 100 of record batches for all topics", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 129 + }, + "hiddenSeries": false, + "id": 416, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_kafka_producer_compression_ratio{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "compression ratio", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "percent", + "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": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Bytes/second write to brokers\n\nvalue = one-minute moving average rate of sent bytes", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 136 + }, + "hiddenSeries": false, + "id": 418, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_kafka_producer_outgoing_byte_rate{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture, broker)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}-{{broker}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Bytes/second write to brokers", + "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": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Bytes/second read from different brokers.\n\nvalue = one-minute moving average rate of all read bytes", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 136 + }, + "hiddenSeries": false, + "id": 417, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_kafka_producer_incoming_byte_rate{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture, broker)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}-{{broker}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Bytes/second read from brokers", + "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": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Requests/second sent to all brokers\n\nvalue = one-minute moving average rate of events per second", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 143 + }, + "hiddenSeries": false, + "id": 419, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_kafka_producer_request_rate{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture, broker)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}-{{broker}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "request rate", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Responses/second received from all brokers\n\nvalue = one-minute moving average of response receive rate", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 143 + }, + "hiddenSeries": false, + "id": 423, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_kafka_producer_response_rate{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture, broker)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}-{{broker}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "response rate", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "the request size in bytes for all brokers\n\nvalue = requests histogram's mean", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 150 + }, + "hiddenSeries": false, + "id": 425, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_kafka_producer_request_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture, broker)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}-{{broker}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "request 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": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "the response size in bytes\n\nvalue = response size histogram's mean", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 12, + "y": 150 + }, + "hiddenSeries": false, + "id": 424, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_kafka_producer_response_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture, broker)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}-{{broker}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "response 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": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "the request latency in ms for all brokers\n\nvalue = request latency histogram's mean", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 157 + }, + "hiddenSeries": false, + "id": 420, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_kafka_producer_request_latency{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture, broker)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}-{{broker}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "the request latency", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "ms", + "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": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "the current number of in-flight requests awaiting a response\n\nvalue = the count of inflight requests.\n\nupdated each 1s, in a good network environment, this value should be very small.", + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 7, + "w": 12, + "x": 0, + "y": 164 + }, + "hiddenSeries": false, + "id": 422, + "legend": { + "alignAsTable": true, + "avg": true, + "current": true, + "max": true, + "min": true, + "rightSide": false, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "dataLinks": [] + }, + "paceLength": 10, + "percentage": false, + "pointradius": 2, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(ticdc_sink_kafka_producer_in_flight_requests{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (changefeed, capture, broker)", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "{{changefeed}}-{{capture}}-{{broker}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "inflight requests", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "none", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "short", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + } + ], + "title": "Kafka Sink", +>>>>>>> 28fe713de (cdc/sink: kafka sink integrate broker level metrics (#4517)) "type": "row" } ], @@ -10758,5 +15161,9 @@ "timezone": "browser", "title": "Test-Cluster-TiCDC", "uid": "YiGL8hBZ1", +<<<<<<< HEAD "version": 32 +======= + "version": 34 +>>>>>>> 28fe713de (cdc/sink: kafka sink integrate broker level metrics (#4517)) }