Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 5 additions & 0 deletions docs/changelog/135635.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,5 @@
pr: 135635
summary: Add executor name attribute to cache miss metrics
area: Search
type: enhancement
issues: []
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@

import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.index.store.LuceneFilesExtensions;
import org.elasticsearch.telemetry.TelemetryProvider;
import org.elasticsearch.telemetry.metric.DoubleHistogram;
Expand All @@ -29,7 +30,9 @@ public class BlobCacheMetrics {
public static final String CACHE_POPULATION_REASON_ATTRIBUTE_KEY = "reason";
public static final String CACHE_POPULATION_SOURCE_ATTRIBUTE_KEY = "source";
public static final String LUCENE_FILE_EXTENSION_ATTRIBUTE_KEY = "file_extension";
public static final String ES_EXECUTOR_ATTRIBUTE_KEY = "executor";
public static final String NON_LUCENE_EXTENSION_TO_RECORD = "other";
public static final String NON_ES_EXECUTOR_TO_RECORD = "other";
public static final String BLOB_CACHE_COUNT_OF_EVICTED_REGIONS_TOTAL = "es.blob_cache.count_of_evicted_regions.total";
public static final String SEARCH_ORIGIN_REMOTE_STORAGE_DOWNLOAD_TOOK_TIME = "es.blob_cache.search_origin.download_took_time.total";

Expand Down Expand Up @@ -198,13 +201,16 @@ public void recordCachePopulationMetrics(
) {
LuceneFilesExtensions luceneFilesExtensions = LuceneFilesExtensions.fromFile(fileName);
String luceneFileExt = luceneFilesExtensions != null ? luceneFilesExtensions.getExtension() : NON_LUCENE_EXTENSION_TO_RECORD;
String executorName = EsExecutors.executorName(Thread.currentThread());
Map<String, Object> metricAttributes = Map.of(
CACHE_POPULATION_REASON_ATTRIBUTE_KEY,
cachePopulationReason.name(),
CACHE_POPULATION_SOURCE_ATTRIBUTE_KEY,
cachePopulationSource.name(),
LUCENE_FILE_EXTENSION_ATTRIBUTE_KEY,
luceneFileExt
luceneFileExt,
ES_EXECUTOR_ATTRIBUTE_KEY,
executorName != null ? executorName : NON_ES_EXECUTOR_TO_RECORD
);
assert bytesCopied > 0 : "We shouldn't be recording zero-sized copies";
cachePopulationBytes.incrementBy(bytesCopied, metricAttributes);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.RelativeByteSizeValue;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.common.util.concurrent.ThrottledTaskRunner;
import org.elasticsearch.core.AbstractRefCounted;
import org.elasticsearch.core.Assertions;
Expand Down Expand Up @@ -69,7 +70,9 @@
import java.util.function.Predicate;
import java.util.stream.Collectors;

import static org.elasticsearch.blobcache.BlobCacheMetrics.ES_EXECUTOR_ATTRIBUTE_KEY;
import static org.elasticsearch.blobcache.BlobCacheMetrics.LUCENE_FILE_EXTENSION_ATTRIBUTE_KEY;
import static org.elasticsearch.blobcache.BlobCacheMetrics.NON_ES_EXECUTOR_TO_RECORD;
import static org.elasticsearch.blobcache.BlobCacheMetrics.NON_LUCENE_EXTENSION_TO_RECORD;

/**
Expand Down Expand Up @@ -1284,6 +1287,7 @@ public void fillCacheRange(
ActionListener<Void> completionListener
) throws IOException {
String blobFileExtension = getFileExtension(resourceDescription);
String executorName = EsExecutors.executorName(Thread.currentThread());
writer.fillCacheRange(
channel,
channelPos,
Expand All @@ -1295,7 +1299,15 @@ public void fillCacheRange(
var elapsedTime = TimeUnit.NANOSECONDS.toMillis(relativeTimeInNanosSupplier.getAsLong() - startTime);
blobCacheMetrics.getCacheMissLoadTimes().record(elapsedTime);
blobCacheMetrics.getCacheMissCounter()
.incrementBy(1L, Map.of(LUCENE_FILE_EXTENSION_ATTRIBUTE_KEY, blobFileExtension));
.incrementBy(
1L,
Map.of(
LUCENE_FILE_EXTENSION_ATTRIBUTE_KEY,
blobFileExtension,
ES_EXECUTOR_ATTRIBUTE_KEY,
executorName != null ? executorName : NON_ES_EXECUTOR_TO_RECORD
)
);
return null;
})
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
import java.util.concurrent.TimeUnit;
import java.util.stream.IntStream;

import static org.elasticsearch.blobcache.BlobCacheMetrics.NON_ES_EXECUTOR_TO_RECORD;
import static org.hamcrest.Matchers.is;

public class BlobCacheMetricsTests extends ESTestCase {
Expand Down Expand Up @@ -46,27 +47,28 @@ public void testRecordCachePopulationMetricsRecordsThroughput() {
cachePopulationReason,
cachePopulationSource
);
String threadName = NON_ES_EXECUTOR_TO_RECORD;

// throughput histogram
Measurement throughputMeasurement = recordingMeterRegistry.getRecorder()
.getMeasurements(InstrumentType.DOUBLE_HISTOGRAM, "es.blob_cache.population.throughput.histogram")
.get(0);
assertEquals(throughputMeasurement.getDouble(), (double) mebiBytesSent / secondsTaken, 0.0);
assertExpectedAttributesPresent(throughputMeasurement, cachePopulationReason, cachePopulationSource, fileExtension);
assertExpectedAttributesPresent(throughputMeasurement, cachePopulationReason, cachePopulationSource, fileExtension, threadName);

// bytes counter
Measurement totalBytesMeasurement = recordingMeterRegistry.getRecorder()
.getMeasurements(InstrumentType.LONG_COUNTER, "es.blob_cache.population.bytes.total")
.get(0);
assertEquals(totalBytesMeasurement.getLong(), ByteSizeValue.ofMb(mebiBytesSent).getBytes());
assertExpectedAttributesPresent(totalBytesMeasurement, cachePopulationReason, cachePopulationSource, fileExtension);
assertExpectedAttributesPresent(totalBytesMeasurement, cachePopulationReason, cachePopulationSource, fileExtension, threadName);

// time counter
Measurement totalTimeMeasurement = recordingMeterRegistry.getRecorder()
.getMeasurements(InstrumentType.LONG_COUNTER, "es.blob_cache.population.time.total")
.get(0);
assertEquals(totalTimeMeasurement.getLong(), TimeUnit.SECONDS.toMillis(secondsTaken));
assertExpectedAttributesPresent(totalTimeMeasurement, cachePopulationReason, cachePopulationSource, fileExtension);
assertExpectedAttributesPresent(totalTimeMeasurement, cachePopulationReason, cachePopulationSource, fileExtension, threadName);

// let us check for 0, avoid div by 0.
checkReadsAndMisses(0, 0, 1);
Expand Down Expand Up @@ -107,10 +109,12 @@ private static void assertExpectedAttributesPresent(
Measurement measurement,
BlobCacheMetrics.CachePopulationReason cachePopulationReason,
CachePopulationSource cachePopulationSource,
String fileExtension
String fileExtension,
String threadName
) {
assertThat(measurement.attributes().get(BlobCacheMetrics.CACHE_POPULATION_REASON_ATTRIBUTE_KEY), is(cachePopulationReason.name()));
assertThat(measurement.attributes().get(BlobCacheMetrics.CACHE_POPULATION_SOURCE_ATTRIBUTE_KEY), is(cachePopulationSource.name()));
assertThat(measurement.attributes().get(BlobCacheMetrics.LUCENE_FILE_EXTENSION_ATTRIBUTE_KEY), is(fileExtension));
assertThat(measurement.attributes().get(BlobCacheMetrics.ES_EXECUTOR_ATTRIBUTE_KEY), is(threadName));
}
}