github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/pkg/sink/kafka/metrics_collector.go (about) 1 // Copyright 2023 PingCAP, Inc. 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // See the License for the specific language governing permissions and 12 // limitations under the License. 13 14 package kafka 15 16 import ( 17 "context" 18 "strconv" 19 "time" 20 21 "github.com/pingcap/log" 22 "github.com/pingcap/tiflow/cdc/model" 23 "github.com/pingcap/tiflow/pkg/util" 24 "github.com/rcrowley/go-metrics" 25 "go.uber.org/zap" 26 ) 27 28 // MetricsCollector is the interface for kafka metrics collector. 29 type MetricsCollector interface { 30 Run(ctx context.Context) 31 } 32 33 const ( 34 // RefreshMetricsInterval specifies the interval of refresh kafka client metrics. 35 RefreshMetricsInterval = 5 * time.Second 36 // refreshClusterMetaInterval specifies the interval of refresh kafka cluster meta. 37 // Do not set it too small, because it will cause too many requests to kafka cluster. 38 // Every request will get all topics and all brokers information. 39 refreshClusterMetaInterval = 30 * time.Minute 40 ) 41 42 // Sarama metrics names, see https://pkg.go.dev/github.com/IBM/sarama#pkg-overview. 43 const ( 44 // Producer level. 45 compressionRatioMetricName = "compression-ratio" 46 recordsPerRequestMetricName = "records-per-request" 47 48 // Broker level. 49 outgoingByteRateMetricNamePrefix = "outgoing-byte-rate-for-broker-" 50 requestRateMetricNamePrefix = "request-rate-for-broker-" 51 requestLatencyInMsMetricNamePrefix = "request-latency-in-ms-for-broker-" 52 requestsInFlightMetricNamePrefix = "requests-in-flight-for-broker-" 53 responseRateMetricNamePrefix = "response-rate-for-broker-" 54 ) 55 56 type saramaMetricsCollector struct { 57 changefeedID model.ChangeFeedID 58 role util.Role 59 // adminClient is used to get broker infos from broker. 60 adminClient ClusterAdminClient 61 brokers map[int32]struct{} 62 registry metrics.Registry 63 } 64 65 // NewSaramaMetricsCollector return a kafka metrics collector based on sarama library. 66 func NewSaramaMetricsCollector( 67 changefeedID model.ChangeFeedID, 68 role util.Role, 69 adminClient ClusterAdminClient, 70 registry metrics.Registry, 71 ) MetricsCollector { 72 return &saramaMetricsCollector{ 73 changefeedID: changefeedID, 74 role: role, 75 adminClient: adminClient, 76 brokers: make(map[int32]struct{}), 77 registry: registry, 78 } 79 } 80 81 func (m *saramaMetricsCollector) Run(ctx context.Context) { 82 // Initialize brokers. 83 m.updateBrokers(ctx) 84 85 refreshMetricsTicker := time.NewTicker(RefreshMetricsInterval) 86 refreshClusterMetaTicker := time.NewTicker(refreshClusterMetaInterval) 87 defer func() { 88 refreshMetricsTicker.Stop() 89 refreshClusterMetaTicker.Stop() 90 m.cleanupMetrics() 91 }() 92 93 for { 94 select { 95 case <-ctx.Done(): 96 log.Info("Kafka metrics collector stopped", 97 zap.String("namespace", m.changefeedID.Namespace), 98 zap.String("changefeed", m.changefeedID.ID)) 99 return 100 case <-refreshMetricsTicker.C: 101 m.collectBrokerMetrics() 102 m.collectProducerMetrics() 103 case <-refreshClusterMetaTicker.C: 104 m.updateBrokers(ctx) 105 } 106 } 107 } 108 109 func (m *saramaMetricsCollector) updateBrokers(ctx context.Context) { 110 start := time.Now() 111 brokers, err := m.adminClient.GetAllBrokers(ctx) 112 if err != nil { 113 log.Warn("Get Kafka brokers failed, "+ 114 "use historical brokers to collect kafka broker level metrics", 115 zap.String("namespace", m.changefeedID.Namespace), 116 zap.String("changefeed", m.changefeedID.ID), 117 zap.Any("role", m.role), 118 zap.Duration("duration", time.Since(start)), 119 zap.Error(err)) 120 return 121 } 122 123 for _, b := range brokers { 124 m.brokers[b.ID] = struct{}{} 125 } 126 } 127 128 func (m *saramaMetricsCollector) collectProducerMetrics() { 129 namespace := m.changefeedID.Namespace 130 changefeedID := m.changefeedID.ID 131 132 compressionRatioMetric := m.registry.Get(compressionRatioMetricName) 133 if histogram, ok := compressionRatioMetric.(metrics.Histogram); ok { 134 compressionRatioGauge. 135 WithLabelValues(namespace, changefeedID). 136 Set(histogram.Snapshot().Mean() / 100) 137 } 138 139 recordsPerRequestMetric := m.registry.Get(recordsPerRequestMetricName) 140 if histogram, ok := recordsPerRequestMetric.(metrics.Histogram); ok { 141 recordsPerRequestGauge. 142 WithLabelValues(namespace, changefeedID). 143 Set(histogram.Snapshot().Mean()) 144 } 145 } 146 147 func (m *saramaMetricsCollector) collectBrokerMetrics() { 148 namespace := m.changefeedID.Namespace 149 changefeedID := m.changefeedID.ID 150 for id := range m.brokers { 151 brokerID := strconv.Itoa(int(id)) 152 153 outgoingByteRateMetric := m.registry.Get( 154 getBrokerMetricName(outgoingByteRateMetricNamePrefix, brokerID)) 155 if meter, ok := outgoingByteRateMetric.(metrics.Meter); ok { 156 OutgoingByteRateGauge. 157 WithLabelValues(namespace, changefeedID, brokerID). 158 Set(meter.Snapshot().RateMean()) 159 } 160 161 requestRateMetric := m.registry.Get( 162 getBrokerMetricName(requestRateMetricNamePrefix, brokerID)) 163 if meter, ok := requestRateMetric.(metrics.Meter); ok { 164 RequestRateGauge. 165 WithLabelValues(namespace, changefeedID, brokerID). 166 Set(meter.Snapshot().RateMean()) 167 } 168 169 requestLatencyMetric := m.registry.Get( 170 getBrokerMetricName(requestLatencyInMsMetricNamePrefix, brokerID)) 171 if histogram, ok := requestLatencyMetric.(metrics.Histogram); ok { 172 RequestLatencyGauge. 173 WithLabelValues(namespace, changefeedID, brokerID). 174 Set(histogram.Snapshot().Mean() / 1000) // convert millisecond to second. 175 } 176 177 requestsInFlightMetric := m.registry.Get(getBrokerMetricName( 178 requestsInFlightMetricNamePrefix, brokerID)) 179 if counter, ok := requestsInFlightMetric.(metrics.Counter); ok { 180 requestsInFlightGauge. 181 WithLabelValues(namespace, changefeedID, brokerID). 182 Set(float64(counter.Snapshot().Count())) 183 } 184 185 responseRateMetric := m.registry.Get(getBrokerMetricName( 186 responseRateMetricNamePrefix, brokerID)) 187 if meter, ok := responseRateMetric.(metrics.Meter); ok { 188 responseRateGauge. 189 WithLabelValues(namespace, changefeedID, brokerID). 190 Set(meter.Snapshot().RateMean()) 191 } 192 } 193 } 194 195 func getBrokerMetricName(prefix, brokerID string) string { 196 return prefix + brokerID 197 } 198 199 func (m *saramaMetricsCollector) cleanupProducerMetrics() { 200 compressionRatioGauge. 201 DeleteLabelValues(m.changefeedID.Namespace, m.changefeedID.ID) 202 recordsPerRequestGauge. 203 DeleteLabelValues(m.changefeedID.Namespace, m.changefeedID.ID) 204 } 205 206 func (m *saramaMetricsCollector) cleanupBrokerMetrics() { 207 namespace := m.changefeedID.Namespace 208 changefeedID := m.changefeedID.ID 209 for id := range m.brokers { 210 brokerID := strconv.Itoa(int(id)) 211 OutgoingByteRateGauge. 212 DeleteLabelValues(namespace, changefeedID, brokerID) 213 RequestRateGauge. 214 DeleteLabelValues(namespace, changefeedID, brokerID) 215 RequestLatencyGauge. 216 DeleteLabelValues(namespace, changefeedID, brokerID) 217 requestsInFlightGauge. 218 DeleteLabelValues(namespace, changefeedID, brokerID) 219 responseRateGauge. 220 DeleteLabelValues(namespace, changefeedID, brokerID) 221 222 } 223 } 224 225 func (m *saramaMetricsCollector) cleanupMetrics() { 226 m.cleanupProducerMetrics() 227 m.cleanupBrokerMetrics() 228 }