Skip to content

Commit d553455

Browse files
authored
Add metrics around the file extensions we request when populating the cache (#123134)
This adds the file extentions for the blobs we request when populating the cache. The possible values for lucene extensions are around 50 and we use a special "other" category for everything else, as a fallback.
1 parent 50aaa1c commit d553455

File tree

2 files changed

+25
-9
lines changed

2 files changed

+25
-9
lines changed

x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/BlobCacheMetrics.java

Lines changed: 10 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -9,6 +9,7 @@
99

1010
import org.apache.logging.log4j.LogManager;
1111
import org.apache.logging.log4j.Logger;
12+
import org.elasticsearch.index.store.LuceneFilesExtensions;
1213
import org.elasticsearch.telemetry.TelemetryProvider;
1314
import org.elasticsearch.telemetry.metric.DoubleHistogram;
1415
import org.elasticsearch.telemetry.metric.LongCounter;
@@ -24,8 +25,8 @@ public class BlobCacheMetrics {
2425
private static final double BYTES_PER_NANOSECONDS_TO_MEBIBYTES_PER_SECOND = 1e9D / (1 << 20);
2526
public static final String CACHE_POPULATION_REASON_ATTRIBUTE_KEY = "reason";
2627
public static final String CACHE_POPULATION_SOURCE_ATTRIBUTE_KEY = "source";
27-
public static final String SHARD_ID_ATTRIBUTE_KEY = "shard_id";
28-
public static final String INDEX_ATTRIBUTE_KEY = "index_name";
28+
public static final String LUCENE_FILE_EXTENSION_ATTRIBUTE_KEY = "file_extension";
29+
public static final String NON_LUCENE_EXTENSION_TO_RECORD = "other";
2930

3031
private final LongCounter cacheMissCounter;
3132
private final LongCounter evictedCountNonZeroFrequency;
@@ -113,22 +114,28 @@ public LongHistogram getCacheMissLoadTimes() {
113114
/**
114115
* Record the various cache population metrics after a chunk is copied to the cache
115116
*
117+
* @param blobName The file that was requested and triggered the cache population.
116118
* @param bytesCopied The number of bytes copied
117119
* @param copyTimeNanos The time taken to copy the bytes in nanoseconds
118120
* @param cachePopulationReason The reason for the cache being populated
119121
* @param cachePopulationSource The source from which the data is being loaded
120122
*/
121123
public void recordCachePopulationMetrics(
124+
String blobName,
122125
int bytesCopied,
123126
long copyTimeNanos,
124127
CachePopulationReason cachePopulationReason,
125128
CachePopulationSource cachePopulationSource
126129
) {
130+
LuceneFilesExtensions luceneFilesExtensions = LuceneFilesExtensions.fromFile(blobName);
131+
String blobFileExtension = luceneFilesExtensions != null ? luceneFilesExtensions.getExtension() : NON_LUCENE_EXTENSION_TO_RECORD;
127132
Map<String, Object> metricAttributes = Map.of(
128133
CACHE_POPULATION_REASON_ATTRIBUTE_KEY,
129134
cachePopulationReason.name(),
130135
CACHE_POPULATION_SOURCE_ATTRIBUTE_KEY,
131-
cachePopulationSource.name()
136+
cachePopulationSource.name(),
137+
LUCENE_FILE_EXTENSION_ATTRIBUTE_KEY,
138+
blobFileExtension
132139
);
133140
assert bytesCopied > 0 : "We shouldn't be recording zero-sized copies";
134141
cachePopulationBytes.incrementBy(bytesCopied, metricAttributes);

x-pack/plugin/blob-cache/src/test/java/org/elasticsearch/blobcache/BlobCacheMetricsTests.java

Lines changed: 15 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -8,14 +8,18 @@
88
package org.elasticsearch.blobcache;
99

1010
import org.elasticsearch.common.unit.ByteSizeValue;
11+
import org.elasticsearch.index.store.LuceneFilesExtensions;
1112
import org.elasticsearch.telemetry.InstrumentType;
1213
import org.elasticsearch.telemetry.Measurement;
1314
import org.elasticsearch.telemetry.RecordingMeterRegistry;
1415
import org.elasticsearch.test.ESTestCase;
1516
import org.junit.Before;
1617

18+
import java.util.Arrays;
1719
import java.util.concurrent.TimeUnit;
1820

21+
import static org.hamcrest.Matchers.is;
22+
1923
public class BlobCacheMetricsTests extends ESTestCase {
2024

2125
private RecordingMeterRegistry recordingMeterRegistry;
@@ -32,7 +36,10 @@ public void testRecordCachePopulationMetricsRecordsThroughput() {
3236
int secondsTaken = randomIntBetween(1, 5);
3337
BlobCacheMetrics.CachePopulationReason cachePopulationReason = randomFrom(BlobCacheMetrics.CachePopulationReason.values());
3438
CachePopulationSource cachePopulationSource = randomFrom(CachePopulationSource.values());
39+
String fileExtension = randomFrom(Arrays.stream(LuceneFilesExtensions.values()).map(LuceneFilesExtensions::getExtension).toList());
40+
String luceneBlobFile = randomAlphanumericOfLength(15) + "." + fileExtension;
3541
metrics.recordCachePopulationMetrics(
42+
luceneBlobFile,
3643
Math.toIntExact(ByteSizeValue.ofMb(mebiBytesSent).getBytes()),
3744
TimeUnit.SECONDS.toNanos(secondsTaken),
3845
cachePopulationReason,
@@ -44,29 +51,31 @@ public void testRecordCachePopulationMetricsRecordsThroughput() {
4451
.getMeasurements(InstrumentType.DOUBLE_HISTOGRAM, "es.blob_cache.population.throughput.histogram")
4552
.get(0);
4653
assertEquals(throughputMeasurement.getDouble(), (double) mebiBytesSent / secondsTaken, 0.0);
47-
assertExpectedAttributesPresent(throughputMeasurement, cachePopulationReason, cachePopulationSource);
54+
assertExpectedAttributesPresent(throughputMeasurement, cachePopulationReason, cachePopulationSource, fileExtension);
4855

4956
// bytes counter
5057
Measurement totalBytesMeasurement = recordingMeterRegistry.getRecorder()
5158
.getMeasurements(InstrumentType.LONG_COUNTER, "es.blob_cache.population.bytes.total")
5259
.get(0);
5360
assertEquals(totalBytesMeasurement.getLong(), ByteSizeValue.ofMb(mebiBytesSent).getBytes());
54-
assertExpectedAttributesPresent(totalBytesMeasurement, cachePopulationReason, cachePopulationSource);
61+
assertExpectedAttributesPresent(totalBytesMeasurement, cachePopulationReason, cachePopulationSource, fileExtension);
5562

5663
// time counter
5764
Measurement totalTimeMeasurement = recordingMeterRegistry.getRecorder()
5865
.getMeasurements(InstrumentType.LONG_COUNTER, "es.blob_cache.population.time.total")
5966
.get(0);
6067
assertEquals(totalTimeMeasurement.getLong(), TimeUnit.SECONDS.toMillis(secondsTaken));
61-
assertExpectedAttributesPresent(totalTimeMeasurement, cachePopulationReason, cachePopulationSource);
68+
assertExpectedAttributesPresent(totalTimeMeasurement, cachePopulationReason, cachePopulationSource, fileExtension);
6269
}
6370

6471
private static void assertExpectedAttributesPresent(
6572
Measurement measurement,
6673
BlobCacheMetrics.CachePopulationReason cachePopulationReason,
67-
CachePopulationSource cachePopulationSource
74+
CachePopulationSource cachePopulationSource,
75+
String fileExtension
6876
) {
69-
assertEquals(measurement.attributes().get(BlobCacheMetrics.CACHE_POPULATION_REASON_ATTRIBUTE_KEY), cachePopulationReason.name());
70-
assertEquals(measurement.attributes().get(BlobCacheMetrics.CACHE_POPULATION_SOURCE_ATTRIBUTE_KEY), cachePopulationSource.name());
77+
assertThat(measurement.attributes().get(BlobCacheMetrics.CACHE_POPULATION_REASON_ATTRIBUTE_KEY), is(cachePopulationReason.name()));
78+
assertThat(measurement.attributes().get(BlobCacheMetrics.CACHE_POPULATION_SOURCE_ATTRIBUTE_KEY), is(cachePopulationSource.name()));
79+
assertThat(measurement.attributes().get(BlobCacheMetrics.LUCENE_FILE_EXTENSION_ATTRIBUTE_KEY), is(fileExtension));
7180
}
7281
}

0 commit comments

Comments
 (0)