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 9a24d28dc99d4..184155b14e74e 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 @@ -17,7 +17,7 @@ import org.opensearch.common.cache.RemovalListener; import org.opensearch.common.cache.RemovalNotification; import org.opensearch.common.cache.policy.CachedQueryResult; -import org.opensearch.common.cache.stats.CacheStats; +import org.opensearch.common.cache.stats.ImmutableCacheStatsHolder; import org.opensearch.common.cache.store.config.CacheConfig; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; @@ -195,7 +195,7 @@ public void close() throws IOException { } @Override - public CacheStats stats() { + public ImmutableCacheStatsHolder stats() { return null; // TODO: in TSC stats PR } 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 3f605646ec9f4..31547aa5b2ed3 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,7 +16,6 @@ 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.CacheStats; import org.opensearch.common.cache.store.builders.ICacheBuilder; import org.opensearch.common.cache.store.config.CacheConfig; 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 7c7c700728074..185d51732a116 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 @@ -24,8 +24,8 @@ import org.opensearch.common.cache.RemovalReason; import org.opensearch.common.cache.serializer.ICacheKeySerializer; import org.opensearch.common.cache.serializer.Serializer; -import org.opensearch.common.cache.stats.CacheStats; -import org.opensearch.common.cache.stats.StatsHolder; +import org.opensearch.common.cache.stats.CacheStatsHolder; +import org.opensearch.common.cache.stats.ImmutableCacheStatsHolder; import org.opensearch.common.cache.store.builders.ICacheBuilder; import org.opensearch.common.cache.store.config.CacheConfig; import org.opensearch.common.collect.Tuple; @@ -113,7 +113,7 @@ public class EhcacheDiskCache implements ICache { private final Class keyType; private final Class valueType; private final TimeValue expireAfterAccess; - private final StatsHolder statsHolder; + private final CacheStatsHolder cacheStatsHolder; private final EhCacheEventListener ehCacheEventListener; private final String threadPoolAlias; private final Settings settings; @@ -162,7 +162,7 @@ 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"); - this.statsHolder = new StatsHolder(dimensionNames); + this.cacheStatsHolder = new CacheStatsHolder(dimensionNames); } @SuppressWarnings({ "rawtypes" }) @@ -277,9 +277,9 @@ public V get(ICacheKey key) { throw new OpenSearchException("Exception occurred while trying to fetch item from ehcache disk cache"); } if (value != null) { - statsHolder.incrementHits(key.dimensions); + cacheStatsHolder.incrementHits(key.dimensions); } else { - statsHolder.incrementMisses(key.dimensions); + cacheStatsHolder.incrementMisses(key.dimensions); } return value; } @@ -315,9 +315,9 @@ public V computeIfAbsent(ICacheKey key, LoadAwareCacheLoader, V> value = compute(key, loader); } if (!loader.isLoaded()) { - statsHolder.incrementHits(key.dimensions); + cacheStatsHolder.incrementHits(key.dimensions); } else { - statsHolder.incrementMisses(key.dimensions); + cacheStatsHolder.incrementMisses(key.dimensions); } return value; } @@ -383,7 +383,7 @@ private V compute(ICacheKey key, LoadAwareCacheLoader, V> loader public void invalidate(ICacheKey key) { try { if (key.getDropStatsForDimensions()) { - statsHolder.removeDimensions(key.dimensions); + cacheStatsHolder.removeDimensions(key.dimensions); } if (key.key != null) { cache.remove(key); @@ -398,7 +398,7 @@ public void invalidate(ICacheKey key) { @Override public void invalidateAll() { cache.clear(); - statsHolder.reset(); + cacheStatsHolder.reset(); } /** @@ -416,7 +416,7 @@ public Iterable> keys() { */ @Override public long count() { - return statsHolder.count(); + return cacheStatsHolder.count(); } @Override @@ -448,8 +448,8 @@ public void close() { * @return CacheStats */ @Override - public CacheStats stats() { - return statsHolder.getCacheStats(); + public ImmutableCacheStatsHolder stats() { + return cacheStatsHolder.getImmutableCacheStatsHolder(); } /** @@ -508,39 +508,39 @@ private long getNewValuePairSize(CacheEvent, ? extends By public void onEvent(CacheEvent, ? extends ByteArrayWrapper> event) { switch (event.getType()) { case CREATED: - statsHolder.incrementEntries(event.getKey().dimensions); - statsHolder.incrementSizeInBytes(event.getKey().dimensions, getNewValuePairSize(event)); + cacheStatsHolder.incrementEntries(event.getKey().dimensions); + cacheStatsHolder.incrementSizeInBytes(event.getKey().dimensions, getNewValuePairSize(event)); assert event.getOldValue() == null; break; case EVICTED: this.removalListener.onRemoval( new RemovalNotification<>(event.getKey(), deserializeValue(event.getOldValue()), RemovalReason.EVICTED) ); - statsHolder.decrementEntries(event.getKey().dimensions); - statsHolder.decrementSizeInBytes(event.getKey().dimensions, getOldValuePairSize(event)); - statsHolder.incrementEvictions(event.getKey().dimensions); + cacheStatsHolder.decrementEntries(event.getKey().dimensions); + cacheStatsHolder.decrementSizeInBytes(event.getKey().dimensions, getOldValuePairSize(event)); + cacheStatsHolder.incrementEvictions(event.getKey().dimensions); assert event.getNewValue() == null; break; case REMOVED: this.removalListener.onRemoval( new RemovalNotification<>(event.getKey(), deserializeValue(event.getOldValue()), RemovalReason.EXPLICIT) ); - statsHolder.decrementEntries(event.getKey().dimensions); - statsHolder.decrementSizeInBytes(event.getKey().dimensions, getOldValuePairSize(event)); + cacheStatsHolder.decrementEntries(event.getKey().dimensions); + cacheStatsHolder.decrementSizeInBytes(event.getKey().dimensions, getOldValuePairSize(event)); assert event.getNewValue() == null; break; case EXPIRED: this.removalListener.onRemoval( new RemovalNotification<>(event.getKey(), deserializeValue(event.getOldValue()), RemovalReason.INVALIDATED) ); - statsHolder.decrementEntries(event.getKey().dimensions); - statsHolder.decrementSizeInBytes(event.getKey().dimensions, getOldValuePairSize(event)); + cacheStatsHolder.decrementEntries(event.getKey().dimensions); + cacheStatsHolder.decrementSizeInBytes(event.getKey().dimensions, getOldValuePairSize(event)); assert event.getNewValue() == null; break; case UPDATED: long newSize = getNewValuePairSize(event); long oldSize = getOldValuePairSize(event); - statsHolder.incrementSizeInBytes(event.getKey().dimensions, newSize - oldSize); + cacheStatsHolder.incrementSizeInBytes(event.getKey().dimensions, newSize - oldSize); break; default: break; 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 00f56acf25dfd..408e1370a9ea3 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 @@ -20,8 +20,7 @@ import org.opensearch.common.cache.RemovalNotification; import org.opensearch.common.cache.serializer.BytesReferenceSerializer; import org.opensearch.common.cache.serializer.Serializer; -import org.opensearch.common.cache.stats.CacheStatsCounterSnapshot; -import org.opensearch.common.cache.stats.MultiDimensionCacheStats; +import org.opensearch.common.cache.stats.CacheStatsSnapshot; import org.opensearch.common.cache.store.config.CacheConfig; import org.opensearch.common.metrics.CounterMetric; import org.opensearch.common.settings.Settings; @@ -829,9 +828,7 @@ public void testInvalidateWithDropDimensions() throws Exception { ICacheKey keyToDrop = keysAdded.get(0); - CacheStatsCounterSnapshot snapshot = ((MultiDimensionCacheStats) ehCacheDiskCachingTier.stats()).getStatsForDimensionValues( - keyToDrop.dimensions - ); + CacheStatsSnapshot snapshot = ehCacheDiskCachingTier.stats().getStatsForDimensionValues(keyToDrop.dimensions); assertNotNull(snapshot); keyToDrop.setDropStatsForDimensions(true); @@ -839,7 +836,7 @@ public void testInvalidateWithDropDimensions() throws Exception { // Now assert the stats are gone for any key that has this combination of dimensions, but still there otherwise for (ICacheKey keyAdded : keysAdded) { - snapshot = ((MultiDimensionCacheStats) ehCacheDiskCachingTier.stats()).getStatsForDimensionValues(keyAdded.dimensions); + snapshot = ehCacheDiskCachingTier.stats().getStatsForDimensionValues(keyAdded.dimensions); if (keyAdded.dimensions.equals(keyToDrop.dimensions)) { assertNull(snapshot); } else { diff --git a/server/src/main/java/org/opensearch/common/cache/ICache.java b/server/src/main/java/org/opensearch/common/cache/ICache.java index b4f611d5f3635..8d8964abf0829 100644 --- a/server/src/main/java/org/opensearch/common/cache/ICache.java +++ b/server/src/main/java/org/opensearch/common/cache/ICache.java @@ -9,7 +9,7 @@ package org.opensearch.common.cache; import org.opensearch.common.annotation.ExperimentalApi; -import org.opensearch.common.cache.stats.CacheStats; +import org.opensearch.common.cache.stats.ImmutableCacheStatsHolder; import org.opensearch.common.cache.store.config.CacheConfig; import java.io.Closeable; @@ -45,7 +45,7 @@ public interface ICache extends Closeable { void refresh(); - CacheStats stats(); + ImmutableCacheStatsHolder stats(); /** * Factory to create objects. diff --git a/server/src/main/java/org/opensearch/common/cache/ICacheKey.java b/server/src/main/java/org/opensearch/common/cache/ICacheKey.java index e1aa9b1c5466c..91b93acd688a3 100644 --- a/server/src/main/java/org/opensearch/common/cache/ICacheKey.java +++ b/server/src/main/java/org/opensearch/common/cache/ICacheKey.java @@ -14,7 +14,8 @@ /** * A key wrapper used for ICache implementations, which carries dimensions with it. - * @param the type of the underlying key + * @param the type of the underlying key. K must implement equals(), or else ICacheKey.equals() + * won't work properly and cache behavior may be incorrect! * * @opensearch.experimental */ diff --git a/server/src/main/java/org/opensearch/common/cache/stats/CacheStats.java b/server/src/main/java/org/opensearch/common/cache/stats/CacheStats.java index e2937abd8ae93..e7136f60b870d 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/CacheStats.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/CacheStats.java @@ -8,28 +8,125 @@ package org.opensearch.common.cache.stats; -import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.common.metrics.CounterMetric; + +import java.util.Objects; /** - * Interface for access to any cache stats. Allows accessing stats by dimension values. - * Stores an immutable snapshot of stats for a cache. The cache maintains its own live counters. - * - * @opensearch.experimental + * A mutable class containing the 5 live metrics tracked by a StatsHolder object. */ -@ExperimentalApi -public interface CacheStats { // TODO: also extends Writeable, ToXContentFragment (in API PR) +public class CacheStats { + CounterMetric hits; + CounterMetric misses; + CounterMetric evictions; + CounterMetric sizeInBytes; + CounterMetric entries; + + public CacheStats(long hits, long misses, long evictions, long sizeInBytes, long entries) { + this.hits = new CounterMetric(); + this.hits.inc(hits); + this.misses = new CounterMetric(); + this.misses.inc(misses); + this.evictions = new CounterMetric(); + this.evictions.inc(evictions); + this.sizeInBytes = new CounterMetric(); + this.sizeInBytes.inc(sizeInBytes); + this.entries = new CounterMetric(); + this.entries.inc(entries); + } + + public CacheStats() { + this(0, 0, 0, 0, 0); + } + + private void internalAdd(long otherHits, long otherMisses, long otherEvictions, long otherSizeInBytes, long otherEntries) { + this.hits.inc(otherHits); + this.misses.inc(otherMisses); + this.evictions.inc(otherEvictions); + this.sizeInBytes.inc(otherSizeInBytes); + this.entries.inc(otherEntries); + } + + public void add(CacheStats other) { + if (other == null) { + return; + } + internalAdd(other.getHits(), other.getMisses(), other.getEvictions(), other.getSizeInBytes(), other.getEntries()); + } + + public void add(CacheStatsSnapshot snapshot) { + if (snapshot == null) { + return; + } + internalAdd(snapshot.getHits(), snapshot.getMisses(), snapshot.getEvictions(), snapshot.getSizeInBytes(), snapshot.getEntries()); + } + + public void subtract(CacheStatsSnapshot other) { + if (other == null) { + return; + } + internalAdd(-other.getHits(), -other.getMisses(), -other.getEvictions(), -other.getSizeInBytes(), -other.getEntries()); + } + + @Override + public int hashCode() { + return Objects.hash(hits.count(), misses.count(), evictions.count(), sizeInBytes.count(), entries.count()); + } + + public void incrementHits() { + hits.inc(); + } + + public void incrementMisses() { + misses.inc(); + } + + public void incrementEvictions() { + evictions.inc(); + } + + public void incrementSizeInBytes(long amount) { + sizeInBytes.inc(amount); + } + + public void decrementSizeInBytes(long amount) { + sizeInBytes.dec(amount); + } + + public void incrementEntries() { + entries.inc(); + } + + public void decrementEntries() { + entries.dec(); + } + + public long getHits() { + return hits.count(); + } - // Method to get all 5 values at once - CacheStatsCounterSnapshot getTotalStats(); + public long getMisses() { + return misses.count(); + } - // Methods to get total values. - long getTotalHits(); + public long getEvictions() { + return evictions.count(); + } - long getTotalMisses(); + public long getSizeInBytes() { + return sizeInBytes.count(); + } - long getTotalEvictions(); + public long getEntries() { + return entries.count(); + } - long getTotalSizeInBytes(); + public void resetSizeAndEntries() { + sizeInBytes = new CounterMetric(); + entries = new CounterMetric(); + } - long getTotalEntries(); + public CacheStatsSnapshot snapshot() { + return new CacheStatsSnapshot(hits.count(), misses.count(), evictions.count(), sizeInBytes.count(), entries.count()); + } } diff --git a/server/src/main/java/org/opensearch/common/cache/stats/CacheStatsCounter.java b/server/src/main/java/org/opensearch/common/cache/stats/CacheStatsCounter.java deleted file mode 100644 index afd9620405d0a..0000000000000 --- a/server/src/main/java/org/opensearch/common/cache/stats/CacheStatsCounter.java +++ /dev/null @@ -1,132 +0,0 @@ -/* - * 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.common.cache.stats; - -import org.opensearch.common.metrics.CounterMetric; - -import java.util.Objects; - -/** - * A class containing the 5 live metrics tracked by a StatsHolder object. Mutable. - */ -public class CacheStatsCounter { - CounterMetric hits; - CounterMetric misses; - CounterMetric evictions; - CounterMetric sizeInBytes; - CounterMetric entries; - - public CacheStatsCounter(long hits, long misses, long evictions, long sizeInBytes, long entries) { - this.hits = new CounterMetric(); - this.hits.inc(hits); - this.misses = new CounterMetric(); - this.misses.inc(misses); - this.evictions = new CounterMetric(); - this.evictions.inc(evictions); - this.sizeInBytes = new CounterMetric(); - this.sizeInBytes.inc(sizeInBytes); - this.entries = new CounterMetric(); - this.entries.inc(entries); - } - - public CacheStatsCounter() { - this(0, 0, 0, 0, 0); - } - - private void internalAdd(long otherHits, long otherMisses, long otherEvictions, long otherSizeInBytes, long otherEntries) { - this.hits.inc(otherHits); - this.misses.inc(otherMisses); - this.evictions.inc(otherEvictions); - this.sizeInBytes.inc(otherSizeInBytes); - this.entries.inc(otherEntries); - } - - public void add(CacheStatsCounter other) { - if (other == null) { - return; - } - internalAdd(other.getHits(), other.getMisses(), other.getEvictions(), other.getSizeInBytes(), other.getEntries()); - } - - public void add(CacheStatsCounterSnapshot snapshot) { - if (snapshot == null) { - return; - } - internalAdd(snapshot.getHits(), snapshot.getMisses(), snapshot.getEvictions(), snapshot.getSizeInBytes(), snapshot.getEntries()); - } - - public void subtract(CacheStatsCounterSnapshot other) { - if (other == null) { - return; - } - internalAdd(-other.getHits(), -other.getMisses(), -other.getEvictions(), -other.getSizeInBytes(), -other.getEntries()); - } - - @Override - public int hashCode() { - return Objects.hash(hits.count(), misses.count(), evictions.count(), sizeInBytes.count(), entries.count()); - } - - public void incrementHits() { - hits.inc(); - } - - public void incrementMisses() { - misses.inc(); - } - - public void incrementEvictions() { - evictions.inc(); - } - - public void incrementSizeInBytes(long amount) { - sizeInBytes.inc(amount); - } - - public void decrementSizeInBytes(long amount) { - sizeInBytes.dec(amount); - } - - public void incrementEntries() { - entries.inc(); - } - - public void decrementEntries() { - entries.dec(); - } - - public long getHits() { - return hits.count(); - } - - public long getMisses() { - return misses.count(); - } - - public long getEvictions() { - return evictions.count(); - } - - public long getSizeInBytes() { - return sizeInBytes.count(); - } - - public long getEntries() { - return entries.count(); - } - - public void resetSizeAndEntries() { - sizeInBytes = new CounterMetric(); - entries = new CounterMetric(); - } - - public CacheStatsCounterSnapshot snapshot() { - return new CacheStatsCounterSnapshot(hits.count(), misses.count(), evictions.count(), sizeInBytes.count(), entries.count()); - } -} diff --git a/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java b/server/src/main/java/org/opensearch/common/cache/stats/CacheStatsHolder.java similarity index 55% rename from server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java rename to server/src/main/java/org/opensearch/common/cache/stats/CacheStatsHolder.java index 559f56ab66272..7103047cf7a3a 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/CacheStatsHolder.java @@ -9,24 +9,25 @@ package org.opensearch.common.cache.stats; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; -import static org.opensearch.common.cache.stats.MultiDimensionCacheStats.MDCSDimensionNode; - /** - * A class caches use to internally keep track of their stats across multiple dimensions. - * Not intended to be exposed outside the cache; for this, use statsHolder.getCacheStats() to create an immutable + * A class ICache implementations use to internally keep track of their stats across multiple dimensions. + * Not intended to be exposed outside the cache; for this, caches use getImmutableCacheStatsHolder() to create an immutable * copy of the current state of the stats. - * Currently, in the IRC, the stats tracked in a StatsHolder will not appear for empty shards that have had no cache + * Currently, in the IRC, the stats tracked in a CacheStatsHolder will not appear for empty shards that have had no cache * operations done on them yet. This might be changed in the future, by exposing a method to add empty nodes to the - * tree in StatsHolder in the ICache interface. + * tree in CacheStatsHolder in the ICache interface. * * @opensearch.experimental */ -public class StatsHolder { +public class CacheStatsHolder { // The list of permitted dimensions. Should be ordered from "outermost" to "innermost", as you would like to // aggregate them in an API response. @@ -35,14 +36,14 @@ public class StatsHolder { // 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 // nodes that share a parent, that parent's dimension value will only be stored once, not many times. - private final DimensionNode statsRoot; + private final Node statsRoot; // To avoid sync problems, obtain a lock before creating or removing nodes in the stats tree. // No lock is needed to edit stats on existing nodes. private final Lock lock = new ReentrantLock(); - public StatsHolder(List dimensionNames) { + public CacheStatsHolder(List dimensionNames) { this.dimensionNames = Collections.unmodifiableList(dimensionNames); - this.statsRoot = new DimensionNode("", true); // The root node has the empty string as its dimension value + this.statsRoot = new Node("", true); // The root node has the empty string as its dimension value } public List getDimensionNames() { @@ -52,15 +53,15 @@ public List getDimensionNames() { // For all these increment functions, the dimensions list comes from the key, and contains all dimensions present in dimensionNames. // The order has to match the order given in dimensionNames. public void incrementHits(List dimensionValues) { - internalIncrement(dimensionValues, DimensionNode::incrementHits, true); + internalIncrement(dimensionValues, Node::incrementHits, true); } public void incrementMisses(List dimensionValues) { - internalIncrement(dimensionValues, DimensionNode::incrementMisses, true); + internalIncrement(dimensionValues, Node::incrementMisses, true); } public void incrementEvictions(List dimensionValues) { - internalIncrement(dimensionValues, DimensionNode::incrementEvictions, true); + internalIncrement(dimensionValues, Node::incrementEvictions, true); } public void incrementSizeInBytes(List dimensionValues, long amountBytes) { @@ -74,11 +75,11 @@ public void decrementSizeInBytes(List dimensionValues, long amountBytes) } public void incrementEntries(List dimensionValues) { - internalIncrement(dimensionValues, DimensionNode::incrementEntries, true); + internalIncrement(dimensionValues, Node::incrementEntries, true); } public void decrementEntries(List dimensionValues) { - internalIncrement(dimensionValues, DimensionNode::decrementEntries, false); + internalIncrement(dimensionValues, Node::decrementEntries, false); } /** @@ -89,9 +90,9 @@ public void reset() { resetHelper(statsRoot); } - private void resetHelper(DimensionNode current) { + private void resetHelper(Node current) { current.resetSizeAndEntries(); - for (DimensionNode child : current.children.values()) { + for (Node child : current.children.values()) { resetHelper(child); } } @@ -101,15 +102,15 @@ public long count() { return statsRoot.getEntries(); } - private void internalIncrement(List dimensionValues, Consumer adder, boolean createNodesIfAbsent) { + private 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); // If we failed to increment, because nodes had to be created, obtain the lock and run again while creating nodes if needed - if (!didIncrement) { + if (!didIncrement && createNodesIfAbsent) { try { lock.lock(); - internalIncrementHelper(dimensionValues, statsRoot, 0, adder, createNodesIfAbsent); + internalIncrementHelper(dimensionValues, statsRoot, 0, adder, true); } finally { lock.unlock(); } @@ -123,9 +124,9 @@ private void internalIncrement(List dimensionValues, Consumer dimensionValues, - DimensionNode node, + Node node, int depth, // Pass in the depth to avoid having to slice the list for each node. - Consumer adder, + Consumer adder, boolean createNodesIfAbsent ) { if (depth == dimensionValues.size()) { @@ -134,7 +135,7 @@ private boolean internalIncrementHelper( return true; } - DimensionNode child = node.getChild(dimensionValues.get(depth)); + Node child = node.getChild(dimensionValues.get(depth)); if (child == null) { if (createNodesIfAbsent) { boolean createMapInChild = depth < dimensionValues.size() - 1; @@ -152,31 +153,31 @@ private boolean internalIncrementHelper( } /** - * Produce an immutable CacheStats representation of these stats. + * Produce an immutable version of these stats. */ - public CacheStats getCacheStats() { - MDCSDimensionNode snapshot = new MDCSDimensionNode("", true, statsRoot.getStatsSnapshot()); + public ImmutableCacheStatsHolder getImmutableCacheStatsHolder() { + ImmutableCacheStatsHolder.Node snapshot = new ImmutableCacheStatsHolder.Node("", true, statsRoot.getStatsSnapshot()); // Traverse the tree and build a corresponding tree of MDCSDimensionNode, to pass to MultiDimensionCacheStats. if (statsRoot.getChildren() != null) { - for (DimensionNode child : statsRoot.getChildren().values()) { - getCacheStatsHelper(child, snapshot); + for (Node child : statsRoot.getChildren().values()) { + getImmutableCacheStatsHelper(child, snapshot); } } - return new MultiDimensionCacheStats(snapshot, dimensionNames); + return new ImmutableCacheStatsHolder(snapshot, dimensionNames); } - private void getCacheStatsHelper(DimensionNode currentNodeInOriginalTree, MDCSDimensionNode parentInNewTree) { - MDCSDimensionNode newNode = createMatchingMDCSDimensionNode(currentNodeInOriginalTree); + private void getImmutableCacheStatsHelper(Node currentNodeInOriginalTree, ImmutableCacheStatsHolder.Node parentInNewTree) { + ImmutableCacheStatsHolder.Node newNode = createMatchingImmutableCacheStatsHolderNode(currentNodeInOriginalTree); parentInNewTree.getChildren().put(newNode.getDimensionValue(), newNode); - for (DimensionNode child : currentNodeInOriginalTree.children.values()) { - getCacheStatsHelper(child, newNode); + for (Node child : currentNodeInOriginalTree.children.values()) { + getImmutableCacheStatsHelper(child, newNode); } } - private MDCSDimensionNode createMatchingMDCSDimensionNode(DimensionNode node) { - CacheStatsCounterSnapshot nodeSnapshot = node.getStatsSnapshot(); + private ImmutableCacheStatsHolder.Node createMatchingImmutableCacheStatsHolderNode(Node node) { + CacheStatsSnapshot nodeSnapshot = node.getStatsSnapshot(); boolean isLeafNode = node.getChildren().isEmpty(); - return new MDCSDimensionNode(node.getDimensionValue(), !isLeafNode, nodeSnapshot); + return new ImmutableCacheStatsHolder.Node(node.getDimensionValue(), !isLeafNode, nodeSnapshot); } public void removeDimensions(List dimensionValues) { @@ -191,16 +192,16 @@ public void removeDimensions(List dimensionValues) { } // Returns a CacheStatsCounterSnapshot object for the stats to decrement if the removal happened, null otherwise. - private CacheStatsCounterSnapshot removeDimensionsHelper(List dimensionValues, DimensionNode node, int depth) { + private CacheStatsSnapshot removeDimensionsHelper(List dimensionValues, Node node, int depth) { if (depth == dimensionValues.size()) { // Pass up a snapshot of the original stats to avoid issues when the original is decremented by other fn invocations return node.getStatsSnapshot(); } - DimensionNode child = node.getChild(dimensionValues.get(depth)); + Node child = node.getChild(dimensionValues.get(depth)); if (child == null) { return null; } - CacheStatsCounterSnapshot statsToDecrement = removeDimensionsHelper(dimensionValues, child, depth + 1); + CacheStatsSnapshot statsToDecrement = removeDimensionsHelper(dimensionValues, child, depth + 1); if (statsToDecrement != null) { // The removal took place, decrement values and remove this node from its parent if it's now empty node.decrementBySnapshot(statsToDecrement); @@ -212,7 +213,92 @@ private CacheStatsCounterSnapshot removeDimensionsHelper(List dimensionV } // pkg-private for testing - DimensionNode getStatsRoot() { + Node getStatsRoot() { return statsRoot; } + + static class Node { + private final String dimensionValue; + // Map from dimensionValue to the DimensionNode for that dimension value. + final Map children; + // The stats for this node. If a leaf node, corresponds to the stats for this combination of dimensions; if not, + // contains the sum of its children's stats. + private CacheStats stats; + + // Used for leaf nodes to avoid allocating many unnecessary maps + private static final Map EMPTY_CHILDREN_MAP = new HashMap<>(); + + Node(String dimensionValue, boolean createChildrenMap) { + this.dimensionValue = dimensionValue; + if (createChildrenMap) { + this.children = new ConcurrentHashMap<>(); + } else { + this.children = EMPTY_CHILDREN_MAP; + } + this.stats = new CacheStats(); + } + + public String getDimensionValue() { + return dimensionValue; + } + + protected Map getChildren() { + // We can safely iterate over ConcurrentHashMap without worrying about thread issues. + return children; + } + + // Functions for modifying internal CacheStatsCounter without callers having to be aware of CacheStatsCounter + + void incrementHits() { + this.stats.incrementHits(); + } + + void incrementMisses() { + this.stats.incrementMisses(); + } + + void incrementEvictions() { + this.stats.incrementEvictions(); + } + + void incrementSizeInBytes(long amountBytes) { + this.stats.incrementSizeInBytes(amountBytes); + } + + void decrementSizeInBytes(long amountBytes) { + this.stats.decrementSizeInBytes(amountBytes); + } + + void incrementEntries() { + this.stats.incrementEntries(); + } + + void decrementEntries() { + this.stats.decrementEntries(); + } + + long getEntries() { + return this.stats.getEntries(); + } + + CacheStatsSnapshot getStatsSnapshot() { + return this.stats.snapshot(); + } + + void decrementBySnapshot(CacheStatsSnapshot snapshot) { + this.stats.subtract(snapshot); + } + + void resetSizeAndEntries() { + this.stats.resetSizeAndEntries(); + } + + Node getChild(String dimensionValue) { + return children.get(dimensionValue); + } + + Node createChild(String dimensionValue, boolean createMapInChild) { + return children.computeIfAbsent(dimensionValue, (key) -> new Node(dimensionValue, createMapInChild)); + } + } } diff --git a/server/src/main/java/org/opensearch/common/cache/stats/CacheStatsCounterSnapshot.java b/server/src/main/java/org/opensearch/common/cache/stats/CacheStatsSnapshot.java similarity index 78% rename from server/src/main/java/org/opensearch/common/cache/stats/CacheStatsCounterSnapshot.java rename to server/src/main/java/org/opensearch/common/cache/stats/CacheStatsSnapshot.java index 3057edd8b2afc..80c3b2855c0dd 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/CacheStatsCounterSnapshot.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/CacheStatsSnapshot.java @@ -17,19 +17,19 @@ import java.util.Objects; /** - * An immutable snapshot of CacheStatsCounter. + * An immutable snapshot of CacheStats. * * @opensearch.experimental */ @ExperimentalApi -public class CacheStatsCounterSnapshot implements Writeable { // TODO: Make this extend ToXContent (in API PR) +public class CacheStatsSnapshot implements Writeable { // TODO: Make this extend ToXContent (in API PR) private final long hits; private final long misses; private final long evictions; private final long sizeInBytes; private final long entries; - public CacheStatsCounterSnapshot(long hits, long misses, long evictions, long sizeInBytes, long entries) { + public CacheStatsSnapshot(long hits, long misses, long evictions, long sizeInBytes, long entries) { this.hits = hits; this.misses = misses; this.evictions = evictions; @@ -37,12 +37,12 @@ public CacheStatsCounterSnapshot(long hits, long misses, long evictions, long si this.entries = entries; } - public CacheStatsCounterSnapshot(StreamInput in) throws IOException { + public CacheStatsSnapshot(StreamInput in) throws IOException { this(in.readVLong(), in.readVLong(), in.readVLong(), in.readVLong(), in.readVLong()); } - public static CacheStatsCounterSnapshot addSnapshots(CacheStatsCounterSnapshot s1, CacheStatsCounterSnapshot s2) { - return new CacheStatsCounterSnapshot( + public static CacheStatsSnapshot addSnapshots(CacheStatsSnapshot s1, CacheStatsSnapshot s2) { + return new CacheStatsSnapshot( s1.hits + s2.hits, s1.misses + s2.misses, s1.evictions + s2.evictions, @@ -85,10 +85,10 @@ public boolean equals(Object o) { if (o == null) { return false; } - if (o.getClass() != CacheStatsCounterSnapshot.class) { + if (o.getClass() != CacheStatsSnapshot.class) { return false; } - CacheStatsCounterSnapshot other = (CacheStatsCounterSnapshot) o; + CacheStatsSnapshot other = (CacheStatsSnapshot) o; return (hits == other.hits) && (misses == other.misses) && (evictions == other.evictions) diff --git a/server/src/main/java/org/opensearch/common/cache/stats/DimensionNode.java b/server/src/main/java/org/opensearch/common/cache/stats/DimensionNode.java deleted file mode 100644 index 7abd9b00d3d9a..0000000000000 --- a/server/src/main/java/org/opensearch/common/cache/stats/DimensionNode.java +++ /dev/null @@ -1,101 +0,0 @@ -/* - * 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.common.cache.stats; - -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; - -/** - * A node in a tree structure, which stores stats in StatsHolder. - */ -class DimensionNode { - private final String dimensionValue; - // Map from dimensionValue to the DimensionNode for that dimension value. - final Map children; - // The stats for this node. If a leaf node, corresponds to the stats for this combination of dimensions; if not, - // contains the sum of its children's stats. - private CacheStatsCounter stats; - - // Used for leaf nodes to avoid allocating many unnecessary maps - private static final Map EMPTY_CHILDREN_MAP = new HashMap<>(); - - DimensionNode(String dimensionValue, boolean createChildrenMap) { - this.dimensionValue = dimensionValue; - if (createChildrenMap) { - this.children = new ConcurrentHashMap<>(); - } else { - this.children = EMPTY_CHILDREN_MAP; - } - this.stats = new CacheStatsCounter(); - } - - public String getDimensionValue() { - return dimensionValue; - } - - protected Map getChildren() { - // We can safely iterate over ConcurrentHashMap without worrying about thread issues. - return children; - } - - // Functions for modifying internal CacheStatsCounter without callers having to be aware of CacheStatsCounter - - void incrementHits() { - this.stats.incrementHits(); - } - - void incrementMisses() { - this.stats.incrementMisses(); - } - - void incrementEvictions() { - this.stats.incrementEvictions(); - } - - void incrementSizeInBytes(long amountBytes) { - this.stats.incrementSizeInBytes(amountBytes); - } - - void decrementSizeInBytes(long amountBytes) { - this.stats.decrementSizeInBytes(amountBytes); - } - - void incrementEntries() { - this.stats.incrementEntries(); - } - - void decrementEntries() { - this.stats.decrementEntries(); - } - - long getEntries() { - return this.stats.getEntries(); - } - - CacheStatsCounterSnapshot getStatsSnapshot() { - return this.stats.snapshot(); - } - - void decrementBySnapshot(CacheStatsCounterSnapshot snapshot) { - this.stats.subtract(snapshot); - } - - void resetSizeAndEntries() { - this.stats.resetSizeAndEntries(); - } - - DimensionNode getChild(String dimensionValue) { - return children.get(dimensionValue); - } - - DimensionNode createChild(String dimensionValue, boolean createMapInChild) { - return children.computeIfAbsent(dimensionValue, (key) -> new DimensionNode(dimensionValue, createMapInChild)); - } -} diff --git a/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java b/server/src/main/java/org/opensearch/common/cache/stats/ImmutableCacheStatsHolder.java similarity index 61% rename from server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java rename to server/src/main/java/org/opensearch/common/cache/stats/ImmutableCacheStatsHolder.java index 627e2a59bc87e..ddcc3f2974d79 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/ImmutableCacheStatsHolder.java @@ -8,60 +8,57 @@ package org.opensearch.common.cache.stats; +import org.opensearch.common.annotation.ExperimentalApi; + import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.TreeMap; /** - * A CacheStats object supporting aggregation over multiple different dimensions. - * Stores a fixed snapshot of a cache's stats; does not allow changes. + * An object storing an immutable snapshot of an entire cache's stats. Accessible outside the cache itself. * * @opensearch.experimental */ -public class MultiDimensionCacheStats implements CacheStats { + +@ExperimentalApi +public class ImmutableCacheStatsHolder { // TODO: extends Writeable, ToXContent // A snapshot of a StatsHolder containing stats maintained by the cache. // Pkg-private for testing. - final MDCSDimensionNode statsRoot; + final Node statsRoot; final List dimensionNames; - public MultiDimensionCacheStats(MDCSDimensionNode statsRoot, List dimensionNames) { + public ImmutableCacheStatsHolder(Node statsRoot, List dimensionNames) { this.statsRoot = statsRoot; this.dimensionNames = dimensionNames; } - @Override - public CacheStatsCounterSnapshot getTotalStats() { + public CacheStatsSnapshot getTotalStats() { return statsRoot.getStats(); } - @Override public long getTotalHits() { return getTotalStats().getHits(); } - @Override public long getTotalMisses() { return getTotalStats().getMisses(); } - @Override public long getTotalEvictions() { return getTotalStats().getEvictions(); } - @Override public long getTotalSizeInBytes() { return getTotalStats().getSizeInBytes(); } - @Override public long getTotalEntries() { return getTotalStats().getEntries(); } - public CacheStatsCounterSnapshot getStatsForDimensionValues(List dimensionValues) { - MDCSDimensionNode current = statsRoot; + public CacheStatsSnapshot getStatsForDimensionValues(List dimensionValues) { + Node current = statsRoot; for (String dimensionValue : dimensionValues) { current = current.children.get(dimensionValue); if (current == null) { @@ -71,17 +68,17 @@ public CacheStatsCounterSnapshot getStatsForDimensionValues(List dimensi return current.stats; } - // A similar class to DimensionNode, which uses an ordered TreeMap and holds immutable CacheStatsCounterSnapshot as its stats. - static class MDCSDimensionNode { + // A similar class to CacheStatsHolder.Node, which uses an ordered TreeMap and holds immutable CacheStatsSnapshot as its stats. + static class Node { private final String dimensionValue; - final Map children; // Map from dimensionValue to the DimensionNode for that dimension value + final Map children; // Map from dimensionValue to the Node for that dimension value // The stats for this node. If a leaf node, corresponds to the stats for this combination of dimensions; if not, // contains the sum of its children's stats. - private CacheStatsCounterSnapshot stats; - private static final Map EMPTY_CHILDREN_MAP = new HashMap<>(); + private CacheStatsSnapshot stats; + private static final Map EMPTY_CHILDREN_MAP = new HashMap<>(); - MDCSDimensionNode(String dimensionValue, boolean createChildrenMap, CacheStatsCounterSnapshot stats) { + Node(String dimensionValue, boolean createChildrenMap, CacheStatsSnapshot stats) { this.dimensionValue = dimensionValue; if (createChildrenMap) { this.children = new TreeMap<>(); // This map should be ordered to enforce a consistent order in API response @@ -91,15 +88,15 @@ static class MDCSDimensionNode { this.stats = stats; } - Map getChildren() { + Map getChildren() { return children; } - public CacheStatsCounterSnapshot getStats() { + public CacheStatsSnapshot getStats() { return stats; } - public void setStats(CacheStatsCounterSnapshot stats) { + public void setStats(CacheStatsSnapshot stats) { this.stats = stats; } @@ -109,7 +106,7 @@ public String getDimensionValue() { } // pkg-private for testing - MDCSDimensionNode getStatsRoot() { + Node getStatsRoot() { return statsRoot; } 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 2e60072d07ed2..29e5667c9f27d 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 @@ -18,8 +18,8 @@ import org.opensearch.common.cache.RemovalNotification; import org.opensearch.common.cache.RemovalReason; import org.opensearch.common.cache.settings.CacheSettings; -import org.opensearch.common.cache.stats.CacheStats; -import org.opensearch.common.cache.stats.StatsHolder; +import org.opensearch.common.cache.stats.CacheStatsHolder; +import org.opensearch.common.cache.stats.ImmutableCacheStatsHolder; import org.opensearch.common.cache.store.builders.ICacheBuilder; import org.opensearch.common.cache.store.config.CacheConfig; import org.opensearch.common.cache.store.settings.OpenSearchOnHeapCacheSettings; @@ -47,7 +47,7 @@ public class OpenSearchOnHeapCache implements ICache, RemovalListener, V> { private final Cache, V> cache; - private final StatsHolder statsHolder; + private final CacheStatsHolder cacheStatsHolder; private final RemovalListener, V> removalListener; private final List dimensionNames; private final ToLongBiFunction, V> weigher; @@ -62,7 +62,7 @@ public OpenSearchOnHeapCache(Builder builder) { } cache = cacheBuilder.build(); this.dimensionNames = Objects.requireNonNull(builder.dimensionNames, "Dimension names can't be null"); - this.statsHolder = new StatsHolder(dimensionNames); + this.cacheStatsHolder = new CacheStatsHolder(dimensionNames); this.removalListener = builder.getRemovalListener(); this.weigher = builder.getWeigher(); } @@ -71,9 +71,9 @@ public OpenSearchOnHeapCache(Builder builder) { public V get(ICacheKey key) { V value = cache.get(key); if (value != null) { - statsHolder.incrementHits(key.dimensions); + cacheStatsHolder.incrementHits(key.dimensions); } else { - statsHolder.incrementMisses(key.dimensions); + cacheStatsHolder.incrementMisses(key.dimensions); } return value; } @@ -81,19 +81,19 @@ public V get(ICacheKey key) { @Override public void put(ICacheKey key, V value) { cache.put(key, value); - statsHolder.incrementEntries(key.dimensions); - statsHolder.incrementSizeInBytes(key.dimensions, weigher.applyAsLong(key, value)); + cacheStatsHolder.incrementEntries(key.dimensions); + cacheStatsHolder.incrementSizeInBytes(key.dimensions, weigher.applyAsLong(key, value)); } @Override public V computeIfAbsent(ICacheKey key, LoadAwareCacheLoader, V> loader) throws Exception { V value = cache.computeIfAbsent(key, key1 -> loader.load(key)); if (!loader.isLoaded()) { - statsHolder.incrementHits(key.dimensions); + cacheStatsHolder.incrementHits(key.dimensions); } else { - statsHolder.incrementMisses(key.dimensions); - statsHolder.incrementEntries(key.dimensions); - statsHolder.incrementSizeInBytes(key.dimensions, cache.getWeigher().applyAsLong(key, value)); + cacheStatsHolder.incrementMisses(key.dimensions); + cacheStatsHolder.incrementEntries(key.dimensions); + cacheStatsHolder.incrementSizeInBytes(key.dimensions, cache.getWeigher().applyAsLong(key, value)); } return value; } @@ -101,7 +101,7 @@ public V computeIfAbsent(ICacheKey key, LoadAwareCacheLoader, V> @Override public void invalidate(ICacheKey key) { if (key.getDropStatsForDimensions()) { - statsHolder.removeDimensions(key.dimensions); + cacheStatsHolder.removeDimensions(key.dimensions); } if (key.key != null) { cache.invalidate(key); @@ -111,7 +111,7 @@ public void invalidate(ICacheKey key) { @Override public void invalidateAll() { cache.invalidateAll(); - statsHolder.reset(); + cacheStatsHolder.reset(); } @Override @@ -121,7 +121,7 @@ public Iterable> keys() { @Override public long count() { - return statsHolder.count(); + return cacheStatsHolder.count(); } @Override @@ -133,22 +133,22 @@ public void refresh() { public void close() {} @Override - public CacheStats stats() { - return statsHolder.getCacheStats(); + public ImmutableCacheStatsHolder stats() { + return cacheStatsHolder.getImmutableCacheStatsHolder(); } @Override public void onRemoval(RemovalNotification, V> notification) { removalListener.onRemoval(notification); - statsHolder.decrementEntries(notification.getKey().dimensions); - statsHolder.decrementSizeInBytes( + cacheStatsHolder.decrementEntries(notification.getKey().dimensions); + cacheStatsHolder.decrementSizeInBytes( notification.getKey().dimensions, cache.getWeigher().applyAsLong(notification.getKey(), notification.getValue()) ); if (RemovalReason.EVICTED.equals(notification.getRemovalReason()) || RemovalReason.CAPACITY.equals(notification.getRemovalReason())) { - statsHolder.incrementEvictions(notification.getKey().dimensions); + cacheStatsHolder.incrementEvictions(notification.getKey().dimensions); } } diff --git a/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java b/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java index bad23591fd727..1dcc8384c43dd 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java +++ b/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java @@ -49,7 +49,7 @@ import org.opensearch.common.cache.policy.CachedQueryResult; import org.opensearch.common.cache.serializer.BytesReferenceSerializer; import org.opensearch.common.cache.service.CacheService; -import org.opensearch.common.cache.stats.CacheStats; +import org.opensearch.common.cache.stats.ImmutableCacheStatsHolder; import org.opensearch.common.cache.store.config.CacheConfig; import org.opensearch.common.lease.Releasable; import org.opensearch.common.lucene.index.OpenSearchDirectoryReader; @@ -763,7 +763,7 @@ long getSizeInBytes() { /** * Returns the current cache stats. Pkg-private for testing. */ - CacheStats getCacheStats() { + ImmutableCacheStatsHolder stats() { return cache.stats(); } diff --git a/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java b/server/src/test/java/org/opensearch/common/cache/stats/CacheStatsHolderTests.java similarity index 57% rename from server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java rename to server/src/test/java/org/opensearch/common/cache/stats/CacheStatsHolderTests.java index d351572e05d74..c757fa0e23fb3 100644 --- a/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java +++ b/server/src/test/java/org/opensearch/common/cache/stats/CacheStatsHolderTests.java @@ -21,112 +21,112 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; -public class StatsHolderTests extends OpenSearchTestCase { +public class CacheStatsHolderTests extends OpenSearchTestCase { public void testAddAndGet() throws Exception { List dimensionNames = List.of("dim1", "dim2", "dim3", "dim4"); - StatsHolder statsHolder = new StatsHolder(dimensionNames); - Map> usedDimensionValues = StatsHolderTests.getUsedDimensionValues(statsHolder, 10); - Map, CacheStatsCounter> expected = StatsHolderTests.populateStats(statsHolder, usedDimensionValues, 1000, 10); + CacheStatsHolder cacheStatsHolder = new CacheStatsHolder(dimensionNames); + Map> usedDimensionValues = CacheStatsHolderTests.getUsedDimensionValues(cacheStatsHolder, 10); + Map, CacheStats> expected = CacheStatsHolderTests.populateStats(cacheStatsHolder, usedDimensionValues, 1000, 10); // test the value in the map is as expected for each distinct combination of values for (List dimensionValues : expected.keySet()) { - CacheStatsCounter expectedCounter = expected.get(dimensionValues); + CacheStats expectedCounter = expected.get(dimensionValues); - CacheStatsCounterSnapshot actualStatsHolder = StatsHolderTests.getNode(dimensionValues, statsHolder.getStatsRoot()) + CacheStatsSnapshot actualStatsHolder = CacheStatsHolderTests.getNode(dimensionValues, cacheStatsHolder.getStatsRoot()) .getStatsSnapshot(); - CacheStatsCounterSnapshot actualCacheStats = getNode(dimensionValues, statsHolder.getStatsRoot()).getStatsSnapshot(); + CacheStatsSnapshot actualCacheStats = getNode(dimensionValues, cacheStatsHolder.getStatsRoot()).getStatsSnapshot(); assertEquals(expectedCounter.snapshot(), actualStatsHolder); assertEquals(expectedCounter.snapshot(), actualCacheStats); } // Check overall total matches - CacheStatsCounter expectedTotal = new CacheStatsCounter(); + CacheStats expectedTotal = new CacheStats(); for (List dims : expected.keySet()) { expectedTotal.add(expected.get(dims)); } - assertEquals(expectedTotal.snapshot(), statsHolder.getStatsRoot().getStatsSnapshot()); + assertEquals(expectedTotal.snapshot(), cacheStatsHolder.getStatsRoot().getStatsSnapshot()); // Check sum of children stats are correct - assertSumOfChildrenStats(statsHolder.getStatsRoot()); + assertSumOfChildrenStats(cacheStatsHolder.getStatsRoot()); } public void testReset() throws Exception { List dimensionNames = List.of("dim1", "dim2"); - StatsHolder statsHolder = new StatsHolder(dimensionNames); - Map> usedDimensionValues = getUsedDimensionValues(statsHolder, 10); - Map, CacheStatsCounter> expected = populateStats(statsHolder, usedDimensionValues, 100, 10); + CacheStatsHolder cacheStatsHolder = new CacheStatsHolder(dimensionNames); + Map> usedDimensionValues = getUsedDimensionValues(cacheStatsHolder, 10); + Map, CacheStats> expected = populateStats(cacheStatsHolder, usedDimensionValues, 100, 10); - statsHolder.reset(); + cacheStatsHolder.reset(); for (List dimensionValues : expected.keySet()) { - CacheStatsCounter originalCounter = expected.get(dimensionValues); + CacheStats originalCounter = expected.get(dimensionValues); originalCounter.sizeInBytes = new CounterMetric(); originalCounter.entries = new CounterMetric(); - DimensionNode node = getNode(dimensionValues, statsHolder.getStatsRoot()); - CacheStatsCounterSnapshot actual = node.getStatsSnapshot(); + CacheStatsHolder.Node node = getNode(dimensionValues, cacheStatsHolder.getStatsRoot()); + CacheStatsSnapshot actual = node.getStatsSnapshot(); assertEquals(originalCounter.snapshot(), actual); } } public void testDropStatsForDimensions() throws Exception { List dimensionNames = List.of("dim1", "dim2"); - StatsHolder statsHolder = new StatsHolder(dimensionNames); + CacheStatsHolder cacheStatsHolder = new CacheStatsHolder(dimensionNames); // Create stats for the following dimension sets List> populatedStats = List.of(List.of("A1", "B1"), List.of("A2", "B2"), List.of("A2", "B3")); for (List dims : populatedStats) { - statsHolder.incrementHits(dims); + cacheStatsHolder.incrementHits(dims); } - assertEquals(3, statsHolder.getStatsRoot().getStatsSnapshot().getHits()); + assertEquals(3, cacheStatsHolder.getStatsRoot().getStatsSnapshot().getHits()); // When we invalidate A2, B2, we should lose the node for B2, but not B3 or A2. - statsHolder.removeDimensions(List.of("A2", "B2")); + cacheStatsHolder.removeDimensions(List.of("A2", "B2")); - assertEquals(2, statsHolder.getStatsRoot().getStatsSnapshot().getHits()); - assertNull(getNode(List.of("A2", "B2"), statsHolder.getStatsRoot())); - assertNotNull(getNode(List.of("A2"), statsHolder.getStatsRoot())); - assertNotNull(getNode(List.of("A2", "B3"), statsHolder.getStatsRoot())); + assertEquals(2, cacheStatsHolder.getStatsRoot().getStatsSnapshot().getHits()); + assertNull(getNode(List.of("A2", "B2"), cacheStatsHolder.getStatsRoot())); + assertNotNull(getNode(List.of("A2"), cacheStatsHolder.getStatsRoot())); + assertNotNull(getNode(List.of("A2", "B3"), cacheStatsHolder.getStatsRoot())); // When we invalidate A1, B1, we should lose the nodes for B1 and also A1, as it has no more children. - statsHolder.removeDimensions(List.of("A1", "B1")); + cacheStatsHolder.removeDimensions(List.of("A1", "B1")); - assertEquals(1, statsHolder.getStatsRoot().getStatsSnapshot().getHits()); - assertNull(getNode(List.of("A1", "B1"), statsHolder.getStatsRoot())); - assertNull(getNode(List.of("A1"), statsHolder.getStatsRoot())); + assertEquals(1, cacheStatsHolder.getStatsRoot().getStatsSnapshot().getHits()); + assertNull(getNode(List.of("A1", "B1"), cacheStatsHolder.getStatsRoot())); + assertNull(getNode(List.of("A1"), cacheStatsHolder.getStatsRoot())); // When we invalidate the last node, all nodes should be deleted except the root node - statsHolder.removeDimensions(List.of("A2", "B3")); - assertEquals(0, statsHolder.getStatsRoot().getStatsSnapshot().getHits()); - assertEquals(0, statsHolder.getStatsRoot().children.size()); + cacheStatsHolder.removeDimensions(List.of("A2", "B3")); + assertEquals(0, cacheStatsHolder.getStatsRoot().getStatsSnapshot().getHits()); + assertEquals(0, cacheStatsHolder.getStatsRoot().children.size()); } public void testCount() throws Exception { List dimensionNames = List.of("dim1", "dim2"); - StatsHolder statsHolder = new StatsHolder(dimensionNames); - Map> usedDimensionValues = getUsedDimensionValues(statsHolder, 10); - Map, CacheStatsCounter> expected = populateStats(statsHolder, usedDimensionValues, 100, 10); + CacheStatsHolder cacheStatsHolder = new CacheStatsHolder(dimensionNames); + Map> usedDimensionValues = getUsedDimensionValues(cacheStatsHolder, 10); + Map, CacheStats> expected = populateStats(cacheStatsHolder, usedDimensionValues, 100, 10); long expectedCount = 0L; - for (CacheStatsCounter counter : expected.values()) { + for (CacheStats counter : expected.values()) { expectedCount += counter.getEntries(); } - assertEquals(expectedCount, statsHolder.count()); + assertEquals(expectedCount, cacheStatsHolder.count()); } public void testConcurrentRemoval() throws Exception { List dimensionNames = List.of("dim1", "dim2"); - StatsHolder statsHolder = new StatsHolder(dimensionNames); + CacheStatsHolder cacheStatsHolder = new CacheStatsHolder(dimensionNames); // Create stats for the following dimension sets List> populatedStats = List.of(List.of("A1", "B1"), List.of("A2", "B2"), List.of("A2", "B3")); for (List dims : populatedStats) { - statsHolder.incrementHits(dims); + cacheStatsHolder.incrementHits(dims); } // Remove (A2, B2) and (A1, B1), before re-adding (A2, B2). At the end we should have stats for (A2, B2) but not (A1, B1). @@ -134,16 +134,16 @@ public void testConcurrentRemoval() throws Exception { Thread[] threads = new Thread[3]; CountDownLatch countDownLatch = new CountDownLatch(3); threads[0] = new Thread(() -> { - statsHolder.removeDimensions(List.of("A2", "B2")); + cacheStatsHolder.removeDimensions(List.of("A2", "B2")); countDownLatch.countDown(); }); threads[1] = new Thread(() -> { - statsHolder.removeDimensions(List.of("A1", "B1")); + cacheStatsHolder.removeDimensions(List.of("A1", "B1")); countDownLatch.countDown(); }); threads[2] = new Thread(() -> { - statsHolder.incrementMisses(List.of("A2", "B2")); - statsHolder.incrementMisses(List.of("A2", "B3")); + cacheStatsHolder.incrementMisses(List.of("A2", "B2")); + cacheStatsHolder.incrementMisses(List.of("A2", "B3")); countDownLatch.countDown(); }); for (Thread thread : threads) { @@ -152,16 +152,16 @@ public void testConcurrentRemoval() throws Exception { Thread.sleep(1); } countDownLatch.await(); - assertNull(getNode(List.of("A1", "B1"), statsHolder.getStatsRoot())); - assertNull(getNode(List.of("A1"), statsHolder.getStatsRoot())); - assertNotNull(getNode(List.of("A2", "B2"), statsHolder.getStatsRoot())); + assertNull(getNode(List.of("A1", "B1"), cacheStatsHolder.getStatsRoot())); + assertNull(getNode(List.of("A1"), cacheStatsHolder.getStatsRoot())); + assertNotNull(getNode(List.of("A2", "B2"), cacheStatsHolder.getStatsRoot())); assertEquals( - new CacheStatsCounterSnapshot(0, 1, 0, 0, 0), - getNode(List.of("A2", "B2"), statsHolder.getStatsRoot()).getStatsSnapshot() + new CacheStatsSnapshot(0, 1, 0, 0, 0), + getNode(List.of("A2", "B2"), cacheStatsHolder.getStatsRoot()).getStatsSnapshot() ); assertEquals( - new CacheStatsCounterSnapshot(1, 1, 0, 0, 0), - getNode(List.of("A2", "B3"), statsHolder.getStatsRoot()).getStatsSnapshot() + new CacheStatsSnapshot(1, 1, 0, 0, 0), + getNode(List.of("A2", "B3"), cacheStatsHolder.getStatsRoot()).getStatsSnapshot() ); } @@ -169,8 +169,8 @@ public void testConcurrentRemoval() throws Exception { * Returns the node found by following these dimension values down from the root node. * Returns null if no such node exists. */ - static DimensionNode getNode(List dimensionValues, DimensionNode root) { - DimensionNode current = root; + static CacheStatsHolder.Node getNode(List dimensionValues, CacheStatsHolder.Node root) { + CacheStatsHolder.Node current = root; for (String dimensionValue : dimensionValues) { current = current.getChildren().get(dimensionValue); if (current == null) { @@ -180,26 +180,26 @@ static DimensionNode getNode(List dimensionValues, DimensionNode root) { return current; } - static Map, CacheStatsCounter> populateStats( - StatsHolder statsHolder, + static Map, CacheStats> populateStats( + CacheStatsHolder cacheStatsHolder, Map> usedDimensionValues, int numDistinctValuePairs, int numRepetitionsPerValue ) throws InterruptedException { - Map, CacheStatsCounter> expected = new ConcurrentHashMap<>(); + Map, CacheStats> expected = new ConcurrentHashMap<>(); Thread[] threads = new Thread[numDistinctValuePairs]; CountDownLatch countDownLatch = new CountDownLatch(numDistinctValuePairs); Random rand = Randomness.get(); List> dimensionsForThreads = new ArrayList<>(); for (int i = 0; i < numDistinctValuePairs; i++) { - dimensionsForThreads.add(getRandomDimList(statsHolder.getDimensionNames(), usedDimensionValues, true, rand)); + dimensionsForThreads.add(getRandomDimList(cacheStatsHolder.getDimensionNames(), usedDimensionValues, true, rand)); int finalI = i; threads[i] = new Thread(() -> { Random threadRand = Randomness.get(); List dimensions = dimensionsForThreads.get(finalI); - expected.computeIfAbsent(dimensions, (key) -> new CacheStatsCounter()); + expected.computeIfAbsent(dimensions, (key) -> new CacheStats()); for (int j = 0; j < numRepetitionsPerValue; j++) { - CacheStatsCounter statsToInc = new CacheStatsCounter( + CacheStats statsToInc = new CacheStats( threadRand.nextInt(10), threadRand.nextInt(10), threadRand.nextInt(10), @@ -211,7 +211,7 @@ static Map, CacheStatsCounter> populateStats( expected.get(dimensions).evictions.inc(statsToInc.getEvictions()); expected.get(dimensions).sizeInBytes.inc(statsToInc.getSizeInBytes()); expected.get(dimensions).entries.inc(statsToInc.getEntries()); - StatsHolderTests.populateStatsHolderFromStatsValueMap(statsHolder, Map.of(dimensions, statsToInc)); + CacheStatsHolderTests.populateStatsHolderFromStatsValueMap(cacheStatsHolder, Map.of(dimensions, statsToInc)); } countDownLatch.countDown(); }); @@ -240,47 +240,47 @@ private static List getRandomDimList( return result; } - static Map> getUsedDimensionValues(StatsHolder statsHolder, int numValuesPerDim) { + static Map> getUsedDimensionValues(CacheStatsHolder cacheStatsHolder, int numValuesPerDim) { Map> usedDimensionValues = new HashMap<>(); - for (int i = 0; i < statsHolder.getDimensionNames().size(); i++) { + for (int i = 0; i < cacheStatsHolder.getDimensionNames().size(); i++) { List values = new ArrayList<>(); for (int j = 0; j < numValuesPerDim; j++) { values.add(UUID.randomUUID().toString()); } - usedDimensionValues.put(statsHolder.getDimensionNames().get(i), values); + usedDimensionValues.put(cacheStatsHolder.getDimensionNames().get(i), values); } return usedDimensionValues; } - private void assertSumOfChildrenStats(DimensionNode current) { + private void assertSumOfChildrenStats(CacheStatsHolder.Node current) { if (!current.children.isEmpty()) { - CacheStatsCounter expectedTotal = new CacheStatsCounter(); - for (DimensionNode child : current.children.values()) { + CacheStats expectedTotal = new CacheStats(); + for (CacheStatsHolder.Node child : current.children.values()) { expectedTotal.add(child.getStatsSnapshot()); } assertEquals(expectedTotal.snapshot(), current.getStatsSnapshot()); - for (DimensionNode child : current.children.values()) { + for (CacheStatsHolder.Node child : current.children.values()) { assertSumOfChildrenStats(child); } } } - static void populateStatsHolderFromStatsValueMap(StatsHolder statsHolder, Map, CacheStatsCounter> statsMap) { - for (Map.Entry, CacheStatsCounter> entry : statsMap.entrySet()) { - CacheStatsCounter stats = entry.getValue(); + static void populateStatsHolderFromStatsValueMap(CacheStatsHolder cacheStatsHolder, Map, CacheStats> statsMap) { + for (Map.Entry, CacheStats> entry : statsMap.entrySet()) { + CacheStats stats = entry.getValue(); List dims = entry.getKey(); for (int i = 0; i < stats.getHits(); i++) { - statsHolder.incrementHits(dims); + cacheStatsHolder.incrementHits(dims); } for (int i = 0; i < stats.getMisses(); i++) { - statsHolder.incrementMisses(dims); + cacheStatsHolder.incrementMisses(dims); } for (int i = 0; i < stats.getEvictions(); i++) { - statsHolder.incrementEvictions(dims); + cacheStatsHolder.incrementEvictions(dims); } - statsHolder.incrementSizeInBytes(dims, stats.getSizeInBytes()); + cacheStatsHolder.incrementSizeInBytes(dims, stats.getSizeInBytes()); for (int i = 0; i < stats.getEntries(); i++) { - statsHolder.incrementEntries(dims); + cacheStatsHolder.incrementEntries(dims); } } } diff --git a/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java b/server/src/test/java/org/opensearch/common/cache/stats/ImmutableCacheStatsHolderTests.java similarity index 54% rename from server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java rename to server/src/test/java/org/opensearch/common/cache/stats/ImmutableCacheStatsHolderTests.java index 460398961d94f..2ae7434a05552 100644 --- a/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java +++ b/server/src/test/java/org/opensearch/common/cache/stats/ImmutableCacheStatsHolderTests.java @@ -13,29 +13,29 @@ import java.util.List; import java.util.Map; -public class MultiDimensionCacheStatsTests extends OpenSearchTestCase { +public class ImmutableCacheStatsHolderTests extends OpenSearchTestCase { public void testGet() throws Exception { List dimensionNames = List.of("dim1", "dim2", "dim3", "dim4"); - StatsHolder statsHolder = new StatsHolder(dimensionNames); - Map> usedDimensionValues = StatsHolderTests.getUsedDimensionValues(statsHolder, 10); - Map, CacheStatsCounter> expected = StatsHolderTests.populateStats(statsHolder, usedDimensionValues, 1000, 10); - MultiDimensionCacheStats stats = (MultiDimensionCacheStats) statsHolder.getCacheStats(); + CacheStatsHolder cacheStatsHolder = new CacheStatsHolder(dimensionNames); + Map> usedDimensionValues = CacheStatsHolderTests.getUsedDimensionValues(cacheStatsHolder, 10); + Map, CacheStats> expected = CacheStatsHolderTests.populateStats(cacheStatsHolder, usedDimensionValues, 1000, 10); + ImmutableCacheStatsHolder stats = cacheStatsHolder.getImmutableCacheStatsHolder(); // test the value in the map is as expected for each distinct combination of values for (List dimensionValues : expected.keySet()) { - CacheStatsCounter expectedCounter = expected.get(dimensionValues); + CacheStats expectedCounter = expected.get(dimensionValues); - CacheStatsCounterSnapshot actualStatsHolder = StatsHolderTests.getNode(dimensionValues, statsHolder.getStatsRoot()) + CacheStatsSnapshot actualStatsHolder = CacheStatsHolderTests.getNode(dimensionValues, cacheStatsHolder.getStatsRoot()) .getStatsSnapshot(); - CacheStatsCounterSnapshot actualCacheStats = getNode(dimensionValues, stats.getStatsRoot()).getStats(); + CacheStatsSnapshot actualCacheStats = getNode(dimensionValues, stats.getStatsRoot()).getStats(); assertEquals(expectedCounter.snapshot(), actualStatsHolder); assertEquals(expectedCounter.snapshot(), actualCacheStats); } // test gets for total (this also checks sum-of-children logic) - CacheStatsCounter expectedTotal = new CacheStatsCounter(); + CacheStats expectedTotal = new CacheStats(); for (List dims : expected.keySet()) { expectedTotal.add(expected.get(dims)); } @@ -52,21 +52,18 @@ public void testGet() throws Exception { public void testEmptyDimsList() throws Exception { // If the dimension list is empty, the tree should have only the root node containing the total stats. - StatsHolder statsHolder = new StatsHolder(List.of()); - Map> usedDimensionValues = StatsHolderTests.getUsedDimensionValues(statsHolder, 100); - StatsHolderTests.populateStats(statsHolder, usedDimensionValues, 10, 100); - MultiDimensionCacheStats stats = (MultiDimensionCacheStats) statsHolder.getCacheStats(); + CacheStatsHolder cacheStatsHolder = new CacheStatsHolder(List.of()); + Map> usedDimensionValues = CacheStatsHolderTests.getUsedDimensionValues(cacheStatsHolder, 100); + CacheStatsHolderTests.populateStats(cacheStatsHolder, usedDimensionValues, 10, 100); + ImmutableCacheStatsHolder stats = cacheStatsHolder.getImmutableCacheStatsHolder(); - MultiDimensionCacheStats.MDCSDimensionNode statsRoot = stats.getStatsRoot(); + ImmutableCacheStatsHolder.Node statsRoot = stats.getStatsRoot(); assertEquals(0, statsRoot.children.size()); assertEquals(stats.getTotalStats(), statsRoot.getStats()); } - private MultiDimensionCacheStats.MDCSDimensionNode getNode( - List dimensionValues, - MultiDimensionCacheStats.MDCSDimensionNode root - ) { - MultiDimensionCacheStats.MDCSDimensionNode current = root; + private ImmutableCacheStatsHolder.Node getNode(List dimensionValues, ImmutableCacheStatsHolder.Node root) { + ImmutableCacheStatsHolder.Node current = root; for (String dimensionValue : dimensionValues) { current = current.getChildren().get(dimensionValue); if (current == null) { @@ -76,14 +73,14 @@ private MultiDimensionCacheStats.MDCSDimensionNode getNode( return current; } - private void assertSumOfChildrenStats(MultiDimensionCacheStats.MDCSDimensionNode current) { + private void assertSumOfChildrenStats(ImmutableCacheStatsHolder.Node current) { if (!current.children.isEmpty()) { - CacheStatsCounter expectedTotal = new CacheStatsCounter(); - for (MultiDimensionCacheStats.MDCSDimensionNode child : current.children.values()) { + CacheStats expectedTotal = new CacheStats(); + for (ImmutableCacheStatsHolder.Node child : current.children.values()) { expectedTotal.add(child.getStats()); } assertEquals(expectedTotal.snapshot(), current.getStats()); - for (MultiDimensionCacheStats.MDCSDimensionNode child : current.children.values()) { + for (ImmutableCacheStatsHolder.Node child : current.children.values()) { assertSumOfChildrenStats(child); } } 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 8b667e86d155c..72b3c2c5bc7df 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 @@ -15,8 +15,7 @@ import org.opensearch.common.cache.LoadAwareCacheLoader; import org.opensearch.common.cache.RemovalListener; import org.opensearch.common.cache.RemovalNotification; -import org.opensearch.common.cache.stats.CacheStatsCounterSnapshot; -import org.opensearch.common.cache.stats.MultiDimensionCacheStats; +import org.opensearch.common.cache.stats.CacheStatsSnapshot; import org.opensearch.common.cache.store.config.CacheConfig; import org.opensearch.common.cache.store.settings.OpenSearchOnHeapCacheSettings; import org.opensearch.common.metrics.CounterMetric; @@ -115,7 +114,7 @@ public void testInvalidateWithDropDimensions() throws Exception { ICacheKey keyToDrop = keysAdded.get(0); - CacheStatsCounterSnapshot snapshot = ((MultiDimensionCacheStats) cache.stats()).getStatsForDimensionValues(keyToDrop.dimensions); + CacheStatsSnapshot snapshot = cache.stats().getStatsForDimensionValues(keyToDrop.dimensions); assertNotNull(snapshot); keyToDrop.setDropStatsForDimensions(true); @@ -123,7 +122,7 @@ public void testInvalidateWithDropDimensions() throws Exception { // Now assert the stats are gone for any key that has this combination of dimensions, but still there otherwise for (ICacheKey keyAdded : keysAdded) { - snapshot = ((MultiDimensionCacheStats) cache.stats()).getStatsForDimensionValues(keyAdded.dimensions); + snapshot = cache.stats().getStatsForDimensionValues(keyAdded.dimensions); if (keyAdded.dimensions.equals(keyToDrop.dimensions)) { assertNull(snapshot); } else { diff --git a/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java b/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java index fbea7424af0c6..09803c097eb80 100644 --- a/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java +++ b/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java @@ -52,8 +52,7 @@ import org.opensearch.common.cache.RemovalReason; import org.opensearch.common.cache.module.CacheModule; import org.opensearch.common.cache.service.CacheService; -import org.opensearch.common.cache.stats.CacheStatsCounterSnapshot; -import org.opensearch.common.cache.stats.MultiDimensionCacheStats; +import org.opensearch.common.cache.stats.CacheStatsSnapshot; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.lucene.index.OpenSearchDirectoryReader; import org.opensearch.common.settings.Settings; @@ -826,9 +825,7 @@ public void testClosingIndexWipesStats() throws Exception { ShardId shardId = indexService.getShard(i).shardId(); List dimensionValues = List.of(shardId.getIndexName(), shardId.toString()); initialDimensionValues.add(dimensionValues); - CacheStatsCounterSnapshot snapshot = ((MultiDimensionCacheStats) cache.getCacheStats()).getStatsForDimensionValues( - dimensionValues - ); + CacheStatsSnapshot snapshot = cache.stats().getStatsForDimensionValues(dimensionValues); assertNotNull(snapshot); // check the values are not empty by confirming entries != 0, this should always be true since the missed value is loaded // into the cache @@ -849,9 +846,7 @@ public void testClosingIndexWipesStats() throws Exception { // Now stats for the closed index should be gone for (List dimensionValues : initialDimensionValues) { - CacheStatsCounterSnapshot snapshot = ((MultiDimensionCacheStats) cache.getCacheStats()).getStatsForDimensionValues( - dimensionValues - ); + CacheStatsSnapshot snapshot = cache.stats().getStatsForDimensionValues(dimensionValues); if (dimensionValues.get(0).equals(indexToCloseName)) { assertNull(snapshot); } else {