Browse Source

Add file extension metadata to cache miss counter from SharedBlobCacheService (#134374)

Tommaso Teofili 1 month ago
parent
commit
4ec6aed8f1

+ 5 - 0
docs/changelog/134374.yaml

@@ -0,0 +1,5 @@
+pr: 134374
+summary: Add file extension metadata to cache miss counter from `SharedBlobCacheService`
+area: Search
+type: enhancement
+issues: []

+ 19 - 0
qa/evil-tests/src/test/java/org/elasticsearch/index/store/LuceneFilesExtensionsTests.java

@@ -9,10 +9,13 @@
 
 package org.elasticsearch.index.store;
 
+import org.apache.lucene.index.IndexFileNames;
 import org.elasticsearch.core.Assertions;
 import org.elasticsearch.core.SuppressForbidden;
 import org.elasticsearch.test.ESTestCase;
 
+import java.util.Locale;
+
 import static org.hamcrest.Matchers.containsString;
 
 public class LuceneFilesExtensionsTests extends ESTestCase {
@@ -21,6 +24,7 @@ public class LuceneFilesExtensionsTests extends ESTestCase {
         if (Assertions.ENABLED) {
             AssertionError e = expectThrows(AssertionError.class, () -> LuceneFilesExtensions.fromExtension("abc"));
             assertThat(e.getMessage(), containsString("unknown Lucene file extension [abc]"));
+            assertFalse(LuceneFilesExtensions.isLuceneExtension("abc"));
 
             setEsAllowUnknownLuceneFileExtensions("true");
             try {
@@ -41,4 +45,19 @@ public class LuceneFilesExtensionsTests extends ESTestCase {
             System.setProperty("es.allow_unknown_lucene_file_extensions", value);
         }
     }
+
+    public void testIsLuceneExtension() {
+        assertFalse(LuceneFilesExtensions.isLuceneExtension(null));
+        assertFalse(LuceneFilesExtensions.isLuceneExtension("bcde"));
+        String randomStringWithLuceneExtension = randomAlphanumericOfLength(10)
+            + "."
+            + LuceneFilesExtensions.values()[randomInt(LuceneFilesExtensions.values().length) - 1].getExtension();
+        String extension = IndexFileNames.getExtension(randomStringWithLuceneExtension);
+        assertTrue(extension + " should be considered a Lucene extension", LuceneFilesExtensions.isLuceneExtension(extension));
+        String upperCaseExtension = extension.toUpperCase(Locale.ROOT);
+        assertFalse(
+            upperCaseExtension + " (uppercase) should not be considered a Lucene extension",
+            LuceneFilesExtensions.isLuceneExtension(upperCaseExtension)
+        );
+    }
 }

+ 5 - 0
server/src/main/java/org/elasticsearch/index/store/LuceneFilesExtensions.java

@@ -165,6 +165,11 @@ public enum LuceneFilesExtensions {
         return null;
     }
 
+    @Nullable
+    public static boolean isLuceneExtension(String ext) {
+        return extensions.containsKey(ext);
+    }
+
     @Nullable
     public static LuceneFilesExtensions fromFile(String fileName) {
         return fromExtension(IndexFileNames.getExtension(fileName));

+ 23 - 2
x-pack/plugin/blob-cache/src/main/java/org/elasticsearch/blobcache/shared/SharedBlobCacheService.java

@@ -9,6 +9,7 @@ package org.elasticsearch.blobcache.shared;
 
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
+import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.action.ActionRunnable;
@@ -37,6 +38,7 @@ import org.elasticsearch.core.Strings;
 import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.env.Environment;
 import org.elasticsearch.env.NodeEnvironment;
+import org.elasticsearch.index.store.LuceneFilesExtensions;
 import org.elasticsearch.monitor.fs.FsProbe;
 import org.elasticsearch.node.NodeRoleSettings;
 import org.elasticsearch.threadpool.ThreadPool;
@@ -67,6 +69,9 @@ import java.util.function.LongSupplier;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
 
+import static org.elasticsearch.blobcache.BlobCacheMetrics.LUCENE_FILE_EXTENSION_ATTRIBUTE_KEY;
+import static org.elasticsearch.blobcache.BlobCacheMetrics.NON_LUCENE_EXTENSION_TO_RECORD;
+
 /**
  * A caching layer on a local node to minimize network roundtrips to the remote blob store.
  */
@@ -1254,7 +1259,8 @@ public class SharedBlobCacheService<KeyType> implements Releasable {
             final ByteRange rangeToWrite,
             final ByteRange rangeToRead,
             final RangeAvailableHandler reader,
-            final RangeMissingHandler writer
+            final RangeMissingHandler writer,
+            String resourceDescription
         ) throws Exception {
             // some cache files can grow after being created, so rangeToWrite can be larger than the initial {@code length}
             assert rangeToWrite.start() >= 0 : rangeToWrite;
@@ -1273,6 +1279,7 @@ public class SharedBlobCacheService<KeyType> implements Releasable {
                     IntConsumer progressUpdater,
                     ActionListener<Void> completionListener
                 ) throws IOException {
+                    String blobFileExtension = getFileExtension(resourceDescription);
                     writer.fillCacheRange(
                         channel,
                         channelPos,
@@ -1283,7 +1290,8 @@ public class SharedBlobCacheService<KeyType> implements Releasable {
                         completionListener.map(unused -> {
                             var elapsedTime = TimeUnit.NANOSECONDS.toMillis(relativeTimeInNanosSupplier.getAsLong() - startTime);
                             blobCacheMetrics.getCacheMissLoadTimes().record(elapsedTime);
-                            blobCacheMetrics.getCacheMissCounter().increment();
+                            blobCacheMetrics.getCacheMissCounter()
+                                .incrementBy(1L, Map.of(LUCENE_FILE_EXTENSION_ATTRIBUTE_KEY, blobFileExtension));
                             return null;
                         })
                     );
@@ -2075,4 +2083,17 @@ public class SharedBlobCacheService<KeyType> implements Releasable {
             }
         }
     }
+
+    private static String getFileExtension(String resourceDescription) {
+        // TODO: consider introspecting resourceDescription for compound files
+        if (resourceDescription.endsWith(LuceneFilesExtensions.CFS.getExtension())) {
+            return LuceneFilesExtensions.CFS.getExtension();
+        }
+        String extension = IndexFileNames.getExtension(resourceDescription);
+        if (LuceneFilesExtensions.isLuceneExtension(extension)) {
+            return extension;
+        } else {
+            return NON_LUCENE_EXTENSION_TO_RECORD;
+        }
+    }
 }

+ 77 - 0
x-pack/plugin/blob-cache/src/test/java/org/elasticsearch/blobcache/shared/SharedBlobCacheServiceTests.java

@@ -43,6 +43,9 @@ import org.elasticsearch.threadpool.ThreadPool;
 
 import java.io.IOException;
 import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Path;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -51,6 +54,8 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.BrokenBarrierException;
 import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -171,6 +176,78 @@ public class SharedBlobCacheServiceTests extends ESTestCase {
         }
     }
 
+    public void testCacheMissOnPopulate() throws Exception {
+        Settings settings = Settings.builder()
+            .put(NODE_NAME_SETTING.getKey(), "node")
+            .put(SharedBlobCacheService.SHARED_CACHE_SIZE_SETTING.getKey(), ByteSizeValue.ofBytes(size(50)).getStringRep())
+            .put(SharedBlobCacheService.SHARED_CACHE_REGION_SIZE_SETTING.getKey(), ByteSizeValue.ofBytes(size(10)).getStringRep())
+            .put("path.home", createTempDir())
+            .build();
+        final DeterministicTaskQueue taskQueue = new DeterministicTaskQueue();
+        RecordingMeterRegistry recordingMeterRegistry = new RecordingMeterRegistry();
+        BlobCacheMetrics metrics = new BlobCacheMetrics(recordingMeterRegistry);
+        ExecutorService ioExecutor = Executors.newCachedThreadPool();
+        try (
+            NodeEnvironment environment = new NodeEnvironment(settings, TestEnvironment.newEnvironment(settings));
+            var cacheService = new SharedBlobCacheService<>(environment, settings, taskQueue.getThreadPool(), ioExecutor, metrics)
+        ) {
+            ByteRange rangeRead = ByteRange.of(0L, 1L);
+            ByteRange rangeWrite = ByteRange.of(0L, 1L);
+            Path tempFile = createTempFile("test", "other");
+            String resourceDescription = tempFile.toAbsolutePath().toString();
+            final var cacheKey = generateCacheKey();
+            SharedBlobCacheService<Object>.CacheFile cacheFile = cacheService.getCacheFile(cacheKey, 1L);
+
+            ByteBuffer writeBuffer = ByteBuffer.allocate(1);
+
+            final int bytesRead = cacheFile.populateAndRead(
+                rangeRead,
+                rangeWrite,
+                (channel, pos, relativePos, len) -> len,
+                (channel, channelPos, streamFactory, relativePos, len, progressUpdater, completionListener) -> {
+                    try (var in = Files.newInputStream(tempFile)) {
+                        SharedBytes.copyToCacheFileAligned(channel, in, channelPos, progressUpdater, writeBuffer.clear());
+                    }
+                    ActionListener.completeWith(completionListener, () -> null);
+                },
+                resourceDescription
+            );
+            assertThat(bytesRead, is(1));
+            List<Measurement> measurements = recordingMeterRegistry.getRecorder()
+                .getMeasurements(InstrumentType.LONG_COUNTER, "es.blob_cache.miss_that_triggered_read.total");
+            Measurement first = measurements.getFirst();
+            assertThat(first.attributes().get("file_extension"), is("other"));
+            assertThat(first.value(), is(1L));
+
+            Path tempFile2 = createTempFile("test", "cfs");
+            resourceDescription = tempFile2.toAbsolutePath().toString();
+            cacheFile = cacheService.getCacheFile(generateCacheKey(), 1L);
+
+            ByteBuffer writeBuffer2 = ByteBuffer.allocate(1);
+
+            final int bytesRead2 = cacheFile.populateAndRead(
+                rangeRead,
+                rangeWrite,
+                (channel, pos, relativePos, len) -> len,
+                (channel, channelPos, streamFactory, relativePos, len, progressUpdater, completionListener) -> {
+                    try (var in = Files.newInputStream(tempFile2)) {
+                        SharedBytes.copyToCacheFileAligned(channel, in, channelPos, progressUpdater, writeBuffer2.clear());
+                    }
+                    ActionListener.completeWith(completionListener, () -> null);
+                },
+                resourceDescription
+            );
+            assertThat(bytesRead2, is(1));
+
+            measurements = recordingMeterRegistry.getRecorder()
+                .getMeasurements(InstrumentType.LONG_COUNTER, "es.blob_cache.miss_that_triggered_read.total");
+            Measurement measurement = measurements.get(1);
+            assertThat(measurement.attributes().get("file_extension"), is("cfs"));
+            assertThat(measurement.value(), is(1L));
+        }
+        ioExecutor.shutdown();
+    }
+
     private static boolean tryEvict(SharedBlobCacheService.CacheFileRegion<Object> region1) {
         if (randomBoolean()) {
             return region1.tryEvict();

+ 2 - 1
x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/store/input/FrozenIndexInput.java

@@ -177,7 +177,8 @@ public final class FrozenIndexInput extends MetadataCachingIndexInput {
                             return null;
                         }
                     }
-                )
+                ),
+                fileInfo.physicalName()
             );
             assert bytesRead == length : bytesRead + " vs " + length;
             byteBufferReference.finish(bytesRead);