From cfdd89c11fb03bbba0178d7c27d0bdab65ffe042 Mon Sep 17 00:00:00 2001 From: Peter Alfonsi Date: Thu, 14 Mar 2024 21:29:46 -0700 Subject: [PATCH 01/17] Added TSC stats implementation Signed-off-by: Peter Alfonsi --- .../common/tier/TieredSpilloverCache.java | 140 +++++-- .../tier/TieredSpilloverCacheStats.java | 158 ++++++++ .../tier/TieredSpilloverCacheStatsTests.java | 361 ++++++++++++++++++ .../opensearch/common/cache/ICacheKey.java | 1 + .../cache/stats/MultiDimensionCacheStats.java | 4 +- .../common/cache/stats/StatsHolder.java | 8 +- 6 files changed, 646 insertions(+), 26 deletions(-) create mode 100644 modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStats.java create mode 100644 modules/cache-common/src/test/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsTests.java 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 13d4492854441..c813855e9f081 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 @@ -15,8 +15,11 @@ import org.opensearch.common.cache.LoadAwareCacheLoader; import org.opensearch.common.cache.RemovalListener; import org.opensearch.common.cache.RemovalNotification; +import org.opensearch.common.cache.RemovalReason; import org.opensearch.common.cache.stats.CacheStats; +import org.opensearch.common.cache.stats.StatsHolder; 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.util.concurrent.ReleasableLock; @@ -30,6 +33,7 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Function; +import java.util.function.ToLongBiFunction; /** * This cache spillover the evicted items from heap tier to disk tier. All the new items are first cached on heap @@ -47,10 +51,18 @@ public class TieredSpilloverCache implements ICache { private final ICache diskCache; private final ICache onHeapCache; + private final RemovalListener, V> onDiskRemovalListener; + private final RemovalListener, V> onHeapRemovalListener; + // The listener for removals from the spillover cache as a whole - // TODO: In TSC stats PR, each tier will have its own separate removal listener. private final RemovalListener, V> removalListener; - private final CacheStats stats; + + // 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 StatsHolder heapStats; + private final StatsHolder diskStats; + + private ToLongBiFunction, V> weigher; private final List dimensionNames; ReadWriteLock readWriteLock = new ReentrantReadWriteLock(); ReleasableLock readLock = new ReleasableLock(readWriteLock.readLock()); @@ -59,21 +71,19 @@ public class TieredSpilloverCache implements ICache { * Maintains caching tiers in ascending order of cache latency. */ private final List> cacheList; + private final List, StatsHolder>> cacheAndStatsList; TieredSpilloverCache(Builder builder) { Objects.requireNonNull(builder.onHeapCacheFactory, "onHeap cache builder can't be null"); Objects.requireNonNull(builder.diskCacheFactory, "disk cache builder 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()) { - diskCache.put(notification.getKey(), notification.getValue()); - } - } - }) + new CacheConfig.Builder().setRemovalListener(onHeapRemovalListener) .setKeyType(builder.cacheConfig.getKeyType()) .setValueType(builder.cacheConfig.getValueType()) .setSettings(builder.cacheConfig.getSettings()) @@ -85,18 +95,26 @@ public void onRemoval(RemovalNotification, V> notification) { ); this.diskCache = builder.diskCacheFactory.create( - new CacheConfig.Builder().setRemovalListener(removalListener) // TODO: change + 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()) .build(), builder.cacheType, builder.cacheFactories ); this.cacheList = Arrays.asList(onHeapCache, diskCache); - this.stats = null; // TODO - in next stats rework PR + this.dimensionNames = builder.cacheConfig.getDimensionNames(); + this.heapStats = new StatsHolder(dimensionNames); + this.diskStats = new StatsHolder(dimensionNames); + this.cacheAndStatsList = List.of( + new Tuple<>(onHeapCache, heapStats), + new Tuple<>(diskCache, diskStats) + ); + } // Package private for testing @@ -118,6 +136,7 @@ public V get(ICacheKey key) { public void put(ICacheKey key, V value) { try (ReleasableLock ignore = writeLock.acquire()) { onHeapCache.put(key, value); + updateStatsOnPut(heapStats, key, value); } } @@ -132,6 +151,10 @@ public V computeIfAbsent(ICacheKey key, LoadAwareCacheLoader, V> V value = null; try (ReleasableLock ignore = writeLock.acquire()) { value = onHeapCache.computeIfAbsent(key, loader); + if (loader.isLoaded()) { + // The value was just computed and added to the cache + updateStatsOnPut(heapStats, key, value); + } } return value; } @@ -143,6 +166,7 @@ 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 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. try (ReleasableLock ignore = writeLock.acquire()) { for (ICache cache : cacheList) { cache.invalidate(key); @@ -157,6 +181,8 @@ public void invalidateAll() { cache.invalidateAll(); } } + heapStats.reset(); + diskStats.reset(); } /** @@ -171,11 +197,7 @@ public Iterable> keys() { @Override public long count() { - long count = 0; - for (ICache cache : cacheList) { - count += cache.count(); - } - return count; + return heapStats.count() + diskStats.count(); } @Override @@ -196,19 +218,19 @@ public void close() throws IOException { @Override public CacheStats stats() { - return stats; + return new TieredSpilloverCacheStats(heapStats.createSnapshot(), diskStats.createSnapshot(), dimensionNames); } private Function, V> getValueFromTieredCache() { return key -> { try (ReleasableLock ignore = readLock.acquire()) { - for (ICache cache : cacheList) { - V value = cache.get(key); + for (Tuple, StatsHolder> pair : cacheAndStatsList) { + V value = pair.v1().get(key); if (value != null) { - // update hit stats + pair.v2().incrementHits(key); return value; } else { - // update miss stats + pair.v2().incrementMisses(key); } } } @@ -216,6 +238,80 @@ private Function, V> getValueFromTieredCache() { }; } + void handleRemovalFromHeapTier(RemovalNotification, V> notification) { + ICacheKey key = notification.getKey(); + + boolean wasEvicted = false; + if (RemovalReason.EVICTED.equals(notification.getRemovalReason()) + || RemovalReason.CAPACITY.equals(notification.getRemovalReason())) { + try (ReleasableLock ignore = writeLock.acquire()) { + diskCache.put(key, notification.getValue()); // spill over to the disk tier and increment its stats + updateStatsOnPut(diskStats, key, notification.getValue()); + } + wasEvicted = true; + } + + else { + // If the removal was for another reason, send this notification to the TSC's removal listener, as the value is leaving the TSC entirely + removalListener.onRemoval(notification); + } + updateStatsOnRemoval(heapStats, wasEvicted, key, notification.getValue()); + } + + void handleRemovalFromDiskTier(RemovalNotification, V> notification) { + // Values removed from the disk tier leave the TSC entirely + removalListener.onRemoval(notification); + + boolean wasEvicted = false; + if (RemovalReason.EVICTED.equals(notification.getRemovalReason()) + || RemovalReason.CAPACITY.equals(notification.getRemovalReason())) { + wasEvicted = true; + } + updateStatsOnRemoval(diskStats, wasEvicted, notification.getKey(), notification.getValue()); + } + + void updateStatsOnRemoval(StatsHolder statsHolder, boolean wasEvicted, ICacheKey key, V value) { + if (wasEvicted) { + statsHolder.incrementEvictions(key); + } + statsHolder.decrementEntries(key); + statsHolder.incrementSizeInBytes(key, -weigher.applyAsLong(key, value)); + } + + void updateStatsOnPut(StatsHolder statsHolder, ICacheKey key, V value) { + statsHolder.incrementEntries(key); + statsHolder.incrementSizeInBytes(key, weigher.applyAsLong(key, value)); + } + + /** + * 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); + } + } + /** * Factory to create TieredSpilloverCache objects. */ diff --git a/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStats.java b/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStats.java new file mode 100644 index 0000000000000..3215283860ae3 --- /dev/null +++ b/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStats.java @@ -0,0 +1,158 @@ +/* + * 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.CacheStats; +import org.opensearch.common.cache.stats.CacheStatsDimension; +import org.opensearch.common.cache.stats.CacheStatsResponse; +import org.opensearch.common.cache.stats.MultiDimensionCacheStats; +import org.opensearch.common.cache.stats.StatsHolder; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + + +/** + * A CacheStats implementation for TieredSpilloverCache, which keeps track of the stats for its tiers. + */ +public class TieredSpilloverCacheStats implements CacheStats { + // Pkg-private for testing + final MultiDimensionCacheStats heapStats; + final MultiDimensionCacheStats diskStats; + + public static final String TIER_DIMENSION_NAME = "tier"; + public static final String TIER_DIMENSION_VALUE_ON_HEAP = "on_heap"; + public static final String TIER_DIMENSION_VALUE_DISK = "disk"; + public static final List HEAP_DIMS = List.of(new CacheStatsDimension(TIER_DIMENSION_NAME, TIER_DIMENSION_VALUE_ON_HEAP)); + public static final List DISK_DIMS = List.of(new CacheStatsDimension(TIER_DIMENSION_NAME, TIER_DIMENSION_VALUE_DISK)); + + public TieredSpilloverCacheStats( + Map heapSnapshot, + Map diskSnapshot, + List dimensionNames) { + this.heapStats = new MultiDimensionCacheStats(heapSnapshot, dimensionNames); + this.diskStats = new MultiDimensionCacheStats(diskSnapshot, dimensionNames); + } + + public TieredSpilloverCacheStats(StreamInput in) throws IOException { + this.heapStats = new MultiDimensionCacheStats(in); + this.diskStats = new MultiDimensionCacheStats(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + heapStats.writeTo(out); + diskStats.writeTo(out); + } + + @Override + public CacheStatsResponse.Snapshot getTotalStats() { + return combineTierResponses(heapStats.getTotalStats(), diskStats.getTotalStats()); + } + + public TreeMap aggregateByLevels(List levels) { + TreeMap result = new TreeMap<>(new MultiDimensionCacheStats.KeyComparator()); + if (levels.contains(TIER_DIMENSION_NAME)) { + // Aggregate by tier. Get the aggregations from each MultiDimensionCacheStats, and combine them into a single + // TreeMap, adding the tier dimension to each TreeMap key. + List noTierLevels = new ArrayList<>(levels); // levels might be immutable + noTierLevels.remove(TIER_DIMENSION_NAME); + TreeMap heapAgg = heapStats.aggregateByLevels(noTierLevels); + TreeMap diskAgg = diskStats.aggregateByLevels(noTierLevels); + + addKeysWithTierDimension(result, heapAgg, TIER_DIMENSION_VALUE_ON_HEAP); + addKeysWithTierDimension(result, diskAgg, TIER_DIMENSION_VALUE_DISK); + } + else { + // Don't aggregate by tier. Get aggregations from each MultiDimensionCacheStats. Combine them using combineTierResponses + // if both aggregations share a key. Otherwise, add directly from the only one which has the key. + TreeMap heapAgg = heapStats.aggregateByLevels(levels); + TreeMap diskAgg = diskStats.aggregateByLevels(levels); + + for (Map.Entry entry : heapAgg.entrySet()) { + CacheStatsResponse.Snapshot heapValue = entry.getValue(); + CacheStatsResponse.Snapshot diskValue = diskAgg.get(entry.getKey()); + StatsHolder.Key key = entry.getKey(); + if (diskValue == null) { + // Only the heap agg has this particular combination of values, add directly to result + result.put(key, heapValue); + } else { + // Both aggregations have this combination, combine them before adding and then remove from diskAgg to avoid double-counting + CacheStatsResponse.Snapshot combined = combineTierResponses(heapValue, diskValue); + result.put(key, combined); + diskAgg.remove(key); + } + } + // The remaining keys are only present in diskAgg + result.putAll(diskAgg); + } + return result; + } + + // Add all keys in originalAggregation to result, but first add tierDimName to the end of the key. + private void addKeysWithTierDimension(TreeMap result, + TreeMap originalAggregation, + String tierDimName) { + for (Map.Entry entry : originalAggregation.entrySet()) { + List newDimensions = new ArrayList<>(entry.getKey().getDimensionValues()); + newDimensions.add(tierDimName); // Tier dimension is at the end as it's the innermost dimension in API responses + StatsHolder.Key newKey = new StatsHolder.Key(newDimensions); + result.put(newKey, entry.getValue()); + } + } + + // pkg-private for testing + static CacheStatsResponse.Snapshot combineTierResponses(CacheStatsResponse.Snapshot heap, CacheStatsResponse.Snapshot disk) { + return new CacheStatsResponse.Snapshot( + heap.getHits() + disk.getHits(), + disk.getMisses(), + disk.getEvictions(), + heap.getSizeInBytes() + disk.getSizeInBytes(), + heap.getEntries() + disk.getEntries() + ); + } + + @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(); + } + + CacheStatsResponse.Snapshot getTotalHeapStats() { + return heapStats.getTotalStats(); + } + + CacheStatsResponse.Snapshot getTotalDiskStats() { + return diskStats.getTotalStats(); + } +} diff --git a/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsTests.java b/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsTests.java new file mode 100644 index 0000000000000..ac57809d1414a --- /dev/null +++ b/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsTests.java @@ -0,0 +1,361 @@ +/* + * 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.Randomness; +import org.opensearch.common.cache.Cache; +import org.opensearch.common.cache.ICacheKey; +import org.opensearch.common.cache.stats.CacheStats; +import org.opensearch.common.cache.stats.CacheStatsDimension; +import org.opensearch.common.cache.stats.CacheStatsResponse; +import org.opensearch.common.cache.stats.StatsHolder; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.common.io.stream.BytesStreamInput; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.TreeMap; +import java.util.UUID; + +import static org.opensearch.cache.common.tier.TieredSpilloverCacheStats.TIER_DIMENSION_NAME; +import static org.opensearch.cache.common.tier.TieredSpilloverCacheStats.TIER_DIMENSION_VALUE_ON_HEAP; +import static org.opensearch.cache.common.tier.TieredSpilloverCacheStats.TIER_DIMENSION_VALUE_DISK; +import static org.opensearch.cache.common.tier.TieredSpilloverCacheStats.combineTierResponses; + +public class TieredSpilloverCacheStatsTests extends OpenSearchTestCase { + private static List dimensionNames = List.of("dim1", "dim2", "dim3"); + private static List tierNames = List.of(TIER_DIMENSION_VALUE_ON_HEAP, TIER_DIMENSION_VALUE_DISK); + public void testGets() throws Exception { + StatsHolder heapStats = new StatsHolder(dimensionNames); + StatsHolder diskStats = new StatsHolder(dimensionNames); + Map> usedDimensionValues = getUsedDimensionValues(heapStats, 10); + Map, CacheStatsResponse>> expected = populateStats(heapStats, diskStats, usedDimensionValues, 100, 2); + TieredSpilloverCacheStats stats = new TieredSpilloverCacheStats(heapStats.createSnapshot(), diskStats.createSnapshot(), dimensionNames); + + CacheStatsResponse heapTotalStats = totalSumExpected(expected.get(TIER_DIMENSION_VALUE_ON_HEAP)); + CacheStatsResponse diskTotalStats = totalSumExpected(expected.get(TIER_DIMENSION_VALUE_DISK)); + CacheStatsResponse.Snapshot totalTSCStats = TieredSpilloverCacheStats.combineTierResponses( + heapTotalStats.snapshot(), diskTotalStats.snapshot()); + + // test total gets + assertEquals(totalTSCStats, stats.getTotalStats()); + + assertEquals(totalTSCStats.getHits(), stats.getTotalHits()); + assertEquals(totalTSCStats.getMisses(), stats.getTotalMisses()); + assertEquals(totalTSCStats.getEvictions(), stats.getTotalEvictions()); + assertEquals(totalTSCStats.getSizeInBytes(), stats.getTotalSizeInBytes()); + assertEquals(totalTSCStats.getEntries(), stats.getTotalEntries()); + + assertEquals(heapTotalStats.snapshot(), stats.getTotalHeapStats()); + assertEquals(diskTotalStats.snapshot(), stats.getTotalDiskStats()); + } + + public void testEmptyDimensionNames() throws Exception { + StatsHolder heapStats = new StatsHolder(List.of()); + StatsHolder diskStats = new StatsHolder(List.of()); + + Map> usedDimensionValues = getUsedDimensionValues(heapStats, 10); + Map, CacheStatsResponse>> expected = populateStats(heapStats, diskStats, usedDimensionValues, 10, 10); + TieredSpilloverCacheStats stats = new TieredSpilloverCacheStats(heapStats.createSnapshot(), diskStats.createSnapshot(), List.of()); + + CacheStatsResponse heapTotalStats = totalSumExpected(expected.get(TIER_DIMENSION_VALUE_ON_HEAP)); + CacheStatsResponse diskTotalStats = totalSumExpected(expected.get(TIER_DIMENSION_VALUE_DISK)); + CacheStatsResponse.Snapshot totalTSCStats = TieredSpilloverCacheStats.combineTierResponses(heapTotalStats.snapshot(), diskTotalStats.snapshot()); + + assertEquals(totalTSCStats, stats.getTotalStats()); + } + + public void testSerialization() throws Exception { + StatsHolder heapStats = new StatsHolder(dimensionNames); + StatsHolder diskStats = new StatsHolder(dimensionNames); + Map> usedDimensionValues = getUsedDimensionValues(heapStats, 10); + Map, CacheStatsResponse>> expected = populateStats(heapStats, diskStats, usedDimensionValues, 100, 2); + TieredSpilloverCacheStats stats = new TieredSpilloverCacheStats(heapStats.createSnapshot(), diskStats.createSnapshot(), dimensionNames); + + BytesStreamOutput os = new BytesStreamOutput(); + stats.writeTo(os); + BytesStreamInput is = new BytesStreamInput(BytesReference.toBytes(os.bytes())); + TieredSpilloverCacheStats deserialized = new TieredSpilloverCacheStats(is); + + assertEquals(stats.heapStats.aggregateByLevels(dimensionNames), deserialized.heapStats.aggregateByLevels(dimensionNames)); + assertEquals(stats.diskStats.aggregateByLevels(dimensionNames), deserialized.diskStats.aggregateByLevels(dimensionNames)); + } + + public void testCombineTierResponses() throws Exception { + CacheStatsResponse.Snapshot heapResponse = new CacheStatsResponse.Snapshot(1,2,3,4,5); + CacheStatsResponse.Snapshot diskResponse = new CacheStatsResponse.Snapshot(2,3,4,5,6); + CacheStatsResponse.Snapshot tscResponse = TieredSpilloverCacheStats.combineTierResponses(heapResponse, diskResponse); + assertEquals(new CacheStatsResponse.Snapshot(3, 3, 4, 9, 11), tscResponse); + } + + public void testAggregationAllLevelsWithoutTier() throws Exception { + StatsHolder heapStats = new StatsHolder(dimensionNames); + StatsHolder diskStats = new StatsHolder(dimensionNames); + Map> usedDimensionValues = getUsedDimensionValues(heapStats, 10); + Map, CacheStatsResponse>> expected = populateStats(heapStats, diskStats, usedDimensionValues, 100, 2); + TieredSpilloverCacheStats stats = new TieredSpilloverCacheStats(heapStats.createSnapshot(), diskStats.createSnapshot(), dimensionNames); + + List levels = new ArrayList<>(dimensionNames); + TreeMap agg = stats.aggregateByLevels(levels); + + for (Map.Entry entry : agg.entrySet()) { + StatsHolder.Key key = entry.getKey(); + Set expectedKey = new HashSet<>(); + for (int i = 0; i < dimensionNames.size(); i++) { + expectedKey.add(new CacheStatsDimension(dimensionNames.get(i), key.getDimensionValues().get(i))); + } + + CacheStatsResponse expectedHeapValue = expected.get(TIER_DIMENSION_VALUE_ON_HEAP).get(expectedKey); + CacheStatsResponse expectedDiskValue = expected.get(TIER_DIMENSION_VALUE_DISK).get(expectedKey); + + assertTrue(expectedHeapValue != null || expectedDiskValue != null); + if (expectedHeapValue != null && expectedDiskValue != null) { + assertEquals(combineTierResponses(expectedHeapValue.snapshot(), expectedDiskValue.snapshot()), entry.getValue()); + } else if (expectedHeapValue != null) { + assertEquals(expectedHeapValue.snapshot(), entry.getValue()); + } else { + // disk value not null + assertEquals(expectedDiskValue.snapshot(), entry.getValue()); + } + } + + } + + public void testAggregationSomeLevelsWithoutTier() throws Exception { + StatsHolder heapStats = new StatsHolder(dimensionNames); + StatsHolder diskStats = new StatsHolder(dimensionNames); + Map> usedDimensionValues = getUsedDimensionValues(heapStats, 10); + Map, CacheStatsResponse>> expected = populateStats(heapStats, diskStats, usedDimensionValues, 100, 2); + TieredSpilloverCacheStats stats = new TieredSpilloverCacheStats(heapStats.createSnapshot(), diskStats.createSnapshot(), dimensionNames); + + for (int i = 0; i < (1 << dimensionNames.size()); i++) { + // Test each combination of possible levels + List levels = new ArrayList<>(); + for (int nameIndex = 0; nameIndex < dimensionNames.size(); nameIndex++) { + if ((i & (1 << nameIndex)) != 0) { + levels.add(dimensionNames.get(nameIndex)); + } + } + if (levels.size() == 0) { + assertThrows(IllegalArgumentException.class, () -> stats.aggregateByLevels(levels)); + } + else { + Map aggregated = stats.aggregateByLevels(levels); + for (Map.Entry aggregatedEntry : aggregated.entrySet()) { + StatsHolder.Key aggregatedKey = aggregatedEntry.getKey(); + Map expectedResponseForTierMap = new HashMap<>(); + for (String tier : new String[]{TIER_DIMENSION_VALUE_ON_HEAP, TIER_DIMENSION_VALUE_DISK}) { + CacheStatsResponse expectedResponseForTier = new CacheStatsResponse(); + for (Set expectedDims : expected.get(tier).keySet()) { + List orderedDimValues = StatsHolder.getOrderedDimensionValues( + new ArrayList<>(expectedDims), + dimensionNames + ); + if (orderedDimValues.containsAll(aggregatedKey.getDimensionValues())) { + expectedResponseForTier.add(expected.get(tier).get(expectedDims)); + } + } + if (expectedResponseForTier.equals(new CacheStatsResponse())) { + expectedResponseForTier = null; // If it's all 0, there were no keys + } + expectedResponseForTierMap.put(tier, expectedResponseForTier); + } + CacheStatsResponse expectedHeapResponse = expectedResponseForTierMap.get(TIER_DIMENSION_VALUE_ON_HEAP); + CacheStatsResponse expectedDiskResponse = expectedResponseForTierMap.get(TIER_DIMENSION_VALUE_DISK); + if (expectedHeapResponse != null && expectedDiskResponse != null) { + assertEquals(combineTierResponses(expectedHeapResponse.snapshot(), expectedDiskResponse.snapshot()), aggregatedEntry.getValue()); + } else if (expectedHeapResponse != null) { + assertEquals(expectedHeapResponse.snapshot(), aggregatedEntry.getValue()); + } else { + assertEquals(expectedDiskResponse.snapshot(), aggregatedEntry.getValue()); + } + } + } + } + } + + public void testAggregationAllLevelsWithTier() throws Exception { + StatsHolder heapStats = new StatsHolder(dimensionNames); + StatsHolder diskStats = new StatsHolder(dimensionNames); + Map> usedDimensionValues = getUsedDimensionValues(heapStats, 10); + Map, CacheStatsResponse>> expected = populateStats(heapStats, diskStats, usedDimensionValues, 100, 2); + TieredSpilloverCacheStats stats = new TieredSpilloverCacheStats(heapStats.createSnapshot(), diskStats.createSnapshot(), dimensionNames); + + List levels = new ArrayList<>(dimensionNames); + levels.add(TIER_DIMENSION_NAME); + TreeMap agg = stats.aggregateByLevels(levels); + + for (Map.Entry entry : agg.entrySet()) { + StatsHolder.Key key = entry.getKey(); + Set expectedKey = new HashSet<>(); + for (int i = 0; i < dimensionNames.size(); i++) { + expectedKey.add(new CacheStatsDimension(dimensionNames.get(i), key.getDimensionValues().get(i))); + } + String aggTierValue = key.getDimensionValues().get(key.getDimensionValues().size()-1); + assertTrue(aggTierValue.equals(TIER_DIMENSION_VALUE_ON_HEAP) || aggTierValue.equals(TIER_DIMENSION_VALUE_DISK)); + + if (aggTierValue.equals(TIER_DIMENSION_VALUE_ON_HEAP)) { + assertEquals(expected.get(TIER_DIMENSION_VALUE_ON_HEAP).get(expectedKey).snapshot(), entry.getValue()); + } else if (aggTierValue.equals(TIER_DIMENSION_VALUE_DISK)) { + assertEquals(expected.get(TIER_DIMENSION_VALUE_DISK).get(expectedKey).snapshot(), entry.getValue()); + } + } + } + + public void testAggregationSomeLevelsWithTier() throws Exception { + StatsHolder heapStats = new StatsHolder(dimensionNames); + StatsHolder diskStats = new StatsHolder(dimensionNames); + Map> usedDimensionValues = getUsedDimensionValues(heapStats, 10); + Map, CacheStatsResponse>> expected = populateStats(heapStats, diskStats, usedDimensionValues, 100, 2); + TieredSpilloverCacheStats stats = new TieredSpilloverCacheStats(heapStats.createSnapshot(), diskStats.createSnapshot(), dimensionNames); + + for (int i = 0; i < (1 << dimensionNames.size()); i++) { + // Test each combination of possible levels + List levels = new ArrayList<>(); + for (int nameIndex = 0; nameIndex < dimensionNames.size(); nameIndex++) { + if ((i & (1 << nameIndex)) != 0) { + levels.add(dimensionNames.get(nameIndex)); + } + } + levels.add(TIER_DIMENSION_NAME); + if (levels.size() == 1) { + assertThrows(IllegalArgumentException.class, () -> stats.aggregateByLevels(levels)); + } + else { + Map aggregated = stats.aggregateByLevels(levels); + for (Map.Entry aggregatedEntry : aggregated.entrySet()) { + StatsHolder.Key aggregatedKey = aggregatedEntry.getKey(); + String aggregatedKeyTier = aggregatedKey.getDimensionValues().get(aggregatedKey.getDimensionValues().size()-1); + CacheStatsResponse expectedResponse = new CacheStatsResponse(); + for (Set expectedDims : expected.get(aggregatedKeyTier).keySet()) { + List orderedDimValues = StatsHolder.getOrderedDimensionValues( + new ArrayList<>(expectedDims), + dimensionNames + ); + orderedDimValues.add(aggregatedKeyTier); + if (orderedDimValues.containsAll(aggregatedKey.getDimensionValues())) { + expectedResponse.add(expected.get(aggregatedKeyTier).get(expectedDims)); + } + } + assertEquals(expectedResponse.snapshot(), aggregatedEntry.getValue()); + } + } + } + } + + private CacheStatsResponse totalSumExpected(Map, CacheStatsResponse> expected) { + CacheStatsResponse result = new CacheStatsResponse(); + for (Set key : expected.keySet()) { + result.add(expected.get(key)); + } + return result; + } + + // Fill the tier stats and return a nested map from tier type and dimensions -> expected response + // Modified from MultiDimensionCacheStatsTests - we can't import it without adding a dependency on server.test module. + private Map, CacheStatsResponse>> populateStats(StatsHolder heapStats, StatsHolder diskStats, Map> usedDimensionValues, int numDistinctValuePairs, int numRepetitionsPerValue) { + Map, CacheStatsResponse>> expected = new HashMap<>(); + expected.put(TIER_DIMENSION_VALUE_ON_HEAP, new HashMap<>()); + expected.put(TIER_DIMENSION_VALUE_DISK, new HashMap<>()); + + Random rand = Randomness.get(); + Map statsHolderMap = Map.of(tierNames.get(0), heapStats, tierNames.get(1), diskStats); + for (String tier : tierNames) { + for (int i = 0; i < numDistinctValuePairs; i++) { + StatsHolder stats = statsHolderMap.get(tier); + List dimensions = getRandomDimList(stats.getDimensionNames(), usedDimensionValues, true, rand); + Set dimSet = new HashSet<>(dimensions); + Map, CacheStatsResponse> tierExpected = expected.get(tier); + if (tierExpected.get(dimSet) == null) { + tierExpected.put(dimSet, new CacheStatsResponse()); + } + ICacheKey dummyKey = getDummyKey(dimensions); + for (int j = 0; j < numRepetitionsPerValue; j++) { + + int numHitIncrements = rand.nextInt(10); + for (int k = 0; k < numHitIncrements; k++) { + stats.incrementHits(dummyKey); + tierExpected.get(new HashSet<>(dimensions)).hits.inc(); + } + + int numMissIncrements = rand.nextInt(10); + for (int k = 0; k < numMissIncrements; k++) { + stats.incrementMisses(dummyKey); + tierExpected.get(new HashSet<>(dimensions)).misses.inc(); + } + + int numEvictionIncrements = rand.nextInt(10); + for (int k = 0; k < numEvictionIncrements; k++) { + stats.incrementEvictions(dummyKey); + tierExpected.get(new HashSet<>(dimensions)).evictions.inc(); + } + + int numMemorySizeIncrements = rand.nextInt(10); + for (int k = 0; k < numMemorySizeIncrements; k++) { + long memIncrementAmount = rand.nextInt(5000); + stats.incrementSizeInBytes(dummyKey, memIncrementAmount); + tierExpected.get(new HashSet<>(dimensions)).sizeInBytes.inc(memIncrementAmount); + } + + int numEntryIncrements = rand.nextInt(9) + 1; + for (int k = 0; k < numEntryIncrements; k++) { + stats.incrementEntries(dummyKey); + tierExpected.get(new HashSet<>(dimensions)).entries.inc(); + } + + int numEntryDecrements = rand.nextInt(numEntryIncrements); + for (int k = 0; k < numEntryDecrements; k++) { + stats.decrementEntries(dummyKey); + tierExpected.get(new HashSet<>(dimensions)).entries.dec(); + } + } + + } + } + return expected; + } + + // Duplicated below functions from MultiDimensionCacheStatsTests. We can't import them without adding a dependency on server.test for this module. + + private List getRandomDimList(List dimensionNames, Map> usedDimensionValues, boolean pickValueForAllDims, Random rand) { + List result = new ArrayList<>(); + for (String dimName : dimensionNames) { + if (pickValueForAllDims || rand.nextBoolean()) { // if pickValueForAllDims, always pick a value for each dimension, otherwise do so 50% of the time + int index = between(0, usedDimensionValues.get(dimName).size() - 1); + result.add(new CacheStatsDimension(dimName, usedDimensionValues.get(dimName).get(index))); + } + } + return result; + } + private Map> getUsedDimensionValues(StatsHolder stats, int numValuesPerDim) { + Map> usedDimensionValues = new HashMap<>(); + for (int i = 0; i < stats.getDimensionNames().size(); i++) { + List values = new ArrayList<>(); + for (int j = 0; j < numValuesPerDim; j++) { + values.add(UUID.randomUUID().toString()); + } + usedDimensionValues.put(stats.getDimensionNames().get(i), values); + } + return usedDimensionValues; + } + + private static ICacheKey getDummyKey(List dims) { + return new ICacheKey<>(null, dims); + } + +} 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 9b56c9585acb2..418167a783cd9 100644 --- a/server/src/main/java/org/opensearch/common/cache/ICacheKey.java +++ b/server/src/main/java/org/opensearch/common/cache/ICacheKey.java @@ -10,6 +10,7 @@ import org.opensearch.common.cache.stats.CacheStatsDimension; +import java.util.Collections; import java.util.List; public class ICacheKey { diff --git a/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java b/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java index 24dfcb475103e..42327065f961b 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java @@ -120,8 +120,8 @@ public TreeMap aggregateByLevels(L } // First compare outermost dimension, then second outermost, etc. - // Pkg-private for testing - static class KeyComparator implements Comparator { + // Public as it's also used by TieredSpilloverCacheStats + public static class KeyComparator implements Comparator { @Override public int compare(StatsHolder.Key k1, StatsHolder.Key k2) { assert k1.dimensionValues.size() == k2.dimensionValues.size(); diff --git a/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java b/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java index 3b68487e0761a..7143dfecc0b5a 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java @@ -107,7 +107,7 @@ private CacheStatsResponse internalGetStats(List dimensions // Get a list of dimension values, ordered according to dimensionNames, from the possibly differently-ordered dimensions passed in. // Static for testing purposes. - static List getOrderedDimensionValues(List dimensions, List dimensionNames) { + public static List getOrderedDimensionValues(List dimensions, List dimensionNames) { List result = new ArrayList<>(); for (String dimensionName : dimensionNames) { for (CacheStatsDimension dim : dimensions) { @@ -134,7 +134,7 @@ public Map createSnapshot() { public static class Key { final List dimensionValues; // The dimensions must be ordered - Key(List dimensionValues) { + public Key(List dimensionValues) { this.dimensionValues = Collections.unmodifiableList(dimensionValues); } @@ -157,5 +157,9 @@ public boolean equals(Object o) { public int hashCode() { return this.dimensionValues.hashCode(); } + + public List getDimensionValues() { + return dimensionValues; + } } } From 1432abb9d418f26b0d6b589564682625512c8af7 Mon Sep 17 00:00:00 2001 From: Peter Alfonsi Date: Thu, 14 Mar 2024 21:30:59 -0700 Subject: [PATCH 02/17] Removed redundant tests Signed-off-by: Peter Alfonsi --- .../tier/TieredSpilloverCacheStatsTests.java | 61 ------------------- 1 file changed, 61 deletions(-) diff --git a/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsTests.java b/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsTests.java index ac57809d1414a..3cd1edcce5871 100644 --- a/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsTests.java +++ b/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsTests.java @@ -103,39 +103,6 @@ public void testCombineTierResponses() throws Exception { assertEquals(new CacheStatsResponse.Snapshot(3, 3, 4, 9, 11), tscResponse); } - public void testAggregationAllLevelsWithoutTier() throws Exception { - StatsHolder heapStats = new StatsHolder(dimensionNames); - StatsHolder diskStats = new StatsHolder(dimensionNames); - Map> usedDimensionValues = getUsedDimensionValues(heapStats, 10); - Map, CacheStatsResponse>> expected = populateStats(heapStats, diskStats, usedDimensionValues, 100, 2); - TieredSpilloverCacheStats stats = new TieredSpilloverCacheStats(heapStats.createSnapshot(), diskStats.createSnapshot(), dimensionNames); - - List levels = new ArrayList<>(dimensionNames); - TreeMap agg = stats.aggregateByLevels(levels); - - for (Map.Entry entry : agg.entrySet()) { - StatsHolder.Key key = entry.getKey(); - Set expectedKey = new HashSet<>(); - for (int i = 0; i < dimensionNames.size(); i++) { - expectedKey.add(new CacheStatsDimension(dimensionNames.get(i), key.getDimensionValues().get(i))); - } - - CacheStatsResponse expectedHeapValue = expected.get(TIER_DIMENSION_VALUE_ON_HEAP).get(expectedKey); - CacheStatsResponse expectedDiskValue = expected.get(TIER_DIMENSION_VALUE_DISK).get(expectedKey); - - assertTrue(expectedHeapValue != null || expectedDiskValue != null); - if (expectedHeapValue != null && expectedDiskValue != null) { - assertEquals(combineTierResponses(expectedHeapValue.snapshot(), expectedDiskValue.snapshot()), entry.getValue()); - } else if (expectedHeapValue != null) { - assertEquals(expectedHeapValue.snapshot(), entry.getValue()); - } else { - // disk value not null - assertEquals(expectedDiskValue.snapshot(), entry.getValue()); - } - } - - } - public void testAggregationSomeLevelsWithoutTier() throws Exception { StatsHolder heapStats = new StatsHolder(dimensionNames); StatsHolder diskStats = new StatsHolder(dimensionNames); @@ -189,34 +156,6 @@ public void testAggregationSomeLevelsWithoutTier() throws Exception { } } - public void testAggregationAllLevelsWithTier() throws Exception { - StatsHolder heapStats = new StatsHolder(dimensionNames); - StatsHolder diskStats = new StatsHolder(dimensionNames); - Map> usedDimensionValues = getUsedDimensionValues(heapStats, 10); - Map, CacheStatsResponse>> expected = populateStats(heapStats, diskStats, usedDimensionValues, 100, 2); - TieredSpilloverCacheStats stats = new TieredSpilloverCacheStats(heapStats.createSnapshot(), diskStats.createSnapshot(), dimensionNames); - - List levels = new ArrayList<>(dimensionNames); - levels.add(TIER_DIMENSION_NAME); - TreeMap agg = stats.aggregateByLevels(levels); - - for (Map.Entry entry : agg.entrySet()) { - StatsHolder.Key key = entry.getKey(); - Set expectedKey = new HashSet<>(); - for (int i = 0; i < dimensionNames.size(); i++) { - expectedKey.add(new CacheStatsDimension(dimensionNames.get(i), key.getDimensionValues().get(i))); - } - String aggTierValue = key.getDimensionValues().get(key.getDimensionValues().size()-1); - assertTrue(aggTierValue.equals(TIER_DIMENSION_VALUE_ON_HEAP) || aggTierValue.equals(TIER_DIMENSION_VALUE_DISK)); - - if (aggTierValue.equals(TIER_DIMENSION_VALUE_ON_HEAP)) { - assertEquals(expected.get(TIER_DIMENSION_VALUE_ON_HEAP).get(expectedKey).snapshot(), entry.getValue()); - } else if (aggTierValue.equals(TIER_DIMENSION_VALUE_DISK)) { - assertEquals(expected.get(TIER_DIMENSION_VALUE_DISK).get(expectedKey).snapshot(), entry.getValue()); - } - } - } - public void testAggregationSomeLevelsWithTier() throws Exception { StatsHolder heapStats = new StatsHolder(dimensionNames); StatsHolder diskStats = new StatsHolder(dimensionNames); From 3605730afdeb5093e9a58c109f189f48cb58f966 Mon Sep 17 00:00:00 2001 From: Peter Alfonsi Date: Fri, 15 Mar 2024 11:49:54 -0700 Subject: [PATCH 03/17] Hooked up API to CacheStats output Signed-off-by: Peter Alfonsi --- .../admin/cluster/node/stats/NodeStats.java | 24 +++++- .../cluster/node/stats/NodesStatsRequest.java | 3 +- .../node/stats/TransportNodesStatsAction.java | 3 +- .../stats/TransportClusterStatsAction.java | 1 + .../admin/indices/stats/CommonStatsFlags.java | 30 +++++++ .../opensearch/common/cache/CacheType.java | 29 ++++++- .../common/cache/service/CacheService.java | 11 +++ .../common/cache/service/NodeCacheStats.java | 82 +++++++++++++++++++ .../common/cache/stats/CacheStats.java | 24 +++++- .../cache/stats/MultiDimensionCacheStats.java | 41 +++++++++- .../common/cache/stats/StatsHolder.java | 2 +- .../main/java/org/opensearch/node/Node.java | 3 +- .../java/org/opensearch/node/NodeService.java | 12 ++- .../admin/cluster/RestNodesStatsAction.java | 18 ++++ .../cluster/node/stats/NodeStatsTests.java | 51 +++++++++++- .../opensearch/cluster/DiskUsageTests.java | 6 ++ .../stats/MultiDimensionCacheStatsTests.java | 6 ++ .../MockInternalClusterInfoService.java | 3 +- .../opensearch/test/InternalTestCluster.java | 1 + 19 files changed, 336 insertions(+), 14 deletions(-) create mode 100644 server/src/main/java/org/opensearch/common/cache/service/NodeCacheStats.java diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java index 8562a7eb37709..ac2daf57f248b 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java @@ -39,6 +39,7 @@ import org.opensearch.cluster.routing.WeightedRoutingStats; import org.opensearch.cluster.service.ClusterManagerThrottlingStats; import org.opensearch.common.Nullable; +import org.opensearch.common.cache.service.NodeCacheStats; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.indices.breaker.AllCircuitBreakerStats; @@ -158,6 +159,9 @@ public class NodeStats extends BaseNodeResponse implements ToXContentFragment { @Nullable private AdmissionControlStats admissionControlStats; + @Nullable + private NodeCacheStats nodeCacheStats; + public NodeStats(StreamInput in) throws IOException { super(in); timestamp = in.readVLong(); @@ -234,6 +238,11 @@ public NodeStats(StreamInput in) throws IOException { } else { admissionControlStats = null; } + if (in.getVersion().onOrAfter(Version.V_3_0_0)) { + nodeCacheStats = in.readOptionalWriteable(NodeCacheStats::new); + } else { + nodeCacheStats = null; + } } public NodeStats( @@ -264,7 +273,8 @@ public NodeStats( @Nullable SearchPipelineStats searchPipelineStats, @Nullable SegmentReplicationRejectionStats segmentReplicationRejectionStats, @Nullable RepositoriesStats repositoriesStats, - @Nullable AdmissionControlStats admissionControlStats + @Nullable AdmissionControlStats admissionControlStats, + @Nullable NodeCacheStats nodeCacheStats ) { super(node); this.timestamp = timestamp; @@ -294,6 +304,7 @@ public NodeStats( this.segmentReplicationRejectionStats = segmentReplicationRejectionStats; this.repositoriesStats = repositoriesStats; this.admissionControlStats = admissionControlStats; + this.nodeCacheStats = nodeCacheStats; } public long getTimestamp() { @@ -451,6 +462,11 @@ public AdmissionControlStats getAdmissionControlStats() { return admissionControlStats; } + @Nullable + public NodeCacheStats getNodeCacheStats() { + return nodeCacheStats; + } + @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); @@ -506,6 +522,9 @@ public void writeTo(StreamOutput out) throws IOException { if (out.getVersion().onOrAfter(Version.V_2_12_0)) { out.writeOptionalWriteable(admissionControlStats); } + if (out.getVersion().onOrAfter(Version.V_3_0_0)) { + out.writeOptionalWriteable(nodeCacheStats); + } } @Override @@ -609,6 +628,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (getAdmissionControlStats() != null) { getAdmissionControlStats().toXContent(builder, params); } + if (getNodeCacheStats() != null) { + getNodeCacheStats().toXContent(builder, params); + } return builder; } } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsRequest.java index 1af56f10b95ee..379836cf442e3 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsRequest.java @@ -219,7 +219,8 @@ public enum Metric { RESOURCE_USAGE_STATS("resource_usage_stats"), SEGMENT_REPLICATION_BACKPRESSURE("segment_replication_backpressure"), REPOSITORIES("repositories"), - ADMISSION_CONTROL("admission_control"); + ADMISSION_CONTROL("admission_control"), + CACHE_STATS("caches"); private String metricName; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java index 1df73d3b4394d..2e93e5e7841cb 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java @@ -128,7 +128,8 @@ protected NodeStats nodeOperation(NodeStatsRequest nodeStatsRequest) { NodesStatsRequest.Metric.RESOURCE_USAGE_STATS.containedIn(metrics), NodesStatsRequest.Metric.SEGMENT_REPLICATION_BACKPRESSURE.containedIn(metrics), NodesStatsRequest.Metric.REPOSITORIES.containedIn(metrics), - NodesStatsRequest.Metric.ADMISSION_CONTROL.containedIn(metrics) + NodesStatsRequest.Metric.ADMISSION_CONTROL.containedIn(metrics), + NodesStatsRequest.Metric.CACHE_STATS.containedIn(metrics) ); } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java index 9c5dcc9e9de3f..e4f483f796f44 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java @@ -172,6 +172,7 @@ protected ClusterStatsNodeResponse nodeOperation(ClusterStatsNodeRequest nodeReq false, false, false, + false, false ); List shardsStats = new ArrayList<>(); diff --git a/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java b/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java index a7d9f95b80f7b..41dd39b9be201 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java @@ -34,6 +34,7 @@ import org.opensearch.Version; import org.opensearch.common.annotation.PublicApi; +import org.opensearch.common.cache.CacheType; import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -63,6 +64,9 @@ public class CommonStatsFlags implements Writeable, Cloneable { private boolean includeAllShardIndexingPressureTrackers = false; private boolean includeOnlyTopIndexingPressureMetrics = false; + // Used for metric CACHE_STATS, to determine which caches to report stats for + private EnumSet includeCaches = null; + /** * @param flags flags to set. If no flags are supplied, default flags will be set. */ @@ -91,6 +95,9 @@ public CommonStatsFlags(StreamInput in) throws IOException { includeUnloadedSegments = in.readBoolean(); includeAllShardIndexingPressureTrackers = in.readBoolean(); includeOnlyTopIndexingPressureMetrics = in.readBoolean(); + if (in.getVersion().onOrAfter(Version.V_3_0_0)) { + includeCaches = in.readEnumSet(CacheType.class); + } } @Override @@ -111,6 +118,9 @@ public void writeTo(StreamOutput out) throws IOException { out.writeBoolean(includeUnloadedSegments); out.writeBoolean(includeAllShardIndexingPressureTrackers); out.writeBoolean(includeOnlyTopIndexingPressureMetrics); + if (out.getVersion().onOrAfter(Version.V_3_0_0)) { + out.writeEnumSet(includeCaches); + } } /** @@ -125,6 +135,7 @@ public CommonStatsFlags all() { includeUnloadedSegments = false; includeAllShardIndexingPressureTrackers = false; includeOnlyTopIndexingPressureMetrics = false; + includeCaches = null; return this; } @@ -140,6 +151,7 @@ public CommonStatsFlags clear() { includeUnloadedSegments = false; includeAllShardIndexingPressureTrackers = false; includeOnlyTopIndexingPressureMetrics = false; + includeCaches = null; return this; } @@ -151,6 +163,10 @@ public Flag[] getFlags() { return flags.toArray(new Flag[0]); } + public EnumSet getIncludeCaches() { + return includeCaches; + } + /** * Sets specific search group stats to retrieve the stats for. Mainly affects search * when enabled. @@ -206,6 +222,20 @@ public CommonStatsFlags includeOnlyTopIndexingPressureMetrics(boolean includeOnl return this; } + public CommonStatsFlags includeCacheType(CacheType cacheType) { + if (includeCaches == null) { + includeCaches = EnumSet.noneOf(CacheType.class); + } + includeCaches.add(cacheType); + return this; + } + + public CommonStatsFlags includeAllCacheTypes() { + includeCaches = EnumSet.noneOf(CacheType.class); + Collections.addAll(includeCaches, CacheType.values()); + return this; + } + public boolean includeUnloadedSegments() { return this.includeUnloadedSegments; } diff --git a/server/src/main/java/org/opensearch/common/cache/CacheType.java b/server/src/main/java/org/opensearch/common/cache/CacheType.java index c5aeb7cd1fa40..46af3d2643e2c 100644 --- a/server/src/main/java/org/opensearch/common/cache/CacheType.java +++ b/server/src/main/java/org/opensearch/common/cache/CacheType.java @@ -10,20 +10,45 @@ import org.opensearch.common.annotation.ExperimentalApi; +import java.util.HashSet; +import java.util.Set; + /** * Cache types available within OpenSearch. */ @ExperimentalApi public enum CacheType { - INDICES_REQUEST_CACHE("indices.requests.cache"); + INDICES_REQUEST_CACHE("indices.requests.cache", "request_cache"); private final String settingPrefix; + private final String apiRepresentation; - CacheType(String settingPrefix) { + CacheType(String settingPrefix, String representation) { this.settingPrefix = settingPrefix; + this.apiRepresentation = representation; } public String getSettingPrefix() { return settingPrefix; } + public String getApiRepresentation() { + return apiRepresentation; + } + + public static CacheType getByRepresentation(String representation) { + for (CacheType cacheType : values()) { + if (cacheType.apiRepresentation.equals(representation)) { + return cacheType; + } + } + throw new IllegalArgumentException("No CacheType with representation = " + representation); + } + + public static Set allRepresentations() { + Set reprs = new HashSet<>(); + for (CacheType cacheType : values()) { + reprs.add(cacheType.apiRepresentation); + } + return reprs; + } } diff --git a/server/src/main/java/org/opensearch/common/cache/service/CacheService.java b/server/src/main/java/org/opensearch/common/cache/service/CacheService.java index b6710e5e4b424..e450fc65fe351 100644 --- a/server/src/main/java/org/opensearch/common/cache/service/CacheService.java +++ b/server/src/main/java/org/opensearch/common/cache/service/CacheService.java @@ -8,10 +8,12 @@ package org.opensearch.common.cache.service; +import org.opensearch.action.admin.indices.stats.CommonStatsFlags; import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.cache.CacheType; import org.opensearch.common.cache.ICache; import org.opensearch.common.cache.settings.CacheSettings; +import org.opensearch.common.cache.stats.CacheStats; import org.opensearch.common.cache.store.OpenSearchOnHeapCache; import org.opensearch.common.cache.store.config.CacheConfig; import org.opensearch.common.settings.Setting; @@ -19,6 +21,7 @@ import org.opensearch.common.util.FeatureFlags; import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.Map; /** @@ -62,4 +65,12 @@ public ICache createCache(CacheConfig config, CacheType cache cacheTypeMap.put(cacheType, iCache); return iCache; } + + public NodeCacheStats stats(CommonStatsFlags flags) { + LinkedHashMap statsMap = new LinkedHashMap<>(); + for (CacheType type : cacheTypeMap.keySet()) { + statsMap.put(type, cacheTypeMap.get(type).stats()); // TODO: We need to force some ordering on cacheTypeMap + } + return new NodeCacheStats(statsMap, flags); + } } diff --git a/server/src/main/java/org/opensearch/common/cache/service/NodeCacheStats.java b/server/src/main/java/org/opensearch/common/cache/service/NodeCacheStats.java new file mode 100644 index 0000000000000..41a0205d987a5 --- /dev/null +++ b/server/src/main/java/org/opensearch/common/cache/service/NodeCacheStats.java @@ -0,0 +1,82 @@ +/* + * 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.service; + +import org.opensearch.action.admin.indices.stats.CommonStatsFlags; +import org.opensearch.common.cache.CacheType; +import org.opensearch.common.cache.stats.CacheStats; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.common.io.stream.Writeable; +import org.opensearch.core.xcontent.ToXContentFragment; +import org.opensearch.core.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Objects; + +public class NodeCacheStats implements ToXContentFragment, Writeable { + private final LinkedHashMap statsByCache; + private final CommonStatsFlags flags; + + public NodeCacheStats(LinkedHashMap statsByCache, CommonStatsFlags flags) { + this.statsByCache = statsByCache; + this.flags = flags; + } + + public NodeCacheStats(StreamInput in) throws IOException { + this.flags = new CommonStatsFlags(in); + Map readMap = in.readMap( + i -> i.readEnum(CacheType.class), + CacheStats::readFromStreamWithClassName + ); + this.statsByCache = new LinkedHashMap<>(readMap); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + flags.writeTo(out); + out.writeMap( + statsByCache, + StreamOutput::writeEnum, + (o, cacheStats) -> cacheStats.writeToWithClassName(o) + ); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + for (CacheType type : statsByCache.keySet()) { + if (flags.getIncludeCaches().contains(type)) { + builder.startObject(type.getApiRepresentation()); + statsByCache.get(type).toXContent(builder, params); + builder.endObject(); + } + } + return builder; + } + + @Override + public boolean equals(Object o) { + if (o == null) { + return false; + } + if (o.getClass() != NodeCacheStats.class) { + return false; + } + NodeCacheStats other = (NodeCacheStats) o; + return statsByCache.equals(other.statsByCache) + && flags.getIncludeCaches().equals(other.flags.getIncludeCaches()); + } + + @Override + public int hashCode() { + return Objects.hash(statsByCache, flags); + } +} 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 4a97ff15ecf2c..4906a41aeea04 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,13 +8,20 @@ package org.opensearch.common.cache.stats; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; +import org.opensearch.core.xcontent.ToXContentFragment; + +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.util.Map; /** * 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. */ -public interface CacheStats extends Writeable {// TODO: also extends ToXContentFragment (in API PR) +public interface CacheStats extends Writeable, ToXContentFragment { // Method to get all 5 values at once CacheStatsResponse.Snapshot getTotalStats(); @@ -29,4 +36,19 @@ public interface CacheStats extends Writeable {// TODO: also extends ToXContentF long getTotalSizeInBytes(); long getTotalEntries(); + + // Used for the readFromStream method to allow deserialization of generic CacheStats objects. + String getClassName(); + + void writeToWithClassName(StreamOutput out) throws IOException; + + static CacheStats readFromStreamWithClassName(StreamInput in) throws IOException { + String className = in.readString(); + //in.reset(); + + if (className.equals(MultiDimensionCacheStats.CLASS_NAME)) { + return new MultiDimensionCacheStats(in); + } + return null; + } } diff --git a/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java b/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java index 24dfcb475103e..584b34d8701b2 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java @@ -10,12 +10,14 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.xcontent.XContentBuilder; import java.io.IOException; import java.util.ArrayList; import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.TreeMap; import java.util.concurrent.ConcurrentHashMap; @@ -29,11 +31,16 @@ public class MultiDimensionCacheStats implements CacheStats { final Map snapshot; final List dimensionNames; + public static String CLASS_NAME = "multidimension"; + public MultiDimensionCacheStats(Map snapshot, List dimensionNames) { this.snapshot = snapshot; this.dimensionNames = dimensionNames; } + /** + * Should not be used with StreamOutputs produced using writeToWithClassName. + */ public MultiDimensionCacheStats(StreamInput in) throws IOException { this.dimensionNames = List.of(in.readStringArray()); Map readMap = in.readMap( @@ -89,6 +96,17 @@ public long getTotalEntries() { return getTotalStats().getEntries(); } + @Override + public String getClassName() { + return CLASS_NAME; + } + + @Override + public void writeToWithClassName(StreamOutput out) throws IOException { + out.writeString(getClassName()); + writeTo(out); + } + /** * Return a TreeMap containing stats values aggregated by the levels passed in. Results are ordered so that * values are grouped by their dimension values. @@ -119,6 +137,14 @@ public TreeMap aggregateByLevels(L return result; } + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + // TODO + builder.startObject("test_output_from_multidim"); + builder.endObject(); + return null; + } + // First compare outermost dimension, then second outermost, etc. // Pkg-private for testing static class KeyComparator implements Comparator { @@ -156,5 +182,18 @@ private int[] getLevelIndices(List levels) { return result; } - // TODO (in API PR): Produce XContent based on aggregateByLevels() + @Override + public boolean equals(Object o) { + if (o == null || o.getClass() != MultiDimensionCacheStats.class) { + return false; + } + MultiDimensionCacheStats other = (MultiDimensionCacheStats) o; + return this.snapshot.equals(other.snapshot) && this.dimensionNames.equals(other.dimensionNames); + } + + @Override + public int hashCode() { + return Objects.hash(this.snapshot, this.dimensionNames); + } + } diff --git a/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java b/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java index 3b68487e0761a..ebb9326deceba 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java @@ -134,7 +134,7 @@ public Map createSnapshot() { public static class Key { final List dimensionValues; // The dimensions must be ordered - Key(List dimensionValues) { + public Key(List dimensionValues) { this.dimensionValues = Collections.unmodifiableList(dimensionValues); } diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 3ef3ae4f6230e..cba62493a2ce5 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -1167,7 +1167,8 @@ protected Node( resourceUsageCollectorService, segmentReplicationStatsTracker, repositoryService, - admissionControlService + admissionControlService, + cacheService ); final SearchService searchService = newSearchService( diff --git a/server/src/main/java/org/opensearch/node/NodeService.java b/server/src/main/java/org/opensearch/node/NodeService.java index 15cc8f3d20bb3..1eb38ea63ad5a 100644 --- a/server/src/main/java/org/opensearch/node/NodeService.java +++ b/server/src/main/java/org/opensearch/node/NodeService.java @@ -41,6 +41,7 @@ import org.opensearch.cluster.routing.WeightedRoutingStats; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Nullable; +import org.opensearch.common.cache.service.CacheService; import org.opensearch.common.settings.Settings; import org.opensearch.common.settings.SettingsFilter; import org.opensearch.common.util.io.IOUtils; @@ -99,6 +100,7 @@ public class NodeService implements Closeable { private final RepositoriesService repositoriesService; private final AdmissionControlService admissionControlService; private final SegmentReplicationStatsTracker segmentReplicationStatsTracker; + private final CacheService cacheService; NodeService( Settings settings, @@ -125,7 +127,8 @@ public class NodeService implements Closeable { ResourceUsageCollectorService resourceUsageCollectorService, SegmentReplicationStatsTracker segmentReplicationStatsTracker, RepositoriesService repositoriesService, - AdmissionControlService admissionControlService + AdmissionControlService admissionControlService, + CacheService cacheService ) { this.settings = settings; this.threadPool = threadPool; @@ -154,6 +157,7 @@ public class NodeService implements Closeable { clusterService.addStateApplier(ingestService); clusterService.addStateApplier(searchPipelineService); this.segmentReplicationStatsTracker = segmentReplicationStatsTracker; + this.cacheService = cacheService; } public NodeInfo info( @@ -236,7 +240,8 @@ public NodeStats stats( boolean resourceUsageStats, boolean segmentReplicationTrackerStats, boolean repositoriesStats, - boolean admissionControl + boolean admissionControl, + boolean cacheService ) { // for indices stats we want to include previous allocated shards stats as well (it will // only be applied to the sensible ones to use, like refresh/merge/flush/indexing stats) @@ -268,7 +273,8 @@ public NodeStats stats( searchPipelineStats ? this.searchPipelineService.stats() : null, segmentReplicationTrackerStats ? this.segmentReplicationStatsTracker.getTotalRejectionStats() : null, repositoriesStats ? this.repositoriesService.getRepositoriesStats() : null, - admissionControl ? this.admissionControlService.stats() : null + admissionControl ? this.admissionControlService.stats() : null, + cacheService ? this.cacheService.stats(indices) : null ); } diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesStatsAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesStatsAction.java index 66b9afda06eb6..9c8c9ee3dd294 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesStatsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesStatsAction.java @@ -36,6 +36,7 @@ import org.opensearch.action.admin.indices.stats.CommonStatsFlags; import org.opensearch.action.admin.indices.stats.CommonStatsFlags.Flag; import org.opensearch.client.node.NodeClient; +import org.opensearch.common.cache.CacheType; import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; @@ -175,6 +176,23 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC nodesStatsRequest.indices(flags); } + } else if (metrics.contains("caches")) { + // Extract the list of caches we want to get stats for from the submetrics (which we get from index_metric) + Set cacheMetrics = Strings.tokenizeByCommaToSet(request.param("index_metric", "_all")); + CommonStatsFlags cacheFlags = new CommonStatsFlags(); + cacheFlags.clear(); + if (cacheMetrics.size() == 1 && cacheMetrics.contains("_all")) { + cacheFlags.includeAllCacheTypes(); + } else { + for (String cacheName : cacheMetrics) { + try { + cacheFlags.includeCacheType(CacheType.getByRepresentation(cacheName)); + } catch (IllegalArgumentException e) { + throw new IllegalArgumentException(unrecognized(request, Set.of(cacheName), CacheType.allRepresentations(), "cache type")); + } + } + } + nodesStatsRequest.indices(cacheFlags); } else if (request.hasParam("index_metric")) { throw new IllegalArgumentException( String.format( diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java index 1b8b6243aa805..337059c5380e3 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java @@ -42,6 +42,12 @@ import org.opensearch.cluster.routing.WeightedRoutingStats; import org.opensearch.cluster.service.ClusterManagerThrottlingStats; import org.opensearch.cluster.service.ClusterStateStats; +import org.opensearch.common.cache.CacheType; +import org.opensearch.common.cache.service.NodeCacheStats; +import org.opensearch.common.cache.stats.CacheStats; +import org.opensearch.common.cache.stats.CacheStatsResponse; +import org.opensearch.common.cache.stats.MultiDimensionCacheStats; +import org.opensearch.common.cache.stats.StatsHolder; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.metrics.OperationStats; import org.opensearch.common.settings.ClusterSettings; @@ -87,6 +93,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.function.Function; @@ -577,6 +584,13 @@ public void testSerialization() throws IOException { deserializedAdmissionControllerStats.getRejectionCount().get(AdmissionControlActionType.INDEXING.getType()) ); } + NodeCacheStats nodeCacheStats = nodeStats.getNodeCacheStats(); + NodeCacheStats deserializedNodeCacheStats = deserializedNodeStats.getNodeCacheStats(); + if (nodeCacheStats == null) { + assertNull(deserializedNodeCacheStats); + } else { + assertEquals(nodeCacheStats, deserializedNodeCacheStats); + } } } } @@ -928,6 +942,40 @@ public void apply(String action, AdmissionControlActionType admissionControlActi NodeIndicesStats indicesStats = getNodeIndicesStats(remoteStoreStats); + NodeCacheStats nodeCacheStats = null; + if (frequently()) { + int numIndices = randomIntBetween(1, 10); + int numShardsPerIndex = randomIntBetween(1, 50); + Map snapshotMap = new HashMap<>(); + List dimensionNames = List.of("index", "shard", "tier"); + for (int indexNum = 0; indexNum < numIndices; indexNum++) { + String indexName = "index" + indexNum; + for (int shardNum = 0; shardNum < numShardsPerIndex; shardNum++) { + String shardName = "[" + indexName + "][" + shardNum + "]"; + for (String tierName : new String[]{"dummy_tier_1", "dummy_tier_2"}) { + CacheStatsResponse.Snapshot response = new CacheStatsResponse.Snapshot( + randomInt(100), + randomInt(100), + randomInt(100), + randomInt(100), + randomInt(100) + ); + snapshotMap.put(new StatsHolder.Key(List.of(indexName, shardName, tierName)), response); + } + } + } + CommonStatsFlags flags = new CommonStatsFlags(); + for (CacheType cacheType : CacheType.values()) { + if (frequently()) { + flags.includeCacheType(cacheType); + } + } + MultiDimensionCacheStats cacheStats = new MultiDimensionCacheStats(snapshotMap, dimensionNames); + LinkedHashMap cacheStatsMap = new LinkedHashMap<>(); + cacheStatsMap.put(CacheType.INDICES_REQUEST_CACHE, cacheStats); + nodeCacheStats = new NodeCacheStats(cacheStatsMap, flags); + } + // TODO: Only remote_store based aspects of NodeIndicesStats are being tested here. // It is possible to test other metrics in NodeIndicesStats as well since it extends Writeable now return new NodeStats( @@ -958,7 +1006,8 @@ public void apply(String action, AdmissionControlActionType admissionControlActi null, segmentReplicationRejectionStats, null, - admissionControlStats + admissionControlStats, + nodeCacheStats ); } diff --git a/server/src/test/java/org/opensearch/cluster/DiskUsageTests.java b/server/src/test/java/org/opensearch/cluster/DiskUsageTests.java index ff47ec3015697..5539dd26dd52d 100644 --- a/server/src/test/java/org/opensearch/cluster/DiskUsageTests.java +++ b/server/src/test/java/org/opensearch/cluster/DiskUsageTests.java @@ -194,6 +194,7 @@ public void testFillDiskUsage() { null, null, null, + null, null ), new NodeStats( @@ -224,6 +225,7 @@ public void testFillDiskUsage() { null, null, null, + null, null ), new NodeStats( @@ -254,6 +256,7 @@ public void testFillDiskUsage() { null, null, null, + null, null ) ); @@ -315,6 +318,7 @@ public void testFillDiskUsageSomeInvalidValues() { null, null, null, + null, null ), new NodeStats( @@ -345,6 +349,7 @@ public void testFillDiskUsageSomeInvalidValues() { null, null, null, + null, null ), new NodeStats( @@ -375,6 +380,7 @@ public void testFillDiskUsageSomeInvalidValues() { null, null, null, + null, null ) ); diff --git a/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java b/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java index c80d10e02f79a..eb0a3f2ffdeb4 100644 --- a/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java +++ b/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java @@ -39,6 +39,12 @@ public void testSerialization() throws Exception { assertEquals(stats.snapshot, deserialized.snapshot); assertEquals(stats.dimensionNames, deserialized.dimensionNames); + + os = new BytesStreamOutput(); + stats.writeToWithClassName(os); + is = new BytesStreamInput(BytesReference.toBytes(os.bytes())); + CacheStats deserializedViaCacheStats = CacheStats.readFromStreamWithClassName(is); + assertEquals(MultiDimensionCacheStats.class, deserializedViaCacheStats.getClass()); } public void testAddAndGet() throws Exception { diff --git a/test/framework/src/main/java/org/opensearch/cluster/MockInternalClusterInfoService.java b/test/framework/src/main/java/org/opensearch/cluster/MockInternalClusterInfoService.java index 1ad6083074025..35ca5d80aeb4e 100644 --- a/test/framework/src/main/java/org/opensearch/cluster/MockInternalClusterInfoService.java +++ b/test/framework/src/main/java/org/opensearch/cluster/MockInternalClusterInfoService.java @@ -124,7 +124,8 @@ List adjustNodesStats(List nodesStats) { nodeStats.getSearchPipelineStats(), nodeStats.getSegmentReplicationRejectionStats(), nodeStats.getRepositoriesStats(), - nodeStats.getAdmissionControlStats() + nodeStats.getAdmissionControlStats(), + nodeStats.getNodeCacheStats() ); }).collect(Collectors.toList()); } diff --git a/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java index c2b964aa96212..ca80c65e58522 100644 --- a/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java @@ -2736,6 +2736,7 @@ public void ensureEstimatedStats() { false, false, false, + false, false ); assertThat( From 7bf4d2e0be434f9af71e75a49fbcf360f03628d8 Mon Sep 17 00:00:00 2001 From: Peter Alfonsi Date: Fri, 15 Mar 2024 16:19:00 -0700 Subject: [PATCH 04/17] Added logic in CacheStats toXContent() Signed-off-by: Peter Alfonsi --- .../cache/stats/CacheStatsResponse.java | 26 ++++- .../cache/stats/MultiDimensionCacheStats.java | 98 ++++++++++++++++++- .../stats/MultiDimensionCacheStatsTests.java | 23 +++++ 3 files changed, 142 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/opensearch/common/cache/stats/CacheStatsResponse.java b/server/src/main/java/org/opensearch/common/cache/stats/CacheStatsResponse.java index 723f64ded6a72..438178291acab 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/CacheStatsResponse.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/CacheStatsResponse.java @@ -12,6 +12,9 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; +import org.opensearch.core.common.unit.ByteSizeValue; +import org.opensearch.core.xcontent.ToXContentFragment; +import org.opensearch.core.xcontent.XContentBuilder; import java.io.IOException; import java.util.Objects; @@ -113,7 +116,7 @@ public Snapshot snapshot() { /** * An immutable snapshot of CacheStatsResponse. */ - public static class Snapshot implements Writeable { // TODO: Make this extend ToXContent (in API PR) + public static class Snapshot implements Writeable, ToXContentFragment { private final long hits; private final long misses; private final long evictions; @@ -191,5 +194,26 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(hits, misses, evictions, sizeInBytes, entries); } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + // We don't write the header in CacheStatsResponse's toXContent, because it doesn't know the name of aggregation it's part of + builder.humanReadableField(CacheStatsResponse.Fields.MEMORY_SIZE_IN_BYTES, CacheStatsResponse.Fields.MEMORY_SIZE, new ByteSizeValue(sizeInBytes)); + builder.field(CacheStatsResponse.Fields.EVICTIONS, evictions); + builder.field(CacheStatsResponse.Fields.HIT_COUNT, hits); + builder.field(CacheStatsResponse.Fields.MISS_COUNT, misses); + builder.field(Fields.ENTRIES, entries); + return builder; + } + } + + static final class Fields { + static final String MEMORY_SIZE = "size"; // TODO: Bad name - think of something better + static final String MEMORY_SIZE_IN_BYTES = "size_in_bytes"; + // TODO: This might not be memory as it could be partially on disk, so I've changed it, but should it be consistent with the earlier field? + static final String EVICTIONS = "evictions"; + static final String HIT_COUNT = "hit_count"; + static final String MISS_COUNT = "miss_count"; + static final String ENTRIES = "entries"; } } diff --git a/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java b/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java index 584b34d8701b2..62abe73641d64 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java @@ -139,12 +139,102 @@ public TreeMap aggregateByLevels(L @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - // TODO - builder.startObject("test_output_from_multidim"); - builder.endObject(); - return null; + // Always show total stats, regardless of levels + getTotalStats().toXContent(builder, params); + + List levels = getLevels(params); + if (levels == null) { + // display total stats only + return builder; + } + + int[] indices = getLevelIndices(levels); // Checks whether levels are valid; throws IllegalArgumentException if not + toXContentForLevels(builder, params, levels); + return builder; + } + + XContentBuilder toXContentForLevels(XContentBuilder builder, Params params, List levels) throws IOException { + TreeMap agg = aggregateByLevels(levels); + StatsHolder.Key lastKey = null; + + int depth = 0; + + for (Map.Entry entry : agg.entrySet()) { + StatsHolder.Key key = entry.getKey(); + if (lastKey == null) { + // Open an object for all relevant dimension values + for (int i = 0; i < key.dimensionValues.size(); i++) { + builder.startObject(levels.get(i)); + System.out.println("New object = " + levels.get(i)); + builder.startObject(key.dimensionValues.get(i)); + System.out.println("New object = " + key.dimensionValues.get(i)); + depth += 2; + } + } + + else { + int innermostCommonDimIndex = getInnermostCommonDimensionIndex(key, lastKey); + // End objects for all dimension values this key doesn't have in common with the last key + int numObjectsToClose = (key.dimensionValues.size() - innermostCommonDimIndex - 2) * 2; + for (int i = 0; i < numObjectsToClose; i++) { + builder.endObject(); + System.out.println("Ended object"); + depth--; + } + // Start new nested objects for all dimension values that this key doesn't have in common with the last key + for (int i = innermostCommonDimIndex + 1; i < key.dimensionValues.size(); i++) { + if (i >= innermostCommonDimIndex + 2) { + // If we're opening an object for a dimension value that's different in the new key, + // also open a new object with the name of that dimension + builder.startObject(levels.get(i)); + System.out.println("New object = " + levels.get(i)); + depth++; + } + builder.startObject(key.dimensionValues.get(i)); + System.out.println("New object = " + key.dimensionValues.get(i)); + depth++; + } + } + + // Finally, write the value + entry.getValue().toXContent(builder, params); + builder.endObject(); // End the object that contains only the snapshot for this key + System.out.println("Ended object"); + depth--; + System.out.println("depth = " + depth); + lastKey = key; + } + // There will always be 2 * levels.size() - 1 objects to close at the end + for (int i = 0; i < levels.size() * 2 - 1; i++) { + builder.endObject(); + System.out.println("Ended object"); + depth--; + } + System.out.println("Final depth = " + depth); + + return builder; + } + private int getInnermostCommonDimensionIndex(StatsHolder.Key key, StatsHolder.Key lastKey) { + assert key.dimensionValues.size() == lastKey.dimensionValues.size(); + for (int i = key.dimensionValues.size() - 1; i >= 0; i--){ + if (key.dimensionValues.get(i).equals(lastKey.dimensionValues.get(i))) { + return i; + } + } + return -1; // No common dimension values + } + + private List getLevels(Params params) { + String levels = params.param("level"); + if (levels == null) { + return null; + } + return List.of(levels.split(",")); + } + + // First compare outermost dimension, then second outermost, etc. // Pkg-private for testing static class KeyComparator implements Comparator { diff --git a/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java b/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java index eb0a3f2ffdeb4..dd9e6ed6579d1 100644 --- a/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java +++ b/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java @@ -11,8 +11,13 @@ import org.opensearch.common.Randomness; import org.opensearch.common.cache.ICacheKey; import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.common.xcontent.XContentType; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.io.stream.BytesStreamInput; +import org.opensearch.core.xcontent.MediaTypeRegistry; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.test.OpenSearchTestCase; import java.util.ArrayList; @@ -164,6 +169,24 @@ public void testAggregateBySomeDimensions() throws Exception { } } + public void testXContentForLevels() throws Exception { + List dimensionNames = List.of("A", "B", "C"); + Map snapshot = Map.of( + new StatsHolder.Key(List.of("A1", "B1", "C1")), new CacheStatsResponse.Snapshot(1,1,1,1,1), + new StatsHolder.Key(List.of("A1", "B1", "C2")), new CacheStatsResponse.Snapshot(2,2,2,2,2), + new StatsHolder.Key(List.of("A1", "B2", "C1")), new CacheStatsResponse.Snapshot(3,3,3,3,3), + new StatsHolder.Key(List.of("A2", "B1", "C3")), new CacheStatsResponse.Snapshot(4,4,4,4,4) + ); + MultiDimensionCacheStats stats = new MultiDimensionCacheStats(snapshot, dimensionNames); + + XContentBuilder builder = XContentFactory.jsonBuilder(); + ToXContent.Params params = ToXContent.EMPTY_PARAMS; + + builder.startObject(); + stats.toXContentForLevels(builder, params, List.of("A", "B", "C")); + builder.endObject(); + } + static Map> getUsedDimensionValues(StatsHolder statsHolder, int numValuesPerDim) { Map> usedDimensionValues = new HashMap<>(); for (int i = 0; i < statsHolder.getDimensionNames().size(); i++) { From ca31bb7b8a297b987b48a5c6053cce98e811062f Mon Sep 17 00:00:00 2001 From: Peter Alfonsi Date: Wed, 27 Mar 2024 16:31:17 -0700 Subject: [PATCH 05/17] Simplified toXContent Signed-off-by: Peter Alfonsi --- .../common/cache/stats/CounterSnapshot.java | 26 ++++++++++++++++++- .../cache/stats/MultiDimensionCacheStats.java | 24 +++++++++++++++-- .../cluster/node/stats/NodeStatsTests.java | 6 ++--- 3 files changed, 50 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/opensearch/common/cache/stats/CounterSnapshot.java b/server/src/main/java/org/opensearch/common/cache/stats/CounterSnapshot.java index df9ecb34e19ee..2ae1b407eeb2b 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/CounterSnapshot.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/CounterSnapshot.java @@ -11,6 +11,9 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; +import org.opensearch.core.common.unit.ByteSizeValue; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; import java.io.IOException; import java.util.Objects; @@ -18,7 +21,7 @@ /** * An immutable snapshot of CacheStatsCounter. */ -public class CounterSnapshot implements Writeable { // TODO: Make this extend ToXContent (in API PR) +public class CounterSnapshot implements Writeable, ToXContent { private final long hits; private final long misses; private final long evictions; @@ -96,4 +99,25 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(hits, misses, evictions, sizeInBytes, entries); } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + // We don't write the header in CacheStatsResponse's toXContent, because it doesn't know the name of aggregation it's part of + builder.humanReadableField(Fields.MEMORY_SIZE_IN_BYTES, Fields.MEMORY_SIZE, new ByteSizeValue(sizeInBytes)); + builder.field(Fields.EVICTIONS, evictions); + builder.field(Fields.HIT_COUNT, hits); + builder.field(Fields.MISS_COUNT, misses); + builder.field(Fields.ENTRIES, entries); + return builder; + } + + static final class Fields { + static final String MEMORY_SIZE = "size"; // TODO: Bad name - think of something better + static final String MEMORY_SIZE_IN_BYTES = "size_in_bytes"; + // TODO: This might not be memory as it could be partially on disk, so I've changed it, but should it be consistent with the earlier field? + static final String EVICTIONS = "evictions"; + static final String HIT_COUNT = "hit_count"; + static final String MISS_COUNT = "miss_count"; + static final String ENTRIES = "entries"; + } } diff --git a/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java b/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java index 8a51d80e310ce..9301797cff013 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java @@ -14,7 +14,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.Objects; @@ -188,10 +187,31 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws XContentBuilder toXContentForLevels(XContentBuilder builder, Params params, List levels) throws IOException { DimensionNode aggregated = aggregateByLevels(levels); + // Depth -1 corresponds to the dummy root node, which has no dimension value and only has children + toXContentForLevelsHelper(-1, aggregated, levels, builder, params); + return builder; + } - return builder; + private void toXContentForLevelsHelper(int depth, DimensionNode current, List levels, XContentBuilder builder, Params params) throws IOException { + if (depth >= 0) { + builder.startObject(current.dimensionValue); + } + if (depth == levels.size() - 1) { + // This is a leaf node + current.getSnapshot().toXContent(builder, params); + } else { + builder.startObject(levels.get(depth+1)); + for (DimensionNode nextNode : current.children.values()) { + toXContentForLevelsHelper(depth+1, nextNode, levels, builder, params); + } + builder.endObject(); + } + + if (depth >= 0) { + builder.endObject(); + } } diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java index 337059c5380e3..9abb877dd369c 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java @@ -45,7 +45,7 @@ import org.opensearch.common.cache.CacheType; import org.opensearch.common.cache.service.NodeCacheStats; import org.opensearch.common.cache.stats.CacheStats; -import org.opensearch.common.cache.stats.CacheStatsResponse; +import org.opensearch.common.cache.stats.CounterSnapshot; import org.opensearch.common.cache.stats.MultiDimensionCacheStats; import org.opensearch.common.cache.stats.StatsHolder; import org.opensearch.common.io.stream.BytesStreamOutput; @@ -946,14 +946,14 @@ public void apply(String action, AdmissionControlActionType admissionControlActi if (frequently()) { int numIndices = randomIntBetween(1, 10); int numShardsPerIndex = randomIntBetween(1, 50); - Map snapshotMap = new HashMap<>(); + Map snapshotMap = new HashMap<>(); List dimensionNames = List.of("index", "shard", "tier"); for (int indexNum = 0; indexNum < numIndices; indexNum++) { String indexName = "index" + indexNum; for (int shardNum = 0; shardNum < numShardsPerIndex; shardNum++) { String shardName = "[" + indexName + "][" + shardNum + "]"; for (String tierName : new String[]{"dummy_tier_1", "dummy_tier_2"}) { - CacheStatsResponse.Snapshot response = new CacheStatsResponse.Snapshot( + CounterSnapshot response = new CounterSnapshot( randomInt(100), randomInt(100), randomInt(100), From 0dd1057bb0440fab4e9320a91c490343ddb8c8f4 Mon Sep 17 00:00:00 2001 From: Peter Alfonsi Date: Thu, 28 Mar 2024 12:29:10 -0700 Subject: [PATCH 06/17] Finished xcontent UT Signed-off-by: Peter Alfonsi --- .../stats/MultiDimensionCacheStatsTests.java | 41 +++++++++++++++++++ 1 file changed, 41 insertions(+) diff --git a/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java b/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java index 2bcf064978a55..59948e1aa86dc 100644 --- a/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java +++ b/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java @@ -12,6 +12,7 @@ import org.opensearch.common.cache.ICacheKey; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.common.xcontent.XContentType; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.io.stream.BytesStreamInput; @@ -28,6 +29,8 @@ import java.util.Random; import java.util.Set; import java.util.UUID; +import java.util.function.BiConsumer; +import java.util.function.Consumer; public class MultiDimensionCacheStatsTests extends OpenSearchTestCase { public void testSerialization() throws Exception { @@ -165,6 +168,44 @@ public void testXContentForLevels() throws Exception { builder.startObject(); stats.toXContentForLevels(builder, params, List.of("A", "B", "C")); builder.endObject(); + String resultString = builder.toString(); + Map result = XContentHelper.convertToMap(MediaTypeRegistry.JSON.xContent(), resultString, true); + + Map> fieldNamesMap = Map.of( + CounterSnapshot.Fields.MEMORY_SIZE_IN_BYTES, (counter, value) -> counter.sizeInBytes.inc(value), + CounterSnapshot.Fields.EVICTIONS, (counter, value) -> counter.evictions.inc(value), + CounterSnapshot.Fields.HIT_COUNT, (counter, value) -> counter.hits.inc(value), + CounterSnapshot.Fields.MISS_COUNT, (counter, value) -> counter. misses.inc(value), + CounterSnapshot.Fields.ENTRIES, (counter, value) -> counter.entries.inc(value) + ); + + for (Map.Entry entry : snapshot.entrySet()) { + List xContentKeys = new ArrayList<>(); + for (int i = 0; i < dimensionNames.size(); i++) { + xContentKeys.add(dimensionNames.get(i)); + xContentKeys.add(entry.getKey().dimensionValues.get(i)); + } + CacheStatsCounter counterFromXContent = new CacheStatsCounter(); + + for (Map.Entry> fieldNamesEntry : fieldNamesMap.entrySet()) { + List fullXContentKeys = new ArrayList<>(xContentKeys); + fullXContentKeys.add(fieldNamesEntry.getKey()); + int valueInXContent = getValueFromNestedXContentMap(result, fullXContentKeys); + BiConsumer incrementer = fieldNamesEntry.getValue(); + incrementer.accept(counterFromXContent, valueInXContent); + } + + CounterSnapshot expected = entry.getValue(); + assertEquals(counterFromXContent.snapshot(), expected); + } + } + + private int getValueFromNestedXContentMap(Map xContentMap, List keys) { + Map current = xContentMap; + for (int i = 0; i < keys.size() - 1; i++) { + current = (Map) current.get(keys.get(i)); + } + return (int) current.get(keys.get(keys.size()-1)); } // Get a map from the list of dimension values to the corresponding leaf node. From 3a90973dd18c2b492c53ee4706a75e69259cb1d1 Mon Sep 17 00:00:00 2001 From: Peter Alfonsi Date: Thu, 28 Mar 2024 13:52:53 -0700 Subject: [PATCH 07/17] readded tests depending on xcontent Signed-off-by: Peter Alfonsi --- .../store/disk/EhCacheDiskCacheTests.java | 112 +++++++++++++++ .../opensearch/common/cache/CacheType.java | 1 + .../common/cache/service/NodeCacheStats.java | 14 +- .../common/cache/stats/CacheStats.java | 3 - .../common/cache/stats/CounterSnapshot.java | 3 +- .../cache/stats/MultiDimensionCacheStats.java | 9 +- .../admin/cluster/RestNodesStatsAction.java | 4 +- .../cluster/node/stats/NodeStatsTests.java | 2 +- .../stats/MultiDimensionCacheStatsTests.java | 49 ++++--- .../common/cache/stats/StatsHolderTests.java | 1 - .../store/OpenSearchOnHeapCacheTests.java | 93 +++++++++++++ .../indices/IndicesRequestCacheTests.java | 127 ++++++++++++++++++ 12 files changed, 377 insertions(+), 41 deletions(-) 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 75ce6727bb39d..8c6d9d3ae4c2b 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,14 +20,20 @@ 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.CacheStats; import org.opensearch.common.cache.stats.CacheStatsDimension; import org.opensearch.common.cache.store.config.CacheConfig; import org.opensearch.common.metrics.CounterMetric; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.bytes.CompositeBytesReference; +import org.opensearch.core.xcontent.MediaTypeRegistry; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.env.NodeEnvironment; import org.opensearch.test.OpenSearchSingleNodeTestCase; @@ -798,6 +804,77 @@ public void testInvalidate() throws Exception { } } + // Modified from OpenSearchOnHeapCacheTests.java + public void testInvalidateWithDropDimensions() throws Exception { + Settings settings = Settings.builder().build(); + List dimensionNames = List.of("dim1", "dim2"); + try (NodeEnvironment env = newNodeEnvironment(settings)) { + ICache ehCacheDiskCachingTier = new EhcacheDiskCache.Builder().setThreadPoolAlias("ehcacheTest") + .setStoragePath(env.nodePaths()[0].indicesPath.toString() + "/request_cache") + .setKeySerializer(new StringSerializer()) + .setValueSerializer(new StringSerializer()) + .setDimensionNames(dimensionNames) + .setKeyType(String.class) + .setValueType(String.class) + .setCacheType(CacheType.INDICES_REQUEST_CACHE) + .setSettings(settings) + .setMaximumWeightInBytes(CACHE_SIZE_IN_BYTES * 20) // bigger so no evictions happen + .setExpireAfterAccess(TimeValue.MAX_VALUE) + .setRemovalListener(new MockRemovalListener<>()) + .setWeigher((key, value) -> 1) + .build(); + + List> keysAdded = new ArrayList<>(); + + for (int i = 0; i < 20; i++) { + ICacheKey key = new ICacheKey<>(UUID.randomUUID().toString(), getRandomDimensions(dimensionNames)); + keysAdded.add(key); + ehCacheDiskCachingTier.put(key, UUID.randomUUID().toString()); + } + + ICacheKey keyToDrop = keysAdded.get(0); + + Map xContentMap = getStatsXContentMap(ehCacheDiskCachingTier.stats(), dimensionNames); + List xContentMapKeys = getXContentMapKeys(keyToDrop); + Map individualSnapshotMap = (Map) getValueFromNestedXContentMap(xContentMap, xContentMapKeys); + assertNotNull(individualSnapshotMap); + assertEquals(5, individualSnapshotMap.size()); // Assert all 5 stats are present and not null + for (Map.Entry entry : individualSnapshotMap.entrySet()) { + Integer value = (Integer) entry.getValue(); + assertNotNull(value); + } + + for (CacheStatsDimension dim : keyToDrop.dimensions) { + dim.setDropStatsOnInvalidation(true); + } + ehCacheDiskCachingTier.invalidate(keyToDrop); + + // Now assert the stats are gone for any key that has this combination of dimensions, but still there otherwise + xContentMap = getStatsXContentMap(ehCacheDiskCachingTier.stats(), dimensionNames); + for (ICacheKey keyAdded : keysAdded) { + xContentMapKeys = getXContentMapKeys(keyAdded); + individualSnapshotMap = (Map) getValueFromNestedXContentMap(xContentMap, xContentMapKeys); + if (keyAdded.dimensions.equals(keyToDrop.dimensions)) { + assertNull(individualSnapshotMap); + } else { + assertNotNull(individualSnapshotMap); + } + } + + ehCacheDiskCachingTier.close(); + } + } + + private List getRandomDimensions(List dimensionNames) { + Random rand = Randomness.get(); + int bound = 3; + List result = new ArrayList<>(); + for (String dimName : dimensionNames) { + result.add(new CacheStatsDimension(dimName, String.valueOf(rand.nextInt(bound)))); + } + return result; + } + private static String generateRandomString(int length) { String characters = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789"; StringBuilder randomString = new StringBuilder(length); @@ -833,6 +910,41 @@ private ToLongBiFunction, String> getWeigher() { }; } + // Helper functions duplicated from server.test; we can't add a dependency on that module + private static Map getStatsXContentMap(CacheStats cacheStats, List levels) throws IOException { + XContentBuilder builder = XContentFactory.jsonBuilder(); + Map paramMap = Map.of("level", String.join(",", levels)); + ToXContent.Params params = new ToXContent.MapParams(paramMap); + + builder.startObject(); + cacheStats.toXContent(builder, params); + builder.endObject(); + + String resultString = builder.toString(); + return XContentHelper.convertToMap(MediaTypeRegistry.JSON.xContent(), resultString, true); + } + + private List getXContentMapKeys(ICacheKey iCacheKey) { + List result = new ArrayList<>(); + for (CacheStatsDimension dim : iCacheKey.dimensions) { + result.add(dim.dimensionName); + result.add(dim.dimensionValue); + } + return result; + } + + public static Object getValueFromNestedXContentMap(Map xContentMap, List keys) { + Map current = xContentMap; + for (int i = 0; i < keys.size() - 1; i++) { + Object next = current.get(keys.get(i)); + if (next == null) { + return null; + } + current = (Map) next; + } + return current.get(keys.get(keys.size() - 1)); + } + static class MockRemovalListener implements RemovalListener, V> { CounterMetric evictionMetric = new CounterMetric(); diff --git a/server/src/main/java/org/opensearch/common/cache/CacheType.java b/server/src/main/java/org/opensearch/common/cache/CacheType.java index 46af3d2643e2c..61442db148067 100644 --- a/server/src/main/java/org/opensearch/common/cache/CacheType.java +++ b/server/src/main/java/org/opensearch/common/cache/CacheType.java @@ -31,6 +31,7 @@ public enum CacheType { public String getSettingPrefix() { return settingPrefix; } + public String getApiRepresentation() { return apiRepresentation; } diff --git a/server/src/main/java/org/opensearch/common/cache/service/NodeCacheStats.java b/server/src/main/java/org/opensearch/common/cache/service/NodeCacheStats.java index 41a0205d987a5..f27df6c8a2a90 100644 --- a/server/src/main/java/org/opensearch/common/cache/service/NodeCacheStats.java +++ b/server/src/main/java/org/opensearch/common/cache/service/NodeCacheStats.java @@ -33,21 +33,14 @@ public NodeCacheStats(LinkedHashMap statsByCache, CommonS public NodeCacheStats(StreamInput in) throws IOException { this.flags = new CommonStatsFlags(in); - Map readMap = in.readMap( - i -> i.readEnum(CacheType.class), - CacheStats::readFromStreamWithClassName - ); + Map readMap = in.readMap(i -> i.readEnum(CacheType.class), CacheStats::readFromStreamWithClassName); this.statsByCache = new LinkedHashMap<>(readMap); } @Override public void writeTo(StreamOutput out) throws IOException { flags.writeTo(out); - out.writeMap( - statsByCache, - StreamOutput::writeEnum, - (o, cacheStats) -> cacheStats.writeToWithClassName(o) - ); + out.writeMap(statsByCache, StreamOutput::writeEnum, (o, cacheStats) -> cacheStats.writeToWithClassName(o)); } @Override @@ -71,8 +64,7 @@ public boolean equals(Object o) { return false; } NodeCacheStats other = (NodeCacheStats) o; - return statsByCache.equals(other.statsByCache) - && flags.getIncludeCaches().equals(other.flags.getIncludeCaches()); + return statsByCache.equals(other.statsByCache) && flags.getIncludeCaches().equals(other.flags.getIncludeCaches()); } @Override 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 5b9b4fe800fa9..510bd4a15b1bb 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 @@ -14,8 +14,6 @@ import org.opensearch.core.xcontent.ToXContentFragment; import java.io.IOException; -import java.lang.reflect.Constructor; -import java.util.Map; /** * Interface for access to any cache stats. Allows accessing stats by dimension values. @@ -44,7 +42,6 @@ public interface CacheStats extends Writeable, ToXContentFragment { static CacheStats readFromStreamWithClassName(StreamInput in) throws IOException { String className = in.readString(); - //in.reset(); if (className.equals(MultiDimensionCacheStats.CLASS_NAME)) { return new MultiDimensionCacheStats(in); diff --git a/server/src/main/java/org/opensearch/common/cache/stats/CounterSnapshot.java b/server/src/main/java/org/opensearch/common/cache/stats/CounterSnapshot.java index 2ae1b407eeb2b..b7dc2a4551ef7 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/CounterSnapshot.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/CounterSnapshot.java @@ -114,7 +114,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws static final class Fields { static final String MEMORY_SIZE = "size"; // TODO: Bad name - think of something better static final String MEMORY_SIZE_IN_BYTES = "size_in_bytes"; - // TODO: This might not be memory as it could be partially on disk, so I've changed it, but should it be consistent with the earlier field? + // TODO: This might not be memory as it could be partially on disk, so I've changed it, but should it be consistent with the earlier + // field? static final String EVICTIONS = "evictions"; static final String HIT_COUNT = "hit_count"; static final String MISS_COUNT = "miss_count"; diff --git a/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java b/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java index 9301797cff013..55a16b575d3ea 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java @@ -168,7 +168,6 @@ DimensionNode aggregateByLevels(List levels) { return root; } - @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { // Always show total stats, regardless of levels @@ -193,7 +192,8 @@ XContentBuilder toXContentForLevels(XContentBuilder builder, Params params, List } - private void toXContentForLevelsHelper(int depth, DimensionNode current, List levels, XContentBuilder builder, Params params) throws IOException { + private void toXContentForLevelsHelper(int depth, DimensionNode current, List levels, XContentBuilder builder, Params params) + throws IOException { if (depth >= 0) { builder.startObject(current.dimensionValue); } @@ -202,9 +202,9 @@ private void toXContentForLevelsHelper(int depth, DimensionNode current, List getLevels(Params params) { String levels = params.param("level"); if (levels == null) { diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesStatsAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesStatsAction.java index 9c8c9ee3dd294..f62eaeb37f41f 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesStatsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesStatsAction.java @@ -188,7 +188,9 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC try { cacheFlags.includeCacheType(CacheType.getByRepresentation(cacheName)); } catch (IllegalArgumentException e) { - throw new IllegalArgumentException(unrecognized(request, Set.of(cacheName), CacheType.allRepresentations(), "cache type")); + throw new IllegalArgumentException( + unrecognized(request, Set.of(cacheName), CacheType.allRepresentations(), "cache type") + ); } } } diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java index 9abb877dd369c..18b74520bb920 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java @@ -952,7 +952,7 @@ public void apply(String action, AdmissionControlActionType admissionControlActi String indexName = "index" + indexNum; for (int shardNum = 0; shardNum < numShardsPerIndex; shardNum++) { String shardName = "[" + indexName + "][" + shardNum + "]"; - for (String tierName : new String[]{"dummy_tier_1", "dummy_tier_2"}) { + for (String tierName : new String[] { "dummy_tier_1", "dummy_tier_2" }) { CounterSnapshot response = new CounterSnapshot( randomInt(100), randomInt(100), diff --git a/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java b/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java index 59948e1aa86dc..99c9f13c80a4d 100644 --- a/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java +++ b/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java @@ -13,7 +13,6 @@ import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.common.xcontent.XContentHelper; -import org.opensearch.common.xcontent.XContentType; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.io.stream.BytesStreamInput; import org.opensearch.core.xcontent.MediaTypeRegistry; @@ -23,14 +22,11 @@ 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.Set; import java.util.UUID; import java.util.function.BiConsumer; -import java.util.function.Consumer; public class MultiDimensionCacheStatsTests extends OpenSearchTestCase { public void testSerialization() throws Exception { @@ -155,10 +151,14 @@ public void testAggregateBySomeDimensions() throws Exception { public void testXContentForLevels() throws Exception { List dimensionNames = List.of("A", "B", "C"); Map snapshot = Map.of( - new StatsHolder.Key(List.of("A1", "B1", "C1")), new CounterSnapshot(1, 1, 1, 1, 1), - new StatsHolder.Key(List.of("A1", "B1", "C2")), new CounterSnapshot(2, 2, 2, 2, 2), - new StatsHolder.Key(List.of("A1", "B2", "C1")), new CounterSnapshot(3, 3, 3, 3, 3), - new StatsHolder.Key(List.of("A2", "B1", "C3")), new CounterSnapshot(4, 4, 4, 4, 4) + new StatsHolder.Key(List.of("A1", "B1", "C1")), + new CounterSnapshot(1, 1, 1, 1, 1), + new StatsHolder.Key(List.of("A1", "B1", "C2")), + new CounterSnapshot(2, 2, 2, 2, 2), + new StatsHolder.Key(List.of("A1", "B2", "C1")), + new CounterSnapshot(3, 3, 3, 3, 3), + new StatsHolder.Key(List.of("A2", "B1", "C3")), + new CounterSnapshot(4, 4, 4, 4, 4) ); MultiDimensionCacheStats stats = new MultiDimensionCacheStats(snapshot, dimensionNames); @@ -172,11 +172,16 @@ public void testXContentForLevels() throws Exception { Map result = XContentHelper.convertToMap(MediaTypeRegistry.JSON.xContent(), resultString, true); Map> fieldNamesMap = Map.of( - CounterSnapshot.Fields.MEMORY_SIZE_IN_BYTES, (counter, value) -> counter.sizeInBytes.inc(value), - CounterSnapshot.Fields.EVICTIONS, (counter, value) -> counter.evictions.inc(value), - CounterSnapshot.Fields.HIT_COUNT, (counter, value) -> counter.hits.inc(value), - CounterSnapshot.Fields.MISS_COUNT, (counter, value) -> counter. misses.inc(value), - CounterSnapshot.Fields.ENTRIES, (counter, value) -> counter.entries.inc(value) + CounterSnapshot.Fields.MEMORY_SIZE_IN_BYTES, + (counter, value) -> counter.sizeInBytes.inc(value), + CounterSnapshot.Fields.EVICTIONS, + (counter, value) -> counter.evictions.inc(value), + CounterSnapshot.Fields.HIT_COUNT, + (counter, value) -> counter.hits.inc(value), + CounterSnapshot.Fields.MISS_COUNT, + (counter, value) -> counter.misses.inc(value), + CounterSnapshot.Fields.ENTRIES, + (counter, value) -> counter.entries.inc(value) ); for (Map.Entry entry : snapshot.entrySet()) { @@ -190,7 +195,7 @@ public void testXContentForLevels() throws Exception { for (Map.Entry> fieldNamesEntry : fieldNamesMap.entrySet()) { List fullXContentKeys = new ArrayList<>(xContentKeys); fullXContentKeys.add(fieldNamesEntry.getKey()); - int valueInXContent = getValueFromNestedXContentMap(result, fullXContentKeys); + int valueInXContent = (int) getValueFromNestedXContentMap(result, fullXContentKeys); BiConsumer incrementer = fieldNamesEntry.getValue(); incrementer.accept(counterFromXContent, valueInXContent); } @@ -200,12 +205,16 @@ public void testXContentForLevels() throws Exception { } } - private int getValueFromNestedXContentMap(Map xContentMap, List keys) { + public static Object getValueFromNestedXContentMap(Map xContentMap, List keys) { Map current = xContentMap; for (int i = 0; i < keys.size() - 1; i++) { - current = (Map) current.get(keys.get(i)); + Object next = current.get(keys.get(i)); + if (next == null) { + return null; + } + current = (Map) next; } - return (int) current.get(keys.get(keys.size()-1)); + return current.get(keys.get(keys.size() - 1)); } // Get a map from the list of dimension values to the corresponding leaf node. @@ -215,7 +224,11 @@ private Map, MultiDimensionCacheStats.DimensionNode> getAllLeafNode return result; } - private void getAllLeafNodesHelper(Map, MultiDimensionCacheStats.DimensionNode> result, MultiDimensionCacheStats.DimensionNode current, List pathToCurrent) { + private void getAllLeafNodesHelper( + Map, MultiDimensionCacheStats.DimensionNode> result, + MultiDimensionCacheStats.DimensionNode current, + List pathToCurrent + ) { if (current.children.isEmpty()) { result.put(pathToCurrent, current); } else { diff --git a/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java b/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java index 2b70a9f637ce7..79e652afb9dbe 100644 --- a/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java +++ b/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java @@ -16,7 +16,6 @@ import java.util.List; import java.util.Map; import java.util.Random; -import java.util.Set; import static org.opensearch.common.cache.stats.MultiDimensionCacheStatsTests.getUsedDimensionValues; import static org.opensearch.common.cache.stats.MultiDimensionCacheStatsTests.populateStats; 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 b7e49e85039a1..df26f60c1b36c 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 @@ -8,21 +8,32 @@ package org.opensearch.common.cache.store; +import org.opensearch.common.Randomness; import org.opensearch.common.cache.CacheType; import org.opensearch.common.cache.ICache; import org.opensearch.common.cache.ICacheKey; import org.opensearch.common.cache.LoadAwareCacheLoader; import org.opensearch.common.cache.RemovalListener; import org.opensearch.common.cache.RemovalNotification; +import org.opensearch.common.cache.stats.CacheStats; import org.opensearch.common.cache.stats.CacheStatsDimension; +import org.opensearch.common.cache.stats.MultiDimensionCacheStatsTests; import org.opensearch.common.cache.store.config.CacheConfig; import org.opensearch.common.cache.store.settings.OpenSearchOnHeapCacheSettings; import org.opensearch.common.metrics.CounterMetric; import org.opensearch.common.settings.Settings; +import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.common.xcontent.XContentHelper; +import org.opensearch.core.xcontent.MediaTypeRegistry; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.test.OpenSearchTestCase; +import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Map; +import java.util.Random; import java.util.UUID; import static org.opensearch.common.cache.store.settings.OpenSearchOnHeapCacheSettings.MAXIMUM_SIZE_IN_BYTES_KEY; @@ -97,6 +108,74 @@ private OpenSearchOnHeapCache getCache(int maxSizeKeys, MockRemo return (OpenSearchOnHeapCache) onHeapCacheFactory.create(cacheConfig, CacheType.INDICES_REQUEST_CACHE, null); } + public void testInvalidateWithDropDimensions() throws Exception { + MockRemovalListener listener = new MockRemovalListener<>(); + int maxKeys = 50; + OpenSearchOnHeapCache cache = getCache(maxKeys, listener); + + List> keysAdded = new ArrayList<>(); + + for (int i = 0; i < maxKeys - 5; i++) { + ICacheKey key = new ICacheKey<>(UUID.randomUUID().toString(), getRandomDimensions()); + keysAdded.add(key); + cache.computeIfAbsent(key, getLoadAwareCacheLoader()); + } + + ICacheKey keyToDrop = keysAdded.get(0); + + Map xContentMap = getStatsXContentMap(cache.stats(), dimensionNames); + List xContentMapKeys = getXContentMapKeys(keyToDrop); + Map individualSnapshotMap = (Map) MultiDimensionCacheStatsTests.getValueFromNestedXContentMap( + xContentMap, + xContentMapKeys + ); + assertNotNull(individualSnapshotMap); + assertEquals(5, individualSnapshotMap.size()); // Assert all 5 stats are present and not null + for (Map.Entry entry : individualSnapshotMap.entrySet()) { + Integer value = (Integer) entry.getValue(); + assertNotNull(value); + } + + for (CacheStatsDimension dim : keyToDrop.dimensions) { + dim.setDropStatsOnInvalidation(true); + } + cache.invalidate(keyToDrop); + + // Now assert the stats are gone for any key that has this combination of dimensions, but still there otherwise + xContentMap = getStatsXContentMap(cache.stats(), dimensionNames); + for (ICacheKey keyAdded : keysAdded) { + xContentMapKeys = getXContentMapKeys(keyAdded); + individualSnapshotMap = (Map) MultiDimensionCacheStatsTests.getValueFromNestedXContentMap( + xContentMap, + xContentMapKeys + ); + if (keyAdded.dimensions.equals(keyToDrop.dimensions)) { + assertNull(individualSnapshotMap); + } else { + assertNotNull(individualSnapshotMap); + } + } + } + + private List getXContentMapKeys(ICacheKey iCacheKey) { + List result = new ArrayList<>(); + for (CacheStatsDimension dim : iCacheKey.dimensions) { + result.add(dim.dimensionName); + result.add(dim.dimensionValue); + } + return result; + } + + private List getRandomDimensions() { + Random rand = Randomness.get(); + int bound = 3; + List result = new ArrayList<>(); + for (String dimName : dimensionNames) { + result.add(new CacheStatsDimension(dimName, String.valueOf(rand.nextInt(bound)))); + } + return result; + } + private static class MockRemovalListener implements RemovalListener, V> { CounterMetric numRemovals; @@ -110,6 +189,20 @@ public void onRemoval(RemovalNotification, V> notification) { } } + // Public as this is used in other tests as well + public static Map getStatsXContentMap(CacheStats cacheStats, List levels) throws IOException { + XContentBuilder builder = XContentFactory.jsonBuilder(); + Map paramMap = Map.of("level", String.join(",", levels)); + ToXContent.Params params = new ToXContent.MapParams(paramMap); + + builder.startObject(); + cacheStats.toXContent(builder, params); + builder.endObject(); + + String resultString = builder.toString(); + return XContentHelper.convertToMap(MediaTypeRegistry.JSON.xContent(), resultString, true); + } + private ICacheKey getICacheKey(String key) { List dims = new ArrayList<>(); for (String dimName : dimensionNames) { diff --git a/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java b/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java index 6143eeb5f13e4..9d4d2ba4aa52d 100644 --- a/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java +++ b/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java @@ -45,12 +45,15 @@ import org.apache.lucene.search.TopDocs; import org.apache.lucene.store.Directory; import org.apache.lucene.util.BytesRef; +import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.CheckedSupplier; import org.opensearch.common.cache.ICacheKey; import org.opensearch.common.cache.RemovalNotification; 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.MultiDimensionCacheStatsTests; +import org.opensearch.common.cache.store.OpenSearchOnHeapCacheTests; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.lucene.index.OpenSearchDirectoryReader; import org.opensearch.common.settings.Settings; @@ -70,6 +73,7 @@ import org.opensearch.index.query.TermQueryBuilder; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.IndexShardState; +import org.opensearch.index.shard.ShardNotFoundException; import org.opensearch.node.Node; import org.opensearch.test.OpenSearchSingleNodeTestCase; import org.opensearch.threadpool.ThreadPool; @@ -77,11 +81,15 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; +import static org.opensearch.indices.IndicesRequestCache.INDEX_DIMENSION_NAME; import static org.opensearch.indices.IndicesRequestCache.INDICES_REQUEST_CACHE_STALENESS_THRESHOLD_SETTING; +import static org.opensearch.indices.IndicesRequestCache.SHARD_ID_DIMENSION_NAME; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -753,6 +761,125 @@ public void testCacheCleanupBasedOnStaleThreshold_StalenessLesserThanThreshold() terminate(threadPool); } + public void testClosingIndexWipesStats() throws Exception { + IndicesService indicesService = getInstanceFromNode(IndicesService.class); + // Create two indices each with multiple shards + int numShards = 3; + Settings indexSettings = Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, numShards).build(); + String indexToKeepName = "test"; + String indexToCloseName = "test2"; + IndexService indexToKeep = createIndex(indexToKeepName, indexSettings); + IndexService indexToClose = createIndex(indexToCloseName, indexSettings); + for (int i = 0; i < numShards; i++) { + // Check we can get all the shards we expect + assertNotNull(indexToKeep.getShard(i)); + assertNotNull(indexToClose.getShard(i)); + } + ThreadPool threadPool = getThreadPool(); + Settings settings = Settings.builder().put(INDICES_REQUEST_CACHE_STALENESS_THRESHOLD_SETTING.getKey(), "0.001%").build(); + IndicesRequestCache cache = new IndicesRequestCache(settings, (shardId -> { + IndexService indexService = null; + try { + indexService = indicesService.indexServiceSafe(shardId.getIndex()); + } catch (IndexNotFoundException ex) { + return Optional.empty(); + } + try { + return Optional.of(new IndicesService.IndexShardCacheEntity(indexService.getShard(shardId.id()))); + } catch (ShardNotFoundException ex) { + return Optional.empty(); + } + }), new CacheModule(new ArrayList<>(), Settings.EMPTY).getCacheService(), threadPool); + Directory dir = newDirectory(); + IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); + + writer.addDocument(newDoc(0, "foo")); + TermQueryBuilder termQuery = new TermQueryBuilder("id", "0"); + BytesReference termBytes = XContentHelper.toXContent(termQuery, MediaTypeRegistry.JSON, false); + if (randomBoolean()) { + writer.flush(); + IOUtils.close(writer); + writer = new IndexWriter(dir, newIndexWriterConfig()); + } + writer.updateDocument(new Term("id", "0"), newDoc(0, "bar")); + DirectoryReader secondReader = OpenSearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "bar", 1)); + + List readersToClose = new ArrayList<>(); + List readersToKeep = new ArrayList<>(); + // Put entries into the cache for each shard + for (IndexService indexService : new IndexService[] { indexToKeep, indexToClose }) { + for (int i = 0; i < numShards; i++) { + IndexShard indexShard = indexService.getShard(i); + IndicesService.IndexShardCacheEntity entity = new IndicesService.IndexShardCacheEntity(indexShard); + DirectoryReader reader = OpenSearchDirectoryReader.wrap(DirectoryReader.open(writer), indexShard.shardId()); + if (indexService == indexToClose) { + readersToClose.add(reader); + } else { + readersToKeep.add(reader); + } + Loader loader = new Loader(reader, 0); + cache.getOrCompute(entity, loader, reader, termBytes); + } + } + + // Check resulting stats + List dimensionNames = List.of(INDEX_DIMENSION_NAME, SHARD_ID_DIMENSION_NAME); + Map xContentMap = OpenSearchOnHeapCacheTests.getStatsXContentMap(cache.getCacheStats(), dimensionNames); + List> initialXContentPaths = new ArrayList<>(); + for (IndexService indexService : new IndexService[] { indexToKeep, indexToClose }) { + for (int i = 0; i < numShards; i++) { + ShardId shardId = indexService.getShard(i).shardId(); + List xContentPath = List.of( + INDEX_DIMENSION_NAME, + shardId.getIndexName(), + SHARD_ID_DIMENSION_NAME, + shardId.toString() + ); + initialXContentPaths.add(xContentPath); + Map individualSnapshotMap = (Map) MultiDimensionCacheStatsTests + .getValueFromNestedXContentMap(xContentMap, xContentPath); + assertNotNull(individualSnapshotMap); + // check the values are not empty by confirming entries != 0, this should always be true since the missed value is loaded + // into the cache + assertNotEquals(0, (int) individualSnapshotMap.get("entries")); + } + } + + // Delete an index + indexToClose.close("test_deletion", true); + // This actually closes the shards associated with the readers, which is necessary for cache cleanup logic + // In this UT, manually close the readers as well; could not figure out how to connect all this up in a UT so that + // we could get readers that were properly connected to an index's directory + for (DirectoryReader reader : readersToClose) { + IOUtils.close(reader); + } + // Trigger cache cleanup + cache.cacheCleanupManager.cleanCache(); + + // Now stats for the closed index should be gone + xContentMap = OpenSearchOnHeapCacheTests.getStatsXContentMap(cache.getCacheStats(), dimensionNames); + for (List path : initialXContentPaths) { + Map individualSnapshotMap = (Map) MultiDimensionCacheStatsTests.getValueFromNestedXContentMap( + xContentMap, + path + ); + if (path.get(1).equals(indexToCloseName)) { + assertNull(individualSnapshotMap); + } else { + assertNotNull(individualSnapshotMap); + // check the values are not empty by confirming entries != 0, this should always be true since the missed value is loaded + // into the cache + assertNotEquals(0, (int) individualSnapshotMap.get("entries")); + } + } + + for (DirectoryReader reader : readersToKeep) { + IOUtils.close(reader); + } + IOUtils.close(secondReader, writer, dir, cache); + terminate(threadPool); + } + public void testEviction() throws Exception { final ByteSizeValue size; { From e6dca8dd1364042cfa45df38802b6e150186d326 Mon Sep 17 00:00:00 2001 From: Peter Alfonsi Date: Fri, 29 Mar 2024 09:51:16 -0700 Subject: [PATCH 08/17] changed statsholder key to contain whole dimension Signed-off-by: Peter Alfonsi --- .../cache/stats/MultiDimensionCacheStats.java | 8 +++--- .../common/cache/stats/StatsHolder.java | 28 +++++++++++-------- .../stats/MultiDimensionCacheStatsTests.java | 8 ++++-- .../common/cache/stats/StatsHolderTests.java | 26 +++++++++++++---- 4 files changed, 47 insertions(+), 23 deletions(-) diff --git a/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java b/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java index 55a16b575d3ea..a5ff8b6c5cca1 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java @@ -44,7 +44,7 @@ public MultiDimensionCacheStats(Map snapshot, public MultiDimensionCacheStats(StreamInput in) throws IOException { this.dimensionNames = List.of(in.readStringArray()); this.snapshot = in.readMap( - i -> new StatsHolder.Key(List.of(i.readArray(StreamInput::readString, String[]::new))), + i -> new StatsHolder.Key(List.of(i.readArray(CacheStatsDimension::new, CacheStatsDimension[]::new))), CounterSnapshot::new ); } @@ -54,7 +54,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeStringArray(dimensionNames.toArray(new String[0])); out.writeMap( snapshot, - (o, key) -> o.writeArray((o1, dimValue) -> o1.writeString((String) dimValue), key.dimensionValues.toArray()), + (o, key) -> o.writeArray((o1, dim) -> ((CacheStatsDimension) dim).writeTo(o1), key.dimensions.toArray()), (o, snapshot) -> snapshot.writeTo(o) ); } @@ -158,9 +158,9 @@ DimensionNode aggregateByLevels(List levels) { DimensionNode root = new DimensionNode(null); for (Map.Entry entry : snapshot.entrySet()) { List levelValues = new ArrayList<>(); // This key's relevant dimension values, which match the levels - List keyDimensionValues = entry.getKey().dimensionValues; + List keyDimensions = entry.getKey().dimensions; for (int levelPosition : levelPositions) { - levelValues.add(keyDimensionValues.get(levelPosition)); + levelValues.add(keyDimensions.get(levelPosition).dimensionValue); } DimensionNode leafNode = root.getNode(levelValues); leafNode.addSnapshot(entry.getValue()); diff --git a/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java b/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java index 31b0fe37751db..11e4a4c622525 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java @@ -137,17 +137,17 @@ private CacheStatsCounter internalGetStats(List dimensions) * Get a valid key from an unordered list of dimensions. */ private Key getKey(List dims) { - return new Key(getOrderedDimensionValues(dims, dimensionNames)); + return new Key(getOrderedDimensions(dims, dimensionNames)); } // Get a list of dimension values, ordered according to dimensionNames, from the possibly differently-ordered dimensions passed in. // Public and static for testing purposes. - public static List getOrderedDimensionValues(List dimensions, List dimensionNames) { - List result = new ArrayList<>(); + public static List getOrderedDimensions(List dimensions, List dimensionNames) { + List result = new ArrayList<>(); for (String dimensionName : dimensionNames) { for (CacheStatsDimension dim : dimensions) { if (dim.dimensionName.equals(dimensionName)) { - result.add(dim.dimensionValue); + result.add(dim); } } } @@ -183,13 +183,17 @@ public void removeDimensions(List dims) { } } + /** + * Check if the Key contains all the dimensions in dims, matching both dimension name and value. + */ boolean keyContainsAllDimensions(Key key, List dims) { for (CacheStatsDimension dim : dims) { int dimensionPosition = dimensionNames.indexOf(dim.dimensionName); if (dimensionPosition == -1) { throw new IllegalArgumentException("Unrecognized dimension: " + dim.dimensionName + " = " + dim.dimensionValue); } - if (!key.dimensionValues.get(dimensionPosition).equals(dim.dimensionValue)) { + String keyDimensionValue = key.dimensions.get(dimensionPosition).dimensionValue; + if (!keyDimensionValue.equals(dim.dimensionValue)) { return false; } } @@ -200,14 +204,14 @@ boolean keyContainsAllDimensions(Key key, List dims) { * Unmodifiable wrapper over a list of dimension values, ordered according to dimensionNames. Pkg-private for testing. */ public static class Key { - final List dimensionValues; // The dimensions must be ordered + final List dimensions; // The dimensions must be ordered - public Key(List dimensionValues) { - this.dimensionValues = Collections.unmodifiableList(dimensionValues); + public Key(List dimensions) { + this.dimensions = Collections.unmodifiableList(dimensions); } - public List getDimensionValues() { - return dimensionValues; + public List getDimensions() { + return dimensions; } @Override @@ -222,12 +226,12 @@ public boolean equals(Object o) { return false; } Key other = (Key) o; - return this.dimensionValues.equals(other.dimensionValues); + return this.dimensions.equals(other.dimensions); } @Override public int hashCode() { - return this.dimensionValues.hashCode(); + return this.dimensions.hashCode(); } } } diff --git a/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java b/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java index 99c9f13c80a4d..6d9361e10aed2 100644 --- a/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java +++ b/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java @@ -61,7 +61,7 @@ public void testAddAndGet() throws Exception { // test the value in the map is as expected for each distinct combination of values for (List dims : expected.keySet()) { CacheStatsCounter expectedCounter = expected.get(dims); - StatsHolder.Key key = new StatsHolder.Key(StatsHolder.getOrderedDimensionValues(dims, dimensionNames)); + StatsHolder.Key key = new StatsHolder.Key(StatsHolder.getOrderedDimensions(dims, dimensionNames)); CounterSnapshot actual = stats.snapshot.get(key); assertEquals(expectedCounter.snapshot(), actual); @@ -134,10 +134,14 @@ public void testAggregateBySomeDimensions() throws Exception { for (Map.Entry, MultiDimensionCacheStats.DimensionNode> aggEntry : aggregatedLeafNodes.entrySet()) { CacheStatsCounter expectedCounter = new CacheStatsCounter(); for (List expectedDims : expected.keySet()) { - List orderedDimValues = StatsHolder.getOrderedDimensionValues( + List orderedDims = StatsHolder.getOrderedDimensions( new ArrayList<>(expectedDims), dimensionNames ); + List orderedDimValues = new ArrayList<>(); + for (CacheStatsDimension dim : orderedDims) { + orderedDimValues.add(dim.dimensionValue); + } if (orderedDimValues.containsAll(aggEntry.getKey())) { expectedCounter.add(expected.get(expectedDims)); } diff --git a/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java b/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java index 79e652afb9dbe..5bdbb6e2ab0e9 100644 --- a/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java +++ b/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java @@ -26,10 +26,18 @@ public class StatsHolderTests extends OpenSearchTestCase { // in MultiDimensionCacheStatsTests.java. public void testKeyEquality() throws Exception { - List dims1 = List.of("1", "2", "3"); + List dims1 = List.of( + new CacheStatsDimension("A", "1"), + new CacheStatsDimension("B", "2"), + new CacheStatsDimension("C", "3") + ); StatsHolder.Key key1 = new StatsHolder.Key(dims1); - List dims2 = List.of("1", "2", "3"); + List dims2 = List.of( + new CacheStatsDimension("A", "1"), + new CacheStatsDimension("B", "2"), + new CacheStatsDimension("C", "3") + ); StatsHolder.Key key2 = new StatsHolder.Key(dims2); assertEquals(key1, key2); @@ -49,7 +57,7 @@ public void testReset() throws Exception { originalCounter.sizeInBytes = new CounterMetric(); originalCounter.entries = new CounterMetric(); - StatsHolder.Key key = new StatsHolder.Key(StatsHolder.getOrderedDimensionValues(dims, dimensionNames)); + StatsHolder.Key key = new StatsHolder.Key(StatsHolder.getOrderedDimensions(dims, dimensionNames)); CacheStatsCounter actual = statsHolder.getStatsMap().get(key); assertEquals(originalCounter, actual); } @@ -68,8 +76,16 @@ public void testKeyContainsAllDimensions() throws Exception { List dims = List.of(new CacheStatsDimension("dim1", "A"), new CacheStatsDimension("dim2", "B")); - StatsHolder.Key matchingKey = new StatsHolder.Key(List.of("A", "B", "C")); - StatsHolder.Key nonMatchingKey = new StatsHolder.Key(List.of("A", "Z", "C")); + StatsHolder.Key matchingKey = new StatsHolder.Key(List.of( + new CacheStatsDimension("dim1", "A"), + new CacheStatsDimension("dim2", "B"), + new CacheStatsDimension("dim3", "C") + )); + StatsHolder.Key nonMatchingKey = new StatsHolder.Key(List.of( + new CacheStatsDimension("dim1", "A"), + new CacheStatsDimension("dim2", "Z"), + new CacheStatsDimension("dim3", "C") + )); assertTrue(statsHolder.keyContainsAllDimensions(matchingKey, dims)); assertFalse(statsHolder.keyContainsAllDimensions(nonMatchingKey, dims)); From 08b688fd2bf03737812026f98eb0f7f9fc9ffdfc Mon Sep 17 00:00:00 2001 From: Peter Alfonsi Date: Mon, 1 Apr 2024 10:31:46 -0700 Subject: [PATCH 09/17] Revert "changed statsholder key to contain whole dimension" This reverts commit e6dca8dd1364042cfa45df38802b6e150186d326. --- .../cache/stats/MultiDimensionCacheStats.java | 8 +++--- .../common/cache/stats/StatsHolder.java | 28 ++++++++----------- .../stats/MultiDimensionCacheStatsTests.java | 8 ++---- .../common/cache/stats/StatsHolderTests.java | 26 ++++------------- 4 files changed, 23 insertions(+), 47 deletions(-) diff --git a/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java b/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java index a5ff8b6c5cca1..55a16b575d3ea 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java @@ -44,7 +44,7 @@ public MultiDimensionCacheStats(Map snapshot, public MultiDimensionCacheStats(StreamInput in) throws IOException { this.dimensionNames = List.of(in.readStringArray()); this.snapshot = in.readMap( - i -> new StatsHolder.Key(List.of(i.readArray(CacheStatsDimension::new, CacheStatsDimension[]::new))), + i -> new StatsHolder.Key(List.of(i.readArray(StreamInput::readString, String[]::new))), CounterSnapshot::new ); } @@ -54,7 +54,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeStringArray(dimensionNames.toArray(new String[0])); out.writeMap( snapshot, - (o, key) -> o.writeArray((o1, dim) -> ((CacheStatsDimension) dim).writeTo(o1), key.dimensions.toArray()), + (o, key) -> o.writeArray((o1, dimValue) -> o1.writeString((String) dimValue), key.dimensionValues.toArray()), (o, snapshot) -> snapshot.writeTo(o) ); } @@ -158,9 +158,9 @@ DimensionNode aggregateByLevels(List levels) { DimensionNode root = new DimensionNode(null); for (Map.Entry entry : snapshot.entrySet()) { List levelValues = new ArrayList<>(); // This key's relevant dimension values, which match the levels - List keyDimensions = entry.getKey().dimensions; + List keyDimensionValues = entry.getKey().dimensionValues; for (int levelPosition : levelPositions) { - levelValues.add(keyDimensions.get(levelPosition).dimensionValue); + levelValues.add(keyDimensionValues.get(levelPosition)); } DimensionNode leafNode = root.getNode(levelValues); leafNode.addSnapshot(entry.getValue()); diff --git a/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java b/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java index 11e4a4c622525..31b0fe37751db 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java @@ -137,17 +137,17 @@ private CacheStatsCounter internalGetStats(List dimensions) * Get a valid key from an unordered list of dimensions. */ private Key getKey(List dims) { - return new Key(getOrderedDimensions(dims, dimensionNames)); + return new Key(getOrderedDimensionValues(dims, dimensionNames)); } // Get a list of dimension values, ordered according to dimensionNames, from the possibly differently-ordered dimensions passed in. // Public and static for testing purposes. - public static List getOrderedDimensions(List dimensions, List dimensionNames) { - List result = new ArrayList<>(); + public static List getOrderedDimensionValues(List dimensions, List dimensionNames) { + List result = new ArrayList<>(); for (String dimensionName : dimensionNames) { for (CacheStatsDimension dim : dimensions) { if (dim.dimensionName.equals(dimensionName)) { - result.add(dim); + result.add(dim.dimensionValue); } } } @@ -183,17 +183,13 @@ public void removeDimensions(List dims) { } } - /** - * Check if the Key contains all the dimensions in dims, matching both dimension name and value. - */ boolean keyContainsAllDimensions(Key key, List dims) { for (CacheStatsDimension dim : dims) { int dimensionPosition = dimensionNames.indexOf(dim.dimensionName); if (dimensionPosition == -1) { throw new IllegalArgumentException("Unrecognized dimension: " + dim.dimensionName + " = " + dim.dimensionValue); } - String keyDimensionValue = key.dimensions.get(dimensionPosition).dimensionValue; - if (!keyDimensionValue.equals(dim.dimensionValue)) { + if (!key.dimensionValues.get(dimensionPosition).equals(dim.dimensionValue)) { return false; } } @@ -204,14 +200,14 @@ boolean keyContainsAllDimensions(Key key, List dims) { * Unmodifiable wrapper over a list of dimension values, ordered according to dimensionNames. Pkg-private for testing. */ public static class Key { - final List dimensions; // The dimensions must be ordered + final List dimensionValues; // The dimensions must be ordered - public Key(List dimensions) { - this.dimensions = Collections.unmodifiableList(dimensions); + public Key(List dimensionValues) { + this.dimensionValues = Collections.unmodifiableList(dimensionValues); } - public List getDimensions() { - return dimensions; + public List getDimensionValues() { + return dimensionValues; } @Override @@ -226,12 +222,12 @@ public boolean equals(Object o) { return false; } Key other = (Key) o; - return this.dimensions.equals(other.dimensions); + return this.dimensionValues.equals(other.dimensionValues); } @Override public int hashCode() { - return this.dimensions.hashCode(); + return this.dimensionValues.hashCode(); } } } diff --git a/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java b/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java index 6d9361e10aed2..99c9f13c80a4d 100644 --- a/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java +++ b/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java @@ -61,7 +61,7 @@ public void testAddAndGet() throws Exception { // test the value in the map is as expected for each distinct combination of values for (List dims : expected.keySet()) { CacheStatsCounter expectedCounter = expected.get(dims); - StatsHolder.Key key = new StatsHolder.Key(StatsHolder.getOrderedDimensions(dims, dimensionNames)); + StatsHolder.Key key = new StatsHolder.Key(StatsHolder.getOrderedDimensionValues(dims, dimensionNames)); CounterSnapshot actual = stats.snapshot.get(key); assertEquals(expectedCounter.snapshot(), actual); @@ -134,14 +134,10 @@ public void testAggregateBySomeDimensions() throws Exception { for (Map.Entry, MultiDimensionCacheStats.DimensionNode> aggEntry : aggregatedLeafNodes.entrySet()) { CacheStatsCounter expectedCounter = new CacheStatsCounter(); for (List expectedDims : expected.keySet()) { - List orderedDims = StatsHolder.getOrderedDimensions( + List orderedDimValues = StatsHolder.getOrderedDimensionValues( new ArrayList<>(expectedDims), dimensionNames ); - List orderedDimValues = new ArrayList<>(); - for (CacheStatsDimension dim : orderedDims) { - orderedDimValues.add(dim.dimensionValue); - } if (orderedDimValues.containsAll(aggEntry.getKey())) { expectedCounter.add(expected.get(expectedDims)); } diff --git a/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java b/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java index 5bdbb6e2ab0e9..79e652afb9dbe 100644 --- a/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java +++ b/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java @@ -26,18 +26,10 @@ public class StatsHolderTests extends OpenSearchTestCase { // in MultiDimensionCacheStatsTests.java. public void testKeyEquality() throws Exception { - List dims1 = List.of( - new CacheStatsDimension("A", "1"), - new CacheStatsDimension("B", "2"), - new CacheStatsDimension("C", "3") - ); + List dims1 = List.of("1", "2", "3"); StatsHolder.Key key1 = new StatsHolder.Key(dims1); - List dims2 = List.of( - new CacheStatsDimension("A", "1"), - new CacheStatsDimension("B", "2"), - new CacheStatsDimension("C", "3") - ); + List dims2 = List.of("1", "2", "3"); StatsHolder.Key key2 = new StatsHolder.Key(dims2); assertEquals(key1, key2); @@ -57,7 +49,7 @@ public void testReset() throws Exception { originalCounter.sizeInBytes = new CounterMetric(); originalCounter.entries = new CounterMetric(); - StatsHolder.Key key = new StatsHolder.Key(StatsHolder.getOrderedDimensions(dims, dimensionNames)); + StatsHolder.Key key = new StatsHolder.Key(StatsHolder.getOrderedDimensionValues(dims, dimensionNames)); CacheStatsCounter actual = statsHolder.getStatsMap().get(key); assertEquals(originalCounter, actual); } @@ -76,16 +68,8 @@ public void testKeyContainsAllDimensions() throws Exception { List dims = List.of(new CacheStatsDimension("dim1", "A"), new CacheStatsDimension("dim2", "B")); - StatsHolder.Key matchingKey = new StatsHolder.Key(List.of( - new CacheStatsDimension("dim1", "A"), - new CacheStatsDimension("dim2", "B"), - new CacheStatsDimension("dim3", "C") - )); - StatsHolder.Key nonMatchingKey = new StatsHolder.Key(List.of( - new CacheStatsDimension("dim1", "A"), - new CacheStatsDimension("dim2", "Z"), - new CacheStatsDimension("dim3", "C") - )); + StatsHolder.Key matchingKey = new StatsHolder.Key(List.of("A", "B", "C")); + StatsHolder.Key nonMatchingKey = new StatsHolder.Key(List.of("A", "Z", "C")); assertTrue(statsHolder.keyContainsAllDimensions(matchingKey, dims)); assertFalse(statsHolder.keyContainsAllDimensions(nonMatchingKey, dims)); From ee808baca14ad53be975eebc46fd16b253670149 Mon Sep 17 00:00:00 2001 From: Peter Alfonsi Date: Mon, 1 Apr 2024 12:38:16 -0700 Subject: [PATCH 10/17] Fixed tests with most recent changes Signed-off-by: Peter Alfonsi --- .../cluster/node/stats/NodeStatsTests.java | 9 ++++++- .../stats/MultiDimensionCacheStatsTests.java | 26 +++++++++++++++---- 2 files changed, 29 insertions(+), 6 deletions(-) diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java index 18b74520bb920..78e2870c8cf9a 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java @@ -45,6 +45,7 @@ import org.opensearch.common.cache.CacheType; import org.opensearch.common.cache.service.NodeCacheStats; import org.opensearch.common.cache.stats.CacheStats; +import org.opensearch.common.cache.stats.CacheStatsDimension; import org.opensearch.common.cache.stats.CounterSnapshot; import org.opensearch.common.cache.stats.MultiDimensionCacheStats; import org.opensearch.common.cache.stats.StatsHolder; @@ -960,7 +961,13 @@ public void apply(String action, AdmissionControlActionType admissionControlActi randomInt(100), randomInt(100) ); - snapshotMap.put(new StatsHolder.Key(List.of(indexName, shardName, tierName)), response); + snapshotMap.put( + new StatsHolder.Key(List.of( + new CacheStatsDimension("testIndexDimensionName", indexName), + new CacheStatsDimension("testShardDimensionName", shardName), + new CacheStatsDimension("testTierDimensionName", tierName) + )), + response); } } } diff --git a/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java b/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java index 6d9361e10aed2..af9682f18298a 100644 --- a/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java +++ b/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java @@ -155,13 +155,29 @@ public void testAggregateBySomeDimensions() throws Exception { public void testXContentForLevels() throws Exception { List dimensionNames = List.of("A", "B", "C"); Map snapshot = Map.of( - new StatsHolder.Key(List.of("A1", "B1", "C1")), + new StatsHolder.Key(List.of( + new CacheStatsDimension("A", "A1"), + new CacheStatsDimension("B", "B1"), + new CacheStatsDimension("C", "C1") + )), new CounterSnapshot(1, 1, 1, 1, 1), - new StatsHolder.Key(List.of("A1", "B1", "C2")), + new StatsHolder.Key(List.of( + new CacheStatsDimension("A", "A1"), + new CacheStatsDimension("B", "B1"), + new CacheStatsDimension("C", "C2") + )), new CounterSnapshot(2, 2, 2, 2, 2), - new StatsHolder.Key(List.of("A1", "B2", "C1")), + new StatsHolder.Key(List.of( + new CacheStatsDimension("A", "A1"), + new CacheStatsDimension("B", "B2"), + new CacheStatsDimension("C", "C1") + )), new CounterSnapshot(3, 3, 3, 3, 3), - new StatsHolder.Key(List.of("A2", "B1", "C3")), + new StatsHolder.Key(List.of( + new CacheStatsDimension("A", "A2"), + new CacheStatsDimension("B", "B1"), + new CacheStatsDimension("C", "C3") + )), new CounterSnapshot(4, 4, 4, 4, 4) ); MultiDimensionCacheStats stats = new MultiDimensionCacheStats(snapshot, dimensionNames); @@ -192,7 +208,7 @@ public void testXContentForLevels() throws Exception { List xContentKeys = new ArrayList<>(); for (int i = 0; i < dimensionNames.size(); i++) { xContentKeys.add(dimensionNames.get(i)); - xContentKeys.add(entry.getKey().dimensionValues.get(i)); + xContentKeys.add(entry.getKey().dimensions.get(i).dimensionValue); } CacheStatsCounter counterFromXContent = new CacheStatsCounter(); From 05541e04e6e661f73d7d09911504a0585b87fe7e Mon Sep 17 00:00:00 2001 From: Peter Alfonsi Date: Mon, 1 Apr 2024 12:55:03 -0700 Subject: [PATCH 11/17] Added store name to api response Signed-off-by: Peter Alfonsi --- .../cache/store/disk/EhcacheDiskCache.java | 2 +- .../cache/stats/MultiDimensionCacheStats.java | 11 ++++++++++- .../opensearch/common/cache/stats/StatsHolder.java | 8 ++++++-- .../common/cache/store/OpenSearchOnHeapCache.java | 2 +- .../admin/cluster/node/stats/NodeStatsTests.java | 2 +- .../cache/stats/MultiDimensionCacheStatsTests.java | 14 ++++++++------ .../common/cache/stats/StatsHolderTests.java | 7 ++++--- 7 files changed, 31 insertions(+), 15 deletions(-) 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 aaa36095aa8be..850df153d6e05 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 @@ -164,7 +164,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.statsHolder = new StatsHolder(dimensionNames, EhcacheDiskCacheFactory.EHCACHE_DISK_CACHE_NAME); } @SuppressWarnings({ "rawtypes" }) diff --git a/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java b/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java index a5ff8b6c5cca1..3c60a5e63d82c 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java @@ -31,11 +31,16 @@ public class MultiDimensionCacheStats implements CacheStats { final Map snapshot; final List dimensionNames; + // The name of the cache type producing these stats. Returned in API response. + final String storeName; + public static String STORE_NAME_FIELD = "store_name"; + public static String CLASS_NAME = "multidimension"; - public MultiDimensionCacheStats(Map snapshot, List dimensionNames) { + public MultiDimensionCacheStats(Map snapshot, List dimensionNames, String storeName) { this.snapshot = snapshot; this.dimensionNames = dimensionNames; + this.storeName = storeName; } /** @@ -47,6 +52,7 @@ public MultiDimensionCacheStats(StreamInput in) throws IOException { i -> new StatsHolder.Key(List.of(i.readArray(CacheStatsDimension::new, CacheStatsDimension[]::new))), CounterSnapshot::new ); + this.storeName = in.readString(); } @Override @@ -57,6 +63,7 @@ public void writeTo(StreamOutput out) throws IOException { (o, key) -> o.writeArray((o1, dim) -> ((CacheStatsDimension) dim).writeTo(o1), key.dimensions.toArray()), (o, snapshot) -> snapshot.writeTo(o) ); + out.writeString(storeName); } @Override @@ -181,6 +188,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws int[] positions = getLevelsInSortedOrder(levels); // Checks whether levels are valid; throws IllegalArgumentException if not toXContentForLevels(builder, params, levels); + // Also add the store name for the cache that produced the stats + builder.field(STORE_NAME_FIELD, storeName); return builder; } diff --git a/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java b/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java index 11e4a4c622525..51060bf9e62cb 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java @@ -37,9 +37,13 @@ public class StatsHolder { // A map from a set of cache stats dimension values -> stats for that ordered list of dimensions. private final ConcurrentMap statsMap; - public StatsHolder(List dimensionNames) { + // The name of the cache type using these stats + private final String storeName; + + public StatsHolder(List dimensionNames, String storeName) { this.dimensionNames = dimensionNames; this.statsMap = new ConcurrentHashMap<>(); + this.storeName = storeName; } public List getDimensionNames() { @@ -164,7 +168,7 @@ public CacheStats getCacheStats() { } // The resulting map is immutable as well as unmodifiable since the backing map is new, not related to statsMap Map immutableSnapshot = Collections.unmodifiableMap(snapshot); - return new MultiDimensionCacheStats(immutableSnapshot, dimensionNames); + return new MultiDimensionCacheStats(immutableSnapshot, dimensionNames, storeName); } /** 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 9c384cda1792e..5c288aedbb382 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 @@ -64,7 +64,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.statsHolder = new StatsHolder(dimensionNames, OpenSearchOnHeapCacheFactory.NAME); this.removalListener = builder.getRemovalListener(); this.weigher = builder.getWeigher(); } diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java index 78e2870c8cf9a..3386cd1183984 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java @@ -977,7 +977,7 @@ public void apply(String action, AdmissionControlActionType admissionControlActi flags.includeCacheType(cacheType); } } - MultiDimensionCacheStats cacheStats = new MultiDimensionCacheStats(snapshotMap, dimensionNames); + MultiDimensionCacheStats cacheStats = new MultiDimensionCacheStats(snapshotMap, dimensionNames, "dummyStoreName"); LinkedHashMap cacheStatsMap = new LinkedHashMap<>(); cacheStatsMap.put(CacheType.INDICES_REQUEST_CACHE, cacheStats); nodeCacheStats = new NodeCacheStats(cacheStatsMap, flags); diff --git a/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java b/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java index af9682f18298a..7d4f01df7f13b 100644 --- a/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java +++ b/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java @@ -29,9 +29,10 @@ import java.util.function.BiConsumer; public class MultiDimensionCacheStatsTests extends OpenSearchTestCase { + private final String storeName = "dummy_store"; public void testSerialization() throws Exception { List dimensionNames = List.of("dim1", "dim2"); - StatsHolder statsHolder = new StatsHolder(dimensionNames); + StatsHolder statsHolder = new StatsHolder(dimensionNames, storeName); Map> usedDimensionValues = getUsedDimensionValues(statsHolder, 10); populateStats(statsHolder, usedDimensionValues, 100, 10); MultiDimensionCacheStats stats = (MultiDimensionCacheStats) statsHolder.getCacheStats(); @@ -43,6 +44,7 @@ public void testSerialization() throws Exception { assertEquals(stats.snapshot, deserialized.snapshot); assertEquals(stats.dimensionNames, deserialized.dimensionNames); + assertEquals(stats.storeName, deserialized.storeName); os = new BytesStreamOutput(); stats.writeToWithClassName(os); @@ -53,7 +55,7 @@ public void testSerialization() throws Exception { public void testAddAndGet() throws Exception { List dimensionNames = List.of("dim1", "dim2", "dim3", "dim4"); - StatsHolder statsHolder = new StatsHolder(dimensionNames); + StatsHolder statsHolder = new StatsHolder(dimensionNames, storeName); Map> usedDimensionValues = getUsedDimensionValues(statsHolder, 10); Map, CacheStatsCounter> expected = populateStats(statsHolder, usedDimensionValues, 1000, 10); MultiDimensionCacheStats stats = (MultiDimensionCacheStats) statsHolder.getCacheStats(); @@ -83,7 +85,7 @@ public void testAddAndGet() throws Exception { public void testEmptyDimsList() throws Exception { // If the dimension list is empty, the map should have only one entry, from the empty set -> the total stats. - StatsHolder statsHolder = new StatsHolder(List.of()); + StatsHolder statsHolder = new StatsHolder(List.of(), storeName); Map> usedDimensionValues = getUsedDimensionValues(statsHolder, 100); populateStats(statsHolder, usedDimensionValues, 10, 100); MultiDimensionCacheStats stats = (MultiDimensionCacheStats) statsHolder.getCacheStats(); @@ -95,7 +97,7 @@ public void testEmptyDimsList() throws Exception { public void testAggregateByAllDimensions() throws Exception { // Aggregating with all dimensions as levels should just give us the same values that were in the original map List dimensionNames = List.of("dim1", "dim2", "dim3", "dim4"); - StatsHolder statsHolder = new StatsHolder(dimensionNames); + StatsHolder statsHolder = new StatsHolder(dimensionNames, storeName); Map> usedDimensionValues = getUsedDimensionValues(statsHolder, 10); Map, CacheStatsCounter> expected = populateStats(statsHolder, usedDimensionValues, 1000, 10); MultiDimensionCacheStats stats = (MultiDimensionCacheStats) statsHolder.getCacheStats(); @@ -112,7 +114,7 @@ public void testAggregateByAllDimensions() throws Exception { public void testAggregateBySomeDimensions() throws Exception { List dimensionNames = List.of("dim1", "dim2", "dim3", "dim4"); - StatsHolder statsHolder = new StatsHolder(dimensionNames); + StatsHolder statsHolder = new StatsHolder(dimensionNames, storeName); Map> usedDimensionValues = getUsedDimensionValues(statsHolder, 10); Map, CacheStatsCounter> expected = populateStats(statsHolder, usedDimensionValues, 1000, 10); MultiDimensionCacheStats stats = (MultiDimensionCacheStats) statsHolder.getCacheStats(); @@ -180,7 +182,7 @@ public void testXContentForLevels() throws Exception { )), new CounterSnapshot(4, 4, 4, 4, 4) ); - MultiDimensionCacheStats stats = new MultiDimensionCacheStats(snapshot, dimensionNames); + MultiDimensionCacheStats stats = new MultiDimensionCacheStats(snapshot, dimensionNames, storeName); XContentBuilder builder = XContentFactory.jsonBuilder(); ToXContent.Params params = ToXContent.EMPTY_PARAMS; diff --git a/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java b/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java index 24542124b92bb..e6f6b9add3730 100644 --- a/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java +++ b/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java @@ -24,6 +24,7 @@ public class StatsHolderTests extends OpenSearchTestCase { // Since StatsHolder does not expose getter methods for aggregating stats, // we test the incrementing functionality in combination with MultiDimensionCacheStats, // in MultiDimensionCacheStatsTests.java. + private final String storeName = "dummy_store"; public void testKeyEquality() throws Exception { List dims1 = List.of( @@ -46,7 +47,7 @@ public void testKeyEquality() throws Exception { public void testReset() throws Exception { List dimensionNames = List.of("dim1", "dim2"); - StatsHolder statsHolder = new StatsHolder(dimensionNames); + StatsHolder statsHolder = new StatsHolder(dimensionNames, storeName); Map> usedDimensionValues = getUsedDimensionValues(statsHolder, 10); Map, CacheStatsCounter> expected = populateStats(statsHolder, usedDimensionValues, 100, 10); @@ -72,7 +73,7 @@ public void testReset() throws Exception { public void testKeyContainsAllDimensions() throws Exception { List dimensionNames = List.of("dim1", "dim2", "dim3"); - StatsHolder statsHolder = new StatsHolder(dimensionNames); + StatsHolder statsHolder = new StatsHolder(dimensionNames, storeName); List dims = List.of(new CacheStatsDimension("dim1", "A"), new CacheStatsDimension("dim2", "B")); @@ -96,7 +97,7 @@ public void testKeyContainsAllDimensions() throws Exception { public void testDropStatsForDimensions() throws Exception { List dimensionNames = List.of("dim1", "dim2", "dim3"); - StatsHolder statsHolder = new StatsHolder(dimensionNames); + StatsHolder statsHolder = new StatsHolder(dimensionNames, storeName); Map> usedDimensionValues = getUsedDimensionValues(statsHolder, 10); populateStats(statsHolder, usedDimensionValues, 100, 10); From 83facb2c8dfaae94f41e226dcc3045c52a7db037 Mon Sep 17 00:00:00 2001 From: Peter Alfonsi Date: Mon, 1 Apr 2024 15:58:12 -0700 Subject: [PATCH 12/17] first chunk of IT Signed-off-by: Peter Alfonsi --- .../indices/IndicesRequestCacheIT.java | 65 +++++++++++++++++++ .../admin/indices/stats/CommonStatsFlags.java | 12 ++-- 2 files changed, 69 insertions(+), 8 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/IndicesRequestCacheIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/IndicesRequestCacheIT.java index 52b4dad553180..1fd8486699ed2 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/IndicesRequestCacheIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/IndicesRequestCacheIT.java @@ -34,17 +34,30 @@ import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; +import org.opensearch.action.admin.cluster.node.stats.NodeStats; +import org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest; +import org.opensearch.action.admin.cluster.node.stats.NodesStatsResponse; import org.opensearch.action.admin.indices.alias.Alias; import org.opensearch.action.admin.indices.forcemerge.ForceMergeResponse; import org.opensearch.action.search.SearchResponse; import org.opensearch.action.search.SearchType; import org.opensearch.client.Client; +import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.common.cache.service.NodeCacheStats; +import org.opensearch.common.cache.stats.CacheStats; import org.opensearch.common.settings.Settings; import org.opensearch.common.time.DateFormatter; import org.opensearch.common.util.FeatureFlags; +import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.common.xcontent.XContentHelper; +import org.opensearch.core.xcontent.MediaTypeRegistry; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.cache.request.RequestCacheStats; import org.opensearch.index.query.QueryBuilders; +import org.opensearch.index.shard.IndexShard; import org.opensearch.search.aggregations.bucket.global.GlobalAggregationBuilder; import org.opensearch.search.aggregations.bucket.histogram.DateHistogramInterval; import org.opensearch.search.aggregations.bucket.histogram.Histogram; @@ -52,6 +65,7 @@ import org.opensearch.test.ParameterizedStaticSettingsOpenSearchIntegTestCase; import org.opensearch.test.hamcrest.OpenSearchAssertions; +import java.io.IOException; import java.time.ZoneId; import java.time.ZoneOffset; import java.time.ZonedDateTime; @@ -59,6 +73,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.List; +import java.util.Map; import static org.opensearch.search.SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING; import static org.opensearch.search.aggregations.AggregationBuilders.dateHistogram; @@ -677,6 +692,34 @@ public void testCacheWithInvalidation() throws Exception { assertCacheState(client, "index", 1, 2); } + public void testCacheStatsAPI() throws Exception { + final String nodeId = internalCluster().startNode(); + Client client = client(nodeId); + assertAcked( + client.admin() + .indices() + .prepareCreate("index") + .setMapping("k", "type=keyword") + .setSettings( + Settings.builder() + .put(IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING.getKey(), true) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + ) + .get() + ); + indexRandom(true, client.prepareIndex("index").setSource("k", "hello")); + ensureSearchable("index"); + SearchResponse resp = client.prepareSearch("index").setRequestCache(true).setQuery(QueryBuilders.termQuery("k", "hello")).get(); + assertSearchResponse(resp); + OpenSearchAssertions.assertAllSuccessful(resp); + assertThat(resp.getHits().getTotalHits().value, equalTo(1L)); + + Map xContentMap = getNodeCacheStatsXContentMap(client, nodeId, List.of()); + int j = 0; + + } + private static void assertCacheState(Client client, String index, long expectedHits, long expectedMisses) { RequestCacheStats requestCacheStats = client.admin() .indices() @@ -694,4 +737,26 @@ private static void assertCacheState(Client client, String index, long expectedH } + private static Map getNodeCacheStatsXContentMap(Client client, String nodeId, List aggregationLevels) throws IOException { + + + NodesStatsResponse nodeStatsResponse = client.admin().cluster() + .prepareNodesStats("data:true") + .addMetric(NodesStatsRequest.Metric.CACHE_STATS.metricName()) + .get(); + Map intermediate = nodeStatsResponse.getNodesMap(); + NodeCacheStats ncs = nodeStatsResponse.getNodes().get(0).getNodeCacheStats(); + + XContentBuilder builder = XContentFactory.jsonBuilder(); + Map paramMap = Map.of("level", String.join(",", aggregationLevels)); + ToXContent.Params params = new ToXContent.MapParams(paramMap); + + builder.startObject(); + ncs.toXContent(builder, params); + builder.endObject(); + + String resultString = builder.toString(); + return XContentHelper.convertToMap(MediaTypeRegistry.JSON.xContent(), resultString, true); + } + } diff --git a/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java b/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java index 41dd39b9be201..cbde1637ea575 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java @@ -65,7 +65,7 @@ public class CommonStatsFlags implements Writeable, Cloneable { private boolean includeOnlyTopIndexingPressureMetrics = false; // Used for metric CACHE_STATS, to determine which caches to report stats for - private EnumSet includeCaches = null; + private EnumSet includeCaches = EnumSet.noneOf(CacheType.class); /** * @param flags flags to set. If no flags are supplied, default flags will be set. @@ -135,7 +135,7 @@ public CommonStatsFlags all() { includeUnloadedSegments = false; includeAllShardIndexingPressureTrackers = false; includeOnlyTopIndexingPressureMetrics = false; - includeCaches = null; + includeCaches = EnumSet.noneOf(CacheType.class); return this; } @@ -151,7 +151,7 @@ public CommonStatsFlags clear() { includeUnloadedSegments = false; includeAllShardIndexingPressureTrackers = false; includeOnlyTopIndexingPressureMetrics = false; - includeCaches = null; + includeCaches = EnumSet.noneOf(CacheType.class); return this; } @@ -223,16 +223,12 @@ public CommonStatsFlags includeOnlyTopIndexingPressureMetrics(boolean includeOnl } public CommonStatsFlags includeCacheType(CacheType cacheType) { - if (includeCaches == null) { - includeCaches = EnumSet.noneOf(CacheType.class); - } includeCaches.add(cacheType); return this; } public CommonStatsFlags includeAllCacheTypes() { - includeCaches = EnumSet.noneOf(CacheType.class); - Collections.addAll(includeCaches, CacheType.values()); + includeCaches = EnumSet.allOf(CacheType.class); return this; } From c385359c87cf0cdd0169bf64f8485e561ca9421a Mon Sep 17 00:00:00 2001 From: Peter Alfonsi Date: Mon, 8 Apr 2024 14:28:22 -0700 Subject: [PATCH 13/17] Finished merge Signed-off-by: Peter Alfonsi --- .../cache/stats/MultiDimensionCacheStats.java | 9 +- .../stats/MultiDimensionCacheStatsTests.java | 156 ++++++------------ .../common/cache/stats/StatsHolderTests.java | 20 +++ 3 files changed, 79 insertions(+), 106 deletions(-) diff --git a/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java b/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java index b10c62c4e51ae..4ce7210e7775e 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java @@ -351,8 +351,6 @@ public boolean equals(Object o) { return false; } return equalsHelper(statsRoot, other.getStatsRoot()); - - //return this.snapshot.equals(other.snapshot) && this.dimensionNames.equals(other.dimensionNames); } private boolean equalsHelper(MDCSDimensionNode thisNode, MDCSDimensionNode otherNode) { @@ -376,9 +374,10 @@ private boolean equalsHelper(MDCSDimensionNode thisNode, MDCSDimensionNode other return allChildrenMatch; } - /*@Override + @Override public int hashCode() { - return Objects.hash(this.snapshot, this.dimensionNames); - }*/ + // Should be sufficient to hash based on the total stats value (found in the root node) + return Objects.hash(statsRoot.stats, dimensionNames); + } } diff --git a/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java b/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java index f8dcda8dc122f..34d30f7ab3552 100644 --- a/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java +++ b/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java @@ -27,6 +27,7 @@ import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; +import java.util.function.BiConsumer; public class MultiDimensionCacheStatsTests extends OpenSearchTestCase { private final String storeName = "dummy_store"; @@ -51,17 +52,6 @@ public void testSerialization() throws Exception { CacheStats deserializedViaCacheStats = CacheStats.readFromStreamWithClassName(is); assertEquals(MultiDimensionCacheStats.class, deserializedViaCacheStats.getClass()); - /*List> pathsInOriginal = new ArrayList<>(); - getAllPathsInTree(stats.getStatsRoot(), new ArrayList<>(), pathsInOriginal); - for (List path : pathsInOriginal) { - MultiDimensionCacheStats.MDCSDimensionNode originalNode = getNode(path, stats.statsRoot); - MultiDimensionCacheStats.MDCSDimensionNode deserializedNode = getNode(path, deserialized.statsRoot); - assertNotNull(deserializedNode); - MultiDimensionCacheStats.MDCSDimensionNode deserializedViaCacheStatsNode = getNode(path, ((MultiDimensionCacheStats) deserializedViaCacheStats).getStatsRoot()); - assertEquals(originalNode.getDimensionValue(), deserializedViaCacheStatsNode.getDimensionValue()); - assertEquals(originalNode.getStatsSnapshot(), deserializedNode.getStatsSnapshot()); - }*/ - assertEquals(stats, deserialized); assertEquals(stats, deserializedViaCacheStats); } @@ -72,14 +62,16 @@ public void testEquals() throws Exception { StatsHolder differentStoreNameStatsHolder = new StatsHolder(dimensionNames, "nonMatchingStoreName"); StatsHolder nonMatchingStatsHolder = new StatsHolder(dimensionNames, storeName); Map> usedDimensionValues = getUsedDimensionValues(statsHolder, 10); - populateStats(statsHolder, usedDimensionValues, 100, 10); + populateStats(List.of(statsHolder, differentStoreNameStatsHolder), usedDimensionValues, 100, 10); populateStats(nonMatchingStatsHolder, usedDimensionValues, 100, 10); MultiDimensionCacheStats stats = (MultiDimensionCacheStats) statsHolder.getCacheStats(); MultiDimensionCacheStats secondStats = (MultiDimensionCacheStats) statsHolder.getCacheStats(); assertEquals(stats, secondStats); - assertNotEquals(stats, nonMatchingStatsHolder); - + MultiDimensionCacheStats nonMatchingStats = (MultiDimensionCacheStats) nonMatchingStatsHolder.getCacheStats(); + assertNotEquals(stats, nonMatchingStats); + MultiDimensionCacheStats differentStoreNameStats = (MultiDimensionCacheStats) differentStoreNameStatsHolder.getCacheStats(); + assertNotEquals(stats, differentStoreNameStats); } public void testAddAndGet() throws Exception { @@ -183,35 +175,17 @@ public void testAggregateBySomeDimensions() throws Exception { } } - /*public void testXContentForLevels() throws Exception { + public void testXContentForLevels() throws Exception { List dimensionNames = List.of("A", "B", "C"); - Map snapshot = Map.of( - new StatsHolder.Key(List.of( - new CacheStatsDimension("A", "A1"), - new CacheStatsDimension("B", "B1"), - new CacheStatsDimension("C", "C1") - )), - new CounterSnapshot(1, 1, 1, 1, 1), - new StatsHolder.Key(List.of( - new CacheStatsDimension("A", "A1"), - new CacheStatsDimension("B", "B1"), - new CacheStatsDimension("C", "C2") - )), - new CounterSnapshot(2, 2, 2, 2, 2), - new StatsHolder.Key(List.of( - new CacheStatsDimension("A", "A1"), - new CacheStatsDimension("B", "B2"), - new CacheStatsDimension("C", "C1") - )), - new CounterSnapshot(3, 3, 3, 3, 3), - new StatsHolder.Key(List.of( - new CacheStatsDimension("A", "A2"), - new CacheStatsDimension("B", "B1"), - new CacheStatsDimension("C", "C3") - )), - new CounterSnapshot(4, 4, 4, 4, 4) - ); - MultiDimensionCacheStats stats = new MultiDimensionCacheStats(snapshot, dimensionNames, storeName); + + StatsHolder statsHolder = new StatsHolder(dimensionNames, storeName); + StatsHolderTests.populateStatsHolderFromStatsValueMap(statsHolder, Map.of( + List.of("A1", "B1", "C1"), new CacheStatsCounter(1, 1, 1, 1, 1), + List.of("A1", "B1", "C2"), new CacheStatsCounter(2, 2, 2, 2, 2), + List.of("A1", "B2", "C1"), new CacheStatsCounter(3, 3, 3, 3, 3), + List.of("A2", "B1", "C3"), new CacheStatsCounter(4, 4, 4, 4, 4) + )); + MultiDimensionCacheStats stats = (MultiDimensionCacheStats) statsHolder.getCacheStats(); XContentBuilder builder = XContentFactory.jsonBuilder(); ToXContent.Params params = ToXContent.EMPTY_PARAMS; @@ -223,23 +197,24 @@ public void testAggregateBySomeDimensions() throws Exception { Map result = XContentHelper.convertToMap(MediaTypeRegistry.JSON.xContent(), resultString, true); Map> fieldNamesMap = Map.of( - CounterSnapshot.Fields.MEMORY_SIZE_IN_BYTES, + CacheStatsCounterSnapshot.Fields.MEMORY_SIZE_IN_BYTES, (counter, value) -> counter.sizeInBytes.inc(value), - CounterSnapshot.Fields.EVICTIONS, + CacheStatsCounterSnapshot.Fields.EVICTIONS, (counter, value) -> counter.evictions.inc(value), - CounterSnapshot.Fields.HIT_COUNT, + CacheStatsCounterSnapshot.Fields.HIT_COUNT, (counter, value) -> counter.hits.inc(value), - CounterSnapshot.Fields.MISS_COUNT, + CacheStatsCounterSnapshot.Fields.MISS_COUNT, (counter, value) -> counter.misses.inc(value), - CounterSnapshot.Fields.ENTRIES, + CacheStatsCounterSnapshot.Fields.ENTRIES, (counter, value) -> counter.entries.inc(value) ); - for (Map.Entry entry : snapshot.entrySet()) { + Map, MultiDimensionCacheStats.MDCSDimensionNode> leafNodes = getAllLeafNodes(stats.getStatsRoot()); + for (Map.Entry, MultiDimensionCacheStats.MDCSDimensionNode> entry : leafNodes.entrySet()) { List xContentKeys = new ArrayList<>(); for (int i = 0; i < dimensionNames.size(); i++) { xContentKeys.add(dimensionNames.get(i)); - xContentKeys.add(entry.getKey().dimensions.get(i).dimensionValue); + xContentKeys.add(entry.getKey().get(i)); } CacheStatsCounter counterFromXContent = new CacheStatsCounter(); @@ -251,10 +226,10 @@ public void testAggregateBySomeDimensions() throws Exception { incrementer.accept(counterFromXContent, valueInXContent); } - CounterSnapshot expected = entry.getValue(); + CacheStatsCounterSnapshot expected = entry.getValue().getStatsSnapshot(); assertEquals(counterFromXContent.snapshot(), expected); } - }*/ + } public static Object getValueFromNestedXContentMap(Map xContentMap, List keys) { Map current = xContentMap; @@ -321,52 +296,47 @@ static Map, CacheStatsCounter> populateStats( Map> usedDimensionValues, int numDistinctValuePairs, int numRepetitionsPerValue + ) throws InterruptedException { + return populateStats(List.of(statsHolder), usedDimensionValues, numDistinctValuePairs, numRepetitionsPerValue); + } + + static Map, CacheStatsCounter> populateStats( + List statsHolders, + Map> usedDimensionValues, + int numDistinctValuePairs, + int numRepetitionsPerValue ) throws InterruptedException { Map, CacheStatsCounter> expected = new ConcurrentHashMap<>(); + for (StatsHolder statsHolder : statsHolders) { + assertEquals(statsHolders.get(0).getDimensionNames(), statsHolder.getDimensionNames()); + } 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(statsHolders.get(0).getDimensionNames(), usedDimensionValues, true, rand)); int finalI = i; threads[i] = new Thread(() -> { - Random threadRand = Randomness.get(); // TODO: This always has the same seed for each thread, causing only 1 set of values + Random threadRand = Randomness.get(); List dimensions = dimensionsForThreads.get(finalI); expected.computeIfAbsent(dimensions, (key) -> new CacheStatsCounter()); - - for (int j = 0; j < numRepetitionsPerValue; j++) { - int numHitIncrements = threadRand.nextInt(10); - for (int k = 0; k < numHitIncrements; k++) { - statsHolder.incrementHits(dimensions); - expected.get(dimensions).hits.inc(); - } - int numMissIncrements = threadRand.nextInt(10); - for (int k = 0; k < numMissIncrements; k++) { - statsHolder.incrementMisses(dimensions); - expected.get(dimensions).misses.inc(); - } - int numEvictionIncrements = threadRand.nextInt(10); - for (int k = 0; k < numEvictionIncrements; k++) { - statsHolder.incrementEvictions(dimensions); - expected.get(dimensions).evictions.inc(); - } - int numMemorySizeIncrements = threadRand.nextInt(10); - for (int k = 0; k < numMemorySizeIncrements; k++) { - long memIncrementAmount = threadRand.nextInt(5000); - statsHolder.incrementSizeInBytes(dimensions, memIncrementAmount); - expected.get(dimensions).sizeInBytes.inc(memIncrementAmount); - } - int numEntryIncrements = threadRand.nextInt(9) + 1; - for (int k = 0; k < numEntryIncrements; k++) { - statsHolder.incrementEntries(dimensions); - expected.get(dimensions).entries.inc(); - } - int numEntryDecrements = threadRand.nextInt(numEntryIncrements); - for (int k = 0; k < numEntryDecrements; k++) { - statsHolder.decrementEntries(dimensions); - expected.get(dimensions).entries.dec(); + for (StatsHolder statsHolder : statsHolders) { + for (int j = 0; j < numRepetitionsPerValue; j++) { + CacheStatsCounter statsToInc = new CacheStatsCounter( + threadRand.nextInt(10), + threadRand.nextInt(10), + threadRand.nextInt(10), + threadRand.nextInt(5000), + threadRand.nextInt(10) + ); + expected.get(dimensions).hits.inc(statsToInc.getHits()); + expected.get(dimensions).misses.inc(statsToInc.getMisses()); + 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)); } } countDownLatch.countDown(); @@ -396,22 +366,6 @@ private static List getRandomDimList( return result; } - private void getAllPathsInTree( - MultiDimensionCacheStats.MDCSDimensionNode currentNode, - List pathToCurrentNode, - List> allPaths - ) { - allPaths.add(pathToCurrentNode); - if (currentNode.getChildren() != null && !currentNode.getChildren().isEmpty()) { - // not a leaf node - for (MultiDimensionCacheStats.MDCSDimensionNode child : currentNode.getChildren().values()) { - List pathToChild = new ArrayList<>(pathToCurrentNode); - pathToChild.add(child.getDimensionValue()); - getAllPathsInTree(child, pathToChild, allPaths); - } - } - } - private MultiDimensionCacheStats.MDCSDimensionNode getNode( List dimensionValues, MultiDimensionCacheStats.MDCSDimensionNode root diff --git a/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java b/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java index a31c56bf2dda7..dba3846fa4630 100644 --- a/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java +++ b/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java @@ -152,4 +152,24 @@ static DimensionNode getNode(List dimensionValues, DimensionNode root) { } return current; } + + static void populateStatsHolderFromStatsValueMap(StatsHolder statsHolder, Map, CacheStatsCounter> statsMap) { + for (Map.Entry, CacheStatsCounter> entry : statsMap.entrySet()) { + CacheStatsCounter stats = entry.getValue(); + List dims = entry.getKey(); + for (int i = 0; i < stats.getHits(); i++) { + statsHolder.incrementHits(dims); + } + for (int i = 0; i < stats.getMisses(); i++) { + statsHolder.incrementMisses(dims); + } + for (int i = 0; i < stats.getEvictions(); i++) { + statsHolder.incrementEvictions(dims); + } + statsHolder.incrementSizeInBytes(dims, stats.getSizeInBytes()); + for (int i = 0; i < stats.getEntries(); i++) { + statsHolder.incrementEntries(dims); + } + } + } } From 96c9493658dc016559068a0186cb7f155f4e3f4d Mon Sep 17 00:00:00 2001 From: Peter Alfonsi Date: Tue, 9 Apr 2024 12:24:37 -0700 Subject: [PATCH 14/17] First draft of single-StatsHolder setup Signed-off-by: Peter Alfonsi --- .../common/tier/TieredSpilloverCache.java | 86 +++-- .../tier/TieredSpilloverCacheStats.java | 156 --------- .../tier/TieredSpilloverCacheStatsTests.java | 298 ------------------ .../tier/TieredSpilloverCacheTests.java | 1 + 4 files changed, 57 insertions(+), 484 deletions(-) delete mode 100644 modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStats.java delete mode 100644 modules/cache-common/src/test/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsTests.java 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 e3d0c6bbf574d..02b3050ced74c 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 @@ -19,6 +19,7 @@ import org.opensearch.common.cache.RemovalReason; import org.opensearch.common.cache.policy.CachedQueryResult; import org.opensearch.common.cache.stats.CacheStats; +import org.opensearch.common.cache.stats.DimensionNode; import org.opensearch.common.cache.stats.StatsHolder; import org.opensearch.common.cache.store.config.CacheConfig; import org.opensearch.common.collect.Tuple; @@ -64,8 +65,9 @@ public class TieredSpilloverCache implements ICache { // 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 StatsHolder heapStats; - private final StatsHolder diskStats; + //private final StatsHolder heapStats; + //private final StatsHolder diskStats; + private final StatsHolder statsHolder; private ToLongBiFunction, V> weigher; private final List dimensionNames; @@ -76,9 +78,14 @@ public class TieredSpilloverCache implements ICache { * Maintains caching tiers in ascending order of cache latency. */ private final List> cacheList; - private final List, StatsHolder>> cacheAndStatsList; + private final List, String>> cacheAndTierValueList; private final List> policies; + // Common values used for tier dimension + public static final String TIER_DIMENSION_NAME = "tier"; + public static final String TIER_DIMENSION_VALUE_ON_HEAP = "on_heap"; + public static final String TIER_DIMENSION_VALUE_DISK = "disk"; + TieredSpilloverCache(Builder builder) { Objects.requireNonNull(builder.onHeapCacheFactory, "onHeap cache builder can't be null"); Objects.requireNonNull(builder.diskCacheFactory, "disk cache builder can't be null"); @@ -116,12 +123,14 @@ public class TieredSpilloverCache implements ICache { this.cacheList = Arrays.asList(onHeapCache, diskCache); this.dimensionNames = builder.cacheConfig.getDimensionNames(); - this.heapStats = new StatsHolder(dimensionNames); - this.diskStats = new StatsHolder(dimensionNames); - this.cacheAndStatsList = List.of( - new Tuple<>(onHeapCache, heapStats), - new Tuple<>(diskCache, diskStats) + //this.heapStats = new StatsHolder(dimensionNames); + //this.diskStats = new StatsHolder(dimensionNames); + this.cacheAndTierValueList = List.of( + new Tuple<>(onHeapCache, TIER_DIMENSION_VALUE_ON_HEAP), + new Tuple<>(diskCache, TIER_DIMENSION_VALUE_DISK) ); + // Pass "tier" as the innermost dimension name, on top of whatever dimensions are specified for the cache as a whole + this.statsHolder = new StatsHolder(addTierValueToDimensionValues(dimensionNames, TIER_DIMENSION_NAME)); this.policies = builder.policies; // Will never be null; builder initializes it to an empty list } @@ -144,7 +153,7 @@ public V get(ICacheKey key) { public void put(ICacheKey key, V value) { try (ReleasableLock ignore = writeLock.acquire()) { onHeapCache.put(key, value); - updateStatsOnPut(heapStats, key, value); + updateStatsOnPut(TIER_DIMENSION_VALUE_ON_HEAP, key, value); } } @@ -161,7 +170,7 @@ public V computeIfAbsent(ICacheKey key, LoadAwareCacheLoader, V> value = onHeapCache.computeIfAbsent(key, loader); if (loader.isLoaded()) { // The value was just computed and added to the cache - updateStatsOnPut(heapStats, key, value); + updateStatsOnPut(TIER_DIMENSION_VALUE_ON_HEAP, key, value); } } return value; @@ -176,8 +185,13 @@ public void invalidate(ICacheKey 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. try (ReleasableLock ignore = writeLock.acquire()) { - for (ICache cache : cacheList) { - cache.invalidate(key); + for (Tuple, String> pair : cacheAndTierValueList) { + //cache.invalidate(key); + if (key.getDropStatsForDimensions()) { + List dimensionValues = addTierValueToDimensionValues(key.dimensions, pair.v2()); + statsHolder.removeDimensions(dimensionValues); + } + pair.v1().invalidate(key); } } } @@ -189,8 +203,7 @@ public void invalidateAll() { cache.invalidateAll(); } } - heapStats.reset(); - diskStats.reset(); + statsHolder.reset(); } /** @@ -205,7 +218,7 @@ public Iterable> keys() { @Override public long count() { - return heapStats.count() + diskStats.count(); + return statsHolder.count(); } @Override @@ -226,21 +239,23 @@ public void close() throws IOException { @Override public CacheStats stats() { - // TODO: Just reuse MDCS //return new TieredSpilloverCacheStats(heapStats.createSnapshot(), diskStats.createSnapshot(), dimensionNames); - return null; + return statsHolder.getCacheStats(); } private Function, V> getValueFromTieredCache() { return key -> { try (ReleasableLock ignore = readLock.acquire()) { - for (Tuple, StatsHolder> pair : cacheAndStatsList) { + for (Tuple, String> pair : cacheAndTierValueList) { V value = pair.v1().get(key); + List dimensionValues = addTierValueToDimensionValues(key.dimensions, pair.v2()); // Get the tier value corresponding to this cache if (value != null) { - pair.v2().incrementHits(key.dimensions); + //pair.v2().incrementHits(key.dimensions); + statsHolder.incrementHits(dimensionValues); return value; } else { - pair.v2().incrementMisses(key.dimensions); + statsHolder.incrementMisses(dimensionValues); + //pair.v2().incrementMisses(key.dimensions); } } } @@ -257,7 +272,7 @@ void handleRemovalFromHeapTier(RemovalNotification, V> notification try (ReleasableLock ignore = writeLock.acquire()) { if (evaluatePolicies(notification.getValue())) { diskCache.put(key, notification.getValue()); // spill over to the disk tier and increment its stats - updateStatsOnPut(diskStats, key, notification.getValue()); + updateStatsOnPut(TIER_DIMENSION_VALUE_DISK, key, notification.getValue()); } } wasEvicted = true; @@ -267,7 +282,7 @@ void handleRemovalFromHeapTier(RemovalNotification, V> notification // If the removal was for another reason, send this notification to the TSC's removal listener, as the value is leaving the TSC entirely removalListener.onRemoval(notification); } - updateStatsOnRemoval(heapStats, wasEvicted, key, notification.getValue()); + updateStatsOnRemoval(TIER_DIMENSION_VALUE_ON_HEAP, wasEvicted, key, notification.getValue()); } void handleRemovalFromDiskTier(RemovalNotification, V> notification) { @@ -279,20 +294,22 @@ void handleRemovalFromDiskTier(RemovalNotification, V> notification || RemovalReason.CAPACITY.equals(notification.getRemovalReason())) { wasEvicted = true; } - updateStatsOnRemoval(diskStats, wasEvicted, notification.getKey(), notification.getValue()); + updateStatsOnRemoval(TIER_DIMENSION_VALUE_DISK, wasEvicted, notification.getKey(), notification.getValue()); } - void updateStatsOnRemoval(StatsHolder statsHolder, boolean wasEvicted, ICacheKey key, V value) { + void updateStatsOnRemoval(String removedFromTierValue, boolean wasEvicted, ICacheKey key, V value) { + List dimensionValues = addTierValueToDimensionValues(key.dimensions, removedFromTierValue); if (wasEvicted) { - statsHolder.incrementEvictions(key.dimensions); + statsHolder.incrementEvictions(dimensionValues); } - statsHolder.decrementEntries(key.dimensions); - statsHolder.decrementSizeInBytes(key.dimensions, weigher.applyAsLong(key, value)); + statsHolder.decrementEntries(dimensionValues); + statsHolder.decrementSizeInBytes(dimensionValues, weigher.applyAsLong(key, value)); } - void updateStatsOnPut(StatsHolder statsHolder, ICacheKey key, V value) { - statsHolder.incrementEntries(key.dimensions); - statsHolder.incrementSizeInBytes(key.dimensions, weigher.applyAsLong(key, value)); + void updateStatsOnPut(String destinationTierValue, ICacheKey key, V value) { + List dimensionValues = addTierValueToDimensionValues(key.dimensions, destinationTierValue); + statsHolder.incrementEntries(dimensionValues); + statsHolder.incrementSizeInBytes(dimensionValues, weigher.applyAsLong(key, value)); } boolean evaluatePolicies(V value) { @@ -304,6 +321,15 @@ boolean evaluatePolicies(V value) { return true; } + /** + * Add tierValue to the end of a copy of the initial dimension values. + */ + private List addTierValueToDimensionValues(List initialDimensions, String tierValue) { + List result = new ArrayList<>(initialDimensions); + result.add(tierValue); + return result; + } + /** * A class which receives removal events from the heap tier. */ diff --git a/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStats.java b/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStats.java deleted file mode 100644 index 350718b693bc4..0000000000000 --- a/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStats.java +++ /dev/null @@ -1,156 +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.cache.common.tier; - -import org.opensearch.common.cache.stats.CacheStats; -import org.opensearch.common.cache.stats.MultiDimensionCacheStats; -import org.opensearch.common.cache.stats.StatsHolder; -import org.opensearch.core.common.io.stream.StreamInput; -import org.opensearch.core.common.io.stream.StreamOutput; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.TreeMap; - - -/** - * A CacheStats implementation for TieredSpilloverCache, which keeps track of the stats for its tiers. - */ -public class TieredSpilloverCacheStats { // implements CacheStats - // Pkg-private for testing - /*final MultiDimensionCacheStats heapStats; - final MultiDimensionCacheStats diskStats; - - public static final String TIER_DIMENSION_NAME = "tier"; - public static final String TIER_DIMENSION_VALUE_ON_HEAP = "on_heap"; - public static final String TIER_DIMENSION_VALUE_DISK = "disk"; - public static final List HEAP_DIMS = List.of(new CacheStatsDimension(TIER_DIMENSION_NAME, TIER_DIMENSION_VALUE_ON_HEAP)); - public static final List DISK_DIMS = List.of(new CacheStatsDimension(TIER_DIMENSION_NAME, TIER_DIMENSION_VALUE_DISK)); - - public TieredSpilloverCacheStats( - Map heapSnapshot, - Map diskSnapshot, - List dimensionNames) { - this.heapStats = new MultiDimensionCacheStats(heapSnapshot, dimensionNames); - this.diskStats = new MultiDimensionCacheStats(diskSnapshot, dimensionNames); - } - - public TieredSpilloverCacheStats(StreamInput in) throws IOException { - this.heapStats = new MultiDimensionCacheStats(in); - this.diskStats = new MultiDimensionCacheStats(in); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - heapStats.writeTo(out); - diskStats.writeTo(out); - } - - @Override - public CacheStatsResponse.Snapshot getTotalStats() { - return combineTierResponses(heapStats.getTotalStats(), diskStats.getTotalStats()); - } - - public TreeMap aggregateByLevels(List levels) { - TreeMap result = new TreeMap<>(new MultiDimensionCacheStats.KeyComparator()); - if (levels.contains(TIER_DIMENSION_NAME)) { - // Aggregate by tier. Get the aggregations from each MultiDimensionCacheStats, and combine them into a single - // TreeMap, adding the tier dimension to each TreeMap key. - List noTierLevels = new ArrayList<>(levels); // levels might be immutable - noTierLevels.remove(TIER_DIMENSION_NAME); - TreeMap heapAgg = heapStats.aggregateByLevels(noTierLevels); - TreeMap diskAgg = diskStats.aggregateByLevels(noTierLevels); - - addKeysWithTierDimension(result, heapAgg, TIER_DIMENSION_VALUE_ON_HEAP); - addKeysWithTierDimension(result, diskAgg, TIER_DIMENSION_VALUE_DISK); - } - else { - // Don't aggregate by tier. Get aggregations from each MultiDimensionCacheStats. Combine them using combineTierResponses - // if both aggregations share a key. Otherwise, add directly from the only one which has the key. - TreeMap heapAgg = heapStats.aggregateByLevels(levels); - TreeMap diskAgg = diskStats.aggregateByLevels(levels); - - for (Map.Entry entry : heapAgg.entrySet()) { - CacheStatsResponse.Snapshot heapValue = entry.getValue(); - CacheStatsResponse.Snapshot diskValue = diskAgg.get(entry.getKey()); - StatsHolder.Key key = entry.getKey(); - if (diskValue == null) { - // Only the heap agg has this particular combination of values, add directly to result - result.put(key, heapValue); - } else { - // Both aggregations have this combination, combine them before adding and then remove from diskAgg to avoid double-counting - CacheStatsResponse.Snapshot combined = combineTierResponses(heapValue, diskValue); - result.put(key, combined); - diskAgg.remove(key); - } - } - // The remaining keys are only present in diskAgg - result.putAll(diskAgg); - } - return result; - } - - // Add all keys in originalAggregation to result, but first add tierDimName to the end of the key. - private void addKeysWithTierDimension(TreeMap result, - TreeMap originalAggregation, - String tierDimName) { - for (Map.Entry entry : originalAggregation.entrySet()) { - List newDimensions = new ArrayList<>(entry.getKey().getDimensionValues()); - newDimensions.add(tierDimName); // Tier dimension is at the end as it's the innermost dimension in API responses - StatsHolder.Key newKey = new StatsHolder.Key(newDimensions); - result.put(newKey, entry.getValue()); - } - } - - // pkg-private for testing - static CacheStatsResponse.Snapshot combineTierResponses(CacheStatsResponse.Snapshot heap, CacheStatsResponse.Snapshot disk) { - return new CacheStatsResponse.Snapshot( - heap.getHits() + disk.getHits(), - disk.getMisses(), - disk.getEvictions(), - heap.getSizeInBytes() + disk.getSizeInBytes(), - heap.getEntries() + disk.getEntries() - ); - } - - @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(); - } - - CacheStatsResponse.Snapshot getTotalHeapStats() { - return heapStats.getTotalStats(); - } - - CacheStatsResponse.Snapshot getTotalDiskStats() { - return diskStats.getTotalStats(); - }*/ -} diff --git a/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsTests.java b/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsTests.java deleted file mode 100644 index 089980e2fe948..0000000000000 --- a/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsTests.java +++ /dev/null @@ -1,298 +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.cache.common.tier; - -import org.opensearch.common.Randomness; -import org.opensearch.common.cache.Cache; -import org.opensearch.common.cache.ICacheKey; -import org.opensearch.common.cache.stats.CacheStats; -import org.opensearch.common.cache.stats.StatsHolder; -import org.opensearch.common.io.stream.BytesStreamOutput; -import org.opensearch.common.settings.Settings; -import org.opensearch.core.common.bytes.BytesReference; -import org.opensearch.core.common.io.stream.BytesStreamInput; -import org.opensearch.test.OpenSearchTestCase; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.Set; -import java.util.TreeMap; -import java.util.UUID; - -/*import static org.opensearch.cache.common.tier.TieredSpilloverCacheStats.TIER_DIMENSION_NAME; -import static org.opensearch.cache.common.tier.TieredSpilloverCacheStats.TIER_DIMENSION_VALUE_ON_HEAP; -import static org.opensearch.cache.common.tier.TieredSpilloverCacheStats.TIER_DIMENSION_VALUE_DISK; -import static org.opensearch.cache.common.tier.TieredSpilloverCacheStats.combineTierResponses;*/ - -public class TieredSpilloverCacheStatsTests extends OpenSearchTestCase { - /*private static List dimensionNames = List.of("dim1", "dim2", "dim3"); - private static List tierNames = List.of(TIER_DIMENSION_VALUE_ON_HEAP, TIER_DIMENSION_VALUE_DISK); - public void testGets() throws Exception { - StatsHolder heapStats = new StatsHolder(dimensionNames); - StatsHolder diskStats = new StatsHolder(dimensionNames); - Map> usedDimensionValues = getUsedDimensionValues(heapStats, 10); - Map, CacheStatsResponse>> expected = populateStats(heapStats, diskStats, usedDimensionValues, 100, 2); - TieredSpilloverCacheStats stats = new TieredSpilloverCacheStats(heapStats.createSnapshot(), diskStats.createSnapshot(), dimensionNames); - - CacheStatsResponse heapTotalStats = totalSumExpected(expected.get(TIER_DIMENSION_VALUE_ON_HEAP)); - CacheStatsResponse diskTotalStats = totalSumExpected(expected.get(TIER_DIMENSION_VALUE_DISK)); - CacheStatsResponse.Snapshot totalTSCStats = TieredSpilloverCacheStats.combineTierResponses( - heapTotalStats.snapshot(), diskTotalStats.snapshot()); - - // test total gets - assertEquals(totalTSCStats, stats.getTotalStats()); - - assertEquals(totalTSCStats.getHits(), stats.getTotalHits()); - assertEquals(totalTSCStats.getMisses(), stats.getTotalMisses()); - assertEquals(totalTSCStats.getEvictions(), stats.getTotalEvictions()); - assertEquals(totalTSCStats.getSizeInBytes(), stats.getTotalSizeInBytes()); - assertEquals(totalTSCStats.getEntries(), stats.getTotalEntries()); - - assertEquals(heapTotalStats.snapshot(), stats.getTotalHeapStats()); - assertEquals(diskTotalStats.snapshot(), stats.getTotalDiskStats()); - } - - public void testEmptyDimensionNames() throws Exception { - StatsHolder heapStats = new StatsHolder(List.of()); - StatsHolder diskStats = new StatsHolder(List.of()); - - Map> usedDimensionValues = getUsedDimensionValues(heapStats, 10); - Map, CacheStatsResponse>> expected = populateStats(heapStats, diskStats, usedDimensionValues, 10, 10); - TieredSpilloverCacheStats stats = new TieredSpilloverCacheStats(heapStats.createSnapshot(), diskStats.createSnapshot(), List.of()); - - CacheStatsResponse heapTotalStats = totalSumExpected(expected.get(TIER_DIMENSION_VALUE_ON_HEAP)); - CacheStatsResponse diskTotalStats = totalSumExpected(expected.get(TIER_DIMENSION_VALUE_DISK)); - CacheStatsResponse.Snapshot totalTSCStats = TieredSpilloverCacheStats.combineTierResponses(heapTotalStats.snapshot(), diskTotalStats.snapshot()); - - assertEquals(totalTSCStats, stats.getTotalStats()); - } - - public void testSerialization() throws Exception { - StatsHolder heapStats = new StatsHolder(dimensionNames); - StatsHolder diskStats = new StatsHolder(dimensionNames); - Map> usedDimensionValues = getUsedDimensionValues(heapStats, 10); - Map, CacheStatsResponse>> expected = populateStats(heapStats, diskStats, usedDimensionValues, 100, 2); - TieredSpilloverCacheStats stats = new TieredSpilloverCacheStats(heapStats.createSnapshot(), diskStats.createSnapshot(), dimensionNames); - - BytesStreamOutput os = new BytesStreamOutput(); - stats.writeTo(os); - BytesStreamInput is = new BytesStreamInput(BytesReference.toBytes(os.bytes())); - TieredSpilloverCacheStats deserialized = new TieredSpilloverCacheStats(is); - - assertEquals(stats.heapStats.aggregateByLevels(dimensionNames), deserialized.heapStats.aggregateByLevels(dimensionNames)); - assertEquals(stats.diskStats.aggregateByLevels(dimensionNames), deserialized.diskStats.aggregateByLevels(dimensionNames)); - } - - public void testCombineTierResponses() throws Exception { - CacheStatsResponse.Snapshot heapResponse = new CacheStatsResponse.Snapshot(1,2,3,4,5); - CacheStatsResponse.Snapshot diskResponse = new CacheStatsResponse.Snapshot(2,3,4,5,6); - CacheStatsResponse.Snapshot tscResponse = TieredSpilloverCacheStats.combineTierResponses(heapResponse, diskResponse); - assertEquals(new CacheStatsResponse.Snapshot(3, 3, 4, 9, 11), tscResponse); - } - - public void testAggregationSomeLevelsWithoutTier() throws Exception { - StatsHolder heapStats = new StatsHolder(dimensionNames); - StatsHolder diskStats = new StatsHolder(dimensionNames); - Map> usedDimensionValues = getUsedDimensionValues(heapStats, 10); - Map, CacheStatsResponse>> expected = populateStats(heapStats, diskStats, usedDimensionValues, 100, 2); - TieredSpilloverCacheStats stats = new TieredSpilloverCacheStats(heapStats.createSnapshot(), diskStats.createSnapshot(), dimensionNames); - - for (int i = 0; i < (1 << dimensionNames.size()); i++) { - // Test each combination of possible levels - List levels = new ArrayList<>(); - for (int nameIndex = 0; nameIndex < dimensionNames.size(); nameIndex++) { - if ((i & (1 << nameIndex)) != 0) { - levels.add(dimensionNames.get(nameIndex)); - } - } - if (levels.size() == 0) { - assertThrows(IllegalArgumentException.class, () -> stats.aggregateByLevels(levels)); - } - else { - Map aggregated = stats.aggregateByLevels(levels); - for (Map.Entry aggregatedEntry : aggregated.entrySet()) { - StatsHolder.Key aggregatedKey = aggregatedEntry.getKey(); - Map expectedResponseForTierMap = new HashMap<>(); - for (String tier : new String[]{TIER_DIMENSION_VALUE_ON_HEAP, TIER_DIMENSION_VALUE_DISK}) { - CacheStatsResponse expectedResponseForTier = new CacheStatsResponse(); - for (Set expectedDims : expected.get(tier).keySet()) { - List orderedDimValues = StatsHolder.getOrderedDimensionValues( - new ArrayList<>(expectedDims), - dimensionNames - ); - if (orderedDimValues.containsAll(aggregatedKey.getDimensionValues())) { - expectedResponseForTier.add(expected.get(tier).get(expectedDims)); - } - } - if (expectedResponseForTier.equals(new CacheStatsResponse())) { - expectedResponseForTier = null; // If it's all 0, there were no keys - } - expectedResponseForTierMap.put(tier, expectedResponseForTier); - } - CacheStatsResponse expectedHeapResponse = expectedResponseForTierMap.get(TIER_DIMENSION_VALUE_ON_HEAP); - CacheStatsResponse expectedDiskResponse = expectedResponseForTierMap.get(TIER_DIMENSION_VALUE_DISK); - if (expectedHeapResponse != null && expectedDiskResponse != null) { - assertEquals(combineTierResponses(expectedHeapResponse.snapshot(), expectedDiskResponse.snapshot()), aggregatedEntry.getValue()); - } else if (expectedHeapResponse != null) { - assertEquals(expectedHeapResponse.snapshot(), aggregatedEntry.getValue()); - } else { - assertEquals(expectedDiskResponse.snapshot(), aggregatedEntry.getValue()); - } - } - } - } - } - - public void testAggregationSomeLevelsWithTier() throws Exception { - StatsHolder heapStats = new StatsHolder(dimensionNames); - StatsHolder diskStats = new StatsHolder(dimensionNames); - Map> usedDimensionValues = getUsedDimensionValues(heapStats, 10); - Map, CacheStatsResponse>> expected = populateStats(heapStats, diskStats, usedDimensionValues, 100, 2); - TieredSpilloverCacheStats stats = new TieredSpilloverCacheStats(heapStats.createSnapshot(), diskStats.createSnapshot(), dimensionNames); - - for (int i = 0; i < (1 << dimensionNames.size()); i++) { - // Test each combination of possible levels - List levels = new ArrayList<>(); - for (int nameIndex = 0; nameIndex < dimensionNames.size(); nameIndex++) { - if ((i & (1 << nameIndex)) != 0) { - levels.add(dimensionNames.get(nameIndex)); - } - } - levels.add(TIER_DIMENSION_NAME); - if (levels.size() == 1) { - assertThrows(IllegalArgumentException.class, () -> stats.aggregateByLevels(levels)); - } - else { - Map aggregated = stats.aggregateByLevels(levels); - for (Map.Entry aggregatedEntry : aggregated.entrySet()) { - StatsHolder.Key aggregatedKey = aggregatedEntry.getKey(); - String aggregatedKeyTier = aggregatedKey.getDimensionValues().get(aggregatedKey.getDimensionValues().size()-1); - CacheStatsResponse expectedResponse = new CacheStatsResponse(); - for (Set expectedDims : expected.get(aggregatedKeyTier).keySet()) { - List orderedDimValues = StatsHolder.getOrderedDimensionValues( - new ArrayList<>(expectedDims), - dimensionNames - ); - orderedDimValues.add(aggregatedKeyTier); - if (orderedDimValues.containsAll(aggregatedKey.getDimensionValues())) { - expectedResponse.add(expected.get(aggregatedKeyTier).get(expectedDims)); - } - } - assertEquals(expectedResponse.snapshot(), aggregatedEntry.getValue()); - } - } - } - } - - private CacheStatsResponse totalSumExpected(Map, CacheStatsResponse> expected) { - CacheStatsResponse result = new CacheStatsResponse(); - for (Set key : expected.keySet()) { - result.add(expected.get(key)); - } - return result; - } - - // Fill the tier stats and return a nested map from tier type and dimensions -> expected response - // Modified from MultiDimensionCacheStatsTests - we can't import it without adding a dependency on server.test module. - private Map, CacheStatsResponse>> populateStats(StatsHolder heapStats, StatsHolder diskStats, Map> usedDimensionValues, int numDistinctValuePairs, int numRepetitionsPerValue) { - Map, CacheStatsResponse>> expected = new HashMap<>(); - expected.put(TIER_DIMENSION_VALUE_ON_HEAP, new HashMap<>()); - expected.put(TIER_DIMENSION_VALUE_DISK, new HashMap<>()); - - Random rand = Randomness.get(); - Map statsHolderMap = Map.of(tierNames.get(0), heapStats, tierNames.get(1), diskStats); - for (String tier : tierNames) { - for (int i = 0; i < numDistinctValuePairs; i++) { - StatsHolder stats = statsHolderMap.get(tier); - List dimensions = getRandomDimList(stats.getDimensionNames(), usedDimensionValues, true, rand); - Set dimSet = new HashSet<>(dimensions); - Map, CacheStatsResponse> tierExpected = expected.get(tier); - if (tierExpected.get(dimSet) == null) { - tierExpected.put(dimSet, new CacheStatsResponse()); - } - ICacheKey dummyKey = getDummyKey(dimensions); - for (int j = 0; j < numRepetitionsPerValue; j++) { - - int numHitIncrements = rand.nextInt(10); - for (int k = 0; k < numHitIncrements; k++) { - stats.incrementHits(dummyKey); - tierExpected.get(new HashSet<>(dimensions)).hits.inc(); - } - - int numMissIncrements = rand.nextInt(10); - for (int k = 0; k < numMissIncrements; k++) { - stats.incrementMisses(dummyKey); - tierExpected.get(new HashSet<>(dimensions)).misses.inc(); - } - - int numEvictionIncrements = rand.nextInt(10); - for (int k = 0; k < numEvictionIncrements; k++) { - stats.incrementEvictions(dummyKey); - tierExpected.get(new HashSet<>(dimensions)).evictions.inc(); - } - - int numMemorySizeIncrements = rand.nextInt(10); - for (int k = 0; k < numMemorySizeIncrements; k++) { - long memIncrementAmount = rand.nextInt(5000); - stats.incrementSizeInBytes(dummyKey, memIncrementAmount); - tierExpected.get(new HashSet<>(dimensions)).sizeInBytes.inc(memIncrementAmount); - } - - int numEntryIncrements = rand.nextInt(9) + 1; - for (int k = 0; k < numEntryIncrements; k++) { - stats.incrementEntries(dummyKey); - tierExpected.get(new HashSet<>(dimensions)).entries.inc(); - } - - int numEntryDecrements = rand.nextInt(numEntryIncrements); - for (int k = 0; k < numEntryDecrements; k++) { - stats.decrementEntries(dummyKey); - tierExpected.get(new HashSet<>(dimensions)).entries.dec(); - } - } - - } - } - return expected; - } - - // Duplicated below functions from MultiDimensionCacheStatsTests. We can't import them without adding a dependency on server.test for this module. - - private List getRandomDimList(List dimensionNames, Map> usedDimensionValues, boolean pickValueForAllDims, Random rand) { - List result = new ArrayList<>(); - for (String dimName : dimensionNames) { - if (pickValueForAllDims || rand.nextBoolean()) { // if pickValueForAllDims, always pick a value for each dimension, otherwise do so 50% of the time - int index = between(0, usedDimensionValues.get(dimName).size() - 1); - result.add(new CacheStatsDimension(dimName, usedDimensionValues.get(dimName).get(index))); - } - } - return result; - } - private Map> getUsedDimensionValues(StatsHolder stats, int numValuesPerDim) { - Map> usedDimensionValues = new HashMap<>(); - for (int i = 0; i < stats.getDimensionNames().size(); i++) { - List values = new ArrayList<>(); - for (int j = 0; j < numValuesPerDim; j++) { - values.add(UUID.randomUUID().toString()); - } - usedDimensionValues.put(stats.getDimensionNames().get(i), values); - } - return usedDimensionValues; - } - - private static ICacheKey getDummyKey(List dims) { - return new ICacheKey<>(null, dims); - }*/ - -} 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 98628203761a7..1637e0da90718 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 @@ -16,6 +16,7 @@ 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.CacheStatsCounterSnapshot; import org.opensearch.common.cache.store.OpenSearchOnHeapCache; import org.opensearch.common.cache.store.config.CacheConfig; import org.opensearch.common.cache.store.settings.OpenSearchOnHeapCacheSettings; From 7fec02dd3853dd78d8e6ce55451bbf18a4c2ef71 Mon Sep 17 00:00:00 2001 From: Peter Alfonsi Date: Tue, 9 Apr 2024 14:20:52 -0700 Subject: [PATCH 15/17] Updated tests to check stats values Signed-off-by: Peter Alfonsi --- .../common/tier/TieredSpilloverCache.java | 15 +- .../cache/common/tier/MockDiskCache.java | 1 + .../tier/TieredSpilloverCacheTests.java | 142 ++++++++++++++++-- .../stats/CacheStatsCounterSnapshot.java | 14 +- 4 files changed, 142 insertions(+), 30 deletions(-) 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 02b3050ced74c..1829703dd0701 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 @@ -19,7 +19,6 @@ import org.opensearch.common.cache.RemovalReason; import org.opensearch.common.cache.policy.CachedQueryResult; import org.opensearch.common.cache.stats.CacheStats; -import org.opensearch.common.cache.stats.DimensionNode; import org.opensearch.common.cache.stats.StatsHolder; import org.opensearch.common.cache.store.config.CacheConfig; import org.opensearch.common.collect.Tuple; @@ -65,10 +64,7 @@ public class TieredSpilloverCache implements ICache { // 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 StatsHolder heapStats; - //private final StatsHolder diskStats; private final StatsHolder statsHolder; - private ToLongBiFunction, V> weigher; private final List dimensionNames; ReadWriteLock readWriteLock = new ReentrantReadWriteLock(); @@ -123,14 +119,12 @@ public class TieredSpilloverCache implements ICache { this.cacheList = Arrays.asList(onHeapCache, diskCache); this.dimensionNames = builder.cacheConfig.getDimensionNames(); - //this.heapStats = new StatsHolder(dimensionNames); - //this.diskStats = new StatsHolder(dimensionNames); this.cacheAndTierValueList = List.of( new Tuple<>(onHeapCache, TIER_DIMENSION_VALUE_ON_HEAP), new Tuple<>(diskCache, TIER_DIMENSION_VALUE_DISK) ); - // Pass "tier" as the innermost dimension name, on top of whatever dimensions are specified for the cache as a whole - this.statsHolder = new StatsHolder(addTierValueToDimensionValues(dimensionNames, TIER_DIMENSION_NAME)); + // Pass "tier" as the innermost dimension name, in addition to whatever dimensions are specified for the cache as a whole + this.statsHolder = new StatsHolder(addTierValueToDimensionValues(dimensionNames, TIER_DIMENSION_NAME), TieredSpilloverCacheFactory.TIERED_SPILLOVER_CACHE_NAME); this.policies = builder.policies; // Will never be null; builder initializes it to an empty list } @@ -159,7 +153,6 @@ public void put(ICacheKey key, V value) { @Override public V computeIfAbsent(ICacheKey key, LoadAwareCacheLoader, V> loader) throws Exception { - V cacheValue = getValueFromTieredCache().apply(key); if (cacheValue == null) { // Add the value to the onHeap cache. We are calling computeIfAbsent which does another get inside. @@ -186,7 +179,6 @@ public void invalidate(ICacheKey key) { // We don't update stats here, as this is handled by the removal listeners for the tiers. try (ReleasableLock ignore = writeLock.acquire()) { for (Tuple, String> pair : cacheAndTierValueList) { - //cache.invalidate(key); if (key.getDropStatsForDimensions()) { List dimensionValues = addTierValueToDimensionValues(key.dimensions, pair.v2()); statsHolder.removeDimensions(dimensionValues); @@ -239,7 +231,6 @@ public void close() throws IOException { @Override public CacheStats stats() { - //return new TieredSpilloverCacheStats(heapStats.createSnapshot(), diskStats.createSnapshot(), dimensionNames); return statsHolder.getCacheStats(); } @@ -250,12 +241,10 @@ private Function, V> getValueFromTieredCache() { V value = pair.v1().get(key); List dimensionValues = addTierValueToDimensionValues(key.dimensions, pair.v2()); // Get the tier value corresponding to this cache if (value != null) { - //pair.v2().incrementHits(key.dimensions); statsHolder.incrementHits(dimensionValues); return value; } else { statsHolder.incrementMisses(dimensionValues); - //pair.v2().incrementMisses(key.dimensions); } } } 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..2d3fe60682db1 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 @@ -71,6 +71,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); } 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 1637e0da90718..46018b8ee8ab1 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 @@ -16,6 +16,7 @@ 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.CacheStats; import org.opensearch.common.cache.stats.CacheStatsCounterSnapshot; import org.opensearch.common.cache.store.OpenSearchOnHeapCache; import org.opensearch.common.cache.store.config.CacheConfig; @@ -25,8 +26,14 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.FeatureFlags; +import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.common.xcontent.XContentHelper; +import org.opensearch.core.xcontent.MediaTypeRegistry; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.test.OpenSearchTestCase; +import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -41,9 +48,10 @@ import java.util.function.Predicate; import static org.opensearch.common.cache.store.settings.OpenSearchOnHeapCacheSettings.MAXIMUM_SIZE_IN_BYTES_KEY; +import static org.opensearch.cache.common.tier.TieredSpilloverCache.TIER_DIMENSION_VALUE_ON_HEAP; +import static org.opensearch.cache.common.tier.TieredSpilloverCache.TIER_DIMENSION_VALUE_DISK; 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"); public void testComputeIfAbsentWithoutAnyOnHeapCacheEviction() throws Exception { @@ -75,6 +83,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); @@ -93,6 +104,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 { @@ -159,12 +177,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, getEntriesForTier(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, getEntriesForTier(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() { @@ -311,6 +342,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, getEntriesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(onHeapCacheSize * keyValueSize, getSizeInBytesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(actualDiskCacheSize, getEntriesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); + tieredSpilloverCache.getOnHeapCache().keys().forEach(onHeapKeys::add); tieredSpilloverCache.getDiskCache().keys().forEach(diskTierKeys::add); @@ -340,6 +380,11 @@ public void testComputeIfAbsentWithEvictionsFromOnHeapCache() throws Exception { 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 { @@ -369,8 +414,10 @@ 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 + getEntriesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK), getEvictionsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); } public void testGetAndCount() throws Exception { @@ -426,7 +473,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; @@ -449,6 +496,8 @@ public void testPut() { ICacheKey key = getICacheKey(UUID.randomUUID().toString()); String value = UUID.randomUUID().toString(); tieredSpilloverCache.put(key, value); + assertEquals(1, getEntriesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(1, tieredSpilloverCache.count()); } public void testPutAndVerifyNewItemsArePresentOnHeapCache() throws Exception { @@ -481,6 +530,9 @@ public void testPutAndVerifyNewItemsArePresentOnHeapCache() throws Exception { tieredSpilloverCache.computeIfAbsent(getICacheKey(UUID.randomUUID().toString()), getLoadAwareCacheLoader()); } + assertEquals(onHeapCacheSize, getEntriesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(0, getEntriesForTier(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++) { @@ -499,9 +551,11 @@ public void testPutAndVerifyNewItemsArePresentOnHeapCache() throws Exception { for (int i = 0; i < actualOnHeapCacheKeys.size(); i++) { assertTrue(newKeyList.contains(actualOnHeapCacheKeys.get(i))); } + assertEquals(onHeapCacheSize, getEntriesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(onHeapCacheSize, getEntriesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); } - public void testInvalidate() { + public void testInvalidate() throws Exception { int onHeapCacheSize = 1; int diskCacheSize = 10; int keyValueSize = 20; @@ -525,11 +579,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()); @@ -539,11 +594,15 @@ public void testInvalidate() { tieredSpilloverCache.put(key2, UUID.randomUUID().toString()); assertEquals(2, tieredSpilloverCache.count()); + assertEquals(1, getEntriesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(1, getEntriesForTier(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, getEntriesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); + assertEquals(1, getEntriesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); assertEquals(1, tieredSpilloverCache.count()); - } public void testCacheKeys() throws Exception { @@ -779,7 +838,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, getEntriesForTier(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 @@ -817,6 +876,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, getEntriesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(1, getEntriesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); assertNotNull(onDiskCache.get(keyToBeEvicted)); } @@ -1123,4 +1186,63 @@ private TieredSpilloverCache intializeTieredSpilloverCache( } return builder.build(); } + + // Helper functions for extracting tier aggregated stats. + private static int getHitsForTier(TieredSpilloverCache tsc, String tierValue) throws IOException { + return getStatsValueForTier(tsc, tierValue, CacheStatsCounterSnapshot.Fields.HIT_COUNT); + } + + private static int getMissesForTier(TieredSpilloverCache tsc, String tierValue) throws IOException { + return getStatsValueForTier(tsc, tierValue, CacheStatsCounterSnapshot.Fields.MISS_COUNT); + } + + private static int getEvictionsForTier(TieredSpilloverCache tsc, String tierValue) throws IOException { + return getStatsValueForTier(tsc, tierValue, CacheStatsCounterSnapshot.Fields.EVICTIONS); + } + + private static int getSizeInBytesForTier(TieredSpilloverCache tsc, String tierValue) throws IOException { + return getStatsValueForTier(tsc, tierValue, CacheStatsCounterSnapshot.Fields.MEMORY_SIZE_IN_BYTES); + } + + private static int getEntriesForTier(TieredSpilloverCache tsc, String tierValue) throws IOException { + return getStatsValueForTier(tsc, tierValue, CacheStatsCounterSnapshot.Fields.ENTRIES); + } + + private static int getStatsValueForTier(TieredSpilloverCache tsc, String tierValue, String fieldName) throws IOException { + CacheStats cacheStats = tsc.stats(); + Map aggregatedXContentMap = getStatsXContentMap(cacheStats, List.of(TieredSpilloverCache.TIER_DIMENSION_NAME)); + Object result = getValueFromNestedXContentMap(aggregatedXContentMap, List.of(TieredSpilloverCache.TIER_DIMENSION_NAME, tierValue, fieldName)); + if (result == null) { + // This can happen if no cache actions have happened for this set of dimensions yet + return 0; + } + return (int) result; + } + + // Duplicated from OpenSearchOnHeapCacheTests.java; we can't add a dependency on server.test + private static Map getStatsXContentMap(CacheStats cacheStats, List levels) throws IOException { + XContentBuilder builder = XContentFactory.jsonBuilder(); + Map paramMap = Map.of("level", String.join(",", levels)); + ToXContent.Params params = new ToXContent.MapParams(paramMap); + + builder.startObject(); + cacheStats.toXContent(builder, params); + builder.endObject(); + + String resultString = builder.toString(); + return XContentHelper.convertToMap(MediaTypeRegistry.JSON.xContent(), resultString, true); + } + + // Duplicated from MultiDimensionCacheStatsTests.java; we can't add a dependency on server.test + private static Object getValueFromNestedXContentMap(Map xContentMap, List keys) { + Map current = xContentMap; + for (int i = 0; i < keys.size() - 1; i++) { + Object next = current.get(keys.get(i)); + if (next == null) { + return null; + } + current = (Map) next; + } + return current.get(keys.get(keys.size() - 1)); + } } diff --git a/server/src/main/java/org/opensearch/common/cache/stats/CacheStatsCounterSnapshot.java b/server/src/main/java/org/opensearch/common/cache/stats/CacheStatsCounterSnapshot.java index b987eadfa751f..fccd1d10da25e 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/CacheStatsCounterSnapshot.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/CacheStatsCounterSnapshot.java @@ -115,14 +115,14 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws return builder; } - static final class Fields { - static final String MEMORY_SIZE = "size"; // TODO: Bad name - think of something better - static final String MEMORY_SIZE_IN_BYTES = "size_in_bytes"; + public static final class Fields { + public static final String MEMORY_SIZE = "size"; // TODO: Bad name - think of something better + public static final String MEMORY_SIZE_IN_BYTES = "size_in_bytes"; // TODO: This might not be memory as it could be partially on disk, so I've changed it, but should it be consistent with the earlier // field? - static final String EVICTIONS = "evictions"; - static final String HIT_COUNT = "hit_count"; - static final String MISS_COUNT = "miss_count"; - static final String ENTRIES = "entries"; + public static final String EVICTIONS = "evictions"; + public static final String HIT_COUNT = "hit_count"; + public static final String MISS_COUNT = "miss_count"; + public static final String ENTRIES = "entries"; } } From 7db638535a5025ddb6508d13519716b8baa187b1 Mon Sep 17 00:00:00 2001 From: Peter Alfonsi Date: Wed, 10 Apr 2024 15:13:14 -0700 Subject: [PATCH 16/17] Revert "Merge branch 'tiramisu-stats-api' into tiramisu-stats-tsc" This reverts commit e7d92614fd18eb43eb20a802772256a8ba326032, reversing changes made to 96c9493658dc016559068a0186cb7f155f4e3f4d. --- .../common/tier/TieredSpilloverCache.java | 2 +- .../cache/store/disk/EhcacheDiskCache.java | 2 +- .../store/disk/EhCacheDiskCacheTests.java | 111 --------- .../indices/IndicesRequestCacheIT.java | 65 ------ .../admin/cluster/node/stats/NodeStats.java | 24 +- .../cluster/node/stats/NodesStatsRequest.java | 3 +- .../node/stats/TransportNodesStatsAction.java | 3 +- .../stats/TransportClusterStatsAction.java | 1 - .../admin/indices/stats/CommonStatsFlags.java | 26 --- .../opensearch/common/cache/CacheType.java | 30 +-- .../opensearch/common/cache/ICacheKey.java | 1 + .../common/cache/service/CacheService.java | 11 - .../common/cache/service/NodeCacheStats.java | 81 ------- .../common/cache/stats/CacheStats.java | 21 +- .../stats/CacheStatsCounterSnapshot.java | 27 +-- .../common/cache/stats/DimensionNode.java | 4 + .../cache/stats/MultiDimensionCacheStats.java | 139 +----------- .../common/cache/stats/StatsHolder.java | 11 +- .../cache/store/OpenSearchOnHeapCache.java | 2 +- .../main/java/org/opensearch/node/Node.java | 3 +- .../java/org/opensearch/node/NodeService.java | 12 +- .../admin/cluster/RestNodesStatsAction.java | 20 -- .../cluster/node/stats/NodeStatsTests.java | 73 +----- .../opensearch/cluster/DiskUsageTests.java | 6 - .../stats/MultiDimensionCacheStatsTests.java | 213 ++++++------------ .../common/cache/stats/StatsHolderTests.java | 29 +-- .../store/OpenSearchOnHeapCacheTests.java | 92 -------- .../indices/IndicesRequestCacheTests.java | 127 ----------- .../MockInternalClusterInfoService.java | 3 +- .../opensearch/test/InternalTestCluster.java | 1 - 30 files changed, 107 insertions(+), 1036 deletions(-) delete mode 100644 server/src/main/java/org/opensearch/common/cache/service/NodeCacheStats.java 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 1829703dd0701..34f5d60b3b987 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 @@ -124,7 +124,7 @@ public class TieredSpilloverCache implements ICache { new Tuple<>(diskCache, TIER_DIMENSION_VALUE_DISK) ); // Pass "tier" as the innermost dimension name, in addition to whatever dimensions are specified for the cache as a whole - this.statsHolder = new StatsHolder(addTierValueToDimensionValues(dimensionNames, TIER_DIMENSION_NAME), TieredSpilloverCacheFactory.TIERED_SPILLOVER_CACHE_NAME); + this.statsHolder = new StatsHolder(addTierValueToDimensionValues(dimensionNames, TIER_DIMENSION_NAME)); this.policies = builder.policies; // Will never be null; builder initializes it to an empty list } 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 516b57342ebb9..7c7c700728074 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 @@ -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, EhcacheDiskCacheFactory.EHCACHE_DISK_CACHE_NAME); + this.statsHolder = new StatsHolder(dimensionNames); } @SuppressWarnings({ "rawtypes" }) 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 82acd0de5743a..510a143b144d5 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,19 +20,13 @@ 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.CacheStats; import org.opensearch.common.cache.store.config.CacheConfig; import org.opensearch.common.metrics.CounterMetric; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; -import org.opensearch.common.xcontent.XContentFactory; -import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.bytes.CompositeBytesReference; -import org.opensearch.core.xcontent.MediaTypeRegistry; -import org.opensearch.core.xcontent.ToXContent; -import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.env.NodeEnvironment; import org.opensearch.test.OpenSearchSingleNodeTestCase; @@ -803,75 +797,6 @@ public void testInvalidate() throws Exception { } } - // Modified from OpenSearchOnHeapCacheTests.java - public void testInvalidateWithDropDimensions() throws Exception { - Settings settings = Settings.builder().build(); - List dimensionNames = List.of("dim1", "dim2"); - try (NodeEnvironment env = newNodeEnvironment(settings)) { - ICache ehCacheDiskCachingTier = new EhcacheDiskCache.Builder().setThreadPoolAlias("ehcacheTest") - .setStoragePath(env.nodePaths()[0].indicesPath.toString() + "/request_cache") - .setKeySerializer(new StringSerializer()) - .setValueSerializer(new StringSerializer()) - .setDimensionNames(dimensionNames) - .setKeyType(String.class) - .setValueType(String.class) - .setCacheType(CacheType.INDICES_REQUEST_CACHE) - .setSettings(settings) - .setMaximumWeightInBytes(CACHE_SIZE_IN_BYTES * 20) // bigger so no evictions happen - .setExpireAfterAccess(TimeValue.MAX_VALUE) - .setRemovalListener(new MockRemovalListener<>()) - .setWeigher((key, value) -> 1) - .build(); - - List> keysAdded = new ArrayList<>(); - - for (int i = 0; i < 20; i++) { - ICacheKey key = new ICacheKey<>(UUID.randomUUID().toString(), getRandomDimensions(dimensionNames)); - keysAdded.add(key); - ehCacheDiskCachingTier.put(key, UUID.randomUUID().toString()); - } - - ICacheKey keyToDrop = keysAdded.get(0); - - Map xContentMap = getStatsXContentMap(ehCacheDiskCachingTier.stats(), dimensionNames); - List xContentMapKeys = getXContentMapKeys(keyToDrop, dimensionNames); - Map individualSnapshotMap = (Map) getValueFromNestedXContentMap(xContentMap, xContentMapKeys); - assertNotNull(individualSnapshotMap); - assertEquals(5, individualSnapshotMap.size()); // Assert all 5 stats are present and not null - for (Map.Entry entry : individualSnapshotMap.entrySet()) { - Integer value = (Integer) entry.getValue(); - assertNotNull(value); - } - - keyToDrop.setDropStatsForDimensions(true); - ehCacheDiskCachingTier.invalidate(keyToDrop); - - // Now assert the stats are gone for any key that has this combination of dimensions, but still there otherwise - xContentMap = getStatsXContentMap(ehCacheDiskCachingTier.stats(), dimensionNames); - for (ICacheKey keyAdded : keysAdded) { - xContentMapKeys = getXContentMapKeys(keyAdded, dimensionNames); - individualSnapshotMap = (Map) getValueFromNestedXContentMap(xContentMap, xContentMapKeys); - if (keyAdded.dimensions.equals(keyToDrop.dimensions)) { - assertNull(individualSnapshotMap); - } else { - assertNotNull(individualSnapshotMap); - } - } - - ehCacheDiskCachingTier.close(); - } - } - - private List getRandomDimensions(List dimensionNames) { - Random rand = Randomness.get(); - int bound = 3; - List result = new ArrayList<>(); - for (String dimName : dimensionNames) { - result.add(String.valueOf(rand.nextInt(bound))); - } - return result; - } - private static String generateRandomString(int length) { String characters = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789"; StringBuilder randomString = new StringBuilder(length); @@ -906,42 +831,6 @@ private ToLongBiFunction, String> getWeigher() { }; } - // Helper functions duplicated from server.test; we can't add a dependency on that module - private static Map getStatsXContentMap(CacheStats cacheStats, List levels) throws IOException { - XContentBuilder builder = XContentFactory.jsonBuilder(); - Map paramMap = Map.of("level", String.join(",", levels)); - ToXContent.Params params = new ToXContent.MapParams(paramMap); - - builder.startObject(); - cacheStats.toXContent(builder, params); - builder.endObject(); - - String resultString = builder.toString(); - return XContentHelper.convertToMap(MediaTypeRegistry.JSON.xContent(), resultString, true); - } - - private List getXContentMapKeys(ICacheKey iCacheKey, List dimensionNames) { - List result = new ArrayList<>(); - assert iCacheKey.dimensions.size() == dimensionNames.size(); - for (int i = 0; i < dimensionNames.size(); i++) { - result.add(dimensionNames.get(i)); - result.add(iCacheKey.dimensions.get(i)); - } - return result; - } - - public static Object getValueFromNestedXContentMap(Map xContentMap, List keys) { - Map current = xContentMap; - for (int i = 0; i < keys.size() - 1; i++) { - Object next = current.get(keys.get(i)); - if (next == null) { - return null; - } - current = (Map) next; - } - return current.get(keys.get(keys.size() - 1)); - } - static class MockRemovalListener implements RemovalListener, V> { CounterMetric evictionMetric = new CounterMetric(); diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/IndicesRequestCacheIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/IndicesRequestCacheIT.java index 1fd8486699ed2..52b4dad553180 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/IndicesRequestCacheIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/IndicesRequestCacheIT.java @@ -34,30 +34,17 @@ import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.opensearch.action.admin.cluster.node.stats.NodeStats; -import org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest; -import org.opensearch.action.admin.cluster.node.stats.NodesStatsResponse; import org.opensearch.action.admin.indices.alias.Alias; import org.opensearch.action.admin.indices.forcemerge.ForceMergeResponse; import org.opensearch.action.search.SearchResponse; import org.opensearch.action.search.SearchType; import org.opensearch.client.Client; -import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.cluster.routing.ShardRouting; -import org.opensearch.common.cache.service.NodeCacheStats; -import org.opensearch.common.cache.stats.CacheStats; import org.opensearch.common.settings.Settings; import org.opensearch.common.time.DateFormatter; import org.opensearch.common.util.FeatureFlags; -import org.opensearch.common.xcontent.XContentFactory; -import org.opensearch.common.xcontent.XContentHelper; -import org.opensearch.core.xcontent.MediaTypeRegistry; -import org.opensearch.core.xcontent.ToXContent; -import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.cache.request.RequestCacheStats; import org.opensearch.index.query.QueryBuilders; -import org.opensearch.index.shard.IndexShard; import org.opensearch.search.aggregations.bucket.global.GlobalAggregationBuilder; import org.opensearch.search.aggregations.bucket.histogram.DateHistogramInterval; import org.opensearch.search.aggregations.bucket.histogram.Histogram; @@ -65,7 +52,6 @@ import org.opensearch.test.ParameterizedStaticSettingsOpenSearchIntegTestCase; import org.opensearch.test.hamcrest.OpenSearchAssertions; -import java.io.IOException; import java.time.ZoneId; import java.time.ZoneOffset; import java.time.ZonedDateTime; @@ -73,7 +59,6 @@ import java.util.Arrays; import java.util.Collection; import java.util.List; -import java.util.Map; import static org.opensearch.search.SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING; import static org.opensearch.search.aggregations.AggregationBuilders.dateHistogram; @@ -692,34 +677,6 @@ public void testCacheWithInvalidation() throws Exception { assertCacheState(client, "index", 1, 2); } - public void testCacheStatsAPI() throws Exception { - final String nodeId = internalCluster().startNode(); - Client client = client(nodeId); - assertAcked( - client.admin() - .indices() - .prepareCreate("index") - .setMapping("k", "type=keyword") - .setSettings( - Settings.builder() - .put(IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING.getKey(), true) - .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) - ) - .get() - ); - indexRandom(true, client.prepareIndex("index").setSource("k", "hello")); - ensureSearchable("index"); - SearchResponse resp = client.prepareSearch("index").setRequestCache(true).setQuery(QueryBuilders.termQuery("k", "hello")).get(); - assertSearchResponse(resp); - OpenSearchAssertions.assertAllSuccessful(resp); - assertThat(resp.getHits().getTotalHits().value, equalTo(1L)); - - Map xContentMap = getNodeCacheStatsXContentMap(client, nodeId, List.of()); - int j = 0; - - } - private static void assertCacheState(Client client, String index, long expectedHits, long expectedMisses) { RequestCacheStats requestCacheStats = client.admin() .indices() @@ -737,26 +694,4 @@ private static void assertCacheState(Client client, String index, long expectedH } - private static Map getNodeCacheStatsXContentMap(Client client, String nodeId, List aggregationLevels) throws IOException { - - - NodesStatsResponse nodeStatsResponse = client.admin().cluster() - .prepareNodesStats("data:true") - .addMetric(NodesStatsRequest.Metric.CACHE_STATS.metricName()) - .get(); - Map intermediate = nodeStatsResponse.getNodesMap(); - NodeCacheStats ncs = nodeStatsResponse.getNodes().get(0).getNodeCacheStats(); - - XContentBuilder builder = XContentFactory.jsonBuilder(); - Map paramMap = Map.of("level", String.join(",", aggregationLevels)); - ToXContent.Params params = new ToXContent.MapParams(paramMap); - - builder.startObject(); - ncs.toXContent(builder, params); - builder.endObject(); - - String resultString = builder.toString(); - return XContentHelper.convertToMap(MediaTypeRegistry.JSON.xContent(), resultString, true); - } - } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java index ac2daf57f248b..8562a7eb37709 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java @@ -39,7 +39,6 @@ import org.opensearch.cluster.routing.WeightedRoutingStats; import org.opensearch.cluster.service.ClusterManagerThrottlingStats; import org.opensearch.common.Nullable; -import org.opensearch.common.cache.service.NodeCacheStats; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.indices.breaker.AllCircuitBreakerStats; @@ -159,9 +158,6 @@ public class NodeStats extends BaseNodeResponse implements ToXContentFragment { @Nullable private AdmissionControlStats admissionControlStats; - @Nullable - private NodeCacheStats nodeCacheStats; - public NodeStats(StreamInput in) throws IOException { super(in); timestamp = in.readVLong(); @@ -238,11 +234,6 @@ public NodeStats(StreamInput in) throws IOException { } else { admissionControlStats = null; } - if (in.getVersion().onOrAfter(Version.V_3_0_0)) { - nodeCacheStats = in.readOptionalWriteable(NodeCacheStats::new); - } else { - nodeCacheStats = null; - } } public NodeStats( @@ -273,8 +264,7 @@ public NodeStats( @Nullable SearchPipelineStats searchPipelineStats, @Nullable SegmentReplicationRejectionStats segmentReplicationRejectionStats, @Nullable RepositoriesStats repositoriesStats, - @Nullable AdmissionControlStats admissionControlStats, - @Nullable NodeCacheStats nodeCacheStats + @Nullable AdmissionControlStats admissionControlStats ) { super(node); this.timestamp = timestamp; @@ -304,7 +294,6 @@ public NodeStats( this.segmentReplicationRejectionStats = segmentReplicationRejectionStats; this.repositoriesStats = repositoriesStats; this.admissionControlStats = admissionControlStats; - this.nodeCacheStats = nodeCacheStats; } public long getTimestamp() { @@ -462,11 +451,6 @@ public AdmissionControlStats getAdmissionControlStats() { return admissionControlStats; } - @Nullable - public NodeCacheStats getNodeCacheStats() { - return nodeCacheStats; - } - @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); @@ -522,9 +506,6 @@ public void writeTo(StreamOutput out) throws IOException { if (out.getVersion().onOrAfter(Version.V_2_12_0)) { out.writeOptionalWriteable(admissionControlStats); } - if (out.getVersion().onOrAfter(Version.V_3_0_0)) { - out.writeOptionalWriteable(nodeCacheStats); - } } @Override @@ -628,9 +609,6 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (getAdmissionControlStats() != null) { getAdmissionControlStats().toXContent(builder, params); } - if (getNodeCacheStats() != null) { - getNodeCacheStats().toXContent(builder, params); - } return builder; } } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsRequest.java index 379836cf442e3..1af56f10b95ee 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsRequest.java @@ -219,8 +219,7 @@ public enum Metric { RESOURCE_USAGE_STATS("resource_usage_stats"), SEGMENT_REPLICATION_BACKPRESSURE("segment_replication_backpressure"), REPOSITORIES("repositories"), - ADMISSION_CONTROL("admission_control"), - CACHE_STATS("caches"); + ADMISSION_CONTROL("admission_control"); private String metricName; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java index 2e93e5e7841cb..1df73d3b4394d 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java @@ -128,8 +128,7 @@ protected NodeStats nodeOperation(NodeStatsRequest nodeStatsRequest) { NodesStatsRequest.Metric.RESOURCE_USAGE_STATS.containedIn(metrics), NodesStatsRequest.Metric.SEGMENT_REPLICATION_BACKPRESSURE.containedIn(metrics), NodesStatsRequest.Metric.REPOSITORIES.containedIn(metrics), - NodesStatsRequest.Metric.ADMISSION_CONTROL.containedIn(metrics), - NodesStatsRequest.Metric.CACHE_STATS.containedIn(metrics) + NodesStatsRequest.Metric.ADMISSION_CONTROL.containedIn(metrics) ); } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java index e4f483f796f44..9c5dcc9e9de3f 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java @@ -172,7 +172,6 @@ protected ClusterStatsNodeResponse nodeOperation(ClusterStatsNodeRequest nodeReq false, false, false, - false, false ); List shardsStats = new ArrayList<>(); diff --git a/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java b/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java index cbde1637ea575..a7d9f95b80f7b 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java @@ -34,7 +34,6 @@ import org.opensearch.Version; import org.opensearch.common.annotation.PublicApi; -import org.opensearch.common.cache.CacheType; import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -64,9 +63,6 @@ public class CommonStatsFlags implements Writeable, Cloneable { private boolean includeAllShardIndexingPressureTrackers = false; private boolean includeOnlyTopIndexingPressureMetrics = false; - // Used for metric CACHE_STATS, to determine which caches to report stats for - private EnumSet includeCaches = EnumSet.noneOf(CacheType.class); - /** * @param flags flags to set. If no flags are supplied, default flags will be set. */ @@ -95,9 +91,6 @@ public CommonStatsFlags(StreamInput in) throws IOException { includeUnloadedSegments = in.readBoolean(); includeAllShardIndexingPressureTrackers = in.readBoolean(); includeOnlyTopIndexingPressureMetrics = in.readBoolean(); - if (in.getVersion().onOrAfter(Version.V_3_0_0)) { - includeCaches = in.readEnumSet(CacheType.class); - } } @Override @@ -118,9 +111,6 @@ public void writeTo(StreamOutput out) throws IOException { out.writeBoolean(includeUnloadedSegments); out.writeBoolean(includeAllShardIndexingPressureTrackers); out.writeBoolean(includeOnlyTopIndexingPressureMetrics); - if (out.getVersion().onOrAfter(Version.V_3_0_0)) { - out.writeEnumSet(includeCaches); - } } /** @@ -135,7 +125,6 @@ public CommonStatsFlags all() { includeUnloadedSegments = false; includeAllShardIndexingPressureTrackers = false; includeOnlyTopIndexingPressureMetrics = false; - includeCaches = EnumSet.noneOf(CacheType.class); return this; } @@ -151,7 +140,6 @@ public CommonStatsFlags clear() { includeUnloadedSegments = false; includeAllShardIndexingPressureTrackers = false; includeOnlyTopIndexingPressureMetrics = false; - includeCaches = EnumSet.noneOf(CacheType.class); return this; } @@ -163,10 +151,6 @@ public Flag[] getFlags() { return flags.toArray(new Flag[0]); } - public EnumSet getIncludeCaches() { - return includeCaches; - } - /** * Sets specific search group stats to retrieve the stats for. Mainly affects search * when enabled. @@ -222,16 +206,6 @@ public CommonStatsFlags includeOnlyTopIndexingPressureMetrics(boolean includeOnl return this; } - public CommonStatsFlags includeCacheType(CacheType cacheType) { - includeCaches.add(cacheType); - return this; - } - - public CommonStatsFlags includeAllCacheTypes() { - includeCaches = EnumSet.allOf(CacheType.class); - return this; - } - public boolean includeUnloadedSegments() { return this.includeUnloadedSegments; } diff --git a/server/src/main/java/org/opensearch/common/cache/CacheType.java b/server/src/main/java/org/opensearch/common/cache/CacheType.java index 61442db148067..c5aeb7cd1fa40 100644 --- a/server/src/main/java/org/opensearch/common/cache/CacheType.java +++ b/server/src/main/java/org/opensearch/common/cache/CacheType.java @@ -10,46 +10,20 @@ import org.opensearch.common.annotation.ExperimentalApi; -import java.util.HashSet; -import java.util.Set; - /** * Cache types available within OpenSearch. */ @ExperimentalApi public enum CacheType { - INDICES_REQUEST_CACHE("indices.requests.cache", "request_cache"); + INDICES_REQUEST_CACHE("indices.requests.cache"); private final String settingPrefix; - private final String apiRepresentation; - CacheType(String settingPrefix, String representation) { + CacheType(String settingPrefix) { this.settingPrefix = settingPrefix; - this.apiRepresentation = representation; } public String getSettingPrefix() { return settingPrefix; } - - public String getApiRepresentation() { - return apiRepresentation; - } - - public static CacheType getByRepresentation(String representation) { - for (CacheType cacheType : values()) { - if (cacheType.apiRepresentation.equals(representation)) { - return cacheType; - } - } - throw new IllegalArgumentException("No CacheType with representation = " + representation); - } - - public static Set allRepresentations() { - Set reprs = new HashSet<>(); - for (CacheType cacheType : values()) { - reprs.add(cacheType.apiRepresentation); - } - return reprs; - } } 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..0eb34952e78f1 100644 --- a/server/src/main/java/org/opensearch/common/cache/ICacheKey.java +++ b/server/src/main/java/org/opensearch/common/cache/ICacheKey.java @@ -10,6 +10,7 @@ import org.opensearch.common.annotation.ExperimentalApi; +import java.util.Collections; import java.util.List; /** diff --git a/server/src/main/java/org/opensearch/common/cache/service/CacheService.java b/server/src/main/java/org/opensearch/common/cache/service/CacheService.java index e450fc65fe351..b6710e5e4b424 100644 --- a/server/src/main/java/org/opensearch/common/cache/service/CacheService.java +++ b/server/src/main/java/org/opensearch/common/cache/service/CacheService.java @@ -8,12 +8,10 @@ package org.opensearch.common.cache.service; -import org.opensearch.action.admin.indices.stats.CommonStatsFlags; import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.cache.CacheType; import org.opensearch.common.cache.ICache; import org.opensearch.common.cache.settings.CacheSettings; -import org.opensearch.common.cache.stats.CacheStats; import org.opensearch.common.cache.store.OpenSearchOnHeapCache; import org.opensearch.common.cache.store.config.CacheConfig; import org.opensearch.common.settings.Setting; @@ -21,7 +19,6 @@ import org.opensearch.common.util.FeatureFlags; import java.util.HashMap; -import java.util.LinkedHashMap; import java.util.Map; /** @@ -65,12 +62,4 @@ public ICache createCache(CacheConfig config, CacheType cache cacheTypeMap.put(cacheType, iCache); return iCache; } - - public NodeCacheStats stats(CommonStatsFlags flags) { - LinkedHashMap statsMap = new LinkedHashMap<>(); - for (CacheType type : cacheTypeMap.keySet()) { - statsMap.put(type, cacheTypeMap.get(type).stats()); // TODO: We need to force some ordering on cacheTypeMap - } - return new NodeCacheStats(statsMap, flags); - } } diff --git a/server/src/main/java/org/opensearch/common/cache/service/NodeCacheStats.java b/server/src/main/java/org/opensearch/common/cache/service/NodeCacheStats.java deleted file mode 100644 index 43ed8c0d0fcd6..0000000000000 --- a/server/src/main/java/org/opensearch/common/cache/service/NodeCacheStats.java +++ /dev/null @@ -1,81 +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.service; - -import org.opensearch.action.admin.indices.stats.CommonStatsFlags; -import org.opensearch.common.annotation.ExperimentalApi; -import org.opensearch.common.cache.CacheType; -import org.opensearch.common.cache.stats.CacheStats; -import org.opensearch.core.common.io.stream.StreamInput; -import org.opensearch.core.common.io.stream.StreamOutput; -import org.opensearch.core.common.io.stream.Writeable; -import org.opensearch.core.xcontent.ToXContentFragment; -import org.opensearch.core.xcontent.XContentBuilder; - -import java.io.IOException; -import java.util.LinkedHashMap; -import java.util.Map; -import java.util.Objects; - -/** - * A class creating XContent responses to cache stats API requests. - * - * @opensearch.experimental - */ -@ExperimentalApi -public class NodeCacheStats implements ToXContentFragment, Writeable { - private final LinkedHashMap statsByCache; - private final CommonStatsFlags flags; - - public NodeCacheStats(LinkedHashMap statsByCache, CommonStatsFlags flags) { - this.statsByCache = statsByCache; - this.flags = flags; - } - - public NodeCacheStats(StreamInput in) throws IOException { - this.flags = new CommonStatsFlags(in); - Map readMap = in.readMap(i -> i.readEnum(CacheType.class), CacheStats::readFromStreamWithClassName); - this.statsByCache = new LinkedHashMap<>(readMap); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - flags.writeTo(out); - out.writeMap(statsByCache, StreamOutput::writeEnum, (o, cacheStats) -> cacheStats.writeToWithClassName(o)); - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - for (CacheType type : statsByCache.keySet()) { - if (flags.getIncludeCaches().contains(type)) { - builder.startObject(type.getApiRepresentation()); - statsByCache.get(type).toXContent(builder, params); - builder.endObject(); - } - } - return builder; - } - - @Override - public boolean equals(Object o) { - if (o == null) { - return false; - } - if (o.getClass() != NodeCacheStats.class) { - return false; - } - NodeCacheStats other = (NodeCacheStats) o; - return statsByCache.equals(other.statsByCache) && flags.getIncludeCaches().equals(other.flags.getIncludeCaches()); - } - - @Override - public int hashCode() { - return Objects.hash(statsByCache, flags); - } -} 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 1e4ae24eb88a1..a552b13aa5f84 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 @@ -9,12 +9,7 @@ package org.opensearch.common.cache.stats; import org.opensearch.common.annotation.ExperimentalApi; -import org.opensearch.core.common.io.stream.StreamInput; -import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; -import org.opensearch.core.xcontent.ToXContentFragment; - -import java.io.IOException; /** * Interface for access to any cache stats. Allows accessing stats by dimension values. @@ -23,7 +18,7 @@ * @opensearch.experimental */ @ExperimentalApi -public interface CacheStats extends Writeable, ToXContentFragment { +public interface CacheStats extends Writeable {// TODO: also extends ToXContentFragment (in API PR) // Method to get all 5 values at once CacheStatsCounterSnapshot getTotalStats(); @@ -38,18 +33,4 @@ public interface CacheStats extends Writeable, ToXContentFragment { long getTotalSizeInBytes(); long getTotalEntries(); - - // Used for the readFromStream method to allow deserialization of generic CacheStats objects. - String getClassName(); - - void writeToWithClassName(StreamOutput out) throws IOException; - - static CacheStats readFromStreamWithClassName(StreamInput in) throws IOException { - String className = in.readString(); - - if (className.equals(MultiDimensionCacheStats.CLASS_NAME)) { - return new MultiDimensionCacheStats(in); - } - return null; - } } diff --git a/server/src/main/java/org/opensearch/common/cache/stats/CacheStatsCounterSnapshot.java b/server/src/main/java/org/opensearch/common/cache/stats/CacheStatsCounterSnapshot.java index fccd1d10da25e..3057edd8b2afc 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/CacheStatsCounterSnapshot.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/CacheStatsCounterSnapshot.java @@ -12,9 +12,6 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; -import org.opensearch.core.common.unit.ByteSizeValue; -import org.opensearch.core.xcontent.ToXContent; -import org.opensearch.core.xcontent.XContentBuilder; import java.io.IOException; import java.util.Objects; @@ -25,7 +22,7 @@ * @opensearch.experimental */ @ExperimentalApi -public class CacheStatsCounterSnapshot implements Writeable, ToXContent { +public class CacheStatsCounterSnapshot implements Writeable { // TODO: Make this extend ToXContent (in API PR) private final long hits; private final long misses; private final long evictions; @@ -103,26 +100,4 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(hits, misses, evictions, sizeInBytes, entries); } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - // We don't write the header in CacheStatsResponse's toXContent, because it doesn't know the name of aggregation it's part of - builder.humanReadableField(Fields.MEMORY_SIZE_IN_BYTES, Fields.MEMORY_SIZE, new ByteSizeValue(sizeInBytes)); - builder.field(Fields.EVICTIONS, evictions); - builder.field(Fields.HIT_COUNT, hits); - builder.field(Fields.MISS_COUNT, misses); - builder.field(Fields.ENTRIES, entries); - return builder; - } - - public static final class Fields { - public static final String MEMORY_SIZE = "size"; // TODO: Bad name - think of something better - public static final String MEMORY_SIZE_IN_BYTES = "size_in_bytes"; - // TODO: This might not be memory as it could be partially on disk, so I've changed it, but should it be consistent with the earlier - // field? - public static final String EVICTIONS = "evictions"; - public static final String HIT_COUNT = "hit_count"; - public static final String MISS_COUNT = "miss_count"; - public static final String ENTRIES = "entries"; - } } 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 index 49f74ea941247..8e89eae80ce79 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/DimensionNode.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/DimensionNode.java @@ -91,6 +91,10 @@ void resetSizeAndEntries() { } DimensionNode getChild(String dimensionValue) { // , boolean createIfAbsent, boolean createMapInChild + /*return children.computeIfAbsent( + dimensionValue, + (key) -> createIfAbsent ? new DimensionNode(dimensionValue, createMapInChild) : null + );*/ return children.get(dimensionValue); } diff --git a/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java b/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java index 4ce7210e7775e..ea4dfcc818350 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/MultiDimensionCacheStats.java @@ -10,13 +10,11 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; -import org.opensearch.core.xcontent.XContentBuilder; import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.TreeMap; /** @@ -31,21 +29,11 @@ public class MultiDimensionCacheStats implements CacheStats { final MDCSDimensionNode statsRoot; final List dimensionNames; - // The name of the cache type producing these stats. Returned in API response. - final String storeName; - public static String STORE_NAME_FIELD = "store_name"; - - public static String CLASS_NAME = "multidimension"; - - public MultiDimensionCacheStats(MDCSDimensionNode statsRoot, List dimensionNames, String storeName) { + public MultiDimensionCacheStats(MDCSDimensionNode statsRoot, List dimensionNames) { this.statsRoot = statsRoot; this.dimensionNames = dimensionNames; - this.storeName = storeName; } - /** - * Should not be used with StreamOutputs produced using writeToWithClassName. - */ public MultiDimensionCacheStats(StreamInput in) throws IOException { // Because we write in preorder order, the parent of the next node we read will always be one of the ancestors // of the last node we read. This allows us to avoid ambiguity if nodes have the same dimension value, without @@ -60,10 +48,9 @@ public MultiDimensionCacheStats(StreamInput in) throws IOException { // Finally, update sum-of-children stats for the root node CacheStatsCounter totalStats = new CacheStatsCounter(); for (MDCSDimensionNode child : statsRoot.children.values()) { - totalStats.add(child.getStatsSnapshot()); + totalStats.add(child.getStats()); } statsRoot.setStats(totalStats.snapshot()); - this.storeName = in.readString(); } @Override @@ -75,14 +62,13 @@ public void writeTo(StreamOutput out) throws IOException { writeDimensionNodeRecursive(out, child, 1); } out.writeBoolean(false); // Write false to signal there are no more nodes - out.writeString(storeName); } private void writeDimensionNodeRecursive(StreamOutput out, MDCSDimensionNode node, int depth) throws IOException { out.writeBoolean(true); // Signals there is a following node to deserialize out.writeVInt(depth); out.writeString(node.getDimensionValue()); - node.getStatsSnapshot().writeTo(out); + node.getStats().writeTo(out); if (node.hasChildren()) { // Not a leaf node @@ -125,7 +111,7 @@ private List readAndAttachDimensionNode(StreamInput in, List< @Override public CacheStatsCounterSnapshot getTotalStats() { - return statsRoot.getStatsSnapshot(); + return statsRoot.getStats(); } @Override @@ -153,24 +139,13 @@ public long getTotalEntries() { return getTotalStats().getEntries(); } - @Override - public String getClassName() { - return CLASS_NAME; - } - - @Override - public void writeToWithClassName(StreamOutput out) throws IOException { - out.writeString(getClassName()); - writeTo(out); - } - /** * Returns a new tree containing the stats aggregated by the levels passed in. The root node is a dummy node, * whose name and value are null. The new tree only has dimensions matching the levels passed in. */ MDCSDimensionNode aggregateByLevels(List levels) { List filteredLevels = filterLevels(levels); - MDCSDimensionNode newRoot = new MDCSDimensionNode(null, statsRoot.getStatsSnapshot()); + MDCSDimensionNode newRoot = new MDCSDimensionNode(null, statsRoot.getStats()); newRoot.createChildrenMap(); for (MDCSDimensionNode child : statsRoot.children.values()) { aggregateByLevelsHelper(newRoot, child, filteredLevels, 0); @@ -194,13 +169,13 @@ void aggregateByLevelsHelper( MDCSDimensionNode nodeInNewTree = parentInNewTree.children.get(dimensionValue); if (nodeInNewTree == null) { // Create new node with stats matching the node from the original tree - nodeInNewTree = new MDCSDimensionNode(dimensionValue, currentInOriginalTree.getStatsSnapshot()); + nodeInNewTree = new MDCSDimensionNode(dimensionValue, currentInOriginalTree.getStats()); parentInNewTree.children.put(dimensionValue, nodeInNewTree); } else { // Otherwise increment existing stats CacheStatsCounterSnapshot newStats = CacheStatsCounterSnapshot.addSnapshots( - nodeInNewTree.getStatsSnapshot(), - currentInOriginalTree.getStatsSnapshot() + nodeInNewTree.getStats(), + currentInOriginalTree.getStats() ); nodeInNewTree.setStats(newStats); } @@ -233,62 +208,6 @@ private List filterLevels(List levels) { return filtered; } - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - // Always show total stats, regardless of levels - getTotalStats().toXContent(builder, params); - - List levels = getLevels(params); - if (levels == null) { - // display total stats only - return builder; - } - - List filteredLevels = filterLevels(levels); - toXContentForLevels(builder, params, filteredLevels); - // Also add the store name for the cache that produced the stats - builder.field(STORE_NAME_FIELD, storeName); - return builder; - } - - XContentBuilder toXContentForLevels(XContentBuilder builder, Params params, List levels) throws IOException { - MDCSDimensionNode aggregated = aggregateByLevels(levels); - // Depth -1 corresponds to the dummy root node, which has no dimension value and only has children - toXContentForLevelsHelper(-1, aggregated, levels, builder, params); - return builder; - - } - - private void toXContentForLevelsHelper(int depth, MDCSDimensionNode current, List levels, XContentBuilder builder, Params params) - throws IOException { - if (depth >= 0) { - builder.startObject(current.dimensionValue); - } - - if (depth == levels.size() - 1) { - // This is a leaf node - current.getStatsSnapshot().toXContent(builder, params); - } else { - builder.startObject(levels.get(depth + 1)); - for (MDCSDimensionNode nextNode : current.children.values()) { - toXContentForLevelsHelper(depth + 1, nextNode, levels, builder, params); - } - builder.endObject(); - } - - if (depth >= 0) { - builder.endObject(); - } - } - - private List getLevels(Params params) { - String levels = params.param("level"); - if (levels == null) { - return null; - } - return List.of(levels.split(",")); - } - // A version of DimensionNode which uses an ordered TreeMap and holds immutable CacheStatsCounterSnapshot as its stats. // TODO: Make this extend from DimensionNode? static class MDCSDimensionNode { @@ -319,7 +238,7 @@ protected Map getChildren() { return children; } - public CacheStatsCounterSnapshot getStatsSnapshot() { + public CacheStatsCounterSnapshot getStats() { return stats; } @@ -341,43 +260,5 @@ MDCSDimensionNode getStatsRoot() { return statsRoot; } - @Override - public boolean equals(Object o) { - if (o == null || o.getClass() != MultiDimensionCacheStats.class) { - return false; - } - MultiDimensionCacheStats other = (MultiDimensionCacheStats) o; - if (!dimensionNames.equals(other.dimensionNames) || !storeName.equals(other.storeName)) { - return false; - } - return equalsHelper(statsRoot, other.getStatsRoot()); - } - - private boolean equalsHelper(MDCSDimensionNode thisNode, MDCSDimensionNode otherNode) { - if (!thisNode.getStatsSnapshot().equals(otherNode.getStatsSnapshot())) { - return false; - } - if (thisNode.children == null && otherNode.children == null) { - // TODO: Simplify this logic once we inherit from normal DimensionNode and have the static empty map thing - return true; - } - if (thisNode.children == null || otherNode.children == null || !thisNode.getChildren().keySet().equals(otherNode.getChildren().keySet())) { - return false; - } - boolean allChildrenMatch = true; - for (String childValue : thisNode.getChildren().keySet()) { - allChildrenMatch = equalsHelper(thisNode.children.get(childValue), otherNode.children.get(childValue)); - if (!allChildrenMatch) { - return false; - } - } - return allChildrenMatch; - } - - @Override - public int hashCode() { - // Should be sufficient to hash based on the total stats value (found in the root node) - return Objects.hash(statsRoot.stats, dimensionNames); - } - + // TODO (in API PR): Produce XContent based on aggregateByLevels() } diff --git a/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java b/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java index f3fbc42b38021..dc161efb05597 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java @@ -17,8 +17,6 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; -import org.opensearch.common.annotation.ExperimentalApi; - import static org.opensearch.common.cache.stats.MultiDimensionCacheStats.MDCSDimensionNode; /** @@ -31,7 +29,6 @@ * * @opensearch.experimental */ -@ExperimentalApi public class StatsHolder { // The list of permitted dimensions. Should be ordered from "outermost" to "innermost", as you would like to @@ -44,12 +41,8 @@ public class StatsHolder { // No lock is needed to edit stats on existing nodes. private final Lock lock = new ReentrantLock(); - // The name of the cache type using these stats - private final String storeName; - - public StatsHolder(List dimensionNames, String storeName) { + public StatsHolder(List dimensionNames) { this.dimensionNames = dimensionNames; - this.storeName = storeName; this.statsRoot = new DimensionNode(null, true); // The root node has no dimension value associated with it, only children } @@ -167,7 +160,7 @@ public CacheStats getCacheStats() { getCacheStatsHelper(child, snapshot); } } - return new MultiDimensionCacheStats(snapshot, dimensionNames, storeName); + return new MultiDimensionCacheStats(snapshot, dimensionNames); } private void getCacheStatsHelper(DimensionNode currentNodeInOriginalTree, MDCSDimensionNode parentInNewTree) { 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 2a68d83456ace..2e60072d07ed2 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 @@ -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, OpenSearchOnHeapCacheFactory.NAME); + this.statsHolder = new StatsHolder(dimensionNames); this.removalListener = builder.getRemovalListener(); this.weigher = builder.getWeigher(); } diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 628381beda3f9..7fa2b6c8ff497 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -1180,8 +1180,7 @@ protected Node( resourceUsageCollectorService, segmentReplicationStatsTracker, repositoryService, - admissionControlService, - cacheService + admissionControlService ); final SearchService searchService = newSearchService( diff --git a/server/src/main/java/org/opensearch/node/NodeService.java b/server/src/main/java/org/opensearch/node/NodeService.java index 1eb38ea63ad5a..15cc8f3d20bb3 100644 --- a/server/src/main/java/org/opensearch/node/NodeService.java +++ b/server/src/main/java/org/opensearch/node/NodeService.java @@ -41,7 +41,6 @@ import org.opensearch.cluster.routing.WeightedRoutingStats; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Nullable; -import org.opensearch.common.cache.service.CacheService; import org.opensearch.common.settings.Settings; import org.opensearch.common.settings.SettingsFilter; import org.opensearch.common.util.io.IOUtils; @@ -100,7 +99,6 @@ public class NodeService implements Closeable { private final RepositoriesService repositoriesService; private final AdmissionControlService admissionControlService; private final SegmentReplicationStatsTracker segmentReplicationStatsTracker; - private final CacheService cacheService; NodeService( Settings settings, @@ -127,8 +125,7 @@ public class NodeService implements Closeable { ResourceUsageCollectorService resourceUsageCollectorService, SegmentReplicationStatsTracker segmentReplicationStatsTracker, RepositoriesService repositoriesService, - AdmissionControlService admissionControlService, - CacheService cacheService + AdmissionControlService admissionControlService ) { this.settings = settings; this.threadPool = threadPool; @@ -157,7 +154,6 @@ public class NodeService implements Closeable { clusterService.addStateApplier(ingestService); clusterService.addStateApplier(searchPipelineService); this.segmentReplicationStatsTracker = segmentReplicationStatsTracker; - this.cacheService = cacheService; } public NodeInfo info( @@ -240,8 +236,7 @@ public NodeStats stats( boolean resourceUsageStats, boolean segmentReplicationTrackerStats, boolean repositoriesStats, - boolean admissionControl, - boolean cacheService + boolean admissionControl ) { // for indices stats we want to include previous allocated shards stats as well (it will // only be applied to the sensible ones to use, like refresh/merge/flush/indexing stats) @@ -273,8 +268,7 @@ public NodeStats stats( searchPipelineStats ? this.searchPipelineService.stats() : null, segmentReplicationTrackerStats ? this.segmentReplicationStatsTracker.getTotalRejectionStats() : null, repositoriesStats ? this.repositoriesService.getRepositoriesStats() : null, - admissionControl ? this.admissionControlService.stats() : null, - cacheService ? this.cacheService.stats(indices) : null + admissionControl ? this.admissionControlService.stats() : null ); } diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesStatsAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesStatsAction.java index f62eaeb37f41f..66b9afda06eb6 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesStatsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesStatsAction.java @@ -36,7 +36,6 @@ import org.opensearch.action.admin.indices.stats.CommonStatsFlags; import org.opensearch.action.admin.indices.stats.CommonStatsFlags.Flag; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.cache.CacheType; import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; @@ -176,25 +175,6 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC nodesStatsRequest.indices(flags); } - } else if (metrics.contains("caches")) { - // Extract the list of caches we want to get stats for from the submetrics (which we get from index_metric) - Set cacheMetrics = Strings.tokenizeByCommaToSet(request.param("index_metric", "_all")); - CommonStatsFlags cacheFlags = new CommonStatsFlags(); - cacheFlags.clear(); - if (cacheMetrics.size() == 1 && cacheMetrics.contains("_all")) { - cacheFlags.includeAllCacheTypes(); - } else { - for (String cacheName : cacheMetrics) { - try { - cacheFlags.includeCacheType(CacheType.getByRepresentation(cacheName)); - } catch (IllegalArgumentException e) { - throw new IllegalArgumentException( - unrecognized(request, Set.of(cacheName), CacheType.allRepresentations(), "cache type") - ); - } - } - } - nodesStatsRequest.indices(cacheFlags); } else if (request.hasParam("index_metric")) { throw new IllegalArgumentException( String.format( diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java index 468085b4dd6ca..1b8b6243aa805 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java @@ -42,12 +42,6 @@ import org.opensearch.cluster.routing.WeightedRoutingStats; import org.opensearch.cluster.service.ClusterManagerThrottlingStats; import org.opensearch.cluster.service.ClusterStateStats; -import org.opensearch.common.cache.CacheType; -import org.opensearch.common.cache.service.NodeCacheStats; -import org.opensearch.common.cache.stats.CacheStats; -import org.opensearch.common.cache.stats.CacheStatsCounterSnapshot; -import org.opensearch.common.cache.stats.MultiDimensionCacheStats; -import org.opensearch.common.cache.stats.StatsHolder; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.metrics.OperationStats; import org.opensearch.common.settings.ClusterSettings; @@ -93,7 +87,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.Iterator; -import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.function.Function; @@ -584,13 +577,6 @@ public void testSerialization() throws IOException { deserializedAdmissionControllerStats.getRejectionCount().get(AdmissionControlActionType.INDEXING.getType()) ); } - NodeCacheStats nodeCacheStats = nodeStats.getNodeCacheStats(); - NodeCacheStats deserializedNodeCacheStats = deserializedNodeStats.getNodeCacheStats(); - if (nodeCacheStats == null) { - assertNull(deserializedNodeCacheStats); - } else { - assertEquals(nodeCacheStats, deserializedNodeCacheStats); - } } } } @@ -942,62 +928,6 @@ public void apply(String action, AdmissionControlActionType admissionControlActi NodeIndicesStats indicesStats = getNodeIndicesStats(remoteStoreStats); - NodeCacheStats nodeCacheStats = null; - if (frequently()) { - int numIndices = randomIntBetween(1, 10); - int numShardsPerIndex = randomIntBetween(1, 50); - //Map snapshotMap = new HashMap<>(); - - List dimensionNames = List.of("index", "shard", "tier"); - StatsHolder statsHolder = new StatsHolder(dimensionNames, "dummyStoreName"); - for (int indexNum = 0; indexNum < numIndices; indexNum++) { - String indexName = "index" + indexNum; - for (int shardNum = 0; shardNum < numShardsPerIndex; shardNum++) { - String shardName = "[" + indexName + "][" + shardNum + "]"; - for (String tierName : new String[] { "dummy_tier_1", "dummy_tier_2" }) { - /*CacheStatsCounterSnapshot response = new CacheStatsCounterSnapshot( - randomInt(100), - randomInt(100), - randomInt(100), - randomInt(100), - randomInt(100) - );*/ - List dimensionValues = List.of(indexName, shardName, tierName); - for (int i = 0; i < randomInt(20); i++){ - statsHolder.incrementHits(dimensionValues); - } - for (int i = 0; i < randomInt(20); i++){ - statsHolder.incrementMisses(dimensionValues); - } - for (int i = 0; i < randomInt(20); i++){ - statsHolder.incrementEvictions(dimensionValues); - } - statsHolder.incrementSizeInBytes(dimensionValues, randomInt(20)); - for (int i = 0; i < randomInt(20); i++){ - statsHolder.incrementEntries(dimensionValues); - } - /*snapshotMap.put( - new StatsHolder.Key(List.of( - new CacheStatsDimension("testIndexDimensionName", indexName), - new CacheStatsDimension("testShardDimensionName", shardName), - new CacheStatsDimension("testTierDimensionName", tierName) - )), - response);*/ - } - } - } - CommonStatsFlags flags = new CommonStatsFlags(); - for (CacheType cacheType : CacheType.values()) { - if (frequently()) { - flags.includeCacheType(cacheType); - } - } - CacheStats cacheStats = statsHolder.getCacheStats(); - LinkedHashMap cacheStatsMap = new LinkedHashMap<>(); - cacheStatsMap.put(CacheType.INDICES_REQUEST_CACHE, cacheStats); - nodeCacheStats = new NodeCacheStats(cacheStatsMap, flags); - } - // TODO: Only remote_store based aspects of NodeIndicesStats are being tested here. // It is possible to test other metrics in NodeIndicesStats as well since it extends Writeable now return new NodeStats( @@ -1028,8 +958,7 @@ public void apply(String action, AdmissionControlActionType admissionControlActi null, segmentReplicationRejectionStats, null, - admissionControlStats, - nodeCacheStats + admissionControlStats ); } diff --git a/server/src/test/java/org/opensearch/cluster/DiskUsageTests.java b/server/src/test/java/org/opensearch/cluster/DiskUsageTests.java index 5539dd26dd52d..ff47ec3015697 100644 --- a/server/src/test/java/org/opensearch/cluster/DiskUsageTests.java +++ b/server/src/test/java/org/opensearch/cluster/DiskUsageTests.java @@ -194,7 +194,6 @@ public void testFillDiskUsage() { null, null, null, - null, null ), new NodeStats( @@ -225,7 +224,6 @@ public void testFillDiskUsage() { null, null, null, - null, null ), new NodeStats( @@ -256,7 +254,6 @@ public void testFillDiskUsage() { null, null, null, - null, null ) ); @@ -318,7 +315,6 @@ public void testFillDiskUsageSomeInvalidValues() { null, null, null, - null, null ), new NodeStats( @@ -349,7 +345,6 @@ public void testFillDiskUsageSomeInvalidValues() { null, null, null, - null, null ), new NodeStats( @@ -380,7 +375,6 @@ public void testFillDiskUsageSomeInvalidValues() { null, null, null, - null, null ) ); diff --git a/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java b/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java index 34d30f7ab3552..ca9354e663e14 100644 --- a/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java +++ b/server/src/test/java/org/opensearch/common/cache/stats/MultiDimensionCacheStatsTests.java @@ -10,13 +10,8 @@ import org.opensearch.common.Randomness; import org.opensearch.common.io.stream.BytesStreamOutput; -import org.opensearch.common.xcontent.XContentFactory; -import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.io.stream.BytesStreamInput; -import org.opensearch.core.xcontent.MediaTypeRegistry; -import org.opensearch.core.xcontent.ToXContent; -import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.test.OpenSearchTestCase; import java.util.ArrayList; @@ -27,13 +22,11 @@ import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; -import java.util.function.BiConsumer; public class MultiDimensionCacheStatsTests extends OpenSearchTestCase { - private final String storeName = "dummy_store"; public void testSerialization() throws Exception { List dimensionNames = List.of("dim1", "dim2", "dim3"); - StatsHolder statsHolder = new StatsHolder(dimensionNames, storeName); + StatsHolder statsHolder = new StatsHolder(dimensionNames); Map> usedDimensionValues = getUsedDimensionValues(statsHolder, 10); populateStats(statsHolder, usedDimensionValues, 100, 10); MultiDimensionCacheStats stats = (MultiDimensionCacheStats) statsHolder.getCacheStats(); @@ -44,39 +37,20 @@ public void testSerialization() throws Exception { MultiDimensionCacheStats deserialized = new MultiDimensionCacheStats(is); assertEquals(stats.dimensionNames, deserialized.dimensionNames); - assertEquals(stats.storeName, deserialized.storeName); - - os = new BytesStreamOutput(); - stats.writeToWithClassName(os); - is = new BytesStreamInput(BytesReference.toBytes(os.bytes())); - CacheStats deserializedViaCacheStats = CacheStats.readFromStreamWithClassName(is); - assertEquals(MultiDimensionCacheStats.class, deserializedViaCacheStats.getClass()); - - assertEquals(stats, deserialized); - assertEquals(stats, deserializedViaCacheStats); - } - - public void testEquals() throws Exception { - List dimensionNames = List.of("dim1", "dim2", "dim3"); - StatsHolder statsHolder = new StatsHolder(dimensionNames, storeName); - StatsHolder differentStoreNameStatsHolder = new StatsHolder(dimensionNames, "nonMatchingStoreName"); - StatsHolder nonMatchingStatsHolder = new StatsHolder(dimensionNames, storeName); - Map> usedDimensionValues = getUsedDimensionValues(statsHolder, 10); - populateStats(List.of(statsHolder, differentStoreNameStatsHolder), usedDimensionValues, 100, 10); - populateStats(nonMatchingStatsHolder, usedDimensionValues, 100, 10); - MultiDimensionCacheStats stats = (MultiDimensionCacheStats) statsHolder.getCacheStats(); - - MultiDimensionCacheStats secondStats = (MultiDimensionCacheStats) statsHolder.getCacheStats(); - assertEquals(stats, secondStats); - MultiDimensionCacheStats nonMatchingStats = (MultiDimensionCacheStats) nonMatchingStatsHolder.getCacheStats(); - assertNotEquals(stats, nonMatchingStats); - MultiDimensionCacheStats differentStoreNameStats = (MultiDimensionCacheStats) differentStoreNameStatsHolder.getCacheStats(); - assertNotEquals(stats, differentStoreNameStats); + List> pathsInOriginal = new ArrayList<>(); + getAllPathsInTree(stats.getStatsRoot(), new ArrayList<>(), pathsInOriginal); + for (List path : pathsInOriginal) { + MultiDimensionCacheStats.MDCSDimensionNode originalNode = getNode(path, stats.statsRoot); + MultiDimensionCacheStats.MDCSDimensionNode deserializedNode = getNode(path, deserialized.statsRoot); + assertNotNull(deserializedNode); + assertEquals(originalNode.getDimensionValue(), deserializedNode.getDimensionValue()); + assertEquals(originalNode.getStats(), deserializedNode.getStats()); + } } public void testAddAndGet() throws Exception { List dimensionNames = List.of("dim1", "dim2", "dim3", "dim4"); - StatsHolder statsHolder = new StatsHolder(dimensionNames, storeName); + StatsHolder statsHolder = new StatsHolder(dimensionNames); Map> usedDimensionValues = getUsedDimensionValues(statsHolder, 10); Map, CacheStatsCounter> expected = populateStats(statsHolder, usedDimensionValues, 1000, 10); MultiDimensionCacheStats stats = (MultiDimensionCacheStats) statsHolder.getCacheStats(); @@ -87,7 +61,7 @@ public void testAddAndGet() throws Exception { CacheStatsCounterSnapshot actualStatsHolder = StatsHolderTests.getNode(dimensionValues, statsHolder.getStatsRoot()) .getStatsSnapshot(); - CacheStatsCounterSnapshot actualCacheStats = getNode(dimensionValues, stats.getStatsRoot()).getStatsSnapshot(); + CacheStatsCounterSnapshot actualCacheStats = getNode(dimensionValues, stats.getStatsRoot()).getStats(); assertEquals(expectedCounter.snapshot(), actualStatsHolder); assertEquals(expectedCounter.snapshot(), actualCacheStats); @@ -111,20 +85,20 @@ public void testAddAndGet() 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(), storeName); + StatsHolder statsHolder = new StatsHolder(List.of()); Map> usedDimensionValues = getUsedDimensionValues(statsHolder, 100); populateStats(statsHolder, usedDimensionValues, 10, 100); MultiDimensionCacheStats stats = (MultiDimensionCacheStats) statsHolder.getCacheStats(); MultiDimensionCacheStats.MDCSDimensionNode statsRoot = stats.getStatsRoot(); assertEquals(0, statsRoot.children.size()); - assertEquals(stats.getTotalStats(), statsRoot.getStatsSnapshot()); + assertEquals(stats.getTotalStats(), statsRoot.getStats()); } public void testAggregateByAllDimensions() throws Exception { // Aggregating with all dimensions as levels should just give us the same values that were in the original map List dimensionNames = List.of("dim1", "dim2", "dim3", "dim4"); - StatsHolder statsHolder = new StatsHolder(dimensionNames, storeName); + StatsHolder statsHolder = new StatsHolder(dimensionNames); Map> usedDimensionValues = getUsedDimensionValues(statsHolder, 10); Map, CacheStatsCounter> expected = populateStats(statsHolder, usedDimensionValues, 1000, 10); MultiDimensionCacheStats stats = (MultiDimensionCacheStats) statsHolder.getCacheStats(); @@ -135,14 +109,14 @@ public void testAggregateByAllDimensions() throws Exception { for (String dimValue : expectedEntry.getKey()) { dimensionValues.add(dimValue); } - assertEquals(expectedEntry.getValue().snapshot(), getNode(dimensionValues, aggregated).getStatsSnapshot()); + assertEquals(expectedEntry.getValue().snapshot(), getNode(dimensionValues, aggregated).getStats()); } assertSumOfChildrenStats(aggregated); } public void testAggregateBySomeDimensions() throws Exception { List dimensionNames = List.of("dim1", "dim2", "dim3", "dim4"); - StatsHolder statsHolder = new StatsHolder(dimensionNames, storeName); + StatsHolder statsHolder = new StatsHolder(dimensionNames); Map> usedDimensionValues = getUsedDimensionValues(statsHolder, 10); Map, CacheStatsCounter> expected = populateStats(statsHolder, usedDimensionValues, 1000, 10); MultiDimensionCacheStats stats = (MultiDimensionCacheStats) statsHolder.getCacheStats(); @@ -168,81 +142,13 @@ public void testAggregateBySomeDimensions() throws Exception { expectedCounter.add(expected.get(expectedDims)); } } - assertEquals(expectedCounter.snapshot(), aggEntry.getValue().getStatsSnapshot()); + assertEquals(expectedCounter.snapshot(), aggEntry.getValue().getStats()); } assertSumOfChildrenStats(aggregated); } } } - public void testXContentForLevels() throws Exception { - List dimensionNames = List.of("A", "B", "C"); - - StatsHolder statsHolder = new StatsHolder(dimensionNames, storeName); - StatsHolderTests.populateStatsHolderFromStatsValueMap(statsHolder, Map.of( - List.of("A1", "B1", "C1"), new CacheStatsCounter(1, 1, 1, 1, 1), - List.of("A1", "B1", "C2"), new CacheStatsCounter(2, 2, 2, 2, 2), - List.of("A1", "B2", "C1"), new CacheStatsCounter(3, 3, 3, 3, 3), - List.of("A2", "B1", "C3"), new CacheStatsCounter(4, 4, 4, 4, 4) - )); - MultiDimensionCacheStats stats = (MultiDimensionCacheStats) statsHolder.getCacheStats(); - - XContentBuilder builder = XContentFactory.jsonBuilder(); - ToXContent.Params params = ToXContent.EMPTY_PARAMS; - - builder.startObject(); - stats.toXContentForLevels(builder, params, List.of("A", "B", "C")); - builder.endObject(); - String resultString = builder.toString(); - Map result = XContentHelper.convertToMap(MediaTypeRegistry.JSON.xContent(), resultString, true); - - Map> fieldNamesMap = Map.of( - CacheStatsCounterSnapshot.Fields.MEMORY_SIZE_IN_BYTES, - (counter, value) -> counter.sizeInBytes.inc(value), - CacheStatsCounterSnapshot.Fields.EVICTIONS, - (counter, value) -> counter.evictions.inc(value), - CacheStatsCounterSnapshot.Fields.HIT_COUNT, - (counter, value) -> counter.hits.inc(value), - CacheStatsCounterSnapshot.Fields.MISS_COUNT, - (counter, value) -> counter.misses.inc(value), - CacheStatsCounterSnapshot.Fields.ENTRIES, - (counter, value) -> counter.entries.inc(value) - ); - - Map, MultiDimensionCacheStats.MDCSDimensionNode> leafNodes = getAllLeafNodes(stats.getStatsRoot()); - for (Map.Entry, MultiDimensionCacheStats.MDCSDimensionNode> entry : leafNodes.entrySet()) { - List xContentKeys = new ArrayList<>(); - for (int i = 0; i < dimensionNames.size(); i++) { - xContentKeys.add(dimensionNames.get(i)); - xContentKeys.add(entry.getKey().get(i)); - } - CacheStatsCounter counterFromXContent = new CacheStatsCounter(); - - for (Map.Entry> fieldNamesEntry : fieldNamesMap.entrySet()) { - List fullXContentKeys = new ArrayList<>(xContentKeys); - fullXContentKeys.add(fieldNamesEntry.getKey()); - int valueInXContent = (int) getValueFromNestedXContentMap(result, fullXContentKeys); - BiConsumer incrementer = fieldNamesEntry.getValue(); - incrementer.accept(counterFromXContent, valueInXContent); - } - - CacheStatsCounterSnapshot expected = entry.getValue().getStatsSnapshot(); - assertEquals(counterFromXContent.snapshot(), expected); - } - } - - public static Object getValueFromNestedXContentMap(Map xContentMap, List keys) { - Map current = xContentMap; - for (int i = 0; i < keys.size() - 1; i++) { - Object next = current.get(keys.get(i)); - if (next == null) { - return null; - } - current = (Map) next; - } - return current.get(keys.get(keys.size() - 1)); - } - // Get a map from the list of dimension values to the corresponding leaf node. private Map, MultiDimensionCacheStats.MDCSDimensionNode> getAllLeafNodes(MultiDimensionCacheStats.MDCSDimensionNode root) { Map, MultiDimensionCacheStats.MDCSDimensionNode> result = new HashMap<>(); @@ -270,9 +176,9 @@ private void assertSumOfChildrenStats(MultiDimensionCacheStats.MDCSDimensionNode if (current.hasChildren()) { CacheStatsCounter expectedTotal = new CacheStatsCounter(); for (MultiDimensionCacheStats.MDCSDimensionNode child : current.children.values()) { - expectedTotal.add(child.getStatsSnapshot()); + expectedTotal.add(child.getStats()); } - assertEquals(expectedTotal.snapshot(), current.getStatsSnapshot()); + assertEquals(expectedTotal.snapshot(), current.getStats()); for (MultiDimensionCacheStats.MDCSDimensionNode child : current.children.values()) { assertSumOfChildrenStats(child); } @@ -296,47 +202,52 @@ static Map, CacheStatsCounter> populateStats( Map> usedDimensionValues, int numDistinctValuePairs, int numRepetitionsPerValue - ) throws InterruptedException { - return populateStats(List.of(statsHolder), usedDimensionValues, numDistinctValuePairs, numRepetitionsPerValue); - } - - static Map, CacheStatsCounter> populateStats( - List statsHolders, - Map> usedDimensionValues, - int numDistinctValuePairs, - int numRepetitionsPerValue ) throws InterruptedException { Map, CacheStatsCounter> expected = new ConcurrentHashMap<>(); - for (StatsHolder statsHolder : statsHolders) { - assertEquals(statsHolders.get(0).getDimensionNames(), statsHolder.getDimensionNames()); - } 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(statsHolders.get(0).getDimensionNames(), usedDimensionValues, true, rand)); + dimensionsForThreads.add(getRandomDimList(statsHolder.getDimensionNames(), usedDimensionValues, true, rand)); int finalI = i; threads[i] = new Thread(() -> { - Random threadRand = Randomness.get(); + Random threadRand = Randomness.get(); // TODO: This always has the same seed for each thread, causing only 1 set of values List dimensions = dimensionsForThreads.get(finalI); expected.computeIfAbsent(dimensions, (key) -> new CacheStatsCounter()); - for (StatsHolder statsHolder : statsHolders) { - for (int j = 0; j < numRepetitionsPerValue; j++) { - CacheStatsCounter statsToInc = new CacheStatsCounter( - threadRand.nextInt(10), - threadRand.nextInt(10), - threadRand.nextInt(10), - threadRand.nextInt(5000), - threadRand.nextInt(10) - ); - expected.get(dimensions).hits.inc(statsToInc.getHits()); - expected.get(dimensions).misses.inc(statsToInc.getMisses()); - 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)); + + for (int j = 0; j < numRepetitionsPerValue; j++) { + int numHitIncrements = threadRand.nextInt(10); + for (int k = 0; k < numHitIncrements; k++) { + statsHolder.incrementHits(dimensions); + expected.get(dimensions).hits.inc(); + } + int numMissIncrements = threadRand.nextInt(10); + for (int k = 0; k < numMissIncrements; k++) { + statsHolder.incrementMisses(dimensions); + expected.get(dimensions).misses.inc(); + } + int numEvictionIncrements = threadRand.nextInt(10); + for (int k = 0; k < numEvictionIncrements; k++) { + statsHolder.incrementEvictions(dimensions); + expected.get(dimensions).evictions.inc(); + } + int numMemorySizeIncrements = threadRand.nextInt(10); + for (int k = 0; k < numMemorySizeIncrements; k++) { + long memIncrementAmount = threadRand.nextInt(5000); + statsHolder.incrementSizeInBytes(dimensions, memIncrementAmount); + expected.get(dimensions).sizeInBytes.inc(memIncrementAmount); + } + int numEntryIncrements = threadRand.nextInt(9) + 1; + for (int k = 0; k < numEntryIncrements; k++) { + statsHolder.incrementEntries(dimensions); + expected.get(dimensions).entries.inc(); + } + int numEntryDecrements = threadRand.nextInt(numEntryIncrements); + for (int k = 0; k < numEntryDecrements; k++) { + statsHolder.decrementEntries(dimensions); + expected.get(dimensions).entries.dec(); } } countDownLatch.countDown(); @@ -366,6 +277,22 @@ private static List getRandomDimList( return result; } + private void getAllPathsInTree( + MultiDimensionCacheStats.MDCSDimensionNode currentNode, + List pathToCurrentNode, + List> allPaths + ) { + allPaths.add(pathToCurrentNode); + if (currentNode.getChildren() != null && !currentNode.getChildren().isEmpty()) { + // not a leaf node + for (MultiDimensionCacheStats.MDCSDimensionNode child : currentNode.getChildren().values()) { + List pathToChild = new ArrayList<>(pathToCurrentNode); + pathToChild.add(child.getDimensionValue()); + getAllPathsInTree(child, pathToChild, allPaths); + } + } + } + private MultiDimensionCacheStats.MDCSDimensionNode getNode( List dimensionValues, MultiDimensionCacheStats.MDCSDimensionNode root diff --git a/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java b/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java index dba3846fa4630..05e5851ce9a50 100644 --- a/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java +++ b/server/src/test/java/org/opensearch/common/cache/stats/StatsHolderTests.java @@ -22,11 +22,10 @@ public class StatsHolderTests extends OpenSearchTestCase { // Since StatsHolder does not expose getter methods for aggregating stats, // we test the incrementing functionality in combination with MultiDimensionCacheStats, // in MultiDimensionCacheStatsTests.java. - private final String storeName = "dummy_store"; public void testReset() throws Exception { List dimensionNames = List.of("dim1", "dim2"); - StatsHolder statsHolder = new StatsHolder(dimensionNames, storeName); + StatsHolder statsHolder = new StatsHolder(dimensionNames); Map> usedDimensionValues = getUsedDimensionValues(statsHolder, 10); Map, CacheStatsCounter> expected = populateStats(statsHolder, usedDimensionValues, 100, 10); @@ -45,7 +44,7 @@ public void testReset() throws Exception { public void testDropStatsForDimensions() throws Exception { List dimensionNames = List.of("dim1", "dim2"); - StatsHolder statsHolder = new StatsHolder(dimensionNames, storeName); + StatsHolder statsHolder = new StatsHolder(dimensionNames); // Create stats for the following dimension sets List> populatedStats = List.of(List.of("A1", "B1"), List.of("A2", "B2"), List.of("A2", "B3")); @@ -81,7 +80,7 @@ public void testDropStatsForDimensions() throws Exception { public void testCount() throws Exception { List dimensionNames = List.of("dim1", "dim2"); - StatsHolder statsHolder = new StatsHolder(dimensionNames, storeName); + StatsHolder statsHolder = new StatsHolder(dimensionNames); Map> usedDimensionValues = getUsedDimensionValues(statsHolder, 10); Map, CacheStatsCounter> expected = populateStats(statsHolder, usedDimensionValues, 100, 10); @@ -94,7 +93,7 @@ public void testCount() throws Exception { public void testConcurrentRemoval() throws Exception { List dimensionNames = List.of("dim1", "dim2"); - StatsHolder statsHolder = new StatsHolder(dimensionNames, storeName); + StatsHolder statsHolder = new StatsHolder(dimensionNames); // Create stats for the following dimension sets List> populatedStats = List.of(List.of("A1", "B1"), List.of("A2", "B2"), List.of("A2", "B3")); @@ -152,24 +151,4 @@ static DimensionNode getNode(List dimensionValues, DimensionNode root) { } return current; } - - static void populateStatsHolderFromStatsValueMap(StatsHolder statsHolder, Map, CacheStatsCounter> statsMap) { - for (Map.Entry, CacheStatsCounter> entry : statsMap.entrySet()) { - CacheStatsCounter stats = entry.getValue(); - List dims = entry.getKey(); - for (int i = 0; i < stats.getHits(); i++) { - statsHolder.incrementHits(dims); - } - for (int i = 0; i < stats.getMisses(); i++) { - statsHolder.incrementMisses(dims); - } - for (int i = 0; i < stats.getEvictions(); i++) { - statsHolder.incrementEvictions(dims); - } - statsHolder.incrementSizeInBytes(dims, stats.getSizeInBytes()); - for (int i = 0; i < stats.getEntries(); i++) { - statsHolder.incrementEntries(dims); - } - } - } } 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 1c6017aa9fb0c..84d4c823e640b 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 @@ -8,31 +8,20 @@ package org.opensearch.common.cache.store; -import org.opensearch.common.Randomness; import org.opensearch.common.cache.CacheType; import org.opensearch.common.cache.ICache; import org.opensearch.common.cache.ICacheKey; import org.opensearch.common.cache.LoadAwareCacheLoader; import org.opensearch.common.cache.RemovalListener; import org.opensearch.common.cache.RemovalNotification; -import org.opensearch.common.cache.stats.CacheStats; -import org.opensearch.common.cache.stats.MultiDimensionCacheStatsTests; import org.opensearch.common.cache.store.config.CacheConfig; import org.opensearch.common.cache.store.settings.OpenSearchOnHeapCacheSettings; import org.opensearch.common.metrics.CounterMetric; import org.opensearch.common.settings.Settings; -import org.opensearch.common.xcontent.XContentFactory; -import org.opensearch.common.xcontent.XContentHelper; -import org.opensearch.core.xcontent.MediaTypeRegistry; -import org.opensearch.core.xcontent.ToXContent; -import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.test.OpenSearchTestCase; -import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.Map; -import java.util.Random; import java.util.UUID; import static org.opensearch.common.cache.store.settings.OpenSearchOnHeapCacheSettings.MAXIMUM_SIZE_IN_BYTES_KEY; @@ -107,73 +96,6 @@ private OpenSearchOnHeapCache getCache(int maxSizeKeys, MockRemo return (OpenSearchOnHeapCache) onHeapCacheFactory.create(cacheConfig, CacheType.INDICES_REQUEST_CACHE, null); } - public void testInvalidateWithDropDimensions() throws Exception { - MockRemovalListener listener = new MockRemovalListener<>(); - int maxKeys = 50; - OpenSearchOnHeapCache cache = getCache(maxKeys, listener); - - List> keysAdded = new ArrayList<>(); - - for (int i = 0; i < maxKeys - 5; i++) { - ICacheKey key = new ICacheKey<>(UUID.randomUUID().toString(), getRandomDimensions()); - keysAdded.add(key); - cache.computeIfAbsent(key, getLoadAwareCacheLoader()); - } - - ICacheKey keyToDrop = keysAdded.get(0); - - Map xContentMap = getStatsXContentMap(cache.stats(), dimensionNames); - List xContentMapKeys = getXContentMapKeys(keyToDrop, dimensionNames); - Map individualSnapshotMap = (Map) MultiDimensionCacheStatsTests.getValueFromNestedXContentMap( - xContentMap, - xContentMapKeys - ); - assertNotNull(individualSnapshotMap); - assertEquals(5, individualSnapshotMap.size()); // Assert all 5 stats are present and not null - for (Map.Entry entry : individualSnapshotMap.entrySet()) { - Integer value = (Integer) entry.getValue(); - assertNotNull(value); - } - - keyToDrop.setDropStatsForDimensions(true); - cache.invalidate(keyToDrop); - - // Now assert the stats are gone for any key that has this combination of dimensions, but still there otherwise - xContentMap = getStatsXContentMap(cache.stats(), dimensionNames); - for (ICacheKey keyAdded : keysAdded) { - xContentMapKeys = getXContentMapKeys(keyAdded, dimensionNames); - individualSnapshotMap = (Map) MultiDimensionCacheStatsTests.getValueFromNestedXContentMap( - xContentMap, - xContentMapKeys - ); - if (keyAdded.dimensions.equals(keyToDrop.dimensions)) { - assertNull(individualSnapshotMap); - } else { - assertNotNull(individualSnapshotMap); - } - } - } - - private List getXContentMapKeys(ICacheKey iCacheKey, List dimensionNames) { - List result = new ArrayList<>(); - assert iCacheKey.dimensions.size() == dimensionNames.size(); - for (int i = 0; i < dimensionNames.size(); i++) { - result.add(dimensionNames.get(i)); - result.add(iCacheKey.dimensions.get(i)); - } - return result; - } - - private List getRandomDimensions() { - Random rand = Randomness.get(); - int bound = 3; - List result = new ArrayList<>(); - for (String dimName : dimensionNames) { - result.add(String.valueOf(rand.nextInt(bound))); - } - return result; - } - private static class MockRemovalListener implements RemovalListener, V> { CounterMetric numRemovals; @@ -187,20 +109,6 @@ public void onRemoval(RemovalNotification, V> notification) { } } - // Public as this is used in other tests as well - public static Map getStatsXContentMap(CacheStats cacheStats, List levels) throws IOException { - XContentBuilder builder = XContentFactory.jsonBuilder(); - Map paramMap = Map.of("level", String.join(",", levels)); - ToXContent.Params params = new ToXContent.MapParams(paramMap); - - builder.startObject(); - cacheStats.toXContent(builder, params); - builder.endObject(); - - String resultString = builder.toString(); - return XContentHelper.convertToMap(MediaTypeRegistry.JSON.xContent(), resultString, true); - } - private ICacheKey getICacheKey(String key) { List dims = new ArrayList<>(); for (String dimName : dimensionNames) { diff --git a/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java b/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java index 9d4d2ba4aa52d..6143eeb5f13e4 100644 --- a/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java +++ b/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java @@ -45,15 +45,12 @@ import org.apache.lucene.search.TopDocs; import org.apache.lucene.store.Directory; import org.apache.lucene.util.BytesRef; -import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.CheckedSupplier; import org.opensearch.common.cache.ICacheKey; import org.opensearch.common.cache.RemovalNotification; 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.MultiDimensionCacheStatsTests; -import org.opensearch.common.cache.store.OpenSearchOnHeapCacheTests; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.lucene.index.OpenSearchDirectoryReader; import org.opensearch.common.settings.Settings; @@ -73,7 +70,6 @@ import org.opensearch.index.query.TermQueryBuilder; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.IndexShardState; -import org.opensearch.index.shard.ShardNotFoundException; import org.opensearch.node.Node; import org.opensearch.test.OpenSearchSingleNodeTestCase; import org.opensearch.threadpool.ThreadPool; @@ -81,15 +77,11 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.List; -import java.util.Map; import java.util.Optional; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; -import static org.opensearch.indices.IndicesRequestCache.INDEX_DIMENSION_NAME; import static org.opensearch.indices.IndicesRequestCache.INDICES_REQUEST_CACHE_STALENESS_THRESHOLD_SETTING; -import static org.opensearch.indices.IndicesRequestCache.SHARD_ID_DIMENSION_NAME; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -761,125 +753,6 @@ public void testCacheCleanupBasedOnStaleThreshold_StalenessLesserThanThreshold() terminate(threadPool); } - public void testClosingIndexWipesStats() throws Exception { - IndicesService indicesService = getInstanceFromNode(IndicesService.class); - // Create two indices each with multiple shards - int numShards = 3; - Settings indexSettings = Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, numShards).build(); - String indexToKeepName = "test"; - String indexToCloseName = "test2"; - IndexService indexToKeep = createIndex(indexToKeepName, indexSettings); - IndexService indexToClose = createIndex(indexToCloseName, indexSettings); - for (int i = 0; i < numShards; i++) { - // Check we can get all the shards we expect - assertNotNull(indexToKeep.getShard(i)); - assertNotNull(indexToClose.getShard(i)); - } - ThreadPool threadPool = getThreadPool(); - Settings settings = Settings.builder().put(INDICES_REQUEST_CACHE_STALENESS_THRESHOLD_SETTING.getKey(), "0.001%").build(); - IndicesRequestCache cache = new IndicesRequestCache(settings, (shardId -> { - IndexService indexService = null; - try { - indexService = indicesService.indexServiceSafe(shardId.getIndex()); - } catch (IndexNotFoundException ex) { - return Optional.empty(); - } - try { - return Optional.of(new IndicesService.IndexShardCacheEntity(indexService.getShard(shardId.id()))); - } catch (ShardNotFoundException ex) { - return Optional.empty(); - } - }), new CacheModule(new ArrayList<>(), Settings.EMPTY).getCacheService(), threadPool); - Directory dir = newDirectory(); - IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); - - writer.addDocument(newDoc(0, "foo")); - TermQueryBuilder termQuery = new TermQueryBuilder("id", "0"); - BytesReference termBytes = XContentHelper.toXContent(termQuery, MediaTypeRegistry.JSON, false); - if (randomBoolean()) { - writer.flush(); - IOUtils.close(writer); - writer = new IndexWriter(dir, newIndexWriterConfig()); - } - writer.updateDocument(new Term("id", "0"), newDoc(0, "bar")); - DirectoryReader secondReader = OpenSearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "bar", 1)); - - List readersToClose = new ArrayList<>(); - List readersToKeep = new ArrayList<>(); - // Put entries into the cache for each shard - for (IndexService indexService : new IndexService[] { indexToKeep, indexToClose }) { - for (int i = 0; i < numShards; i++) { - IndexShard indexShard = indexService.getShard(i); - IndicesService.IndexShardCacheEntity entity = new IndicesService.IndexShardCacheEntity(indexShard); - DirectoryReader reader = OpenSearchDirectoryReader.wrap(DirectoryReader.open(writer), indexShard.shardId()); - if (indexService == indexToClose) { - readersToClose.add(reader); - } else { - readersToKeep.add(reader); - } - Loader loader = new Loader(reader, 0); - cache.getOrCompute(entity, loader, reader, termBytes); - } - } - - // Check resulting stats - List dimensionNames = List.of(INDEX_DIMENSION_NAME, SHARD_ID_DIMENSION_NAME); - Map xContentMap = OpenSearchOnHeapCacheTests.getStatsXContentMap(cache.getCacheStats(), dimensionNames); - List> initialXContentPaths = new ArrayList<>(); - for (IndexService indexService : new IndexService[] { indexToKeep, indexToClose }) { - for (int i = 0; i < numShards; i++) { - ShardId shardId = indexService.getShard(i).shardId(); - List xContentPath = List.of( - INDEX_DIMENSION_NAME, - shardId.getIndexName(), - SHARD_ID_DIMENSION_NAME, - shardId.toString() - ); - initialXContentPaths.add(xContentPath); - Map individualSnapshotMap = (Map) MultiDimensionCacheStatsTests - .getValueFromNestedXContentMap(xContentMap, xContentPath); - assertNotNull(individualSnapshotMap); - // check the values are not empty by confirming entries != 0, this should always be true since the missed value is loaded - // into the cache - assertNotEquals(0, (int) individualSnapshotMap.get("entries")); - } - } - - // Delete an index - indexToClose.close("test_deletion", true); - // This actually closes the shards associated with the readers, which is necessary for cache cleanup logic - // In this UT, manually close the readers as well; could not figure out how to connect all this up in a UT so that - // we could get readers that were properly connected to an index's directory - for (DirectoryReader reader : readersToClose) { - IOUtils.close(reader); - } - // Trigger cache cleanup - cache.cacheCleanupManager.cleanCache(); - - // Now stats for the closed index should be gone - xContentMap = OpenSearchOnHeapCacheTests.getStatsXContentMap(cache.getCacheStats(), dimensionNames); - for (List path : initialXContentPaths) { - Map individualSnapshotMap = (Map) MultiDimensionCacheStatsTests.getValueFromNestedXContentMap( - xContentMap, - path - ); - if (path.get(1).equals(indexToCloseName)) { - assertNull(individualSnapshotMap); - } else { - assertNotNull(individualSnapshotMap); - // check the values are not empty by confirming entries != 0, this should always be true since the missed value is loaded - // into the cache - assertNotEquals(0, (int) individualSnapshotMap.get("entries")); - } - } - - for (DirectoryReader reader : readersToKeep) { - IOUtils.close(reader); - } - IOUtils.close(secondReader, writer, dir, cache); - terminate(threadPool); - } - public void testEviction() throws Exception { final ByteSizeValue size; { diff --git a/test/framework/src/main/java/org/opensearch/cluster/MockInternalClusterInfoService.java b/test/framework/src/main/java/org/opensearch/cluster/MockInternalClusterInfoService.java index 35ca5d80aeb4e..1ad6083074025 100644 --- a/test/framework/src/main/java/org/opensearch/cluster/MockInternalClusterInfoService.java +++ b/test/framework/src/main/java/org/opensearch/cluster/MockInternalClusterInfoService.java @@ -124,8 +124,7 @@ List adjustNodesStats(List nodesStats) { nodeStats.getSearchPipelineStats(), nodeStats.getSegmentReplicationRejectionStats(), nodeStats.getRepositoriesStats(), - nodeStats.getAdmissionControlStats(), - nodeStats.getNodeCacheStats() + nodeStats.getAdmissionControlStats() ); }).collect(Collectors.toList()); } diff --git a/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java index ca80c65e58522..c2b964aa96212 100644 --- a/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java @@ -2736,7 +2736,6 @@ public void ensureEstimatedStats() { false, false, false, - false, false ); assertThat( From 585e4cb6a0e3a52393f008a7aa2d7a71c4066139 Mon Sep 17 00:00:00 2001 From: Peter Alfonsi Date: Wed, 10 Apr 2024 15:25:32 -0700 Subject: [PATCH 17/17] Simplified tsc stats testing logic Signed-off-by: Peter Alfonsi --- .../common/tier/TieredSpilloverCache.java | 10 ++- .../tier/TieredSpilloverCacheTests.java | 82 +++++++------------ .../opensearch/common/cache/ICacheKey.java | 1 - .../common/cache/stats/StatsHolder.java | 4 - 4 files changed, 35 insertions(+), 62 deletions(-) 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 34f5d60b3b987..8ef72a467a14d 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 @@ -37,8 +37,8 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Function; -import java.util.function.ToLongBiFunction; import java.util.function.Predicate; +import java.util.function.ToLongBiFunction; /** * This cache spillover the evicted items from heap tier to disk tier. All the new items are first cached on heap @@ -239,7 +239,8 @@ private Function, V> getValueFromTieredCache() { try (ReleasableLock ignore = readLock.acquire()) { for (Tuple, String> pair : cacheAndTierValueList) { V value = pair.v1().get(key); - List dimensionValues = addTierValueToDimensionValues(key.dimensions, pair.v2()); // Get the tier value corresponding to this cache + List dimensionValues = addTierValueToDimensionValues(key.dimensions, pair.v2()); // Get the tier value + // corresponding to this cache if (value != null) { statsHolder.incrementHits(dimensionValues); return value; @@ -268,7 +269,8 @@ void handleRemovalFromHeapTier(RemovalNotification, V> notification } else { - // If the removal was for another reason, send this notification to the TSC's removal listener, as the value is leaving the TSC entirely + // If the removal was for another reason, 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()); @@ -324,9 +326,11 @@ private List addTierValueToDimensionValues(List initialDimension */ 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); 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 46018b8ee8ab1..d01bff684b277 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 @@ -16,8 +16,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.CacheStats; import org.opensearch.common.cache.stats.CacheStatsCounterSnapshot; +import org.opensearch.common.cache.stats.MultiDimensionCacheStats; import org.opensearch.common.cache.store.OpenSearchOnHeapCache; import org.opensearch.common.cache.store.config.CacheConfig; import org.opensearch.common.cache.store.settings.OpenSearchOnHeapCacheSettings; @@ -26,11 +26,6 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.FeatureFlags; -import org.opensearch.common.xcontent.XContentFactory; -import org.opensearch.common.xcontent.XContentHelper; -import org.opensearch.core.xcontent.MediaTypeRegistry; -import org.opensearch.core.xcontent.ToXContent; -import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.test.OpenSearchTestCase; import java.io.IOException; @@ -47,9 +42,9 @@ import java.util.function.Function; import java.util.function.Predicate; -import static org.opensearch.common.cache.store.settings.OpenSearchOnHeapCacheSettings.MAXIMUM_SIZE_IN_BYTES_KEY; -import static org.opensearch.cache.common.tier.TieredSpilloverCache.TIER_DIMENSION_VALUE_ON_HEAP; import static org.opensearch.cache.common.tier.TieredSpilloverCache.TIER_DIMENSION_VALUE_DISK; +import static org.opensearch.cache.common.tier.TieredSpilloverCache.TIER_DIMENSION_VALUE_ON_HEAP; +import static org.opensearch.common.cache.store.settings.OpenSearchOnHeapCacheSettings.MAXIMUM_SIZE_IN_BYTES_KEY; public class TieredSpilloverCacheTests extends OpenSearchTestCase { static final List dimensionNames = List.of("dim1", "dim2", "dim3"); @@ -417,7 +412,10 @@ public void testComputeIfAbsentWithEvictionsFromTieredCache() throws Exception { 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 + getEntriesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK), getEvictionsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals( + evictions + getEntriesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK), + getEvictionsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP) + ); } public void testGetAndCount() throws Exception { @@ -1188,61 +1186,37 @@ private TieredSpilloverCache intializeTieredSpilloverCache( } // Helper functions for extracting tier aggregated stats. - private static int getHitsForTier(TieredSpilloverCache tsc, String tierValue) throws IOException { - return getStatsValueForTier(tsc, tierValue, CacheStatsCounterSnapshot.Fields.HIT_COUNT); - } - - private static int getMissesForTier(TieredSpilloverCache tsc, String tierValue) throws IOException { - return getStatsValueForTier(tsc, tierValue, CacheStatsCounterSnapshot.Fields.MISS_COUNT); + private long getHitsForTier(TieredSpilloverCache tsc, String tierValue) throws IOException { + return getStatsSnapshotForTier(tsc, tierValue).getHits(); } - private static int getEvictionsForTier(TieredSpilloverCache tsc, String tierValue) throws IOException { - return getStatsValueForTier(tsc, tierValue, CacheStatsCounterSnapshot.Fields.EVICTIONS); + private long getMissesForTier(TieredSpilloverCache tsc, String tierValue) throws IOException { + return getStatsSnapshotForTier(tsc, tierValue).getMisses(); } - private static int getSizeInBytesForTier(TieredSpilloverCache tsc, String tierValue) throws IOException { - return getStatsValueForTier(tsc, tierValue, CacheStatsCounterSnapshot.Fields.MEMORY_SIZE_IN_BYTES); + private long getEvictionsForTier(TieredSpilloverCache tsc, String tierValue) throws IOException { + return getStatsSnapshotForTier(tsc, tierValue).getEvictions(); } - private static int getEntriesForTier(TieredSpilloverCache tsc, String tierValue) throws IOException { - return getStatsValueForTier(tsc, tierValue, CacheStatsCounterSnapshot.Fields.ENTRIES); + private long getSizeInBytesForTier(TieredSpilloverCache tsc, String tierValue) throws IOException { + return getStatsSnapshotForTier(tsc, tierValue).getSizeInBytes(); } - private static int getStatsValueForTier(TieredSpilloverCache tsc, String tierValue, String fieldName) throws IOException { - CacheStats cacheStats = tsc.stats(); - Map aggregatedXContentMap = getStatsXContentMap(cacheStats, List.of(TieredSpilloverCache.TIER_DIMENSION_NAME)); - Object result = getValueFromNestedXContentMap(aggregatedXContentMap, List.of(TieredSpilloverCache.TIER_DIMENSION_NAME, tierValue, fieldName)); - if (result == null) { - // This can happen if no cache actions have happened for this set of dimensions yet - return 0; - } - return (int) result; + private long getEntriesForTier(TieredSpilloverCache tsc, String tierValue) throws IOException { + return getStatsSnapshotForTier(tsc, tierValue).getEntries(); } - // Duplicated from OpenSearchOnHeapCacheTests.java; we can't add a dependency on server.test - private static Map getStatsXContentMap(CacheStats cacheStats, List levels) throws IOException { - XContentBuilder builder = XContentFactory.jsonBuilder(); - Map paramMap = Map.of("level", String.join(",", levels)); - ToXContent.Params params = new ToXContent.MapParams(paramMap); - - builder.startObject(); - cacheStats.toXContent(builder, params); - builder.endObject(); - - String resultString = builder.toString(); - return XContentHelper.convertToMap(MediaTypeRegistry.JSON.xContent(), resultString, true); - } - - // Duplicated from MultiDimensionCacheStatsTests.java; we can't add a dependency on server.test - private static Object getValueFromNestedXContentMap(Map xContentMap, List keys) { - Map current = xContentMap; - for (int i = 0; i < keys.size() - 1; i++) { - Object next = current.get(keys.get(i)); - if (next == null) { - return null; - } - current = (Map) next; + private CacheStatsCounterSnapshot getStatsSnapshotForTier(TieredSpilloverCache tsc, String tierValue) throws IOException { + MultiDimensionCacheStats cacheStats = (MultiDimensionCacheStats) tsc.stats(); + // 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); + CacheStatsCounterSnapshot snapshot = cacheStats.getStatsForDimensionValues(mockDimensions); + if (snapshot == null) { + return new CacheStatsCounterSnapshot(0, 0, 0, 0, 0); // This can happen if no cache actions have happened for this set of + // dimensions yet } - return current.get(keys.get(keys.size() - 1)); + return snapshot; } } 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 0eb34952e78f1..e1aa9b1c5466c 100644 --- a/server/src/main/java/org/opensearch/common/cache/ICacheKey.java +++ b/server/src/main/java/org/opensearch/common/cache/ICacheKey.java @@ -10,7 +10,6 @@ import org.opensearch.common.annotation.ExperimentalApi; -import java.util.Collections; import java.util.List; /** diff --git a/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java b/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java index 81ffdef0ab8d3..271264ce7b71c 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/StatsHolder.java @@ -8,11 +8,7 @@ package org.opensearch.common.cache.stats; -import java.util.ArrayList; -import java.util.Collections; 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;