|
62 | 62 | import org.apache.cassandra.utils.memory.BufferPool; |
63 | 63 | import org.apache.cassandra.utils.memory.BufferPools; |
64 | 64 | import org.github.jamm.Unmetered; |
| 65 | +import java.util.List; |
| 66 | +import java.util.Map; |
| 67 | +import java.util.HashMap; |
| 68 | +import java.util.Collections; |
| 69 | +import java.util.stream.Collectors; |
| 70 | +import java.util.stream.Stream; |
65 | 71 |
|
66 | 72 | public class ChunkCache |
67 | 73 | implements RemovalListener<ChunkCache.Key, ChunkCache.Chunk>, CacheSize |
@@ -770,4 +776,80 @@ public int sizeOfFile(File file) { |
770 | 776 | long mask = - (1 << (CHUNK_SIZE_LOG2_BITS + READER_TYPE_BITS)); |
771 | 777 | return (int) cacheAsMap.keySet().stream().filter(x -> (x.readerId & mask) == fileId).count(); |
772 | 778 | } |
| 779 | + |
| 780 | + public static class ChunkCacheInspectionEntry |
| 781 | + { |
| 782 | + public final String filePath; |
| 783 | + public final long position; |
| 784 | + public final int size; |
| 785 | + public final boolean isReady; |
| 786 | + |
| 787 | + public ChunkCacheInspectionEntry(String filePath, long position, int size, boolean isReady) |
| 788 | + { |
| 789 | + this.filePath = filePath; |
| 790 | + this.position = position; |
| 791 | + this.size = size; |
| 792 | + this.isReady = isReady; |
| 793 | + } |
| 794 | + |
| 795 | + @Override |
| 796 | + public String toString() |
| 797 | + { |
| 798 | + return String.format("Chunk{file='%s', pos=%d, size=%d}", filePath, position, size); |
| 799 | + } |
| 800 | + } |
| 801 | + |
| 802 | + /** |
| 803 | + * Inspects the "hottest" (most frequently/recently used) chunks in the cache. |
| 804 | + * |
| 805 | + * @param limit maximum number of entries to return |
| 806 | + * @return List of entries sorted from hottest to less hot |
| 807 | + */ |
| 808 | + public List<ChunkCacheInspectionEntry> getHotEntries(int limit) |
| 809 | + { |
| 810 | + return inspectCacheSegments(limit, true); |
| 811 | + } |
| 812 | + |
| 813 | + /** |
| 814 | + * Inspects the "coldest" (candidates for eviction) chunks in the cache. |
| 815 | + * |
| 816 | + * @param limit maximum number of entries to return |
| 817 | + * @return List of entries sorted from coldest to warmer |
| 818 | + */ |
| 819 | + public List<ChunkCacheInspectionEntry> getColdEntries(int limit) |
| 820 | + { |
| 821 | + return inspectCacheSegments(limit, false); |
| 822 | + } |
| 823 | + |
| 824 | + private List<ChunkCacheInspectionEntry> inspectCacheSegments(int limit, boolean hottest) |
| 825 | + { |
| 826 | + if (!enabled) |
| 827 | + return Collections.emptyList(); |
| 828 | + |
| 829 | + Map<Long, String> idToPath = new HashMap<>(fileIdMap.size()); |
| 830 | + fileIdMap.forEach((file, id) -> idToPath.put(id, file.toString())); |
| 831 | + |
| 832 | + int shift = CHUNK_SIZE_LOG2_BITS + READER_TYPE_BITS; |
| 833 | + |
| 834 | + return synchronousCache.policy().eviction() |
| 835 | + .map(policy -> { |
| 836 | + Map<Key, Chunk> orderedMap = hottest |
| 837 | + ? policy.hottest(limit) |
| 838 | + : policy.coldest(limit); |
| 839 | + |
| 840 | + return orderedMap.entrySet().stream().map(entry -> { |
| 841 | + Key key = entry.getKey(); |
| 842 | + Chunk chunk = entry.getValue(); |
| 843 | + |
| 844 | + long fileId = key.readerId >>> shift; |
| 845 | + |
| 846 | + String path = idToPath.getOrDefault(fileId, "Unknown/StaleFile"); |
| 847 | + |
| 848 | + int size = (chunk != null) ? chunk.capacity() : 0; |
| 849 | + |
| 850 | + return new ChunkCacheInspectionEntry(path, key.position, size, chunk != null); |
| 851 | + }).collect(Collectors.toList()); |
| 852 | + }) |
| 853 | + .orElse(Collections.emptyList()); |
| 854 | + } |
773 | 855 | } |
0 commit comments