From dc520dcb9751ea3116992e9edbe097724ba670d5 Mon Sep 17 00:00:00 2001 From: Peter Alfonsi Date: Thu, 2 May 2024 17:29:07 -0700 Subject: [PATCH] [Tiered Caching] Adds stats implementation for TieredSpilloverCache (#13236) Stats rework part 4 of 4 --------- Signed-off-by: Peter Alfonsi Co-authored-by: Peter Alfonsi --- CHANGELOG.md | 1 + .../TieredSpilloverCacheIT.java | 2 +- .../common/tier/TieredSpilloverCache.java | 250 ++++++++++++---- .../tier/TieredSpilloverCacheStatsHolder.java | 161 +++++++++++ .../cache/common/tier/MockDiskCache.java | 27 +- .../tier/TieredSpilloverCacheTests.java | 266 ++++++++++++++++-- .../cache/store/disk/EhcacheDiskCache.java | 11 +- .../store/disk/EhCacheDiskCacheTests.java | 32 +++ .../cache/stats/DefaultCacheStatsHolder.java | 30 +- .../cache/store/OpenSearchOnHeapCache.java | 18 +- .../cache/store/builders/ICacheBuilder.java | 11 + .../cache/store/config/CacheConfig.java | 13 + .../store/OpenSearchOnHeapCacheTests.java | 33 ++- 13 files changed, 755 insertions(+), 100 deletions(-) create mode 100644 modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsHolder.java diff --git a/CHANGELOG.md b/CHANGELOG.md index ba55a80f5d1d6..cdaa1904e2522 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -22,6 +22,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [Tiered Caching] Add dimension-based stats to ICache implementations. ([#12531](https://github.com/opensearch-project/OpenSearch/pull/12531)) - Add changes for overriding remote store and replication settings during snapshot restore. ([#11868](https://github.com/opensearch-project/OpenSearch/pull/11868)) - Add an individual setting of rate limiter for segment replication ([#12959](https://github.com/opensearch-project/OpenSearch/pull/12959)) +- [Tiered Caching] Add dimension-based stats to TieredSpilloverCache ([#13236](https://github.com/opensearch-project/OpenSearch/pull/13236)) - [Tiered Caching] Expose new cache stats API ([#13237](https://github.com/opensearch-project/OpenSearch/pull/13237)) - [Streaming Indexing] Ensure support of the new transport by security plugin ([#13174](https://github.com/opensearch-project/OpenSearch/pull/13174)) - Add cluster setting to dynamically configure the buckets for filter rewrite optimization. ([#13179](https://github.com/opensearch-project/OpenSearch/pull/13179)) diff --git a/modules/cache-common/src/internalClusterTest/java/org.opensearch.cache.common.tier/TieredSpilloverCacheIT.java b/modules/cache-common/src/internalClusterTest/java/org.opensearch.cache.common.tier/TieredSpilloverCacheIT.java index cbe16a690c104..bfc184cff0566 100644 --- a/modules/cache-common/src/internalClusterTest/java/org.opensearch.cache.common.tier/TieredSpilloverCacheIT.java +++ b/modules/cache-common/src/internalClusterTest/java/org.opensearch.cache.common.tier/TieredSpilloverCacheIT.java @@ -550,7 +550,7 @@ public MockDiskCachePlugin() {} @Override public Map getCacheFactoryMap() { - return Map.of(MockDiskCache.MockDiskCacheFactory.NAME, new MockDiskCache.MockDiskCacheFactory(0, 1000)); + return Map.of(MockDiskCache.MockDiskCacheFactory.NAME, new MockDiskCache.MockDiskCacheFactory(0, 1000, false)); } @Override diff --git a/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCache.java b/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCache.java index bca81ebd958ce..9942651ccdd67 100644 --- a/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCache.java +++ b/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCache.java @@ -20,6 +20,7 @@ import org.opensearch.common.cache.policy.CachedQueryResult; import org.opensearch.common.cache.stats.ImmutableCacheStatsHolder; import org.opensearch.common.cache.store.config.CacheConfig; +import org.opensearch.common.collect.Tuple; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; @@ -34,12 +35,16 @@ import java.util.Map; import java.util.NoSuchElementException; import java.util.Objects; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Function; import java.util.function.Predicate; +import java.util.function.ToLongBiFunction; import static org.opensearch.cache.common.tier.TieredSpilloverCacheSettings.DISK_CACHE_ENABLED_SETTING_MAP; +import static org.opensearch.cache.common.tier.TieredSpilloverCacheStatsHolder.TIER_DIMENSION_VALUE_DISK; +import static org.opensearch.cache.common.tier.TieredSpilloverCacheStatsHolder.TIER_DIMENSION_VALUE_ON_HEAP; /** * This cache spillover the evicted items from heap tier to disk tier. All the new items are first cached on heap @@ -60,9 +65,17 @@ public class TieredSpilloverCache implements ICache { private final ICache diskCache; private final ICache onHeapCache; - // 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. + // Removal listeners for the individual tiers + private final RemovalListener, V> onDiskRemovalListener; + private final RemovalListener, V> onHeapRemovalListener; + + // Removal listener from the spillover cache as a whole private final RemovalListener, V> removalListener; + + // In future we want to just read the stats from the individual tiers' statsHolder objects, but this isn't + // possible right now because of the way computeIfAbsent is implemented. + private final TieredSpilloverCacheStatsHolder statsHolder; + private ToLongBiFunction, V> weigher; private final List dimensionNames; ReadWriteLock readWriteLock = new ReentrantReadWriteLock(); ReleasableLock readLock = new ReleasableLock(readWriteLock.readLock()); @@ -70,7 +83,7 @@ public class TieredSpilloverCache implements ICache { /** * Maintains caching tiers in ascending order of cache latency. */ - private final Map, Boolean> caches; + private final Map, TierInfo> caches; private final List> policies; TieredSpilloverCache(Builder builder) { @@ -80,21 +93,12 @@ public class TieredSpilloverCache implements ICache { Objects.requireNonNull(builder.cacheConfig.getClusterSettings(), "cluster settings can't be null"); this.removalListener = Objects.requireNonNull(builder.removalListener, "Removal listener can't be null"); + this.onHeapRemovalListener = new HeapTierRemovalListener(this); + this.onDiskRemovalListener = new DiskTierRemovalListener(this); + this.weigher = Objects.requireNonNull(builder.cacheConfig.getWeigher(), "Weigher can't be null"); + this.onHeapCache = builder.onHeapCacheFactory.create( - new CacheConfig.Builder().setRemovalListener(new RemovalListener, V>() { - @Override - public void onRemoval(RemovalNotification, V> notification) { - try (ReleasableLock ignore = writeLock.acquire()) { - if (caches.get(diskCache) - && SPILLOVER_REMOVAL_REASONS.contains(notification.getRemovalReason()) - && evaluatePolicies(notification.getValue())) { - diskCache.put(notification.getKey(), notification.getValue()); - } else { - removalListener.onRemoval(notification); - } - } - } - }) + new CacheConfig.Builder().setRemovalListener(onHeapRemovalListener) .setKeyType(builder.cacheConfig.getKeyType()) .setValueType(builder.cacheConfig.getValueType()) .setSettings(builder.cacheConfig.getSettings()) @@ -103,18 +107,33 @@ && evaluatePolicies(notification.getValue())) { .setMaxSizeInBytes(builder.cacheConfig.getMaxSizeInBytes()) .setExpireAfterAccess(builder.cacheConfig.getExpireAfterAccess()) .setClusterSettings(builder.cacheConfig.getClusterSettings()) + .setStatsTrackingEnabled(false) .build(), builder.cacheType, builder.cacheFactories ); - this.diskCache = builder.diskCacheFactory.create(builder.cacheConfig, builder.cacheType, builder.cacheFactories); + this.diskCache = builder.diskCacheFactory.create( + new CacheConfig.Builder().setRemovalListener(onDiskRemovalListener) + .setKeyType(builder.cacheConfig.getKeyType()) + .setValueType(builder.cacheConfig.getValueType()) + .setSettings(builder.cacheConfig.getSettings()) + .setWeigher(builder.cacheConfig.getWeigher()) + .setDimensionNames(builder.cacheConfig.getDimensionNames()) + .setStatsTrackingEnabled(false) + .build(), + builder.cacheType, + builder.cacheFactories + ); Boolean isDiskCacheEnabled = DISK_CACHE_ENABLED_SETTING_MAP.get(builder.cacheType).get(builder.cacheConfig.getSettings()); - LinkedHashMap, Boolean> cacheListMap = new LinkedHashMap<>(); - cacheListMap.put(onHeapCache, true); - cacheListMap.put(diskCache, isDiskCacheEnabled); + LinkedHashMap, TierInfo> cacheListMap = new LinkedHashMap<>(); + cacheListMap.put(onHeapCache, new TierInfo(true, TIER_DIMENSION_VALUE_ON_HEAP)); + cacheListMap.put(diskCache, new TierInfo(isDiskCacheEnabled, TIER_DIMENSION_VALUE_DISK)); this.caches = Collections.synchronizedMap(cacheListMap); + this.dimensionNames = builder.cacheConfig.getDimensionNames(); + // Pass "tier" as the innermost dimension name, in addition to whatever dimensions are specified for the cache as a whole + this.statsHolder = new TieredSpilloverCacheStatsHolder(dimensionNames, isDiskCacheEnabled); this.policies = builder.policies; // Will never be null; builder initializes it to an empty list builder.cacheConfig.getClusterSettings() .addSettingsUpdateConsumer(DISK_CACHE_ENABLED_SETTING_MAP.get(builder.cacheType), this::enableDisableDiskCache); @@ -134,25 +153,38 @@ ICache getDiskCache() { void enableDisableDiskCache(Boolean isDiskCacheEnabled) { // When disk cache is disabled, we are not clearing up the disk cache entries yet as that should be part of // separate cache/clear API. - this.caches.put(diskCache, isDiskCacheEnabled); + this.caches.put(diskCache, new TierInfo(isDiskCacheEnabled, TIER_DIMENSION_VALUE_DISK)); + this.statsHolder.setDiskCacheEnabled(isDiskCacheEnabled); } @Override public V get(ICacheKey key) { - return getValueFromTieredCache().apply(key); + Tuple cacheValueTuple = getValueFromTieredCache(true).apply(key); + if (cacheValueTuple == null) { + return null; + } + return cacheValueTuple.v1(); } @Override public void put(ICacheKey key, V value) { try (ReleasableLock ignore = writeLock.acquire()) { onHeapCache.put(key, value); + updateStatsOnPut(TIER_DIMENSION_VALUE_ON_HEAP, key, value); } } @Override public V computeIfAbsent(ICacheKey key, LoadAwareCacheLoader, V> loader) throws Exception { - V cacheValue = getValueFromTieredCache().apply(key); - if (cacheValue == null) { + // Don't capture stats in the initial getValueFromTieredCache(). If we have concurrent requests for the same key, + // and it only has to be loaded one time, we should report one miss and the rest hits. But, if we do stats in + // getValueFromTieredCache(), + // we will see all misses. Instead, handle stats in computeIfAbsent(). + Tuple cacheValueTuple = getValueFromTieredCache(false).apply(key); + List heapDimensionValues = statsHolder.getDimensionsWithTierValue(key.dimensions, TIER_DIMENSION_VALUE_ON_HEAP); + List diskDimensionValues = statsHolder.getDimensionsWithTierValue(key.dimensions, TIER_DIMENSION_VALUE_DISK); + + if (cacheValueTuple == null) { // Add the value to the onHeap cache. We are calling computeIfAbsent which does another get inside. // This is needed as there can be many requests for the same key at the same time and we only want to load // the value once. @@ -160,19 +192,49 @@ public V computeIfAbsent(ICacheKey key, LoadAwareCacheLoader, V> try (ReleasableLock ignore = writeLock.acquire()) { value = onHeapCache.computeIfAbsent(key, loader); } + // Handle stats + if (loader.isLoaded()) { + // The value was just computed and added to the cache by this thread. Register a miss for the heap cache, and the disk cache + // if present + updateStatsOnPut(TIER_DIMENSION_VALUE_ON_HEAP, key, value); + statsHolder.incrementMisses(heapDimensionValues); + if (caches.get(diskCache).isEnabled()) { + statsHolder.incrementMisses(diskDimensionValues); + } + } else { + // Another thread requesting this key already loaded the value. Register a hit for the heap cache + statsHolder.incrementHits(heapDimensionValues); + } return value; + } else { + // Handle stats for an initial hit from getValueFromTieredCache() + if (cacheValueTuple.v2().equals(TIER_DIMENSION_VALUE_ON_HEAP)) { + // A hit for the heap tier + statsHolder.incrementHits(heapDimensionValues); + } else if (cacheValueTuple.v2().equals(TIER_DIMENSION_VALUE_DISK)) { + // Miss for the heap tier, hit for the disk tier + statsHolder.incrementMisses(heapDimensionValues); + statsHolder.incrementHits(diskDimensionValues); + } } - return cacheValue; + return cacheValueTuple.v1(); } @Override public void invalidate(ICacheKey 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 count hits/misses stats, so ignoring it for now. - try (ReleasableLock ignore = writeLock.acquire()) { - for (Map.Entry, Boolean> cacheEntry : caches.entrySet()) { - cacheEntry.getKey().invalidate(key); + // also trigger a hit/miss listener event, so ignoring it for now. + // We don't update stats here, as this is handled by the removal listeners for the tiers. + for (Map.Entry, TierInfo> cacheEntry : caches.entrySet()) { + if (key.getDropStatsForDimensions()) { + List dimensionValues = statsHolder.getDimensionsWithTierValue(key.dimensions, cacheEntry.getValue().tierName); + statsHolder.removeDimensions(dimensionValues); + } + if (key.key != null) { + try (ReleasableLock ignore = writeLock.acquire()) { + cacheEntry.getKey().invalidate(key); + } } } } @@ -180,10 +242,11 @@ public void invalidate(ICacheKey key) { @Override public void invalidateAll() { try (ReleasableLock ignore = writeLock.acquire()) { - for (Map.Entry, Boolean> cacheEntry : caches.entrySet()) { + for (Map.Entry, TierInfo> cacheEntry : caches.entrySet()) { cacheEntry.getKey().invalidateAll(); } } + statsHolder.reset(); } /** @@ -194,7 +257,7 @@ public void invalidateAll() { @Override public Iterable> keys() { List>> iterableList = new ArrayList<>(); - for (Map.Entry, Boolean> cacheEntry : caches.entrySet()) { + for (Map.Entry, TierInfo> cacheEntry : caches.entrySet()) { iterableList.add(cacheEntry.getKey().keys()); } Iterable>[] iterables = (Iterable>[]) iterableList.toArray(new Iterable[0]); @@ -203,19 +266,15 @@ public Iterable> keys() { @Override public long count() { - long count = 0; - for (Map.Entry, Boolean> cacheEntry : caches.entrySet()) { - // Count for all the tiers irrespective of whether they are enabled or not. As eventually - // this will turn to zero once cache is cleared up either via invalidation or manually. - count += cacheEntry.getKey().count(); - } - return count; + // Count for all the tiers irrespective of whether they are enabled or not. As eventually + // this will turn to zero once cache is cleared up either via invalidation or manually. + return statsHolder.count(); } @Override public void refresh() { try (ReleasableLock ignore = writeLock.acquire()) { - for (Map.Entry, Boolean> cacheEntry : caches.entrySet()) { + for (Map.Entry, TierInfo> cacheEntry : caches.entrySet()) { cacheEntry.getKey().refresh(); } } @@ -223,7 +282,7 @@ public void refresh() { @Override public void close() throws IOException { - for (Map.Entry, Boolean> cacheEntry : caches.entrySet()) { + for (Map.Entry, TierInfo> cacheEntry : caches.entrySet()) { // Close all the caches here irrespective of whether they are enabled or not. cacheEntry.getKey().close(); } @@ -231,25 +290,76 @@ public void close() throws IOException { @Override public ImmutableCacheStatsHolder stats(String[] levels) { - return null; // TODO: in TSC stats PR + return statsHolder.getImmutableCacheStatsHolder(levels); } - private Function, V> getValueFromTieredCache() { + /** + * Get a value from the tiered cache, and the name of the tier it was found in. + * @param captureStats Whether to record hits/misses for this call of the function + * @return A tuple of the value and the name of the tier it was found in. + */ + private Function, Tuple> getValueFromTieredCache(boolean captureStats) { return key -> { try (ReleasableLock ignore = readLock.acquire()) { - for (Map.Entry, Boolean> cacheEntry : caches.entrySet()) { - if (cacheEntry.getValue()) { + for (Map.Entry, TierInfo> cacheEntry : caches.entrySet()) { + if (cacheEntry.getValue().isEnabled()) { V value = cacheEntry.getKey().get(key); + // Get the tier value corresponding to this cache + String tierValue = cacheEntry.getValue().tierName; + List dimensionValues = statsHolder.getDimensionsWithTierValue(key.dimensions, tierValue); if (value != null) { - return value; + if (captureStats) { + statsHolder.incrementHits(dimensionValues); + } + return new Tuple<>(value, tierValue); + } else if (captureStats) { + statsHolder.incrementMisses(dimensionValues); } } } + return null; } - return null; }; } + void handleRemovalFromHeapTier(RemovalNotification, V> notification) { + ICacheKey key = notification.getKey(); + boolean wasEvicted = SPILLOVER_REMOVAL_REASONS.contains(notification.getRemovalReason()); + if (caches.get(diskCache).isEnabled() && wasEvicted && evaluatePolicies(notification.getValue())) { + try (ReleasableLock ignore = writeLock.acquire()) { + diskCache.put(key, notification.getValue()); // spill over to the disk tier and increment its stats + } + updateStatsOnPut(TIER_DIMENSION_VALUE_DISK, key, notification.getValue()); + } else { + // If the value is not going to the disk cache, send this notification to the TSC's removal listener + // as the value is leaving the TSC entirely + removalListener.onRemoval(notification); + } + updateStatsOnRemoval(TIER_DIMENSION_VALUE_ON_HEAP, wasEvicted, key, notification.getValue()); + } + + void handleRemovalFromDiskTier(RemovalNotification, V> notification) { + // Values removed from the disk tier leave the TSC entirely + removalListener.onRemoval(notification); + boolean wasEvicted = SPILLOVER_REMOVAL_REASONS.contains(notification.getRemovalReason()); + updateStatsOnRemoval(TIER_DIMENSION_VALUE_DISK, wasEvicted, notification.getKey(), notification.getValue()); + } + + void updateStatsOnRemoval(String removedFromTierValue, boolean wasEvicted, ICacheKey key, V value) { + List dimensionValues = statsHolder.getDimensionsWithTierValue(key.dimensions, removedFromTierValue); + if (wasEvicted) { + statsHolder.incrementEvictions(dimensionValues); + } + statsHolder.decrementItems(dimensionValues); + statsHolder.decrementSizeInBytes(dimensionValues, weigher.applyAsLong(key, value)); + } + + void updateStatsOnPut(String destinationTierValue, ICacheKey key, V value) { + List dimensionValues = statsHolder.getDimensionsWithTierValue(key.dimensions, destinationTierValue); + statsHolder.incrementItems(dimensionValues); + statsHolder.incrementSizeInBytes(dimensionValues, weigher.applyAsLong(key, value)); + } + boolean evaluatePolicies(V value) { for (Predicate policy : policies) { if (!policy.test(value)) { @@ -259,6 +369,38 @@ boolean evaluatePolicies(V value) { return true; } + /** + * A class which receives removal events from the heap tier. + */ + private class HeapTierRemovalListener implements RemovalListener, V> { + private final TieredSpilloverCache tsc; + + HeapTierRemovalListener(TieredSpilloverCache tsc) { + this.tsc = tsc; + } + + @Override + public void onRemoval(RemovalNotification, V> notification) { + tsc.handleRemovalFromHeapTier(notification); + } + } + + /** + * A class which receives removal events from the disk tier. + */ + private class DiskTierRemovalListener implements RemovalListener, V> { + private final TieredSpilloverCache tsc; + + DiskTierRemovalListener(TieredSpilloverCache tsc) { + this.tsc = tsc; + } + + @Override + public void onRemoval(RemovalNotification, V> notification) { + tsc.handleRemovalFromDiskTier(notification); + } + } + /** * ConcatenatedIterables which combines cache iterables and supports remove() functionality as well if underlying * iterator supports it. @@ -320,6 +462,20 @@ public void remove() { } } + private class TierInfo { + AtomicBoolean isEnabled; + final String tierName; + + TierInfo(boolean isEnabled, String tierName) { + this.isEnabled = new AtomicBoolean(isEnabled); + this.tierName = tierName; + } + + boolean isEnabled() { + return isEnabled.get(); + } + } + /** * Factory to create TieredSpilloverCache objects. */ diff --git a/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsHolder.java b/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsHolder.java new file mode 100644 index 0000000000000..d17059e8dee94 --- /dev/null +++ b/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsHolder.java @@ -0,0 +1,161 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.cache.common.tier; + +import org.opensearch.common.cache.stats.DefaultCacheStatsHolder; + +import java.util.ArrayList; +import java.util.List; +import java.util.function.Consumer; + +/** + * A tier-aware version of DefaultCacheStatsHolder. Overrides the incrementer functions, as we can't just add the on-heap + * and disk stats to get a total for the cache as a whole. If the disk tier is present, the total hits, size, and entries + * should be the sum of both tiers' values, but the total misses and evictions should be the disk tier's values. + * When the disk tier isn't present, on-heap misses and evictions should contribute to the total. + * + * For example, if the heap tier has 5 misses and the disk tier has 4, the total cache has had 4 misses, not 9. + * The same goes for evictions. Other stats values add normally. + * + * This means for misses and evictions, if we are incrementing for the on-heap tier and the disk tier is present, + * we have to increment only the leaf nodes corresponding to the on-heap tier itself, and not its ancestors, + * which correspond to totals including both tiers. If the disk tier is not present, we do increment the ancestor nodes. + */ +public class TieredSpilloverCacheStatsHolder extends DefaultCacheStatsHolder { + + /** Whether the disk cache is currently enabled. */ + private boolean diskCacheEnabled; + + // Common values used for tier dimension + + /** The name for the tier dimension. */ + public static final String TIER_DIMENSION_NAME = "tier"; + + /** Dimension value for on-heap cache, like OpenSearchOnHeapCache.*/ + public static final String TIER_DIMENSION_VALUE_ON_HEAP = "on_heap"; + + /** Dimension value for on-disk cache, like EhcacheDiskCache. */ + public static final String TIER_DIMENSION_VALUE_DISK = "disk"; + + /** + * Constructor for the stats holder. + * @param originalDimensionNames the original dimension names, not including TIER_DIMENSION_NAME + * @param diskCacheEnabled whether the disk tier starts out enabled + */ + public TieredSpilloverCacheStatsHolder(List originalDimensionNames, boolean diskCacheEnabled) { + super( + getDimensionNamesWithTier(originalDimensionNames), + TieredSpilloverCache.TieredSpilloverCacheFactory.TIERED_SPILLOVER_CACHE_NAME + ); + this.diskCacheEnabled = diskCacheEnabled; + } + + private static List getDimensionNamesWithTier(List dimensionNames) { + List dimensionNamesWithTier = new ArrayList<>(dimensionNames); + dimensionNamesWithTier.add(TIER_DIMENSION_NAME); + return dimensionNamesWithTier; + } + + /** + * Add tierValue to the end of a copy of the initial dimension values, so they can appropriately be used in this stats holder. + */ + List getDimensionsWithTierValue(List initialDimensions, String tierValue) { + List result = new ArrayList<>(initialDimensions); + result.add(tierValue); + return result; + } + + private String validateTierDimensionValue(List dimensionValues) { + String tierDimensionValue = dimensionValues.get(dimensionValues.size() - 1); + assert tierDimensionValue.equals(TIER_DIMENSION_VALUE_ON_HEAP) || tierDimensionValue.equals(TIER_DIMENSION_VALUE_DISK) + : "Invalid tier dimension value"; + return tierDimensionValue; + } + + @Override + public void incrementHits(List dimensionValues) { + validateTierDimensionValue(dimensionValues); + // Hits from either tier should be included in the total values. + super.incrementHits(dimensionValues); + } + + @Override + public void incrementMisses(List dimensionValues) { + final String tierValue = validateTierDimensionValue(dimensionValues); + + // If the disk tier is present, only misses from the disk tier should be included in total values. + Consumer missIncrementer = (node) -> { + if (tierValue.equals(TIER_DIMENSION_VALUE_ON_HEAP) && diskCacheEnabled) { + // If on-heap tier, increment only the leaf node corresponding to the on heap values; not the total values in its parent + // nodes + if (node.isAtLowestLevel()) { + node.incrementMisses(); + } + } else { + // If disk tier, or on-heap tier with a disabled disk tier, increment the leaf node and its parents + node.incrementMisses(); + } + }; + internalIncrement(dimensionValues, missIncrementer, true); + } + + @Override + public void incrementEvictions(List dimensionValues) { + final String tierValue = validateTierDimensionValue(dimensionValues); + + // If the disk tier is present, only evictions from the disk tier should be included in total values. + Consumer evictionsIncrementer = (node) -> { + if (tierValue.equals(TIER_DIMENSION_VALUE_ON_HEAP) && diskCacheEnabled) { + // If on-heap tier, increment only the leaf node corresponding to the on heap values; not the total values in its parent + // nodes + if (node.isAtLowestLevel()) { + node.incrementEvictions(); + } + } else { + // If disk tier, or on-heap tier with a disabled disk tier, increment the leaf node and its parents + node.incrementEvictions(); + } + }; + internalIncrement(dimensionValues, evictionsIncrementer, true); + } + + @Override + public void incrementSizeInBytes(List dimensionValues, long amountBytes) { + validateTierDimensionValue(dimensionValues); + // Size from either tier should be included in the total values. + super.incrementSizeInBytes(dimensionValues, amountBytes); + } + + // For decrements, we should not create nodes if they are absent. This protects us from erroneously decrementing values for keys + // which have been entirely deleted, for example in an async removal listener. + @Override + public void decrementSizeInBytes(List dimensionValues, long amountBytes) { + validateTierDimensionValue(dimensionValues); + // Size from either tier should be included in the total values. + super.decrementSizeInBytes(dimensionValues, amountBytes); + } + + @Override + public void incrementItems(List dimensionValues) { + validateTierDimensionValue(dimensionValues); + // Entries from either tier should be included in the total values. + super.incrementItems(dimensionValues); + } + + @Override + public void decrementItems(List dimensionValues) { + validateTierDimensionValue(dimensionValues); + // Entries from either tier should be included in the total values. + super.decrementItems(dimensionValues); + } + + void setDiskCacheEnabled(boolean diskCacheEnabled) { + this.diskCacheEnabled = diskCacheEnabled; + } +} diff --git a/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/MockDiskCache.java b/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/MockDiskCache.java index 8aed3f004e7b2..2058faa5181b1 100644 --- a/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/MockDiskCache.java +++ b/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/MockDiskCache.java @@ -16,11 +16,15 @@ import org.opensearch.common.cache.RemovalNotification; import org.opensearch.common.cache.RemovalReason; import org.opensearch.common.cache.serializer.Serializer; +import org.opensearch.common.cache.stats.CacheStatsHolder; +import org.opensearch.common.cache.stats.DefaultCacheStatsHolder; import org.opensearch.common.cache.stats.ImmutableCacheStatsHolder; +import org.opensearch.common.cache.stats.NoopCacheStatsHolder; import org.opensearch.common.cache.store.builders.ICacheBuilder; import org.opensearch.common.cache.store.config.CacheConfig; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.NoSuchElementException; import java.util.concurrent.ConcurrentHashMap; @@ -32,12 +36,19 @@ public class MockDiskCache implements ICache { long delay; private final RemovalListener, V> removalListener; + private final CacheStatsHolder statsHolder; // Only update for number of entries; this is only used to test statsTrackingEnabled logic + // in TSC - public MockDiskCache(int maxSize, long delay, RemovalListener, V> removalListener) { + public MockDiskCache(int maxSize, long delay, RemovalListener, V> removalListener, boolean statsTrackingEnabled) { this.maxSize = maxSize; this.delay = delay; this.removalListener = removalListener; this.cache = new ConcurrentHashMap, V>(); + if (statsTrackingEnabled) { + this.statsHolder = new DefaultCacheStatsHolder(List.of(), "mock_disk_cache"); + } else { + this.statsHolder = NoopCacheStatsHolder.getInstance(); + } } @Override @@ -50,6 +61,7 @@ public V get(ICacheKey key) { public void put(ICacheKey key, V value) { if (this.cache.size() >= maxSize) { // For simplification this.removalListener.onRemoval(new RemovalNotification<>(key, value, RemovalReason.EVICTED)); + this.statsHolder.decrementItems(List.of()); } try { Thread.sleep(delay); @@ -57,6 +69,7 @@ public void put(ICacheKey key, V value) { throw new RuntimeException(e); } this.cache.put(key, value); + this.statsHolder.incrementItems(List.of()); } @Override @@ -73,6 +86,7 @@ public V computeIfAbsent(ICacheKey key, LoadAwareCacheLoader, V> @Override public void invalidate(ICacheKey key) { + removalListener.onRemoval(new RemovalNotification<>(key, cache.get(key), RemovalReason.INVALIDATED)); this.cache.remove(key); } @@ -96,7 +110,9 @@ public void refresh() {} @Override public ImmutableCacheStatsHolder stats() { - return null; + // To allow testing of statsTrackingEnabled logic in TSC, return a dummy ImmutableCacheStatsHolder with the + // right number of entries, unless statsTrackingEnabled is false + return statsHolder.getImmutableCacheStatsHolder(null); } @Override @@ -114,10 +130,12 @@ public static class MockDiskCacheFactory implements Factory { public static final String NAME = "mockDiskCache"; final long delay; final int maxSize; + final boolean statsTrackingEnabled; - public MockDiskCacheFactory(long delay, int maxSize) { + public MockDiskCacheFactory(long delay, int maxSize, boolean statsTrackingEnabled) { this.delay = delay; this.maxSize = maxSize; + this.statsTrackingEnabled = statsTrackingEnabled; } @Override @@ -128,6 +146,7 @@ public ICache create(CacheConfig config, CacheType cacheType, .setMaxSize(maxSize) .setDeliberateDelay(delay) .setRemovalListener(config.getRemovalListener()) + .setStatsTrackingEnabled(config.getStatsTrackingEnabled()) .build(); } @@ -146,7 +165,7 @@ public static class Builder extends ICacheBuilder { @Override public ICache build() { - return new MockDiskCache(this.maxSize, this.delay, this.getRemovalListener()); + return new MockDiskCache(this.maxSize, this.delay, this.getRemovalListener(), getStatsTrackingEnabled()); } public Builder setMaxSize(int maxSize) { diff --git a/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/TieredSpilloverCacheTests.java b/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/TieredSpilloverCacheTests.java index 1ecb63414dc68..6d5ee91326338 100644 --- a/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/TieredSpilloverCacheTests.java +++ b/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/TieredSpilloverCacheTests.java @@ -8,6 +8,7 @@ package org.opensearch.cache.common.tier; +import org.opensearch.common.Randomness; import org.opensearch.common.cache.CacheType; import org.opensearch.common.cache.ICache; import org.opensearch.common.cache.ICacheKey; @@ -16,6 +17,8 @@ import org.opensearch.common.cache.RemovalNotification; import org.opensearch.common.cache.policy.CachedQueryResult; import org.opensearch.common.cache.settings.CacheSettings; +import org.opensearch.common.cache.stats.ImmutableCacheStats; +import org.opensearch.common.cache.stats.ImmutableCacheStatsHolder; import org.opensearch.common.cache.store.OpenSearchOnHeapCache; import org.opensearch.common.cache.store.config.CacheConfig; import org.opensearch.common.cache.store.settings.OpenSearchOnHeapCacheSettings; @@ -28,11 +31,13 @@ import org.opensearch.test.OpenSearchTestCase; import org.junit.Before; +import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Random; import java.util.UUID; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; @@ -44,10 +49,12 @@ import static org.opensearch.cache.common.tier.TieredSpilloverCacheSettings.DISK_CACHE_ENABLED_SETTING_MAP; import static org.opensearch.cache.common.tier.TieredSpilloverCacheSettings.TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP; +import static org.opensearch.cache.common.tier.TieredSpilloverCacheStatsHolder.TIER_DIMENSION_NAME; +import static org.opensearch.cache.common.tier.TieredSpilloverCacheStatsHolder.TIER_DIMENSION_VALUE_DISK; +import static org.opensearch.cache.common.tier.TieredSpilloverCacheStatsHolder.TIER_DIMENSION_VALUE_ON_HEAP; import static org.opensearch.common.cache.store.settings.OpenSearchOnHeapCacheSettings.MAXIMUM_SIZE_IN_BYTES_KEY; public class TieredSpilloverCacheTests extends OpenSearchTestCase { - // TODO: TSC stats impl is in a future PR. Parts of tests which use stats values are missing for now. static final List dimensionNames = List.of("dim1", "dim2", "dim3"); private ClusterSettings clusterSettings; @@ -89,6 +96,9 @@ public void testComputeIfAbsentWithoutAnyOnHeapCacheEviction() throws Exception tieredSpilloverCache.computeIfAbsent(key, tieredCacheLoader); } assertEquals(0, removalListener.evictionsMetric.count()); + assertEquals(numOfItems1, getMissesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(0, getHitsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(0, getEvictionsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); // Try to hit cache again with some randomization. int numOfItems2 = randomIntBetween(1, onHeapCacheSize / 2 - 1); @@ -107,6 +117,13 @@ public void testComputeIfAbsentWithoutAnyOnHeapCacheEviction() throws Exception } } assertEquals(0, removalListener.evictionsMetric.count()); + assertEquals(cacheHit, getHitsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(numOfItems1 + cacheMiss, getMissesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(0, getEvictionsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + + assertEquals(0, getHitsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); + assertEquals(numOfItems1 + cacheMiss, getMissesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); + assertEquals(0, getEvictionsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); } public void testComputeIfAbsentWithFactoryBasedCacheCreation() throws Exception { @@ -160,7 +177,7 @@ public void testComputeIfAbsentWithFactoryBasedCacheCreation() throws Exception OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory.NAME, new OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory(), MockDiskCache.MockDiskCacheFactory.NAME, - new MockDiskCache.MockDiskCacheFactory(0, randomIntBetween(100, 300)) + new MockDiskCache.MockDiskCacheFactory(0, randomIntBetween(100, 300), false) ) ); @@ -174,12 +191,25 @@ public void testComputeIfAbsentWithFactoryBasedCacheCreation() throws Exception LoadAwareCacheLoader, String> tieredCacheLoader = getLoadAwareCacheLoader(); tieredSpilloverCache.computeIfAbsent(getICacheKey(key), tieredCacheLoader); } + + int expectedDiskEntries = numOfItems1 - onHeapCacheSize; tieredSpilloverCache.getOnHeapCache().keys().forEach(onHeapKeys::add); tieredSpilloverCache.getDiskCache().keys().forEach(diskTierKeys::add); - // Verify on heap cache size. + // Verify on heap cache stats. assertEquals(onHeapCacheSize, tieredSpilloverCache.getOnHeapCache().count()); - // Verify disk cache size. - assertEquals(numOfItems1 - onHeapCacheSize, tieredSpilloverCache.getDiskCache().count()); + assertEquals(onHeapCacheSize, getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(numOfItems1, getMissesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(0, getHitsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(expectedDiskEntries, getEvictionsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(onHeapCacheSize * keyValueSize, getSizeInBytesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + + // Verify disk cache stats. + assertEquals(expectedDiskEntries, tieredSpilloverCache.getDiskCache().count()); + assertEquals(expectedDiskEntries, getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); + assertEquals(0, getHitsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); + assertEquals(numOfItems1, getMissesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); + assertEquals(0, getEvictionsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); + assertEquals(expectedDiskEntries * keyValueSize, getSizeInBytesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); } public void testWithFactoryCreationWithOnHeapCacheNotPresent() { @@ -222,7 +252,7 @@ public void testWithFactoryCreationWithOnHeapCacheNotPresent() { OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory.NAME, new OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory(), MockDiskCache.MockDiskCacheFactory.NAME, - new MockDiskCache.MockDiskCacheFactory(0, randomIntBetween(100, 300)) + new MockDiskCache.MockDiskCacheFactory(0, randomIntBetween(100, 300), false) ) ) ); @@ -267,7 +297,7 @@ public void testWithFactoryCreationWithDiskCacheNotPresent() { OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory.NAME, new OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory(), MockDiskCache.MockDiskCacheFactory.NAME, - new MockDiskCache.MockDiskCacheFactory(0, randomIntBetween(100, 300)) + new MockDiskCache.MockDiskCacheFactory(0, randomIntBetween(100, 300), false) ) ) ); @@ -307,7 +337,7 @@ public void testComputeIfAbsentWithEvictionsFromOnHeapCache() throws Exception { .setClusterSettings(clusterSettings) .build(); - ICache.Factory mockDiskCacheFactory = new MockDiskCache.MockDiskCacheFactory(0, diskCacheSize); + ICache.Factory mockDiskCacheFactory = new MockDiskCache.MockDiskCacheFactory(0, diskCacheSize, false); TieredSpilloverCache tieredSpilloverCache = new TieredSpilloverCache.Builder() .setOnHeapCacheFactory(onHeapCacheFactory) @@ -327,6 +357,15 @@ public void testComputeIfAbsentWithEvictionsFromOnHeapCache() throws Exception { tieredSpilloverCache.computeIfAbsent(key, tieredCacheLoader); } + long actualDiskCacheSize = tieredSpilloverCache.getDiskCache().count(); + + assertEquals(numOfItems1, getMissesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(0, getHitsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(actualDiskCacheSize, getEvictionsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(onHeapCacheSize, getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(onHeapCacheSize * keyValueSize, getSizeInBytesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(actualDiskCacheSize, getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); + tieredSpilloverCache.getOnHeapCache().keys().forEach(onHeapKeys::add); tieredSpilloverCache.getDiskCache().keys().forEach(diskTierKeys::add); @@ -350,12 +389,18 @@ public void testComputeIfAbsentWithEvictionsFromOnHeapCache() throws Exception { assertFalse(loadAwareCacheLoader.isLoaded()); } } - for (int iter = 0; iter < randomIntBetween(50, 200); iter++) { + int numRandom = randomIntBetween(50, 200); + for (int iter = 0; iter < numRandom; iter++) { // Hit cache with randomized key which is expected to miss cache always. LoadAwareCacheLoader, String> tieredCacheLoader = getLoadAwareCacheLoader(); tieredSpilloverCache.computeIfAbsent(getICacheKey(UUID.randomUUID().toString()), tieredCacheLoader); cacheMiss++; } + + assertEquals(numOfItems1 + cacheMiss + diskCacheHit, getMissesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(onHeapCacheHit, getHitsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(cacheMiss + numOfItems1, getMissesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); + assertEquals(diskCacheHit, getHitsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); } public void testComputeIfAbsentWithEvictionsFromTieredCache() throws Exception { @@ -385,8 +430,13 @@ public void testComputeIfAbsentWithEvictionsFromTieredCache() throws Exception { tieredSpilloverCache.computeIfAbsent(getICacheKey(UUID.randomUUID().toString()), tieredCacheLoader); } - int evictions = numOfItems - (totalSize); + int evictions = numOfItems - (totalSize); // Evictions from the cache as a whole assertEquals(evictions, removalListener.evictionsMetric.count()); + assertEquals(evictions, getEvictionsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); + assertEquals( + evictions + getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK), + getEvictionsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP) + ); } public void testGetAndCount() throws Exception { @@ -442,7 +492,7 @@ public void testGetAndCount() throws Exception { assertEquals(numOfItems1, tieredSpilloverCache.count()); } - public void testPut() { + public void testPut() throws Exception { int onHeapCacheSize = randomIntBetween(10, 30); int diskCacheSize = randomIntBetween(onHeapCacheSize + 1, 100); int keyValueSize = 50; @@ -465,6 +515,8 @@ public void testPut() { ICacheKey key = getICacheKey(UUID.randomUUID().toString()); String value = UUID.randomUUID().toString(); tieredSpilloverCache.put(key, value); + assertEquals(1, getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(1, tieredSpilloverCache.count()); } public void testPutAndVerifyNewItemsArePresentOnHeapCache() throws Exception { @@ -497,6 +549,9 @@ public void testPutAndVerifyNewItemsArePresentOnHeapCache() throws Exception { tieredSpilloverCache.computeIfAbsent(getICacheKey(UUID.randomUUID().toString()), getLoadAwareCacheLoader()); } + assertEquals(onHeapCacheSize, getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(0, getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); + // Again try to put OnHeap cache capacity amount of new items. List> newKeyList = new ArrayList<>(); for (int i = 0; i < onHeapCacheSize; i++) { @@ -515,9 +570,11 @@ public void testPutAndVerifyNewItemsArePresentOnHeapCache() throws Exception { for (int i = 0; i < actualOnHeapCacheKeys.size(); i++) { assertTrue(newKeyList.contains(actualOnHeapCacheKeys.get(i))); } + assertEquals(onHeapCacheSize, getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(onHeapCacheSize, getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); } - public void testInvalidate() { + public void testInvalidate() throws Exception { int onHeapCacheSize = 1; int diskCacheSize = 10; int keyValueSize = 20; @@ -541,11 +598,12 @@ public void testInvalidate() { String value = UUID.randomUUID().toString(); // First try to invalidate without the key present in cache. tieredSpilloverCache.invalidate(key); - // assertEquals(0, tieredSpilloverCache.stats().getEvictionsByDimensions(HEAP_DIMS)); + assertEquals(0, getEvictionsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); // Now try to invalidate with the key present in onHeap cache. tieredSpilloverCache.put(key, value); tieredSpilloverCache.invalidate(key); + assertEquals(0, getEvictionsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); // Evictions metric shouldn't increase for invalidations. assertEquals(0, tieredSpilloverCache.count()); @@ -555,11 +613,15 @@ public void testInvalidate() { tieredSpilloverCache.put(key2, UUID.randomUUID().toString()); assertEquals(2, tieredSpilloverCache.count()); + assertEquals(1, getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(1, getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); // Again invalidate older key, leaving one in heap tier and zero in disk tier tieredSpilloverCache.invalidate(key); + assertEquals(0, getEvictionsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); + assertEquals(0, getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); + assertEquals(1, getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); assertEquals(1, tieredSpilloverCache.count()); - } public void testCacheKeys() throws Exception { @@ -752,6 +814,9 @@ public String load(ICacheKey key) { } } assertEquals(1, numberOfTimesKeyLoaded); // It should be loaded only once. + // We should see only one heap miss, and the rest hits + assertEquals(1, getMissesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(numberOfSameKeys - 1, getHitsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); } public void testConcurrencyForEvictionFlowFromOnHeapToDiskTier() throws Exception { @@ -760,7 +825,7 @@ public void testConcurrencyForEvictionFlowFromOnHeapToDiskTier() throws Exceptio MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); ICache.Factory onHeapCacheFactory = new OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory(); - ICache.Factory diskCacheFactory = new MockDiskCache.MockDiskCacheFactory(500, diskCacheSize); + ICache.Factory diskCacheFactory = new MockDiskCache.MockDiskCacheFactory(500, diskCacheSize, false); CacheConfig cacheConfig = new CacheConfig.Builder().setKeyType(String.class) .setKeyType(String.class) .setWeigher((k, v) -> 150) @@ -796,7 +861,7 @@ public void testConcurrencyForEvictionFlowFromOnHeapToDiskTier() throws Exceptio // Put first key on tiered cache. Will go into onHeap cache. tieredSpilloverCache.computeIfAbsent(keyToBeEvicted, getLoadAwareCacheLoader()); - // assertEquals(1, tieredSpilloverCache.stats().getEntriesByDimensions(HEAP_DIMS)); + assertEquals(1, getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); CountDownLatch countDownLatch = new CountDownLatch(1); CountDownLatch countDownLatch1 = new CountDownLatch(1); // Put second key on tiered cache. Will cause eviction of first key from onHeap cache and should go into @@ -834,6 +899,10 @@ public void testConcurrencyForEvictionFlowFromOnHeapToDiskTier() throws Exceptio assertEquals(1, tieredSpilloverCache.getOnHeapCache().count()); assertEquals(1, onDiskCache.count()); + + assertEquals(1, getEvictionsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(1, getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(1, getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); assertNotNull(onDiskCache.get(keyToBeEvicted)); } @@ -961,7 +1030,7 @@ public CachedQueryResult.PolicyValues apply(String s) { OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory.NAME, new OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory(), MockDiskCache.MockDiskCacheFactory.NAME, - new MockDiskCache.MockDiskCacheFactory(0, randomIntBetween(100, 300)) + new MockDiskCache.MockDiskCacheFactory(0, randomIntBetween(100, 300), false) ) ); @@ -1053,7 +1122,6 @@ public void testGetPutAndInvalidateWithDiskCacheDisabled() throws Exception { int diskCacheSize = randomIntBetween(onHeapCacheSize + 1, 100); int keyValueSize = 50; int totalSize = onHeapCacheSize + diskCacheSize; - MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); TieredSpilloverCache tieredSpilloverCache = initializeTieredSpilloverCache( keyValueSize, @@ -1122,6 +1190,128 @@ public void testGetPutAndInvalidateWithDiskCacheDisabled() throws Exception { assertEquals(0, tieredSpilloverCache.count()); } + public void testTiersDoNotTrackStats() throws Exception { + int onHeapCacheSize = randomIntBetween(10, 30); + int diskCacheSize = randomIntBetween(onHeapCacheSize + 1, 100); + int keyValueSize = 50; + MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); + TieredSpilloverCache tieredSpilloverCache = initializeTieredSpilloverCache( + keyValueSize, + diskCacheSize, + removalListener, + Settings.builder() + .put( + OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) + .get(MAXIMUM_SIZE_IN_BYTES_KEY) + .getKey(), + onHeapCacheSize * keyValueSize + "b" + ) + .build(), + 0 + ); + + // do some gets to put entries in both tiers + int numMisses = onHeapCacheSize + randomIntBetween(10, 20); + for (int iter = 0; iter < numMisses; iter++) { + ICacheKey key = getICacheKey(UUID.randomUUID().toString()); + LoadAwareCacheLoader, String> tieredCacheLoader = getLoadAwareCacheLoader(); + tieredSpilloverCache.computeIfAbsent(key, tieredCacheLoader); + } + assertNotEquals(new ImmutableCacheStats(0, 0, 0, 0, 0), tieredSpilloverCache.stats().getTotalStats()); + assertEquals(new ImmutableCacheStats(0, 0, 0, 0, 0), tieredSpilloverCache.getOnHeapCache().stats().getTotalStats()); + ImmutableCacheStats diskStats = tieredSpilloverCache.getDiskCache().stats().getTotalStats(); + assertEquals(new ImmutableCacheStats(0, 0, 0, 0, 0), diskStats); + } + + public void testTierStatsAddCorrectly() throws Exception { + /* We expect the total stats to be: + * totalHits = heapHits + diskHits + * totalMisses = diskMisses + * totalEvictions = diskEvictions + * totalSize = heapSize + diskSize + * totalEntries = heapEntries + diskEntries + */ + + int onHeapCacheSize = randomIntBetween(10, 30); + int diskCacheSize = randomIntBetween(onHeapCacheSize + 1, 100); + int keyValueSize = 50; + MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); + TieredSpilloverCache tieredSpilloverCache = initializeTieredSpilloverCache( + keyValueSize, + diskCacheSize, + removalListener, + Settings.builder() + .put( + OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) + .get(MAXIMUM_SIZE_IN_BYTES_KEY) + .getKey(), + onHeapCacheSize * keyValueSize + "b" + ) + .build(), + 0 + ); + + List> usedKeys = new ArrayList<>(); + // Fill the cache, getting some entries + evictions for both tiers + int numMisses = onHeapCacheSize + diskCacheSize + randomIntBetween(10, 20); + for (int iter = 0; iter < numMisses; iter++) { + ICacheKey key = getICacheKey(UUID.randomUUID().toString()); + usedKeys.add(key); + LoadAwareCacheLoader, String> tieredCacheLoader = getLoadAwareCacheLoader(); + tieredSpilloverCache.computeIfAbsent(key, tieredCacheLoader); + } + // Also do some random hits + Random rand = Randomness.get(); + int approxNumHits = 30; + for (int i = 0; i < approxNumHits; i++) { + LoadAwareCacheLoader, String> tieredCacheLoader = getLoadAwareCacheLoader(); + ICacheKey key = usedKeys.get(rand.nextInt(usedKeys.size())); + tieredSpilloverCache.computeIfAbsent(key, tieredCacheLoader); + } + + ImmutableCacheStats totalStats = tieredSpilloverCache.stats().getTotalStats(); + ImmutableCacheStats heapStats = getStatsSnapshotForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP); + ImmutableCacheStats diskStats = getStatsSnapshotForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK); + + assertEquals(totalStats.getHits(), heapStats.getHits() + diskStats.getHits()); + assertEquals(totalStats.getMisses(), diskStats.getMisses()); + assertEquals(totalStats.getEvictions(), diskStats.getEvictions()); + assertEquals(totalStats.getSizeInBytes(), heapStats.getSizeInBytes() + diskStats.getSizeInBytes()); + assertEquals(totalStats.getItems(), heapStats.getItems() + diskStats.getItems()); + + // Also check the heap stats don't have zero misses or evictions + assertNotEquals(0, heapStats.getMisses()); + assertNotEquals(0, heapStats.getEvictions()); + + // Now turn off the disk tier and do more misses and evictions from the heap tier. + // These should be added to the totals, as the disk tier is now absent + long missesBeforeDisablingDiskCache = totalStats.getMisses(); + long evictionsBeforeDisablingDiskCache = totalStats.getEvictions(); + long heapTierEvictionsBeforeDisablingDiskCache = heapStats.getEvictions(); + + clusterSettings.applySettings( + Settings.builder().put(DISK_CACHE_ENABLED_SETTING_MAP.get(CacheType.INDICES_REQUEST_CACHE).getKey(), false).build() + ); + + int newMisses = randomIntBetween(10, 30); + for (int i = 0; i < newMisses; i++) { + LoadAwareCacheLoader, String> tieredCacheLoader = getLoadAwareCacheLoader(); + tieredSpilloverCache.computeIfAbsent(getICacheKey(UUID.randomUUID().toString()), tieredCacheLoader); + } + + totalStats = tieredSpilloverCache.stats().getTotalStats(); + heapStats = getStatsSnapshotForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP); + assertEquals(missesBeforeDisablingDiskCache + newMisses, totalStats.getMisses()); + assertEquals(heapTierEvictionsBeforeDisablingDiskCache + newMisses, heapStats.getEvictions()); + assertEquals(evictionsBeforeDisablingDiskCache + newMisses, totalStats.getEvictions()); + + // Turn the disk cache back on in cluster settings for other tests + clusterSettings.applySettings( + Settings.builder().put(DISK_CACHE_ENABLED_SETTING_MAP.get(CacheType.INDICES_REQUEST_CACHE).getKey(), true).build() + ); + + } + private List getMockDimensions() { List dims = new ArrayList<>(); for (String dimensionName : dimensionNames) { @@ -1237,7 +1427,7 @@ private TieredSpilloverCache intializeTieredSpilloverCache( ) .setClusterSettings(clusterSettings) .build(); - ICache.Factory mockDiskCacheFactory = new MockDiskCache.MockDiskCacheFactory(diskDeliberateDelay, diskCacheSize); + ICache.Factory mockDiskCacheFactory = new MockDiskCache.MockDiskCacheFactory(diskDeliberateDelay, diskCacheSize, false); TieredSpilloverCache.Builder builder = new TieredSpilloverCache.Builder().setCacheType( CacheType.INDICES_REQUEST_CACHE @@ -1251,4 +1441,42 @@ private TieredSpilloverCache intializeTieredSpilloverCache( } return builder.build(); } + + // Helper functions for extracting tier aggregated stats. + private long getHitsForTier(TieredSpilloverCache tsc, String tierValue) throws IOException { + return getStatsSnapshotForTier(tsc, tierValue).getHits(); + } + + private long getMissesForTier(TieredSpilloverCache tsc, String tierValue) throws IOException { + return getStatsSnapshotForTier(tsc, tierValue).getMisses(); + } + + private long getEvictionsForTier(TieredSpilloverCache tsc, String tierValue) throws IOException { + return getStatsSnapshotForTier(tsc, tierValue).getEvictions(); + } + + private long getSizeInBytesForTier(TieredSpilloverCache tsc, String tierValue) throws IOException { + return getStatsSnapshotForTier(tsc, tierValue).getSizeInBytes(); + } + + private long getItemsForTier(TieredSpilloverCache tsc, String tierValue) throws IOException { + return getStatsSnapshotForTier(tsc, tierValue).getItems(); + } + + private ImmutableCacheStats getStatsSnapshotForTier(TieredSpilloverCache tsc, String tierValue) throws IOException { + List levelsList = new ArrayList<>(dimensionNames); + levelsList.add(TIER_DIMENSION_NAME); + String[] levels = levelsList.toArray(new String[0]); + ImmutableCacheStatsHolder cacheStats = tsc.stats(levels); + // Since we always use the same list of dimensions from getMockDimensions() in keys for these tests, we can get all the stats values + // for a given tier with a single node in MDCS + List mockDimensions = getMockDimensions(); + mockDimensions.add(tierValue); + ImmutableCacheStats snapshot = cacheStats.getStatsForDimensionValues(mockDimensions); + if (snapshot == null) { + return new ImmutableCacheStats(0, 0, 0, 0, 0); // This can happen if no cache actions have happened for this set of + // dimensions yet + } + return snapshot; + } } diff --git a/plugins/cache-ehcache/src/main/java/org/opensearch/cache/store/disk/EhcacheDiskCache.java b/plugins/cache-ehcache/src/main/java/org/opensearch/cache/store/disk/EhcacheDiskCache.java index 93c54a48d59da..9a4dce1067b61 100644 --- a/plugins/cache-ehcache/src/main/java/org/opensearch/cache/store/disk/EhcacheDiskCache.java +++ b/plugins/cache-ehcache/src/main/java/org/opensearch/cache/store/disk/EhcacheDiskCache.java @@ -27,6 +27,7 @@ import org.opensearch.common.cache.stats.CacheStatsHolder; import org.opensearch.common.cache.stats.DefaultCacheStatsHolder; import org.opensearch.common.cache.stats.ImmutableCacheStatsHolder; +import org.opensearch.common.cache.stats.NoopCacheStatsHolder; import org.opensearch.common.cache.store.builders.ICacheBuilder; import org.opensearch.common.cache.store.config.CacheConfig; import org.opensearch.common.collect.Tuple; @@ -163,8 +164,13 @@ private EhcacheDiskCache(Builder builder) { this.ehCacheEventListener = new EhCacheEventListener(builder.getRemovalListener(), builder.getWeigher()); this.cache = buildCache(Duration.ofMillis(expireAfterAccess.getMillis()), builder); List dimensionNames = Objects.requireNonNull(builder.dimensionNames, "Dimension names can't be null"); - // If this cache is being used, FeatureFlags.PLUGGABLE_CACHE is already on, so we can always use the DefaultCacheStatsHolder. - this.cacheStatsHolder = new DefaultCacheStatsHolder(dimensionNames, EhcacheDiskCacheFactory.EHCACHE_DISK_CACHE_NAME); + if (builder.getStatsTrackingEnabled()) { + // If this cache is being used, FeatureFlags.PLUGGABLE_CACHE is already on, so we can always use the DefaultCacheStatsHolder + // unless statsTrackingEnabled is explicitly set to false in CacheConfig. + this.cacheStatsHolder = new DefaultCacheStatsHolder(dimensionNames, EhcacheDiskCacheFactory.EHCACHE_DISK_CACHE_NAME); + } else { + this.cacheStatsHolder = NoopCacheStatsHolder.getInstance(); + } } @SuppressWarnings({ "rawtypes" }) @@ -414,6 +420,7 @@ public Iterable> keys() { /** * Gives the current count of keys in disk cache. + * If enableStatsTracking is set to false in the builder, always returns 0. * @return current count of keys */ @Override diff --git a/plugins/cache-ehcache/src/test/java/org/opensearch/cache/store/disk/EhCacheDiskCacheTests.java b/plugins/cache-ehcache/src/test/java/org/opensearch/cache/store/disk/EhCacheDiskCacheTests.java index f2bfe1209a4c7..29551befd3e9f 100644 --- a/plugins/cache-ehcache/src/test/java/org/opensearch/cache/store/disk/EhCacheDiskCacheTests.java +++ b/plugins/cache-ehcache/src/test/java/org/opensearch/cache/store/disk/EhCacheDiskCacheTests.java @@ -850,6 +850,38 @@ public void testInvalidateWithDropDimensions() throws Exception { } } + public void testStatsTrackingDisabled() throws Exception { + Settings settings = Settings.builder().build(); + MockRemovalListener removalListener = new MockRemovalListener<>(); + ToLongBiFunction, String> weigher = getWeigher(); + try (NodeEnvironment env = newNodeEnvironment(settings)) { + ICache ehcacheTest = new EhcacheDiskCache.Builder().setThreadPoolAlias("ehcacheTest") + .setStoragePath(env.nodePaths()[0].indicesPath.toString() + "/request_cache") + .setIsEventListenerModeSync(true) + .setKeyType(String.class) + .setValueType(String.class) + .setKeySerializer(new StringSerializer()) + .setValueSerializer(new StringSerializer()) + .setDimensionNames(List.of(dimensionName)) + .setCacheType(CacheType.INDICES_REQUEST_CACHE) + .setSettings(settings) + .setExpireAfterAccess(TimeValue.MAX_VALUE) + .setMaximumWeightInBytes(CACHE_SIZE_IN_BYTES) + .setRemovalListener(removalListener) + .setWeigher(weigher) + .setStatsTrackingEnabled(false) + .build(); + int randomKeys = randomIntBetween(10, 100); + for (int i = 0; i < randomKeys; i++) { + ICacheKey iCacheKey = getICacheKey(UUID.randomUUID().toString()); + ehcacheTest.put(iCacheKey, UUID.randomUUID().toString()); + assertEquals(0, ehcacheTest.count()); // Expect count of 0 if NoopCacheStatsHolder is used + assertEquals(new ImmutableCacheStats(0, 0, 0, 0, 0), ehcacheTest.stats().getTotalStats()); + } + ehcacheTest.close(); + } + } + private List getRandomDimensions(List dimensionNames) { Random rand = Randomness.get(); int bound = 3; diff --git a/server/src/main/java/org/opensearch/common/cache/stats/DefaultCacheStatsHolder.java b/server/src/main/java/org/opensearch/common/cache/stats/DefaultCacheStatsHolder.java index 5574e345b6d3d..ea92c8e81b8f0 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/DefaultCacheStatsHolder.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/DefaultCacheStatsHolder.java @@ -32,7 +32,7 @@ public class DefaultCacheStatsHolder implements CacheStatsHolder { // The list of permitted dimensions. Should be ordered from "outermost" to "innermost", as you would like to // aggregate them in an API response. - private final List dimensionNames; + protected final List dimensionNames; // A tree structure based on dimension values, which stores stats values in its leaf nodes. // Non-leaf nodes have stats matching the sum of their children. // We use a tree structure, rather than a map with concatenated keys, to save on memory usage. If there are many leaf @@ -115,7 +115,7 @@ public long count() { return statsRoot.getEntries(); } - private void internalIncrement(List dimensionValues, Consumer adder, boolean createNodesIfAbsent) { + protected void internalIncrement(List dimensionValues, Consumer adder, boolean createNodesIfAbsent) { assert dimensionValues.size() == dimensionNames.size(); // First try to increment without creating nodes boolean didIncrement = internalIncrementHelper(dimensionValues, statsRoot, 0, adder, false); @@ -213,7 +213,10 @@ Node getStatsRoot() { return statsRoot; } - static class Node { + /** + * Nodes that make up the tree in the stats holder. + */ + protected static class Node { private final String dimensionValue; // Map from dimensionValue to the DimensionNode for that dimension value. final Map children; @@ -245,23 +248,23 @@ protected Map getChildren() { // Functions for modifying internal CacheStatsCounter without callers having to be aware of CacheStatsCounter - void incrementHits() { + public void incrementHits() { this.stats.incrementHits(); } - void incrementMisses() { + public void incrementMisses() { this.stats.incrementMisses(); } - void incrementEvictions() { + public void incrementEvictions() { this.stats.incrementEvictions(); } - void incrementSizeInBytes(long amountBytes) { + public void incrementSizeInBytes(long amountBytes) { this.stats.incrementSizeInBytes(amountBytes); } - void decrementSizeInBytes(long amountBytes) { + public void decrementSizeInBytes(long amountBytes) { this.stats.decrementSizeInBytes(amountBytes); } @@ -296,5 +299,16 @@ Node getChild(String dimensionValue) { Node createChild(String dimensionValue, boolean createMapInChild) { return children.computeIfAbsent(dimensionValue, (key) -> new Node(dimensionValue, createMapInChild)); } + + /** + * Return whether this is a leaf node which is at the lowest level of the tree. + * Does not return true if this is a node at a higher level whose children are still being constructed. + * @return if this is a leaf node at the lowest level + */ + public boolean isAtLowestLevel() { + // Compare by value to the empty children map, to ensure we don't get false positives for nodes + // which are in the process of having children added + return children == EMPTY_CHILDREN_MAP; + } } } diff --git a/server/src/main/java/org/opensearch/common/cache/store/OpenSearchOnHeapCache.java b/server/src/main/java/org/opensearch/common/cache/store/OpenSearchOnHeapCache.java index f4cf9f3a8fa61..569653bec2a3d 100644 --- a/server/src/main/java/org/opensearch/common/cache/store/OpenSearchOnHeapCache.java +++ b/server/src/main/java/org/opensearch/common/cache/store/OpenSearchOnHeapCache.java @@ -53,6 +53,7 @@ public class OpenSearchOnHeapCache implements ICache, RemovalListene private final RemovalListener, V> removalListener; private final List dimensionNames; private final ToLongBiFunction, V> weigher; + private final boolean statsTrackingEnabled; public OpenSearchOnHeapCache(Builder builder) { CacheBuilder, V> cacheBuilder = CacheBuilder., V>builder() @@ -64,12 +65,11 @@ public OpenSearchOnHeapCache(Builder builder) { } cache = cacheBuilder.build(); this.dimensionNames = Objects.requireNonNull(builder.dimensionNames, "Dimension names can't be null"); - // Use noop stats when pluggable caching is off - boolean useNoopStats = !FeatureFlags.PLUGGABLE_CACHE_SETTING.get(builder.getSettings()); - if (useNoopStats) { - this.cacheStatsHolder = NoopCacheStatsHolder.getInstance(); - } else { + this.statsTrackingEnabled = builder.getStatsTrackingEnabled(); + if (statsTrackingEnabled) { this.cacheStatsHolder = new DefaultCacheStatsHolder(dimensionNames, OpenSearchOnHeapCacheFactory.NAME); + } else { + this.cacheStatsHolder = NoopCacheStatsHolder.getInstance(); } this.removalListener = builder.getRemovalListener(); this.weigher = builder.getWeigher(); @@ -171,8 +171,9 @@ public static class OpenSearchOnHeapCacheFactory implements Factory { public ICache create(CacheConfig config, CacheType cacheType, Map cacheFactories) { Map> settingList = OpenSearchOnHeapCacheSettings.getSettingListForCacheType(cacheType); Settings settings = config.getSettings(); + boolean statsTrackingEnabled = statsTrackingEnabled(config.getSettings(), config.getStatsTrackingEnabled()); ICacheBuilder builder = new Builder().setDimensionNames(config.getDimensionNames()) - .setSettings(config.getSettings()) + .setStatsTrackingEnabled(statsTrackingEnabled) .setMaximumWeightInBytes(((ByteSizeValue) settingList.get(MAXIMUM_SIZE_IN_BYTES_KEY).get(settings)).getBytes()) .setExpireAfterAccess(((TimeValue) settingList.get(EXPIRE_AFTER_ACCESS_KEY).get(settings))) .setWeigher(config.getWeigher()) @@ -193,6 +194,11 @@ public ICache create(CacheConfig config, CacheType cacheType, public String getCacheName() { return NAME; } + + private boolean statsTrackingEnabled(Settings settings, boolean statsTrackingEnabledConfig) { + // Don't track stats when pluggable caching is off, or when explicitly set to false in the CacheConfig + return FeatureFlags.PLUGGABLE_CACHE_SETTING.get(settings) && statsTrackingEnabledConfig; + } } /** diff --git a/server/src/main/java/org/opensearch/common/cache/store/builders/ICacheBuilder.java b/server/src/main/java/org/opensearch/common/cache/store/builders/ICacheBuilder.java index ac90fcc85ffef..a308d1db88258 100644 --- a/server/src/main/java/org/opensearch/common/cache/store/builders/ICacheBuilder.java +++ b/server/src/main/java/org/opensearch/common/cache/store/builders/ICacheBuilder.java @@ -37,6 +37,8 @@ public abstract class ICacheBuilder { private RemovalListener, V> removalListener; + private boolean statsTrackingEnabled = true; + public ICacheBuilder() {} public ICacheBuilder setMaximumWeightInBytes(long sizeInBytes) { @@ -64,6 +66,11 @@ public ICacheBuilder setRemovalListener(RemovalListener, V> r return this; } + public ICacheBuilder setStatsTrackingEnabled(boolean statsTrackingEnabled) { + this.statsTrackingEnabled = statsTrackingEnabled; + return this; + } + public long getMaxWeightInBytes() { return maxWeightInBytes; } @@ -84,5 +91,9 @@ public Settings getSettings() { return settings; } + public boolean getStatsTrackingEnabled() { + return statsTrackingEnabled; + } + public abstract ICache build(); } diff --git a/server/src/main/java/org/opensearch/common/cache/store/config/CacheConfig.java b/server/src/main/java/org/opensearch/common/cache/store/config/CacheConfig.java index 15cbdbd021d71..0c54ac57a9b18 100644 --- a/server/src/main/java/org/opensearch/common/cache/store/config/CacheConfig.java +++ b/server/src/main/java/org/opensearch/common/cache/store/config/CacheConfig.java @@ -68,6 +68,8 @@ public class CacheConfig { private final ClusterSettings clusterSettings; + private final boolean statsTrackingEnabled; + private CacheConfig(Builder builder) { this.keyType = builder.keyType; this.valueType = builder.valueType; @@ -81,6 +83,7 @@ private CacheConfig(Builder builder) { this.maxSizeInBytes = builder.maxSizeInBytes; this.expireAfterAccess = builder.expireAfterAccess; this.clusterSettings = builder.clusterSettings; + this.statsTrackingEnabled = builder.statsTrackingEnabled; } public Class getKeyType() { @@ -131,6 +134,10 @@ public ClusterSettings getClusterSettings() { return clusterSettings; } + public boolean getStatsTrackingEnabled() { + return statsTrackingEnabled; + } + /** * Builder class to build Cache config related parameters. * @param Type of key. @@ -155,6 +162,7 @@ public static class Builder { private TimeValue expireAfterAccess; private ClusterSettings clusterSettings; + private boolean statsTrackingEnabled = true; public Builder() {} @@ -218,6 +226,11 @@ public Builder setClusterSettings(ClusterSettings clusterSettings) { return this; } + public Builder setStatsTrackingEnabled(boolean statsTrackingEnabled) { + this.statsTrackingEnabled = statsTrackingEnabled; + return this; + } + public CacheConfig build() { return new CacheConfig<>(this); } diff --git a/server/src/test/java/org/opensearch/common/cache/store/OpenSearchOnHeapCacheTests.java b/server/src/test/java/org/opensearch/common/cache/store/OpenSearchOnHeapCacheTests.java index f5885d03f1850..f227db6fee2d1 100644 --- a/server/src/test/java/org/opensearch/common/cache/store/OpenSearchOnHeapCacheTests.java +++ b/server/src/test/java/org/opensearch/common/cache/store/OpenSearchOnHeapCacheTests.java @@ -39,7 +39,7 @@ public void testStats() throws Exception { MockRemovalListener listener = new MockRemovalListener<>(); int maxKeys = between(10, 50); int numEvicted = between(10, 20); - OpenSearchOnHeapCache cache = getCache(maxKeys, listener, true); + OpenSearchOnHeapCache cache = getCache(maxKeys, listener, true, true); // When the pluggable caches setting is on, we should get stats as expected from cache.stats(). @@ -82,21 +82,26 @@ public void testStats() throws Exception { } public void testStatsWithoutPluggableCaches() throws Exception { - // When the pluggable caches setting is off, we should get all-zero stats from cache.stats(), but count() should still work. + // When the pluggable caches setting is off, or when we manually set statsTrackingEnabled = false in the config, + // we should get all-zero stats from cache.stats(), but count() should still work. MockRemovalListener listener = new MockRemovalListener<>(); int maxKeys = between(10, 50); int numEvicted = between(10, 20); - OpenSearchOnHeapCache cache = getCache(maxKeys, listener, false); - List> keysAdded = new ArrayList<>(); - int numAdded = maxKeys + numEvicted; - for (int i = 0; i < numAdded; i++) { - ICacheKey key = getICacheKey(UUID.randomUUID().toString()); - keysAdded.add(key); - cache.computeIfAbsent(key, getLoadAwareCacheLoader()); + OpenSearchOnHeapCache pluggableCachesOffCache = getCache(maxKeys, listener, false, true); + OpenSearchOnHeapCache manuallySetNoopStatsCache = getCache(maxKeys, listener, true, false); + List> caches = List.of(pluggableCachesOffCache, manuallySetNoopStatsCache); - assertEquals(Math.min(maxKeys, i + 1), cache.count()); - assertZeroStats(cache.stats()); + for (OpenSearchOnHeapCache cache : caches) { + int numAdded = maxKeys + numEvicted; + for (int i = 0; i < numAdded; i++) { + ICacheKey key = getICacheKey(UUID.randomUUID().toString()); + cache.computeIfAbsent(key, getLoadAwareCacheLoader()); + + assertEquals(Math.min(maxKeys, i + 1), cache.count()); + ImmutableCacheStatsHolder stats = cache.stats(); + assertZeroStats(cache.stats()); + } } } @@ -107,7 +112,8 @@ private void assertZeroStats(ImmutableCacheStatsHolder stats) { private OpenSearchOnHeapCache getCache( int maxSizeKeys, MockRemovalListener listener, - boolean pluggableCachesSetting + boolean pluggableCachesSetting, + boolean statsTrackingEnabled ) { ICache.Factory onHeapCacheFactory = new OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory(); Settings settings = Settings.builder() @@ -127,6 +133,7 @@ private OpenSearchOnHeapCache getCache( .setSettings(settings) .setDimensionNames(dimensionNames) .setMaxSizeInBytes(maxSizeKeys * keyValueSize) + .setStatsTrackingEnabled(statsTrackingEnabled) .build(); return (OpenSearchOnHeapCache) onHeapCacheFactory.create(cacheConfig, CacheType.INDICES_REQUEST_CACHE, null); } @@ -134,7 +141,7 @@ private OpenSearchOnHeapCache getCache( public void testInvalidateWithDropDimensions() throws Exception { MockRemovalListener listener = new MockRemovalListener<>(); int maxKeys = 50; - OpenSearchOnHeapCache cache = getCache(maxKeys, listener, true); + OpenSearchOnHeapCache cache = getCache(maxKeys, listener, true, true); List> keysAdded = new ArrayList<>();