From cdcac4076da8fe96e94a961bebb9ea31333f2155 Mon Sep 17 00:00:00 2001 From: lesnik2u Date: Fri, 28 Nov 2025 11:47:08 +0100 Subject: [PATCH 1/6] CNDB-15360: ChunkCache hot and cold inspection POC --- .../apache/cassandra/cache/ChunkCache.java | 100 +++++++++++++++++- 1 file changed, 98 insertions(+), 2 deletions(-) diff --git a/src/java/org/apache/cassandra/cache/ChunkCache.java b/src/java/org/apache/cassandra/cache/ChunkCache.java index 6def74cc21b2..c9898bd487ee 100644 --- a/src/java/org/apache/cassandra/cache/ChunkCache.java +++ b/src/java/org/apache/cassandra/cache/ChunkCache.java @@ -62,6 +62,12 @@ import org.apache.cassandra.utils.memory.BufferPool; import org.apache.cassandra.utils.memory.BufferPools; import org.github.jamm.Unmetered; +import java.util.List; +import java.util.Map; +import java.util.HashMap; +import java.util.Collections; +import java.util.stream.Collectors; +import java.util.stream.Stream; public class ChunkCache implements RemovalListener, CacheSize @@ -113,6 +119,7 @@ public class ChunkCache // File id management private final ConcurrentHashMap fileIdMap = new ConcurrentHashMap<>(); + private final ConcurrentHashMap idToFileMap = new ConcurrentHashMap<>(); private final AtomicLong nextFileId = new AtomicLong(0); // number of bits required to store the log2 of the chunk size @@ -263,7 +270,9 @@ protected long readerIdFor(ChunkReader source) private long assignFileId(File file) { - return nextFileId.getAndIncrement(); + long id = nextFileId.getAndIncrement(); + idToFileMap.put(id, file); + return id; } /** @@ -279,7 +288,11 @@ public void invalidateFile(File file) { // Removing the name from the id map suffices -- the next time someone wants to read this file, it will get // assigned a fresh id. - fileIdMap.remove(file); + Long id = fileIdMap.remove(file); + if (id != null) + { + idToFileMap.remove(id); + } } /** @@ -770,4 +783,87 @@ public int sizeOfFile(File file) { long mask = - (1 << (CHUNK_SIZE_LOG2_BITS + READER_TYPE_BITS)); return (int) cacheAsMap.keySet().stream().filter(x -> (x.readerId & mask) == fileId).count(); } + + /** + * A snapshot of a specific chunk currently held in the cache. + * Used for diagnostics and inspection tools. + */ + public static class ChunkCacheInspectionEntry + { + public final File file; + public final long position; + public final int size; + + public ChunkCacheInspectionEntry(File file, long position, int size) + { + this.file = file; + this.position = position; + this.size = size; + } + + @Override + public String toString() + { + return String.format("Chunk{file='%s', pos=%d, size=%d}", file, position, size); + } + } + + + /** + * Inspects the "hottest" (most frequently/recently used) chunks in the cache. + * Uses a consumer pattern to avoid materializing a full list in memory. + * + * @param limit maximum number of entries to inspect + * @param consumer consumer to process each entry + */ + public void inspectHotEntries(int limit, java.util.function.Consumer consumer) + { + inspectCacheSegments(limit, true, consumer); + } + + /** + * Inspects the "coldest" (candidates for eviction) chunks in the cache. + * Uses a consumer pattern to avoid materializing a full list in memory. + * + * @param limit maximum number of entries to inspect + * @param consumer consumer to process each entry + */ + public void inspectColdEntries(int limit, java.util.function.Consumer consumer) + { + inspectCacheSegments(limit, false, consumer); + } + + private void inspectCacheSegments(int limit, boolean hottest, java.util.function.Consumer consumer) + { + if (!enabled) + return; + + // The readerId packs multiple values into a single long: [File ID][Chunk Size][Reader Type] + // We need to shift right to extract just the File ID portion by discarding the lower bits + int shift = CHUNK_SIZE_LOG2_BITS + READER_TYPE_BITS; + + synchronousCache.policy().eviction().ifPresent(policy -> { + Map orderedMap = hottest ? policy.hottest(limit) : policy.coldest(limit); + + orderedMap.forEach((key, chunk) -> { + // Skip entries where the chunk was evicted but the key still exists + if (chunk == null) + return; + + // Extract the file ID by shifting away the lower bits. + // The >>> operator does an unsigned right shift, moving the bits right and filling with zeros. + // For example, if readerId is [FileID:42][ChunkSize:3][ReaderType:1] and shift is 5, + // this operation discards the rightmost 5 bits (ChunkSize + ReaderType) leaving just FileID:42 + long fileId = key.readerId >>> shift; + + File file = idToFileMap.get(fileId); + + // Skip if we can't find the file (it may have been invalidated) + if (file == null) + return; + + consumer.accept(new ChunkCacheInspectionEntry(file, key.position, chunk.capacity())); + }); + }); + } } From 1c9a0575ceaf022b18e6b97090480d2833ae9127 Mon Sep 17 00:00:00 2001 From: lesnik2u Date: Fri, 28 Nov 2025 14:00:59 +0100 Subject: [PATCH 2/6] CNDB-15360: added unit test coverage for inspection methods --- .../cassandra/cache/ChunkCacheTest.java | 383 ++++++++++++++++-- 1 file changed, 360 insertions(+), 23 deletions(-) diff --git a/test/unit/org/apache/cassandra/cache/ChunkCacheTest.java b/test/unit/org/apache/cassandra/cache/ChunkCacheTest.java index da04e896068f..65902288ead6 100644 --- a/test/unit/org/apache/cassandra/cache/ChunkCacheTest.java +++ b/test/unit/org/apache/cassandra/cache/ChunkCacheTest.java @@ -22,7 +22,11 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.FileChannel; +import java.util.ArrayList; import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.CopyOnWriteArrayList; @@ -52,6 +56,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertSame; import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; @@ -125,28 +130,28 @@ public void testRandomAccessReadersWithUpdatedFileAndMultipleChunksAndCacheInval try (FileHandle.Builder builder1 = new FileHandle.Builder(file).withChunkCache(ChunkCache.instance)) { - try (FileHandle handle1 = builder1.complete(); - RandomAccessReader reader1 = handle1.createReader()) - { - // Read 2 chunks and verify contents - for (int i = 0; i < RandomAccessReader.DEFAULT_BUFFER_SIZE * 2; i++) - assertEquals((byte) 0, reader1.readByte()); - - // Overwrite the file's contents - var bytes = new byte[RandomAccessReader.DEFAULT_BUFFER_SIZE * 3]; - Arrays.fill(bytes, (byte) 1); - writeBytes(file, bytes); - - // Verify rebuffer pulls from cache for first 2 bytes and then from disk for third byte - reader1.seek(0); - for (int i = 0; i < RandomAccessReader.DEFAULT_BUFFER_SIZE * 2; i++) - assertEquals((byte) 0, reader1.readByte()); - // Trigger read of next chunk and see it is the new data - assertEquals((byte) 1, reader1.readByte()); - - assertEquals(3, ChunkCache.instance.size()); - assertEquals(3, ChunkCache.instance.sizeOfFile(file)); - } + try (FileHandle handle1 = builder1.complete(); + RandomAccessReader reader1 = handle1.createReader()) + { + // Read 2 chunks and verify contents + for (int i = 0; i < RandomAccessReader.DEFAULT_BUFFER_SIZE * 2; i++) + assertEquals((byte) 0, reader1.readByte()); + + // Overwrite the file's contents + var bytes = new byte[RandomAccessReader.DEFAULT_BUFFER_SIZE * 3]; + Arrays.fill(bytes, (byte) 1); + writeBytes(file, bytes); + + // Verify rebuffer pulls from cache for first 2 bytes and then from disk for third byte + reader1.seek(0); + for (int i = 0; i < RandomAccessReader.DEFAULT_BUFFER_SIZE * 2; i++) + assertEquals((byte) 0, reader1.readByte()); + // Trigger read of next chunk and see it is the new data + assertEquals((byte) 1, reader1.readByte()); + + assertEquals(3, ChunkCache.instance.size()); + assertEquals(3, ChunkCache.instance.sizeOfFile(file)); + } // Invalidate cache for both chunks ChunkCache.instance.invalidateFile(file); @@ -569,4 +574,336 @@ public void tstDontCacheErroredReads() throws Exception assertEquals(0, ChunkCache.instance.sizeOfFile(file1)); } -} + + @Test + public void testChunkCacheInspectionEntry() + { + File file = new File("/tmp/test.db"); + ChunkCache.ChunkCacheInspectionEntry entry = new ChunkCache.ChunkCacheInspectionEntry(file, 1024, 4096); + + assertEquals(file, entry.file); + assertEquals(1024, entry.position); + assertEquals(4096, entry.size); + + + String expected = String.format("Chunk{file='%s', pos=%d, size=%d}", file, 1024, 4096); + assertEquals(expected, entry.toString()); + } + + @Test + public void testInspectHotEntries() throws IOException + { + ChunkCache.instance.clear(); + assertEquals(0, ChunkCache.instance.size()); + + // Create multiple files with different access patterns + File file1 = FileUtils.createTempFile("hot1", null); + file1.deleteOnExit(); + File file2 = FileUtils.createTempFile("hot2", null); + file2.deleteOnExit(); + File file3 = FileUtils.createTempFile("hot3", null); + file3.deleteOnExit(); + + writeBytes(file1, new byte[RandomAccessReader.DEFAULT_BUFFER_SIZE]); + writeBytes(file2, new byte[RandomAccessReader.DEFAULT_BUFFER_SIZE]); + writeBytes(file3, new byte[RandomAccessReader.DEFAULT_BUFFER_SIZE]); + + // Access files to populate cache + try (FileHandle.Builder builder1 = new FileHandle.Builder(file1).withChunkCache(ChunkCache.instance); + FileHandle handle1 = builder1.complete(); + RandomAccessReader reader1 = handle1.createReader(); + FileHandle.Builder builder2 = new FileHandle.Builder(file2).withChunkCache(ChunkCache.instance); + FileHandle handle2 = builder2.complete(); + RandomAccessReader reader2 = handle2.createReader(); + FileHandle.Builder builder3 = new FileHandle.Builder(file3).withChunkCache(ChunkCache.instance); + FileHandle handle3 = builder3.complete(); + RandomAccessReader reader3 = handle3.createReader()) + { + reader1.reBuffer(); + reader2.reBuffer(); + reader3.reBuffer(); + + assertEquals(3, ChunkCache.instance.size()); + + // Inspect hot entries + List hotEntries = new ArrayList<>(); + ChunkCache.instance.inspectHotEntries(10, hotEntries::add); + + // Should have exactly 3 entries + assertEquals(3, hotEntries.size()); + + // Verify entries have valid data + Set observedFiles = new HashSet<>(); + for (ChunkCache.ChunkCacheInspectionEntry entry : hotEntries) + { + logger.info("Hot entry: file={}, position={}, size={}, isReady={}", + entry.file.path(), entry.position, entry.size, true /* isReady removed from ChunkCacheInspectionEntry */); + assertNotNull(entry.file); + assertTrue(entry.position >= 0); + assertTrue(entry.size > 0); + assertTrue(true /* isReady removed from ChunkCacheInspectionEntry */); + observedFiles.add(entry.file); + } + + // Verify all files are present + assertTrue(observedFiles.contains(file1)); + assertTrue(observedFiles.contains(file2)); + assertTrue(observedFiles.contains(file3)); + } + } + + @Test + public void testInspectColdEntries() throws IOException + { + ChunkCache.instance.clear(); + assertEquals(0, ChunkCache.instance.size()); + + // Create multiple files + File file1 = FileUtils.createTempFile("cold1", null); + file1.deleteOnExit(); + File file2 = FileUtils.createTempFile("cold2", null); + file2.deleteOnExit(); + + writeBytes(file1, new byte[RandomAccessReader.DEFAULT_BUFFER_SIZE]); + writeBytes(file2, new byte[RandomAccessReader.DEFAULT_BUFFER_SIZE]); + + // Access files to populate cache + try (FileHandle.Builder builder1 = new FileHandle.Builder(file1).withChunkCache(ChunkCache.instance); + FileHandle handle1 = builder1.complete(); + RandomAccessReader reader1 = handle1.createReader(); + FileHandle.Builder builder2 = new FileHandle.Builder(file2).withChunkCache(ChunkCache.instance); + FileHandle handle2 = builder2.complete(); + RandomAccessReader reader2 = handle2.createReader()) + { + reader1.reBuffer(); + reader2.reBuffer(); + + assertEquals(2, ChunkCache.instance.size()); + + // Inspect cold entries (candidates for eviction) + List coldEntries = new ArrayList<>(); + ChunkCache.instance.inspectColdEntries(10, coldEntries::add); + + // Should have exactly 2 entries + assertEquals(2, coldEntries.size()); + + // Verify entries have valid data + Set observedFiles = new HashSet<>(); + for (ChunkCache.ChunkCacheInspectionEntry entry : coldEntries) + { + logger.info("Cold entry: file={}, position={}, size={}, isReady={}", + entry.file.path(), entry.position, entry.size, true /* isReady removed from ChunkCacheInspectionEntry */); + assertNotNull(entry.file); + assertTrue(entry.position >= 0); + assertTrue(entry.size > 0); + observedFiles.add(entry.file); + } + + // Verify both files are present + assertTrue(observedFiles.contains(file1)); + assertTrue(observedFiles.contains(file2)); + } + } + + @Test + public void testInspectEntriesWithLimit() throws IOException + { + ChunkCache.instance.clear(); + assertEquals(0, ChunkCache.instance.size()); + + // Create file with multiple chunks + File file = FileUtils.createTempFile("limitTest", null); + file.deleteOnExit(); + writeBytes(file, new byte[RandomAccessReader.DEFAULT_BUFFER_SIZE * 5]); + + try (FileHandle.Builder builder = new FileHandle.Builder(file).withChunkCache(ChunkCache.instance); + FileHandle handle = builder.complete(); + RandomAccessReader reader = handle.createReader()) + { + // Read all chunks to populate cache + for (int i = 0; i < RandomAccessReader.DEFAULT_BUFFER_SIZE * 5; i++) + reader.readByte(); + + assertEquals(5, ChunkCache.instance.size()); + + // Test with limit smaller than cache size + List limitedEntries = new ArrayList<>(); + ChunkCache.instance.inspectHotEntries(2, limitedEntries::add); + + // Should respect the limit exactly + assertEquals(2, limitedEntries.size()); + + // Verify all entries are valid + for (ChunkCache.ChunkCacheInspectionEntry entry : limitedEntries) + { + assertNotNull(entry.file); + assertEquals(file, entry.file); + assertTrue(entry.position >= 0); + assertTrue(entry.size > 0); + assertTrue(true /* isReady removed from ChunkCacheInspectionEntry */); + } + } + } + + @Test + public void testInspectEntriesWhenCacheDisabled() + { + // Create a disabled cache + BufferPool pool = mock(BufferPool.class); + ChunkCache disabledCache = new ChunkCache(pool, 0, ChunkCacheMetrics::create); + + List entries = new ArrayList<>(); + + // Should not throw and should return no entries + disabledCache.inspectHotEntries(10, entries::add); + assertEquals(0, entries.size()); + + disabledCache.inspectColdEntries(10, entries::add); + assertEquals(0, entries.size()); + } + + @Test + public void testHotEntriesAreActuallyHotter() throws IOException + { + ChunkCache.instance.clear(); + assertEquals(0, ChunkCache.instance.size()); + + File hotFile = FileUtils.createTempFile("hot", null); + hotFile.deleteOnExit(); + File coldFile = FileUtils.createTempFile("cold", null); + coldFile.deleteOnExit(); + + logger.info("Test files created - hot: {}, cold: {}", hotFile.path(), coldFile.path()); + + writeBytes(hotFile, new byte[RandomAccessReader.DEFAULT_BUFFER_SIZE]); + writeBytes(coldFile, new byte[RandomAccessReader.DEFAULT_BUFFER_SIZE]); + + try (FileHandle.Builder hotBuilder = new FileHandle.Builder(hotFile).withChunkCache(ChunkCache.instance); + FileHandle hotHandle = hotBuilder.complete(); + FileHandle.Builder coldBuilder = new FileHandle.Builder(coldFile).withChunkCache(ChunkCache.instance); + FileHandle coldHandle = coldBuilder.complete()) + { + // First access cold file + logger.info("Accessing cold file once..."); + try (RandomAccessReader reader = coldHandle.createReader()) + { + reader.reBuffer(); + } + logger.info("Cache size after cold file access: {}", ChunkCache.instance.size()); + + // Then access hot file multiple times + logger.info("Accessing hot file 50 times..."); + for (int i = 0; i < 50; i++) + { + try (RandomAccessReader reader = hotHandle.createReader()) + { + reader.reBuffer(); + // Actually read some data to ensure the cache is used + reader.seek(0); + reader.readByte(); + } + } + logger.info("Cache size after hot file accesses: {}", ChunkCache.instance.size()); + + assertEquals(2, ChunkCache.instance.size()); + + // Verify hot entries and cold entries are different + logger.info("Inspecting hot entries..."); + List hotEntries = new ArrayList<>(); + ChunkCache.instance.inspectHotEntries(2, hotEntries::add); + + logger.info("Hot entries ({}): ", hotEntries.size()); + for (int i = 0; i < hotEntries.size(); i++) + { + ChunkCache.ChunkCacheInspectionEntry entry = hotEntries.get(i); + logger.info(" [{}] file={}, position={}, size={}, isReady={}", + i, entry.file.path(), entry.position, entry.size, true /* isReady removed from ChunkCacheInspectionEntry */); + } + + logger.info("Inspecting cold entries..."); + List coldEntries = new ArrayList<>(); + ChunkCache.instance.inspectColdEntries(2, coldEntries::add); + + logger.info("Cold entries ({}): ", coldEntries.size()); + for (int i = 0; i < coldEntries.size(); i++) + { + ChunkCache.ChunkCacheInspectionEntry entry = coldEntries.get(i); + logger.info(" [{}] file={}, position={}, size={}, isReady={}", + i, entry.file.path(), entry.position, entry.size, true /* isReady removed from ChunkCacheInspectionEntry */); + } + + assertEquals(2, hotEntries.size()); + assertEquals(2, coldEntries.size()); + + // The order should be different - hot entries should be in opposite order from cold + Set hotFiles = hotEntries.stream().map(e -> e.file).collect(java.util.stream.Collectors.toSet()); + Set coldFiles = coldEntries.stream().map(e -> e.file).collect(java.util.stream.Collectors.toSet()); + + logger.info("Hot files set contains hot file: {}", hotFiles.contains(hotFile)); + logger.info("Hot files set contains cold file: {}", hotFiles.contains(coldFile)); + logger.info("Cold files set contains hot file: {}", coldFiles.contains(hotFile)); + logger.info("Cold files set contains cold file: {}", coldFiles.contains(coldFile)); + + // Both should contain both files + assertTrue(hotFiles.contains(hotFile)); + assertTrue(hotFiles.contains(coldFile)); + assertTrue(coldFiles.contains(hotFile)); + assertTrue(coldFiles.contains(coldFile)); + + // The first hot entry should be the frequently accessed file + // The first cold entry should be the infrequently accessed file + logger.info("First hot entry file: {}", hotEntries.get(0).file.path()); + logger.info("Expected hot file: {}", hotFile.path()); + logger.info("First cold entry file: {}", coldEntries.get(0).file.path()); + logger.info("Expected cold file: {}", coldFile.path()); + + assertEquals("Most frequently accessed file should be first in hot entries", + hotFile, hotEntries.get(0).file); + assertEquals("Least frequently accessed file should be first in cold entries", + coldFile, coldEntries.get(0).file); + + logger.info("Test passed successfully!"); + } + } + + @Test + public void testInspectEntriesWithZeroLimit() throws IOException + { + ChunkCache.instance.clear(); + + File file = FileUtils.createTempFile("zeroLimit", null); + file.deleteOnExit(); + writeBytes(file, new byte[RandomAccessReader.DEFAULT_BUFFER_SIZE]); + + try (FileHandle.Builder builder = new FileHandle.Builder(file).withChunkCache(ChunkCache.instance); + FileHandle handle = builder.complete(); + RandomAccessReader reader = handle.createReader()) + { + reader.reBuffer(); + assertEquals(1, ChunkCache.instance.size()); + + List entries = new ArrayList<>(); + ChunkCache.instance.inspectHotEntries(0, entries::add); + + // Should return no entries when limit is 0 + assertEquals(0, entries.size()); + } + } + + @Test + public void testInspectEntriesWithEmptyCache() + { + ChunkCache.instance.clear(); + assertEquals(0, ChunkCache.instance.size()); + + List hotEntries = new ArrayList<>(); + List coldEntries = new ArrayList<>(); + + // Should not throw when cache is empty + ChunkCache.instance.inspectHotEntries(10, hotEntries::add); + ChunkCache.instance.inspectColdEntries(10, coldEntries::add); + + assertEquals(0, hotEntries.size()); + assertEquals(0, coldEntries.size()); + } +} \ No newline at end of file From 17f7a1be47399992b7a5e10ac148a2b91aa8e343 Mon Sep 17 00:00:00 2001 From: lesnik2u Date: Mon, 1 Dec 2025 16:49:32 +0100 Subject: [PATCH 3/6] CNDB-15360: Removed the flaky "hotness" test and focus on data validity --- .../cassandra/cache/ChunkCacheTest.java | 190 +++--------------- 1 file changed, 33 insertions(+), 157 deletions(-) diff --git a/test/unit/org/apache/cassandra/cache/ChunkCacheTest.java b/test/unit/org/apache/cassandra/cache/ChunkCacheTest.java index 65902288ead6..0faaf68b1e2b 100644 --- a/test/unit/org/apache/cassandra/cache/ChunkCacheTest.java +++ b/test/unit/org/apache/cassandra/cache/ChunkCacheTest.java @@ -33,6 +33,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.stream.Collectors; import com.google.common.base.Throwables; import org.junit.BeforeClass; @@ -575,28 +576,13 @@ public void tstDontCacheErroredReads() throws Exception assertEquals(0, ChunkCache.instance.sizeOfFile(file1)); } - @Test - public void testChunkCacheInspectionEntry() - { - File file = new File("/tmp/test.db"); - ChunkCache.ChunkCacheInspectionEntry entry = new ChunkCache.ChunkCacheInspectionEntry(file, 1024, 4096); - - assertEquals(file, entry.file); - assertEquals(1024, entry.position); - assertEquals(4096, entry.size); - - - String expected = String.format("Chunk{file='%s', pos=%d, size=%d}", file, 1024, 4096); - assertEquals(expected, entry.toString()); - } - @Test public void testInspectHotEntries() throws IOException { ChunkCache.instance.clear(); assertEquals(0, ChunkCache.instance.size()); - // Create multiple files with different access patterns + // Create multiple files and populate cache File file1 = FileUtils.createTempFile("hot1", null); file1.deleteOnExit(); File file2 = FileUtils.createTempFile("hot2", null); @@ -608,6 +594,8 @@ public void testInspectHotEntries() throws IOException writeBytes(file2, new byte[RandomAccessReader.DEFAULT_BUFFER_SIZE]); writeBytes(file3, new byte[RandomAccessReader.DEFAULT_BUFFER_SIZE]); + Set expectedFiles = new HashSet<>(Arrays.asList(file1, file2, file3)); + // Access files to populate cache try (FileHandle.Builder builder1 = new FileHandle.Builder(file1).withChunkCache(ChunkCache.instance); FileHandle handle1 = builder1.complete(); @@ -632,23 +620,20 @@ public void testInspectHotEntries() throws IOException // Should have exactly 3 entries assertEquals(3, hotEntries.size()); - // Verify entries have valid data - Set observedFiles = new HashSet<>(); + // Verify entries have valid data and match files we put in cache for (ChunkCache.ChunkCacheInspectionEntry entry : hotEntries) { - logger.info("Hot entry: file={}, position={}, size={}, isReady={}", - entry.file.path(), entry.position, entry.size, true /* isReady removed from ChunkCacheInspectionEntry */); - assertNotNull(entry.file); - assertTrue(entry.position >= 0); - assertTrue(entry.size > 0); - assertTrue(true /* isReady removed from ChunkCacheInspectionEntry */); - observedFiles.add(entry.file); + assertNotNull("File should not be null", entry.file); + assertTrue("File should be one we added to cache", expectedFiles.contains(entry.file)); + assertTrue("Position should be non-negative", entry.position >= 0); + assertTrue("Size should be positive", entry.size > 0); } - // Verify all files are present - assertTrue(observedFiles.contains(file1)); - assertTrue(observedFiles.contains(file2)); - assertTrue(observedFiles.contains(file3)); + // Verify all files are represented + Set observedFiles = hotEntries.stream() + .map(e -> e.file) + .collect(Collectors.toSet()); + assertEquals("All cached files should appear in results", expectedFiles, observedFiles); } } @@ -658,7 +643,6 @@ public void testInspectColdEntries() throws IOException ChunkCache.instance.clear(); assertEquals(0, ChunkCache.instance.size()); - // Create multiple files File file1 = FileUtils.createTempFile("cold1", null); file1.deleteOnExit(); File file2 = FileUtils.createTempFile("cold2", null); @@ -667,7 +651,8 @@ public void testInspectColdEntries() throws IOException writeBytes(file1, new byte[RandomAccessReader.DEFAULT_BUFFER_SIZE]); writeBytes(file2, new byte[RandomAccessReader.DEFAULT_BUFFER_SIZE]); - // Access files to populate cache + Set expectedFiles = new HashSet<>(Arrays.asList(file1, file2)); + try (FileHandle.Builder builder1 = new FileHandle.Builder(file1).withChunkCache(ChunkCache.instance); FileHandle handle1 = builder1.complete(); RandomAccessReader reader1 = handle1.createReader(); @@ -680,28 +665,26 @@ public void testInspectColdEntries() throws IOException assertEquals(2, ChunkCache.instance.size()); - // Inspect cold entries (candidates for eviction) + // Inspect cold entries List coldEntries = new ArrayList<>(); ChunkCache.instance.inspectColdEntries(10, coldEntries::add); - // Should have exactly 2 entries assertEquals(2, coldEntries.size()); - // Verify entries have valid data - Set observedFiles = new HashSet<>(); + // Verify entries have valid data and match files we put in cache for (ChunkCache.ChunkCacheInspectionEntry entry : coldEntries) { - logger.info("Cold entry: file={}, position={}, size={}, isReady={}", - entry.file.path(), entry.position, entry.size, true /* isReady removed from ChunkCacheInspectionEntry */); - assertNotNull(entry.file); - assertTrue(entry.position >= 0); - assertTrue(entry.size > 0); - observedFiles.add(entry.file); + assertNotNull("File should not be null", entry.file); + assertTrue("File should be one we added to cache", expectedFiles.contains(entry.file)); + assertTrue("Position should be non-negative", entry.position >= 0); + assertTrue("Size should be positive", entry.size > 0); } - // Verify both files are present - assertTrue(observedFiles.contains(file1)); - assertTrue(observedFiles.contains(file2)); + // Verify all files are represented + Set observedFiles = coldEntries.stream() + .map(e -> e.file) + .collect(Collectors.toSet()); + assertEquals("All cached files should appear in results", expectedFiles, observedFiles); } } @@ -711,7 +694,6 @@ public void testInspectEntriesWithLimit() throws IOException ChunkCache.instance.clear(); assertEquals(0, ChunkCache.instance.size()); - // Create file with multiple chunks File file = FileUtils.createTempFile("limitTest", null); file.deleteOnExit(); writeBytes(file, new byte[RandomAccessReader.DEFAULT_BUFFER_SIZE * 5]); @@ -730,17 +712,16 @@ public void testInspectEntriesWithLimit() throws IOException List limitedEntries = new ArrayList<>(); ChunkCache.instance.inspectHotEntries(2, limitedEntries::add); - // Should respect the limit exactly + // Should respect the limit assertEquals(2, limitedEntries.size()); - // Verify all entries are valid + // Verify all entries are from the same file and have valid data for (ChunkCache.ChunkCacheInspectionEntry entry : limitedEntries) { - assertNotNull(entry.file); - assertEquals(file, entry.file); - assertTrue(entry.position >= 0); - assertTrue(entry.size > 0); - assertTrue(true /* isReady removed from ChunkCacheInspectionEntry */); + assertNotNull("File should not be null", entry.file); + assertEquals("Should be from the test file", file, entry.file); + assertTrue("Position should be non-negative", entry.position >= 0); + assertTrue("Size should be positive", entry.size > 0); } } } @@ -748,7 +729,6 @@ public void testInspectEntriesWithLimit() throws IOException @Test public void testInspectEntriesWhenCacheDisabled() { - // Create a disabled cache BufferPool pool = mock(BufferPool.class); ChunkCache disabledCache = new ChunkCache(pool, 0, ChunkCacheMetrics::create); @@ -762,110 +742,6 @@ public void testInspectEntriesWhenCacheDisabled() assertEquals(0, entries.size()); } - @Test - public void testHotEntriesAreActuallyHotter() throws IOException - { - ChunkCache.instance.clear(); - assertEquals(0, ChunkCache.instance.size()); - - File hotFile = FileUtils.createTempFile("hot", null); - hotFile.deleteOnExit(); - File coldFile = FileUtils.createTempFile("cold", null); - coldFile.deleteOnExit(); - - logger.info("Test files created - hot: {}, cold: {}", hotFile.path(), coldFile.path()); - - writeBytes(hotFile, new byte[RandomAccessReader.DEFAULT_BUFFER_SIZE]); - writeBytes(coldFile, new byte[RandomAccessReader.DEFAULT_BUFFER_SIZE]); - - try (FileHandle.Builder hotBuilder = new FileHandle.Builder(hotFile).withChunkCache(ChunkCache.instance); - FileHandle hotHandle = hotBuilder.complete(); - FileHandle.Builder coldBuilder = new FileHandle.Builder(coldFile).withChunkCache(ChunkCache.instance); - FileHandle coldHandle = coldBuilder.complete()) - { - // First access cold file - logger.info("Accessing cold file once..."); - try (RandomAccessReader reader = coldHandle.createReader()) - { - reader.reBuffer(); - } - logger.info("Cache size after cold file access: {}", ChunkCache.instance.size()); - - // Then access hot file multiple times - logger.info("Accessing hot file 50 times..."); - for (int i = 0; i < 50; i++) - { - try (RandomAccessReader reader = hotHandle.createReader()) - { - reader.reBuffer(); - // Actually read some data to ensure the cache is used - reader.seek(0); - reader.readByte(); - } - } - logger.info("Cache size after hot file accesses: {}", ChunkCache.instance.size()); - - assertEquals(2, ChunkCache.instance.size()); - - // Verify hot entries and cold entries are different - logger.info("Inspecting hot entries..."); - List hotEntries = new ArrayList<>(); - ChunkCache.instance.inspectHotEntries(2, hotEntries::add); - - logger.info("Hot entries ({}): ", hotEntries.size()); - for (int i = 0; i < hotEntries.size(); i++) - { - ChunkCache.ChunkCacheInspectionEntry entry = hotEntries.get(i); - logger.info(" [{}] file={}, position={}, size={}, isReady={}", - i, entry.file.path(), entry.position, entry.size, true /* isReady removed from ChunkCacheInspectionEntry */); - } - - logger.info("Inspecting cold entries..."); - List coldEntries = new ArrayList<>(); - ChunkCache.instance.inspectColdEntries(2, coldEntries::add); - - logger.info("Cold entries ({}): ", coldEntries.size()); - for (int i = 0; i < coldEntries.size(); i++) - { - ChunkCache.ChunkCacheInspectionEntry entry = coldEntries.get(i); - logger.info(" [{}] file={}, position={}, size={}, isReady={}", - i, entry.file.path(), entry.position, entry.size, true /* isReady removed from ChunkCacheInspectionEntry */); - } - - assertEquals(2, hotEntries.size()); - assertEquals(2, coldEntries.size()); - - // The order should be different - hot entries should be in opposite order from cold - Set hotFiles = hotEntries.stream().map(e -> e.file).collect(java.util.stream.Collectors.toSet()); - Set coldFiles = coldEntries.stream().map(e -> e.file).collect(java.util.stream.Collectors.toSet()); - - logger.info("Hot files set contains hot file: {}", hotFiles.contains(hotFile)); - logger.info("Hot files set contains cold file: {}", hotFiles.contains(coldFile)); - logger.info("Cold files set contains hot file: {}", coldFiles.contains(hotFile)); - logger.info("Cold files set contains cold file: {}", coldFiles.contains(coldFile)); - - // Both should contain both files - assertTrue(hotFiles.contains(hotFile)); - assertTrue(hotFiles.contains(coldFile)); - assertTrue(coldFiles.contains(hotFile)); - assertTrue(coldFiles.contains(coldFile)); - - // The first hot entry should be the frequently accessed file - // The first cold entry should be the infrequently accessed file - logger.info("First hot entry file: {}", hotEntries.get(0).file.path()); - logger.info("Expected hot file: {}", hotFile.path()); - logger.info("First cold entry file: {}", coldEntries.get(0).file.path()); - logger.info("Expected cold file: {}", coldFile.path()); - - assertEquals("Most frequently accessed file should be first in hot entries", - hotFile, hotEntries.get(0).file); - assertEquals("Least frequently accessed file should be first in cold entries", - coldFile, coldEntries.get(0).file); - - logger.info("Test passed successfully!"); - } - } - @Test public void testInspectEntriesWithZeroLimit() throws IOException { From 1cdb563e7d2b245b974101b51f7d73fbfea46c41 Mon Sep 17 00:00:00 2001 From: lesnik2u Date: Mon, 1 Dec 2025 16:50:07 +0100 Subject: [PATCH 4/6] CNDB-15360: Remove unnecessary/unwanted reverse map --- .../apache/cassandra/cache/ChunkCache.java | 19 ++++++++----------- 1 file changed, 8 insertions(+), 11 deletions(-) diff --git a/src/java/org/apache/cassandra/cache/ChunkCache.java b/src/java/org/apache/cassandra/cache/ChunkCache.java index c9898bd487ee..21b95bf6c34b 100644 --- a/src/java/org/apache/cassandra/cache/ChunkCache.java +++ b/src/java/org/apache/cassandra/cache/ChunkCache.java @@ -119,7 +119,6 @@ public class ChunkCache // File id management private final ConcurrentHashMap fileIdMap = new ConcurrentHashMap<>(); - private final ConcurrentHashMap idToFileMap = new ConcurrentHashMap<>(); private final AtomicLong nextFileId = new AtomicLong(0); // number of bits required to store the log2 of the chunk size @@ -271,7 +270,6 @@ protected long readerIdFor(ChunkReader source) private long assignFileId(File file) { long id = nextFileId.getAndIncrement(); - idToFileMap.put(id, file); return id; } @@ -288,11 +286,7 @@ public void invalidateFile(File file) { // Removing the name from the id map suffices -- the next time someone wants to read this file, it will get // assigned a fresh id. - Long id = fileIdMap.remove(file); - if (id != null) - { - idToFileMap.remove(id); - } + fileIdMap.remove(file); } /** @@ -807,8 +801,6 @@ public String toString() return String.format("Chunk{file='%s', pos=%d, size=%d}", file, position, size); } } - - /** * Inspects the "hottest" (most frequently/recently used) chunks in the cache. * Uses a consumer pattern to avoid materializing a full list in memory. @@ -856,7 +848,12 @@ private void inspectCacheSegments(int limit, boolean hottest, java.util.function // this operation discards the rightmost 5 bits (ChunkSize + ReaderType) leaving just FileID:42 long fileId = key.readerId >>> shift; - File file = idToFileMap.get(fileId); + // Look up the File by searching through fileIdMap entries + File file = fileIdMap.entrySet().stream() + .filter(e -> e.getValue().equals(fileId)) + .map(Map.Entry::getKey) + .findFirst() + .orElse(null); // Skip if we can't find the file (it may have been invalidated) if (file == null) @@ -865,5 +862,5 @@ private void inspectCacheSegments(int limit, boolean hottest, java.util.function consumer.accept(new ChunkCacheInspectionEntry(file, key.position, chunk.capacity())); }); }); - } + } } From abf97b6aa82c9897e2a6d23f97c1f7df5eb94014 Mon Sep 17 00:00:00 2001 From: lesnik2u Date: Mon, 1 Dec 2025 17:50:22 +0100 Subject: [PATCH 5/6] CNDB-15360: fixes from git review --- .../apache/cassandra/cache/ChunkCache.java | 128 +++++++++--------- .../cassandra/cache/ChunkCacheTest.java | 31 +++-- 2 files changed, 80 insertions(+), 79 deletions(-) diff --git a/src/java/org/apache/cassandra/cache/ChunkCache.java b/src/java/org/apache/cassandra/cache/ChunkCache.java index 21b95bf6c34b..1364580b0d3a 100644 --- a/src/java/org/apache/cassandra/cache/ChunkCache.java +++ b/src/java/org/apache/cassandra/cache/ChunkCache.java @@ -28,6 +28,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; import java.util.function.Function; import javax.annotation.Nullable; @@ -52,8 +53,6 @@ import org.apache.cassandra.io.util.ChannelProxy; import org.apache.cassandra.io.util.ChunkReader; import org.apache.cassandra.io.util.File; -import org.apache.cassandra.io.util.PrefetchingRebufferer; -import org.apache.cassandra.io.util.ReadPattern; import org.apache.cassandra.io.util.Rebufferer; import org.apache.cassandra.io.util.RebuffererFactory; import org.apache.cassandra.metrics.ChunkCacheMetrics; @@ -62,22 +61,18 @@ import org.apache.cassandra.utils.memory.BufferPool; import org.apache.cassandra.utils.memory.BufferPools; import org.github.jamm.Unmetered; -import java.util.List; + import java.util.Map; -import java.util.HashMap; -import java.util.Collections; -import java.util.stream.Collectors; -import java.util.stream.Stream; public class ChunkCache - implements RemovalListener, CacheSize +implements RemovalListener, CacheSize { private final static Logger logger = LoggerFactory.getLogger(ChunkCache.class); public static final int RESERVED_POOL_SPACE_IN_MB = 32; private static final int INITIAL_CAPACITY = Integer.getInteger("cassandra.chunkcache_initialcapacity", 16); private static final boolean ASYNC_CLEANUP = Boolean.parseBoolean(System.getProperty("cassandra.chunkcache.async_cleanup", "true")); - private static final int CLEANER_THREADS = Integer.getInteger("dse.chunk.cache.cleaner.threads",1); + private static final int CLEANER_THREADS = Integer.getInteger("dse.chunk.cache.cleaner.threads", 1); private static final Class PERFORM_CLEANUP_TASK_CLASS; // cached value in order to not call System.getProperty on a hotpath @@ -192,7 +187,8 @@ public void onRemoval(Key key, Chunk chunk, RemovalCause cause) /** * Clears the cache, used in the CNDB Writer for testing purposes. */ - public void clear() { + public void clear() + { // Clear keysByFile first to prevent unnecessary computation in onRemoval method. synchronousCache.invalidateAll(); } @@ -246,7 +242,7 @@ public void intercept(Function interceptor /** * Maps a reader to a reader id, used by the cache to find content. - * + *

