diff --git a/cdc/sink/producer/kafka/kafka.go b/cdc/sink/producer/kafka/kafka.go index dd13cfdfaba..2888587c029 100644 --- a/cdc/sink/producer/kafka/kafka.go +++ b/cdc/sink/producer/kafka/kafka.go @@ -54,6 +54,7 @@ type kafkaSaramaProducer struct { // 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. clientLock sync.RWMutex + admin kafka.ClusterAdminClient client sarama.Client asyncProducer sarama.AsyncProducer syncProducer sarama.SyncProducer @@ -262,6 +263,16 @@ func (k *kafkaSaramaProducer) Close() error { zap.String("changefeed", k.id), zap.Any("role", k.role)) } + 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() return nil } @@ -329,12 +340,6 @@ func NewKafkaSaramaProducer(ctx context.Context, topic string, config *Config, if err != nil { return nil, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) } - defer func() { - if err := admin.Close(); err != nil { - log.Warn("close kafka cluster admin failed", zap.Error(err), - zap.String("changefeed", changefeedID), zap.Any("role", role)) - } - }() if err := validateAndCreateTopic(admin, topic, config, cfg, opts); err != nil { return nil, cerror.WrapError(cerror.ErrKafkaNewSaramaProducer, err) @@ -361,6 +366,7 @@ func NewKafkaSaramaProducer(ctx context.Context, topic string, config *Config, return nil, err } k := &kafkaSaramaProducer{ + admin: admin, client: client, asyncProducer: asyncProducer, syncProducer: syncProducer, @@ -380,7 +386,7 @@ func NewKafkaSaramaProducer(ctx context.Context, topic string, config *Config, role: role, metricsMonitor: NewSaramaMetricsMonitor(cfg.MetricRegistry, - util.CaptureAddrFromCtx(ctx), changefeedID), + util.CaptureAddrFromCtx(ctx), changefeedID, admin), } go func() { if err := k.run(ctx); err != nil && errors.Cause(err) != context.Canceled { diff --git a/cdc/sink/producer/kafka/metrics.go b/cdc/sink/producer/kafka/metrics.go index 600e2eb3940..844850938cc 100644 --- a/cdc/sink/producer/kafka/metrics.go +++ b/cdc/sink/producer/kafka/metrics.go @@ -14,12 +14,17 @@ 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 ( - // batch-size + // Histogram update by the `batch-size` batchSizeGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: "ticdc", @@ -28,7 +33,7 @@ var ( Help: "the number of bytes sent per partition per request for all topics", }, []string{"capture", "changefeed"}) - // record-send-rate + // meter mark by total records count recordSendRateGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: "ticdc", @@ -38,6 +43,7 @@ var ( }, []string{"capture", "changefeed"}) // records-per-request + // histogram update by all records count. recordPerRequestGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: "ticdc", @@ -46,7 +52,7 @@ var ( Help: "the number of records sent per request for all topics", }, []string{"capture", "changefeed"}) - // compression-ratio + // histogram update by `compression-ratio`. compressionRatioGauge = prometheus.NewGaugeVec( prometheus.GaugeOpts{ Namespace: "ticdc", @@ -54,6 +60,80 @@ var ( 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 @@ -62,14 +142,34 @@ func InitMetrics(registry *prometheus.Registry) { 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 { @@ -77,42 +177,135 @@ type saramaMetricsMonitor struct { 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.Mean()) + 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.Rate1()) + 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.Mean()) + 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.Mean()) + compressionRatioGauge.WithLabelValues(sm.captureAddr, sm.changefeedID).Set(histogram.Snapshot().Mean()) } } -func NewSaramaMetricsMonitor(registry metrics.Registry, captureAddr, changefeedID string) *saramaMetricsMonitor { +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 a1f00411376..9a9f8c9a4c2 100644 --- a/metrics/grafana/ticdc.json +++ b/metrics/grafana/ticdc.json @@ -124,8 +124,8 @@ "editable": true, "gnetId": null, "graphTooltip": 1, - "id": 33, - "iteration": 1644481309661, + "id": 25, + "iteration": 1644826138916, "links": [], "panels": [ { @@ -3503,7 +3503,7 @@ "h": 4, "w": 12, "x": 0, - "y": 4 + "y": 84 }, "hiddenSeries": false, "id": 218, @@ -3597,7 +3597,7 @@ "h": 4, "w": 12, "x": 12, - "y": 4 + "y": 84 }, "hiddenSeries": false, "id": 229, @@ -3691,7 +3691,7 @@ "h": 4, "w": 12, "x": 0, - "y": 8 + "y": 88 }, "hiddenSeries": false, "id": 228, @@ -3785,7 +3785,7 @@ "h": 4, "w": 12, "x": 12, - "y": 8 + "y": 88 }, "hiddenSeries": false, "id": 220, @@ -3879,7 +3879,7 @@ "h": 4, "w": 12, "x": 0, - "y": 12 + "y": 92 }, "hiddenSeries": false, "id": 219, @@ -3973,7 +3973,7 @@ "h": 4, "w": 12, "x": 12, - "y": 12 + "y": 92 }, "hiddenSeries": false, "id": 224, @@ -4067,7 +4067,7 @@ "h": 4, "w": 12, "x": 0, - "y": 16 + "y": 96 }, "hiddenSeries": false, "id": 223, @@ -4161,7 +4161,7 @@ "h": 4, "w": 12, "x": 12, - "y": 16 + "y": 96 }, "hiddenSeries": false, "id": 221, @@ -4255,7 +4255,7 @@ "h": 4, "w": 12, "x": 0, - "y": 20 + "y": 100 }, "hiddenSeries": false, "id": 222, @@ -4349,7 +4349,7 @@ "h": 4, "w": 12, "x": 12, - "y": 20 + "y": 100 }, "hiddenSeries": false, "id": 226, @@ -4443,7 +4443,7 @@ "h": 4, "w": 12, "x": 0, - "y": 24 + "y": 104 }, "hiddenSeries": false, "id": 227, @@ -4539,7 +4539,7 @@ "h": 4, "w": 12, "x": 12, - "y": 24 + "y": 104 }, "hiddenSeries": false, "id": 225, @@ -4635,7 +4635,7 @@ "h": 4, "w": 12, "x": 0, - "y": 28 + "y": 108 }, "hiddenSeries": false, "id": 108, @@ -4730,7 +4730,7 @@ "h": 4, "w": 12, "x": 12, - "y": 28 + "y": 108 }, "hiddenSeries": false, "id": 240, @@ -12350,8 +12350,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.6.47:8400", - "value": "172.16.6.47:8400" + "text": "172.16.6.111:8300", + "value": "172.16.6.111:8300" } }, "seriesOverrides": [ @@ -12536,8 +12536,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.6.47:8400", - "value": "172.16.6.47:8400" + "text": "172.16.6.111:8300", + "value": "172.16.6.111:8300" } }, "seriesOverrides": [], @@ -12643,8 +12643,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.6.47:8400", - "value": "172.16.6.47:8400" + "text": "172.16.6.111:8300", + "value": "172.16.6.111:8300" } }, "seriesOverrides": [ @@ -12765,8 +12765,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.6.47:8400", - "value": "172.16.6.47:8400" + "text": "172.16.6.111:8300", + "value": "172.16.6.111:8300" } }, "seriesOverrides": [ @@ -12863,8 +12863,8 @@ "scopedVars": { "runtime_instance": { "selected": false, - "text": "172.16.6.47:8400", - "value": "172.16.6.47:8400" + "text": "172.16.6.111:8300", + "value": "172.16.6.111:8300" } }, "title": "Runtime $runtime_instance", @@ -12878,7 +12878,7 @@ "x": 0, "y": 10 }, - "id": 411, + "id": 426, "panels": [ { "aliasColors": {}, @@ -12898,7 +12898,7 @@ "x": 0, "y": 6 }, - "id": 412, + "id": 427, "legend": { "alignAsTable": false, "avg": false, @@ -12925,7 +12925,7 @@ "renderer": "flot", "repeat": null, "repeatDirection": "h", - "repeatIteration": 1644481309661, + "repeatIteration": 1644826138916, "repeatPanelId": 157, "repeatedByRow": true, "scopedVars": { @@ -12936,8 +12936,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.6.47:8401", - "value": "172.16.6.47:8401" + "text": "172.16.7.51:8300", + "value": "172.16.7.51:8300" } }, "seriesOverrides": [ @@ -13087,7 +13087,7 @@ "x": 12, "y": 6 }, - "id": 413, + "id": 428, "legend": { "alignAsTable": false, "avg": false, @@ -13114,7 +13114,7 @@ "renderer": "flot", "repeat": null, "repeatDirection": "h", - "repeatIteration": 1644481309661, + "repeatIteration": 1644826138916, "repeatPanelId": 158, "repeatedByRow": true, "scopedVars": { @@ -13125,8 +13125,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.6.47:8401", - "value": "172.16.6.47:8401" + "text": "172.16.7.51:8300", + "value": "172.16.7.51:8300" } }, "seriesOverrides": [], @@ -13203,7 +13203,7 @@ "x": 0, "y": 13 }, - "id": 414, + "id": 429, "legend": { "alignAsTable": false, "avg": false, @@ -13224,7 +13224,7 @@ "pointradius": 5, "points": false, "renderer": "flot", - "repeatIteration": 1644481309661, + "repeatIteration": 1644826138916, "repeatPanelId": 160, "repeatedByRow": true, "scopedVars": { @@ -13235,8 +13235,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.6.47:8401", - "value": "172.16.6.47:8401" + "text": "172.16.7.51:8300", + "value": "172.16.7.51:8300" } }, "seriesOverrides": [ @@ -13329,7 +13329,7 @@ "x": 12, "y": 13 }, - "id": 415, + "id": 430, "legend": { "alignAsTable": false, "avg": false, @@ -13349,7 +13349,7 @@ "pointradius": 5, "points": false, "renderer": "flot", - "repeatIteration": 1644481309661, + "repeatIteration": 1644826138916, "repeatPanelId": 161, "repeatedByRow": true, "scopedVars": { @@ -13360,8 +13360,8 @@ }, "runtime_instance": { "selected": false, - "text": "172.16.6.47:8401", - "value": "172.16.6.47:8401" + "text": "172.16.7.51:8300", + "value": "172.16.7.51:8300" } }, "seriesOverrides": [ @@ -13455,13 +13455,13 @@ } ], "repeat": null, - "repeatIteration": 1644481309661, + "repeatIteration": 1644826138916, "repeatPanelId": 155, "scopedVars": { "runtime_instance": { "selected": false, - "text": "172.16.6.47:8401", - "value": "172.16.6.47:8401" + "text": "172.16.7.51:8300", + "value": "172.16.7.51:8300" } }, "title": "Runtime $runtime_instance", @@ -13836,7 +13836,7 @@ "type": "row" }, { - "collapsed": false, + "collapsed": true, "gridPos": { "h": 1, "w": 24, @@ -13844,421 +13844,1150 @@ "y": 12 }, "id": 402, - "panels": [], - "title": "Kafka Sink", - "type": "row" - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of bytes sent per partition per request for all topics", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 13 - }, - "hiddenSeries": false, - "id": 410, - "legend": { - "alignAsTable": true, - "avg": true, - "current": true, - "max": true, - "min": true, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "max(ticdc_sink_kafka_producer_batch_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}", - "refId": "A" - }, - { - "expr": "sum(ticdc_sink_kafka_producer_batch_size{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture,changefeed)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}-{{capture}}", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "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": "The number of records sent per request for all topics", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 12, - "y": 13 - }, - "hiddenSeries": false, - "id": 408, - "legend": { - "alignAsTable": true, - "avg": true, - "current": true, - "max": true, - "min": true, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "max(ticdc_sink_kafka_producer_record_send_rate{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}", - "refId": "A" - }, - { - "expr": "sum(ticdc_sink_kafka_producer_record_send_rate{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture,changefeed)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}-{{capture}}", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "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": "wps", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": false - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "description": "The number of records sent per request for all topics", - "fill": 1, - "fillGradient": 0, - "gridPos": { - "h": 7, - "w": 12, - "x": 0, - "y": 20 - }, - "hiddenSeries": false, - "id": 409, - "legend": { - "alignAsTable": true, - "avg": true, - "current": true, - "max": true, - "min": true, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "max(ticdc_sink_kafka_producer_records_per_request{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}", - "refId": "A" - }, - { - "expr": "sum(ticdc_sink_kafka_producer_records_per_request{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture,changefeed)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}-{{capture}}", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "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": 20 - }, - "hiddenSeries": false, - "id": 416, - "legend": { - "alignAsTable": true, - "avg": true, - "current": true, - "max": true, - "min": true, - "rightSide": true, - "show": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null", - "options": { - "dataLinks": [] - }, - "paceLength": 10, - "percentage": false, - "pointradius": 2, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "expr": "max(ticdc_sink_kafka_producer_compression_ratio{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\"}) by (changefeed)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}", - "refId": "A" - }, - { - "expr": "sum(ticdc_sink_kafka_producer_compression_ratio{tidb_cluster=\"$tidb_cluster\", changefeed=~\"$changefeed\", capture=~\"$capture\"}) by (capture,changefeed)", - "format": "time_series", - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{changefeed}}-{{capture}}", - "refId": "B" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "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 - } + "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", + "type": "row" } ], "refresh": "1m", @@ -14480,5 +15209,5 @@ "timezone": "browser", "title": "Test-Cluster-TiCDC", "uid": "YiGL8hBZ1", - "version": 33 + "version": 34 }