forked from apache/cassandra
-
Notifications
You must be signed in to change notification settings - Fork 21
[WIP] CNDB-15360 Chunk Cache inspection POC #2140
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Open
lesnik2u
wants to merge
2
commits into
main
Choose a base branch
from
CNDB-15360-ChunkCache
base: main
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -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<ChunkCache.Key, ChunkCache.Chunk>, CacheSize | ||
|
|
@@ -113,6 +119,7 @@ public class ChunkCache | |
|
|
||
| // File id management | ||
| private final ConcurrentHashMap<File, Long> fileIdMap = new ConcurrentHashMap<>(); | ||
| private final ConcurrentHashMap<Long, File> 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 | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. what about ChunkCacheEntryMetadata ? |
||
| { | ||
| 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<ChunkCacheInspectionEntry> 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<ChunkCacheInspectionEntry> consumer) | ||
| { | ||
| inspectCacheSegments(limit, false, consumer); | ||
| } | ||
|
|
||
| private void inspectCacheSegments(int limit, boolean hottest, java.util.function.Consumer<ChunkCacheInspectionEntry> 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<Key, Chunk> 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())); | ||
| }); | ||
| }); | ||
| } | ||
| } | ||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this is going to use lot of memory (and use CPU on hot paths) for a feature that will be used only in DEV/TEST to investigate the behavior of the ChunkCache, it is not really worth to take this way.