Skip to content

Commit a6ec31f

Browse files
authored
Merge pull request #611 from Aiven-Open/jeqo/docs-metrics-core
refactor: expose core metric names for doc generation
2 parents 4821be9 + 7c09a41 commit a6ec31f

File tree

9 files changed

+775
-200
lines changed

9 files changed

+775
-200
lines changed

core/src/main/java/io/aiven/kafka/tieredstorage/fetch/cache/ChunkCache.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -47,8 +47,8 @@
4747
import com.github.benmanes.caffeine.cache.Weigher;
4848

4949
public abstract class ChunkCache<T> implements ChunkManager, Configurable {
50-
private static final String METRIC_GROUP = "chunk-cache-metrics";
51-
private static final String THREAD_POOL_METRIC_GROUP = "chunk-cache-thread-pool-metrics";
50+
public static final String METRIC_GROUP = "chunk-cache-metrics";
51+
public static final String THREAD_POOL_METRIC_GROUP = "chunk-cache-thread-pool-metrics";
5252

5353
private final ChunkManager chunkManager;
5454
private ExecutorService executor;

core/src/main/java/io/aiven/kafka/tieredstorage/fetch/index/MemorySegmentIndexesCache.java

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -49,9 +49,10 @@
4949
public class MemorySegmentIndexesCache implements SegmentIndexesCache {
5050
private static final Logger log = LoggerFactory.getLogger(MemorySegmentIndexesCache.class);
5151

52+
public static final String METRIC_GROUP = "segment-indexes-cache-metrics";
53+
public static final String THREAD_POOL_METRIC_GROUP = "segment-indexes-cache-thread-pool-metrics";
54+
5255
private static final long DEFAULT_MAX_SIZE_BYTES = 10 * 1024 * 1024;
53-
private static final String METRIC_GROUP = "segment-indexes-cache-metrics";
54-
private static final String THREAD_POOL_METRIC_GROUP = "segment-indexes-cache-thread-pool-metrics";
5556

5657
private final CaffeineStatsCounter statsCounter = new CaffeineStatsCounter(METRIC_GROUP);
5758

core/src/main/java/io/aiven/kafka/tieredstorage/fetch/manifest/MemorySegmentManifestCache.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -46,8 +46,8 @@
4646

4747
public class MemorySegmentManifestCache implements SegmentManifestCache {
4848
private static final Logger log = LoggerFactory.getLogger(MemorySegmentManifestCache.class);
49-
private static final String METRIC_GROUP = "segment-manifest-cache-metrics";
50-
private static final String THREAD_POOL_METRIC_GROUP = "segment-manifest-cache-thread-pool-metrics";
49+
public static final String METRIC_GROUP = "segment-manifest-cache-metrics";
50+
public static final String THREAD_POOL_METRIC_GROUP = "segment-manifest-cache-thread-pool-metrics";
5151
private static final long DEFAULT_MAX_SIZE = 1000L;
5252
private static final long DEFAULT_RETENTION_MS = 3_600_000;
5353

Lines changed: 138 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,138 @@
1+
/*
2+
* Copyright 2024 Aiven Oy
3+
*
4+
* Licensed under the Apache License, Version 2.0 (the "License");
5+
* you may not use this file except in compliance with the License.
6+
* You may obtain a copy of the License at
7+
*
8+
* http://www.apache.org/licenses/LICENSE-2.0
9+
*
10+
* Unless required by applicable law or agreed to in writing, software
11+
* distributed under the License is distributed on an "AS IS" BASIS,
12+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13+
* See the License for the specific language governing permissions and
14+
* limitations under the License.
15+
*/
16+
17+
package io.aiven.kafka.tieredstorage.metrics;
18+
19+
import java.util.List;
20+
21+
import org.apache.kafka.common.MetricNameTemplate;
22+
23+
public class CaffeineMetricsRegistry {
24+
public static final String METRIC_CONTEXT = "aiven.kafka.server.tieredstorage.cache";
25+
26+
static final String CACHE_HITS = "cache-hits";
27+
static final String CACHE_HITS_TOTAL = CACHE_HITS + "-total";
28+
static final String CACHE_MISSES = "cache-misses";
29+
static final String CACHE_MISSES_TOTAL = CACHE_MISSES + "-total";
30+
static final String CACHE_LOAD = "cache-load";
31+
static final String CACHE_LOAD_SUCCESS = CACHE_LOAD + "-success";
32+
static final String CACHE_LOAD_SUCCESS_TOTAL = CACHE_LOAD_SUCCESS + "-total";
33+
static final String CACHE_LOAD_SUCCESS_TIME = CACHE_LOAD_SUCCESS + "-time";
34+
static final String CACHE_LOAD_SUCCESS_TIME_TOTAL = CACHE_LOAD_SUCCESS_TIME + "-total";
35+
static final String CACHE_LOAD_FAILURE = CACHE_LOAD + "-failure";
36+
static final String CACHE_LOAD_FAILURE_TOTAL = CACHE_LOAD_FAILURE + "-total";
37+
static final String CACHE_LOAD_FAILURE_TIME = CACHE_LOAD_FAILURE + "-time";
38+
static final String CACHE_LOAD_FAILURE_TIME_TOTAL = CACHE_LOAD_FAILURE_TIME + "-total";
39+
40+
static final String CACHE_EVICTION = "cache-eviction";
41+
static final String CACHE_EVICTION_TOTAL = CACHE_EVICTION + "-total";
42+
static final String CACHE_EVICTION_WEIGHT = CACHE_EVICTION + "-weight";
43+
static final String CACHE_EVICTION_WEIGHT_TOTAL = CACHE_EVICTION_WEIGHT + "-total";
44+
45+
static final String CACHE_SIZE = "cache-size";
46+
static final String CACHE_SIZE_TOTAL = CACHE_SIZE + "-total";
47+
48+
final String groupName;
49+
50+
final MetricNameTemplate cacheHitsMetricName;
51+
final MetricNameTemplate cacheMissesMetricName;
52+
final MetricNameTemplate cacheLoadSuccessMetricName;
53+
final MetricNameTemplate cacheLoadSuccessTimeMetricName;
54+
final MetricNameTemplate cacheLoadFailureMetricName;
55+
final MetricNameTemplate cacheLoadFailureTimeMetricName;
56+
final MetricNameTemplate cacheEvictionMetricName;
57+
final MetricNameTemplate cacheEvictionByCauseMetricName;
58+
final MetricNameTemplate cacheEvictionWeightMetricName;
59+
final MetricNameTemplate cacheEvictionWeightByCauseMetricName;
60+
final MetricNameTemplate cacheSizeTotalMetricName;
61+
62+
public CaffeineMetricsRegistry(final String groupName) {
63+
this.groupName = groupName;
64+
cacheHitsMetricName = new MetricNameTemplate(
65+
CACHE_HITS_TOTAL,
66+
groupName,
67+
"Cache hits"
68+
);
69+
cacheMissesMetricName = new MetricNameTemplate(
70+
CACHE_MISSES_TOTAL,
71+
groupName,
72+
"Cache misses"
73+
);
74+
cacheLoadSuccessMetricName = new MetricNameTemplate(
75+
CACHE_LOAD_SUCCESS_TOTAL,
76+
groupName,
77+
"Successful load of a new entry"
78+
);
79+
cacheLoadSuccessTimeMetricName = new MetricNameTemplate(
80+
CACHE_LOAD_SUCCESS_TIME_TOTAL,
81+
groupName,
82+
"Time to load a new entry"
83+
);
84+
cacheLoadFailureMetricName = new MetricNameTemplate(
85+
CACHE_LOAD_FAILURE_TOTAL,
86+
groupName,
87+
"Failures to load a new entry"
88+
);
89+
cacheLoadFailureTimeMetricName = new MetricNameTemplate(
90+
CACHE_LOAD_FAILURE_TIME_TOTAL,
91+
groupName,
92+
"Time when failing to load a new entry"
93+
);
94+
cacheEvictionMetricName = new MetricNameTemplate(
95+
CACHE_EVICTION_TOTAL,
96+
groupName,
97+
"Eviction of an entry from the cache"
98+
);
99+
cacheEvictionByCauseMetricName = new MetricNameTemplate(
100+
CACHE_EVICTION_TOTAL,
101+
groupName,
102+
"Eviction of an entry from the cache tagged by cause",
103+
"cause"
104+
);
105+
cacheEvictionWeightMetricName = new MetricNameTemplate(
106+
CACHE_EVICTION_WEIGHT_TOTAL,
107+
groupName,
108+
"Weight of evicted entry"
109+
);
110+
cacheEvictionWeightByCauseMetricName = new MetricNameTemplate(
111+
CACHE_EVICTION_WEIGHT_TOTAL,
112+
groupName,
113+
"Weight of evicted entry tagged by cause",
114+
"cause"
115+
);
116+
cacheSizeTotalMetricName = new MetricNameTemplate(
117+
CACHE_SIZE_TOTAL,
118+
groupName,
119+
"Estimated number of entries in the cache"
120+
);
121+
}
122+
123+
public List<MetricNameTemplate> all() {
124+
return List.of(
125+
cacheHitsMetricName,
126+
cacheMissesMetricName,
127+
cacheLoadSuccessMetricName,
128+
cacheLoadSuccessTimeMetricName,
129+
cacheLoadFailureMetricName,
130+
cacheLoadFailureTimeMetricName,
131+
cacheEvictionMetricName,
132+
cacheEvictionByCauseMetricName,
133+
cacheEvictionWeightMetricName,
134+
cacheEvictionWeightByCauseMetricName,
135+
cacheSizeTotalMetricName
136+
);
137+
}
138+
}

core/src/main/java/io/aiven/kafka/tieredstorage/metrics/CaffeineStatsCounter.java

Lines changed: 77 additions & 50 deletions
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,17 @@
3434
import com.github.benmanes.caffeine.cache.stats.CacheStats;
3535
import com.github.benmanes.caffeine.cache.stats.StatsCounter;
3636

37+
import static io.aiven.kafka.tieredstorage.metrics.CaffeineMetricsRegistry.CACHE_EVICTION;
38+
import static io.aiven.kafka.tieredstorage.metrics.CaffeineMetricsRegistry.CACHE_EVICTION_WEIGHT;
39+
import static io.aiven.kafka.tieredstorage.metrics.CaffeineMetricsRegistry.CACHE_HITS;
40+
import static io.aiven.kafka.tieredstorage.metrics.CaffeineMetricsRegistry.CACHE_LOAD_FAILURE;
41+
import static io.aiven.kafka.tieredstorage.metrics.CaffeineMetricsRegistry.CACHE_LOAD_FAILURE_TIME;
42+
import static io.aiven.kafka.tieredstorage.metrics.CaffeineMetricsRegistry.CACHE_LOAD_SUCCESS;
43+
import static io.aiven.kafka.tieredstorage.metrics.CaffeineMetricsRegistry.CACHE_LOAD_SUCCESS_TIME;
44+
import static io.aiven.kafka.tieredstorage.metrics.CaffeineMetricsRegistry.CACHE_MISSES;
45+
import static io.aiven.kafka.tieredstorage.metrics.CaffeineMetricsRegistry.CACHE_SIZE;
46+
import static io.aiven.kafka.tieredstorage.metrics.CaffeineMetricsRegistry.METRIC_CONTEXT;
47+
3748
/**
3849
* Records cache metrics managed by Caffeine {@code Cache#stats}.
3950
*
@@ -43,29 +54,6 @@
4354
* <a href="https://github.com/micrometer-metrics/micrometer/blob/main/micrometer-core/src/main/java/io/micrometer/core/instrument/binder/cache/CaffeineStatsCounter.java">Micrometer</a>
4455
*/
4556
public class CaffeineStatsCounter implements StatsCounter {
46-
47-
static final String CACHE_HITS = "cache-hits";
48-
static final String CACHE_HITS_TOTAL = CACHE_HITS + "-total";
49-
static final String CACHE_MISSES = "cache-misses";
50-
static final String CACHE_MISSES_TOTAL = CACHE_MISSES + "-total";
51-
static final String CACHE_LOAD = "cache-load";
52-
static final String CACHE_LOAD_SUCCESS = CACHE_LOAD + "-success";
53-
static final String CACHE_LOAD_SUCCESS_TOTAL = CACHE_LOAD_SUCCESS + "-total";
54-
static final String CACHE_LOAD_SUCCESS_TIME = CACHE_LOAD_SUCCESS + "-time";
55-
static final String CACHE_LOAD_SUCCESS_TIME_TOTAL = CACHE_LOAD_SUCCESS_TIME + "-total";
56-
static final String CACHE_LOAD_FAILURE = CACHE_LOAD + "-failure";
57-
static final String CACHE_LOAD_FAILURE_TOTAL = CACHE_LOAD_FAILURE + "-total";
58-
static final String CACHE_LOAD_FAILURE_TIME = CACHE_LOAD_FAILURE + "-time";
59-
static final String CACHE_LOAD_FAILURE_TIME_TOTAL = CACHE_LOAD_FAILURE_TIME + "-total";
60-
61-
static final String CACHE_EVICTION = "cache-eviction";
62-
static final String CACHE_EVICTION_TOTAL = CACHE_EVICTION + "-total";
63-
static final String CACHE_EVICTION_WEIGHT = CACHE_EVICTION + "-weight";
64-
static final String CACHE_EVICTION_WEIGHT_TOTAL = CACHE_EVICTION_WEIGHT + "-total";
65-
66-
static final String CACHE_SIZE = "cache-size";
67-
static final String CACHE_SIZE_TOTAL = CACHE_SIZE + "-total";
68-
6957
private final org.apache.kafka.common.metrics.Metrics metrics;
7058

7159
private final LongAdder cacheHitCount;
@@ -78,7 +66,7 @@ public class CaffeineStatsCounter implements StatsCounter {
7866
private final LongAdder cacheEvictionWeightTotal;
7967
private final ConcurrentHashMap<RemovalCause, LongAdder> cacheEvictionCountByCause;
8068
private final ConcurrentHashMap<RemovalCause, LongAdder> cacheEvictionWeightByCause;
81-
private final String groupName;
69+
private final CaffeineMetricsRegistry metricsRegistry;
8270

8371
public CaffeineStatsCounter(final String groupName) {
8472
cacheHitCount = new LongAdder();
@@ -90,8 +78,6 @@ public CaffeineStatsCounter(final String groupName) {
9078
cacheEvictionCountTotal = new LongAdder();
9179
cacheEvictionWeightTotal = new LongAdder();
9280

93-
this.groupName = groupName;
94-
9581
cacheEvictionCountByCause = new ConcurrentHashMap<>();
9682
Arrays.stream(RemovalCause.values()).forEach(cause -> cacheEvictionCountByCause.put(cause, new LongAdder()));
9783

@@ -102,42 +88,84 @@ public CaffeineStatsCounter(final String groupName) {
10288

10389
metrics = new org.apache.kafka.common.metrics.Metrics(
10490
new MetricConfig(), List.of(reporter), Time.SYSTEM,
105-
new KafkaMetricsContext("aiven.kafka.server.tieredstorage.cache")
91+
new KafkaMetricsContext(METRIC_CONTEXT)
10692
);
10793

108-
initSensor(CACHE_HITS, CACHE_HITS_TOTAL, cacheHitCount);
109-
initSensor(CACHE_MISSES, CACHE_MISSES_TOTAL, cacheMissCount);
110-
initSensor(CACHE_LOAD_SUCCESS, CACHE_LOAD_SUCCESS_TOTAL, cacheLoadSuccessCount);
111-
initSensor(CACHE_LOAD_SUCCESS_TIME, CACHE_LOAD_SUCCESS_TIME_TOTAL, cacheLoadSuccessTimeTotal);
112-
initSensor(CACHE_LOAD_FAILURE, CACHE_LOAD_FAILURE_TOTAL, cacheLoadFailureCount);
113-
initSensor(CACHE_LOAD_FAILURE_TIME, CACHE_LOAD_FAILURE_TIME_TOTAL, cacheLoadFailureTimeTotal);
114-
initSensor(CACHE_EVICTION, CACHE_EVICTION_TOTAL, cacheEvictionCountTotal);
94+
metricsRegistry = new CaffeineMetricsRegistry(groupName);
95+
initSensor(
96+
metricsRegistry.cacheHitsMetricName,
97+
CACHE_HITS,
98+
cacheHitCount
99+
);
100+
initSensor(
101+
metricsRegistry.cacheMissesMetricName,
102+
CACHE_MISSES,
103+
cacheMissCount
104+
);
105+
initSensor(
106+
metricsRegistry.cacheLoadSuccessMetricName,
107+
CACHE_LOAD_SUCCESS,
108+
cacheLoadSuccessCount
109+
);
110+
initSensor(
111+
metricsRegistry.cacheLoadSuccessTimeMetricName,
112+
CACHE_LOAD_SUCCESS_TIME,
113+
cacheLoadSuccessTimeTotal
114+
);
115+
initSensor(
116+
metricsRegistry.cacheLoadFailureMetricName,
117+
CACHE_LOAD_FAILURE,
118+
cacheLoadFailureCount
119+
);
120+
initSensor(
121+
metricsRegistry.cacheLoadFailureTimeMetricName,
122+
CACHE_LOAD_FAILURE_TIME,
123+
cacheLoadFailureTimeTotal
124+
);
125+
initSensor(
126+
metricsRegistry.cacheEvictionMetricName,
127+
CACHE_EVICTION,
128+
cacheEvictionCountTotal
129+
);
115130
Arrays.stream(RemovalCause.values()).forEach(cause ->
116-
initSensor("cause." + cause.name() + "." + CACHE_EVICTION, CACHE_EVICTION_TOTAL,
117-
cacheEvictionCountByCause.get(cause), () -> Map.of("cause", cause.name()), "cause")
131+
initSensor(
132+
metricsRegistry.cacheEvictionByCauseMetricName,
133+
"cause." + cause.name() + "." + CACHE_EVICTION,
134+
cacheEvictionCountByCause.get(cause),
135+
() -> Map.of("cause", cause.name())
136+
)
118137
);
119138

120-
initSensor(CACHE_EVICTION_WEIGHT, CACHE_EVICTION_WEIGHT_TOTAL, cacheEvictionWeightTotal);
139+
initSensor(
140+
metricsRegistry.cacheEvictionWeightMetricName,
141+
CACHE_EVICTION_WEIGHT,
142+
cacheEvictionWeightTotal
143+
);
121144

122145
Arrays.stream(RemovalCause.values()).forEach(cause ->
123-
initSensor("cause." + cause.name() + "." + CACHE_EVICTION, CACHE_EVICTION_WEIGHT_TOTAL,
124-
cacheEvictionWeightByCause.get(cause), () -> Map.of("cause", cause.name()), "cause")
146+
initSensor(
147+
metricsRegistry.cacheEvictionWeightByCauseMetricName,
148+
"cause." + cause.name() + "." + CACHE_EVICTION,
149+
cacheEvictionWeightByCause.get(cause),
150+
() -> Map.of("cause", cause.name())
151+
)
125152
);
126153
}
127154

128-
private void initSensor(final String sensorName,
129-
final String metricName,
130-
final LongAdder value,
131-
final Supplier<Map<String, String>> tagsSupplier,
132-
final String... tagNames) {
133-
final var name = new MetricNameTemplate(metricName, groupName, "", tagNames);
155+
private void initSensor(
156+
final MetricNameTemplate metricNameTemplate,
157+
final String sensorName,
158+
final LongAdder value,
159+
final Supplier<Map<String, String>> tagsSupplier
160+
) {
134161
new SensorProvider(metrics, sensorName, tagsSupplier)
135-
.with(name, new MeasurableValue(value::sum))
162+
.with(metricNameTemplate, new MeasurableValue(value::sum))
136163
.get();
137164
}
138165

139-
private void initSensor(final String sensorName, final String metricName, final LongAdder value) {
140-
initSensor(sensorName, metricName, value, Collections::emptyMap);
166+
private void initSensor(final MetricNameTemplate metricNameTemplate, final String sensorName,
167+
final LongAdder value) {
168+
initSensor(metricNameTemplate, sensorName, value, Collections::emptyMap);
141169
}
142170

143171
@Override
@@ -193,9 +221,8 @@ public void recordHit() {
193221
* @param sizeSupplier operation from cache to provide cache size value
194222
*/
195223
public void registerSizeMetric(final Supplier<Long> sizeSupplier) {
196-
final var name = new MetricNameTemplate(CACHE_SIZE_TOTAL, groupName, "");
197224
new SensorProvider(metrics, CACHE_SIZE)
198-
.with(name, new MeasurableValue(sizeSupplier))
225+
.with(metricsRegistry.cacheSizeTotalMetricName, new MeasurableValue(sizeSupplier))
199226
.get();
200227
}
201228

core/src/main/java/io/aiven/kafka/tieredstorage/metrics/Metrics.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,7 @@
3535
import org.slf4j.Logger;
3636
import org.slf4j.LoggerFactory;
3737

38+
import static io.aiven.kafka.tieredstorage.metrics.MetricsRegistry.METRIC_CONTEXT;
3839
import static io.aiven.kafka.tieredstorage.metrics.MetricsRegistry.OBJECT_UPLOAD;
3940
import static io.aiven.kafka.tieredstorage.metrics.MetricsRegistry.OBJECT_UPLOAD_BYTES;
4041
import static io.aiven.kafka.tieredstorage.metrics.MetricsRegistry.SEGMENT_COPY_TIME;
@@ -69,7 +70,7 @@ public Metrics(final Time time, final MetricConfig metricConfig) {
6970
metricConfig,
7071
List.of(reporter),
7172
time,
72-
new KafkaMetricsContext("aiven.kafka.server.tieredstorage")
73+
new KafkaMetricsContext(METRIC_CONTEXT)
7374
);
7475

7576
metricsRegistry = new MetricsRegistry();

0 commit comments

Comments
 (0)