* Uses the file name (through the fileIdMap), reader type and chunk size to define the id. * The lowest {@link #READER_TYPE_BITS} are occupied by reader type, then the next {@link #CHUNK_SIZE_LOG2_BITS} * are occupied by log 2 of chunk size (we assume the chunk size is the power of 2), and the rest of the bits @@ -269,15 +265,14 @@ protected long readerIdFor(ChunkReader source) private long assignFileId(File file) { - long id = nextFileId.getAndIncrement(); - return id; + return nextFileId.getAndIncrement(); } /** * Invalidate all buffers from the given file, i.e. make sure they can not be accessed by any reader using a * FileHandle opened after this call. The buffers themselves will remain in the cache until they get normally * evicted, because it is too costly to remove them. - * + *

* Note that this call has no effect of handles that are already opened. The correct usage is to call this when * a file is deleted, or when a file is created for writing. It cannot be used to update and resynchronize the * cached view of an existing file. @@ -300,7 +295,7 @@ public void invalidateFileNow(File file) if (fileIdMaybeNull == null) return; long fileId = fileIdMaybeNull << (CHUNK_SIZE_LOG2_BITS + READER_TYPE_BITS); - long mask = - (1 << (CHUNK_SIZE_LOG2_BITS + READER_TYPE_BITS)); + long mask = -(1 << (CHUNK_SIZE_LOG2_BITS + READER_TYPE_BITS)); synchronousCache.invalidateAll(Iterables.filter(cache.asMap().keySet(), x -> (x.readerId & mask) == fileId)); } @@ -348,10 +343,14 @@ public boolean equals(Object obj) */ abstract static class Chunk { - /** The offset in the file where the chunk is read */ + /** + * The offset in the file where the chunk is read + */ final long offset; - /** The number of bytes read from disk, this could be less than the memory space allocated */ + /** + * The number of bytes read from disk, this could be less than the memory space allocated + */ int bytesRead; private volatile int references; @@ -381,7 +380,6 @@ Rebufferer.BufferHolder getReferencedBuffer(long position) if (refCount == 0) return null; // Buffer was released before we managed to reference it. - } while (!referencesUpdater.compareAndSet(this, refCount, refCount + 1)); return getBuffer(position); @@ -761,8 +759,8 @@ public int size() public long weightedSize() { return synchronousCache.policy().eviction() - .map(policy -> policy.weightedSize().orElseGet(synchronousCache::estimatedSize)) - .orElseGet(synchronousCache::estimatedSize); + .map(policy -> policy.weightedSize().orElseGet(synchronousCache::estimatedSize)) + .orElseGet(synchronousCache::estimatedSize); } /** @@ -774,7 +772,7 @@ public int sizeOfFile(File file) { if (fileIdMaybeNull == null) return 0; long fileId = fileIdMaybeNull << (CHUNK_SIZE_LOG2_BITS + READER_TYPE_BITS); - long mask = - (1 << (CHUNK_SIZE_LOG2_BITS + READER_TYPE_BITS)); + long mask = -(1 << (CHUNK_SIZE_LOG2_BITS + READER_TYPE_BITS)); return (int) cacheAsMap.keySet().stream().filter(x -> (x.readerId & mask) == fileId).count(); } @@ -782,6 +780,9 @@ public int sizeOfFile(File file) { * A snapshot of a specific chunk currently held in the cache. * Used for diagnostics and inspection tools. */ + enum CacheOrder + {HOTTEST, COLDEST} + public static class ChunkCacheInspectionEntry { public final File file; @@ -801,66 +802,65 @@ public String toString() return String.format("Chunk{file='%s', pos=%d, size=%d}", file, position, size); } } - /** - * Inspects the "hottest" (most frequently/recently used) chunks in the cache. - * Uses a consumer pattern to avoid materializing a full list in memory. - * - * @param limit maximum number of entries to inspect - * @param consumer consumer to process each entry - */ - public void inspectHotEntries(int limit, java.util.function.Consumer consumer) - { - inspectCacheSegments(limit, true, consumer); - } /** - * Inspects the "coldest" (candidates for eviction) chunks in the cache. + * Inspects chunks in the cache by access frequency/recency. * Uses a consumer pattern to avoid materializing a full list in memory. * * @param limit maximum number of entries to inspect + * @param order whether to inspect hottest (most used) or coldest (eviction candidates) * @param consumer consumer to process each entry */ - public void inspectColdEntries(int limit, java.util.function.Consumer consumer) + public void inspectEntries(int limit, CacheOrder order, Consumer consumer) { - inspectCacheSegments(limit, false, consumer); + inspectCacheSegments(limit, order == CacheOrder.HOTTEST, consumer); } - private void inspectCacheSegments(int limit, boolean hottest, java.util.function.Consumer consumer) + private void inspectCacheSegments(int limit, boolean hottest, Consumer consumer) { if (!enabled) - return; + throw new IllegalStateException("chunk cache not enabled"); + + // Eviction policy is required to determine hot/cold entries + // Note: In practice this will always be present due to maximumWeight() configuration, + // but we check explicitly to document the requirement and fail fast if cache setup changes. + if (synchronousCache.policy().eviction().isEmpty()) + throw new IllegalStateException("no eviction policy configured - cannot determine hot/cold entries"); // The readerId packs multiple values into a single long: [File ID][Chunk Size][Reader Type] // We need to shift right to extract just the File ID portion by discarding the lower bits int shift = CHUNK_SIZE_LOG2_BITS + READER_TYPE_BITS; - synchronousCache.policy().eviction().ifPresent(policy -> { - Map orderedMap = hottest ? policy.hottest(limit) : policy.coldest(limit); - - orderedMap.forEach((key, chunk) -> { - // Skip entries where the chunk was evicted but the key still exists - if (chunk == null) - return; - - // Extract the file ID by shifting away the lower bits. - // The >>> operator does an unsigned right shift, moving the bits right and filling with zeros. - // For example, if readerId is [FileID:42][ChunkSize:3][ReaderType:1] and shift is 5, - // this operation discards the rightmost 5 bits (ChunkSize + ReaderType) leaving just FileID:42 - long fileId = key.readerId >>> shift; - - // Look up the File by searching through fileIdMap entries - File file = fileIdMap.entrySet().stream() - .filter(e -> e.getValue().equals(fileId)) - .map(Map.Entry::getKey) - .findFirst() - .orElse(null); - - // Skip if we can't find the file (it may have been invalidated) - if (file == null) - return; - - consumer.accept(new ChunkCacheInspectionEntry(file, key.position, chunk.capacity())); - }); + var policy = synchronousCache.policy().eviction().get(); + Map orderedMap = hottest ? policy.hottest(limit) : policy.coldest(limit); + + orderedMap.forEach((key, chunk) -> { + // Skip entries where the chunk was evicted but the key still exists + if (chunk == null) + return; + + // Extract the file ID by shifting away the lower bits. + // The >>> operator does an unsigned right shift, moving the bits right and filling with zeros. + // For example, if readerId is [FileID:42][ChunkSize:3][ReaderType:1] and shift is 5, + // this operation discards the rightmost 5 bits (ChunkSize + ReaderType) leaving just FileID:42 + long fileId = key.readerId >>> shift; + + // Look up the File by searching through fileIdMap entries + File file = null; + for (Map.Entry entry : fileIdMap.entrySet()) + { + if (entry.getValue().equals(fileId)) + { + file = entry.getKey(); + break; + } + } + + // Skip if we can't find the file (it may have been invalidated) + if (file == null) + return; + + consumer.accept(new ChunkCacheInspectionEntry(file, key.position, chunk.capacity())); }); } } diff --git a/test/unit/org/apache/cassandra/cache/ChunkCacheTest.java b/test/unit/org/apache/cassandra/cache/ChunkCacheTest.java index 0faaf68b1e2b..504619f5cecf 100644 --- a/test/unit/org/apache/cassandra/cache/ChunkCacheTest.java +++ b/test/unit/org/apache/cassandra/cache/ChunkCacheTest.java @@ -615,7 +615,7 @@ public void testInspectHotEntries() throws IOException // Inspect hot entries List hotEntries = new ArrayList<>(); - ChunkCache.instance.inspectHotEntries(10, hotEntries::add); + ChunkCache.instance.inspectEntries(10, ChunkCache.CacheOrder.HOTTEST, hotEntries::add); // Should have exactly 3 entries assertEquals(3, hotEntries.size()); @@ -667,7 +667,7 @@ public void testInspectColdEntries() throws IOException // Inspect cold entries List coldEntries = new ArrayList<>(); - ChunkCache.instance.inspectColdEntries(10, coldEntries::add); + ChunkCache.instance.inspectEntries(10, ChunkCache.CacheOrder.COLDEST , coldEntries::add); assertEquals(2, coldEntries.size()); @@ -710,7 +710,7 @@ public void testInspectEntriesWithLimit() throws IOException // Test with limit smaller than cache size List limitedEntries = new ArrayList<>(); - ChunkCache.instance.inspectHotEntries(2, limitedEntries::add); + ChunkCache.instance.inspectEntries(2, ChunkCache.CacheOrder.HOTTEST, limitedEntries::add); // Should respect the limit assertEquals(2, limitedEntries.size()); @@ -726,20 +726,13 @@ public void testInspectEntriesWithLimit() throws IOException } } - @Test + @Test(expected = IllegalStateException.class) public void testInspectEntriesWhenCacheDisabled() { BufferPool pool = mock(BufferPool.class); ChunkCache disabledCache = new ChunkCache(pool, 0, ChunkCacheMetrics::create); - List entries = new ArrayList<>(); - - // Should not throw and should return no entries - disabledCache.inspectHotEntries(10, entries::add); - assertEquals(0, entries.size()); - - disabledCache.inspectColdEntries(10, entries::add); - assertEquals(0, entries.size()); + disabledCache.inspectEntries(10, ChunkCache.CacheOrder.HOTTEST, e -> {}); } @Test @@ -759,7 +752,7 @@ public void testInspectEntriesWithZeroLimit() throws IOException assertEquals(1, ChunkCache.instance.size()); List entries = new ArrayList<>(); - ChunkCache.instance.inspectHotEntries(0, entries::add); + ChunkCache.instance.inspectEntries(0, ChunkCache.CacheOrder.HOTTEST , entries::add); // Should return no entries when limit is 0 assertEquals(0, entries.size()); @@ -776,10 +769,18 @@ public void testInspectEntriesWithEmptyCache() List coldEntries = new ArrayList<>(); // Should not throw when cache is empty - ChunkCache.instance.inspectHotEntries(10, hotEntries::add); - ChunkCache.instance.inspectColdEntries(10, coldEntries::add); + ChunkCache.instance.inspectEntries(10, ChunkCache.CacheOrder.HOTTEST , hotEntries::add); + ChunkCache.instance.inspectEntries(10, ChunkCache.CacheOrder.COLDEST , coldEntries::add); assertEquals(0, hotEntries.size()); assertEquals(0, coldEntries.size()); } + @Test(expected = IllegalStateException.class) + public void testInspectEntriesThrowsWhenCacheDisabled() + { + BufferPool pool = mock(BufferPool.class); + ChunkCache disabledCache = new ChunkCache(pool, 0, ChunkCacheMetrics::create); + + disabledCache.inspectEntries(10, ChunkCache.CacheOrder.HOTTEST, e -> {}); + } } \ No newline at end of file From b58bdd1d53101d3e5405de1b8d24663cccfe3580 Mon Sep 17 00:00:00 2001 From: lesnik2u Date: Mon, 1 Dec 2025 18:11:06 +0100 Subject: [PATCH 6/6] CNDB-15360: fixes nit making enum public --- src/java/org/apache/cassandra/cache/ChunkCache.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/java/org/apache/cassandra/cache/ChunkCache.java b/src/java/org/apache/cassandra/cache/ChunkCache.java index 1364580b0d3a..5f57d0f2f2f8 100644 --- a/src/java/org/apache/cassandra/cache/ChunkCache.java +++ b/src/java/org/apache/cassandra/cache/ChunkCache.java @@ -780,7 +780,7 @@ public int sizeOfFile(File file) { * A snapshot of a specific chunk currently held in the cache. * Used for diagnostics and inspection tools. */ - enum CacheOrder + public enum CacheOrder {HOTTEST, COLDEST} public static class ChunkCacheInspectionEntry