Skip to content
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

[Tiered Caching] Stats rework (1/3): Interfaces and implementations for individual tiers #12531

Merged
merged 82 commits into from
Apr 12, 2024
Merged
Show file tree
Hide file tree
Changes from 35 commits
Commits
Show all changes
82 commits
Select commit Hold shift + click to select a range
f6e2330
Changes ICache to use ICacheKey
Mar 4, 2024
9e147c8
Added CacheStats interface
Mar 4, 2024
a1f249a
added cacheStats impl
Mar 4, 2024
3783684
Added serializers
Mar 4, 2024
3623858
Changes to ICache builders etc
Mar 4, 2024
818c43d
added cache dimension tests
Mar 4, 2024
64fb5ea
Changed on heap cache to align with new interfaces
Mar 4, 2024
9dd7e1b
Made TSC use the new interface changes
Mar 4, 2024
09be0ad
Made stats() part of ICache interface
Mar 4, 2024
52098b0
Changed ehcache to work with new changes to interfaces
Mar 4, 2024
04d0b62
spotlessApply
Mar 4, 2024
ad2b4ce
Suppresses rawtypes warnings where needed in ehcache
Mar 4, 2024
9a29a46
More spotlessApply
Mar 5, 2024
17e93fb
Changes TSC tests to work with new changes
Mar 5, 2024
1c122e3
Merge remote-tracking branch 'upstream/main' into tiramisu-stats-tiers
Mar 5, 2024
5cf3585
More spotlessApply
Mar 5, 2024
9d27f38
Suppressed unchecked cast warnings where needed
Mar 5, 2024
f1affc1
more unchecked
Mar 5, 2024
f841d7a
Fixed missing dimension names in TSC on heap cache builder
Mar 5, 2024
5f62582
Fixed failing UT
Mar 5, 2024
0345e27
commented out test which leaks threads
Mar 5, 2024
25852a1
Javadocs
Mar 5, 2024
70f6222
Rerunning gradle for flaky test failure
Mar 5, 2024
6a4a195
rerun gradle again
Mar 6, 2024
cd0d430
Fixed flaky ehcache test
Mar 6, 2024
0d82d9b
Merge remote-tracking branch 'upstream/main' into tiramisu-stats-tiers
Mar 6, 2024
17c2202
Addressed Sagar's comments
Mar 6, 2024
ca13c5f
Removed commented-out stats lines in TSC
Mar 7, 2024
a40211f
Addressed Sagar's minor comments
Mar 8, 2024
a2d1986
Addressed Sagar's comment on allowing user to pick which dimension co…
Mar 9, 2024
5411ad7
Addressed Michael's comments
Mar 14, 2024
a0ff075
Addressed Michael's round 2 comments
Mar 14, 2024
81f4806
Merge remote-tracking branch 'upstream/main' into tiramisu-stats-tiers
Mar 14, 2024
25570e4
Integrate stats changes with most recent IRC changes
Mar 15, 2024
ad01eec
Fixed failing test
Mar 15, 2024
4828327
Addressed Sagar's comments
Mar 15, 2024
a54e545
Merge remote-tracking branch 'upstream/main' into tiramisu-stats-tiers
Mar 19, 2024
4d2da13
Addressed minor comments + stats reset on index deletion
Mar 21, 2024
4f955fd
Merge remote-tracking branch 'upstream/main' into tiramisu-stats-tiers
Mar 21, 2024
5ecdcff
Misc cleanup/reorganization
Mar 26, 2024
fb3baaa
Changed aggregateByLevels to return tree structure
Mar 27, 2024
b5bded2
changed statsholder key to contain whole dimension
Mar 29, 2024
7363dba
spotlessapply
Mar 29, 2024
c31ee32
Fixed IRC dimension names
Apr 1, 2024
e2a9536
Changes StatsHolder to originally store in a tree structure
Apr 2, 2024
7dfe706
Removed CacheStatsDimension
Apr 2, 2024
30f90e9
misc cleanup
Apr 2, 2024
d208412
Addressed Ankit's comments
Apr 3, 2024
c57fd95
Fixed bug in StatsHolder.count()
Apr 3, 2024
5483054
Misc simplifications
Apr 4, 2024
42c326b
changelog
Apr 4, 2024
2ee05fe
Added stats check when removing nodes
Apr 4, 2024
96c531d
deserialization tweak
Apr 4, 2024
1119068
Made increment and removeDimension logic recursive
Apr 4, 2024
a7e6092
Separated out dimensionnode for the two classes, more recursive work
Apr 4, 2024
0474564
Misc cleanup
Apr 4, 2024
2ef1d2a
Addressed minor comments
Apr 5, 2024
5f9589a
misc simplifications
Apr 8, 2024
ae936dc
Addressed Ankit's synchronization + getter comments
Apr 8, 2024
de701ab
Merge remote-tracking branch 'upstream/main' into tiramisu-stats-tiers
Apr 8, 2024
a4e867d
fixed bug in populateStats + API annotations
Apr 8, 2024
c81254f
Added comments
Apr 8, 2024
14f6488
removed commented code
Apr 9, 2024
51523e3
cleanup
Apr 10, 2024
2d2e9b0
Misc simplifications to MultiDimensionCacheStats
Apr 10, 2024
1e2273b
Moved stats removal tests to this PR
Apr 10, 2024
93270f1
Addressed Ankit's comments
Apr 10, 2024
13289d3
Merge remote-tracking branch 'upstream/main' into tiramisu-stats-tiers
Apr 11, 2024
93d53d5
Made reading dimension node in MDCS recursive
Apr 11, 2024
fabb315
Added javadocs for icachekeyserializer
Apr 11, 2024
2c7f431
spotlessApply
Apr 11, 2024
fddd56e
Moved aggregation logic to API PR
Apr 11, 2024
c0c7b8e
Moved some tests from MDCS to StatsHolderTests
Apr 11, 2024
c96f474
Removed serialization logic of MDCS from this PR
Apr 11, 2024
9c037e1
Merge remote-tracking branch 'upstream/main' into tiramisu-stats-tiers
Apr 11, 2024
c0b3dd2
removed unnecessary fns from this pr
Apr 11, 2024
2f59ee7
Renamed classes to make more sense, removed interface
Apr 12, 2024
da9e485
Merge remote-tracking branch 'upstream/main' into tiramisu-stats-tiers
Apr 12, 2024
f60fb08
Renamed snapshot -> ImmutableCacheStats
Apr 12, 2024
f465a22
Fixed flaky ehcache test
Apr 12, 2024
5283470
Addressed Michael's comment
Apr 12, 2024
54e12a3
Improves code coverage
Apr 12, 2024
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
Original file line number Diff line number Diff line change
Expand Up @@ -11,9 +11,11 @@
import org.opensearch.common.annotation.ExperimentalApi;
import org.opensearch.common.cache.CacheType;
import org.opensearch.common.cache.ICache;
import org.opensearch.common.cache.ICacheKey;
import org.opensearch.common.cache.LoadAwareCacheLoader;
import org.opensearch.common.cache.RemovalListener;
import org.opensearch.common.cache.RemovalNotification;
import org.opensearch.common.cache.stats.CacheStats;
import org.opensearch.common.cache.store.config.CacheConfig;
import org.opensearch.common.settings.Setting;
import org.opensearch.common.settings.Settings;
Expand Down Expand Up @@ -44,7 +46,12 @@ public class TieredSpilloverCache<K, V> implements ICache<K, V> {

private final ICache<K, V> diskCache;
private final ICache<K, V> onHeapCache;
private final RemovalListener<K, V> removalListener;

// The listener for removals from the spillover cache as a whole
// TODO: In TSC stats PR, each tier will have its own separate removal listener.
private final RemovalListener<ICacheKey<K>, V> removalListener;
private final CacheStats stats;
private final List<String> dimensionNames;
ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
ReleasableLock readLock = new ReleasableLock(readWriteLock.readLock());
ReleasableLock writeLock = new ReleasableLock(readWriteLock.writeLock());
Expand All @@ -59,9 +66,9 @@ public class TieredSpilloverCache<K, V> implements ICache<K, V> {
this.removalListener = Objects.requireNonNull(builder.removalListener, "Removal listener can't be null");

this.onHeapCache = builder.onHeapCacheFactory.create(
new CacheConfig.Builder<K, V>().setRemovalListener(new RemovalListener<K, V>() {
new CacheConfig.Builder<K, V>().setRemovalListener(new RemovalListener<ICacheKey<K>, V>() {
@Override
public void onRemoval(RemovalNotification<K, V> notification) {
public void onRemoval(RemovalNotification<ICacheKey<K>, V> notification) {
try (ReleasableLock ignore = writeLock.acquire()) {
diskCache.put(notification.getKey(), notification.getValue());
}
Expand All @@ -71,13 +78,25 @@ public void onRemoval(RemovalNotification<K, V> notification) {
.setValueType(builder.cacheConfig.getValueType())
.setSettings(builder.cacheConfig.getSettings())
.setWeigher(builder.cacheConfig.getWeigher())
.setDimensionNames(builder.cacheConfig.getDimensionNames())
.build(),
builder.cacheType,
builder.cacheFactories

);
this.diskCache = builder.diskCacheFactory.create(builder.cacheConfig, builder.cacheType, builder.cacheFactories);
this.diskCache = builder.diskCacheFactory.create(
new CacheConfig.Builder<K, V>().setRemovalListener(removalListener) // TODO: change
.setKeyType(builder.cacheConfig.getKeyType())
.setValueType(builder.cacheConfig.getValueType())
.setSettings(builder.cacheConfig.getSettings())
.setWeigher(builder.cacheConfig.getWeigher())
.build(),
builder.cacheType,
builder.cacheFactories
);
peteralfonsi marked this conversation as resolved.
Show resolved Hide resolved
this.cacheList = Arrays.asList(onHeapCache, diskCache);
this.stats = null; // TODO - in next stats rework PR
this.dimensionNames = builder.cacheConfig.getDimensionNames();
}

// Package private for testing
Expand All @@ -91,19 +110,19 @@ ICache<K, V> getDiskCache() {
}

@Override
public V get(K key) {
public V get(ICacheKey<K> key) {
return getValueFromTieredCache().apply(key);
}

@Override
public void put(K key, V value) {
public void put(ICacheKey<K> key, V value) {
try (ReleasableLock ignore = writeLock.acquire()) {
onHeapCache.put(key, value);
}
}

@Override
public V computeIfAbsent(K key, LoadAwareCacheLoader<K, V> loader) throws Exception {
public V computeIfAbsent(ICacheKey<K> key, LoadAwareCacheLoader<ICacheKey<K>, V> loader) throws Exception {

V cacheValue = getValueFromTieredCache().apply(key);
if (cacheValue == null) {
Expand All @@ -120,7 +139,7 @@ public V computeIfAbsent(K key, LoadAwareCacheLoader<K, V> loader) throws Except
}

@Override
public void invalidate(K key) {
public void invalidate(ICacheKey<K> key) {
// We are trying to invalidate the key from all caches though it would be present in only of them.
// Doing this as we don't know where it is located. We could do a get from both and check that, but what will
// also trigger a hit/miss listener event, so ignoring it for now.
Expand All @@ -146,7 +165,7 @@ public void invalidateAll() {
*/
@SuppressWarnings("unchecked")
@Override
public Iterable<K> keys() {
public Iterable<ICacheKey<K>> keys() {
return Iterables.concat(onHeapCache.keys(), diskCache.keys());
}

Expand Down Expand Up @@ -175,7 +194,12 @@ public void close() throws IOException {
}
}

private Function<K, V> getValueFromTieredCache() {
@Override
public CacheStats stats() {
return stats;
}

private Function<ICacheKey<K>, V> getValueFromTieredCache() {
return key -> {
try (ReleasableLock ignore = readLock.acquire()) {
for (ICache<K, V> cache : cacheList) {
Expand Down Expand Up @@ -253,7 +277,7 @@ public String getCacheName() {
public static class Builder<K, V> {
private ICache.Factory onHeapCacheFactory;
private ICache.Factory diskCacheFactory;
private RemovalListener<K, V> removalListener;
private RemovalListener<ICacheKey<K>, V> removalListener;
private CacheConfig<K, V> cacheConfig;
private CacheType cacheType;
private Map<String, ICache.Factory> cacheFactories;
Expand Down Expand Up @@ -288,7 +312,7 @@ public Builder<K, V> setDiskCacheFactory(ICache.Factory diskCacheFactory) {
* @param removalListener Removal listener
* @return builder
*/
public Builder<K, V> setRemovalListener(RemovalListener<K, V> removalListener) {
public Builder<K, V> setRemovalListener(RemovalListener<ICacheKey<K>, V> removalListener) {
this.removalListener = removalListener;
return this;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -10,10 +10,12 @@

import org.opensearch.common.cache.CacheType;
import org.opensearch.common.cache.ICache;
import org.opensearch.common.cache.ICacheKey;
import org.opensearch.common.cache.LoadAwareCacheLoader;
import org.opensearch.common.cache.RemovalListener;
import org.opensearch.common.cache.RemovalNotification;
import org.opensearch.common.cache.RemovalReason;
import org.opensearch.common.cache.stats.CacheStats;
import org.opensearch.common.cache.store.builders.ICacheBuilder;
import org.opensearch.common.cache.store.config.CacheConfig;

Expand All @@ -22,27 +24,27 @@

public class MockDiskCache<K, V> implements ICache<K, V> {

Map<K, V> cache;
Map<ICacheKey<K>, V> cache;
int maxSize;
long delay;

private final RemovalListener<K, V> removalListener;
private final RemovalListener<ICacheKey<K>, V> removalListener;

public MockDiskCache(int maxSize, long delay, RemovalListener<K, V> removalListener) {
public MockDiskCache(int maxSize, long delay, RemovalListener<ICacheKey<K>, V> removalListener) {
this.maxSize = maxSize;
this.delay = delay;
this.removalListener = removalListener;
this.cache = new ConcurrentHashMap<K, V>();
this.cache = new ConcurrentHashMap<ICacheKey<K>, V>();
}

@Override
public V get(K key) {
public V get(ICacheKey<K> key) {
V value = cache.get(key);
return value;
}

@Override
public void put(K key, V value) {
public void put(ICacheKey<K> key, V value) {
if (this.cache.size() >= maxSize) { // For simplification
this.removalListener.onRemoval(new RemovalNotification<>(key, value, RemovalReason.EVICTED));
}
Expand All @@ -55,7 +57,7 @@ public void put(K key, V value) {
}

@Override
public V computeIfAbsent(K key, LoadAwareCacheLoader<K, V> loader) {
public V computeIfAbsent(ICacheKey<K> key, LoadAwareCacheLoader<ICacheKey<K>, V> loader) {
V value = cache.computeIfAbsent(key, key1 -> {
try {
return loader.load(key);
Expand All @@ -67,7 +69,7 @@ public V computeIfAbsent(K key, LoadAwareCacheLoader<K, V> loader) {
}

@Override
public void invalidate(K key) {
public void invalidate(ICacheKey<K> key) {
this.cache.remove(key);
}

Expand All @@ -77,7 +79,7 @@ public void invalidateAll() {
}

@Override
public Iterable<K> keys() {
public Iterable<ICacheKey<K>> keys() {
return this.cache.keySet();
}

Expand All @@ -89,6 +91,11 @@ public long count() {
@Override
public void refresh() {}

@Override
public CacheStats stats() {
return null;
}

@Override
public void close() {

Expand Down
Loading
Loading