forked from apache/pulsar
/
AbstractMetrics.java
248 lines (210 loc) · 8.74 KB
/
AbstractMetrics.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you 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, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.pulsar.broker.stats.metrics;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import org.apache.bookkeeper.mledger.ManagedLedgerFactoryMXBean;
import org.apache.bookkeeper.mledger.impl.ManagedLedgerFactoryImpl;
import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
import org.apache.bookkeeper.mledger.impl.ManagedLedgerMBeanImpl;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.common.policies.data.TopicStats;
import org.apache.pulsar.common.stats.Metrics;
abstract class AbstractMetrics {
protected static final String METRICS_VERSION_SUFFIX = "v2";
protected static final Pattern V2_LEDGER_NAME_PATTERN = Pattern.compile("^(([^/]+)/([^/]+)/([^/]+))/(.*)$");
protected static final double[] ENTRY_LATENCY_BUCKETS_MS =
new double[ManagedLedgerMBeanImpl.ENTRY_LATENCY_BUCKETS_USEC.length];
static {
// Convert buckets boundaries from usec to millis
for (int i = 0; i < ManagedLedgerMBeanImpl.ENTRY_LATENCY_BUCKETS_USEC.length; i++) {
ENTRY_LATENCY_BUCKETS_MS[i] = ManagedLedgerMBeanImpl.ENTRY_LATENCY_BUCKETS_USEC[i] / 1000.0;
}
}
protected static final double[] ENTRY_SIZE_BUCKETS_BYTES =
new double[ManagedLedgerMBeanImpl.ENTRY_SIZE_BUCKETS_BYTES.length];
static {
// Convert buckets boundaries from usec to millis
for (int i = 0; i < ManagedLedgerMBeanImpl.ENTRY_SIZE_BUCKETS_BYTES.length; i++) {
ENTRY_SIZE_BUCKETS_BYTES[i] = ManagedLedgerMBeanImpl.ENTRY_SIZE_BUCKETS_BYTES[i];
}
}
// simple abstract for the buckets, their boundaries and pre-calculated keys
// pre-calculating the keys avoids a lot of object allocations during metric collection
static class Buckets {
private final double[] boundaries;
private final String[] bucketKeys;
Buckets(String metricKey, double[] boundaries) {
this.boundaries = boundaries;
this.bucketKeys = generateBucketKeys(metricKey, boundaries);
}
private static String[] generateBucketKeys(String mkey, double[] boundaries) {
String[] keys = new String[boundaries.length + 1];
for (int i = 0; i < boundaries.length + 1; i++) {
String bucketKey;
double value;
// example of key : "<metric_key>_0.0_0.5"
if (i == 0 && boundaries.length > 0) {
bucketKey = String.format("%s_0.0_%1.1f", mkey, boundaries[i]);
} else if (i < boundaries.length) {
bucketKey = String.format("%s_%1.1f_%1.1f", mkey, boundaries[i - 1], boundaries[i]);
} else {
bucketKey = String.format("%s_OVERFLOW", mkey);
}
keys[i] = bucketKey;
}
return keys;
}
public void populateBucketEntries(Map<String, Double> map, long[] bucketValues, int period) {
// bucket values should be one more that the boundaries to have the last element as OVERFLOW
if (bucketValues != null && bucketValues.length != boundaries.length + 1) {
throw new RuntimeException("Bucket boundary and value array length mismatch");
}
for (int i = 0; i < boundaries.length + 1; i++) {
double value = (bucketValues == null) ? 0.0D : ((double) bucketValues[i] / (period > 0 ? period : 1));
map.compute(bucketKeys[i], (key, currentValue) -> (currentValue == null ? 0.0d : currentValue) + value);
}
}
}
protected final PulsarService pulsar;
abstract List<Metrics> generate();
AbstractMetrics(PulsarService pulsar) {
this.pulsar = pulsar;
}
/**
* Creates a metrics with empty immutable dimension.
* <p>
* Use this for metrics that doesn't need any dimension - i.e global metrics
*
* @return
*/
protected Metrics createMetrics() {
return createMetrics(new HashMap<String, String>());
}
protected Metrics createMetrics(Map<String, String> dimensionMap) {
// create with current version
return Metrics.create(dimensionMap);
}
/**
* Returns the managed ledger cache statistics from ML factory.
*
* @return
*/
protected ManagedLedgerFactoryMXBean getManagedLedgerCacheStats() {
return ((ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory()).getCacheStats();
}
/**
* Returns managed ledgers map from ML factory.
*
* @return
*/
protected Map<String, ManagedLedgerImpl> getManagedLedgers() {
return ((ManagedLedgerFactoryImpl) pulsar.getManagedLedgerFactory()).getManagedLedgers();
}
protected String getLocalClusterName() {
return pulsar.getConfiguration().getClusterName();
}
protected double average(List<Double> values) {
double average = 0;
if (values.size() > 0) {
double sum = 0;
for (Double value : values) {
sum += value;
}
average = sum / values.size();
}
return average;
}
protected double sum(List<Double> values) {
double sum = 0;
if (values.size() > 0) {
for (Double value : values) {
sum += value;
}
}
return sum;
}
protected String parseNamespaceFromLedgerName(String ledgerName) {
Matcher m = V2_LEDGER_NAME_PATTERN.matcher(ledgerName);
if (m.matches()) {
return m.group(1);
} else {
throw new RuntimeException("Failed to parse the namespace from ledger name : " + ledgerName);
}
}
/**
* Creates a dimension key for metrics.
*
* @param namespace Namespace of metric
* @return
*/
protected Metrics createMetricsByDimension(String namespace) {
Map<String, String> dimensionMap = Maps.newHashMap();
dimensionMap.put("namespace", namespace);
return createMetrics(dimensionMap);
}
/**
* Creates a dimension key for replication metrics.
*
* @param namespace
* @param fromClusterName
* @param toClusterName
* @return
*/
protected Metrics createMetricsByDimension(String namespace, String fromClusterName, String toClusterName) {
Map<String, String> dimensionMap = Maps.newHashMap();
dimensionMap.put("namespace", namespace);
dimensionMap.put("from_cluster", fromClusterName);
dimensionMap.put("to_cluster", toClusterName);
return createMetrics(dimensionMap);
}
protected void populateAggregationMap(Map<String, List<Double>> map, String mkey, double value) {
map.computeIfAbsent(mkey, __ -> Lists.newArrayList()).add(value);
}
protected void populateAggregationMapWithSum(Map<String, Double> map, String mkey, double value) {
Double val = map.getOrDefault(mkey, 0.0);
map.put(mkey, val + value);
}
protected void populateMaxMap(Map<String, Long> map, String mkey, long value) {
Long existingValue = map.get(mkey);
if (existingValue == null || value > existingValue) {
map.put(mkey, value);
}
}
/**
* Helper to manage populating topics map.
*
* @param ledgersByDimensionMap
* @param metrics
* @param ledger
*/
protected void populateDimensionMap(Map<Metrics, List<ManagedLedgerImpl>> ledgersByDimensionMap, Metrics metrics,
ManagedLedgerImpl ledger) {
ledgersByDimensionMap.computeIfAbsent(metrics, __ -> Lists.newArrayList()).add(ledger);
}
protected void populateDimensionMap(Map<Metrics, List<TopicStats>> topicsStatsByDimensionMap,
Metrics metrics, TopicStats destStats) {
topicsStatsByDimensionMap.computeIfAbsent(metrics, __ -> Lists.newArrayList()).add(destStats);
}
}