diff --git a/CHANGELOG.md b/CHANGELOG.md index ff5178b16f1ad..081db6c94d2f9 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -12,6 +12,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Implement WithFieldName interface in ValuesSourceAggregationBuilder & FieldSortBuilder ([#15916](https://github.com/opensearch-project/OpenSearch/pull/15916)) - Add successfulSearchShardIndices in searchRequestContext ([#15967](https://github.com/opensearch-project/OpenSearch/pull/15967)) - Fallback to Remote cluster-state on Term-Version check mismatch - ([#15424](https://github.com/opensearch-project/OpenSearch/pull/15424)) +- [Tiered Caching] Segmented cache changes ([#16047](https://github.com/opensearch-project/OpenSearch/pull/16047)) - Add support for msearch API to pass search pipeline name - ([#15923](https://github.com/opensearch-project/OpenSearch/pull/15923)) - Add success and failure metrics for async shard fetch ([#15976](https://github.com/opensearch-project/OpenSearch/pull/15976)) - [S3 Repository] Change default retry mechanism of s3 clients to Standard Mode ([#15978](https://github.com/opensearch-project/OpenSearch/pull/15978)) diff --git a/modules/cache-common/src/internalClusterTest/java/org/opensearch/cache/common/tier/TieredSpilloverCacheBaseIT.java b/modules/cache-common/src/internalClusterTest/java/org/opensearch/cache/common/tier/TieredSpilloverCacheBaseIT.java new file mode 100644 index 0000000000000..01371ca8eeefb --- /dev/null +++ b/modules/cache-common/src/internalClusterTest/java/org/opensearch/cache/common/tier/TieredSpilloverCacheBaseIT.java @@ -0,0 +1,57 @@ +/* + * 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.CacheType; +import org.opensearch.common.cache.settings.CacheSettings; +import org.opensearch.common.cache.store.OpenSearchOnHeapCache; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.FeatureFlags; +import org.opensearch.test.OpenSearchIntegTestCase; + +public class TieredSpilloverCacheBaseIT extends OpenSearchIntegTestCase { + + public Settings defaultSettings(String onHeapCacheSizeInBytesOrPercentage, int numberOfSegments) { + return Settings.builder() + .put(FeatureFlags.PLUGGABLE_CACHE, "true") + .put( + CacheSettings.getConcreteStoreNameSettingForCacheType(CacheType.INDICES_REQUEST_CACHE).getKey(), + TieredSpilloverCache.TieredSpilloverCacheFactory.TIERED_SPILLOVER_CACHE_NAME + ) + .put( + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_NAME.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), + OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory.NAME + ) + .put( + TieredSpilloverCacheSettings.TIERED_SPILLOVER_DISK_STORE_NAME.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), + MockDiskCache.MockDiskCacheFactory.NAME + ) + .put( + TieredSpilloverCacheSettings.TIERED_SPILLOVER_SEGMENTS.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), + numberOfSegments + ) + .put( + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), + onHeapCacheSizeInBytesOrPercentage + ) + .build(); + } + + public int getNumberOfSegments() { + return randomFrom(1, 2, 4, 8, 16, 64, 128, 256); + } +} diff --git a/modules/cache-common/src/internalClusterTest/java/org/opensearch/cache/common/tier/TieredSpilloverCacheIT.java b/modules/cache-common/src/internalClusterTest/java/org/opensearch/cache/common/tier/TieredSpilloverCacheIT.java index 02be0990eb136..d58e36c036510 100644 --- a/modules/cache-common/src/internalClusterTest/java/org/opensearch/cache/common/tier/TieredSpilloverCacheIT.java +++ b/modules/cache-common/src/internalClusterTest/java/org/opensearch/cache/common/tier/TieredSpilloverCacheIT.java @@ -22,12 +22,8 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.cache.CacheType; import org.opensearch.common.cache.ICache; -import org.opensearch.common.cache.settings.CacheSettings; -import org.opensearch.common.cache.store.OpenSearchOnHeapCache; -import org.opensearch.common.cache.store.settings.OpenSearchOnHeapCacheSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; -import org.opensearch.common.util.FeatureFlags; import org.opensearch.index.cache.request.RequestCacheStats; import org.opensearch.index.query.QueryBuilders; import org.opensearch.indices.IndicesRequestCache; @@ -43,13 +39,15 @@ import java.util.Arrays; import java.util.Collection; import java.util.List; +import java.util.Locale; import java.util.Map; +import java.util.UUID; import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.opensearch.common.cache.store.settings.OpenSearchOnHeapCacheSettings.MAXIMUM_SIZE_IN_BYTES_KEY; +import static org.opensearch.common.cache.settings.CacheSettings.INVALID_SEGMENT_COUNT_EXCEPTION_MESSAGE; import static org.opensearch.indices.IndicesService.INDICES_CACHE_CLEAN_INTERVAL_SETTING; import static org.opensearch.search.aggregations.AggregationBuilders.dateHistogram; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; @@ -58,43 +56,15 @@ import static org.hamcrest.Matchers.greaterThan; @OpenSearchIntegTestCase.ClusterScope(numDataNodes = 0, scope = OpenSearchIntegTestCase.Scope.TEST) -public class TieredSpilloverCacheIT extends OpenSearchIntegTestCase { +public class TieredSpilloverCacheIT extends TieredSpilloverCacheBaseIT { @Override protected Collection> nodePlugins() { return Arrays.asList(TieredSpilloverCachePlugin.class, MockDiskCachePlugin.class); } - static Settings defaultSettings(String onHeapCacheSizeInBytesOrPercentage) { - return Settings.builder() - .put(FeatureFlags.PLUGGABLE_CACHE, "true") - .put( - CacheSettings.getConcreteStoreNameSettingForCacheType(CacheType.INDICES_REQUEST_CACHE).getKey(), - TieredSpilloverCache.TieredSpilloverCacheFactory.TIERED_SPILLOVER_CACHE_NAME - ) - .put( - TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_NAME.getConcreteSettingForNamespace( - CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() - ).getKey(), - OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory.NAME - ) - .put( - TieredSpilloverCacheSettings.TIERED_SPILLOVER_DISK_STORE_NAME.getConcreteSettingForNamespace( - CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() - ).getKey(), - MockDiskCache.MockDiskCacheFactory.NAME - ) - .put( - OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) - .get(MAXIMUM_SIZE_IN_BYTES_KEY) - .getKey(), - onHeapCacheSizeInBytesOrPercentage - ) - .build(); - } - public void testPluginsAreInstalled() { - internalCluster().startNode(Settings.builder().put(defaultSettings("1%")).build()); + internalCluster().startNode(Settings.builder().put(defaultSettings("1%", getNumberOfSegments())).build()); NodesInfoRequest nodesInfoRequest = new NodesInfoRequest(); nodesInfoRequest.addMetric(NodesInfoRequest.Metric.PLUGINS.metricName()); NodesInfoResponse nodesInfoResponse = OpenSearchIntegTestCase.client().admin().cluster().nodesInfo(nodesInfoRequest).actionGet(); @@ -111,7 +81,8 @@ public void testPluginsAreInstalled() { } public void testSanityChecksWithIndicesRequestCache() throws InterruptedException { - internalCluster().startNodes(3, Settings.builder().put(defaultSettings("1%")).build()); + int numberOfSegments = getNumberOfSegments(); + internalCluster().startNodes(3, Settings.builder().put(defaultSettings("1%", numberOfSegments)).build()); Client client = client(); assertAcked( client.admin() @@ -147,9 +118,97 @@ public void testSanityChecksWithIndicesRequestCache() throws InterruptedExceptio ); } + public void testWithDynamicTookTimePolicyWithMultiSegments() throws Exception { + int numberOfSegments = getNumberOfSegments(); + int onHeapCacheSizePerSegmentInBytes = 800; // Per cache entry below is around ~700 bytes, so keeping this + // just a bit higher so that each segment can atleast hold 1 entry. + int onHeapCacheSizeInBytes = onHeapCacheSizePerSegmentInBytes * numberOfSegments; + internalCluster().startNode(Settings.builder().put(defaultSettings(onHeapCacheSizeInBytes + "b", numberOfSegments)).build()); + Client client = client(); + 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) + .put("index.refresh_interval", -1) + ) + .get() + ); + // Set a very high value for took time policy so that no items evicted from onHeap cache are spilled + // to disk. And then hit requests so that few items are cached into cache. + ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest().transientSettings( + Settings.builder() + .put( + TieredSpilloverCacheSettings.TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP.get(CacheType.INDICES_REQUEST_CACHE).getKey(), + new TimeValue(100, TimeUnit.SECONDS) + ) + .build() + ); + assertAcked(internalCluster().client().admin().cluster().updateSettings(updateSettingsRequest).get()); + int numberOfIndexedItems = numberOfSegments + 1; // Best case if all keys are distributed among different + // segment, atleast one of the segment will have 2 entries and we will see evictions. + for (int iterator = 0; iterator < numberOfIndexedItems; iterator++) { + indexRandom(true, client.prepareIndex("index").setSource("k" + iterator, "hello" + iterator)); + } + ensureSearchable("index"); + refreshAndWaitForReplication(); + // Force merge the index to ensure there can be no background merges during the subsequent searches that would invalidate the cache + ForceMergeResponse forceMergeResponse = client.admin().indices().prepareForceMerge("index").setFlush(true).get(); + OpenSearchAssertions.assertAllSuccessful(forceMergeResponse); + long perQuerySizeInCacheInBytes = -1; + for (int iterator = 0; iterator < numberOfIndexedItems; iterator++) { + SearchResponse resp = client.prepareSearch("index") + .setRequestCache(true) + .setQuery(QueryBuilders.termQuery("k" + iterator, "hello" + iterator)) + .get(); + if (perQuerySizeInCacheInBytes == -1) { + RequestCacheStats requestCacheStats = getRequestCacheStats(client, "index"); + perQuerySizeInCacheInBytes = requestCacheStats.getMemorySizeInBytes(); + } + assertSearchResponse(resp); + } + RequestCacheStats requestCacheStats = getRequestCacheStats(client, "index"); + // Considering disk cache won't be used due to took time policy having a high value, we expect overall cache + // size to be less than or equal to onHeapCache size. + assertTrue(requestCacheStats.getMemorySizeInBytes() <= onHeapCacheSizeInBytes); + assertEquals(numberOfIndexedItems, requestCacheStats.getMissCount()); + // We should atleast one eviction considering disk cache isn't able to hold anything due to policy. + assertTrue(requestCacheStats.getEvictions() > 0); + assertEquals(0, requestCacheStats.getHitCount()); + long lastEvictionSeen = requestCacheStats.getEvictions(); + + // Decrease took time policy to zero so that disk cache also comes into play. Now we should be able + // to cache all entries. + updateSettingsRequest = new ClusterUpdateSettingsRequest().transientSettings( + Settings.builder() + .put( + TieredSpilloverCacheSettings.TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP.get(CacheType.INDICES_REQUEST_CACHE).getKey(), + new TimeValue(0, TimeUnit.MILLISECONDS) + ) + .build() + ); + assertAcked(internalCluster().client().admin().cluster().updateSettings(updateSettingsRequest).get()); + for (int iterator = 0; iterator < numberOfIndexedItems * 2; iterator++) { + SearchResponse resp = client.prepareSearch("index") + .setRequestCache(true) + .setQuery(QueryBuilders.termQuery(UUID.randomUUID().toString(), UUID.randomUUID().toString())) + .get(); + assertSearchResponse(resp); + } + + requestCacheStats = getRequestCacheStats(client, "index"); + // We shouldn't see any new evictions now. + assertEquals(lastEvictionSeen, requestCacheStats.getEvictions()); + } + public void testWithDynamicTookTimePolicy() throws Exception { int onHeapCacheSizeInBytes = 2000; - internalCluster().startNode(Settings.builder().put(defaultSettings(onHeapCacheSizeInBytes + "b")).build()); + internalCluster().startNode(Settings.builder().put(defaultSettings(onHeapCacheSizeInBytes + "b", 1)).build()); Client client = client(); assertAcked( client.admin() @@ -271,9 +330,10 @@ public void testWithDynamicTookTimePolicy() throws Exception { public void testInvalidationWithIndicesRequestCache() throws Exception { int onHeapCacheSizeInBytes = 2000; + int numberOfSegments = getNumberOfSegments(); internalCluster().startNode( Settings.builder() - .put(defaultSettings(onHeapCacheSizeInBytes + "b")) + .put(defaultSettings(onHeapCacheSizeInBytes + "b", numberOfSegments)) .put(INDICES_CACHE_CLEAN_INTERVAL_SETTING.getKey(), new TimeValue(1)) .build() ); @@ -354,10 +414,11 @@ public void testInvalidationWithIndicesRequestCache() throws Exception { } public void testWithExplicitCacheClear() throws Exception { + int numberOfSegments = getNumberOfSegments(); int onHeapCacheSizeInBytes = 2000; internalCluster().startNode( Settings.builder() - .put(defaultSettings(onHeapCacheSizeInBytes + "b")) + .put(defaultSettings(onHeapCacheSizeInBytes + "b", numberOfSegments)) .put(INDICES_CACHE_CLEAN_INTERVAL_SETTING.getKey(), new TimeValue(1)) .build() ); @@ -426,10 +487,13 @@ public void testWithExplicitCacheClear() throws Exception { } public void testWithDynamicDiskCacheSetting() throws Exception { - int onHeapCacheSizeInBytes = 10; // Keep it low so that all items are cached onto disk. + int numberOfSegments = getNumberOfSegments(); + int onHeapCacheSizeInBytes = randomIntBetween(numberOfSegments + 1, numberOfSegments * 2); // Keep it low so + // that all items are + // cached onto disk. internalCluster().startNode( Settings.builder() - .put(defaultSettings(onHeapCacheSizeInBytes + "b")) + .put(defaultSettings(onHeapCacheSizeInBytes + "b", numberOfSegments)) .put(INDICES_CACHE_CLEAN_INTERVAL_SETTING.getKey(), new TimeValue(1)) .build() ); @@ -540,6 +604,27 @@ public void testWithDynamicDiskCacheSetting() throws Exception { assertEquals(0, lastKnownHitCount - requestCacheStats.getHitCount()); } + public void testWithInvalidSegmentNumberSetting() throws Exception { + int numberOfSegments = getNumberOfSegments(); + int onHeapCacheSizeInBytes = randomIntBetween(numberOfSegments + 1, numberOfSegments * 2); // Keep it low so + // that all items are + // cached onto disk. + assertThrows( + String.format( + Locale.ROOT, + INVALID_SEGMENT_COUNT_EXCEPTION_MESSAGE, + TieredSpilloverCache.TieredSpilloverCacheFactory.TIERED_SPILLOVER_CACHE_NAME + ), + IllegalArgumentException.class, + () -> internalCluster().startNode( + Settings.builder() + .put(defaultSettings(onHeapCacheSizeInBytes + "b", 300)) + .put(INDICES_CACHE_CLEAN_INTERVAL_SETTING.getKey(), new TimeValue(1)) + .build() + ) + ); + } + private RequestCacheStats getRequestCacheStats(Client client, String indexName) { return client.admin().indices().prepareStats(indexName).setRequestCache(true).get().getTotal().getRequestCache(); } @@ -550,7 +635,7 @@ public MockDiskCachePlugin() {} @Override public Map getCacheFactoryMap() { - return Map.of(MockDiskCache.MockDiskCacheFactory.NAME, new MockDiskCache.MockDiskCacheFactory(0, 1000, false)); + return Map.of(MockDiskCache.MockDiskCacheFactory.NAME, new MockDiskCache.MockDiskCacheFactory(0, 10000, false, 1)); } @Override diff --git a/modules/cache-common/src/internalClusterTest/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsIT.java b/modules/cache-common/src/internalClusterTest/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsIT.java index 783b6083e9226..fe6bd7050a8f3 100644 --- a/modules/cache-common/src/internalClusterTest/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsIT.java +++ b/modules/cache-common/src/internalClusterTest/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsIT.java @@ -45,16 +45,16 @@ // Use a single data node to simplify accessing cache stats across different shards. @OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) -public class TieredSpilloverCacheStatsIT extends OpenSearchIntegTestCase { +public class TieredSpilloverCacheStatsIT extends TieredSpilloverCacheBaseIT { @Override protected Collection> nodePlugins() { return Arrays.asList(TieredSpilloverCachePlugin.class, TieredSpilloverCacheIT.MockDiskCachePlugin.class); } - private final String HEAP_CACHE_SIZE_STRING = "10000B"; - private final int HEAP_CACHE_SIZE = 10_000; - private final String index1Name = "index1"; - private final String index2Name = "index2"; + private static final String HEAP_CACHE_SIZE_STRING = "10000B"; + private static final int HEAP_CACHE_SIZE = 10_000; + private static final String index1Name = "index1"; + private static final String index2Name = "index2"; /** * Test aggregating by indices @@ -63,7 +63,7 @@ public void testIndicesLevelAggregation() throws Exception { internalCluster().startNodes( 1, Settings.builder() - .put(TieredSpilloverCacheIT.defaultSettings(HEAP_CACHE_SIZE_STRING)) + .put(defaultSettings(HEAP_CACHE_SIZE_STRING, 1)) .put( TieredSpilloverCacheSettings.TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP.get(CacheType.INDICES_REQUEST_CACHE).getKey(), new TimeValue(0, TimeUnit.SECONDS) @@ -116,7 +116,7 @@ public void testIndicesAndTierLevelAggregation() throws Exception { internalCluster().startNodes( 1, Settings.builder() - .put(TieredSpilloverCacheIT.defaultSettings(HEAP_CACHE_SIZE_STRING)) + .put(defaultSettings(HEAP_CACHE_SIZE_STRING, 1)) .put( TieredSpilloverCacheSettings.TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP.get(CacheType.INDICES_REQUEST_CACHE).getKey(), new TimeValue(0, TimeUnit.SECONDS) @@ -196,7 +196,7 @@ public void testTierLevelAggregation() throws Exception { internalCluster().startNodes( 1, Settings.builder() - .put(TieredSpilloverCacheIT.defaultSettings(HEAP_CACHE_SIZE_STRING)) + .put(defaultSettings(HEAP_CACHE_SIZE_STRING, 1)) .put( TieredSpilloverCacheSettings.TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP.get(CacheType.INDICES_REQUEST_CACHE).getKey(), new TimeValue(0, TimeUnit.SECONDS) @@ -205,7 +205,6 @@ public void testTierLevelAggregation() throws Exception { ); Client client = client(); Map values = setupCacheForAggregationTests(client); - // Get values for tiers alone and check they add correctly across indices ImmutableCacheStatsHolder tiersOnlyStatsHolder = getNodeCacheStatsResult(client, List.of(TIER_DIMENSION_NAME)); ImmutableCacheStats totalHeapExpectedStats = returnNullIfAllZero( @@ -238,7 +237,7 @@ public void testInvalidLevelsAreIgnored() throws Exception { internalCluster().startNodes( 1, Settings.builder() - .put(TieredSpilloverCacheIT.defaultSettings(HEAP_CACHE_SIZE_STRING)) + .put(defaultSettings(HEAP_CACHE_SIZE_STRING, getNumberOfSegments())) .put( TieredSpilloverCacheSettings.TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP.get(CacheType.INDICES_REQUEST_CACHE).getKey(), new TimeValue(0, TimeUnit.SECONDS) @@ -289,7 +288,7 @@ public void testStatsMatchOldApi() throws Exception { internalCluster().startNodes( 1, Settings.builder() - .put(TieredSpilloverCacheIT.defaultSettings(HEAP_CACHE_SIZE_STRING)) + .put(defaultSettings(HEAP_CACHE_SIZE_STRING, getNumberOfSegments())) .put( TieredSpilloverCacheSettings.TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP.get(CacheType.INDICES_REQUEST_CACHE).getKey(), new TimeValue(0, TimeUnit.SECONDS) @@ -342,6 +341,82 @@ public void testStatsMatchOldApi() throws Exception { assertEquals(oldAPIStats.getMemorySizeInBytes(), totalStats.getSizeInBytes()); } + public void testStatsWithMultipleSegments() throws Exception { + int numberOfSegments = randomFrom(2, 4, 8, 16, 64); + int singleSearchSizeApproxUpperBound = 700; // We know this from other tests and manually verifying + int heap_cache_size_per_segment = singleSearchSizeApproxUpperBound * numberOfSegments; // Worst case if all + // keys land up in same segment, it would still be able to accommodate. + internalCluster().startNodes( + 1, + Settings.builder() + .put(defaultSettings(heap_cache_size_per_segment * numberOfSegments + "B", numberOfSegments)) + .put( + TieredSpilloverCacheSettings.TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP.get(CacheType.INDICES_REQUEST_CACHE).getKey(), + new TimeValue(0, TimeUnit.SECONDS) + ) + .build() + ); + Client client = client(); + startIndex(client, index1Name); + // First search one time to calculate item size + searchIndex(client, index1Name, 0); + // get total stats + long singleSearchSize = getTotalStats(client).getSizeInBytes(); + // Now try to hit queries same as number of segments. All these should be able to reside inside onHeap cache. + for (int i = 1; i < numberOfSegments; i++) { + searchIndex(client, index1Name, i); + } + ImmutableCacheStatsHolder allLevelsStatsHolder = getNodeCacheStatsResult( + client, + List.of(IndicesRequestCache.INDEX_DIMENSION_NAME, TIER_DIMENSION_NAME) + ); + ImmutableCacheStats index1OnHeapExpectedStats = returnNullIfAllZero( + new ImmutableCacheStats(0, numberOfSegments, 0, singleSearchSize * numberOfSegments, numberOfSegments) + ); + assertEquals( + index1OnHeapExpectedStats, + allLevelsStatsHolder.getStatsForDimensionValues(List.of(index1Name, TIER_DIMENSION_VALUE_ON_HEAP)) + ); + ImmutableCacheStats index1DiskCacheExpectedStats = returnNullIfAllZero(new ImmutableCacheStats(0, numberOfSegments, 0, 0, 0)); + assertEquals( + index1DiskCacheExpectedStats, + allLevelsStatsHolder.getStatsForDimensionValues(List.of(index1Name, TIER_DIMENSION_VALUE_DISK)) + ); + + // Now fire same queries to get some hits + for (int i = 0; i < numberOfSegments; i++) { + searchIndex(client, index1Name, i); + } + allLevelsStatsHolder = getNodeCacheStatsResult(client, List.of(IndicesRequestCache.INDEX_DIMENSION_NAME, TIER_DIMENSION_NAME)); + index1OnHeapExpectedStats = returnNullIfAllZero( + new ImmutableCacheStats(numberOfSegments, numberOfSegments, 0, singleSearchSize * numberOfSegments, numberOfSegments) + ); + assertEquals( + index1OnHeapExpectedStats, + allLevelsStatsHolder.getStatsForDimensionValues(List.of(index1Name, TIER_DIMENSION_VALUE_ON_HEAP)) + ); + + // Now try to evict from onheap cache by adding numberOfSegments ^ 2 which will guarantee this. + for (int i = numberOfSegments; i < numberOfSegments + numberOfSegments * numberOfSegments; i++) { + searchIndex(client, index1Name, i); + } + allLevelsStatsHolder = getNodeCacheStatsResult(client, List.of(IndicesRequestCache.INDEX_DIMENSION_NAME, TIER_DIMENSION_NAME)); + ImmutableCacheStats onHeapCacheStat = allLevelsStatsHolder.getStatsForDimensionValues( + List.of(index1Name, TIER_DIMENSION_VALUE_ON_HEAP) + ); + // Jut verifying evictions happened as can't fetch the exact number considering we don't have a way to get + // segment number for queries. + assertTrue(onHeapCacheStat.getEvictions() > 0); + ImmutableCacheStats diskCacheStat = allLevelsStatsHolder.getStatsForDimensionValues(List.of(index1Name, TIER_DIMENSION_VALUE_DISK)); + + // Similarly verify items are present on disk cache now + assertEquals(onHeapCacheStat.getEvictions(), diskCacheStat.getItems()); + assertTrue(diskCacheStat.getSizeInBytes() > 0); + assertTrue(diskCacheStat.getMisses() > 0); + assertTrue(diskCacheStat.getHits() == 0); + assertTrue(diskCacheStat.getEvictions() == 0); + } + private void startIndex(Client client, String indexName) throws InterruptedException { assertAcked( client.admin() @@ -373,6 +448,7 @@ private Map setupCacheForAggregationTests(Client client) throws searchIndex(client, index1Name, 0); // get total stats long singleSearchSize = getTotalStats(client).getSizeInBytes(); + int itemsOnHeapAfterTest = HEAP_CACHE_SIZE / (int) singleSearchSize; // As the heap tier evicts, the items on it after the test will // be the same as its max capacity int itemsOnDiskAfterTest = 1 + randomInt(30); // The first one we search (to get the size) always goes to disk @@ -416,7 +492,6 @@ private Map setupCacheForAggregationTests(Client client) throws for (int i = itemsOnDiskAfterTest + itemsOnHeapIndex1AfterTest; i < itemsOnDiskAfterTest + itemsOnHeapAfterTest; i++) { searchIndex(client, index2Name, i); } - // Get some hits on all combinations of indices and tiers for (int i = itemsOnDiskAfterTest; i < itemsOnDiskAfterTest + hitsOnHeapIndex1; i++) { // heap hits for index 1 @@ -499,6 +574,7 @@ private static ImmutableCacheStatsHolder getNodeCacheStatsResult(Client client, .addMetric(NodesStatsRequest.Metric.CACHE_STATS.metricName()) .setIndices(statsFlags) .get(); + // Can always get the first data node as there's only one in this test suite assertEquals(1, nodeStatsResponse.getNodes().size()); NodeCacheStats ncs = nodeStatsResponse.getNodes().get(0).getNodeCacheStats(); 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 f69c56808b2a1..ab5335ca0ca66 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 @@ -34,6 +34,7 @@ import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.NoSuchElementException; import java.util.Objects; @@ -49,8 +50,13 @@ import java.util.function.ToLongBiFunction; import static org.opensearch.cache.common.tier.TieredSpilloverCacheSettings.DISK_CACHE_ENABLED_SETTING_MAP; +import static org.opensearch.cache.common.tier.TieredSpilloverCacheSettings.TIERED_SPILLOVER_DISK_STORE_SIZE; +import static org.opensearch.cache.common.tier.TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE; +import static org.opensearch.cache.common.tier.TieredSpilloverCacheSettings.TIERED_SPILLOVER_SEGMENTS; import static org.opensearch.cache.common.tier.TieredSpilloverCacheStatsHolder.TIER_DIMENSION_VALUE_DISK; import static org.opensearch.cache.common.tier.TieredSpilloverCacheStatsHolder.TIER_DIMENSION_VALUE_ON_HEAP; +import static org.opensearch.common.cache.settings.CacheSettings.INVALID_SEGMENT_COUNT_EXCEPTION_MESSAGE; +import static org.opensearch.common.cache.settings.CacheSettings.VALID_SEGMENT_COUNT_VALUES; /** * This cache spillover the evicted items from heap tier to disk tier. All the new items are first cached on heap @@ -69,29 +75,16 @@ public class TieredSpilloverCache implements ICache { private static final List SPILLOVER_REMOVAL_REASONS = List.of(RemovalReason.EVICTED, RemovalReason.CAPACITY); private static final Logger logger = LogManager.getLogger(TieredSpilloverCache.class); - private final ICache diskCache; - private final ICache onHeapCache; - - // Removal listeners for the individual tiers - private final RemovalListener, V> onDiskRemovalListener; - private final RemovalListener, V> onHeapRemovalListener; - - // Removal listener from the spillover cache as a whole - private final RemovalListener, V> removalListener; + static final String ZERO_SEGMENT_COUNT_EXCEPTION_MESSAGE = "Segment count cannot be less than one for tiered cache"; // In future we want to just read the stats from the individual tiers' statsHolder objects, but this isn't // possible right now because of the way computeIfAbsent is implemented. private final TieredSpilloverCacheStatsHolder statsHolder; - private ToLongBiFunction, V> weigher; private final List dimensionNames; - ReadWriteLock readWriteLock = new ReentrantReadWriteLock(); - ReleasableLock readLock = new ReleasableLock(readWriteLock.readLock()); - ReleasableLock writeLock = new ReleasableLock(readWriteLock.writeLock()); - /** - * Maintains caching tiers in ascending order of cache latency. - */ - private final Map, TierInfo> caches; - private final List> policies; + + private final int numberOfSegments; + + final TieredSpilloverCacheSegment[] tieredSpilloverCacheSegments; /** * This map is used to handle concurrent requests for same key in computeIfAbsent() to ensure we load the value @@ -99,224 +92,525 @@ public class TieredSpilloverCache implements ICache { */ Map, CompletableFuture, V>>> completableFutureMap = new ConcurrentHashMap<>(); + @SuppressWarnings({ "unchecked", "rawtypes" }) 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"); Objects.requireNonNull(builder.cacheConfig, "cache config can't be null"); - Objects.requireNonNull(builder.cacheConfig.getClusterSettings(), "cluster settings can't be null"); - this.removalListener = Objects.requireNonNull(builder.removalListener, "Removal listener can't be null"); - - this.onHeapRemovalListener = new HeapTierRemovalListener(this); - this.onDiskRemovalListener = new DiskTierRemovalListener(this); - this.weigher = Objects.requireNonNull(builder.cacheConfig.getWeigher(), "Weigher can't be null"); - - this.onHeapCache = builder.onHeapCacheFactory.create( - new CacheConfig.Builder().setRemovalListener(onHeapRemovalListener) - .setKeyType(builder.cacheConfig.getKeyType()) - .setValueType(builder.cacheConfig.getValueType()) - .setSettings(builder.cacheConfig.getSettings()) - .setWeigher(builder.cacheConfig.getWeigher()) - .setDimensionNames(builder.cacheConfig.getDimensionNames()) - .setMaxSizeInBytes(builder.cacheConfig.getMaxSizeInBytes()) - .setExpireAfterAccess(builder.cacheConfig.getExpireAfterAccess()) - .setClusterSettings(builder.cacheConfig.getClusterSettings()) - .setStatsTrackingEnabled(false) - .build(), - builder.cacheType, - builder.cacheFactories - - ); - this.diskCache = builder.diskCacheFactory.create( - new CacheConfig.Builder().setRemovalListener(onDiskRemovalListener) - .setKeyType(builder.cacheConfig.getKeyType()) - .setValueType(builder.cacheConfig.getValueType()) - .setSettings(builder.cacheConfig.getSettings()) - .setWeigher(builder.cacheConfig.getWeigher()) - .setKeySerializer(builder.cacheConfig.getKeySerializer()) - .setValueSerializer(builder.cacheConfig.getValueSerializer()) - .setDimensionNames(builder.cacheConfig.getDimensionNames()) - .setStatsTrackingEnabled(false) - .build(), - builder.cacheType, - builder.cacheFactories - ); + Objects.requireNonNull(builder.cacheConfig.getSettings(), "settings can't be null"); + if (builder.numberOfSegments <= 0) { + throw new IllegalArgumentException(ZERO_SEGMENT_COUNT_EXCEPTION_MESSAGE); + } + this.numberOfSegments = builder.numberOfSegments; Boolean isDiskCacheEnabled = DISK_CACHE_ENABLED_SETTING_MAP.get(builder.cacheType).get(builder.cacheConfig.getSettings()); - LinkedHashMap, TierInfo> cacheListMap = new LinkedHashMap<>(); - cacheListMap.put(onHeapCache, new TierInfo(true, TIER_DIMENSION_VALUE_ON_HEAP)); - cacheListMap.put(diskCache, new TierInfo(isDiskCacheEnabled, TIER_DIMENSION_VALUE_DISK)); - this.caches = Collections.synchronizedMap(cacheListMap); - this.dimensionNames = builder.cacheConfig.getDimensionNames(); // Pass "tier" as the innermost dimension name, in addition to whatever dimensions are specified for the cache as a whole this.statsHolder = new TieredSpilloverCacheStatsHolder(dimensionNames, isDiskCacheEnabled); - this.policies = builder.policies; // Will never be null; builder initializes it to an empty list + long onHeapCachePerSegmentSizeInBytes = builder.onHeapCacheSizeInBytes / this.numberOfSegments; + long diskCachePerSegmentSizeInBytes = builder.diskCacheSizeInBytes / this.numberOfSegments; + if (onHeapCachePerSegmentSizeInBytes <= 0) { + throw new IllegalArgumentException("Per segment size for onHeap cache within Tiered cache should be " + "greater than 0"); + } + if (diskCachePerSegmentSizeInBytes <= 0) { + throw new IllegalArgumentException("Per segment size for disk cache within Tiered cache should be " + "greater than 0"); + } + this.tieredSpilloverCacheSegments = new TieredSpilloverCacheSegment[this.numberOfSegments]; + for (int i = 0; i < numberOfSegments; i++) { + tieredSpilloverCacheSegments[i] = new TieredSpilloverCacheSegment( + builder, + statsHolder, + i + 1, + this.numberOfSegments, + onHeapCachePerSegmentSizeInBytes, + diskCachePerSegmentSizeInBytes + ); + } builder.cacheConfig.getClusterSettings() .addSettingsUpdateConsumer(DISK_CACHE_ENABLED_SETTING_MAP.get(builder.cacheType), this::enableDisableDiskCache); } - // Package private for testing - ICache getOnHeapCache() { - return onHeapCache; - } + static class TieredSpilloverCacheSegment implements ICache { - // Package private for testing - ICache getDiskCache() { - return diskCache; - } + private final ICache diskCache; + private final ICache onHeapCache; - // Package private for testing. - void enableDisableDiskCache(Boolean isDiskCacheEnabled) { - // When disk cache is disabled, we are not clearing up the disk cache entries yet as that should be part of - // separate cache/clear API. - this.caches.put(diskCache, new TierInfo(isDiskCacheEnabled, TIER_DIMENSION_VALUE_DISK)); - this.statsHolder.setDiskCacheEnabled(isDiskCacheEnabled); - } + // Removal listeners for the individual tiers + private final RemovalListener, V> onDiskRemovalListener; + private final RemovalListener, V> onHeapRemovalListener; - @Override - public V get(ICacheKey key) { - Tuple cacheValueTuple = getValueFromTieredCache(true).apply(key); - if (cacheValueTuple == null) { - return null; + // Removal listener from the spillover cache as a whole + private final RemovalListener, V> removalListener; + + private ToLongBiFunction, V> weigher; + ReadWriteLock readWriteLock = new ReentrantReadWriteLock(); + ReleasableLock readLock = new ReleasableLock(readWriteLock.readLock()); + ReleasableLock writeLock = new ReleasableLock(readWriteLock.writeLock()); + + private final Map, TierInfo> caches; + + private final List> policies; + + private final TieredSpilloverCacheStatsHolder statsHolder; + + /** + * This map is used to handle concurrent requests for same key in computeIfAbsent() to ensure we load the value + * only once. + */ + Map, CompletableFuture, V>>> completableFutureMap = new ConcurrentHashMap<>(); + + TieredSpilloverCacheSegment( + Builder builder, + TieredSpilloverCacheStatsHolder statsHolder, + int segmentNumber, + int numberOfSegments, + long onHeapCacheSizeInBytes, + long diskCacheSizeInBytes + ) { + Objects.requireNonNull(builder.onHeapCacheFactory, "onHeap cache builder can't be null"); + Objects.requireNonNull(builder.diskCacheFactory, "disk cache builder can't be null"); + Objects.requireNonNull(builder.cacheConfig, "cache config can't be null"); + Objects.requireNonNull(builder.cacheConfig.getClusterSettings(), "cluster settings can't be null"); + Objects.requireNonNull(builder.cacheConfig.getStoragePath(), "Storage path shouldn't be null"); + this.removalListener = Objects.requireNonNull(builder.removalListener, "Removal listener can't be null"); + this.statsHolder = statsHolder; + + 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(onHeapRemovalListener) + .setKeyType(builder.cacheConfig.getKeyType()) + .setValueType(builder.cacheConfig.getValueType()) + .setSettings(builder.cacheConfig.getSettings()) + .setWeigher(builder.cacheConfig.getWeigher()) + .setDimensionNames(builder.cacheConfig.getDimensionNames()) + .setMaxSizeInBytes(onHeapCacheSizeInBytes) + .setExpireAfterAccess(builder.cacheConfig.getExpireAfterAccess()) + .setClusterSettings(builder.cacheConfig.getClusterSettings()) + .setSegmentCount(1) // We don't need to make underlying caches multi-segmented + .setStatsTrackingEnabled(false) + .setCacheAlias("tiered_on_heap#" + segmentNumber) + .build(), + builder.cacheType, + builder.cacheFactories + + ); + this.diskCache = builder.diskCacheFactory.create( + new CacheConfig.Builder().setRemovalListener(onDiskRemovalListener) + .setKeyType(builder.cacheConfig.getKeyType()) + .setValueType(builder.cacheConfig.getValueType()) + .setSettings(builder.cacheConfig.getSettings()) + .setWeigher(builder.cacheConfig.getWeigher()) + .setKeySerializer(builder.cacheConfig.getKeySerializer()) + .setValueSerializer(builder.cacheConfig.getValueSerializer()) + .setDimensionNames(builder.cacheConfig.getDimensionNames()) + .setSegmentCount(1) // We don't need to make underlying caches multi-segmented + .setStatsTrackingEnabled(false) + .setMaxSizeInBytes(diskCacheSizeInBytes) + .setStoragePath(builder.cacheConfig.getStoragePath() + "/" + segmentNumber) + .setCacheAlias("tiered_disk_cache#" + segmentNumber) + .build(), + builder.cacheType, + builder.cacheFactories + ); + + Boolean isDiskCacheEnabled = DISK_CACHE_ENABLED_SETTING_MAP.get(builder.cacheType).get(builder.cacheConfig.getSettings()); + LinkedHashMap, TierInfo> cacheListMap = new LinkedHashMap<>(); + cacheListMap.put(onHeapCache, new TierInfo(true, TIER_DIMENSION_VALUE_ON_HEAP)); + cacheListMap.put(diskCache, new TierInfo(isDiskCacheEnabled, TIER_DIMENSION_VALUE_DISK)); + this.caches = Collections.synchronizedMap(cacheListMap); + this.policies = builder.policies; // Will never be null; builder initializes it to an empty list } - return cacheValueTuple.v1(); - } - @Override - public void put(ICacheKey key, V value) { - try (ReleasableLock ignore = writeLock.acquire()) { - onHeapCache.put(key, value); - updateStatsOnPut(TIER_DIMENSION_VALUE_ON_HEAP, key, value); + // Package private for testing + ICache getOnHeapCache() { + return onHeapCache; } - } - @Override - public V computeIfAbsent(ICacheKey key, LoadAwareCacheLoader, V> loader) throws Exception { - // Don't capture stats in the initial getValueFromTieredCache(). If we have concurrent requests for the same key, - // and it only has to be loaded one time, we should report one miss and the rest hits. But, if we do stats in - // getValueFromTieredCache(), - // we will see all misses. Instead, handle stats in computeIfAbsent(). - Tuple cacheValueTuple; - CompletableFuture, V>> future = null; - try (ReleasableLock ignore = readLock.acquire()) { - cacheValueTuple = getValueFromTieredCache(false).apply(key); + // Package private for testing + ICache getDiskCache() { + return diskCache; + } + + void enableDisableDiskCache(Boolean isDiskCacheEnabled) { + // When disk cache is disabled, we are not clearing up the disk cache entries yet as that should be part of + // separate cache/clear API. + this.caches.put(diskCache, new TierInfo(isDiskCacheEnabled, TIER_DIMENSION_VALUE_DISK)); + this.statsHolder.setDiskCacheEnabled(isDiskCacheEnabled); + } + + @Override + public V get(ICacheKey key) { + Tuple cacheValueTuple = getValueFromTieredCache(true).apply(key); if (cacheValueTuple == null) { - // Only one of the threads will succeed putting a future into map for the same key. - // Rest will fetch existing future and wait on that to complete. - future = completableFutureMap.putIfAbsent(key, new CompletableFuture<>()); + return null; } + return cacheValueTuple.v1(); } - List heapDimensionValues = statsHolder.getDimensionsWithTierValue(key.dimensions, TIER_DIMENSION_VALUE_ON_HEAP); - List diskDimensionValues = statsHolder.getDimensionsWithTierValue(key.dimensions, TIER_DIMENSION_VALUE_DISK); - - if (cacheValueTuple == null) { - // Add the value to the onHeap cache. We are calling computeIfAbsent which does another get inside. - // This is needed as there can be many requests for the same key at the same time and we only want to load - // the value once. - V value = compute(key, loader, future); - // Handle stats - if (loader.isLoaded()) { - // The value was just computed and added to the cache by this thread. Register a miss for the heap cache, and the disk cache - // if present + + @Override + public void put(ICacheKey key, V value) { + // First check in case the key is already present in either of tiers. + Tuple cacheValueTuple = getValueFromTieredCache(true).apply(key); + if (cacheValueTuple == null) { + // In case it is not present in any tier, put it inside onHeap cache by default. + try (ReleasableLock ignore = writeLock.acquire()) { + onHeapCache.put(key, value); + } updateStatsOnPut(TIER_DIMENSION_VALUE_ON_HEAP, key, value); - statsHolder.incrementMisses(heapDimensionValues); - if (caches.get(diskCache).isEnabled()) { - statsHolder.incrementMisses(diskDimensionValues); + } else { + // Put it inside desired tier. + try (ReleasableLock ignore = writeLock.acquire()) { + for (Map.Entry, TierInfo> entry : this.caches.entrySet()) { + if (cacheValueTuple.v2().equals(entry.getValue().tierName)) { + entry.getKey().put(key, value); + } + } + updateStatsOnPut(cacheValueTuple.v2(), key, value); + } + } + } + + @Override + public V computeIfAbsent(ICacheKey key, LoadAwareCacheLoader, V> loader) throws Exception { + // Don't capture stats in the initial getValueFromTieredCache(). If we have concurrent requests for the same key, + // and it only has to be loaded one time, we should report one miss and the rest hits. But, if we do stats in + // getValueFromTieredCache(), + // we will see all misses. Instead, handle stats in computeIfAbsent(). + Tuple cacheValueTuple; + CompletableFuture, V>> future = null; + try (ReleasableLock ignore = readLock.acquire()) { + cacheValueTuple = getValueFromTieredCache(false).apply(key); + if (cacheValueTuple == null) { + // Only one of the threads will succeed putting a future into map for the same key. + // Rest will fetch existing future and wait on that to complete. + future = completableFutureMap.putIfAbsent(key, new CompletableFuture<>()); + } + } + List heapDimensionValues = statsHolder.getDimensionsWithTierValue(key.dimensions, TIER_DIMENSION_VALUE_ON_HEAP); + List diskDimensionValues = statsHolder.getDimensionsWithTierValue(key.dimensions, TIER_DIMENSION_VALUE_DISK); + + if (cacheValueTuple == null) { + // Add the value to the onHeap cache. We are calling computeIfAbsent which does another get inside. + // This is needed as there can be many requests for the same key at the same time and we only want to load + // the value once. + V value = compute(key, loader, future); + // Handle stats + if (loader.isLoaded()) { + // The value was just computed and added to the cache by this thread. Register a miss for the heap cache, and the disk + // cache + // if present + updateStatsOnPut(TIER_DIMENSION_VALUE_ON_HEAP, key, value); + statsHolder.incrementMisses(heapDimensionValues); + if (caches.get(diskCache).isEnabled()) { + statsHolder.incrementMisses(diskDimensionValues); + } + } else { + // Another thread requesting this key already loaded the value. Register a hit for the heap cache + statsHolder.incrementHits(heapDimensionValues); } + return value; } else { - // Another thread requesting this key already loaded the value. Register a hit for the heap cache - statsHolder.incrementHits(heapDimensionValues); + // Handle stats for an initial hit from getValueFromTieredCache() + if (cacheValueTuple.v2().equals(TIER_DIMENSION_VALUE_ON_HEAP)) { + // A hit for the heap tier + statsHolder.incrementHits(heapDimensionValues); + } else if (cacheValueTuple.v2().equals(TIER_DIMENSION_VALUE_DISK)) { + // Miss for the heap tier, hit for the disk tier + statsHolder.incrementMisses(heapDimensionValues); + statsHolder.incrementHits(diskDimensionValues); + } + } + return cacheValueTuple.v1(); + } + + private V compute(ICacheKey key, LoadAwareCacheLoader, V> loader, CompletableFuture, V>> future) + throws Exception { + // Handler to handle results post-processing. Takes a tuple or exception as an input and returns + // the value. Also before returning value, puts the value in cache. + BiFunction, V>, Throwable, Void> handler = (pair, ex) -> { + if (pair != null) { + try (ReleasableLock ignore = writeLock.acquire()) { + onHeapCache.put(pair.v1(), pair.v2()); + } catch (Exception e) { + // TODO: Catch specific exceptions to know whether this resulted from cache or underlying removal + // listeners/stats. Needs better exception handling at underlying layers.For now swallowing + // exception. + logger.warn("Exception occurred while putting item onto heap cache", e); + } + } else { + if (ex != null) { + logger.warn("Exception occurred while trying to compute the value", ex); + } + } + completableFutureMap.remove(key);// Remove key from map as not needed anymore. + return null; + }; + V value = null; + if (future == null) { + future = completableFutureMap.get(key); + future.handle(handler); + try { + value = loader.load(key); + } catch (Exception ex) { + future.completeExceptionally(ex); + throw new ExecutionException(ex); + } + if (value == null) { + NullPointerException npe = new NullPointerException("Loader returned a null value"); + future.completeExceptionally(npe); + throw new ExecutionException(npe); + } else { + future.complete(new Tuple<>(key, value)); + } + } else { + try { + value = future.get().v2(); + } catch (InterruptedException ex) { + throw new IllegalStateException(ex); + } } return value; - } else { - // Handle stats for an initial hit from getValueFromTieredCache() - if (cacheValueTuple.v2().equals(TIER_DIMENSION_VALUE_ON_HEAP)) { - // A hit for the heap tier - statsHolder.incrementHits(heapDimensionValues); - } else if (cacheValueTuple.v2().equals(TIER_DIMENSION_VALUE_DISK)) { - // Miss for the heap tier, hit for the disk tier - statsHolder.incrementMisses(heapDimensionValues); - statsHolder.incrementHits(diskDimensionValues); + } + + @Override + public void invalidate(ICacheKey key) { + for (Map.Entry, TierInfo> cacheEntry : caches.entrySet()) { + if (key.getDropStatsForDimensions()) { + List dimensionValues = statsHolder.getDimensionsWithTierValue(key.dimensions, cacheEntry.getValue().tierName); + statsHolder.removeDimensions(dimensionValues); + } + if (key.key != null) { + try (ReleasableLock ignore = writeLock.acquire()) { + cacheEntry.getKey().invalidate(key); + } + } } } - return cacheValueTuple.v1(); - } - private V compute(ICacheKey key, LoadAwareCacheLoader, V> loader, CompletableFuture, V>> future) - throws Exception { - // Handler to handle results post processing. Takes a tuple or exception as an input and returns - // the value. Also before returning value, puts the value in cache. - BiFunction, V>, Throwable, Void> handler = (pair, ex) -> { - if (pair != null) { - try (ReleasableLock ignore = writeLock.acquire()) { - onHeapCache.put(pair.v1(), pair.v2()); - } catch (Exception e) { - // TODO: Catch specific exceptions to know whether this resulted from cache or underlying removal - // listeners/stats. Needs better exception handling at underlying layers.For now swallowing - // exception. - logger.warn("Exception occurred while putting item onto heap cache", e); + @Override + public void invalidateAll() { + try (ReleasableLock ignore = writeLock.acquire()) { + for (Map.Entry, TierInfo> cacheEntry : caches.entrySet()) { + cacheEntry.getKey().invalidateAll(); } - } else { - if (ex != null) { - logger.warn("Exception occurred while trying to compute the value", ex); + } + statsHolder.reset(); + } + + @SuppressWarnings({ "unchecked" }) + @Override + public Iterable> keys() { + List>> iterableList = new ArrayList<>(); + for (Map.Entry, TierInfo> cacheEntry : caches.entrySet()) { + iterableList.add(cacheEntry.getKey().keys()); + } + Iterable>[] iterables = (Iterable>[]) iterableList.toArray(new Iterable[0]); + return new ConcatenatedIterables<>(iterables); + } + + @Override + public long count() { + return onHeapCache.count() + diskCache.count(); + } + + @Override + public void refresh() { + try (ReleasableLock ignore = writeLock.acquire()) { + for (Map.Entry, TierInfo> cacheEntry : caches.entrySet()) { + cacheEntry.getKey().refresh(); } } - completableFutureMap.remove(key);// Remove key from map as not needed anymore. + } + + @Override + public ImmutableCacheStatsHolder stats(String[] levels) { return null; - }; - V value = null; - if (future == null) { - future = completableFutureMap.get(key); - future.handle(handler); - try { - value = loader.load(key); - } catch (Exception ex) { - future.completeExceptionally(ex); - throw new ExecutionException(ex); + } + + @Override + public void close() throws IOException { + for (Map.Entry, TierInfo> cacheEntry : caches.entrySet()) { + // Close all the caches here irrespective of whether they are enabled or not. + cacheEntry.getKey().close(); } - if (value == null) { - NullPointerException npe = new NullPointerException("Loader returned a null value"); - future.completeExceptionally(npe); - throw new ExecutionException(npe); - } else { - future.complete(new Tuple<>(key, value)); + } + + void handleRemovalFromHeapTier(RemovalNotification, V> notification) { + ICacheKey key = notification.getKey(); + boolean wasEvicted = SPILLOVER_REMOVAL_REASONS.contains(notification.getRemovalReason()); + boolean countEvictionTowardsTotal = false; // Don't count this eviction towards the cache's total if it ends up in the disk tier + boolean exceptionOccurredOnDiskCachePut = false; + boolean canCacheOnDisk = caches.get(diskCache).isEnabled() && wasEvicted && evaluatePolicies(notification.getValue()); + if (canCacheOnDisk) { + try (ReleasableLock ignore = writeLock.acquire()) { + diskCache.put(key, notification.getValue()); // spill over to the disk tier and increment its stats + } catch (Exception ex) { + // TODO: Catch specific exceptions. Needs better exception handling. We are just swallowing exception + // in this case as it shouldn't cause upstream request to fail. + logger.warn("Exception occurred while putting item to disk cache", ex); + exceptionOccurredOnDiskCachePut = true; + } + if (!exceptionOccurredOnDiskCachePut) { + updateStatsOnPut(TIER_DIMENSION_VALUE_DISK, key, notification.getValue()); + } } - } else { - try { - value = future.get().v2(); - } catch (InterruptedException ex) { - throw new IllegalStateException(ex); + if (!canCacheOnDisk || exceptionOccurredOnDiskCachePut) { + // If the value is not going to the disk cache, send this notification to the TSC's removal listener + // as the value is leaving the TSC entirely + removalListener.onRemoval(notification); + countEvictionTowardsTotal = true; } + updateStatsOnRemoval(TIER_DIMENSION_VALUE_ON_HEAP, wasEvicted, key, notification.getValue(), countEvictionTowardsTotal); } - return value; - } - @Override - public void invalidate(ICacheKey key) { - // We are trying to invalidate the key from all caches though it would be present in only of them. - // Doing this as we don't know where it is located. We could do a get from both and check that, but what will - // also trigger a hit/miss listener event, so ignoring it for now. - // We don't update stats here, as this is handled by the removal listeners for the tiers. - for (Map.Entry, TierInfo> cacheEntry : caches.entrySet()) { - if (key.getDropStatsForDimensions()) { - List dimensionValues = statsHolder.getDimensionsWithTierValue(key.dimensions, cacheEntry.getValue().tierName); - statsHolder.removeDimensions(dimensionValues); + boolean evaluatePolicies(V value) { + for (Predicate policy : policies) { + if (!policy.test(value)) { + return false; + } } - if (key.key != null) { - try (ReleasableLock ignore = writeLock.acquire()) { - cacheEntry.getKey().invalidate(key); + return true; + } + + /** + * Get a value from the tiered cache, and the name of the tier it was found in. + * @param captureStats Whether to record hits/misses for this call of the function + * @return A tuple of the value and the name of the tier it was found in. + */ + private Function, Tuple> getValueFromTieredCache(boolean captureStats) { + return key -> { + try (ReleasableLock ignore = readLock.acquire()) { + for (Map.Entry, TierInfo> cacheEntry : caches.entrySet()) { + if (cacheEntry.getValue().isEnabled()) { + V value = cacheEntry.getKey().get(key); + // Get the tier value corresponding to this cache + String tierValue = cacheEntry.getValue().tierName; + List dimensionValues = statsHolder.getDimensionsWithTierValue(key.dimensions, tierValue); + if (value != null) { + if (captureStats) { + statsHolder.incrementHits(dimensionValues); + } + return new Tuple<>(value, tierValue); + } else if (captureStats) { + statsHolder.incrementMisses(dimensionValues); + } + } + } + return null; } + }; + } + + void handleRemovalFromDiskTier(RemovalNotification, V> notification) { + // Values removed from the disk tier leave the TSC entirely + removalListener.onRemoval(notification); + boolean wasEvicted = SPILLOVER_REMOVAL_REASONS.contains(notification.getRemovalReason()); + updateStatsOnRemoval(TIER_DIMENSION_VALUE_DISK, wasEvicted, notification.getKey(), notification.getValue(), true); + } + + void updateStatsOnRemoval( + String removedFromTierValue, + boolean wasEvicted, + ICacheKey key, + V value, + boolean countEvictionTowardsTotal + ) { + List dimensionValues = statsHolder.getDimensionsWithTierValue(key.dimensions, removedFromTierValue); + if (wasEvicted) { + statsHolder.incrementEvictions(dimensionValues, countEvictionTowardsTotal); + } + statsHolder.decrementItems(dimensionValues); + statsHolder.decrementSizeInBytes(dimensionValues, weigher.applyAsLong(key, value)); + } + + void updateStatsOnPut(String destinationTierValue, ICacheKey key, V value) { + List dimensionValues = statsHolder.getDimensionsWithTierValue(key.dimensions, destinationTierValue); + statsHolder.incrementItems(dimensionValues); + statsHolder.incrementSizeInBytes(dimensionValues, weigher.applyAsLong(key, value)); + } + + /** + * A class which receives removal events from the heap tier. + */ + private class HeapTierRemovalListener implements RemovalListener, V> { + private final TieredSpilloverCacheSegment tsc; + + HeapTierRemovalListener(TieredSpilloverCacheSegment 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 TieredSpilloverCacheSegment tsc; + + DiskTierRemovalListener(TieredSpilloverCacheSegment tsc) { + this.tsc = tsc; + } + + @Override + public void onRemoval(RemovalNotification, V> notification) { + tsc.handleRemovalFromDiskTier(notification); } } } + // Package private for testing. + void enableDisableDiskCache(Boolean isDiskCacheEnabled) { + for (int iter = 0; iter < this.numberOfSegments; iter++) { + tieredSpilloverCacheSegments[iter].enableDisableDiskCache(isDiskCacheEnabled); + } + this.statsHolder.setDiskCacheEnabled(isDiskCacheEnabled); + } + + // Package private for testing. + TieredSpilloverCacheSegment getTieredCacheSegment(ICacheKey key) { + return tieredSpilloverCacheSegments[getSegmentNumber(key)]; + } + + int getSegmentNumber(ICacheKey key) { + return key.hashCode() & (this.numberOfSegments - 1); + } + + int getNumberOfSegments() { + return tieredSpilloverCacheSegments.length; + } + + @Override + public V get(ICacheKey key) { + TieredSpilloverCacheSegment tieredSpilloverCacheSegment = getTieredCacheSegment(key); + return tieredSpilloverCacheSegment.get(key); + } + + @Override + public void put(ICacheKey key, V value) { + TieredSpilloverCacheSegment tieredSpilloverCacheSegment = getTieredCacheSegment(key); + tieredSpilloverCacheSegment.put(key, value); + } + + @Override + public V computeIfAbsent(ICacheKey key, LoadAwareCacheLoader, V> loader) throws Exception { + TieredSpilloverCacheSegment tieredSpilloverCacheSegment = getTieredCacheSegment(key); + return tieredSpilloverCacheSegment.computeIfAbsent(key, loader); + } + + @Override + public void invalidate(ICacheKey key) { + TieredSpilloverCacheSegment tieredSpilloverCacheSegment = getTieredCacheSegment(key); + tieredSpilloverCacheSegment.invalidate(key); + } + @Override public void invalidateAll() { - try (ReleasableLock ignore = writeLock.acquire()) { - for (Map.Entry, TierInfo> cacheEntry : caches.entrySet()) { - cacheEntry.getKey().invalidateAll(); - } + for (int iter = 0; iter < this.numberOfSegments; iter++) { + tieredSpilloverCacheSegments[iter].invalidateAll(); } - statsHolder.reset(); } /** @@ -327,8 +621,8 @@ public void invalidateAll() { @Override public Iterable> keys() { List>> iterableList = new ArrayList<>(); - for (Map.Entry, TierInfo> cacheEntry : caches.entrySet()) { - iterableList.add(cacheEntry.getKey().keys()); + for (int iter = 0; iter < this.numberOfSegments; iter++) { + iterableList.add(tieredSpilloverCacheSegments[iter].keys()); } Iterable>[] iterables = (Iterable>[]) iterableList.toArray(new Iterable[0]); return new ConcatenatedIterables<>(iterables); @@ -343,18 +637,15 @@ public long count() { @Override public void refresh() { - try (ReleasableLock ignore = writeLock.acquire()) { - for (Map.Entry, TierInfo> cacheEntry : caches.entrySet()) { - cacheEntry.getKey().refresh(); - } + for (int iter = 0; iter < this.numberOfSegments; iter++) { + tieredSpilloverCacheSegments[iter].refresh(); } } @Override public void close() throws IOException { - for (Map.Entry, TierInfo> cacheEntry : caches.entrySet()) { - // Close all the caches here irrespective of whether they are enabled or not. - cacheEntry.getKey().close(); + for (int iter = 0; iter < this.numberOfSegments; iter++) { + tieredSpilloverCacheSegments[iter].close(); } } @@ -363,130 +654,44 @@ public ImmutableCacheStatsHolder stats(String[] levels) { return statsHolder.getImmutableCacheStatsHolder(levels); } - /** - * Get a value from the tiered cache, and the name of the tier it was found in. - * @param captureStats Whether to record hits/misses for this call of the function - * @return A tuple of the value and the name of the tier it was found in. - */ - private Function, Tuple> getValueFromTieredCache(boolean captureStats) { - return key -> { - try (ReleasableLock ignore = readLock.acquire()) { - for (Map.Entry, TierInfo> cacheEntry : caches.entrySet()) { - if (cacheEntry.getValue().isEnabled()) { - V value = cacheEntry.getKey().get(key); - // Get the tier value corresponding to this cache - String tierValue = cacheEntry.getValue().tierName; - List dimensionValues = statsHolder.getDimensionsWithTierValue(key.dimensions, tierValue); - if (value != null) { - if (captureStats) { - statsHolder.incrementHits(dimensionValues); - } - return new Tuple<>(value, tierValue); - } else if (captureStats) { - statsHolder.incrementMisses(dimensionValues); - } - } - } - return null; - } - }; - } - - void handleRemovalFromHeapTier(RemovalNotification, V> notification) { - ICacheKey key = notification.getKey(); - boolean wasEvicted = SPILLOVER_REMOVAL_REASONS.contains(notification.getRemovalReason()); - boolean countEvictionTowardsTotal = false; // Don't count this eviction towards the cache's total if it ends up in the disk tier - boolean exceptionOccurredOnDiskCachePut = false; - boolean canCacheOnDisk = caches.get(diskCache).isEnabled() && wasEvicted && evaluatePolicies(notification.getValue()); - if (canCacheOnDisk) { - try (ReleasableLock ignore = writeLock.acquire()) { - diskCache.put(key, notification.getValue()); // spill over to the disk tier and increment its stats - } catch (Exception ex) { - // TODO: Catch specific exceptions. Needs better exception handling. We are just swallowing exception - // in this case as it shouldn't cause upstream request to fail. - logger.warn("Exception occurred while putting item to disk cache", ex); - exceptionOccurredOnDiskCachePut = true; - } - if (!exceptionOccurredOnDiskCachePut) { - updateStatsOnPut(TIER_DIMENSION_VALUE_DISK, key, notification.getValue()); - } - } - if (!canCacheOnDisk || exceptionOccurredOnDiskCachePut) { - // If the value is not going to the disk cache, send this notification to the TSC's removal listener - // as the value is leaving the TSC entirely - removalListener.onRemoval(notification); - countEvictionTowardsTotal = true; + // Package private for testing. + @SuppressWarnings({ "unchecked" }) + Iterable> getOnHeapCacheKeys() { + List>> iterableList = new ArrayList<>(); + for (int iter = 0; iter < this.numberOfSegments; iter++) { + iterableList.add(tieredSpilloverCacheSegments[iter].onHeapCache.keys()); } - updateStatsOnRemoval(TIER_DIMENSION_VALUE_ON_HEAP, wasEvicted, key, notification.getValue(), countEvictionTowardsTotal); - } - - void handleRemovalFromDiskTier(RemovalNotification, V> notification) { - // Values removed from the disk tier leave the TSC entirely - removalListener.onRemoval(notification); - boolean wasEvicted = SPILLOVER_REMOVAL_REASONS.contains(notification.getRemovalReason()); - updateStatsOnRemoval(TIER_DIMENSION_VALUE_DISK, wasEvicted, notification.getKey(), notification.getValue(), true); - } - - void updateStatsOnRemoval( - String removedFromTierValue, - boolean wasEvicted, - ICacheKey key, - V value, - boolean countEvictionTowardsTotal - ) { - List dimensionValues = statsHolder.getDimensionsWithTierValue(key.dimensions, removedFromTierValue); - if (wasEvicted) { - statsHolder.incrementEvictions(dimensionValues, countEvictionTowardsTotal); - } - statsHolder.decrementItems(dimensionValues); - statsHolder.decrementSizeInBytes(dimensionValues, weigher.applyAsLong(key, value)); - } - - void updateStatsOnPut(String destinationTierValue, ICacheKey key, V value) { - List dimensionValues = statsHolder.getDimensionsWithTierValue(key.dimensions, destinationTierValue); - statsHolder.incrementItems(dimensionValues); - statsHolder.incrementSizeInBytes(dimensionValues, weigher.applyAsLong(key, value)); + Iterable>[] iterables = (Iterable>[]) iterableList.toArray(new Iterable[0]); + return new ConcatenatedIterables<>(iterables); } - boolean evaluatePolicies(V value) { - for (Predicate policy : policies) { - if (!policy.test(value)) { - return false; - } + // Package private for testing. + @SuppressWarnings({ "unchecked" }) + Iterable> getDiskCacheKeys() { + List>> iterableList = new ArrayList<>(); + for (int iter = 0; iter < this.numberOfSegments; iter++) { + iterableList.add(tieredSpilloverCacheSegments[iter].diskCache.keys()); } - return true; + Iterable>[] iterables = (Iterable>[]) iterableList.toArray(new Iterable[0]); + return new ConcatenatedIterables<>(iterables); } - /** - * 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); + // Package private for testing. + long onHeapCacheCount() { + long onHeapCacheEntries = 0; + for (int iter = 0; iter < this.numberOfSegments; iter++) { + onHeapCacheEntries += tieredSpilloverCacheSegments[iter].onHeapCache.count(); } + return onHeapCacheEntries; } - /** - * 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); + // Package private for testing. + long diskCacheCount() { + long diskCacheEntries = 0; + for (int iter = 0; iter < this.numberOfSegments; iter++) { + diskCacheEntries += tieredSpilloverCacheSegments[iter].diskCache.count(); } + return diskCacheEntries; } /** @@ -550,7 +755,7 @@ public void remove() { } } - private class TierInfo { + private static class TierInfo { AtomicBoolean isEnabled; final String tierName; @@ -611,12 +816,29 @@ public ICache create(CacheConfig config, CacheType cacheType, "Cached result parser fn can't be null" ); + int numberOfSegments = TIERED_SPILLOVER_SEGMENTS.getConcreteSettingForNamespace(cacheType.getSettingPrefix()).get(settings); + + if (!VALID_SEGMENT_COUNT_VALUES.contains(numberOfSegments)) { + throw new IllegalArgumentException( + String.format(Locale.ROOT, INVALID_SEGMENT_COUNT_EXCEPTION_MESSAGE, TIERED_SPILLOVER_CACHE_NAME) + ); + } + + long onHeapCacheSize = TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace(cacheType.getSettingPrefix()) + .get(settings) + .getBytes(); + long diskCacheSize = TIERED_SPILLOVER_DISK_STORE_SIZE.getConcreteSettingForNamespace(cacheType.getSettingPrefix()) + .get(settings); + return new Builder().setDiskCacheFactory(diskCacheFactory) .setOnHeapCacheFactory(onHeapCacheFactory) .setRemovalListener(config.getRemovalListener()) .setCacheConfig(config) .setCacheType(cacheType) + .setNumberOfSegments(numberOfSegments) .addPolicy(new TookTimePolicy(diskPolicyThreshold, cachedResultParser, config.getClusterSettings(), cacheType)) + .setOnHeapCacheSizeInBytes(onHeapCacheSize) + .setDiskCacheSize(diskCacheSize) .build(); } @@ -640,6 +862,10 @@ public static class Builder { private Map cacheFactories; private final ArrayList> policies = new ArrayList<>(); + private int numberOfSegments; + private long onHeapCacheSizeInBytes; + private long diskCacheSizeInBytes; + /** * Default constructor */ @@ -725,6 +951,36 @@ public Builder addPolicies(List> policies) { return this; } + /** + * Sets number of segments for tiered cache + * @param numberOfSegments number of segments + * @return builder + */ + public Builder setNumberOfSegments(int numberOfSegments) { + this.numberOfSegments = numberOfSegments; + return this; + } + + /** + * Sets onHeap cache size + * @param onHeapCacheSizeInBytes size of onHeap cache in bytes + * @return builder + */ + public Builder setOnHeapCacheSizeInBytes(long onHeapCacheSizeInBytes) { + this.onHeapCacheSizeInBytes = onHeapCacheSizeInBytes; + return this; + } + + /** + * Sets disk cache siz + * @param diskCacheSizeInBytes size of diskCache in bytes + * @return buider + */ + public Builder setDiskCacheSize(long diskCacheSizeInBytes) { + this.diskCacheSizeInBytes = diskCacheSizeInBytes; + return this; + } + /** * Build tiered spillover cache. * @return TieredSpilloverCache diff --git a/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCachePlugin.java b/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCachePlugin.java index 1c10e51630460..bf522b42b70ca 100644 --- a/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCachePlugin.java +++ b/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCachePlugin.java @@ -65,6 +65,15 @@ public List> getSettings() { if (FeatureFlags.PLUGGABLE_CACHE_SETTING.get(settings)) { settingList.add(DISK_CACHE_ENABLED_SETTING_MAP.get(cacheType)); } + settingList.add( + TieredSpilloverCacheSettings.TIERED_SPILLOVER_SEGMENTS.getConcreteSettingForNamespace(cacheType.getSettingPrefix()) + ); + settingList.add( + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace(cacheType.getSettingPrefix()) + ); + settingList.add( + TieredSpilloverCacheSettings.TIERED_SPILLOVER_DISK_STORE_SIZE.getConcreteSettingForNamespace(cacheType.getSettingPrefix()) + ); } return settingList; } diff --git a/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCacheSettings.java b/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCacheSettings.java index e8e441d6bd3a6..122d00af3bd1e 100644 --- a/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCacheSettings.java +++ b/modules/cache-common/src/main/java/org/opensearch/cache/common/tier/TieredSpilloverCacheSettings.java @@ -11,11 +11,16 @@ import org.opensearch.common.cache.CacheType; import org.opensearch.common.settings.Setting; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.unit.ByteSizeValue; +import org.opensearch.threadpool.ThreadPool; import java.util.HashMap; +import java.util.Locale; import java.util.Map; import java.util.concurrent.TimeUnit; +import static org.opensearch.common.cache.settings.CacheSettings.INVALID_SEGMENT_COUNT_EXCEPTION_MESSAGE; +import static org.opensearch.common.cache.settings.CacheSettings.VALID_SEGMENT_COUNT_VALUES; import static org.opensearch.common.settings.Setting.Property.NodeScope; /** @@ -23,6 +28,16 @@ */ public class TieredSpilloverCacheSettings { + /** + * Default cache size in bytes ie 1gb. + */ + public static final long DEFAULT_DISK_CACHE_SIZE_IN_BYTES = 1073741824L; + + /** + * Minimum disk cache size ie 10mb. May not make such sense to keep a value smaller than this. + */ + public static final long MIN_DISK_CACHE_SIZE_IN_BYTES = 10485760L; + /** * Setting which defines the onHeap cache store to be used in TieredSpilloverCache. * @@ -50,6 +65,43 @@ public class TieredSpilloverCacheSettings { (key) -> Setting.boolSetting(key, true, NodeScope, Setting.Property.Dynamic) ); + /** + * Setting defining the number of segments within Tiered cache + */ + public static final Setting.AffixSetting TIERED_SPILLOVER_SEGMENTS = Setting.suffixKeySetting( + TieredSpilloverCache.TieredSpilloverCacheFactory.TIERED_SPILLOVER_CACHE_NAME + ".segments", + (key) -> Setting.intSetting(key, defaultSegments(), 1, k -> { + if (!VALID_SEGMENT_COUNT_VALUES.contains(k)) { + throw new IllegalArgumentException( + String.format( + Locale.ROOT, + INVALID_SEGMENT_COUNT_EXCEPTION_MESSAGE, + TieredSpilloverCache.TieredSpilloverCacheFactory.TIERED_SPILLOVER_CACHE_NAME + ) + ); + } + }, NodeScope) + ); + + /** + * Setting which defines the onHeap cache size to be used within tiered cache. + * + * Pattern: {cache_type}.tiered_spillover.onheap.store.size + * Example: indices.request.cache.tiered_spillover.onheap.store.size + */ + public static final Setting.AffixSetting TIERED_SPILLOVER_ONHEAP_STORE_SIZE = Setting.suffixKeySetting( + TieredSpilloverCache.TieredSpilloverCacheFactory.TIERED_SPILLOVER_CACHE_NAME + ".onheap.store.size", + (key) -> Setting.memorySizeSetting(key, "1%", NodeScope) + ); + + /** + * Setting which defines the disk cache size to be used within tiered cache. + */ + public static final Setting.AffixSetting TIERED_SPILLOVER_DISK_STORE_SIZE = Setting.suffixKeySetting( + TieredSpilloverCache.TieredSpilloverCacheFactory.TIERED_SPILLOVER_CACHE_NAME + ".disk.store.size", + (key) -> Setting.longSetting(key, DEFAULT_DISK_CACHE_SIZE_IN_BYTES, MIN_DISK_CACHE_SIZE_IN_BYTES, NodeScope) + ); + /** * Setting defining the minimum took time for a query to be allowed into the disk cache. */ @@ -96,6 +148,23 @@ public class TieredSpilloverCacheSettings { DISK_CACHE_ENABLED_SETTING_MAP = diskCacheSettingMap; } + /** + * Returns the default segment count to be used within TieredCache. + * @return default segment count + */ + public static int defaultSegments() { + // For now, we use number of search threads as the default segment count. If needed each cache type can + // configure its own segmentCount via setting in the future. + int defaultSegmentCount = ThreadPool.searchThreadPoolSize(Runtime.getRuntime().availableProcessors()); + // Now round it off to the next power of 2 as we don't support any other values. + for (int segmentValue : VALID_SEGMENT_COUNT_VALUES) { + if (defaultSegmentCount <= segmentValue) { + return segmentValue; + } + } + return VALID_SEGMENT_COUNT_VALUES.get(VALID_SEGMENT_COUNT_VALUES.size() - 1); + } + /** * Default constructor */ 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 69e2060f7ea2f..fcddd489a27aa 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 @@ -62,6 +62,7 @@ public void put(ICacheKey key, V value) { if (this.cache.size() >= maxSize) { // For simplification this.removalListener.onRemoval(new RemovalNotification<>(key, value, RemovalReason.EVICTED)); this.statsHolder.decrementItems(List.of()); + return; } try { Thread.sleep(delay); @@ -86,8 +87,10 @@ 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); + V value = this.cache.remove(key); + if (value != null) { + removalListener.onRemoval(new RemovalNotification<>(key, cache.get(key), RemovalReason.INVALIDATED)); + } } @Override @@ -131,11 +134,13 @@ public static class MockDiskCacheFactory implements Factory { final long delay; final int maxSize; final boolean statsTrackingEnabled; + final int keyValueSize; - public MockDiskCacheFactory(long delay, int maxSize, boolean statsTrackingEnabled) { + public MockDiskCacheFactory(long delay, int maxSize, boolean statsTrackingEnabled, int keyValueSize) { this.delay = delay; this.maxSize = maxSize; this.statsTrackingEnabled = statsTrackingEnabled; + this.keyValueSize = keyValueSize; } @Override @@ -145,13 +150,21 @@ public ICache create(CacheConfig config, CacheType cacheType, // cache would require. assert config.getKeySerializer() != null; assert config.getValueSerializer() != null; - return new Builder().setKeySerializer((Serializer) config.getKeySerializer()) + MockDiskCache.Builder builder = (Builder) new Builder().setKeySerializer( + (Serializer) config.getKeySerializer() + ) .setValueSerializer((Serializer) config.getValueSerializer()) - .setMaxSize(maxSize) .setDeliberateDelay(delay) .setRemovalListener(config.getRemovalListener()) - .setStatsTrackingEnabled(config.getStatsTrackingEnabled()) - .build(); + .setStatsTrackingEnabled(config.getStatsTrackingEnabled()); + + // For mock disk cache, size refers to number of entries for simplicity. + if (config.getMaxSizeInBytes() > 0) { + builder.setMaxSize(Math.toIntExact(config.getMaxSizeInBytes())); + } else { + builder.setMaxSize(maxSize); + } + return builder.build(); } @Override 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 c6440a1e1797f..1215a2130ac2d 100644 --- a/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/TieredSpilloverCacheTests.java +++ b/modules/cache-common/src/test/java/org/opensearch/cache/common/tier/TieredSpilloverCacheTests.java @@ -8,6 +8,7 @@ package org.opensearch.cache.common.tier; +import org.opensearch.OpenSearchException; import org.opensearch.common.Randomness; import org.opensearch.common.cache.CacheType; import org.opensearch.common.cache.ICache; @@ -29,6 +30,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.FeatureFlags; +import org.opensearch.env.NodeEnvironment; import org.opensearch.test.OpenSearchTestCase; import org.junit.Before; @@ -54,11 +56,16 @@ import java.util.function.Function; import java.util.function.Predicate; +import static org.opensearch.cache.common.tier.TieredSpilloverCache.ZERO_SEGMENT_COUNT_EXCEPTION_MESSAGE; import static org.opensearch.cache.common.tier.TieredSpilloverCacheSettings.DISK_CACHE_ENABLED_SETTING_MAP; +import static org.opensearch.cache.common.tier.TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE; +import static org.opensearch.cache.common.tier.TieredSpilloverCacheSettings.TIERED_SPILLOVER_SEGMENTS; import static org.opensearch.cache.common.tier.TieredSpilloverCacheSettings.TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP; import static org.opensearch.cache.common.tier.TieredSpilloverCacheStatsHolder.TIER_DIMENSION_NAME; import static org.opensearch.cache.common.tier.TieredSpilloverCacheStatsHolder.TIER_DIMENSION_VALUE_DISK; import static org.opensearch.cache.common.tier.TieredSpilloverCacheStatsHolder.TIER_DIMENSION_VALUE_ON_HEAP; +import static org.opensearch.common.cache.settings.CacheSettings.INVALID_SEGMENT_COUNT_EXCEPTION_MESSAGE; +import static org.opensearch.common.cache.settings.CacheSettings.VALID_SEGMENT_COUNT_VALUES; import static org.opensearch.common.cache.store.settings.OpenSearchOnHeapCacheSettings.MAXIMUM_SIZE_IN_BYTES_KEY; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doThrow; @@ -89,13 +96,14 @@ public void testComputeIfAbsentWithoutAnyOnHeapCacheEviction() throws Exception removalListener, Settings.builder() .put( - OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) - .get(MAXIMUM_SIZE_IN_BYTES_KEY) - .getKey(), + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), onHeapCacheSize * keyValueSize + "b" ) .build(), - 0 + 0, + 1 ); int numOfItems1 = randomIntBetween(1, onHeapCacheSize / 2 - 1); List> keys = new ArrayList<>(); @@ -160,18 +168,19 @@ public void testComputeIfAbsentWithFactoryBasedCacheCreation() throws Exception MockDiskCache.MockDiskCacheFactory.NAME ) .put( - OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) - .get(MAXIMUM_SIZE_IN_BYTES_KEY) - .getKey(), + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), onHeapCacheSize * keyValueSize + "b" ) .put( CacheSettings.getConcreteStoreNameSettingForCacheType(CacheType.INDICES_REQUEST_CACHE).getKey(), TieredSpilloverCache.TieredSpilloverCacheFactory.TIERED_SPILLOVER_CACHE_NAME ) + .put(TIERED_SPILLOVER_SEGMENTS.getConcreteSettingForNamespace(CacheType.INDICES_REQUEST_CACHE.getSettingPrefix()).getKey(), 1) .put(FeatureFlags.PLUGGABLE_CACHE, "true") .build(); - + String storagePath = getStoragePath(settings); ICache tieredSpilloverICache = new TieredSpilloverCache.TieredSpilloverCacheFactory().create( new CacheConfig.Builder().setKeyType(String.class) .setKeyType(String.class) @@ -182,15 +191,16 @@ public void testComputeIfAbsentWithFactoryBasedCacheCreation() throws Exception .setSettings(settings) .setDimensionNames(dimensionNames) .setCachedResultParser(s -> new CachedQueryResult.PolicyValues(20_000_000L)) // Values will always appear to have taken - // 20_000_000 ns = 20 ms to compute + // 20_000_000 ns = 20 ms to compute .setClusterSettings(clusterSettings) + .setStoragePath(storagePath) .build(), CacheType.INDICES_REQUEST_CACHE, Map.of( OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory.NAME, new OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory(), MockDiskCache.MockDiskCacheFactory.NAME, - new MockDiskCache.MockDiskCacheFactory(0, randomIntBetween(100, 300), false) + new MockDiskCache.MockDiskCacheFactory(0, randomIntBetween(100, 300), false, keyValueSize) ) ); @@ -206,10 +216,10 @@ public void testComputeIfAbsentWithFactoryBasedCacheCreation() throws Exception } int expectedDiskEntries = numOfItems1 - onHeapCacheSize; - tieredSpilloverCache.getOnHeapCache().keys().forEach(onHeapKeys::add); - tieredSpilloverCache.getDiskCache().keys().forEach(diskTierKeys::add); + tieredSpilloverCache.tieredSpilloverCacheSegments[0].getOnHeapCache().keys().forEach(onHeapKeys::add); + tieredSpilloverCache.tieredSpilloverCacheSegments[0].getDiskCache().keys().forEach(diskTierKeys::add); // Verify on heap cache stats. - assertEquals(onHeapCacheSize, tieredSpilloverCache.getOnHeapCache().count()); + assertEquals(onHeapCacheSize, tieredSpilloverCache.tieredSpilloverCacheSegments[0].getOnHeapCache().count()); assertEquals(onHeapCacheSize, getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); assertEquals(numOfItems1, getMissesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); assertEquals(0, getHitsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); @@ -217,7 +227,7 @@ public void testComputeIfAbsentWithFactoryBasedCacheCreation() throws Exception assertEquals(onHeapCacheSize * keyValueSize, getSizeInBytesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); // Verify disk cache stats. - assertEquals(expectedDiskEntries, tieredSpilloverCache.getDiskCache().count()); + assertEquals(expectedDiskEntries, tieredSpilloverCache.tieredSpilloverCacheSegments[0].getDiskCache().count()); assertEquals(expectedDiskEntries, getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); assertEquals(0, getHitsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); assertEquals(numOfItems1, getMissesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); @@ -225,6 +235,149 @@ public void testComputeIfAbsentWithFactoryBasedCacheCreation() throws Exception assertEquals(expectedDiskEntries * keyValueSize, getSizeInBytesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); } + public void testComputeIfAbsentWithSegmentedCache() throws Exception { + int onHeapCacheSize = randomIntBetween(300, 600); + int diskCacheSize = randomIntBetween(700, 1200); + int totalSize = onHeapCacheSize + diskCacheSize; + int numberOfSegments = getNumberOfSegments(); + int keyValueSize = 11; + MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); + int onHeapCacheSizeInBytes = onHeapCacheSize * keyValueSize; + Map expectedSegmentOnHeapCacheSize = getSegmentOnHeapCacheSize( + numberOfSegments, + onHeapCacheSizeInBytes, + keyValueSize + ); + int totalOnHeapEntries = 0; + int totalOnDiskEntries = 0; + // Set the desired settings needed to create a TieredSpilloverCache object with INDICES_REQUEST_CACHE cacheType. + Settings settings = Settings.builder() + .put( + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_NAME.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), + OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory.NAME + ) + .put( + TIERED_SPILLOVER_SEGMENTS.getConcreteSettingForNamespace(CacheType.INDICES_REQUEST_CACHE.getSettingPrefix()).getKey(), + numberOfSegments + ) + .put( + TieredSpilloverCacheSettings.TIERED_SPILLOVER_DISK_STORE_NAME.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), + MockDiskCache.MockDiskCacheFactory.NAME + ) + .put( + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), + onHeapCacheSizeInBytes + "b" + ) + .put( + CacheSettings.getConcreteStoreNameSettingForCacheType(CacheType.INDICES_REQUEST_CACHE).getKey(), + TieredSpilloverCache.TieredSpilloverCacheFactory.TIERED_SPILLOVER_CACHE_NAME + ) + .put(FeatureFlags.PLUGGABLE_CACHE, "true") + .build(); + String storagePath = getStoragePath(settings); + ICache tieredSpilloverICache = new TieredSpilloverCache.TieredSpilloverCacheFactory().create( + new CacheConfig.Builder().setKeyType(String.class) + .setKeyType(String.class) + .setWeigher((k, v) -> keyValueSize) + .setRemovalListener(removalListener) + .setKeySerializer(new StringSerializer()) + .setValueSerializer(new StringSerializer()) + .setSettings(settings) + .setDimensionNames(dimensionNames) + .setCachedResultParser(s -> new CachedQueryResult.PolicyValues(20_000_000L)) // Values will always appear to have taken + // 20_000_000 ns = 20 ms to compute + .setClusterSettings(clusterSettings) + .setStoragePath(storagePath) + .setSegmentCount(numberOfSegments) + .build(), + CacheType.INDICES_REQUEST_CACHE, + Map.of( + OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory.NAME, + new OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory(), + MockDiskCache.MockDiskCacheFactory.NAME, + // Make disk cache big enough to hold all entries + new MockDiskCache.MockDiskCacheFactory(0, diskCacheSize * 500, false, keyValueSize) + ) + ); + TieredSpilloverCache tieredSpilloverCache = (TieredSpilloverCache) tieredSpilloverICache; + TieredSpilloverCache.TieredSpilloverCacheSegment[] tieredSpilloverCacheSegments = + tieredSpilloverCache.tieredSpilloverCacheSegments; + assertEquals(numberOfSegments, tieredSpilloverCacheSegments.length); + + Map, TieredSpilloverCache.TieredSpilloverCacheSegment> tieredSpilloverCacheSegmentMap = + new HashMap<>(); + + int numOfItems1 = randomIntBetween(onHeapCacheSize + 1, totalSize); + List> onHeapKeys = new ArrayList<>(); + List> diskTierKeys = new ArrayList<>(); + Map expectedNumberOfEntriesInSegment = new HashMap<>(); + for (int iter = 0; iter < numOfItems1; iter++) { + String key = UUID.randomUUID().toString(); + LoadAwareCacheLoader, String> tieredCacheLoader = getLoadAwareCacheLoader(); + ICacheKey iCacheKey = getICacheKey(key); + int keySegment = tieredSpilloverCache.getSegmentNumber(iCacheKey); + if (expectedNumberOfEntriesInSegment.get(keySegment) == null) { + expectedNumberOfEntriesInSegment.put(keySegment, Integer.valueOf(1)); + } else { + Integer updatedValue = expectedNumberOfEntriesInSegment.get(keySegment) + 1; + expectedNumberOfEntriesInSegment.put(keySegment, updatedValue); + } + tieredSpilloverCacheSegmentMap.put(iCacheKey, tieredSpilloverCache.getTieredCacheSegment(iCacheKey)); + tieredSpilloverCache.computeIfAbsent(iCacheKey, tieredCacheLoader); + } + + // We now calculate expected onHeap cache entries and then verify it later. + for (int i = 0; i < numberOfSegments; i++) { + if (expectedNumberOfEntriesInSegment.get(i) == null) { + continue; + } + if (expectedNumberOfEntriesInSegment.get(i) >= expectedSegmentOnHeapCacheSize.get(i)) { + totalOnHeapEntries += expectedSegmentOnHeapCacheSize.get(i); + totalOnDiskEntries += expectedNumberOfEntriesInSegment.get(i) - expectedSegmentOnHeapCacheSize.get(i); + } else { + // In this case onHeap cache wasn't utilized fully. + totalOnHeapEntries += expectedNumberOfEntriesInSegment.get(i); + } + } + + tieredSpilloverCache.getOnHeapCacheKeys().forEach(onHeapKeys::add); + tieredSpilloverCache.getDiskCacheKeys().forEach(diskTierKeys::add); + // Verify on heap cache stats. + assertEquals(totalOnHeapEntries, tieredSpilloverCache.onHeapCacheCount()); + assertEquals(totalOnHeapEntries, getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(numOfItems1, getMissesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(0, getHitsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(totalOnDiskEntries, getEvictionsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(totalOnHeapEntries * keyValueSize, getSizeInBytesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + + // Verify disk cache stats. + assertEquals(totalOnDiskEntries, tieredSpilloverCache.diskCacheCount()); + assertEquals(totalOnDiskEntries, getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); + assertEquals(0, getHitsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); + assertEquals(numOfItems1, getMissesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); + assertEquals(0, getEvictionsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); + assertEquals(totalOnDiskEntries * keyValueSize, getSizeInBytesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); + + // Verify the keys for onHeap and disk cache + + for (ICacheKey key : onHeapKeys) { + assertNotNull(tieredSpilloverCache.get(key)); + } + assertEquals(numOfItems1, getMissesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + assertEquals(onHeapKeys.size(), getHitsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); + for (ICacheKey key : diskTierKeys) { + assertNotNull(tieredSpilloverCache.get(key)); + } + assertEquals(numOfItems1, getMissesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); + assertEquals(diskTierKeys.size(), getHitsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); + } + public void testWithFactoryCreationWithOnHeapCacheNotPresent() { int onHeapCacheSize = randomIntBetween(10, 30); int keyValueSize = 50; @@ -239,9 +392,9 @@ public void testWithFactoryCreationWithOnHeapCacheNotPresent() { MockDiskCache.MockDiskCacheFactory.NAME ) .put( - OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) - .get(MAXIMUM_SIZE_IN_BYTES_KEY) - .getKey(), + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), onHeapCacheSize * keyValueSize + "b" ) .put( @@ -265,7 +418,7 @@ public void testWithFactoryCreationWithOnHeapCacheNotPresent() { OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory.NAME, new OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory(), MockDiskCache.MockDiskCacheFactory.NAME, - new MockDiskCache.MockDiskCacheFactory(0, randomIntBetween(100, 300), false) + new MockDiskCache.MockDiskCacheFactory(0, randomIntBetween(100, 300), false, keyValueSize) ) ) ); @@ -289,9 +442,9 @@ public void testWithFactoryCreationWithDiskCacheNotPresent() { OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory.NAME ) .put( - OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) - .get(MAXIMUM_SIZE_IN_BYTES_KEY) - .getKey(), + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), onHeapCacheSize * keyValueSize + "b" ) .build(); @@ -310,7 +463,7 @@ public void testWithFactoryCreationWithDiskCacheNotPresent() { OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory.NAME, new OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory(), MockDiskCache.MockDiskCacheFactory.NAME, - new MockDiskCache.MockDiskCacheFactory(0, randomIntBetween(100, 300), false) + new MockDiskCache.MockDiskCacheFactory(0, randomIntBetween(100, 300), false, keyValueSize) ) ) ); @@ -327,6 +480,20 @@ public void testComputeIfAbsentWithEvictionsFromOnHeapCache() throws Exception { int keyValueSize = 50; MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); ICache.Factory onHeapCacheFactory = new OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory(); + + Settings settings = Settings.builder() + .put( + CacheSettings.getConcreteStoreNameSettingForCacheType(CacheType.INDICES_REQUEST_CACHE).getKey(), + TieredSpilloverCache.TieredSpilloverCacheFactory.TIERED_SPILLOVER_CACHE_NAME + ) + .put(FeatureFlags.PLUGGABLE_CACHE, "true") + .put( + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), + onHeapCacheSize * keyValueSize + "b" + ) + .build(); CacheConfig cacheConfig = new CacheConfig.Builder().setKeyType(String.class) .setKeyType(String.class) .setWeigher((k, v) -> keyValueSize) @@ -334,25 +501,12 @@ public void testComputeIfAbsentWithEvictionsFromOnHeapCache() throws Exception { .setKeySerializer(new StringSerializer()) .setValueSerializer(new StringSerializer()) .setDimensionNames(dimensionNames) - .setSettings( - Settings.builder() - .put( - CacheSettings.getConcreteStoreNameSettingForCacheType(CacheType.INDICES_REQUEST_CACHE).getKey(), - TieredSpilloverCache.TieredSpilloverCacheFactory.TIERED_SPILLOVER_CACHE_NAME - ) - .put(FeatureFlags.PLUGGABLE_CACHE, "true") - .put( - OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) - .get(MAXIMUM_SIZE_IN_BYTES_KEY) - .getKey(), - onHeapCacheSize * keyValueSize + "b" - ) - .build() - ) + .setSettings(settings) + .setStoragePath(getStoragePath(settings)) .setClusterSettings(clusterSettings) .build(); - ICache.Factory mockDiskCacheFactory = new MockDiskCache.MockDiskCacheFactory(0, diskCacheSize, false); + ICache.Factory mockDiskCacheFactory = new MockDiskCache.MockDiskCacheFactory(0, diskCacheSize, false, keyValueSize); TieredSpilloverCache tieredSpilloverCache = new TieredSpilloverCache.Builder() .setOnHeapCacheFactory(onHeapCacheFactory) @@ -360,6 +514,9 @@ public void testComputeIfAbsentWithEvictionsFromOnHeapCache() throws Exception { .setCacheConfig(cacheConfig) .setRemovalListener(removalListener) .setCacheType(CacheType.INDICES_REQUEST_CACHE) + .setNumberOfSegments(1) + .setOnHeapCacheSizeInBytes(onHeapCacheSize * keyValueSize) + .setDiskCacheSize(diskCacheSize) .build(); // Put values in cache more than it's size and cause evictions from onHeap. @@ -372,7 +529,7 @@ public void testComputeIfAbsentWithEvictionsFromOnHeapCache() throws Exception { tieredSpilloverCache.computeIfAbsent(key, tieredCacheLoader); } - long actualDiskCacheSize = tieredSpilloverCache.getDiskCache().count(); + long actualDiskCacheSize = tieredSpilloverCache.diskCacheCount(); assertEquals(numOfItems1, getMissesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); assertEquals(0, getHitsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); @@ -381,8 +538,8 @@ public void testComputeIfAbsentWithEvictionsFromOnHeapCache() throws Exception { assertEquals(onHeapCacheSize * keyValueSize, getSizeInBytesForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); assertEquals(actualDiskCacheSize, getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); - tieredSpilloverCache.getOnHeapCache().keys().forEach(onHeapKeys::add); - tieredSpilloverCache.getDiskCache().keys().forEach(diskTierKeys::add); + tieredSpilloverCache.getOnHeapCacheKeys().forEach(onHeapKeys::add); + tieredSpilloverCache.getDiskCacheKeys().forEach(diskTierKeys::add); // Try to hit cache again with some randomization. int numOfItems2 = randomIntBetween(50, 200); @@ -420,11 +577,11 @@ public void testComputeIfAbsentWithEvictionsFromOnHeapCache() throws Exception { } public void testComputeIfAbsentWithEvictionsFromTieredCache() throws Exception { - int onHeapCacheSize = randomIntBetween(10, 30); - int diskCacheSize = randomIntBetween(onHeapCacheSize + 1, 100); + int onHeapCacheSize = randomIntBetween(300, 600); + int diskCacheSize = randomIntBetween(700, 1200); int totalSize = onHeapCacheSize + diskCacheSize; - int keyValueSize = 50; - + int numberOfSegments = getNumberOfSegments(); + int keyValueSize = 11; MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); TieredSpilloverCache tieredSpilloverCache = initializeTieredSpilloverCache( keyValueSize, @@ -432,25 +589,53 @@ public void testComputeIfAbsentWithEvictionsFromTieredCache() throws Exception { removalListener, Settings.builder() .put( - OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) - .get(MAXIMUM_SIZE_IN_BYTES_KEY) - .getKey(), + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), onHeapCacheSize * keyValueSize + "b" ) .build(), - 0 + 0, + numberOfSegments + ); + Map onHeapCacheExpectedSize = getSegmentOnHeapCacheSize( + numberOfSegments, + onHeapCacheSize * keyValueSize, + keyValueSize ); + Map mockDiskCacheExpectedSize = getSegmentMockDiskCacheSize(numberOfSegments, diskCacheSize); + Map perSegmentEntryCapacity = new HashMap<>(); + for (int i = 0; i < numberOfSegments; i++) { + int totalEntriesForSegment = onHeapCacheExpectedSize.get(i) + mockDiskCacheExpectedSize.get(i); + perSegmentEntryCapacity.put(i, totalEntriesForSegment); + } int numOfItems = randomIntBetween(totalSize + 1, totalSize * 3); + Map segmentSizeTracker = new HashMap<>(); + int expectedEvictions = 0; for (int iter = 0; iter < numOfItems; iter++) { + ICacheKey iCacheKey = getICacheKey(UUID.randomUUID().toString()); + int keySegment = tieredSpilloverCache.getSegmentNumber(iCacheKey); LoadAwareCacheLoader, String> tieredCacheLoader = getLoadAwareCacheLoader(); - tieredSpilloverCache.computeIfAbsent(getICacheKey(UUID.randomUUID().toString()), tieredCacheLoader); + tieredSpilloverCache.computeIfAbsent(iCacheKey, tieredCacheLoader); + if (segmentSizeTracker.get(keySegment) == null) { + segmentSizeTracker.put(keySegment, Integer.valueOf(1)); + } else { + Integer updatedValue = segmentSizeTracker.get(keySegment) + 1; + segmentSizeTracker.put(keySegment, updatedValue); + } } - - int evictions = numOfItems - (totalSize); // Evictions from the cache as a whole - assertEquals(evictions, removalListener.evictionsMetric.count()); - assertEquals(evictions, getEvictionsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); + for (int i = 0; i < numberOfSegments; i++) { + if (segmentSizeTracker.get(i) == null) { + continue; + } + if (segmentSizeTracker.get(i) > perSegmentEntryCapacity.get(i)) { + expectedEvictions += segmentSizeTracker.get(i) - perSegmentEntryCapacity.get(i); + } + } + assertEquals(expectedEvictions, removalListener.evictionsMetric.count()); + assertEquals(expectedEvictions, getEvictionsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); assertEquals( - evictions + getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK), + expectedEvictions + getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK), getEvictionsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP) ); } @@ -468,13 +653,14 @@ public void testGetAndCount() throws Exception { removalListener, Settings.builder() .put( - OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) - .get(MAXIMUM_SIZE_IN_BYTES_KEY) - .getKey(), + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), onHeapCacheSize * keyValueSize + "b" ) .build(), - 0 + 0, + 1 ); int numOfItems1 = randomIntBetween(onHeapCacheSize + 1, totalSize); @@ -509,8 +695,9 @@ public void testGetAndCount() throws Exception { } public void testPut() throws Exception { - int onHeapCacheSize = randomIntBetween(10, 30); - int diskCacheSize = randomIntBetween(onHeapCacheSize + 1, 100); + int numberOfSegments = getNumberOfSegments(); + int onHeapCacheSize = randomIntBetween(10, 30) * numberOfSegments; + int diskCacheSize = randomIntBetween(onHeapCacheSize + 1, onHeapCacheSize * 2); int keyValueSize = 50; MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); @@ -520,13 +707,14 @@ public void testPut() throws Exception { removalListener, Settings.builder() .put( - OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) - .get(MAXIMUM_SIZE_IN_BYTES_KEY) - .getKey(), + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), onHeapCacheSize * keyValueSize + "b" ) .build(), - 0 + 0, + numberOfSegments ); ICacheKey key = getICacheKey(UUID.randomUUID().toString()); String value = UUID.randomUUID().toString(); @@ -541,7 +729,7 @@ public void testPutAndVerifyNewItemsArePresentOnHeapCache() throws Exception { int keyValueSize = 50; MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); - + int numberOfSegments = 1; TieredSpilloverCache tieredSpilloverCache = initializeTieredSpilloverCache( keyValueSize, diskCacheSize, @@ -552,13 +740,14 @@ public void testPutAndVerifyNewItemsArePresentOnHeapCache() throws Exception { TieredSpilloverCache.TieredSpilloverCacheFactory.TIERED_SPILLOVER_CACHE_NAME ) .put( - OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) - .get(MAXIMUM_SIZE_IN_BYTES_KEY) - .getKey(), + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), (onHeapCacheSize * keyValueSize) + "b" ) .build(), - 0 + 0, + numberOfSegments ); for (int i = 0; i < onHeapCacheSize; i++) { @@ -580,7 +769,7 @@ public void testPutAndVerifyNewItemsArePresentOnHeapCache() throws Exception { // Verify that new items are part of onHeap cache. List> actualOnHeapCacheKeys = new ArrayList<>(); - tieredSpilloverCache.getOnHeapCache().keys().forEach(actualOnHeapCacheKeys::add); + tieredSpilloverCache.getOnHeapCacheKeys().forEach(actualOnHeapCacheKeys::add); assertEquals(newKeyList.size(), actualOnHeapCacheKeys.size()); for (int i = 0; i < actualOnHeapCacheKeys.size(); i++) { @@ -594,7 +783,6 @@ public void testInvalidate() throws Exception { int onHeapCacheSize = 1; int diskCacheSize = 10; int keyValueSize = 20; - MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); TieredSpilloverCache tieredSpilloverCache = initializeTieredSpilloverCache( keyValueSize, @@ -602,13 +790,14 @@ public void testInvalidate() throws Exception { removalListener, Settings.builder() .put( - OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) - .get(MAXIMUM_SIZE_IN_BYTES_KEY) - .getKey(), + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), onHeapCacheSize * keyValueSize + "b" ) .build(), - 0 + 0, + 1 ); ICacheKey key = getICacheKey(UUID.randomUUID().toString()); String value = UUID.randomUUID().toString(); @@ -652,13 +841,14 @@ public void testCacheKeys() throws Exception { removalListener, Settings.builder() .put( - OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) - .get(MAXIMUM_SIZE_IN_BYTES_KEY) - .getKey(), + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), onHeapCacheSize * keyValueSize + "b" ) .build(), - 0 + 0, + 1 ); List> onHeapKeys = new ArrayList<>(); List> diskTierKeys = new ArrayList<>(); @@ -678,8 +868,8 @@ public void testCacheKeys() throws Exception { List> actualOnHeapKeys = new ArrayList<>(); List> actualOnDiskKeys = new ArrayList<>(); - Iterable> onHeapiterable = tieredSpilloverCache.getOnHeapCache().keys(); - Iterable> onDiskiterable = tieredSpilloverCache.getDiskCache().keys(); + Iterable> onHeapiterable = tieredSpilloverCache.getOnHeapCacheKeys(); + Iterable> onDiskiterable = tieredSpilloverCache.getDiskCacheKeys(); onHeapiterable.iterator().forEachRemaining(actualOnHeapKeys::add); onDiskiterable.iterator().forEachRemaining(actualOnDiskKeys::add); for (ICacheKey onHeapKey : onHeapKeys) { @@ -713,16 +903,18 @@ public void testRefresh() { diskCacheSize, removalListener, Settings.EMPTY, - 0 + 0, + 1 ); tieredSpilloverCache.refresh(); } public void testInvalidateAll() throws Exception { - int onHeapCacheSize = randomIntBetween(10, 30); - int diskCacheSize = randomIntBetween(60, 100); - int keyValueSize = 50; + int onHeapCacheSize = randomIntBetween(300, 600); + int diskCacheSize = randomIntBetween(700, 1200); int totalSize = onHeapCacheSize + diskCacheSize; + int numberOfSegments = getNumberOfSegments(); + int keyValueSize = 50; MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); TieredSpilloverCache tieredSpilloverCache = initializeTieredSpilloverCache( @@ -731,30 +923,29 @@ public void testInvalidateAll() throws Exception { removalListener, Settings.builder() .put( - OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) - .get(MAXIMUM_SIZE_IN_BYTES_KEY) - .getKey(), + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), onHeapCacheSize * keyValueSize + "b" ) .build(), - 0 + 0, + numberOfSegments ); // Put values in cache more than it's size and cause evictions from onHeap. int numOfItems1 = randomIntBetween(onHeapCacheSize + 1, totalSize); - List> onHeapKeys = new ArrayList<>(); - List> diskTierKeys = new ArrayList<>(); for (int iter = 0; iter < numOfItems1; iter++) { ICacheKey key = getICacheKey(UUID.randomUUID().toString()); - if (iter > (onHeapCacheSize - 1)) { - // All these are bound to go to disk based cache. - diskTierKeys.add(key); - } else { - onHeapKeys.add(key); - } LoadAwareCacheLoader, String> tieredCacheLoader = getLoadAwareCacheLoader(); tieredSpilloverCache.computeIfAbsent(key, tieredCacheLoader); } - assertEquals(numOfItems1, tieredSpilloverCache.count()); + assertEquals( + getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP) + getItemsForTier( + tieredSpilloverCache, + TIER_DIMENSION_VALUE_DISK + ), + tieredSpilloverCache.count() + ); tieredSpilloverCache.invalidateAll(); assertEquals(0, tieredSpilloverCache.count()); } @@ -767,9 +958,9 @@ public void testComputeIfAbsentConcurrently() throws Exception { MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); Settings settings = Settings.builder() .put( - OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) - .get(MAXIMUM_SIZE_IN_BYTES_KEY) - .getKey(), + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), onHeapCacheSize * keyValueSize + "b" ) .build(); @@ -779,7 +970,8 @@ public void testComputeIfAbsentConcurrently() throws Exception { diskCacheSize, removalListener, settings, - 0 + 0, + 1 ); int numberOfSameKeys = randomIntBetween(400, onHeapCacheSize - 1); @@ -837,16 +1029,17 @@ public String load(ICacheKey key) { } public void testComputIfAbsentConcurrentlyWithMultipleKeys() throws Exception { - int onHeapCacheSize = randomIntBetween(300, 500); + int numberOfSegments = getNumberOfSegments(); + int onHeapCacheSize = randomIntBetween(300, 500) * numberOfSegments; // Able to support all keys in case of + // skewness as well. int diskCacheSize = randomIntBetween(600, 700); int keyValueSize = 50; - MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); Settings settings = Settings.builder() .put( - OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) - .get(MAXIMUM_SIZE_IN_BYTES_KEY) - .getKey(), + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), onHeapCacheSize * keyValueSize + "b" ) .build(); @@ -856,7 +1049,8 @@ public void testComputIfAbsentConcurrentlyWithMultipleKeys() throws Exception { diskCacheSize, removalListener, settings, - 0 + 0, + numberOfSegments ); int iterations = 10; @@ -942,9 +1136,9 @@ public void testComputeIfAbsentWithOnHeapCacheThrowingExceptionOnPut() throws Ex MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); Settings settings = Settings.builder() .put( - OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) - .get(MAXIMUM_SIZE_IN_BYTES_KEY) - .getKey(), + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), onHeapCacheSize * keyValueSize + "b" ) .build(); @@ -952,15 +1146,18 @@ public void testComputeIfAbsentWithOnHeapCacheThrowingExceptionOnPut() throws Ex ICache mockOnHeapCache = mock(ICache.class); when(onHeapCacheFactory.create(any(), any(), any())).thenReturn(mockOnHeapCache); doThrow(new RuntimeException("Testing")).when(mockOnHeapCache).put(any(), any()); - CacheConfig cacheConfig = getCacheConfig(keyValueSize, settings, removalListener); - ICache.Factory mockDiskCacheFactory = new MockDiskCache.MockDiskCacheFactory(0, diskCacheSize, false); + CacheConfig cacheConfig = getCacheConfig(keyValueSize, settings, removalListener, 1); + ICache.Factory mockDiskCacheFactory = new MockDiskCache.MockDiskCacheFactory(0, diskCacheSize, false, keyValueSize); TieredSpilloverCache tieredSpilloverCache = getTieredSpilloverCache( onHeapCacheFactory, mockDiskCacheFactory, cacheConfig, null, - removalListener + removalListener, + 1, + onHeapCacheSize * keyValueSize, + diskCacheSize ); String value = ""; value = tieredSpilloverCache.computeIfAbsent(getICacheKey("test"), new LoadAwareCacheLoader<>() { @@ -980,20 +1177,20 @@ public String load(ICacheKey key) { @SuppressWarnings({ "rawtypes", "unchecked" }) public void testComputeIfAbsentWithDiskCacheThrowingExceptionOnPut() throws Exception { - int onHeapCacheSize = 0; + int onHeapCacheSize = 1; int keyValueSize = 50; MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); Settings settings = Settings.builder() .put( - OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) - .get(MAXIMUM_SIZE_IN_BYTES_KEY) - .getKey(), + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), onHeapCacheSize * keyValueSize + "b" ) .build(); ICache.Factory onHeapCacheFactory = new OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory(); - CacheConfig cacheConfig = getCacheConfig(keyValueSize, settings, removalListener); + CacheConfig cacheConfig = getCacheConfig(keyValueSize, settings, removalListener, 1); ICache.Factory mockDiskCacheFactory = mock(MockDiskCache.MockDiskCacheFactory.class); ICache mockDiskCache = mock(ICache.class); when(mockDiskCacheFactory.create(any(), any(), any())).thenReturn(mockDiskCache); @@ -1004,10 +1201,25 @@ public void testComputeIfAbsentWithDiskCacheThrowingExceptionOnPut() throws Exce mockDiskCacheFactory, cacheConfig, null, - removalListener + removalListener, + 1, + onHeapCacheSize * keyValueSize, + 200 ); String response = ""; + // This first computeIfAbsent ensures onHeap cache has 1 item present and rest will be evicted to disk. + tieredSpilloverCache.computeIfAbsent(getICacheKey("test1"), new LoadAwareCacheLoader<>() { + @Override + public boolean isLoaded() { + return false; + } + + @Override + public String load(ICacheKey key) { + return "test1"; + } + }); response = tieredSpilloverCache.computeIfAbsent(getICacheKey("test"), new LoadAwareCacheLoader<>() { @Override public boolean isLoaded() { @@ -1050,29 +1262,37 @@ public void testConcurrencyForEvictionFlowFromOnHeapToDiskTier() throws Exceptio MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); ICache.Factory onHeapCacheFactory = new OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory(); - ICache.Factory diskCacheFactory = new MockDiskCache.MockDiskCacheFactory(500, diskCacheSize, false); + ICache.Factory diskCacheFactory = new MockDiskCache.MockDiskCacheFactory(500, diskCacheSize, false, 1); + + Settings settings = Settings.builder() + .put( + CacheSettings.getConcreteStoreNameSettingForCacheType(CacheType.INDICES_REQUEST_CACHE).getKey(), + TieredSpilloverCache.TieredSpilloverCacheFactory.TIERED_SPILLOVER_CACHE_NAME + ) + .put(FeatureFlags.PLUGGABLE_CACHE, "true") + .put( + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), + 200 + "b" + ) + .build(); + String storagePath; + try (NodeEnvironment environment = newNodeEnvironment(settings)) { + storagePath = environment.nodePaths()[0].path + "/test"; + } catch (IOException e) { + throw new OpenSearchException("Exception occurred", e); + } + CacheConfig cacheConfig = new CacheConfig.Builder().setKeyType(String.class) .setKeyType(String.class) .setWeigher((k, v) -> 150) .setRemovalListener(removalListener) .setKeySerializer(new StringSerializer()) .setValueSerializer(new StringSerializer()) - .setSettings( - Settings.builder() - .put( - CacheSettings.getConcreteStoreNameSettingForCacheType(CacheType.INDICES_REQUEST_CACHE).getKey(), - TieredSpilloverCache.TieredSpilloverCacheFactory.TIERED_SPILLOVER_CACHE_NAME - ) - .put(FeatureFlags.PLUGGABLE_CACHE, "true") - .put( - OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) - .get(MAXIMUM_SIZE_IN_BYTES_KEY) - .getKey(), - 200 + "b" - ) - .build() - ) + .setSettings(settings) .setClusterSettings(clusterSettings) + .setStoragePath(storagePath) .setDimensionNames(dimensionNames) .build(); TieredSpilloverCache tieredSpilloverCache = new TieredSpilloverCache.Builder() @@ -1080,6 +1300,9 @@ public void testConcurrencyForEvictionFlowFromOnHeapToDiskTier() throws Exceptio .setDiskCacheFactory(diskCacheFactory) .setRemovalListener(removalListener) .setCacheConfig(cacheConfig) + .setOnHeapCacheSizeInBytes(200) + .setDiskCacheSize(diskCacheSize) + .setNumberOfSegments(1) .setCacheType(CacheType.INDICES_REQUEST_CACHE) .build(); @@ -1104,9 +1327,7 @@ public void testConcurrencyForEvictionFlowFromOnHeapToDiskTier() throws Exceptio }); thread.start(); assertBusy(() -> { assertTrue(loadAwareCacheLoader.isLoaded()); }, 100, TimeUnit.MILLISECONDS); // We wait for new key to be loaded - // after which it eviction flow is - // guaranteed to occur. - ICache onDiskCache = tieredSpilloverCache.getDiskCache(); + // after which it eviction flow is guaranteed to occur. // Now on a different thread, try to get key(above one which got evicted) from tiered cache. We expect this // should return not null value as it should be present on diskCache. @@ -1124,13 +1345,13 @@ public void testConcurrencyForEvictionFlowFromOnHeapToDiskTier() throws Exceptio assertNotNull(actualValue.get()); countDownLatch1.await(); - assertEquals(1, tieredSpilloverCache.getOnHeapCache().count()); - assertEquals(1, onDiskCache.count()); + assertEquals(1, tieredSpilloverCache.onHeapCacheCount()); + assertEquals(1, tieredSpilloverCache.diskCacheCount()); assertEquals(1, getEvictionsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); assertEquals(1, getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP)); assertEquals(1, getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK)); - assertNotNull(onDiskCache.get(keyToBeEvicted)); + assertNotNull(tieredSpilloverCache.getTieredCacheSegment(keyToBeEvicted).getDiskCache().get(keyToBeEvicted)); } public void testDiskTierPolicies() throws Exception { @@ -1140,7 +1361,6 @@ public void testDiskTierPolicies() throws Exception { policies.add(new AllowEvenLengths()); int keyValueSize = 50; - int onHeapCacheSize = 0; MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); TieredSpilloverCache tieredSpilloverCache = intializeTieredSpilloverCache( keyValueSize, @@ -1148,14 +1368,15 @@ public void testDiskTierPolicies() throws Exception { removalListener, Settings.builder() .put( - OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) - .get(MAXIMUM_SIZE_IN_BYTES_KEY) - .getKey(), - onHeapCacheSize * keyValueSize + "b" + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), + keyValueSize - 1 + "b" ) .build(), 0, - policies + policies, + 1 ); Map keyValuePairs = new HashMap<>(); @@ -1233,15 +1454,16 @@ public void testTookTimePolicyFromFactory() throws Exception { MockDiskCache.MockDiskCacheFactory.NAME ) .put( - OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) - .get(MAXIMUM_SIZE_IN_BYTES_KEY) - .getKey(), + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), onHeapCacheSize * keyValueSize + "b" ) .put( TieredSpilloverCacheSettings.TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP.get(CacheType.INDICES_REQUEST_CACHE).getKey(), new TimeValue(timeValueThresholdNanos / 1_000_000) ) + .put(TIERED_SPILLOVER_SEGMENTS.getConcreteSettingForNamespace(CacheType.INDICES_REQUEST_CACHE.getSettingPrefix()).getKey(), 1) .build(); ICache tieredSpilloverICache = new TieredSpilloverCache.TieredSpilloverCacheFactory().create( @@ -1261,13 +1483,14 @@ public CachedQueryResult.PolicyValues apply(String s) { } }) .setClusterSettings(clusterSettings) + .setStoragePath(getStoragePath(settings)) .build(), CacheType.INDICES_REQUEST_CACHE, Map.of( OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory.NAME, new OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory(), MockDiskCache.MockDiskCacheFactory.NAME, - new MockDiskCache.MockDiskCacheFactory(0, randomIntBetween(100, 300), false) + new MockDiskCache.MockDiskCacheFactory(0, randomIntBetween(100, 300), false, keyValueSize) ) ); @@ -1279,13 +1502,14 @@ public CachedQueryResult.PolicyValues apply(String s) { } assertEquals(tookTimeMap.size(), tieredSpilloverCache.count()); - // Ensure all these keys get evicted from the on heap tier by adding > heap tier size worth of random keys + // Ensure all these keys get evicted from the on heap tier by adding > heap tier size worth of random keys (this works as we have 1 + // segment) for (int i = 0; i < onHeapCacheSize; i++) { tieredSpilloverCache.computeIfAbsent(getICacheKey(UUID.randomUUID().toString()), getLoadAwareCacheLoader(keyValueMap)); } - ICache onHeapCache = tieredSpilloverCache.getOnHeapCache(); for (String key : tookTimeMap.keySet()) { - assertNull(onHeapCache.get(getICacheKey(key))); + ICacheKey iCacheKey = getICacheKey(key); + assertNull(tieredSpilloverCache.getTieredCacheSegment(iCacheKey).getOnHeapCache().get(iCacheKey)); } // Now the original keys should be in the disk tier if the policy allows them, or misses if not @@ -1320,10 +1544,10 @@ public void testMinimumThresholdSettingValue() throws Exception { public void testPutWithDiskCacheDisabledSetting() throws Exception { int onHeapCacheSize = randomIntBetween(10, 30); - int diskCacheSize = randomIntBetween(onHeapCacheSize + 1, 100); + int diskCacheSize = randomIntBetween(300, 500); int keyValueSize = 50; int totalSize = onHeapCacheSize + diskCacheSize; - + int numberOfSegments = getNumberOfSegments(); MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); TieredSpilloverCache tieredSpilloverCache = initializeTieredSpilloverCache( keyValueSize, @@ -1331,14 +1555,15 @@ public void testPutWithDiskCacheDisabledSetting() throws Exception { removalListener, Settings.builder() .put( - OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) - .get(MAXIMUM_SIZE_IN_BYTES_KEY) - .getKey(), + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), onHeapCacheSize * keyValueSize + "b" ) .put(DISK_CACHE_ENABLED_SETTING_MAP.get(CacheType.INDICES_REQUEST_CACHE).getKey(), false) .build(), - 0 + 0, + numberOfSegments ); int numOfItems1 = randomIntBetween(onHeapCacheSize + 1, totalSize); // Create more items than onHeap cache. @@ -1347,59 +1572,95 @@ public void testPutWithDiskCacheDisabledSetting() throws Exception { LoadAwareCacheLoader, String> loadAwareCacheLoader = getLoadAwareCacheLoader(); tieredSpilloverCache.computeIfAbsent(key, loadAwareCacheLoader); } - ICache onHeapCache = tieredSpilloverCache.getOnHeapCache(); - ICache diskCache = tieredSpilloverCache.getDiskCache(); - assertEquals(onHeapCacheSize, onHeapCache.count()); - assertEquals(0, diskCache.count()); // Disk cache shouldn't have anything considering it is disabled. - assertEquals(numOfItems1 - onHeapCacheSize, removalListener.evictionsMetric.count()); + + assertEquals(getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP), tieredSpilloverCache.onHeapCacheCount()); + assertEquals(0, tieredSpilloverCache.diskCacheCount()); // Disk cache shouldn't have anything considering it is + // disabled. + assertEquals(numOfItems1 - tieredSpilloverCache.onHeapCacheCount(), removalListener.evictionsMetric.count()); } public void testGetPutAndInvalidateWithDiskCacheDisabled() throws Exception { - int onHeapCacheSize = randomIntBetween(10, 30); - int diskCacheSize = randomIntBetween(onHeapCacheSize + 1, 100); - int keyValueSize = 50; + int onHeapCacheSize = randomIntBetween(300, 400); + int diskCacheSize = randomIntBetween(onHeapCacheSize + 1, 500); + int keyValueSize = 12; int totalSize = onHeapCacheSize + diskCacheSize; + MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); + int numberOfSegments = getNumberOfSegments(); TieredSpilloverCache tieredSpilloverCache = initializeTieredSpilloverCache( keyValueSize, diskCacheSize, removalListener, Settings.builder() .put( - OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) - .get(MAXIMUM_SIZE_IN_BYTES_KEY) - .getKey(), + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), onHeapCacheSize * keyValueSize + "b" ) .build(), - 0 + 0, + numberOfSegments ); - + Map onHeapCacheExpectedSize = getSegmentOnHeapCacheSize( + numberOfSegments, + onHeapCacheSize * keyValueSize, + keyValueSize + ); + Map mockDiskCacheExpectedSize = getSegmentMockDiskCacheSize(numberOfSegments, diskCacheSize); + Map perSegmentEntryCapacity = new HashMap<>(); + for (int i = 0; i < numberOfSegments; i++) { + int totalEntriesForSegment = onHeapCacheExpectedSize.get(i) + mockDiskCacheExpectedSize.get(i); + perSegmentEntryCapacity.put(i, totalEntriesForSegment); + } int numOfItems1 = randomIntBetween(onHeapCacheSize + 1, totalSize - 1); // Create more items than onHeap // cache to cause spillover. + Map segmentSizeTracker = new HashMap<>(); for (int iter = 0; iter < numOfItems1; iter++) { ICacheKey key = getICacheKey(UUID.randomUUID().toString()); + int keySegment = tieredSpilloverCache.getSegmentNumber(key); LoadAwareCacheLoader, String> loadAwareCacheLoader = getLoadAwareCacheLoader(); tieredSpilloverCache.computeIfAbsent(key, loadAwareCacheLoader); + if (segmentSizeTracker.get(keySegment) == null) { + segmentSizeTracker.put(keySegment, Integer.valueOf(1)); + } else { + Integer updatedValue = segmentSizeTracker.get(keySegment) + 1; + segmentSizeTracker.put(keySegment, updatedValue); + } + } + int expectedEvictions = 0; + for (int i = 0; i < numberOfSegments; i++) { + if (segmentSizeTracker.get(i) == null) { + continue; + } + if (segmentSizeTracker.get(i) > perSegmentEntryCapacity.get(i)) { + expectedEvictions += segmentSizeTracker.get(i) - perSegmentEntryCapacity.get(i); + } } - ICache onHeapCache = tieredSpilloverCache.getOnHeapCache(); - ICache diskCache = tieredSpilloverCache.getDiskCache(); List> diskCacheKeys = new ArrayList<>(); - tieredSpilloverCache.getDiskCache().keys().forEach(diskCacheKeys::add); - long actualDiskCacheCount = diskCache.count(); + tieredSpilloverCache.getDiskCacheKeys().forEach(diskCacheKeys::add); + long actualDiskCacheCount = tieredSpilloverCache.diskCacheCount(); long actualTieredCacheCount = tieredSpilloverCache.count(); - assertEquals(onHeapCacheSize, onHeapCache.count()); - assertEquals(numOfItems1 - onHeapCacheSize, actualDiskCacheCount); - assertEquals(0, removalListener.evictionsMetric.count()); - assertEquals(numOfItems1, actualTieredCacheCount); + assertEquals(getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP), tieredSpilloverCache.onHeapCacheCount()); + assertEquals(getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_DISK), actualDiskCacheCount); + assertEquals(expectedEvictions, removalListener.evictionsMetric.count()); + assertEquals( + getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP) + getItemsForTier( + tieredSpilloverCache, + TIER_DIMENSION_VALUE_DISK + ), + actualTieredCacheCount + ); for (ICacheKey diskKey : diskCacheKeys) { assertNotNull(tieredSpilloverCache.get(diskKey)); } - tieredSpilloverCache.enableDisableDiskCache(false); // Disable disk cache now. int numOfItems2 = totalSize - numOfItems1; for (int iter = 0; iter < numOfItems2; iter++) { ICacheKey key = getICacheKey(UUID.randomUUID().toString()); + int keySegment = tieredSpilloverCache.getSegmentNumber(key); + TieredSpilloverCache.TieredSpilloverCacheSegment segment = + tieredSpilloverCache.tieredSpilloverCacheSegments[keySegment]; LoadAwareCacheLoader, String> loadAwareCacheLoader = getLoadAwareCacheLoader(); tieredSpilloverCache.computeIfAbsent(key, loadAwareCacheLoader); } @@ -1407,20 +1668,21 @@ public void testGetPutAndInvalidateWithDiskCacheDisabled() throws Exception { assertNull(tieredSpilloverCache.get(diskKey)); // Considering disk cache is disabled, we shouldn't find // these keys. } - assertEquals(onHeapCacheSize, onHeapCache.count()); // Should remain same. - assertEquals(0, diskCache.count() - actualDiskCacheCount); // Considering it is disabled now, shouldn't cache + assertEquals(getItemsForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP), tieredSpilloverCache.onHeapCacheCount()); // Should + // remain + // same. + assertEquals(0, tieredSpilloverCache.diskCacheCount() - actualDiskCacheCount); // Considering it is disabled now, shouldn't + // cache // any more items. - assertEquals(numOfItems2, removalListener.evictionsMetric.count()); // Considering onHeap cache was already - // full, we should all existing onHeap entries being evicted. - assertEquals(0, tieredSpilloverCache.count() - actualTieredCacheCount); // Count still returns disk cache - // entries count as they haven't been cleared yet. + assertTrue(removalListener.evictionsMetric.count() > 0); + // Considering onHeap cache was already full, we should have some onHeap entries being evicted. long lastKnownTieredCacheEntriesCount = tieredSpilloverCache.count(); // Clear up disk cache keys. for (ICacheKey diskKey : diskCacheKeys) { tieredSpilloverCache.invalidate(diskKey); } - assertEquals(0, diskCache.count()); + assertEquals(0, tieredSpilloverCache.diskCacheCount()); assertEquals(lastKnownTieredCacheEntriesCount - diskCacheKeys.size(), tieredSpilloverCache.count()); tieredSpilloverCache.invalidateAll(); // Clear up all the keys. @@ -1438,13 +1700,14 @@ public void testTiersDoNotTrackStats() throws Exception { removalListener, Settings.builder() .put( - OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) - .get(MAXIMUM_SIZE_IN_BYTES_KEY) - .getKey(), + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), onHeapCacheSize * keyValueSize + "b" ) .build(), - 0 + 0, + 1 ); // do some gets to put entries in both tiers @@ -1455,8 +1718,11 @@ public void testTiersDoNotTrackStats() throws Exception { tieredSpilloverCache.computeIfAbsent(key, tieredCacheLoader); } assertNotEquals(new ImmutableCacheStats(0, 0, 0, 0, 0), tieredSpilloverCache.stats().getTotalStats()); - assertEquals(new ImmutableCacheStats(0, 0, 0, 0, 0), tieredSpilloverCache.getOnHeapCache().stats().getTotalStats()); - ImmutableCacheStats diskStats = tieredSpilloverCache.getDiskCache().stats().getTotalStats(); + assertEquals( + new ImmutableCacheStats(0, 0, 0, 0, 0), + tieredSpilloverCache.tieredSpilloverCacheSegments[0].getOnHeapCache().stats().getTotalStats() + ); + ImmutableCacheStats diskStats = tieredSpilloverCache.tieredSpilloverCacheSegments[0].getDiskCache().stats().getTotalStats(); assertEquals(new ImmutableCacheStats(0, 0, 0, 0, 0), diskStats); } @@ -1469,8 +1735,9 @@ public void testTierStatsAddCorrectly() throws Exception { * totalEntries = heapEntries + diskEntries */ - int onHeapCacheSize = randomIntBetween(10, 30); - int diskCacheSize = randomIntBetween(onHeapCacheSize + 1, 100); + int onHeapCacheSize = randomIntBetween(300, 600); + int diskCacheSize = randomIntBetween(700, 1200); + int numberOfSegments = getNumberOfSegments(); int keyValueSize = 50; MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); TieredSpilloverCache tieredSpilloverCache = initializeTieredSpilloverCache( @@ -1479,13 +1746,14 @@ public void testTierStatsAddCorrectly() throws Exception { removalListener, Settings.builder() .put( - OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) - .get(MAXIMUM_SIZE_IN_BYTES_KEY) - .getKey(), + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), onHeapCacheSize * keyValueSize + "b" ) .build(), - 0 + 0, + numberOfSegments ); List> usedKeys = new ArrayList<>(); @@ -1529,16 +1797,28 @@ public void testTierStatsAddCorrectly() throws Exception { clusterSettings.applySettings( Settings.builder().put(DISK_CACHE_ENABLED_SETTING_MAP.get(CacheType.INDICES_REQUEST_CACHE).getKey(), false).build() ); - - int newMisses = randomIntBetween(10, 30); - for (int i = 0; i < newMisses; i++) { + Map onHeapExpectedSize = getSegmentOnHeapCacheSize( + numberOfSegments, + onHeapCacheSize * keyValueSize, + keyValueSize + ); + int numOfItems = randomIntBetween(10, 30); + int newMisses = 0; + for (int i = 0; i < numOfItems; i++) { LoadAwareCacheLoader, String> tieredCacheLoader = getLoadAwareCacheLoader(); - tieredSpilloverCache.computeIfAbsent(getICacheKey(UUID.randomUUID().toString()), tieredCacheLoader); + ICacheKey iCacheKey = getICacheKey(UUID.randomUUID().toString()); + int keySegment = tieredSpilloverCache.getSegmentNumber(iCacheKey); + if (tieredSpilloverCache.tieredSpilloverCacheSegments[keySegment].getOnHeapCache().count() >= onHeapExpectedSize.get( + keySegment + )) { + newMisses++; + } + tieredSpilloverCache.computeIfAbsent(iCacheKey, tieredCacheLoader); } totalStats = tieredSpilloverCache.stats().getTotalStats(); heapStats = getStatsSnapshotForTier(tieredSpilloverCache, TIER_DIMENSION_VALUE_ON_HEAP); - assertEquals(missesBeforeDisablingDiskCache + newMisses, totalStats.getMisses()); + assertEquals(missesBeforeDisablingDiskCache + numOfItems, totalStats.getMisses()); assertEquals(heapTierEvictionsBeforeDisablingDiskCache + newMisses, heapStats.getEvictions()); assertEquals(evictionsBeforeDisablingDiskCache + newMisses, totalStats.getEvictions()); @@ -1546,7 +1826,290 @@ public void testTierStatsAddCorrectly() throws Exception { clusterSettings.applySettings( Settings.builder().put(DISK_CACHE_ENABLED_SETTING_MAP.get(CacheType.INDICES_REQUEST_CACHE).getKey(), true).build() ); + } + + public void testPutForAKeyWhichAlreadyExists() { + int onHeapCacheSize = 1; + int diskCacheSize = 3; + int keyValueSize = 1; + MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); + TieredSpilloverCache tieredSpilloverCache = initializeTieredSpilloverCache( + keyValueSize, + diskCacheSize, + removalListener, + Settings.builder() + .put( + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), + onHeapCacheSize * keyValueSize + "b" + ) + .build(), + 0, + 1 + ); + ICacheKey key1 = getICacheKey("key1"); + ICacheKey key2 = getICacheKey("key2"); + tieredSpilloverCache.put(key1, "key1"); // Goes to onHeap cache. + tieredSpilloverCache.put(key2, "key2"); // Goes to onHeap cache. And key1 evicted to disk cache. + List> diskKeys = new ArrayList<>(); + List> onHeapCacheKeys = new ArrayList<>(); + tieredSpilloverCache.tieredSpilloverCacheSegments[0].getDiskCache().keys().forEach(diskKeys::add); + tieredSpilloverCache.tieredSpilloverCacheSegments[0].getOnHeapCache().keys().forEach(onHeapCacheKeys::add); + assertEquals(1, onHeapCacheKeys.size()); + assertEquals(1, diskKeys.size()); + assertTrue(onHeapCacheKeys.contains(key2)); + assertTrue(diskKeys.contains(key1)); + assertEquals("key1", tieredSpilloverCache.get(key1)); + + // Now try to put key1 again onto tiered cache with new value. + tieredSpilloverCache.put(key1, "dummy"); + diskKeys.clear(); + onHeapCacheKeys.clear(); + tieredSpilloverCache.tieredSpilloverCacheSegments[0].getDiskCache().keys().forEach(diskKeys::add); + tieredSpilloverCache.tieredSpilloverCacheSegments[0].getOnHeapCache().keys().forEach(onHeapCacheKeys::add); + assertEquals(1, onHeapCacheKeys.size()); + assertEquals(1, diskKeys.size()); + assertTrue(onHeapCacheKeys.contains(key2)); + assertTrue(diskKeys.contains(key1)); + assertEquals("dummy", tieredSpilloverCache.get(key1)); + } + public void testTieredCacheThrowingExceptionOnPerSegmentSizeBeingZero() { + int onHeapCacheSize = 10; + int diskCacheSize = randomIntBetween(700, 1200); + int keyValueSize = 1; + MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); + assertThrows( + IllegalArgumentException.class, + () -> initializeTieredSpilloverCache( + keyValueSize, + diskCacheSize, + removalListener, + Settings.builder() + .put( + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), + onHeapCacheSize * keyValueSize + "b" + ) + .build(), + 0, + 256 + ) + ); + } + + public void testTieredCacheWithZeroNumberOfSegments() { + int onHeapCacheSize = 10; + int diskCacheSize = randomIntBetween(700, 1200); + int keyValueSize = 1; + MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); + assertThrows( + ZERO_SEGMENT_COUNT_EXCEPTION_MESSAGE, + IllegalArgumentException.class, + () -> initializeTieredSpilloverCache( + keyValueSize, + diskCacheSize, + removalListener, + Settings.builder() + .put( + OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) + .get(MAXIMUM_SIZE_IN_BYTES_KEY) + .getKey(), + onHeapCacheSize * keyValueSize + "b" + ) + .build(), + 0, + 0 + ) + ); + } + + public void testWithInvalidSegmentNumber() throws Exception { + int onHeapCacheSize = 10; + int diskCacheSize = randomIntBetween(700, 1200); + int keyValueSize = 1; + MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); + Settings settings = Settings.builder() + .put( + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_NAME.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), + OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory.NAME + ) + .put( + TieredSpilloverCacheSettings.TIERED_SPILLOVER_DISK_STORE_NAME.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), + MockDiskCache.MockDiskCacheFactory.NAME + ) + .put( + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), + onHeapCacheSize * keyValueSize + "b" + ) + .put( + CacheSettings.getConcreteStoreNameSettingForCacheType(CacheType.INDICES_REQUEST_CACHE).getKey(), + TieredSpilloverCache.TieredSpilloverCacheFactory.TIERED_SPILLOVER_CACHE_NAME + ) + .put(TIERED_SPILLOVER_SEGMENTS.getConcreteSettingForNamespace(CacheType.INDICES_REQUEST_CACHE.getSettingPrefix()).getKey(), 1) + .put(FeatureFlags.PLUGGABLE_CACHE, "true") + .put(TIERED_SPILLOVER_SEGMENTS.getConcreteSettingForNamespace(CacheType.INDICES_REQUEST_CACHE.getSettingPrefix()).getKey(), 3) + .build(); + String storagePath = getStoragePath(settings); + assertThrows( + INVALID_SEGMENT_COUNT_EXCEPTION_MESSAGE, + IllegalArgumentException.class, + () -> new TieredSpilloverCache.TieredSpilloverCacheFactory().create( + new CacheConfig.Builder().setKeyType(String.class) + .setKeyType(String.class) + .setWeigher((k, v) -> keyValueSize) + .setRemovalListener(removalListener) + .setKeySerializer(new StringSerializer()) + .setValueSerializer(new StringSerializer()) + .setSettings(settings) + .setDimensionNames(dimensionNames) + .setCachedResultParser(s -> new CachedQueryResult.PolicyValues(20_000_000L)) // Values will always appear to have taken + // 20_000_000 ns = 20 ms to compute + .setClusterSettings(clusterSettings) + .setStoragePath(storagePath) + .build(), + CacheType.INDICES_REQUEST_CACHE, + Map.of( + OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory.NAME, + new OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory(), + MockDiskCache.MockDiskCacheFactory.NAME, + new MockDiskCache.MockDiskCacheFactory(0, randomIntBetween(100, 300), false, keyValueSize) + ) + ) + ); + } + + public void testWithVeryLowDiskCacheSize() throws Exception { + int onHeapCacheSize = 10; + int diskCacheSize = randomIntBetween(700, 1200); + int keyValueSize = 1; + MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); + Settings settings = Settings.builder() + .put( + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_NAME.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), + OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory.NAME + ) + .put( + TieredSpilloverCacheSettings.TIERED_SPILLOVER_DISK_STORE_NAME.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), + MockDiskCache.MockDiskCacheFactory.NAME + ) + .put( + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), + onHeapCacheSize * keyValueSize + "b" + ) + .put( + CacheSettings.getConcreteStoreNameSettingForCacheType(CacheType.INDICES_REQUEST_CACHE).getKey(), + TieredSpilloverCache.TieredSpilloverCacheFactory.TIERED_SPILLOVER_CACHE_NAME + ) + .put( + TieredSpilloverCacheSettings.TIERED_SPILLOVER_DISK_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), + 1L + ) + .put(FeatureFlags.PLUGGABLE_CACHE, "true") + .put(TIERED_SPILLOVER_SEGMENTS.getConcreteSettingForNamespace(CacheType.INDICES_REQUEST_CACHE.getSettingPrefix()).getKey(), 2) + .build(); + String storagePath = getStoragePath(settings); + assertThrows( + IllegalArgumentException.class, + () -> new TieredSpilloverCache.TieredSpilloverCacheFactory().create( + new CacheConfig.Builder().setKeyType(String.class) + .setKeyType(String.class) + .setWeigher((k, v) -> keyValueSize) + .setRemovalListener(removalListener) + .setKeySerializer(new StringSerializer()) + .setValueSerializer(new StringSerializer()) + .setSettings(settings) + .setDimensionNames(dimensionNames) + .setCachedResultParser(s -> new CachedQueryResult.PolicyValues(20_000_000L)) // Values will always appear to have taken + // 20_000_000 ns = 20 ms to compute + .setClusterSettings(clusterSettings) + .setStoragePath(storagePath) + .build(), + CacheType.INDICES_REQUEST_CACHE, + Map.of( + OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory.NAME, + new OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory(), + MockDiskCache.MockDiskCacheFactory.NAME, + new MockDiskCache.MockDiskCacheFactory(0, randomIntBetween(100, 300), false, keyValueSize) + ) + ) + ); + } + + public void testTieredCacheDefaultSegmentCount() { + int onHeapCacheSize = 500; + int keyValueSize = 1; + + MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); + Settings settings = Settings.builder() + .put( + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_NAME.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), + OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory.NAME + ) + .put( + TieredSpilloverCacheSettings.TIERED_SPILLOVER_DISK_STORE_NAME.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), + MockDiskCache.MockDiskCacheFactory.NAME + ) + .put( + TieredSpilloverCacheSettings.TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).getKey(), + onHeapCacheSize * keyValueSize + "b" + ) + .put( + CacheSettings.getConcreteStoreNameSettingForCacheType(CacheType.INDICES_REQUEST_CACHE).getKey(), + TieredSpilloverCache.TieredSpilloverCacheFactory.TIERED_SPILLOVER_CACHE_NAME + ) + .put(FeatureFlags.PLUGGABLE_CACHE, "true") + .build(); + String storagePath = getStoragePath(settings); + + TieredSpilloverCache tieredSpilloverCache = (TieredSpilloverCache< + String, + String>) new TieredSpilloverCache.TieredSpilloverCacheFactory().create( + new CacheConfig.Builder().setKeyType(String.class) + .setKeyType(String.class) + .setWeigher((k, v) -> keyValueSize) + .setRemovalListener(removalListener) + .setKeySerializer(new StringSerializer()) + .setValueSerializer(new StringSerializer()) + .setSettings(settings) + .setDimensionNames(dimensionNames) + .setCachedResultParser(s -> new CachedQueryResult.PolicyValues(20_000_000L)) // Values will always appear to have taken + // 20_000_000 ns = 20 ms to compute + .setClusterSettings(clusterSettings) + .setStoragePath(storagePath) + .build(), + CacheType.INDICES_REQUEST_CACHE, + Map.of( + OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory.NAME, + new OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory(), + MockDiskCache.MockDiskCacheFactory.NAME, + new MockDiskCache.MockDiskCacheFactory(0, randomIntBetween(100, 300), false, keyValueSize) + ) + ); + assertEquals(TieredSpilloverCacheSettings.defaultSegments(), tieredSpilloverCache.getNumberOfSegments()); + assertTrue(VALID_SEGMENT_COUNT_VALUES.contains(tieredSpilloverCache.getNumberOfSegments())); } private List getMockDimensions() { @@ -1631,7 +2194,10 @@ private TieredSpilloverCache getTieredSpilloverCache( ICache.Factory mockDiskCacheFactory, CacheConfig cacheConfig, List> policies, - RemovalListener, String> removalListener + RemovalListener, String> removalListener, + int numberOfSegments, + long onHeapCacheSizeInBytes, + long diskCacheSize ) { TieredSpilloverCache.Builder builder = new TieredSpilloverCache.Builder().setCacheType( CacheType.INDICES_REQUEST_CACHE @@ -1639,6 +2205,9 @@ private TieredSpilloverCache getTieredSpilloverCache( .setRemovalListener(removalListener) .setOnHeapCacheFactory(onHeapCacheFactory) .setDiskCacheFactory(mockDiskCacheFactory) + .setNumberOfSegments(numberOfSegments) + .setDiskCacheSize(diskCacheSize) + .setOnHeapCacheSizeInBytes(onHeapCacheSizeInBytes) .setCacheConfig(cacheConfig); if (policies != null) { builder.addPolicies(policies); @@ -1654,7 +2223,27 @@ private TieredSpilloverCache initializeTieredSpilloverCache( long diskDeliberateDelay ) { - return intializeTieredSpilloverCache(keyValueSize, diskCacheSize, removalListener, settings, diskDeliberateDelay, null); + return intializeTieredSpilloverCache(keyValueSize, diskCacheSize, removalListener, settings, diskDeliberateDelay, null, 256); + } + + private TieredSpilloverCache initializeTieredSpilloverCache( + int keyValueSize, + int diskCacheSize, + RemovalListener, String> removalListener, + Settings settings, + long diskDeliberateDelay, + int numberOfSegments + + ) { + return intializeTieredSpilloverCache( + keyValueSize, + diskCacheSize, + removalListener, + settings, + diskDeliberateDelay, + null, + numberOfSegments + ); } private TieredSpilloverCache intializeTieredSpilloverCache( @@ -1663,9 +2252,19 @@ private TieredSpilloverCache intializeTieredSpilloverCache( RemovalListener, String> removalListener, Settings settings, long diskDeliberateDelay, - List> policies + List> policies, + int numberOfSegments ) { ICache.Factory onHeapCacheFactory = new OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory(); + String storagePath; + try (NodeEnvironment environment = newNodeEnvironment(settings)) { + storagePath = environment.nodePaths()[0].path + "/test"; + } catch (IOException e) { + throw new OpenSearchException("Exception occurred", e); + } + long onHeapCacheSizeInBytes = TIERED_SPILLOVER_ONHEAP_STORE_SIZE.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ).get(settings).getBytes(); CacheConfig cacheConfig = new CacheConfig.Builder().setKeyType(String.class) .setKeyType(String.class) .setWeigher((k, v) -> keyValueSize) @@ -1685,17 +2284,39 @@ private TieredSpilloverCache intializeTieredSpilloverCache( .build() ) .setClusterSettings(clusterSettings) + .setStoragePath(storagePath) .build(); - ICache.Factory mockDiskCacheFactory = new MockDiskCache.MockDiskCacheFactory(diskDeliberateDelay, diskCacheSize, false); + ICache.Factory mockDiskCacheFactory = new MockDiskCache.MockDiskCacheFactory( + diskDeliberateDelay, + diskCacheSize, + false, + keyValueSize + ); - return getTieredSpilloverCache(onHeapCacheFactory, mockDiskCacheFactory, cacheConfig, policies, removalListener); + return getTieredSpilloverCache( + onHeapCacheFactory, + mockDiskCacheFactory, + cacheConfig, + policies, + removalListener, + numberOfSegments, + onHeapCacheSizeInBytes, + diskCacheSize + ); } private CacheConfig getCacheConfig( int keyValueSize, Settings settings, - RemovalListener, String> removalListener + RemovalListener, String> removalListener, + int numberOfSegments ) { + String storagePath; + try (NodeEnvironment environment = newNodeEnvironment(settings)) { + storagePath = environment.nodePaths()[0].path + "/test"; + } catch (IOException e) { + throw new OpenSearchException("Exception occurred", e); + } return new CacheConfig.Builder().setKeyType(String.class) .setKeyType(String.class) .setWeigher((k, v) -> keyValueSize) @@ -1715,6 +2336,8 @@ private CacheConfig getCacheConfig( .build() ) .setClusterSettings(clusterSettings) + .setStoragePath(storagePath) + .setSegmentCount(numberOfSegments) .build(); } @@ -1756,6 +2379,16 @@ private ImmutableCacheStats getStatsSnapshotForTier(TieredSpilloverCache t return snapshot; } + private String getStoragePath(Settings settings) { + String storagePath; + try (NodeEnvironment environment = newNodeEnvironment(settings)) { + storagePath = environment.nodePaths()[0].path + "/test"; + } catch (IOException e) { + throw new OpenSearchException("Exception occurred", e); + } + return storagePath; + } + private void verifyComputeIfAbsentThrowsException( Class expectedException, LoadAwareCacheLoader, String> loader, @@ -1780,7 +2413,8 @@ private void verifyComputeIfAbsentThrowsException( diskCacheSize, removalListener, settings, - 0 + 0, + 1 ); int numberOfSameKeys = randomIntBetween(10, onHeapCacheSize - 1); @@ -1816,6 +2450,29 @@ private void verifyComputeIfAbsentThrowsException( assertEquals(0, tieredSpilloverCache.completableFutureMap.size()); } + private int getNumberOfSegments() { + return randomFrom(2, 4, 8, 16, 32, 64, 128, 256); + } + + private Map getSegmentOnHeapCacheSize(int numberOfSegments, int onHeapCacheSizeInBytes, int keyValueSize) { + Map expectedSegmentOnHeapCacheSize = new HashMap<>(); + for (int i = 0; i < numberOfSegments; i++) { + int perSegmentOnHeapCacheSizeBytes = onHeapCacheSizeInBytes / numberOfSegments; + int perSegmentOnHeapCacheEntries = perSegmentOnHeapCacheSizeBytes / keyValueSize; + expectedSegmentOnHeapCacheSize.put(i, perSegmentOnHeapCacheEntries); + } + return expectedSegmentOnHeapCacheSize; + } + + private Map getSegmentMockDiskCacheSize(int numberOfSegments, int diskCacheSize) { + Map expectedSegmentDiskCacheSize = new HashMap<>(); + for (int i = 0; i < numberOfSegments; i++) { + int perSegmentDiskCacheEntries = diskCacheSize / numberOfSegments; + expectedSegmentDiskCacheSize.put(i, perSegmentDiskCacheEntries); + } + return expectedSegmentDiskCacheSize; + } + private ImmutableCacheStats getTotalStatsSnapshot(TieredSpilloverCache tsc) throws IOException { ImmutableCacheStatsHolder cacheStats = tsc.stats(new String[0]); return cacheStats.getStatsForDimensionValues(List.of()); diff --git a/plugins/cache-ehcache/src/main/java/org/opensearch/cache/EhcacheDiskCacheSettings.java b/plugins/cache-ehcache/src/main/java/org/opensearch/cache/EhcacheDiskCacheSettings.java index d173155080f6a..cbc104f2d0b00 100644 --- a/plugins/cache-ehcache/src/main/java/org/opensearch/cache/EhcacheDiskCacheSettings.java +++ b/plugins/cache-ehcache/src/main/java/org/opensearch/cache/EhcacheDiskCacheSettings.java @@ -88,7 +88,7 @@ public class EhcacheDiskCacheSettings { */ public static final Setting.AffixSetting DISK_CACHE_ALIAS_SETTING = Setting.suffixKeySetting( EhcacheDiskCache.EhcacheDiskCacheFactory.EHCACHE_DISK_CACHE_NAME + ".alias", - (key) -> Setting.simpleString(key, "", NodeScope) + (key) -> Setting.simpleString(key, "ehcache_disk", NodeScope) ); /** 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 4a95b04de3952..0fa0f8162bb98 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 @@ -101,14 +101,18 @@ public class EhcacheDiskCache implements ICache { private static final Logger logger = LogManager.getLogger(EhcacheDiskCache.class); // Unique id associated with this cache. - private final static String UNIQUE_ID = UUID.randomUUID().toString(); - private final static String THREAD_POOL_ALIAS_PREFIX = "ehcachePool"; + final static String UNIQUE_ID = UUID.randomUUID().toString(); + final static String THREAD_POOL_ALIAS_PREFIX = "ehcachePool"; + final static int MINIMUM_MAX_SIZE_IN_BYTES = 1024 * 100; // 100KB + final static String CACHE_DATA_CLEANUP_DURING_INITIALIZATION_EXCEPTION = "Failed to delete ehcache disk cache under " + + "path: %s during initialization. Please clean this up manually and restart the process"; + // A Cache manager can create many caches. private final PersistentCacheManager cacheManager; // Disk cache. Using ByteArrayWrapper to compare two byte[] by values rather than the default reference checks @SuppressWarnings({ "rawtypes" }) // We have to use the raw type as there's no way to pass the "generic class" to ehcache - private Cache cache; + private final Cache cache; private final long maxWeightInBytes; private final String storagePath; private final Class keyType; @@ -124,10 +128,6 @@ public class EhcacheDiskCache implements ICache { private final Serializer keySerializer; private final Serializer valueSerializer; - final static int MINIMUM_MAX_SIZE_IN_BYTES = 1024 * 100; // 100KB - final static String CACHE_DATA_CLEANUP_DURING_INITIALIZATION_EXCEPTION = "Failed to delete ehcache disk cache under " - + "path: %s during initialization. Please clean this up manually and restart the process"; - /** * Used in computeIfAbsent to synchronize loading of a given key. This is needed as ehcache doesn't provide a * computeIfAbsent method. @@ -199,6 +199,12 @@ private Cache buildCache(Duration expireAfterAccess // Creating the cache requires permissions specified in plugin-security.policy return AccessController.doPrivileged((PrivilegedAction>) () -> { try { + int segmentCount = (Integer) EhcacheDiskCacheSettings.getSettingListForCacheType(cacheType) + .get(DISK_SEGMENT_KEY) + .get(settings); + if (builder.getNumberOfSegments() > 0) { + segmentCount = builder.getNumberOfSegments(); + } return this.cacheManager.createCache( this.diskCacheAlias, CacheConfigurationBuilder.newCacheConfigurationBuilder( @@ -232,7 +238,7 @@ public Duration getExpiryForUpdate( (Integer) EhcacheDiskCacheSettings.getSettingListForCacheType(cacheType) .get(DISK_WRITE_CONCURRENCY_KEY) .get(settings), - (Integer) EhcacheDiskCacheSettings.getSettingListForCacheType(cacheType).get(DISK_SEGMENT_KEY).get(settings) + segmentCount ) ) .withKeySerializer(new KeySerializerWrapper(keySerializer)) @@ -709,8 +715,19 @@ public ICache create(CacheConfig config, CacheType cacheType, throw new IllegalArgumentException("EhcacheDiskCache requires a value serializer of type Serializer"); } - return new Builder().setStoragePath((String) settingList.get(DISK_STORAGE_PATH_KEY).get(settings)) - .setDiskCacheAlias((String) settingList.get(DISK_CACHE_ALIAS_KEY).get(settings)) + String storagePath = (String) settingList.get(DISK_STORAGE_PATH_KEY).get(settings); + // If we read the storage path directly from the setting, we have to add the segment number at the end. + if (storagePath == null || storagePath.isBlank()) { + // In case storage path is not explicitly set by user, use default path. + // Since this comes from the TSC, it already has the segment number at the end. + storagePath = config.getStoragePath(); + } + String diskCacheAlias = (String) settingList.get(DISK_CACHE_ALIAS_KEY).get(settings); + if (config.getCacheAlias() != null && !config.getCacheAlias().isBlank()) { + diskCacheAlias = config.getCacheAlias(); + } + EhcacheDiskCache.Builder builder = (Builder) new Builder().setStoragePath(storagePath) + .setDiskCacheAlias(diskCacheAlias) .setIsEventListenerModeSync((Boolean) settingList.get(DISK_LISTENER_MODE_SYNC_KEY).get(settings)) .setCacheType(cacheType) .setKeyType((config.getKeyType())) @@ -721,9 +738,21 @@ public ICache create(CacheConfig config, CacheType cacheType, .setWeigher(config.getWeigher()) .setRemovalListener(config.getRemovalListener()) .setExpireAfterAccess((TimeValue) settingList.get(DISK_CACHE_EXPIRE_AFTER_ACCESS_KEY).get(settings)) - .setMaximumWeightInBytes((Long) settingList.get(DISK_MAX_SIZE_IN_BYTES_KEY).get(settings)) - .setSettings(settings) - .build(); + .setSettings(settings); + long maxSizeInBytes = (Long) settingList.get(DISK_MAX_SIZE_IN_BYTES_KEY).get(settings); + // If config value is set, use this instead. + if (config.getMaxSizeInBytes() > 0) { + builder.setMaximumWeightInBytes(config.getMaxSizeInBytes()); + } else { + builder.setMaximumWeightInBytes(maxSizeInBytes); + } + int segmentCount = (Integer) EhcacheDiskCacheSettings.getSettingListForCacheType(cacheType).get(DISK_SEGMENT_KEY).get(settings); + if (config.getSegmentCount() > 0) { + builder.setNumberOfSegments(config.getSegmentCount()); + } else { + builder.setNumberOfSegments(segmentCount); + } + return builder.build(); } @Override diff --git a/plugins/cache-ehcache/src/test/java/org/opensearch/cache/store/disk/EhCacheDiskCacheTests.java b/plugins/cache-ehcache/src/test/java/org/opensearch/cache/store/disk/EhCacheDiskCacheTests.java index 2bc24227bb513..a0d0aa4ec4914 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 @@ -433,6 +433,74 @@ public String load(ICacheKey key) { } } + public void testComputeIfAbsentConcurrentlyWithMultipleEhcacheDiskCache() throws IOException { + Settings settings = Settings.builder().build(); + MockRemovalListener removalListener = new MockRemovalListener<>(); + List> iCaches = new ArrayList<>(); + int segments = 4; + try (NodeEnvironment env = newNodeEnvironment(settings)) { + ICache.Factory ehcacheFactory = new EhcacheDiskCache.EhcacheDiskCacheFactory(); + for (int i = 1; i <= segments; i++) { + ICache ehcacheTest = ehcacheFactory.create( + new CacheConfig.Builder().setValueType(String.class) + .setKeyType(String.class) + .setRemovalListener(removalListener) + .setKeySerializer(new StringSerializer()) + .setValueSerializer(new StringSerializer()) + .setDimensionNames(List.of(dimensionName)) + .setWeigher(getWeigher()) + .setMaxSizeInBytes(CACHE_SIZE_IN_BYTES * 100) + .setSettings( + Settings.builder() + .put( + EhcacheDiskCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) + .get(DISK_MAX_SIZE_IN_BYTES_KEY) + .getKey(), + CACHE_SIZE_IN_BYTES + ) + .put( + EhcacheDiskCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) + .get(DISK_STORAGE_PATH_KEY) + .getKey(), + env.nodePaths()[0].indicesPath.toString() + "/request_cache/" + i + ) + .put( + EhcacheDiskCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) + .get(DISK_LISTENER_MODE_SYNC_KEY) + .getKey(), + true + ) + .build() + ) + .build(), + CacheType.INDICES_REQUEST_CACHE, + Map.of() + ); + iCaches.add(ehcacheTest); + } + int randomKeys = randomIntBetween(100, 300); + Map, String> keyValueMap = new HashMap<>(); + for (int i = 0; i < randomKeys; i++) { + keyValueMap.put(getICacheKey(UUID.randomUUID().toString()), UUID.randomUUID().toString()); + } + for (Map.Entry, String> entry : keyValueMap.entrySet()) { + ICache ehcacheTest = iCaches.get(entry.getKey().hashCode() & (segments - 1)); + ehcacheTest.put(entry.getKey(), entry.getValue()); + } + for (Map.Entry, String> entry : keyValueMap.entrySet()) { + ICache ehcacheTest = iCaches.get(entry.getKey().hashCode() & (segments - 1)); + String value = ehcacheTest.get(entry.getKey()); + assertEquals(entry.getValue(), value); + } + int count = 0; + for (int i = 0; i < segments; i++) { + count += iCaches.get(i).count(); + iCaches.get(i).close(); + } + assertEquals(randomKeys, count); + } + } + public void testComputeIfAbsentConcurrentlyAndThrowsException() throws Exception { Settings settings = Settings.builder().build(); MockRemovalListener removalListener = new MockRemovalListener<>(); diff --git a/server/src/main/java/org/opensearch/common/cache/Cache.java b/server/src/main/java/org/opensearch/common/cache/Cache.java index caae81e4387b4..e01a1223955ed 100644 --- a/server/src/main/java/org/opensearch/common/cache/Cache.java +++ b/server/src/main/java/org/opensearch/common/cache/Cache.java @@ -115,8 +115,20 @@ public class Cache { // the removal callback private RemovalListener removalListener = notification -> {}; - // use CacheBuilder to construct - Cache() {} + private final int numberOfSegments; + public static final int NUMBER_OF_SEGMENTS = 256; + + Cache(final int numberOfSegments) { + if (numberOfSegments != -1) { + this.numberOfSegments = numberOfSegments; + } else { + this.numberOfSegments = NUMBER_OF_SEGMENTS; + } + this.segments = new CacheSegment[this.numberOfSegments]; + for (int i = 0; i < this.numberOfSegments; i++) { + this.segments[i] = new CacheSegment<>(); + } + } void setExpireAfterAccessNanos(long expireAfterAccessNanos) { if (expireAfterAccessNanos <= 0) { @@ -366,15 +378,8 @@ void eviction() { } } - public static final int NUMBER_OF_SEGMENTS = 256; @SuppressWarnings("unchecked") - private final CacheSegment[] segments = new CacheSegment[NUMBER_OF_SEGMENTS]; - - { - for (int i = 0; i < segments.length; i++) { - segments[i] = new CacheSegment<>(); - } - } + private final CacheSegment[] segments; Entry head; Entry tail; @@ -382,6 +387,10 @@ void eviction() { // lock protecting mutations to the LRU list private final ReleasableLock lruLock = new ReleasableLock(new ReentrantLock()); + int getNumberOfSegments() { + return numberOfSegments; + } + /** * Returns the value to which the specified key is mapped, or null if this map contains no mapping for the key. * @@ -588,9 +597,9 @@ public void invalidate(K key, V value) { public void invalidateAll() { Entry h; - boolean[] haveSegmentLock = new boolean[NUMBER_OF_SEGMENTS]; + boolean[] haveSegmentLock = new boolean[this.numberOfSegments]; try { - for (int i = 0; i < NUMBER_OF_SEGMENTS; i++) { + for (int i = 0; i < this.numberOfSegments; i++) { segments[i].segmentLock.writeLock().lock(); haveSegmentLock[i] = true; } @@ -607,7 +616,7 @@ public void invalidateAll() { weight = 0; } } finally { - for (int i = NUMBER_OF_SEGMENTS - 1; i >= 0; i--) { + for (int i = this.numberOfSegments - 1; i >= 0; i--) { if (haveSegmentLock[i]) { segments[i].segmentLock.writeLock().unlock(); } @@ -940,6 +949,6 @@ public ToLongBiFunction getWeigher() { } private CacheSegment getCacheSegment(K key) { - return segments[key.hashCode() & 0xff]; + return segments[key.hashCode() & (this.numberOfSegments - 1)]; } } diff --git a/server/src/main/java/org/opensearch/common/cache/CacheBuilder.java b/server/src/main/java/org/opensearch/common/cache/CacheBuilder.java index b6d7979aa4108..8f93e3358a375 100644 --- a/server/src/main/java/org/opensearch/common/cache/CacheBuilder.java +++ b/server/src/main/java/org/opensearch/common/cache/CacheBuilder.java @@ -34,9 +34,13 @@ import org.opensearch.common.unit.TimeValue; +import java.util.Locale; import java.util.Objects; import java.util.function.ToLongBiFunction; +import static org.opensearch.common.cache.settings.CacheSettings.INVALID_SEGMENT_COUNT_EXCEPTION_MESSAGE; +import static org.opensearch.common.cache.settings.CacheSettings.VALID_SEGMENT_COUNT_VALUES; + /** * The cache builder. * @@ -48,6 +52,7 @@ public class CacheBuilder { private long expireAfterWriteNanos = -1; private ToLongBiFunction weigher; private RemovalListener removalListener; + private int numberOfSegments = -1; public static CacheBuilder builder() { return new CacheBuilder<>(); @@ -55,6 +60,14 @@ public static CacheBuilder builder() { private CacheBuilder() {} + public CacheBuilder setNumberOfSegments(int numberOfSegments) { + if (!VALID_SEGMENT_COUNT_VALUES.contains(numberOfSegments)) { + throw new IllegalArgumentException(String.format(Locale.ROOT, INVALID_SEGMENT_COUNT_EXCEPTION_MESSAGE, "Cache")); + } + this.numberOfSegments = numberOfSegments; + return this; + } + public CacheBuilder setMaximumWeight(long maximumWeight) { if (maximumWeight < 0) { throw new IllegalArgumentException("maximumWeight < 0"); @@ -108,7 +121,7 @@ public CacheBuilder removalListener(RemovalListener removalListener) } public Cache build() { - Cache cache = new Cache<>(); + Cache cache = new Cache<>(numberOfSegments); if (maximumWeight != -1) { cache.setMaximumWeight(maximumWeight); } diff --git a/server/src/main/java/org/opensearch/common/cache/settings/CacheSettings.java b/server/src/main/java/org/opensearch/common/cache/settings/CacheSettings.java index 43a047f0f22c6..7a810d92e3d0d 100644 --- a/server/src/main/java/org/opensearch/common/cache/settings/CacheSettings.java +++ b/server/src/main/java/org/opensearch/common/cache/settings/CacheSettings.java @@ -12,12 +12,25 @@ import org.opensearch.common.cache.CacheType; import org.opensearch.common.settings.Setting; +import java.util.List; + /** * Settings related to cache. */ @ExperimentalApi public class CacheSettings { + /** + * Only includes values which is power of 2 as we use bitwise logic: (key AND (segmentCount -1)) to calculate + * segmentNumber which works well only with such values. Stores it in sorted order. + */ + public static final List VALID_SEGMENT_COUNT_VALUES = List.of(1, 2, 4, 8, 16, 32, 64, 128, 256); + + /** + * Exception message for invalid segment number. + */ + public static final String INVALID_SEGMENT_COUNT_EXCEPTION_MESSAGE = "Cache: %s segment count should be power of two up-to 256"; + /** * Used to store cache store name for desired cache types within OpenSearch. * Setting pattern: {cache_type}.store.name 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 569653bec2a3d..571383a9fce6a 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 @@ -54,8 +54,10 @@ public class OpenSearchOnHeapCache implements ICache, RemovalListene private final List dimensionNames; private final ToLongBiFunction, V> weigher; private final boolean statsTrackingEnabled; + private final long maximumWeight; public OpenSearchOnHeapCache(Builder builder) { + this.maximumWeight = builder.getMaxWeightInBytes(); CacheBuilder, V> cacheBuilder = CacheBuilder., V>builder() .setMaximumWeight(builder.getMaxWeightInBytes()) .weigher(builder.getWeigher()) @@ -63,6 +65,9 @@ public OpenSearchOnHeapCache(Builder builder) { if (builder.getExpireAfterAcess() != null) { cacheBuilder.setExpireAfterAccess(builder.getExpireAfterAcess()); } + if (builder.getNumberOfSegments() > 0) { + cacheBuilder.setNumberOfSegments(builder.getNumberOfSegments()); + } cache = cacheBuilder.build(); this.dimensionNames = Objects.requireNonNull(builder.dimensionNames, "Dimension names can't be null"); this.statsTrackingEnabled = builder.getStatsTrackingEnabled(); @@ -75,6 +80,11 @@ public OpenSearchOnHeapCache(Builder builder) { this.weigher = builder.getWeigher(); } + // package private for testing + long getMaximumWeight() { + return this.maximumWeight; + } + @Override public V get(ICacheKey key) { V value = cache.get(key); @@ -174,18 +184,33 @@ public ICache create(CacheConfig config, CacheType cacheType, boolean statsTrackingEnabled = statsTrackingEnabled(config.getSettings(), config.getStatsTrackingEnabled()); ICacheBuilder builder = new Builder().setDimensionNames(config.getDimensionNames()) .setStatsTrackingEnabled(statsTrackingEnabled) - .setMaximumWeightInBytes(((ByteSizeValue) settingList.get(MAXIMUM_SIZE_IN_BYTES_KEY).get(settings)).getBytes()) .setExpireAfterAccess(((TimeValue) settingList.get(EXPIRE_AFTER_ACCESS_KEY).get(settings))) .setWeigher(config.getWeigher()) .setRemovalListener(config.getRemovalListener()); Setting cacheSettingForCacheType = CacheSettings.CACHE_TYPE_STORE_NAME.getConcreteSettingForNamespace( cacheType.getSettingPrefix() ); + long maxSizeInBytes = ((ByteSizeValue) settingList.get(MAXIMUM_SIZE_IN_BYTES_KEY).get(settings)).getBytes(); + + if (config.getMaxSizeInBytes() > 0) { // If this is passed from upstream(like tieredCache), then use this + // instead. + builder.setMaximumWeightInBytes(config.getMaxSizeInBytes()); + } else { + builder.setMaximumWeightInBytes(maxSizeInBytes); + } + if (config.getSegmentCount() > 0) { + builder.setNumberOfSegments(config.getSegmentCount()); + } else { + builder.setNumberOfSegments(-1); // By default it will use 256 segments. + } + String storeName = cacheSettingForCacheType.get(settings); if (!FeatureFlags.PLUGGABLE_CACHE_SETTING.get(settings) || (storeName == null || storeName.isBlank())) { // For backward compatibility as the user intent is to use older settings. builder.setMaximumWeightInBytes(config.getMaxSizeInBytes()); builder.setExpireAfterAccess(config.getExpireAfterAccess()); + builder.setNumberOfSegments(-1); // By default it will use 256 as we don't want to use this setting + // when user wants to use older default onHeap cache settings. } return builder.build(); } diff --git a/server/src/main/java/org/opensearch/common/cache/store/builders/ICacheBuilder.java b/server/src/main/java/org/opensearch/common/cache/store/builders/ICacheBuilder.java index a308d1db88258..d35e22ffdc978 100644 --- a/server/src/main/java/org/opensearch/common/cache/store/builders/ICacheBuilder.java +++ b/server/src/main/java/org/opensearch/common/cache/store/builders/ICacheBuilder.java @@ -39,6 +39,8 @@ public abstract class ICacheBuilder { private boolean statsTrackingEnabled = true; + private int numberOfSegments; + public ICacheBuilder() {} public ICacheBuilder setMaximumWeightInBytes(long sizeInBytes) { @@ -71,6 +73,11 @@ public ICacheBuilder setStatsTrackingEnabled(boolean statsTrackingEnabled) return this; } + public ICacheBuilder setNumberOfSegments(int numberOfSegments) { + this.numberOfSegments = numberOfSegments; + return this; + } + public long getMaxWeightInBytes() { return maxWeightInBytes; } @@ -79,6 +86,10 @@ public TimeValue getExpireAfterAcess() { return expireAfterAcess; } + public int getNumberOfSegments() { + return numberOfSegments; + } + public ToLongBiFunction, V> getWeigher() { return weigher; } diff --git a/server/src/main/java/org/opensearch/common/cache/store/config/CacheConfig.java b/server/src/main/java/org/opensearch/common/cache/store/config/CacheConfig.java index 0c54ac57a9b18..ddad416c251a9 100644 --- a/server/src/main/java/org/opensearch/common/cache/store/config/CacheConfig.java +++ b/server/src/main/java/org/opensearch/common/cache/store/config/CacheConfig.java @@ -70,6 +70,12 @@ public class CacheConfig { private final boolean statsTrackingEnabled; + private final String storagePath; + + private final int segmentCount; + + private final String cacheAlias; + private CacheConfig(Builder builder) { this.keyType = builder.keyType; this.valueType = builder.valueType; @@ -84,6 +90,9 @@ private CacheConfig(Builder builder) { this.expireAfterAccess = builder.expireAfterAccess; this.clusterSettings = builder.clusterSettings; this.statsTrackingEnabled = builder.statsTrackingEnabled; + this.storagePath = builder.storagePath; + this.segmentCount = builder.segmentCount; + this.cacheAlias = builder.cacheAlias; } public Class getKeyType() { @@ -138,6 +147,18 @@ public boolean getStatsTrackingEnabled() { return statsTrackingEnabled; } + public String getStoragePath() { + return storagePath; + } + + public int getSegmentCount() { + return segmentCount; + } + + public String getCacheAlias() { + return cacheAlias; + } + /** * Builder class to build Cache config related parameters. * @param Type of key. @@ -163,6 +184,9 @@ public static class Builder { private TimeValue expireAfterAccess; private ClusterSettings clusterSettings; private boolean statsTrackingEnabled = true; + private String storagePath; + private int segmentCount; + private String cacheAlias; public Builder() {} @@ -231,6 +255,21 @@ public Builder setStatsTrackingEnabled(boolean statsTrackingEnabled) { return this; } + public Builder setStoragePath(String storagePath) { + this.storagePath = storagePath; + return this; + } + + public Builder setSegmentCount(int segmentCount) { + this.segmentCount = segmentCount; + return this; + } + + public Builder setCacheAlias(String cacheAlias) { + this.cacheAlias = cacheAlias; + return this; + } + public CacheConfig build() { return new CacheConfig<>(this); } diff --git a/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java b/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java index 416cd73abbd25..6dc36f952569f 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java +++ b/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java @@ -68,6 +68,7 @@ import org.opensearch.core.common.io.stream.Writeable; import org.opensearch.core.common.unit.ByteSizeValue; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.env.NodeEnvironment; import org.opensearch.index.shard.IndexShard; import org.opensearch.threadpool.ThreadPool; @@ -167,7 +168,8 @@ public final class IndicesRequestCache implements RemovalListener> cacheEntityFunction, CacheService cacheService, ThreadPool threadPool, - ClusterService clusterService + ClusterService clusterService, + NodeEnvironment nodeEnvironment ) { this.size = INDICES_CACHE_QUERY_SIZE.get(settings); this.expire = INDICES_CACHE_QUERY_EXPIRE.exists(settings) ? INDICES_CACHE_QUERY_EXPIRE.get(settings) : null; @@ -202,6 +204,7 @@ public final class IndicesRequestCache implements RemovalListener keys = new HashSet<>(); Cache cache = CacheBuilder.builder() .setMaximumWeight(numberOfEntries / 2) + .setNumberOfSegments(numberOfSegments) .removalListener(notification -> { keys.remove(notification.getKey()); evictions.incrementAndGet(); @@ -114,11 +116,13 @@ public void testCacheStats() { // check that the evicted entries were evicted in LRU order (first the odds in a batch, then the evens in a batch) // for each batch public void testCacheEvictions() { + int numberOfSegments = randomFrom(1, 2, 4, 8, 16, 64, 128, 256); int maximumWeight = randomIntBetween(1, numberOfEntries); AtomicLong evictions = new AtomicLong(); List evictedKeys = new ArrayList<>(); Cache cache = CacheBuilder.builder() .setMaximumWeight(maximumWeight) + .setNumberOfSegments(numberOfSegments) .removalListener(notification -> { evictions.incrementAndGet(); evictedKeys.add(notification.getKey()); @@ -173,11 +177,13 @@ public void testCacheEvictions() { // cache some entries and exceed the maximum weight, then check that the cache has the expected weight and the // expected evictions occurred public void testWeigher() { + int numberOfSegments = randomFrom(1, 2, 4, 8, 16, 64, 128, 256); int maximumWeight = 2 * numberOfEntries; int weight = randomIntBetween(2, 10); AtomicLong evictions = new AtomicLong(); Cache cache = CacheBuilder.builder() .setMaximumWeight(maximumWeight) + .setNumberOfSegments(numberOfSegments) .weigher((k, v) -> weight) .removalListener(notification -> evictions.incrementAndGet()) .build(); @@ -212,7 +218,8 @@ public void testWeight() { // cache some entries, randomly invalidate some of them, then check that the number of cached entries is correct public void testCount() { - Cache cache = CacheBuilder.builder().build(); + int numberOfSegments = randomFrom(1, 2, 4, 8, 16, 64, 128, 256); + Cache cache = CacheBuilder.builder().setNumberOfSegments(numberOfSegments).build(); int count = 0; for (int i = 0; i < numberOfEntries; i++) { count++; @@ -230,8 +237,9 @@ public void testCount() { // cache some entries, step the clock forward, cache some more entries, step the clock forward and then check that // the first batch of cached entries expired and were removed public void testExpirationAfterAccess() { + int numberOfSegments = randomFrom(1, 2, 4, 8, 16, 64, 128, 256); AtomicLong now = new AtomicLong(); - Cache cache = new Cache() { + Cache cache = new Cache(numberOfSegments) { @Override protected long now() { return now.get(); @@ -267,8 +275,9 @@ protected long now() { } public void testSimpleExpireAfterAccess() { + int numberOfSegments = randomFrom(1, 2, 4, 8, 16, 64, 128, 256); AtomicLong now = new AtomicLong(); - Cache cache = new Cache() { + Cache cache = new Cache(numberOfSegments) { @Override protected long now() { return now.get(); @@ -289,8 +298,9 @@ protected long now() { } public void testExpirationAfterWrite() { + int numberOfSegments = randomFrom(1, 2, 4, 8, 16, 64, 128, 256); AtomicLong now = new AtomicLong(); - Cache cache = new Cache() { + Cache cache = new Cache(numberOfSegments) { @Override protected long now() { return now.get(); @@ -329,8 +339,9 @@ protected long now() { } public void testComputeIfAbsentAfterExpiration() throws ExecutionException { + int numberOfSegments = randomFrom(1, 2, 4, 8, 16, 64, 128, 256); AtomicLong now = new AtomicLong(); - Cache cache = new Cache() { + Cache cache = new Cache(numberOfSegments) { @Override protected long now() { return now.get(); @@ -352,8 +363,10 @@ protected long now() { } public void testComputeIfAbsentDeadlock() throws BrokenBarrierException, InterruptedException { + int numberOfSegments = randomFrom(1, 2, 4, 8, 16, 64, 128, 256); final int numberOfThreads = randomIntBetween(2, 32); final Cache cache = CacheBuilder.builder() + .setNumberOfSegments(numberOfSegments) .setExpireAfterAccess(TimeValue.timeValueNanos(1)) .build(); @@ -386,8 +399,9 @@ public void testComputeIfAbsentDeadlock() throws BrokenBarrierException, Interru // randomly promote some entries, step the clock forward, then check that the promoted entries remain and the // non-promoted entries were removed public void testPromotion() { + int numberOfSegments = randomFrom(1, 2, 4, 8, 16, 64, 128, 256); AtomicLong now = new AtomicLong(); - Cache cache = new Cache() { + Cache cache = new Cache(numberOfSegments) { @Override protected long now() { return now.get(); @@ -420,7 +434,8 @@ protected long now() { // randomly invalidate some cached entries, then check that a lookup for each of those and only those keys is null public void testInvalidate() { - Cache cache = CacheBuilder.builder().build(); + int numberOfSegments = randomFrom(1, 2, 4, 8, 16, 64, 128, 256); + Cache cache = CacheBuilder.builder().setNumberOfSegments(numberOfSegments).build(); for (int i = 0; i < numberOfEntries; i++) { cache.put(i, Integer.toString(i)); } @@ -443,11 +458,15 @@ public void testInvalidate() { // randomly invalidate some cached entries, then check that we receive invalidate notifications for those and only // those entries public void testNotificationOnInvalidate() { + int numberOfSegments = randomFrom(1, 2, 4, 8, 16, 64, 128, 256); Set notifications = new HashSet<>(); - Cache cache = CacheBuilder.builder().removalListener(notification -> { - assertEquals(RemovalReason.INVALIDATED, notification.getRemovalReason()); - notifications.add(notification.getKey()); - }).build(); + Cache cache = CacheBuilder.builder() + .setNumberOfSegments(numberOfSegments) + .removalListener(notification -> { + assertEquals(RemovalReason.INVALIDATED, notification.getRemovalReason()); + notifications.add(notification.getKey()); + }) + .build(); for (int i = 0; i < numberOfEntries; i++) { cache.put(i, Integer.toString(i)); } @@ -491,11 +510,15 @@ public void testInvalidateWithValue() { // randomly invalidate some cached entries, then check that we receive invalidate notifications for those and only // those entries public void testNotificationOnInvalidateWithValue() { + int numberOfSegments = randomFrom(1, 2, 4, 8, 16, 64, 128, 256); Set notifications = new HashSet<>(); - Cache cache = CacheBuilder.builder().removalListener(notification -> { - assertEquals(RemovalReason.INVALIDATED, notification.getRemovalReason()); - notifications.add(notification.getKey()); - }).build(); + Cache cache = CacheBuilder.builder() + .setNumberOfSegments(numberOfSegments) + .removalListener(notification -> { + assertEquals(RemovalReason.INVALIDATED, notification.getRemovalReason()); + notifications.add(notification.getKey()); + }) + .build(); for (int i = 0; i < numberOfEntries; i++) { cache.put(i, Integer.toString(i)); } @@ -607,8 +630,9 @@ public void testNotificationOnReplace() { } public void testComputeIfAbsentLoadsSuccessfully() { + int numberOfSegments = randomFrom(1, 2, 4, 8, 16, 64, 128, 256); Map map = new HashMap<>(); - Cache cache = CacheBuilder.builder().build(); + Cache cache = CacheBuilder.builder().setNumberOfSegments(numberOfSegments).build(); for (int i = 0; i < numberOfEntries; i++) { try { cache.computeIfAbsent(i, k -> { @@ -931,4 +955,12 @@ public void testRemoveUsingValuesIterator() { assertEquals(RemovalReason.INVALIDATED, removalNotifications.get(i).getRemovalReason()); } } + + public void testWithInvalidSegmentNumber() { + assertThrows( + "Number of segments for cache should be a power of two up-to 256", + IllegalArgumentException.class, + () -> CacheBuilder.builder().setMaximumWeight(1000).setNumberOfSegments(21).build() + ); + } } 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 f227db6fee2d1..45a7b273eb41e 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 @@ -105,6 +105,37 @@ public void testStatsWithoutPluggableCaches() throws Exception { } } + public void testWithCacheConfigSettings() { + MockRemovalListener listener = new MockRemovalListener<>(); + int maxKeys = between(10, 50); + ICache.Factory onHeapCacheFactory = new OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory(); + Settings settings = Settings.builder() + .put( + OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) + .get(MAXIMUM_SIZE_IN_BYTES_KEY) + .getKey(), + 1000 + "b" // Setting some random value which shouldn't be honored. + ) + .put(FeatureFlags.PLUGGABLE_CACHE, true) + .build(); + + CacheConfig cacheConfig = new CacheConfig.Builder().setKeyType(String.class) + .setValueType(String.class) + .setWeigher((k, v) -> keyValueSize) + .setRemovalListener(listener) + .setSettings(settings) + .setDimensionNames(dimensionNames) + .setMaxSizeInBytes(maxKeys * keyValueSize) // this should get honored + .setStatsTrackingEnabled(true) + .build(); + OpenSearchOnHeapCache onHeapCache = (OpenSearchOnHeapCache) onHeapCacheFactory.create( + cacheConfig, + CacheType.INDICES_REQUEST_CACHE, + null + ); + assertEquals(maxKeys * keyValueSize, onHeapCache.getMaximumWeight()); + } + private void assertZeroStats(ImmutableCacheStatsHolder stats) { assertEquals(new ImmutableCacheStats(0, 0, 0, 0, 0), stats.getTotalStats()); } diff --git a/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java b/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java index 10688de3ab0ae..1a3aece74b3e2 100644 --- a/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java +++ b/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java @@ -73,6 +73,7 @@ import org.opensearch.core.index.shard.ShardId; import org.opensearch.core.xcontent.MediaTypeRegistry; import org.opensearch.core.xcontent.XContentHelper; +import org.opensearch.env.NodeEnvironment; import org.opensearch.index.IndexNotFoundException; import org.opensearch.index.IndexService; import org.opensearch.index.cache.request.RequestCacheStats; @@ -851,15 +852,18 @@ public void testAddingToCleanupKeyToCountMapWorksAppropriatelyWithMultipleThread assertFalse(concurrentModificationExceptionDetected.get()); } - private IndicesRequestCache getIndicesRequestCache(Settings settings) { + private IndicesRequestCache getIndicesRequestCache(Settings settings) throws IOException { IndicesService indicesService = getInstanceFromNode(IndicesService.class); - return new IndicesRequestCache( - settings, - indicesService.indicesRequestCache.cacheEntityLookup, - new CacheModule(new ArrayList<>(), Settings.EMPTY).getCacheService(), - threadPool, - ClusterServiceUtils.createClusterService(threadPool) - ); + try (NodeEnvironment env = newNodeEnvironment(settings)) { + return new IndicesRequestCache( + settings, + indicesService.indicesRequestCache.cacheEntityLookup, + new CacheModule(new ArrayList<>(), Settings.EMPTY).getCacheService(), + threadPool, + ClusterServiceUtils.createClusterService(threadPool), + env + ); + } } private DirectoryReader getReader(IndexWriter writer, ShardId shardId) throws IOException { @@ -913,23 +917,26 @@ public void testClosingIndexWipesStats() throws Exception { .put(INDICES_REQUEST_CACHE_STALENESS_THRESHOLD_SETTING.getKey(), "0.001%") .put(FeatureFlags.PLUGGABLE_CACHE, true) .build(); - 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, - ClusterServiceUtils.createClusterService(threadPool) - ); + try (NodeEnvironment env = newNodeEnvironment(settings)) { + 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, + ClusterServiceUtils.createClusterService(threadPool), + env + ); + } writer.addDocument(newDoc(0, "foo")); TermQueryBuilder termQuery = new TermQueryBuilder("id", "0"); @@ -1058,6 +1065,7 @@ public void testEviction() throws Exception { IOUtils.close(reader, secondReader, writer, dir, cache); } indexShard = createIndex("test1").getShard(0); + NodeEnvironment environment = newNodeEnvironment(); IndicesRequestCache cache = new IndicesRequestCache( // TODO: Add wiggle room to max size to allow for overhead of ICacheKey. This can be removed once API PR goes in, as it updates // the old API to account for the ICacheKey overhead. @@ -1065,7 +1073,8 @@ public void testEviction() throws Exception { (shardId -> Optional.of(new IndicesService.IndexShardCacheEntity(indexShard))), new CacheModule(new ArrayList<>(), Settings.EMPTY).getCacheService(), threadPool, - ClusterServiceUtils.createClusterService(threadPool) + ClusterServiceUtils.createClusterService(threadPool), + environment ); dir = newDirectory(); writer = new IndexWriter(dir, newIndexWriterConfig()); @@ -1085,7 +1094,7 @@ public void testEviction() throws Exception { assertEquals("baz", value3.streamInput().readString()); assertEquals(2, cache.count()); assertEquals(1, indexShard.requestCache().stats().getEvictions()); - IOUtils.close(reader, secondReader, thirdReader); + IOUtils.close(reader, secondReader, thirdReader, environment); } public void testClearAllEntityIdentity() throws Exception {