diff --git a/CHANGELOG.md b/CHANGELOG.md index 492da15def01b..9b41f1cd4ca01 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -11,12 +11,14 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Allow setting KEYSTORE_PASSWORD through env variable ([#12865](https://github.com/opensearch-project/OpenSearch/pull/12865)) - [Concurrent Segment Search] Perform buildAggregation concurrently and support Composite Aggregations ([#12697](https://github.com/opensearch-project/OpenSearch/pull/12697)) - [Concurrent Segment Search] Disable concurrent segment search for system indices and throttled requests ([#12954](https://github.com/opensearch-project/OpenSearch/pull/12954)) +- [Tiered Caching] Make took time caching policy setting dynamic ([#13063](https://github.com/opensearch-project/OpenSearch/pull/13063)) - Derived fields support to derive field values at query time without indexing ([#12569](https://github.com/opensearch-project/OpenSearch/pull/12569)) - Detect breaking changes on pull requests ([#9044](https://github.com/opensearch-project/OpenSearch/pull/9044)) - Add cluster primary balance contraint for rebalancing with buffer ([#12656](https://github.com/opensearch-project/OpenSearch/pull/12656)) - [Remote Store] Make translog transfer timeout configurable ([#12704](https://github.com/opensearch-project/OpenSearch/pull/12704)) - Reject Resize index requests (i.e, split, shrink and clone), While DocRep to SegRep migration is in progress.([#12686](https://github.com/opensearch-project/OpenSearch/pull/12686)) - Add support for more than one protocol for transport ([#12967](https://github.com/opensearch-project/OpenSearch/pull/12967)) +- [Tiered Caching] Add dimension-based stats to ICache implementations. ([#12531](https://github.com/opensearch-project/OpenSearch/pull/12531)) - Add changes for overriding remote store and replication settings during snapshot restore. ([#11868](https://github.com/opensearch-project/OpenSearch/pull/11868)) ### Dependencies @@ -35,6 +37,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [BWC and API enforcement] Enforcing the presence of API annotations at build time ([#12872](https://github.com/opensearch-project/OpenSearch/pull/12872)) - Improve built-in secure transports support ([#12907](https://github.com/opensearch-project/OpenSearch/pull/12907)) - Update links to documentation in rest-api-spec ([#13043](https://github.com/opensearch-project/OpenSearch/pull/13043)) +- Refactoring globMatch using simpleMatchWithNormalizedStrings from Regex ([#13104](https://github.com/opensearch-project/OpenSearch/pull/13104)) ### Deprecated diff --git a/libs/common/src/main/java/org/opensearch/common/Glob.java b/libs/common/src/main/java/org/opensearch/common/Glob.java index daf045dd49e3a..b390a3ca84182 100644 --- a/libs/common/src/main/java/org/opensearch/common/Glob.java +++ b/libs/common/src/main/java/org/opensearch/common/Glob.java @@ -52,34 +52,35 @@ public static boolean globMatch(String pattern, String str) { if (pattern == null || str == null) { return false; } - int firstIndex = pattern.indexOf('*'); - if (firstIndex == -1) { - return pattern.equals(str); - } - if (firstIndex == 0) { - if (pattern.length() == 1) { - return true; - } - int nextIndex = pattern.indexOf('*', firstIndex + 1); - if (nextIndex == -1) { - return str.endsWith(pattern.substring(1)); - } else if (nextIndex == 1) { - // Double wildcard "**" - skipping the first "*" - return globMatch(pattern.substring(1), str); + int sIdx = 0, pIdx = 0, match = 0, wildcardIdx = -1; + while (sIdx < str.length()) { + // both chars matching, incrementing both pointers + if (pIdx < pattern.length() && str.charAt(sIdx) == pattern.charAt(pIdx)) { + sIdx++; + pIdx++; + } else if (pIdx < pattern.length() && pattern.charAt(pIdx) == '*') { + // wildcard found, only incrementing pattern pointer + wildcardIdx = pIdx; + match = sIdx; + pIdx++; + } else if (wildcardIdx != -1) { + // last pattern pointer was a wildcard, incrementing string pointer + pIdx = wildcardIdx + 1; + match++; + sIdx = match; + } else { + // current pattern pointer is not a wildcard, last pattern pointer was also not a wildcard + // characters do not match + return false; } - String part = pattern.substring(1, nextIndex); - int partIndex = str.indexOf(part); - while (partIndex != -1) { - if (globMatch(pattern.substring(nextIndex), str.substring(partIndex + part.length()))) { - return true; - } - partIndex = str.indexOf(part, partIndex + 1); - } - return false; } - return (str.length() >= firstIndex - && pattern.substring(0, firstIndex).equals(str.substring(0, firstIndex)) - && globMatch(pattern.substring(firstIndex), str.substring(firstIndex))); + + // check for remaining characters in pattern + while (pIdx < pattern.length() && pattern.charAt(pIdx) == '*') { + pIdx++; + } + + return pIdx == pattern.length(); } } diff --git a/libs/x-content/src/test/java/org/opensearch/common/xcontent/XContentParserTests.java b/libs/x-content/src/test/java/org/opensearch/common/xcontent/XContentParserTests.java index 81a2b0e290121..16816a03d742d 100644 --- a/libs/x-content/src/test/java/org/opensearch/common/xcontent/XContentParserTests.java +++ b/libs/x-content/src/test/java/org/opensearch/common/xcontent/XContentParserTests.java @@ -85,8 +85,8 @@ public class XContentParserTests extends OpenSearchTestCase { () -> randomAlphaOfLengthBetween(1, SmileXContent.DEFAULT_MAX_STRING_LEN), /* YAML parser limitation */ XContentType.YAML, - /* use 75% of the limit, difficult to get the exact size of the content right */ - () -> randomRealisticUnicodeOfCodepointLengthBetween(1, (int) (YamlXContent.DEFAULT_CODEPOINT_LIMIT * 0.75)) + /* use 50% of the limit, difficult to get the exact size of the content right */ + () -> randomRealisticUnicodeOfCodepointLengthBetween(1, (int) (YamlXContent.DEFAULT_CODEPOINT_LIMIT * 0.50)) ); private static final Map> OFF_LIMIT_GENERATORS = Map.of( 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 568ac4d188c51..977a66c53b7e8 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 @@ -12,21 +12,31 @@ import org.opensearch.action.admin.cluster.node.info.NodesInfoRequest; import org.opensearch.action.admin.cluster.node.info.NodesInfoResponse; import org.opensearch.action.admin.cluster.node.info.PluginsAndModules; +import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; +import org.opensearch.action.admin.indices.cache.clear.ClearIndicesCacheRequest; +import org.opensearch.action.admin.indices.cache.clear.ClearIndicesCacheResponse; +import org.opensearch.action.admin.indices.forcemerge.ForceMergeResponse; import org.opensearch.action.search.SearchResponse; import org.opensearch.action.search.SearchType; import org.opensearch.client.Client; +import org.opensearch.cluster.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; import org.opensearch.plugins.CachePlugin; import org.opensearch.plugins.Plugin; import org.opensearch.plugins.PluginInfo; import org.opensearch.search.aggregations.bucket.histogram.DateHistogramInterval; import org.opensearch.test.OpenSearchIntegTestCase; +import org.opensearch.test.hamcrest.OpenSearchAssertions; import org.junit.Assert; import java.time.ZoneId; @@ -34,15 +44,20 @@ import java.util.Collection; import java.util.List; import java.util.Map; +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.indices.IndicesService.INDICES_CACHE_CLEAN_INTERVAL_SETTING; import static org.opensearch.search.aggregations.AggregationBuilders.dateHistogram; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertNoFailures; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertSearchResponse; import static org.hamcrest.Matchers.greaterThan; +@OpenSearchIntegTestCase.ClusterScope(numDataNodes = 0, scope = OpenSearchIntegTestCase.Scope.TEST) public class TieredSpilloverCacheIT extends OpenSearchIntegTestCase { @Override @@ -50,15 +65,9 @@ protected Collection> nodePlugins() { return Arrays.asList(TieredSpilloverCachePlugin.class, MockDiskCachePlugin.class); } - @Override - protected Settings featureFlagSettings() { - return Settings.builder().put(super.featureFlagSettings()).put(FeatureFlags.PLUGGABLE_CACHE, "true").build(); - } - - @Override - protected Settings nodeSettings(int nodeOrdinal) { + private Settings defaultSettings(String onHeapCacheSizeInBytesOrPecentage) { return Settings.builder() - .put(super.nodeSettings(nodeOrdinal)) + .put(FeatureFlags.PLUGGABLE_CACHE, "true") .put( CacheSettings.getConcreteStoreNameSettingForCacheType(CacheType.INDICES_REQUEST_CACHE).getKey(), TieredSpilloverCache.TieredSpilloverCacheFactory.TIERED_SPILLOVER_CACHE_NAME @@ -75,10 +84,17 @@ protected Settings nodeSettings(int nodeOrdinal) { ).getKey(), MockDiskCache.MockDiskCacheFactory.NAME ) + .put( + OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) + .get(MAXIMUM_SIZE_IN_BYTES_KEY) + .getKey(), + onHeapCacheSizeInBytesOrPecentage + ) .build(); } public void testPluginsAreInstalled() { + internalCluster().startNode(Settings.builder().put(defaultSettings("1%")).build()); NodesInfoRequest nodesInfoRequest = new NodesInfoRequest(); nodesInfoRequest.addMetric(NodesInfoRequest.Metric.PLUGINS.metricName()); NodesInfoResponse nodesInfoResponse = OpenSearchIntegTestCase.client().admin().cluster().nodesInfo(nodesInfoRequest).actionGet(); @@ -90,11 +106,12 @@ public void testPluginsAreInstalled() { .collect(Collectors.toList()); Assert.assertTrue( pluginInfos.stream() - .anyMatch(pluginInfo -> pluginInfo.getName().equals("org.opensearch.cache.common" + ".tier.TieredSpilloverCachePlugin")) + .anyMatch(pluginInfo -> pluginInfo.getName().equals("org.opensearch.cache.common.tier.TieredSpilloverCachePlugin")) ); } public void testSanityChecksWithIndicesRequestCache() throws InterruptedException { + internalCluster().startNodes(3, Settings.builder().put(defaultSettings("1%")).build()); Client client = client(); assertAcked( client.admin() @@ -118,10 +135,7 @@ public void testSanityChecksWithIndicesRequestCache() throws InterruptedExceptio .setSize(0) .setSearchType(SearchType.QUERY_THEN_FETCH) .addAggregation( - dateHistogram("histo").field("f") - .timeZone(ZoneId.of("+01:00")) - .minDocCount(0) - .dateHistogramInterval(DateHistogramInterval.MONTH) + dateHistogram("histo").field("f").timeZone(ZoneId.of("+01:00")).minDocCount(0).calendarInterval(DateHistogramInterval.MONTH) ) .get(); assertSearchResponse(r1); @@ -133,6 +147,288 @@ public void testSanityChecksWithIndicesRequestCache() throws InterruptedExceptio ); } + public void testWithDynamicTookTimePolicy() throws Exception { + int onHeapCacheSizeInBytes = 2000; + internalCluster().startNode(Settings.builder().put(defaultSettings(onHeapCacheSizeInBytes + "b")).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() + ); + // Step 1 : 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 = randomIntBetween(6, 10); + 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); + long entriesInCache = requestCacheStats.getMemorySizeInBytes() / perQuerySizeInCacheInBytes; + // All should be misses in the first attempt + assertEquals(numberOfIndexedItems, requestCacheStats.getMissCount()); + assertEquals(numberOfIndexedItems - entriesInCache, requestCacheStats.getEvictions()); + assertEquals(0, requestCacheStats.getHitCount()); + + // Step 2: Again hit same set of queries as above, we still won't see any hits as items keeps getting evicted. + for (int iterator = 0; iterator < numberOfIndexedItems; iterator++) { + SearchResponse resp = client.prepareSearch("index") + .setRequestCache(true) + .setQuery(QueryBuilders.termQuery("k" + iterator, "hello" + iterator)) + .get(); + assertSearchResponse(resp); + } + requestCacheStats = getRequestCacheStats(client, "index"); + // We still won't get any hits as items keep getting evicted in LRU fashion due to limited cache size. + assertTrue(requestCacheStats.getMemorySizeInBytes() <= onHeapCacheSizeInBytes); + assertEquals(numberOfIndexedItems * 2, requestCacheStats.getMissCount()); + assertEquals(numberOfIndexedItems * 2 - entriesInCache, requestCacheStats.getEvictions()); + assertEquals(0, requestCacheStats.getHitCount()); + long lastEvictionSeen = requestCacheStats.getEvictions(); + + // Step 3: 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; iterator++) { + SearchResponse resp = client.prepareSearch("index") + .setRequestCache(true) + .setQuery(QueryBuilders.termQuery("k" + iterator, "hello" + iterator)) + .get(); + assertSearchResponse(resp); + } + requestCacheStats = getRequestCacheStats(client, "index"); + // All entries should get cached. + assertEquals(numberOfIndexedItems * perQuerySizeInCacheInBytes, requestCacheStats.getMemorySizeInBytes()); + // No more evictions seen when compared with last step. + assertEquals(0, requestCacheStats.getEvictions() - lastEvictionSeen); + // Hit count should be equal to number of cache entries present in previous step. + assertEquals(entriesInCache, requestCacheStats.getHitCount()); + assertEquals(numberOfIndexedItems * 3 - entriesInCache, requestCacheStats.getMissCount()); + long lastHitCountSeen = requestCacheStats.getHitCount(); + long lastMissCountSeen = requestCacheStats.getMissCount(); + + // Step 4: Again hit the same requests, we should get hits for all entries. + for (int iterator = 0; iterator < numberOfIndexedItems; iterator++) { + SearchResponse resp = client.prepareSearch("index") + .setRequestCache(true) + .setQuery(QueryBuilders.termQuery("k" + iterator, "hello" + iterator)) + .get(); + assertSearchResponse(resp); + } + requestCacheStats = getRequestCacheStats(client, "index"); + // All entries should get cached. + assertEquals(numberOfIndexedItems * perQuerySizeInCacheInBytes, requestCacheStats.getMemorySizeInBytes()); + // No more evictions seen when compared with last step. + assertEquals(0, requestCacheStats.getEvictions() - lastEvictionSeen); + assertEquals(lastHitCountSeen + numberOfIndexedItems, requestCacheStats.getHitCount()); + assertEquals(0, lastMissCountSeen - requestCacheStats.getMissCount()); + } + + public void testInvalidationWithIndicesRequestCache() throws Exception { + int onHeapCacheSizeInBytes = 2000; + internalCluster().startNode( + Settings.builder() + .put(defaultSettings(onHeapCacheSizeInBytes + "b")) + .put(INDICES_CACHE_CLEAN_INTERVAL_SETTING.getKey(), new TimeValue(1)) + .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() + ); + // Update took time policy to zero so that all entries are eligible to be cached on disk. + ClusterUpdateSettingsRequest 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()); + int numberOfIndexedItems = randomIntBetween(5, 10); + 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"); + assertEquals(numberOfIndexedItems, requestCacheStats.getMissCount()); + assertEquals(0, requestCacheStats.getHitCount()); + assertEquals(0, requestCacheStats.getEvictions()); + assertEquals(perQuerySizeInCacheInBytes * numberOfIndexedItems, requestCacheStats.getMemorySizeInBytes()); + for (int iterator = 0; iterator < numberOfIndexedItems; iterator++) { + SearchResponse resp = client.prepareSearch("index") + .setRequestCache(true) + .setQuery(QueryBuilders.termQuery("k" + iterator, "hello" + iterator)) + .get(); + assertSearchResponse(resp); + } + requestCacheStats = client.admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache(); + assertEquals(numberOfIndexedItems, requestCacheStats.getHitCount()); + assertEquals(numberOfIndexedItems, requestCacheStats.getMissCount()); + assertEquals(perQuerySizeInCacheInBytes * numberOfIndexedItems, requestCacheStats.getMemorySizeInBytes()); + assertEquals(0, requestCacheStats.getEvictions()); + // Explicit refresh would invalidate cache entries. + refreshAndWaitForReplication(); + assertBusy(() -> { + // Explicit refresh should clear up cache entries + assertTrue(getRequestCacheStats(client, "index").getMemorySizeInBytes() == 0); + }, 1, TimeUnit.SECONDS); + requestCacheStats = client.admin().indices().prepareStats("index").setRequestCache(true).get().getTotal().getRequestCache(); + assertEquals(0, requestCacheStats.getMemorySizeInBytes()); + // Hits and misses stats shouldn't get cleared up. + assertEquals(numberOfIndexedItems, requestCacheStats.getHitCount()); + assertEquals(numberOfIndexedItems, requestCacheStats.getMissCount()); + } + + public void testWithExplicitCacheClear() throws Exception { + int onHeapCacheSizeInBytes = 2000; + internalCluster().startNode( + Settings.builder() + .put(defaultSettings(onHeapCacheSizeInBytes + "b")) + .put(INDICES_CACHE_CLEAN_INTERVAL_SETTING.getKey(), new TimeValue(1)) + .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() + ); + // Update took time policy to zero so that all entries are eligible to be cached on disk. + ClusterUpdateSettingsRequest 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()); + int numberOfIndexedItems = randomIntBetween(5, 10); + 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"); + assertEquals(numberOfIndexedItems, requestCacheStats.getMissCount()); + assertEquals(0, requestCacheStats.getHitCount()); + assertEquals(0, requestCacheStats.getEvictions()); + assertEquals(perQuerySizeInCacheInBytes * numberOfIndexedItems, requestCacheStats.getMemorySizeInBytes()); + + // Explicit clear the cache. + ClearIndicesCacheRequest request = new ClearIndicesCacheRequest("index"); + ClearIndicesCacheResponse response = client.admin().indices().clearCache(request).get(); + assertNoFailures(response); + + assertBusy(() -> { + // All entries should get cleared up. + assertTrue(getRequestCacheStats(client, "index").getMemorySizeInBytes() == 0); + }, 1, TimeUnit.SECONDS); + } + + private RequestCacheStats getRequestCacheStats(Client client, String indexName) { + return client.admin().indices().prepareStats(indexName).setRequestCache(true).get().getTotal().getRequestCache(); + } + public static class MockDiskCachePlugin extends Plugin implements CachePlugin { public MockDiskCachePlugin() {} diff --git a/modules/cache-common/src/main/java/org/opensearch/cache/common/policy/TookTimePolicy.java b/modules/cache-common/src/main/java/org/opensearch/cache/common/policy/TookTimePolicy.java index 96ef027c17187..4bc26803acf4c 100644 --- a/modules/cache-common/src/main/java/org/opensearch/cache/common/policy/TookTimePolicy.java +++ b/modules/cache-common/src/main/java/org/opensearch/cache/common/policy/TookTimePolicy.java @@ -13,12 +13,16 @@ package org.opensearch.cache.common.policy; +import org.opensearch.common.cache.CacheType; import org.opensearch.common.cache.policy.CachedQueryResult; +import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.unit.TimeValue; import java.util.function.Function; import java.util.function.Predicate; +import static org.opensearch.cache.common.tier.TieredSpilloverCacheSettings.TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP; + /** * A cache tier policy which accepts queries whose took time is greater than some threshold. * The threshold should be set to approximately the time it takes to get a result from the cache tier. @@ -30,7 +34,7 @@ public class TookTimePolicy implements Predicate { /** * The minimum took time to allow a query. Set to TimeValue.ZERO to let all data through. */ - private final TimeValue threshold; + private TimeValue threshold; /** * Function which extracts the relevant PolicyValues from a serialized CachedQueryResult @@ -41,13 +45,25 @@ public class TookTimePolicy implements Predicate { * Constructs a took time policy. * @param threshold the threshold * @param cachedResultParser the function providing policy values + * @param clusterSettings cluster settings + * @param cacheType cache type */ - public TookTimePolicy(TimeValue threshold, Function cachedResultParser) { + public TookTimePolicy( + TimeValue threshold, + Function cachedResultParser, + ClusterSettings clusterSettings, + CacheType cacheType + ) { if (threshold.compareTo(TimeValue.ZERO) < 0) { throw new IllegalArgumentException("Threshold for TookTimePolicy must be >= 0ms but was " + threshold.getStringRep()); } this.threshold = threshold; this.cachedResultParser = cachedResultParser; + clusterSettings.addSettingsUpdateConsumer(TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP.get(cacheType), this::setThreshold); + } + + private void setThreshold(TimeValue threshold) { + this.threshold = threshold; } /** 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 00a8eec93acc9..ae3d9f1dbcf62 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 @@ -12,22 +12,26 @@ import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.cache.CacheType; import org.opensearch.common.cache.ICache; +import org.opensearch.common.cache.ICacheKey; import org.opensearch.common.cache.LoadAwareCacheLoader; import org.opensearch.common.cache.RemovalListener; import org.opensearch.common.cache.RemovalNotification; +import org.opensearch.common.cache.RemovalReason; import org.opensearch.common.cache.policy.CachedQueryResult; +import org.opensearch.common.cache.stats.ImmutableCacheStatsHolder; import org.opensearch.common.cache.store.config.CacheConfig; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.ReleasableLock; -import org.opensearch.common.util.iterable.Iterables; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.NoSuchElementException; import java.util.Objects; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -47,9 +51,16 @@ @ExperimentalApi public class TieredSpilloverCache implements ICache { + // Used to avoid caching stale entries in lower tiers. + private static final List SPILLOVER_REMOVAL_REASONS = List.of(RemovalReason.EVICTED, RemovalReason.CAPACITY); + private final ICache diskCache; private final ICache onHeapCache; - private final RemovalListener removalListener; + + // The listener for removals from the spillover cache as a whole + // TODO: In TSC stats PR, each tier will have its own separate removal listener. + private final RemovalListener, V> removalListener; + private final List dimensionNames; ReadWriteLock readWriteLock = new ReentrantReadWriteLock(); ReleasableLock readLock = new ReleasableLock(readWriteLock.readLock()); ReleasableLock writeLock = new ReleasableLock(readWriteLock.writeLock()); @@ -65,12 +76,15 @@ public class TieredSpilloverCache implements ICache { this.removalListener = Objects.requireNonNull(builder.removalListener, "Removal listener can't be null"); this.onHeapCache = builder.onHeapCacheFactory.create( - new CacheConfig.Builder().setRemovalListener(new RemovalListener() { + new CacheConfig.Builder().setRemovalListener(new RemovalListener, V>() { @Override - public void onRemoval(RemovalNotification notification) { + public void onRemoval(RemovalNotification, V> notification) { try (ReleasableLock ignore = writeLock.acquire()) { - if (evaluatePolicies(notification.getValue())) { + if (SPILLOVER_REMOVAL_REASONS.contains(notification.getRemovalReason()) + && evaluatePolicies(notification.getValue())) { diskCache.put(notification.getKey(), notification.getValue()); + } else { + removalListener.onRemoval(notification); } } } @@ -79,8 +93,10 @@ public void onRemoval(RemovalNotification notification) { .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()) .build(), builder.cacheType, builder.cacheFactories @@ -88,7 +104,7 @@ public void onRemoval(RemovalNotification notification) { ); this.diskCache = builder.diskCacheFactory.create(builder.cacheConfig, builder.cacheType, builder.cacheFactories); this.cacheList = Arrays.asList(onHeapCache, diskCache); - + this.dimensionNames = builder.cacheConfig.getDimensionNames(); this.policies = builder.policies; // Will never be null; builder initializes it to an empty list } @@ -103,19 +119,19 @@ ICache getDiskCache() { } @Override - public V get(K key) { + public V get(ICacheKey key) { return getValueFromTieredCache().apply(key); } @Override - public void put(K key, V value) { + public void put(ICacheKey key, V value) { try (ReleasableLock ignore = writeLock.acquire()) { onHeapCache.put(key, value); } } @Override - public V computeIfAbsent(K key, LoadAwareCacheLoader loader) throws Exception { + public V computeIfAbsent(ICacheKey key, LoadAwareCacheLoader, V> loader) throws Exception { V cacheValue = getValueFromTieredCache().apply(key); if (cacheValue == null) { @@ -132,7 +148,7 @@ public V computeIfAbsent(K key, LoadAwareCacheLoader loader) throws Except } @Override - public void invalidate(K key) { + 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. @@ -156,10 +172,11 @@ public void invalidateAll() { * Provides an iteration over both onHeap and disk keys. This is not protected from any mutations to the cache. * @return An iterable over (onHeap + disk) keys */ - @SuppressWarnings("unchecked") + @SuppressWarnings({ "unchecked" }) @Override - public Iterable keys() { - return Iterables.concat(onHeapCache.keys(), diskCache.keys()); + public Iterable> keys() { + Iterable>[] iterables = (Iterable>[]) new Iterable[] { onHeapCache.keys(), diskCache.keys() }; + return new ConcatenatedIterables>(iterables); } @Override @@ -187,7 +204,12 @@ public void close() throws IOException { } } - private Function getValueFromTieredCache() { + @Override + public ImmutableCacheStatsHolder stats() { + return null; // TODO: in TSC stats PR + } + + private Function, V> getValueFromTieredCache() { return key -> { try (ReleasableLock ignore = readLock.acquire()) { for (ICache cache : cacheList) { @@ -213,6 +235,67 @@ boolean evaluatePolicies(V value) { return true; } + /** + * ConcatenatedIterables which combines cache iterables and supports remove() functionality as well if underlying + * iterator supports it. + * @param Type of key. + */ + static class ConcatenatedIterables implements Iterable { + + final Iterable[] iterables; + + ConcatenatedIterables(Iterable[] iterables) { + this.iterables = iterables; + } + + @SuppressWarnings({ "unchecked" }) + @Override + public Iterator iterator() { + Iterator[] iterators = (Iterator[]) new Iterator[iterables.length]; + for (int i = 0; i < iterables.length; i++) { + iterators[i] = iterables[i].iterator(); + } + return new ConcatenatedIterator<>(iterators); + } + + static class ConcatenatedIterator implements Iterator { + private final Iterator[] iterators; + private int currentIteratorIndex; + private Iterator currentIterator; + + public ConcatenatedIterator(Iterator[] iterators) { + this.iterators = iterators; + this.currentIteratorIndex = 0; + this.currentIterator = iterators[currentIteratorIndex]; + } + + @Override + public boolean hasNext() { + while (!currentIterator.hasNext()) { + currentIteratorIndex++; + if (currentIteratorIndex == iterators.length) { + return false; + } + currentIterator = iterators[currentIteratorIndex]; + } + return true; + } + + @Override + public T next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return currentIterator.next(); + } + + @Override + public void remove() { + currentIterator.remove(); + } + } + } + /** * Factory to create TieredSpilloverCache objects. */ @@ -253,8 +336,7 @@ public ICache create(CacheConfig config, CacheType cacheType, } ICache.Factory diskCacheFactory = cacheFactories.get(diskCacheStoreName); - TimeValue diskPolicyThreshold = TieredSpilloverCacheSettings.TIERED_SPILLOVER_DISK_TOOK_TIME_THRESHOLD - .getConcreteSettingForNamespace(cacheType.getSettingPrefix()) + TimeValue diskPolicyThreshold = TieredSpilloverCacheSettings.TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP.get(cacheType) .get(settings); Function cachedResultParser = Objects.requireNonNull( config.getCachedResultParser(), @@ -266,7 +348,7 @@ public ICache create(CacheConfig config, CacheType cacheType, .setRemovalListener(config.getRemovalListener()) .setCacheConfig(config) .setCacheType(cacheType) - .addPolicy(new TookTimePolicy(diskPolicyThreshold, cachedResultParser)) + .addPolicy(new TookTimePolicy(diskPolicyThreshold, cachedResultParser, config.getClusterSettings(), cacheType)) .build(); } @@ -284,7 +366,7 @@ public String getCacheName() { public static class Builder { private ICache.Factory onHeapCacheFactory; private ICache.Factory diskCacheFactory; - private RemovalListener removalListener; + private RemovalListener, V> removalListener; private CacheConfig cacheConfig; private CacheType cacheType; private Map cacheFactories; @@ -320,7 +402,7 @@ public Builder setDiskCacheFactory(ICache.Factory diskCacheFactory) { * @param removalListener Removal listener * @return builder */ - public Builder setRemovalListener(RemovalListener removalListener) { + public Builder setRemovalListener(RemovalListener, V> removalListener) { this.removalListener = removalListener; return this; } 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 0cc8a711faaf5..dfd40199d859e 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 @@ -18,6 +18,8 @@ import java.util.List; import java.util.Map; +import static org.opensearch.cache.common.tier.TieredSpilloverCacheSettings.TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP; + /** * Plugin for TieredSpilloverCache. */ @@ -51,11 +53,7 @@ public List> getSettings() { settingList.add( TieredSpilloverCacheSettings.TIERED_SPILLOVER_DISK_STORE_NAME.getConcreteSettingForNamespace(cacheType.getSettingPrefix()) ); - settingList.add( - TieredSpilloverCacheSettings.TIERED_SPILLOVER_DISK_TOOK_TIME_THRESHOLD.getConcreteSettingForNamespace( - cacheType.getSettingPrefix() - ) - ); + settingList.add(TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP.get(cacheType)); } 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 684307960b8a5..b89e8c517a351 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 @@ -8,9 +8,12 @@ package org.opensearch.cache.common.tier; +import org.opensearch.common.cache.CacheType; import org.opensearch.common.settings.Setting; import org.opensearch.common.unit.TimeValue; +import java.util.HashMap; +import java.util.Map; import java.util.concurrent.TimeUnit; import static org.opensearch.common.settings.Setting.Property.NodeScope; @@ -42,17 +45,36 @@ public class TieredSpilloverCacheSettings { /** * Setting defining the minimum took time for a query to be allowed into the disk cache. */ - public static final Setting.AffixSetting TIERED_SPILLOVER_DISK_TOOK_TIME_THRESHOLD = Setting.suffixKeySetting( + private static final Setting.AffixSetting TIERED_SPILLOVER_DISK_TOOK_TIME_THRESHOLD = Setting.suffixKeySetting( TieredSpilloverCache.TieredSpilloverCacheFactory.TIERED_SPILLOVER_CACHE_NAME + ".disk.store.policies.took_time.threshold", (key) -> Setting.timeSetting( key, new TimeValue(10, TimeUnit.MILLISECONDS), // Default value for this setting TimeValue.ZERO, // Minimum value for this setting - NodeScope + NodeScope, + Setting.Property.Dynamic ) ); - // 10 ms was chosen as a safe value based on proof of concept, where we saw disk latencies in this range. - // Will be tuned further with future benchmarks. + + /** + * Stores took time policy settings for various cache types as these are dynamic so that can be registered and + * retrieved accordingly. + */ + public static final Map> TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP; + + /** + * Fetches concrete took time policy settings. + */ + static { + Map> concreteTookTimePolicySettingMap = new HashMap<>(); + for (CacheType cacheType : CacheType.values()) { + concreteTookTimePolicySettingMap.put( + cacheType, + TIERED_SPILLOVER_DISK_TOOK_TIME_THRESHOLD.getConcreteSettingForNamespace(cacheType.getSettingPrefix()) + ); + } + TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP = concreteTookTimePolicySettingMap; + } /** * Default constructor diff --git a/modules/cache-common/src/test/java/org/opensearch/cache/common/policy/TookTimePolicyTests.java b/modules/cache-common/src/test/java/org/opensearch/cache/common/policy/TookTimePolicyTests.java index 237c9c7b79db4..000067280e50d 100644 --- a/modules/cache-common/src/test/java/org/opensearch/cache/common/policy/TookTimePolicyTests.java +++ b/modules/cache-common/src/test/java/org/opensearch/cache/common/policy/TookTimePolicyTests.java @@ -12,20 +12,27 @@ import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TotalHits; import org.opensearch.common.Randomness; +import org.opensearch.common.cache.CacheType; import org.opensearch.common.cache.policy.CachedQueryResult; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.lucene.search.TopDocsAndMaxScore; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.search.DocValueFormat; import org.opensearch.search.query.QuerySearchResult; import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; import java.io.IOException; +import java.util.HashSet; import java.util.Random; import java.util.function.Function; +import static org.opensearch.cache.common.tier.TieredSpilloverCacheSettings.TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP; + public class TookTimePolicyTests extends OpenSearchTestCase { private final Function transformationFunction = (data) -> { try { @@ -35,8 +42,17 @@ public class TookTimePolicyTests extends OpenSearchTestCase { } }; + private ClusterSettings clusterSettings; + + @Before + public void setup() { + Settings settings = Settings.EMPTY; + clusterSettings = new ClusterSettings(settings, new HashSet<>()); + clusterSettings.registerSetting(TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP.get(CacheType.INDICES_REQUEST_CACHE)); + } + private TookTimePolicy getTookTimePolicy(TimeValue threshold) { - return new TookTimePolicy<>(threshold, transformationFunction); + return new TookTimePolicy<>(threshold, transformationFunction, clusterSettings, CacheType.INDICES_REQUEST_CACHE); } public void testTookTimePolicy() throws Exception { 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 d8a6eb480a5a5..0d98503af635f 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 @@ -10,40 +10,44 @@ import org.opensearch.common.cache.CacheType; import org.opensearch.common.cache.ICache; +import org.opensearch.common.cache.ICacheKey; import org.opensearch.common.cache.LoadAwareCacheLoader; import org.opensearch.common.cache.RemovalListener; import org.opensearch.common.cache.RemovalNotification; import org.opensearch.common.cache.RemovalReason; import org.opensearch.common.cache.serializer.Serializer; +import org.opensearch.common.cache.stats.ImmutableCacheStatsHolder; import org.opensearch.common.cache.store.builders.ICacheBuilder; import org.opensearch.common.cache.store.config.CacheConfig; +import java.util.Iterator; import java.util.Map; +import java.util.NoSuchElementException; import java.util.concurrent.ConcurrentHashMap; public class MockDiskCache implements ICache { - Map cache; + Map, V> cache; int maxSize; long delay; - private final RemovalListener removalListener; + private final RemovalListener, V> removalListener; - public MockDiskCache(int maxSize, long delay, RemovalListener removalListener) { + public MockDiskCache(int maxSize, long delay, RemovalListener, V> removalListener) { this.maxSize = maxSize; this.delay = delay; this.removalListener = removalListener; - this.cache = new ConcurrentHashMap(); + this.cache = new ConcurrentHashMap, V>(); } @Override - public V get(K key) { + public V get(ICacheKey key) { V value = cache.get(key); return value; } @Override - public void put(K key, V value) { + public void put(ICacheKey key, V value) { if (this.cache.size() >= maxSize) { // For simplification this.removalListener.onRemoval(new RemovalNotification<>(key, value, RemovalReason.EVICTED)); } @@ -56,7 +60,7 @@ public void put(K key, V value) { } @Override - public V computeIfAbsent(K key, LoadAwareCacheLoader loader) { + public V computeIfAbsent(ICacheKey key, LoadAwareCacheLoader, V> loader) { V value = cache.computeIfAbsent(key, key1 -> { try { return loader.load(key); @@ -68,7 +72,7 @@ public V computeIfAbsent(K key, LoadAwareCacheLoader loader) { } @Override - public void invalidate(K key) { + public void invalidate(ICacheKey key) { this.cache.remove(key); } @@ -78,8 +82,8 @@ public void invalidateAll() { } @Override - public Iterable keys() { - return this.cache.keySet(); + public Iterable> keys() { + return () -> new CacheKeyIterator<>(cache, removalListener); } @Override @@ -90,6 +94,11 @@ public long count() { @Override public void refresh() {} + @Override + public ImmutableCacheStatsHolder stats() { + return null; + } + @Override public void close() { @@ -156,4 +165,48 @@ public Builder setValueSerializer(Serializer valueSerializer) { } } + + /** + * Provides a iterator over keys. + * @param Type of key + * @param Type of value + */ + static class CacheKeyIterator implements Iterator { + private final Iterator> entryIterator; + private final Map cache; + private final RemovalListener removalListener; + private K currentKey; + + public CacheKeyIterator(Map cache, RemovalListener removalListener) { + this.entryIterator = cache.entrySet().iterator(); + this.removalListener = removalListener; + this.cache = cache; + } + + @Override + public boolean hasNext() { + return entryIterator.hasNext(); + } + + @Override + public K next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + Map.Entry entry = entryIterator.next(); + currentKey = entry.getKey(); + return currentKey; + } + + @Override + public void remove() { + if (currentKey == null) { + throw new IllegalStateException("No element to remove"); + } + V value = cache.get(currentKey); + cache.remove(currentKey); + this.removalListener.onRemoval(new RemovalNotification<>(currentKey, value, RemovalReason.INVALIDATED)); + currentKey = null; + } + } } 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 b132952834f06..bf9f8fd22d793 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 @@ -10,6 +10,7 @@ import org.opensearch.common.cache.CacheType; import org.opensearch.common.cache.ICache; +import org.opensearch.common.cache.ICacheKey; import org.opensearch.common.cache.LoadAwareCacheLoader; import org.opensearch.common.cache.RemovalListener; import org.opensearch.common.cache.RemovalNotification; @@ -19,14 +20,17 @@ import org.opensearch.common.cache.store.config.CacheConfig; import org.opensearch.common.cache.store.settings.OpenSearchOnHeapCacheSettings; import org.opensearch.common.metrics.CounterMetric; +import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.FeatureFlags; import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; import java.util.ArrayList; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.UUID; @@ -38,16 +42,28 @@ import java.util.function.Function; import java.util.function.Predicate; +import static org.opensearch.cache.common.tier.TieredSpilloverCacheSettings.TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP; import static org.opensearch.common.cache.store.settings.OpenSearchOnHeapCacheSettings.MAXIMUM_SIZE_IN_BYTES_KEY; public class TieredSpilloverCacheTests extends OpenSearchTestCase { + // TODO: TSC stats impl is in a future PR. Parts of tests which use stats values are missing for now. + static final List dimensionNames = List.of("dim1", "dim2", "dim3"); + + private ClusterSettings clusterSettings; + + @Before + public void setup() { + Settings settings = Settings.EMPTY; + clusterSettings = new ClusterSettings(settings, new HashSet<>()); + clusterSettings.registerSetting(TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP.get(CacheType.INDICES_REQUEST_CACHE)); + } public void testComputeIfAbsentWithoutAnyOnHeapCacheEviction() throws Exception { int onHeapCacheSize = randomIntBetween(10, 30); int keyValueSize = 50; MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); - TieredSpilloverCache tieredSpilloverCache = intializeTieredSpilloverCache( + TieredSpilloverCache tieredSpilloverCache = initializeTieredSpilloverCache( keyValueSize, randomIntBetween(1, 4), removalListener, @@ -62,12 +78,12 @@ public void testComputeIfAbsentWithoutAnyOnHeapCacheEviction() throws Exception 0 ); int numOfItems1 = randomIntBetween(1, onHeapCacheSize / 2 - 1); - List keys = new ArrayList<>(); + List> keys = new ArrayList<>(); // Put values in cache. for (int iter = 0; iter < numOfItems1; iter++) { - String key = UUID.randomUUID().toString(); + ICacheKey key = getICacheKey(UUID.randomUUID().toString()); keys.add(key); - LoadAwareCacheLoader tieredCacheLoader = getLoadAwareCacheLoader(); + LoadAwareCacheLoader, String> tieredCacheLoader = getLoadAwareCacheLoader(); tieredSpilloverCache.computeIfAbsent(key, tieredCacheLoader); } assertEquals(0, removalListener.evictionsMetric.count()); @@ -84,7 +100,7 @@ public void testComputeIfAbsentWithoutAnyOnHeapCacheEviction() throws Exception tieredSpilloverCache.computeIfAbsent(keys.get(index), getLoadAwareCacheLoader()); } else { // Hit cache with randomized key which is expected to miss cache always. - tieredSpilloverCache.computeIfAbsent(UUID.randomUUID().toString(), getLoadAwareCacheLoader()); + tieredSpilloverCache.computeIfAbsent(getICacheKey(UUID.randomUUID().toString()), getLoadAwareCacheLoader()); cacheMiss++; } } @@ -132,8 +148,10 @@ public void testComputeIfAbsentWithFactoryBasedCacheCreation() throws Exception .setWeigher((k, v) -> keyValueSize) .setRemovalListener(removalListener) .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) .build(), CacheType.INDICES_REQUEST_CACHE, Map.of( @@ -147,11 +165,15 @@ public void testComputeIfAbsentWithFactoryBasedCacheCreation() throws Exception TieredSpilloverCache tieredSpilloverCache = (TieredSpilloverCache) tieredSpilloverICache; int numOfItems1 = randomIntBetween(onHeapCacheSize + 1, totalSize); + List> onHeapKeys = new ArrayList<>(); + List> diskTierKeys = new ArrayList<>(); for (int iter = 0; iter < numOfItems1; iter++) { String key = UUID.randomUUID().toString(); - LoadAwareCacheLoader tieredCacheLoader = getLoadAwareCacheLoader(); - tieredSpilloverCache.computeIfAbsent(key, tieredCacheLoader); + LoadAwareCacheLoader, String> tieredCacheLoader = getLoadAwareCacheLoader(); + tieredSpilloverCache.computeIfAbsent(getICacheKey(key), tieredCacheLoader); } + tieredSpilloverCache.getOnHeapCache().keys().forEach(onHeapKeys::add); + tieredSpilloverCache.getDiskCache().keys().forEach(diskTierKeys::add); // Verify on heap cache size. assertEquals(onHeapCacheSize, tieredSpilloverCache.getOnHeapCache().count()); // Verify disk cache size. @@ -264,6 +286,7 @@ public void testComputeIfAbsentWithEvictionsFromOnHeapCache() throws Exception { .setKeyType(String.class) .setWeigher((k, v) -> keyValueSize) .setRemovalListener(removalListener) + .setDimensionNames(dimensionNames) .setSettings( Settings.builder() .put( @@ -293,20 +316,17 @@ public void testComputeIfAbsentWithEvictionsFromOnHeapCache() throws Exception { // 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<>(); + List> onHeapKeys = new ArrayList<>(); + List> diskTierKeys = new ArrayList<>(); for (int iter = 0; iter < numOfItems1; iter++) { - String key = UUID.randomUUID().toString(); - LoadAwareCacheLoader tieredCacheLoader = getLoadAwareCacheLoader(); + ICacheKey key = getICacheKey(UUID.randomUUID().toString()); + LoadAwareCacheLoader, String> tieredCacheLoader = getLoadAwareCacheLoader(); tieredSpilloverCache.computeIfAbsent(key, tieredCacheLoader); } tieredSpilloverCache.getOnHeapCache().keys().forEach(onHeapKeys::add); tieredSpilloverCache.getDiskCache().keys().forEach(diskTierKeys::add); - assertEquals(tieredSpilloverCache.getOnHeapCache().count(), onHeapKeys.size()); - assertEquals(tieredSpilloverCache.getDiskCache().count(), diskTierKeys.size()); - // Try to hit cache again with some randomization. int numOfItems2 = randomIntBetween(50, 200); int onHeapCacheHit = 0; @@ -316,21 +336,21 @@ public void testComputeIfAbsentWithEvictionsFromOnHeapCache() throws Exception { if (randomBoolean()) { // Hit cache with key stored in onHeap cache. onHeapCacheHit++; int index = randomIntBetween(0, onHeapKeys.size() - 1); - LoadAwareCacheLoader loadAwareCacheLoader = getLoadAwareCacheLoader(); + LoadAwareCacheLoader, String> loadAwareCacheLoader = getLoadAwareCacheLoader(); tieredSpilloverCache.computeIfAbsent(onHeapKeys.get(index), loadAwareCacheLoader); assertFalse(loadAwareCacheLoader.isLoaded()); } else { // Hit cache with key stored in disk cache. diskCacheHit++; int index = randomIntBetween(0, diskTierKeys.size() - 1); - LoadAwareCacheLoader loadAwareCacheLoader = getLoadAwareCacheLoader(); + LoadAwareCacheLoader, String> loadAwareCacheLoader = getLoadAwareCacheLoader(); tieredSpilloverCache.computeIfAbsent(diskTierKeys.get(index), loadAwareCacheLoader); assertFalse(loadAwareCacheLoader.isLoaded()); } } for (int iter = 0; iter < randomIntBetween(50, 200); iter++) { // Hit cache with randomized key which is expected to miss cache always. - LoadAwareCacheLoader tieredCacheLoader = getLoadAwareCacheLoader(); - tieredSpilloverCache.computeIfAbsent(UUID.randomUUID().toString(), tieredCacheLoader); + LoadAwareCacheLoader, String> tieredCacheLoader = getLoadAwareCacheLoader(); + tieredSpilloverCache.computeIfAbsent(getICacheKey(UUID.randomUUID().toString()), tieredCacheLoader); cacheMiss++; } } @@ -342,7 +362,7 @@ public void testComputeIfAbsentWithEvictionsFromTieredCache() throws Exception { int keyValueSize = 50; MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); - TieredSpilloverCache tieredSpilloverCache = intializeTieredSpilloverCache( + TieredSpilloverCache tieredSpilloverCache = initializeTieredSpilloverCache( keyValueSize, diskCacheSize, removalListener, @@ -358,9 +378,10 @@ public void testComputeIfAbsentWithEvictionsFromTieredCache() throws Exception { ); int numOfItems = randomIntBetween(totalSize + 1, totalSize * 3); for (int iter = 0; iter < numOfItems; iter++) { - LoadAwareCacheLoader tieredCacheLoader = getLoadAwareCacheLoader(); - tieredSpilloverCache.computeIfAbsent(UUID.randomUUID().toString(), tieredCacheLoader); + LoadAwareCacheLoader, String> tieredCacheLoader = getLoadAwareCacheLoader(); + tieredSpilloverCache.computeIfAbsent(getICacheKey(UUID.randomUUID().toString()), tieredCacheLoader); } + int evictions = numOfItems - (totalSize); assertEquals(evictions, removalListener.evictionsMetric.count()); } @@ -372,7 +393,7 @@ public void testGetAndCount() throws Exception { int totalSize = onHeapCacheSize + diskCacheSize; MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); - TieredSpilloverCache tieredSpilloverCache = intializeTieredSpilloverCache( + TieredSpilloverCache tieredSpilloverCache = initializeTieredSpilloverCache( keyValueSize, diskCacheSize, removalListener, @@ -388,17 +409,17 @@ public void testGetAndCount() throws Exception { ); int numOfItems1 = randomIntBetween(onHeapCacheSize + 1, totalSize); - List onHeapKeys = new ArrayList<>(); - List diskTierKeys = new ArrayList<>(); + List> onHeapKeys = new ArrayList<>(); + List> diskTierKeys = new ArrayList<>(); for (int iter = 0; iter < numOfItems1; iter++) { - String key = UUID.randomUUID().toString(); + 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 loadAwareCacheLoader = getLoadAwareCacheLoader(); + LoadAwareCacheLoader, String> loadAwareCacheLoader = getLoadAwareCacheLoader(); tieredSpilloverCache.computeIfAbsent(key, loadAwareCacheLoader); } @@ -412,7 +433,7 @@ public void testGetAndCount() throws Exception { assertNotNull(tieredSpilloverCache.get(diskTierKeys.get(index))); } } else { - assertNull(tieredSpilloverCache.get(UUID.randomUUID().toString())); + assertNull(tieredSpilloverCache.get(getICacheKey(UUID.randomUUID().toString()))); } } assertEquals(numOfItems1, tieredSpilloverCache.count()); @@ -424,7 +445,7 @@ public void testPut() { int keyValueSize = 50; MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); - TieredSpilloverCache tieredSpilloverCache = intializeTieredSpilloverCache( + TieredSpilloverCache tieredSpilloverCache = initializeTieredSpilloverCache( keyValueSize, diskCacheSize, removalListener, @@ -438,10 +459,9 @@ public void testPut() { .build(), 0 ); - String key = UUID.randomUUID().toString(); + ICacheKey key = getICacheKey(UUID.randomUUID().toString()); String value = UUID.randomUUID().toString(); tieredSpilloverCache.put(key, value); - assertEquals(1, tieredSpilloverCache.count()); } public void testPutAndVerifyNewItemsArePresentOnHeapCache() throws Exception { @@ -451,7 +471,7 @@ public void testPutAndVerifyNewItemsArePresentOnHeapCache() throws Exception { MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); - TieredSpilloverCache tieredSpilloverCache = intializeTieredSpilloverCache( + TieredSpilloverCache tieredSpilloverCache = initializeTieredSpilloverCache( keyValueSize, diskCacheSize, removalListener, @@ -471,52 +491,27 @@ public void testPutAndVerifyNewItemsArePresentOnHeapCache() throws Exception { ); for (int i = 0; i < onHeapCacheSize; i++) { - tieredSpilloverCache.computeIfAbsent(UUID.randomUUID().toString(), new LoadAwareCacheLoader<>() { - @Override - public boolean isLoaded() { - return false; - } - - @Override - public String load(String key) { - return UUID.randomUUID().toString(); - } - }); + tieredSpilloverCache.computeIfAbsent(getICacheKey(UUID.randomUUID().toString()), getLoadAwareCacheLoader()); } - assertEquals(onHeapCacheSize, tieredSpilloverCache.getOnHeapCache().count()); - assertEquals(0, tieredSpilloverCache.getDiskCache().count()); - // Again try to put OnHeap cache capacity amount of new items. - List newKeyList = new ArrayList<>(); + List> newKeyList = new ArrayList<>(); for (int i = 0; i < onHeapCacheSize; i++) { - newKeyList.add(UUID.randomUUID().toString()); + newKeyList.add(getICacheKey(UUID.randomUUID().toString())); } for (int i = 0; i < newKeyList.size(); i++) { - tieredSpilloverCache.computeIfAbsent(newKeyList.get(i), new LoadAwareCacheLoader<>() { - @Override - public boolean isLoaded() { - return false; - } - - @Override - public String load(String key) { - return UUID.randomUUID().toString(); - } - }); + tieredSpilloverCache.computeIfAbsent(newKeyList.get(i), getLoadAwareCacheLoader()); } // Verify that new items are part of onHeap cache. - List actualOnHeapCacheKeys = new ArrayList<>(); + List> actualOnHeapCacheKeys = new ArrayList<>(); tieredSpilloverCache.getOnHeapCache().keys().forEach(actualOnHeapCacheKeys::add); assertEquals(newKeyList.size(), actualOnHeapCacheKeys.size()); for (int i = 0; i < actualOnHeapCacheKeys.size(); i++) { assertTrue(newKeyList.contains(actualOnHeapCacheKeys.get(i))); } - assertEquals(onHeapCacheSize, tieredSpilloverCache.getOnHeapCache().count()); - assertEquals(onHeapCacheSize, tieredSpilloverCache.getDiskCache().count()); } public void testInvalidate() { @@ -525,7 +520,7 @@ public void testInvalidate() { int keyValueSize = 20; MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); - TieredSpilloverCache tieredSpilloverCache = intializeTieredSpilloverCache( + TieredSpilloverCache tieredSpilloverCache = initializeTieredSpilloverCache( keyValueSize, diskCacheSize, removalListener, @@ -539,24 +534,29 @@ public void testInvalidate() { .build(), 0 ); - String key = UUID.randomUUID().toString(); + ICacheKey key = getICacheKey(UUID.randomUUID().toString()); String value = UUID.randomUUID().toString(); // First try to invalidate without the key present in cache. tieredSpilloverCache.invalidate(key); + // assertEquals(0, tieredSpilloverCache.stats().getEvictionsByDimensions(HEAP_DIMS)); // Now try to invalidate with the key present in onHeap cache. tieredSpilloverCache.put(key, value); tieredSpilloverCache.invalidate(key); + // Evictions metric shouldn't increase for invalidations. assertEquals(0, tieredSpilloverCache.count()); tieredSpilloverCache.put(key, value); // Put another key/value so that one of the item is evicted to disk cache. - String key2 = UUID.randomUUID().toString(); + ICacheKey key2 = getICacheKey(UUID.randomUUID().toString()); tieredSpilloverCache.put(key2, UUID.randomUUID().toString()); + assertEquals(2, tieredSpilloverCache.count()); - // Again invalidate older key + + // Again invalidate older key, leaving one in heap tier and zero in disk tier tieredSpilloverCache.invalidate(key); assertEquals(1, tieredSpilloverCache.count()); + } public void testCacheKeys() throws Exception { @@ -565,7 +565,7 @@ public void testCacheKeys() throws Exception { int keyValueSize = 50; MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); - TieredSpilloverCache tieredSpilloverCache = intializeTieredSpilloverCache( + TieredSpilloverCache tieredSpilloverCache = initializeTieredSpilloverCache( keyValueSize, diskCacheSize, removalListener, @@ -579,46 +579,46 @@ public void testCacheKeys() throws Exception { .build(), 0 ); - List onHeapKeys = new ArrayList<>(); - List diskTierKeys = new ArrayList<>(); + List> onHeapKeys = new ArrayList<>(); + List> diskTierKeys = new ArrayList<>(); // During first round add onHeapCacheSize entries. Will go to onHeap cache initially. for (int i = 0; i < onHeapCacheSize; i++) { - String key = UUID.randomUUID().toString(); + ICacheKey key = getICacheKey(UUID.randomUUID().toString()); diskTierKeys.add(key); tieredSpilloverCache.computeIfAbsent(key, getLoadAwareCacheLoader()); } // In another round, add another onHeapCacheSize entries. These will go to onHeap and above ones will be // evicted to onDisk cache. for (int i = 0; i < onHeapCacheSize; i++) { - String key = UUID.randomUUID().toString(); + ICacheKey key = getICacheKey(UUID.randomUUID().toString()); onHeapKeys.add(key); tieredSpilloverCache.computeIfAbsent(key, getLoadAwareCacheLoader()); } - List actualOnHeapKeys = new ArrayList<>(); - List actualOnDiskKeys = new ArrayList<>(); - Iterable onHeapiterable = tieredSpilloverCache.getOnHeapCache().keys(); - Iterable onDiskiterable = tieredSpilloverCache.getDiskCache().keys(); + List> actualOnHeapKeys = new ArrayList<>(); + List> actualOnDiskKeys = new ArrayList<>(); + Iterable> onHeapiterable = tieredSpilloverCache.getOnHeapCache().keys(); + Iterable> onDiskiterable = tieredSpilloverCache.getDiskCache().keys(); onHeapiterable.iterator().forEachRemaining(actualOnHeapKeys::add); onDiskiterable.iterator().forEachRemaining(actualOnDiskKeys::add); - for (String onHeapKey : onHeapKeys) { + for (ICacheKey onHeapKey : onHeapKeys) { assertTrue(actualOnHeapKeys.contains(onHeapKey)); } - for (String onDiskKey : actualOnDiskKeys) { + for (ICacheKey onDiskKey : actualOnDiskKeys) { assertTrue(actualOnDiskKeys.contains(onDiskKey)); } // Testing keys() which returns all keys. - List actualMergedKeys = new ArrayList<>(); - List expectedMergedKeys = new ArrayList<>(); + List> actualMergedKeys = new ArrayList<>(); + List> expectedMergedKeys = new ArrayList<>(); expectedMergedKeys.addAll(onHeapKeys); expectedMergedKeys.addAll(diskTierKeys); - Iterable mergedIterable = tieredSpilloverCache.keys(); + Iterable> mergedIterable = tieredSpilloverCache.keys(); mergedIterable.iterator().forEachRemaining(actualMergedKeys::add); assertEquals(expectedMergedKeys.size(), actualMergedKeys.size()); - for (String key : expectedMergedKeys) { + for (ICacheKey key : expectedMergedKeys) { assertTrue(actualMergedKeys.contains(key)); } } @@ -627,7 +627,7 @@ public void testRefresh() { int diskCacheSize = randomIntBetween(60, 100); MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); - TieredSpilloverCache tieredSpilloverCache = intializeTieredSpilloverCache( + TieredSpilloverCache tieredSpilloverCache = initializeTieredSpilloverCache( 50, diskCacheSize, removalListener, @@ -644,7 +644,7 @@ public void testInvalidateAll() throws Exception { int totalSize = onHeapCacheSize + diskCacheSize; MockCacheRemovalListener removalListener = new MockCacheRemovalListener<>(); - TieredSpilloverCache tieredSpilloverCache = intializeTieredSpilloverCache( + TieredSpilloverCache tieredSpilloverCache = initializeTieredSpilloverCache( keyValueSize, diskCacheSize, removalListener, @@ -660,17 +660,17 @@ public void testInvalidateAll() throws Exception { ); // 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<>(); + List> onHeapKeys = new ArrayList<>(); + List> diskTierKeys = new ArrayList<>(); for (int iter = 0; iter < numOfItems1; iter++) { - String key = UUID.randomUUID().toString(); + 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 tieredCacheLoader = getLoadAwareCacheLoader(); + LoadAwareCacheLoader, String> tieredCacheLoader = getLoadAwareCacheLoader(); tieredSpilloverCache.computeIfAbsent(key, tieredCacheLoader); } assertEquals(numOfItems1, tieredSpilloverCache.count()); @@ -693,7 +693,7 @@ public void testComputeIfAbsentConcurrently() throws Exception { ) .build(); - TieredSpilloverCache tieredSpilloverCache = intializeTieredSpilloverCache( + TieredSpilloverCache tieredSpilloverCache = initializeTieredSpilloverCache( keyValueSize, diskCacheSize, removalListener, @@ -702,19 +702,19 @@ public void testComputeIfAbsentConcurrently() throws Exception { ); int numberOfSameKeys = randomIntBetween(10, onHeapCacheSize - 1); - String key = UUID.randomUUID().toString(); + ICacheKey key = getICacheKey(UUID.randomUUID().toString()); String value = UUID.randomUUID().toString(); Thread[] threads = new Thread[numberOfSameKeys]; Phaser phaser = new Phaser(numberOfSameKeys + 1); CountDownLatch countDownLatch = new CountDownLatch(numberOfSameKeys); // To wait for all threads to finish. - List> loadAwareCacheLoaderList = new CopyOnWriteArrayList<>(); + List, String>> loadAwareCacheLoaderList = new CopyOnWriteArrayList<>(); for (int i = 0; i < numberOfSameKeys; i++) { threads[i] = new Thread(() -> { try { - LoadAwareCacheLoader loadAwareCacheLoader = new LoadAwareCacheLoader<>() { + LoadAwareCacheLoader, String> loadAwareCacheLoader = new LoadAwareCacheLoader<>() { boolean isLoaded = false; @Override @@ -723,7 +723,7 @@ public boolean isLoaded() { } @Override - public String load(String key) { + public String load(ICacheKey key) { isLoaded = true; return value; } @@ -743,7 +743,7 @@ public String load(String key) { int numberOfTimesKeyLoaded = 0; assertEquals(numberOfSameKeys, loadAwareCacheLoaderList.size()); for (int i = 0; i < loadAwareCacheLoaderList.size(); i++) { - LoadAwareCacheLoader loader = loadAwareCacheLoaderList.get(i); + LoadAwareCacheLoader, String> loader = loadAwareCacheLoaderList.get(i); if (loader.isLoaded()) { numberOfTimesKeyLoaded++; } @@ -777,6 +777,7 @@ public void testConcurrencyForEvictionFlowFromOnHeapToDiskTier() throws Exceptio ) .build() ) + .setDimensionNames(dimensionNames) .build(); TieredSpilloverCache tieredSpilloverCache = new TieredSpilloverCache.Builder() .setOnHeapCacheFactory(onHeapCacheFactory) @@ -786,26 +787,17 @@ public void testConcurrencyForEvictionFlowFromOnHeapToDiskTier() throws Exceptio .setCacheType(CacheType.INDICES_REQUEST_CACHE) .build(); - String keyToBeEvicted = "key1"; - String secondKey = "key2"; + ICacheKey keyToBeEvicted = getICacheKey("key1"); + ICacheKey secondKey = getICacheKey("key2"); // Put first key on tiered cache. Will go into onHeap cache. - tieredSpilloverCache.computeIfAbsent(keyToBeEvicted, new LoadAwareCacheLoader<>() { - @Override - public boolean isLoaded() { - return false; - } - - @Override - public String load(String key) { - return UUID.randomUUID().toString(); - } - }); + tieredSpilloverCache.computeIfAbsent(keyToBeEvicted, getLoadAwareCacheLoader()); + // assertEquals(1, tieredSpilloverCache.stats().getEntriesByDimensions(HEAP_DIMS)); CountDownLatch countDownLatch = new CountDownLatch(1); CountDownLatch countDownLatch1 = new CountDownLatch(1); // Put second key on tiered cache. Will cause eviction of first key from onHeap cache and should go into // disk cache. - LoadAwareCacheLoader loadAwareCacheLoader = getLoadAwareCacheLoader(); + LoadAwareCacheLoader, String> loadAwareCacheLoader = getLoadAwareCacheLoader(); Thread thread = new Thread(() -> { try { tieredSpilloverCache.computeIfAbsent(secondKey, loadAwareCacheLoader); @@ -816,7 +808,7 @@ public String load(String key) { }); thread.start(); assertBusy(() -> { assertTrue(loadAwareCacheLoader.isLoaded()); }, 100, TimeUnit.MILLISECONDS); // We wait for new key to be loaded - // after which it eviction flow is + // after which it eviction flow is // guaranteed to occur. ICache onDiskCache = tieredSpilloverCache.getDiskCache(); @@ -835,20 +827,12 @@ public String load(String key) { countDownLatch.await(); assertNotNull(actualValue.get()); countDownLatch1.await(); + assertEquals(1, tieredSpilloverCache.getOnHeapCache().count()); assertEquals(1, onDiskCache.count()); assertNotNull(onDiskCache.get(keyToBeEvicted)); } - class MockCacheRemovalListener implements RemovalListener { - final CounterMetric evictionsMetric = new CounterMetric(); - - @Override - public void onRemoval(RemovalNotification notification) { - evictionsMetric.inc(); - } - } - public void testDiskTierPolicies() throws Exception { // For policy function, allow if what it receives starts with "a" and string is even length ArrayList> policies = new ArrayList<>(); @@ -887,26 +871,14 @@ public void testDiskTierPolicies() throws Exception { keyValuePairs.put("key5", ""); expectedOutputs.put("key5", false); - LoadAwareCacheLoader loader = new LoadAwareCacheLoader() { - boolean isLoaded = false; - - @Override - public boolean isLoaded() { - return isLoaded; - } - - @Override - public String load(String key) throws Exception { - isLoaded = true; - return keyValuePairs.get(key); - } - }; + LoadAwareCacheLoader, String> loader = getLoadAwareCacheLoader(keyValuePairs); for (String key : keyValuePairs.keySet()) { + ICacheKey iCacheKey = getICacheKey(key); Boolean expectedOutput = expectedOutputs.get(key); - String value = tieredSpilloverCache.computeIfAbsent(key, loader); + String value = tieredSpilloverCache.computeIfAbsent(iCacheKey, loader); assertEquals(keyValuePairs.get(key), value); - String result = tieredSpilloverCache.get(key); + String result = tieredSpilloverCache.get(iCacheKey); if (expectedOutput) { // Should retrieve from disk tier if it was accepted assertEquals(keyValuePairs.get(key), result); @@ -959,9 +931,7 @@ public void testTookTimePolicyFromFactory() throws Exception { onHeapCacheSize * keyValueSize + "b" ) .put( - TieredSpilloverCacheSettings.TIERED_SPILLOVER_DISK_TOOK_TIME_THRESHOLD.getConcreteSettingForNamespace( - CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() - ).getKey(), + TieredSpilloverCacheSettings.TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP.get(CacheType.INDICES_REQUEST_CACHE).getKey(), new TimeValue(timeValueThresholdNanos / 1_000_000) ) .build(); @@ -973,12 +943,14 @@ public void testTookTimePolicyFromFactory() throws Exception { .setRemovalListener(removalListener) .setSettings(settings) .setMaxSizeInBytes(onHeapCacheSize * keyValueSize) + .setDimensionNames(dimensionNames) .setCachedResultParser(new Function() { @Override public CachedQueryResult.PolicyValues apply(String s) { return new CachedQueryResult.PolicyValues(tookTimeMap.get(s)); } }) + .setClusterSettings(clusterSettings) .build(), CacheType.INDICES_REQUEST_CACHE, Map.of( @@ -993,22 +965,22 @@ public CachedQueryResult.PolicyValues apply(String s) { // First add all our values to the on heap cache for (String key : tookTimeMap.keySet()) { - tieredSpilloverCache.computeIfAbsent(key, getLoadAwareCacheLoader(keyValueMap)); + tieredSpilloverCache.computeIfAbsent(getICacheKey(key), getLoadAwareCacheLoader(keyValueMap)); } 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 for (int i = 0; i < onHeapCacheSize; i++) { - tieredSpilloverCache.computeIfAbsent(UUID.randomUUID().toString(), getLoadAwareCacheLoader(keyValueMap)); + tieredSpilloverCache.computeIfAbsent(getICacheKey(UUID.randomUUID().toString()), getLoadAwareCacheLoader(keyValueMap)); } ICache onHeapCache = tieredSpilloverCache.getOnHeapCache(); for (String key : tookTimeMap.keySet()) { - assertNull(onHeapCache.get(key)); + assertNull(onHeapCache.get(getICacheKey(key))); } // Now the original keys should be in the disk tier if the policy allows them, or misses if not for (String key : tookTimeMap.keySet()) { - String computedValue = tieredSpilloverCache.get(key); + String computedValue = tieredSpilloverCache.get(getICacheKey(key)); String mapValue = keyValueMap.get(key); Long tookTime = tookTimeMap.get(mapValue); if (tookTime != null && tookTime > timeValueThresholdNanos) { @@ -1024,8 +996,9 @@ public CachedQueryResult.PolicyValues apply(String s) { public void testMinimumThresholdSettingValue() throws Exception { // Confirm we can't set TieredSpilloverCache.TieredSpilloverCacheFactory.TIERED_SPILLOVER_DISK_TOOK_TIME_THRESHOLD to below // TimeValue.ZERO (for example, MINUS_ONE) - Setting concreteSetting = TieredSpilloverCacheSettings.TIERED_SPILLOVER_DISK_TOOK_TIME_THRESHOLD - .getConcreteSettingForNamespace(CacheType.INDICES_REQUEST_CACHE.getSettingPrefix()); + Setting concreteSetting = TieredSpilloverCacheSettings.TOOK_TIME_POLICY_CONCRETE_SETTINGS_MAP.get( + CacheType.INDICES_REQUEST_CACHE + ); TimeValue validDuration = new TimeValue(0, TimeUnit.MILLISECONDS); Settings validSettings = Settings.builder().put(concreteSetting.getKey(), validDuration).build(); @@ -1035,6 +1008,27 @@ public void testMinimumThresholdSettingValue() throws Exception { assertEquals(validDuration, concreteSetting.get(validSettings)); } + private List getMockDimensions() { + List dims = new ArrayList<>(); + for (String dimensionName : dimensionNames) { + dims.add("0"); + } + return dims; + } + + private ICacheKey getICacheKey(String key) { + return new ICacheKey<>(key, getMockDimensions()); + } + + class MockCacheRemovalListener implements RemovalListener, V> { + final CounterMetric evictionsMetric = new CounterMetric(); + + @Override + public void onRemoval(RemovalNotification, V> notification) { + evictionsMetric.inc(); + } + } + private static class AllowFirstLetterA implements Predicate { @Override public boolean test(String data) { @@ -1053,12 +1047,12 @@ public boolean test(String data) { } } - private LoadAwareCacheLoader getLoadAwareCacheLoader() { + private LoadAwareCacheLoader, String> getLoadAwareCacheLoader() { return new LoadAwareCacheLoader<>() { boolean isLoaded = false; @Override - public String load(String key) { + public String load(ICacheKey key) { isLoaded = true; return UUID.randomUUID().toString(); } @@ -1070,14 +1064,14 @@ public boolean isLoaded() { }; } - private LoadAwareCacheLoader getLoadAwareCacheLoader(Map keyValueMap) { + private LoadAwareCacheLoader, String> getLoadAwareCacheLoader(Map keyValueMap) { return new LoadAwareCacheLoader<>() { boolean isLoaded = false; @Override - public String load(String key) { + public String load(ICacheKey key) { isLoaded = true; - String mapValue = keyValueMap.get(key); + String mapValue = keyValueMap.get(key.key); if (mapValue == null) { mapValue = UUID.randomUUID().toString(); } @@ -1091,10 +1085,10 @@ public boolean isLoaded() { }; } - private TieredSpilloverCache intializeTieredSpilloverCache( + private TieredSpilloverCache initializeTieredSpilloverCache( int keyValueSize, int diskCacheSize, - RemovalListener removalListener, + RemovalListener, String> removalListener, Settings settings, long diskDeliberateDelay @@ -1105,7 +1099,7 @@ private TieredSpilloverCache intializeTieredSpilloverCache( private TieredSpilloverCache intializeTieredSpilloverCache( int keyValueSize, int diskCacheSize, - RemovalListener removalListener, + RemovalListener, String> removalListener, Settings settings, long diskDeliberateDelay, List> policies @@ -1114,6 +1108,8 @@ private TieredSpilloverCache intializeTieredSpilloverCache( CacheConfig cacheConfig = new CacheConfig.Builder().setKeyType(String.class) .setKeyType(String.class) .setWeigher((k, v) -> keyValueSize) + .setSettings(settings) + .setDimensionNames(dimensionNames) .setRemovalListener(removalListener) .setSettings( Settings.builder() 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 edb2c900be46c..185d51732a116 100644 --- a/plugins/cache-ehcache/src/main/java/org/opensearch/cache/store/disk/EhcacheDiskCache.java +++ b/plugins/cache-ehcache/src/main/java/org/opensearch/cache/store/disk/EhcacheDiskCache.java @@ -17,15 +17,18 @@ import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.cache.CacheType; import org.opensearch.common.cache.ICache; +import org.opensearch.common.cache.ICacheKey; import org.opensearch.common.cache.LoadAwareCacheLoader; import org.opensearch.common.cache.RemovalListener; import org.opensearch.common.cache.RemovalNotification; import org.opensearch.common.cache.RemovalReason; +import org.opensearch.common.cache.serializer.ICacheKeySerializer; import org.opensearch.common.cache.serializer.Serializer; +import org.opensearch.common.cache.stats.CacheStatsHolder; +import org.opensearch.common.cache.stats.ImmutableCacheStatsHolder; import org.opensearch.common.cache.store.builders.ICacheBuilder; import org.opensearch.common.cache.store.config.CacheConfig; import org.opensearch.common.collect.Tuple; -import org.opensearch.common.metrics.CounterMetric; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; @@ -40,6 +43,7 @@ import java.time.Duration; import java.util.Arrays; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.NoSuchElementException; import java.util.Objects; @@ -49,6 +53,7 @@ import java.util.concurrent.ExecutionException; import java.util.function.BiFunction; import java.util.function.Supplier; +import java.util.function.ToLongBiFunction; import org.ehcache.Cache; import org.ehcache.CachePersistenceException; @@ -101,21 +106,20 @@ public class EhcacheDiskCache implements ICache { private final PersistentCacheManager cacheManager; // Disk cache. Using ByteArrayWrapper to compare two byte[] by values rather than the default reference checks - private Cache cache; + @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 long maxWeightInBytes; private final String storagePath; private final Class keyType; private final Class valueType; private final TimeValue expireAfterAccess; + private final CacheStatsHolder cacheStatsHolder; private final EhCacheEventListener ehCacheEventListener; private final String threadPoolAlias; private final Settings settings; - private final RemovalListener removalListener; + private final RemovalListener, V> removalListener; private final CacheType cacheType; private final String diskCacheAlias; - // TODO: Move count to stats once those changes are ready. - private final CounterMetric entries = new CounterMetric(); - private final Serializer keySerializer; private final Serializer valueSerializer; @@ -123,7 +127,7 @@ public class EhcacheDiskCache implements ICache { * Used in computeIfAbsent to synchronize loading of a given key. This is needed as ehcache doesn't provide a * computeIfAbsent method. */ - Map>> completableFutureMap = new ConcurrentHashMap<>(); + Map, CompletableFuture, V>>> completableFutureMap = new ConcurrentHashMap<>(); private EhcacheDiskCache(Builder builder) { this.keyType = Objects.requireNonNull(builder.keyType, "Key type shouldn't be null"); @@ -154,31 +158,39 @@ private EhcacheDiskCache(Builder builder) { this.cacheManager = buildCacheManager(); Objects.requireNonNull(builder.getRemovalListener(), "Removal listener can't be null"); this.removalListener = builder.getRemovalListener(); - this.ehCacheEventListener = new EhCacheEventListener(builder.getRemovalListener()); + Objects.requireNonNull(builder.getWeigher(), "Weigher can't be null"); + this.ehCacheEventListener = new EhCacheEventListener(builder.getRemovalListener(), builder.getWeigher()); this.cache = buildCache(Duration.ofMillis(expireAfterAccess.getMillis()), builder); + List dimensionNames = Objects.requireNonNull(builder.dimensionNames, "Dimension names can't be null"); + this.cacheStatsHolder = new CacheStatsHolder(dimensionNames); } - private Cache buildCache(Duration expireAfterAccess, Builder builder) { + @SuppressWarnings({ "rawtypes" }) + private Cache buildCache(Duration expireAfterAccess, Builder builder) { try { return this.cacheManager.createCache( this.diskCacheAlias, CacheConfigurationBuilder.newCacheConfigurationBuilder( - this.keyType, + ICacheKey.class, ByteArrayWrapper.class, ResourcePoolsBuilder.newResourcePoolsBuilder().disk(maxWeightInBytes, MemoryUnit.B) ).withExpiry(new ExpiryPolicy<>() { @Override - public Duration getExpiryForCreation(K key, ByteArrayWrapper value) { + public Duration getExpiryForCreation(ICacheKey key, ByteArrayWrapper value) { return INFINITE; } @Override - public Duration getExpiryForAccess(K key, Supplier value) { + public Duration getExpiryForAccess(ICacheKey key, Supplier value) { return expireAfterAccess; } @Override - public Duration getExpiryForUpdate(K key, Supplier oldValue, ByteArrayWrapper newValue) { + public Duration getExpiryForUpdate( + ICacheKey key, + Supplier oldValue, + ByteArrayWrapper newValue + ) { return INFINITE; } }) @@ -192,7 +204,7 @@ public Duration getExpiryForUpdate(K key, Supplier o (Integer) EhcacheDiskCacheSettings.getSettingListForCacheType(cacheType).get(DISK_SEGMENT_KEY).get(settings) ) ) - .withKeySerializer(new KeySerializerWrapper(keySerializer)) + .withKeySerializer(new KeySerializerWrapper(keySerializer)) .withValueSerializer(new ByteArrayWrapperSerializer()) // We pass ByteArrayWrapperSerializer as ehcache's value serializer. If V is an interface, and we pass its // serializer directly to ehcache, ehcache requires the classes match exactly before/after serialization. @@ -225,7 +237,7 @@ private CacheEventListenerConfigurationBuilder getListenerConfiguration(Builder< } // Package private for testing - Map>> getCompletableFutureMap() { + Map, CompletableFuture, V>>> getCompletableFutureMap() { return completableFutureMap; } @@ -254,7 +266,7 @@ private PersistentCacheManager buildCacheManager() { } @Override - public V get(K key) { + public V get(ICacheKey key) { if (key == null) { throw new IllegalArgumentException("Key passed to ehcache disk cache was null."); } @@ -264,6 +276,11 @@ public V get(K key) { } catch (CacheLoadingException ex) { throw new OpenSearchException("Exception occurred while trying to fetch item from ehcache disk cache"); } + if (value != null) { + cacheStatsHolder.incrementHits(key.dimensions); + } else { + cacheStatsHolder.incrementMisses(key.dimensions); + } return value; } @@ -273,7 +290,7 @@ public V get(K key) { * @param value Type of value. */ @Override - public void put(K key, V value) { + public void put(ICacheKey key, V value) { try { cache.put(key, serializeValue(value)); } catch (CacheWritingException ex) { @@ -289,26 +306,31 @@ public void put(K key, V value) { * @throws Exception when either internal get or put calls fail. */ @Override - public V computeIfAbsent(K key, LoadAwareCacheLoader loader) throws Exception { - // Ehache doesn't provide any computeIfAbsent function. Exposes putIfAbsent but that works differently and is + public V computeIfAbsent(ICacheKey key, LoadAwareCacheLoader, V> loader) throws Exception { + // Ehcache doesn't provide any computeIfAbsent function. Exposes putIfAbsent but that works differently and is // not performant in case there are multiple concurrent request for same key. Below is our own custom // implementation of computeIfAbsent on top of ehcache. Inspired by OpenSearch Cache implementation. V value = deserializeValue(cache.get(key)); if (value == null) { value = compute(key, loader); } + if (!loader.isLoaded()) { + cacheStatsHolder.incrementHits(key.dimensions); + } else { + cacheStatsHolder.incrementMisses(key.dimensions); + } return value; } - private V compute(K key, LoadAwareCacheLoader loader) throws Exception { + private V compute(ICacheKey key, LoadAwareCacheLoader, V> loader) throws Exception { // A future that returns a pair of key/value. - CompletableFuture> completableFuture = new CompletableFuture<>(); + CompletableFuture, V>> completableFuture = new CompletableFuture<>(); // Only one of the threads will succeed putting a future into map for the same key. // Rest will fetch existing future. - CompletableFuture> future = completableFutureMap.putIfAbsent(key, completableFuture); + CompletableFuture, V>> future = completableFutureMap.putIfAbsent(key, completableFuture); // 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, Throwable, V> handler = (pair, ex) -> { + BiFunction, V>, Throwable, V> handler = (pair, ex) -> { V value = null; if (pair != null) { cache.put(pair.v1(), serializeValue(pair.v2())); @@ -358,9 +380,14 @@ private V compute(K key, LoadAwareCacheLoader loader) throws Exception { * @param key key to be invalidated. */ @Override - public void invalidate(K key) { + public void invalidate(ICacheKey key) { try { - cache.remove(key); + if (key.getDropStatsForDimensions()) { + cacheStatsHolder.removeDimensions(key.dimensions); + } + if (key.key != null) { + cache.remove(key); + } } catch (CacheWritingException ex) { // Handle throw new RuntimeException(ex); @@ -371,7 +398,7 @@ public void invalidate(K key) { @Override public void invalidateAll() { cache.clear(); - this.entries.dec(this.entries.count()); // reset to zero. + cacheStatsHolder.reset(); } /** @@ -379,7 +406,7 @@ public void invalidateAll() { * @return Iterable */ @Override - public Iterable keys() { + public Iterable> keys() { return () -> new EhCacheKeyIterator<>(cache.iterator()); } @@ -389,7 +416,7 @@ public Iterable keys() { */ @Override public long count() { - return entries.count(); + return cacheStatsHolder.count(); } @Override @@ -416,15 +443,25 @@ public void close() { } } + /** + * Relevant stats for this cache. + * @return CacheStats + */ + @Override + public ImmutableCacheStatsHolder stats() { + return cacheStatsHolder.getImmutableCacheStatsHolder(); + } + /** * This iterator wraps ehCache iterator and only iterates over its keys. * @param Type of key */ - class EhCacheKeyIterator implements Iterator { + @SuppressWarnings({ "rawtypes", "unchecked" }) + class EhCacheKeyIterator implements Iterator> { - Iterator> iterator; + Iterator> iterator; - EhCacheKeyIterator(Iterator> iterator) { + EhCacheKeyIterator(Iterator> iterator) { this.iterator = iterator; } @@ -434,7 +471,7 @@ public boolean hasNext() { } @Override - public K next() { + public ICacheKey next() { if (!hasNext()) { throw new NoSuchElementException(); } @@ -450,43 +487,60 @@ public void remove() { /** * Wrapper over Ehcache original listener to listen to desired events and notify desired subscribers. */ - class EhCacheEventListener implements CacheEventListener { + class EhCacheEventListener implements CacheEventListener, ByteArrayWrapper> { + private final RemovalListener, V> removalListener; + private ToLongBiFunction, V> weigher; - private final RemovalListener removalListener; - - EhCacheEventListener(RemovalListener removalListener) { + EhCacheEventListener(RemovalListener, V> removalListener, ToLongBiFunction, V> weigher) { this.removalListener = removalListener; + this.weigher = weigher; + } + + private long getOldValuePairSize(CacheEvent, ? extends ByteArrayWrapper> event) { + return weigher.applyAsLong(event.getKey(), deserializeValue(event.getOldValue())); + } + + private long getNewValuePairSize(CacheEvent, ? extends ByteArrayWrapper> event) { + return weigher.applyAsLong(event.getKey(), deserializeValue(event.getNewValue())); } @Override - public void onEvent(CacheEvent event) { + public void onEvent(CacheEvent, ? extends ByteArrayWrapper> event) { switch (event.getType()) { case CREATED: - entries.inc(); + cacheStatsHolder.incrementEntries(event.getKey().dimensions); + cacheStatsHolder.incrementSizeInBytes(event.getKey().dimensions, getNewValuePairSize(event)); assert event.getOldValue() == null; break; case EVICTED: this.removalListener.onRemoval( new RemovalNotification<>(event.getKey(), deserializeValue(event.getOldValue()), RemovalReason.EVICTED) ); - entries.dec(); + cacheStatsHolder.decrementEntries(event.getKey().dimensions); + cacheStatsHolder.decrementSizeInBytes(event.getKey().dimensions, getOldValuePairSize(event)); + cacheStatsHolder.incrementEvictions(event.getKey().dimensions); assert event.getNewValue() == null; break; case REMOVED: - entries.dec(); this.removalListener.onRemoval( new RemovalNotification<>(event.getKey(), deserializeValue(event.getOldValue()), RemovalReason.EXPLICIT) ); + cacheStatsHolder.decrementEntries(event.getKey().dimensions); + cacheStatsHolder.decrementSizeInBytes(event.getKey().dimensions, getOldValuePairSize(event)); assert event.getNewValue() == null; break; case EXPIRED: this.removalListener.onRemoval( new RemovalNotification<>(event.getKey(), deserializeValue(event.getOldValue()), RemovalReason.INVALIDATED) ); - entries.dec(); + cacheStatsHolder.decrementEntries(event.getKey().dimensions); + cacheStatsHolder.decrementSizeInBytes(event.getKey().dimensions, getOldValuePairSize(event)); assert event.getNewValue() == null; break; case UPDATED: + long newSize = getNewValuePairSize(event); + long oldSize = getOldValuePairSize(event); + cacheStatsHolder.incrementSizeInBytes(event.getKey().dimensions, newSize - oldSize); break; default: break; @@ -495,13 +549,14 @@ public void onEvent(CacheEvent event) { } /** - * Wrapper over Serializer which is compatible with ehcache's serializer requirements. + * Wrapper over ICacheKeySerializer which is compatible with ehcache's serializer requirements. */ - private class KeySerializerWrapper implements org.ehcache.spi.serialization.Serializer { - private Serializer serializer; + @SuppressWarnings({ "rawtypes", "unchecked" }) + private class KeySerializerWrapper implements org.ehcache.spi.serialization.Serializer { + private ICacheKeySerializer serializer; - public KeySerializerWrapper(Serializer keySerializer) { - this.serializer = keySerializer; + public KeySerializerWrapper(Serializer internalKeySerializer) { + this.serializer = new ICacheKeySerializer<>(internalKeySerializer); } // This constructor must be present, but does not have to work as we are not actually persisting the disk @@ -510,19 +565,19 @@ public KeySerializerWrapper(Serializer keySerializer) { public KeySerializerWrapper(ClassLoader classLoader, FileBasedPersistenceContext persistenceContext) {} @Override - public ByteBuffer serialize(T object) throws SerializerException { + public ByteBuffer serialize(ICacheKey object) throws SerializerException { return ByteBuffer.wrap(serializer.serialize(object)); } @Override - public T read(ByteBuffer binary) throws ClassNotFoundException, SerializerException { + public ICacheKey read(ByteBuffer binary) throws ClassNotFoundException, SerializerException { byte[] arr = new byte[binary.remaining()]; binary.get(arr); return serializer.deserialize(arr); } @Override - public boolean equals(T object, ByteBuffer binary) throws ClassNotFoundException, SerializerException { + public boolean equals(ICacheKey object, ByteBuffer binary) throws ClassNotFoundException, SerializerException { byte[] arr = new byte[binary.remaining()]; binary.get(arr); return serializer.equals(object, arr); @@ -566,8 +621,7 @@ public boolean equals(ByteArrayWrapper object, ByteBuffer binary) throws ClassNo * @return the serialized value */ private ByteArrayWrapper serializeValue(V value) { - ByteArrayWrapper result = new ByteArrayWrapper(valueSerializer.serialize(value)); - return result; + return new ByteArrayWrapper(valueSerializer.serialize(value)); } /** @@ -625,6 +679,8 @@ public ICache create(CacheConfig config, CacheType cacheType, .setValueType(config.getValueType()) .setKeySerializer(keySerializer) .setValueSerializer(valueSerializer) + .setDimensionNames(config.getDimensionNames()) + .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)) @@ -658,6 +714,7 @@ public static class Builder extends ICacheBuilder { private Class keyType; private Class valueType; + private List dimensionNames; private Serializer keySerializer; private Serializer valueSerializer; @@ -736,6 +793,16 @@ public Builder setIsEventListenerModeSync(boolean isEventListenerModeSync) return this; } + /** + * Sets the allowed dimension names for keys that will enter this cache. + * @param dimensionNames A list of dimension names this cache will accept + * @return builder + */ + public Builder setDimensionNames(List dimensionNames) { + this.dimensionNames = dimensionNames; + return this; + } + /** * Sets the key serializer for this cache. * @param keySerializer the key serializer @@ -764,7 +831,7 @@ public EhcacheDiskCache build() { /** * A wrapper over byte[], with equals() that works using Arrays.equals(). - * Necessary due to a bug in Ehcache. + * Necessary due to a limitation in how Ehcache compares byte[]. */ static class ByteArrayWrapper { private final byte[] value; 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 3a98ad2fef6b1..06ebed08d7525 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 @@ -14,11 +14,13 @@ import org.opensearch.common.Randomness; import org.opensearch.common.cache.CacheType; import org.opensearch.common.cache.ICache; +import org.opensearch.common.cache.ICacheKey; import org.opensearch.common.cache.LoadAwareCacheLoader; import org.opensearch.common.cache.RemovalListener; import org.opensearch.common.cache.RemovalNotification; import org.opensearch.common.cache.serializer.BytesReferenceSerializer; import org.opensearch.common.cache.serializer.Serializer; +import org.opensearch.common.cache.stats.ImmutableCacheStats; import org.opensearch.common.cache.store.config.CacheConfig; import org.opensearch.common.metrics.CounterMetric; import org.opensearch.common.settings.Settings; @@ -43,6 +45,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.Phaser; +import java.util.function.ToLongBiFunction; import static org.opensearch.cache.EhcacheDiskCacheSettings.DISK_LISTENER_MODE_SYNC_KEY; import static org.opensearch.cache.EhcacheDiskCacheSettings.DISK_MAX_SIZE_IN_BYTES_KEY; @@ -53,10 +56,12 @@ public class EhCacheDiskCacheTests extends OpenSearchSingleNodeTestCase { private static final int CACHE_SIZE_IN_BYTES = 1024 * 101; + private final String dimensionName = "shardId"; public void testBasicGetAndPut() throws IOException { Settings settings = Settings.builder().build(); MockRemovalListener removalListener = new MockRemovalListener<>(); + ToLongBiFunction, String> weigher = getWeigher(); try (NodeEnvironment env = newNodeEnvironment(settings)) { ICache ehcacheTest = new EhcacheDiskCache.Builder().setThreadPoolAlias("ehcacheTest") .setStoragePath(env.nodePaths()[0].indicesPath.toString() + "/request_cache") @@ -65,32 +70,42 @@ public void testBasicGetAndPut() throws IOException { .setValueType(String.class) .setKeySerializer(new StringSerializer()) .setValueSerializer(new StringSerializer()) + .setDimensionNames(List.of(dimensionName)) .setCacheType(CacheType.INDICES_REQUEST_CACHE) .setSettings(settings) .setExpireAfterAccess(TimeValue.MAX_VALUE) .setMaximumWeightInBytes(CACHE_SIZE_IN_BYTES) .setRemovalListener(removalListener) + .setWeigher(weigher) .build(); int randomKeys = randomIntBetween(10, 100); + long expectedSize = 0; Map keyValueMap = new HashMap<>(); for (int i = 0; i < randomKeys; i++) { keyValueMap.put(UUID.randomUUID().toString(), UUID.randomUUID().toString()); } for (Map.Entry entry : keyValueMap.entrySet()) { - ehcacheTest.put(entry.getKey(), entry.getValue()); + ICacheKey iCacheKey = getICacheKey(entry.getKey()); + ehcacheTest.put(iCacheKey, entry.getValue()); + expectedSize += weigher.applyAsLong(iCacheKey, entry.getValue()); } for (Map.Entry entry : keyValueMap.entrySet()) { - String value = ehcacheTest.get(entry.getKey()); + String value = ehcacheTest.get(getICacheKey(entry.getKey())); assertEquals(entry.getValue(), value); } + assertEquals(randomKeys, ehcacheTest.stats().getTotalEntries()); + assertEquals(randomKeys, ehcacheTest.stats().getTotalHits()); + assertEquals(expectedSize, ehcacheTest.stats().getTotalSizeInBytes()); assertEquals(randomKeys, ehcacheTest.count()); // Validate misses int expectedNumberOfMisses = randomIntBetween(10, 200); for (int i = 0; i < expectedNumberOfMisses; i++) { - ehcacheTest.get(UUID.randomUUID().toString()); + ehcacheTest.get(getICacheKey(UUID.randomUUID().toString())); } + assertEquals(expectedNumberOfMisses, ehcacheTest.stats().getTotalMisses()); + ehcacheTest.close(); } } @@ -105,6 +120,8 @@ public void testBasicGetAndPutUsingFactory() throws IOException { .setRemovalListener(removalListener) .setKeySerializer(new StringSerializer()) .setValueSerializer(new StringSerializer()) + .setDimensionNames(List.of(dimensionName)) + .setWeigher(getWeigher()) .setSettings( Settings.builder() .put( @@ -132,14 +149,14 @@ public void testBasicGetAndPutUsingFactory() throws IOException { Map.of() ); int randomKeys = randomIntBetween(10, 100); - Map keyValueMap = new HashMap<>(); + Map, String> keyValueMap = new HashMap<>(); for (int i = 0; i < randomKeys; i++) { - keyValueMap.put(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + keyValueMap.put(getICacheKey(UUID.randomUUID().toString()), UUID.randomUUID().toString()); } - for (Map.Entry entry : keyValueMap.entrySet()) { + for (Map.Entry, String> entry : keyValueMap.entrySet()) { ehcacheTest.put(entry.getKey(), entry.getValue()); } - for (Map.Entry entry : keyValueMap.entrySet()) { + for (Map.Entry, String> entry : keyValueMap.entrySet()) { String value = ehcacheTest.get(entry.getKey()); assertEquals(entry.getValue(), value); } @@ -148,7 +165,7 @@ public void testBasicGetAndPutUsingFactory() throws IOException { // Validate misses int expectedNumberOfMisses = randomIntBetween(10, 200); for (int i = 0; i < expectedNumberOfMisses; i++) { - ehcacheTest.get(UUID.randomUUID().toString()); + ehcacheTest.get(getICacheKey(UUID.randomUUID().toString())); } ehcacheTest.close(); @@ -167,22 +184,24 @@ public void testConcurrentPut() throws Exception { .setValueType(String.class) .setKeySerializer(new StringSerializer()) .setValueSerializer(new StringSerializer()) + .setDimensionNames(List.of(dimensionName)) .setCacheType(CacheType.INDICES_REQUEST_CACHE) .setSettings(settings) .setExpireAfterAccess(TimeValue.MAX_VALUE) .setMaximumWeightInBytes(CACHE_SIZE_IN_BYTES) .setRemovalListener(removalListener) + .setWeigher(getWeigher()) .build(); int randomKeys = randomIntBetween(20, 100); Thread[] threads = new Thread[randomKeys]; Phaser phaser = new Phaser(randomKeys + 1); CountDownLatch countDownLatch = new CountDownLatch(randomKeys); - Map keyValueMap = new HashMap<>(); + Map, String> keyValueMap = new HashMap<>(); int j = 0; for (int i = 0; i < randomKeys; i++) { - keyValueMap.put(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + keyValueMap.put(getICacheKey(UUID.randomUUID().toString()), UUID.randomUUID().toString()); } - for (Map.Entry entry : keyValueMap.entrySet()) { + for (Map.Entry, String> entry : keyValueMap.entrySet()) { threads[j] = new Thread(() -> { phaser.arriveAndAwaitAdvance(); ehcacheTest.put(entry.getKey(), entry.getValue()); @@ -193,11 +212,12 @@ public void testConcurrentPut() throws Exception { } phaser.arriveAndAwaitAdvance(); // Will trigger parallel puts above. countDownLatch.await(); // Wait for all threads to finish - for (Map.Entry entry : keyValueMap.entrySet()) { + for (Map.Entry, String> entry : keyValueMap.entrySet()) { String value = ehcacheTest.get(entry.getKey()); assertEquals(entry.getValue(), value); } assertEquals(randomKeys, ehcacheTest.count()); + assertEquals(randomKeys, ehcacheTest.stats().getTotalEntries()); ehcacheTest.close(); } } @@ -214,11 +234,13 @@ public void testEhcacheParallelGets() throws Exception { .setValueType(String.class) .setKeySerializer(new StringSerializer()) .setValueSerializer(new StringSerializer()) + .setDimensionNames(List.of(dimensionName)) .setCacheType(CacheType.INDICES_REQUEST_CACHE) .setSettings(settings) .setExpireAfterAccess(TimeValue.MAX_VALUE) .setMaximumWeightInBytes(CACHE_SIZE_IN_BYTES) .setRemovalListener(removalListener) + .setWeigher(getWeigher()) .build(); int randomKeys = randomIntBetween(20, 100); Thread[] threads = new Thread[randomKeys]; @@ -230,13 +252,13 @@ public void testEhcacheParallelGets() throws Exception { keyValueMap.put(UUID.randomUUID().toString(), UUID.randomUUID().toString()); } for (Map.Entry entry : keyValueMap.entrySet()) { - ehcacheTest.put(entry.getKey(), entry.getValue()); + ehcacheTest.put(getICacheKey(entry.getKey()), entry.getValue()); } assertEquals(keyValueMap.size(), ehcacheTest.count()); for (Map.Entry entry : keyValueMap.entrySet()) { threads[j] = new Thread(() -> { phaser.arriveAndAwaitAdvance(); - assertEquals(entry.getValue(), ehcacheTest.get(entry.getKey())); + assertEquals(entry.getValue(), ehcacheTest.get(getICacheKey(entry.getKey()))); countDownLatch.countDown(); }); threads[j].start(); @@ -244,6 +266,7 @@ public void testEhcacheParallelGets() throws Exception { } phaser.arriveAndAwaitAdvance(); // Will trigger parallel puts above. countDownLatch.await(); // Wait for all threads to finish + assertEquals(randomKeys, ehcacheTest.stats().getTotalHits()); ehcacheTest.close(); } } @@ -259,11 +282,13 @@ public void testEhcacheKeyIterator() throws Exception { .setValueType(String.class) .setKeySerializer(new StringSerializer()) .setValueSerializer(new StringSerializer()) + .setDimensionNames(List.of(dimensionName)) .setCacheType(CacheType.INDICES_REQUEST_CACHE) .setSettings(settings) .setExpireAfterAccess(TimeValue.MAX_VALUE) .setMaximumWeightInBytes(CACHE_SIZE_IN_BYTES) .setRemovalListener(new MockRemovalListener<>()) + .setWeigher(getWeigher()) .build(); int randomKeys = randomIntBetween(2, 100); @@ -272,12 +297,12 @@ public void testEhcacheKeyIterator() throws Exception { keyValueMap.put(UUID.randomUUID().toString(), UUID.randomUUID().toString()); } for (Map.Entry entry : keyValueMap.entrySet()) { - ehcacheTest.put(entry.getKey(), entry.getValue()); + ehcacheTest.put(getICacheKey(entry.getKey()), entry.getValue()); } - Iterator keys = ehcacheTest.keys().iterator(); + Iterator> keys = ehcacheTest.keys().iterator(); int keysCount = 0; while (keys.hasNext()) { - String key = keys.next(); + ICacheKey key = keys.next(); keysCount++; assertNotNull(ehcacheTest.get(key)); } @@ -289,6 +314,7 @@ public void testEhcacheKeyIterator() throws Exception { public void testEvictions() throws Exception { Settings settings = Settings.builder().build(); MockRemovalListener removalListener = new MockRemovalListener<>(); + ToLongBiFunction, String> weigher = getWeigher(); try (NodeEnvironment env = newNodeEnvironment(settings)) { ICache ehcacheTest = new EhcacheDiskCache.Builder().setDiskCacheAlias("test1") .setStoragePath(env.nodePaths()[0].indicesPath.toString() + "/request_cache") @@ -298,11 +324,13 @@ public void testEvictions() throws Exception { .setValueType(String.class) .setKeySerializer(new StringSerializer()) .setValueSerializer(new StringSerializer()) + .setDimensionNames(List.of(dimensionName)) .setCacheType(CacheType.INDICES_REQUEST_CACHE) .setSettings(settings) .setExpireAfterAccess(TimeValue.MAX_VALUE) .setMaximumWeightInBytes(CACHE_SIZE_IN_BYTES) .setRemovalListener(removalListener) + .setWeigher(weigher) .build(); // Generate a string with 100 characters @@ -311,9 +339,10 @@ public void testEvictions() throws Exception { // Trying to generate more than 100kb to cause evictions. for (int i = 0; i < 1000; i++) { String key = "Key" + i; - ehcacheTest.put(key, value); + ehcacheTest.put(getICacheKey(key), value); } assertEquals(660, removalListener.evictionMetric.count()); + assertEquals(660, ehcacheTest.stats().getTotalEvictions()); ehcacheTest.close(); } } @@ -330,11 +359,13 @@ public void testComputeIfAbsentConcurrently() throws Exception { .setValueType(String.class) .setKeySerializer(new StringSerializer()) .setValueSerializer(new StringSerializer()) + .setDimensionNames(List.of(dimensionName)) .setCacheType(CacheType.INDICES_REQUEST_CACHE) .setSettings(settings) .setExpireAfterAccess(TimeValue.MAX_VALUE) .setMaximumWeightInBytes(CACHE_SIZE_IN_BYTES) .setRemovalListener(removalListener) + .setWeigher(getWeigher()) .build(); int numberOfRequest = 2;// randomIntBetween(200, 400); @@ -344,12 +375,12 @@ public void testComputeIfAbsentConcurrently() throws Exception { Phaser phaser = new Phaser(numberOfRequest + 1); CountDownLatch countDownLatch = new CountDownLatch(numberOfRequest); - List> loadAwareCacheLoaderList = new CopyOnWriteArrayList<>(); + List, String>> loadAwareCacheLoaderList = new CopyOnWriteArrayList<>(); // Try to hit different request with the same key concurrently. Verify value is only loaded once. for (int i = 0; i < numberOfRequest; i++) { threads[i] = new Thread(() -> { - LoadAwareCacheLoader loadAwareCacheLoader = new LoadAwareCacheLoader<>() { + LoadAwareCacheLoader, String> loadAwareCacheLoader = new LoadAwareCacheLoader<>() { boolean isLoaded; @Override @@ -358,7 +389,7 @@ public boolean isLoaded() { } @Override - public String load(String key) { + public String load(ICacheKey key) { isLoaded = true; return value; } @@ -366,7 +397,7 @@ public String load(String key) { loadAwareCacheLoaderList.add(loadAwareCacheLoader); phaser.arriveAndAwaitAdvance(); try { - assertEquals(value, ehcacheTest.computeIfAbsent(key, loadAwareCacheLoader)); + assertEquals(value, ehcacheTest.computeIfAbsent(getICacheKey(key), loadAwareCacheLoader)); } catch (Exception e) { throw new RuntimeException(e); } @@ -384,6 +415,9 @@ public String load(String key) { } assertEquals(1, numberOfTimesValueLoaded); assertEquals(0, ((EhcacheDiskCache) ehcacheTest).getCompletableFutureMap().size()); + assertEquals(1, ehcacheTest.stats().getTotalMisses()); + assertEquals(1, ehcacheTest.stats().getTotalEntries()); + assertEquals(numberOfRequest - 1, ehcacheTest.stats().getTotalHits()); assertEquals(1, ehcacheTest.count()); ehcacheTest.close(); } @@ -401,11 +435,13 @@ public void testComputeIfAbsentConcurrentlyAndThrowsException() throws Exception .setValueType(String.class) .setKeySerializer(new StringSerializer()) .setValueSerializer(new StringSerializer()) + .setDimensionNames(List.of(dimensionName)) .setCacheType(CacheType.INDICES_REQUEST_CACHE) .setSettings(settings) .setExpireAfterAccess(TimeValue.MAX_VALUE) .setMaximumWeightInBytes(CACHE_SIZE_IN_BYTES) .setRemovalListener(removalListener) + .setWeigher(getWeigher()) .build(); int numberOfRequest = randomIntBetween(200, 400); @@ -414,12 +450,12 @@ public void testComputeIfAbsentConcurrentlyAndThrowsException() throws Exception Phaser phaser = new Phaser(numberOfRequest + 1); CountDownLatch countDownLatch = new CountDownLatch(numberOfRequest); - List> loadAwareCacheLoaderList = new CopyOnWriteArrayList<>(); + List, String>> loadAwareCacheLoaderList = new CopyOnWriteArrayList<>(); // Try to hit different request with the same key concurrently. Loader throws exception. for (int i = 0; i < numberOfRequest; i++) { threads[i] = new Thread(() -> { - LoadAwareCacheLoader loadAwareCacheLoader = new LoadAwareCacheLoader<>() { + LoadAwareCacheLoader, String> loadAwareCacheLoader = new LoadAwareCacheLoader<>() { boolean isLoaded; @Override @@ -428,14 +464,14 @@ public boolean isLoaded() { } @Override - public String load(String key) throws Exception { + public String load(ICacheKey key) throws Exception { isLoaded = true; throw new RuntimeException("Exception"); } }; loadAwareCacheLoaderList.add(loadAwareCacheLoader); phaser.arriveAndAwaitAdvance(); - assertThrows(ExecutionException.class, () -> ehcacheTest.computeIfAbsent(key, loadAwareCacheLoader)); + assertThrows(ExecutionException.class, () -> ehcacheTest.computeIfAbsent(getICacheKey(key), loadAwareCacheLoader)); countDownLatch.countDown(); }); threads[i].start(); @@ -460,11 +496,13 @@ public void testComputeIfAbsentWithNullValueLoading() throws Exception { .setValueType(String.class) .setKeySerializer(new StringSerializer()) .setValueSerializer(new StringSerializer()) + .setDimensionNames(List.of(dimensionName)) .setCacheType(CacheType.INDICES_REQUEST_CACHE) .setSettings(settings) .setExpireAfterAccess(TimeValue.MAX_VALUE) .setMaximumWeightInBytes(CACHE_SIZE_IN_BYTES) .setRemovalListener(removalListener) + .setWeigher(getWeigher()) .build(); int numberOfRequest = randomIntBetween(200, 400); @@ -473,12 +511,12 @@ public void testComputeIfAbsentWithNullValueLoading() throws Exception { Phaser phaser = new Phaser(numberOfRequest + 1); CountDownLatch countDownLatch = new CountDownLatch(numberOfRequest); - List> loadAwareCacheLoaderList = new CopyOnWriteArrayList<>(); + List, String>> loadAwareCacheLoaderList = new CopyOnWriteArrayList<>(); // Try to hit different request with the same key concurrently. Loader throws exception. for (int i = 0; i < numberOfRequest; i++) { threads[i] = new Thread(() -> { - LoadAwareCacheLoader loadAwareCacheLoader = new LoadAwareCacheLoader<>() { + LoadAwareCacheLoader, String> loadAwareCacheLoader = new LoadAwareCacheLoader<>() { boolean isLoaded; @Override @@ -487,7 +525,7 @@ public boolean isLoaded() { } @Override - public String load(String key) throws Exception { + public String load(ICacheKey key) throws Exception { isLoaded = true; return null; } @@ -495,11 +533,11 @@ public String load(String key) throws Exception { loadAwareCacheLoaderList.add(loadAwareCacheLoader); phaser.arriveAndAwaitAdvance(); try { - ehcacheTest.computeIfAbsent(key, loadAwareCacheLoader); + ehcacheTest.computeIfAbsent(getICacheKey(key), loadAwareCacheLoader); } catch (Exception ex) { assertThat(ex.getCause(), instanceOf(NullPointerException.class)); } - assertThrows(ExecutionException.class, () -> ehcacheTest.computeIfAbsent(key, loadAwareCacheLoader)); + assertThrows(ExecutionException.class, () -> ehcacheTest.computeIfAbsent(getICacheKey(key), loadAwareCacheLoader)); countDownLatch.countDown(); }); threads[i].start(); @@ -512,42 +550,119 @@ public String load(String key) throws Exception { } } - public void testEhcacheKeyIteratorWithRemove() throws IOException { + public void testMemoryTracking() throws Exception { + // Test all cases for EhCacheEventListener.onEvent and check stats memory usage is updated correctly Settings settings = Settings.builder().build(); + ToLongBiFunction, String> weigher = getWeigher(); + int initialKeyLength = 40; + int initialValueLength = 40; + long sizeForOneInitialEntry = weigher.applyAsLong( + new ICacheKey<>(generateRandomString(initialKeyLength), getMockDimensions()), + generateRandomString(initialValueLength) + ); + int maxEntries = 2000; try (NodeEnvironment env = newNodeEnvironment(settings)) { ICache ehcacheTest = new EhcacheDiskCache.Builder().setDiskCacheAlias("test1") .setThreadPoolAlias("ehcacheTest") .setStoragePath(env.nodePaths()[0].indicesPath.toString() + "/request_cache") - .setIsEventListenerModeSync(true) .setKeyType(String.class) .setValueType(String.class) .setKeySerializer(new StringSerializer()) .setValueSerializer(new StringSerializer()) + .setDimensionNames(List.of(dimensionName)) + .setIsEventListenerModeSync(true) // Test fails if async; probably not all updates happen before checking stats .setCacheType(CacheType.INDICES_REQUEST_CACHE) .setSettings(settings) .setExpireAfterAccess(TimeValue.MAX_VALUE) + .setMaximumWeightInBytes(maxEntries * sizeForOneInitialEntry) + .setRemovalListener(new MockRemovalListener<>()) + .setWeigher(weigher) + .build(); + long expectedSize = 0; + + // Test CREATED case + int numInitialKeys = randomIntBetween(10, 100); + ArrayList> initialKeys = new ArrayList<>(); + for (int i = 0; i < numInitialKeys; i++) { + ICacheKey key = new ICacheKey<>(generateRandomString(initialKeyLength), getMockDimensions()); + String value = generateRandomString(initialValueLength); + ehcacheTest.put(key, value); + initialKeys.add(key); + expectedSize += weigher.applyAsLong(key, value); + assertEquals(expectedSize, ehcacheTest.stats().getTotalStats().getSizeInBytes()); + } + + // Test UPDATED case + HashMap, String> updatedValues = new HashMap<>(); + for (int i = 0; i < numInitialKeys * 0.5; i++) { + int newLengthDifference = randomIntBetween(-20, 20); + String newValue = generateRandomString(initialValueLength + newLengthDifference); + ehcacheTest.put(initialKeys.get(i), newValue); + updatedValues.put(initialKeys.get(i), newValue); + expectedSize += newLengthDifference; + assertEquals(expectedSize, ehcacheTest.stats().getTotalStats().getSizeInBytes()); + } + + // Test REMOVED case by removing all updated keys + for (int i = 0; i < numInitialKeys * 0.5; i++) { + ICacheKey removedKey = initialKeys.get(i); + ehcacheTest.invalidate(removedKey); + expectedSize -= weigher.applyAsLong(removedKey, updatedValues.get(removedKey)); + assertEquals(expectedSize, ehcacheTest.stats().getTotalStats().getSizeInBytes()); + } + + // Test EVICTED case by adding entries past the cap and ensuring memory size stays as what we expect + for (int i = 0; i < maxEntries - ehcacheTest.count(); i++) { + ICacheKey key = new ICacheKey<>(generateRandomString(initialKeyLength), getMockDimensions()); + String value = generateRandomString(initialValueLength); + ehcacheTest.put(key, value); + } + // TODO: Ehcache incorrectly evicts at 30-40% of max size. Fix this test once we figure out why. + // Since the EVICTED and EXPIRED cases use the same code as REMOVED, we should be ok on testing them for now. + // assertEquals(maxEntries * sizeForOneInitialEntry, ehcacheTest.stats().getTotalMemorySize()); + + ehcacheTest.close(); + } + } + + public void testEhcacheKeyIteratorWithRemove() throws IOException { + Settings settings = Settings.builder().build(); + try (NodeEnvironment env = newNodeEnvironment(settings)) { + ICache ehcacheTest = new EhcacheDiskCache.Builder().setDiskCacheAlias("test1") + .setThreadPoolAlias("ehcacheTest") + .setStoragePath(env.nodePaths()[0].indicesPath.toString() + "/request_cache") + .setIsEventListenerModeSync(true) + .setKeySerializer(new StringSerializer()) + .setValueSerializer(new StringSerializer()) + .setDimensionNames(List.of(dimensionName)) + .setCacheType(CacheType.INDICES_REQUEST_CACHE) + .setKeyType(String.class) + .setValueType(String.class) + .setSettings(settings) + .setExpireAfterAccess(TimeValue.MAX_VALUE) .setMaximumWeightInBytes(CACHE_SIZE_IN_BYTES) .setRemovalListener(new MockRemovalListener<>()) + .setWeigher(getWeigher()) .build(); int randomKeys = randomIntBetween(2, 100); for (int i = 0; i < randomKeys; i++) { - ehcacheTest.put(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + ehcacheTest.put(getICacheKey(UUID.randomUUID().toString()), UUID.randomUUID().toString()); } long originalSize = ehcacheTest.count(); assertEquals(randomKeys, originalSize); // Now try removing subset of keys and verify - List removedKeyList = new ArrayList<>(); - for (Iterator iterator = ehcacheTest.keys().iterator(); iterator.hasNext();) { - String key = iterator.next(); + List> removedKeyList = new ArrayList<>(); + for (Iterator> iterator = ehcacheTest.keys().iterator(); iterator.hasNext();) { + ICacheKey key = iterator.next(); if (randomBoolean()) { removedKeyList.add(key); iterator.remove(); } } // Verify the removed key doesn't exist anymore. - for (String ehcacheKey : removedKeyList) { + for (ICacheKey ehcacheKey : removedKeyList) { assertNull(ehcacheTest.get(ehcacheKey)); } // Verify ehcache entry size again. @@ -568,22 +683,24 @@ public void testInvalidateAll() throws Exception { .setValueType(String.class) .setKeySerializer(new StringSerializer()) .setValueSerializer(new StringSerializer()) + .setDimensionNames(List.of(dimensionName)) .setCacheType(CacheType.INDICES_REQUEST_CACHE) .setSettings(settings) .setExpireAfterAccess(TimeValue.MAX_VALUE) .setMaximumWeightInBytes(CACHE_SIZE_IN_BYTES) .setRemovalListener(removalListener) + .setWeigher(getWeigher()) .build(); int randomKeys = randomIntBetween(10, 100); - Map keyValueMap = new HashMap<>(); + Map, String> keyValueMap = new HashMap<>(); for (int i = 0; i < randomKeys; i++) { - keyValueMap.put(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + keyValueMap.put(getICacheKey(UUID.randomUUID().toString()), UUID.randomUUID().toString()); } - for (Map.Entry entry : keyValueMap.entrySet()) { + for (Map.Entry, String> entry : keyValueMap.entrySet()) { ehcacheTest.put(entry.getKey(), entry.getValue()); } ehcacheTest.invalidateAll(); // clear all the entries. - for (Map.Entry entry : keyValueMap.entrySet()) { + for (Map.Entry, String> entry : keyValueMap.entrySet()) { // Verify that value is null for a removed entry. assertNull(ehcacheTest.get(entry.getKey())); } @@ -600,6 +717,7 @@ public void testBasicGetAndPutBytesReference() throws Exception { .setStoragePath(env.nodePaths()[0].indicesPath.toString() + "/request_cache") .setKeySerializer(new StringSerializer()) .setValueSerializer(new BytesReferenceSerializer()) + .setDimensionNames(List.of(dimensionName)) .setKeyType(String.class) .setValueType(BytesReference.class) .setCacheType(CacheType.INDICES_REQUEST_CACHE) @@ -607,15 +725,16 @@ public void testBasicGetAndPutBytesReference() throws Exception { .setMaximumWeightInBytes(CACHE_SIZE_IN_BYTES * 20) // bigger so no evictions happen .setExpireAfterAccess(TimeValue.MAX_VALUE) .setRemovalListener(new MockRemovalListener<>()) + .setWeigher((key, value) -> 1) .build(); int randomKeys = randomIntBetween(10, 100); int valueLength = 100; Random rand = Randomness.get(); - Map keyValueMap = new HashMap<>(); + Map, BytesReference> keyValueMap = new HashMap<>(); for (int i = 0; i < randomKeys; i++) { byte[] valueBytes = new byte[valueLength]; rand.nextBytes(valueBytes); - keyValueMap.put(UUID.randomUUID().toString(), new BytesArray(valueBytes)); + keyValueMap.put(getICacheKey(UUID.randomUUID().toString()), new BytesArray(valueBytes)); // Test a non-BytesArray implementation of BytesReference. byte[] compositeBytes1 = new byte[valueLength]; @@ -623,12 +742,12 @@ public void testBasicGetAndPutBytesReference() throws Exception { rand.nextBytes(compositeBytes1); rand.nextBytes(compositeBytes2); BytesReference composite = CompositeBytesReference.of(new BytesArray(compositeBytes1), new BytesArray(compositeBytes2)); - keyValueMap.put(UUID.randomUUID().toString(), composite); + keyValueMap.put(getICacheKey(UUID.randomUUID().toString()), composite); } - for (Map.Entry entry : keyValueMap.entrySet()) { + for (Map.Entry, BytesReference> entry : keyValueMap.entrySet()) { ehCacheDiskCachingTier.put(entry.getKey(), entry.getValue()); } - for (Map.Entry entry : keyValueMap.entrySet()) { + for (Map.Entry, BytesReference> entry : keyValueMap.entrySet()) { BytesReference value = ehCacheDiskCachingTier.get(entry.getKey()); assertEquals(entry.getValue(), value); } @@ -647,29 +766,31 @@ public void testInvalidate() throws Exception { .setKeySerializer(new StringSerializer()) .setValueSerializer(new StringSerializer()) .setValueType(String.class) + .setDimensionNames(List.of(dimensionName)) .setCacheType(CacheType.INDICES_REQUEST_CACHE) .setSettings(settings) .setExpireAfterAccess(TimeValue.MAX_VALUE) .setMaximumWeightInBytes(CACHE_SIZE_IN_BYTES) .setRemovalListener(removalListener) + .setWeigher(getWeigher()) .build(); int randomKeys = randomIntBetween(10, 100); - Map keyValueMap = new HashMap<>(); + Map, String> keyValueMap = new HashMap<>(); for (int i = 0; i < randomKeys; i++) { - keyValueMap.put(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + keyValueMap.put(getICacheKey(UUID.randomUUID().toString()), UUID.randomUUID().toString()); } - for (Map.Entry entry : keyValueMap.entrySet()) { + for (Map.Entry, String> entry : keyValueMap.entrySet()) { ehcacheTest.put(entry.getKey(), entry.getValue()); } assertEquals(keyValueMap.size(), ehcacheTest.count()); - List removedKeyList = new ArrayList<>(); - for (Map.Entry entry : keyValueMap.entrySet()) { + List> removedKeyList = new ArrayList<>(); + for (Map.Entry, String> entry : keyValueMap.entrySet()) { if (randomBoolean()) { removedKeyList.add(entry.getKey()); ehcacheTest.invalidate(entry.getKey()); } } - for (String removedKey : removedKeyList) { + for (ICacheKey removedKey : removedKeyList) { assertNull(ehcacheTest.get(removedKey)); } assertEquals(keyValueMap.size() - removedKeyList.size(), ehcacheTest.count()); @@ -677,6 +798,67 @@ public void testInvalidate() throws Exception { } } + // Modified from OpenSearchOnHeapCacheTests.java + public void testInvalidateWithDropDimensions() throws Exception { + Settings settings = Settings.builder().build(); + List dimensionNames = List.of("dim1", "dim2"); + try (NodeEnvironment env = newNodeEnvironment(settings)) { + ICache ehCacheDiskCachingTier = new EhcacheDiskCache.Builder().setThreadPoolAlias("ehcacheTest") + .setStoragePath(env.nodePaths()[0].indicesPath.toString() + "/request_cache") + .setKeySerializer(new StringSerializer()) + .setValueSerializer(new StringSerializer()) + .setIsEventListenerModeSync(true) + .setDimensionNames(dimensionNames) + .setKeyType(String.class) + .setValueType(String.class) + .setCacheType(CacheType.INDICES_REQUEST_CACHE) + .setSettings(settings) + .setMaximumWeightInBytes(CACHE_SIZE_IN_BYTES * 20) // bigger so no evictions happen + .setExpireAfterAccess(TimeValue.MAX_VALUE) + .setRemovalListener(new MockRemovalListener<>()) + .setWeigher((key, value) -> 1) + .build(); + + List> keysAdded = new ArrayList<>(); + + for (int i = 0; i < 20; i++) { + ICacheKey key = new ICacheKey<>(UUID.randomUUID().toString(), getRandomDimensions(dimensionNames)); + keysAdded.add(key); + ehCacheDiskCachingTier.put(key, UUID.randomUUID().toString()); + } + + ICacheKey keyToDrop = keysAdded.get(0); + + ImmutableCacheStats snapshot = ehCacheDiskCachingTier.stats().getStatsForDimensionValues(keyToDrop.dimensions); + assertNotNull(snapshot); + + keyToDrop.setDropStatsForDimensions(true); + ehCacheDiskCachingTier.invalidate(keyToDrop); + + // Now assert the stats are gone for any key that has this combination of dimensions, but still there otherwise + for (ICacheKey keyAdded : keysAdded) { + snapshot = ehCacheDiskCachingTier.stats().getStatsForDimensionValues(keyAdded.dimensions); + if (keyAdded.dimensions.equals(keyToDrop.dimensions)) { + assertNull(snapshot); + } else { + assertNotNull(snapshot); + } + } + + ehCacheDiskCachingTier.close(); + } + } + + private List getRandomDimensions(List dimensionNames) { + Random rand = Randomness.get(); + int bound = 3; + List result = new ArrayList<>(); + for (String dimName : dimensionNames) { + result.add(String.valueOf(rand.nextInt(bound))); + } + return result; + } + private static String generateRandomString(int length) { String characters = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz0123456789"; StringBuilder randomString = new StringBuilder(length); @@ -689,12 +871,34 @@ private static String generateRandomString(int length) { return randomString.toString(); } - static class MockRemovalListener implements RemovalListener { + private List getMockDimensions() { + return List.of("0"); + } + + private ICacheKey getICacheKey(String key) { + return new ICacheKey<>(key, getMockDimensions()); + } + + private ToLongBiFunction, String> getWeigher() { + return (iCacheKey, value) -> { + // Size consumed by key + long totalSize = iCacheKey.key.length(); + for (String dim : iCacheKey.dimensions) { + totalSize += dim.length(); + } + totalSize += 10; // The ICacheKeySerializer writes 2 VInts to record array lengths, which can be 1-5 bytes each + // Size consumed by value + totalSize += value.length(); + return totalSize; + }; + } + + static class MockRemovalListener implements RemovalListener, V> { CounterMetric evictionMetric = new CounterMetric(); @Override - public void onRemoval(RemovalNotification notification) { + public void onRemoval(RemovalNotification, V> notification) { evictionMetric.inc(); } } diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3BlobContainer.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3BlobContainer.java index 1f23a09a047f2..14829a066ca3a 100644 --- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3BlobContainer.java +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3BlobContainer.java @@ -62,6 +62,7 @@ import software.amazon.awssdk.services.s3.model.UploadPartRequest; import software.amazon.awssdk.services.s3.model.UploadPartResponse; import software.amazon.awssdk.services.s3.paginators.ListObjectsV2Iterable; +import software.amazon.awssdk.utils.CollectionUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -77,6 +78,7 @@ import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.BlobStoreException; import org.opensearch.common.blobstore.DeleteResult; +import org.opensearch.common.blobstore.FetchBlobResult; import org.opensearch.common.blobstore.stream.read.ReadContext; import org.opensearch.common.blobstore.stream.write.WriteContext; import org.opensearch.common.blobstore.stream.write.WritePriority; @@ -138,6 +140,13 @@ public boolean blobExists(String blobName) { } } + @ExperimentalApi + @Override + public FetchBlobResult readBlobWithMetadata(String blobName) throws IOException { + S3RetryingInputStream s3RetryingInputStream = new S3RetryingInputStream(blobStore, buildKey(blobName)); + return new FetchBlobResult(s3RetryingInputStream, s3RetryingInputStream.getMetadata()); + } + @Override public InputStream readBlob(String blobName) throws IOException { return new S3RetryingInputStream(blobStore, buildKey(blobName)); @@ -169,12 +178,27 @@ public long readBlobPreferredLength() { */ @Override public void writeBlob(String blobName, InputStream inputStream, long blobSize, boolean failIfAlreadyExists) throws IOException { + writeBlobWithMetadata(blobName, inputStream, blobSize, failIfAlreadyExists, null); + } + + /** + * Write blob with its object metadata. + */ + @ExperimentalApi + @Override + public void writeBlobWithMetadata( + String blobName, + InputStream inputStream, + long blobSize, + boolean failIfAlreadyExists, + @Nullable Map metadata + ) throws IOException { assert inputStream.markSupported() : "No mark support on inputStream breaks the S3 SDK's ability to retry requests"; SocketAccess.doPrivilegedIOException(() -> { if (blobSize <= getLargeBlobThresholdInBytes()) { - executeSingleUpload(blobStore, buildKey(blobName), inputStream, blobSize); + executeSingleUpload(blobStore, buildKey(blobName), inputStream, blobSize, metadata); } else { - executeMultipartUpload(blobStore, buildKey(blobName), inputStream, blobSize); + executeMultipartUpload(blobStore, buildKey(blobName), inputStream, blobSize, metadata); } return null; }); @@ -190,7 +214,8 @@ public void asyncBlobUpload(WriteContext writeContext, ActionListener comp writeContext.getUploadFinalizer(), writeContext.doRemoteDataIntegrityCheck(), writeContext.getExpectedChecksum(), - blobStore.isUploadRetryEnabled() + blobStore.isUploadRetryEnabled(), + writeContext.getMetadata() ); try { if (uploadRequest.getContentLength() > ByteSizeUnit.GB.toBytes(10) && blobStore.isRedirectLargeUploads()) { @@ -203,7 +228,8 @@ public void asyncBlobUpload(WriteContext writeContext, ActionListener comp blobStore, uploadRequest.getKey(), inputStream.getInputStream(), - uploadRequest.getContentLength() + uploadRequest.getContentLength(), + uploadRequest.getMetadata() ); completionListener.onResponse(null); } catch (Exception ex) { @@ -542,8 +568,13 @@ private String buildKey(String blobName) { /** * Uploads a blob using a single upload request */ - void executeSingleUpload(final S3BlobStore blobStore, final String blobName, final InputStream input, final long blobSize) - throws IOException { + void executeSingleUpload( + final S3BlobStore blobStore, + final String blobName, + final InputStream input, + final long blobSize, + final Map metadata + ) throws IOException { // Extra safety checks if (blobSize > MAX_FILE_SIZE.getBytes()) { @@ -560,6 +591,10 @@ void executeSingleUpload(final S3BlobStore blobStore, final String blobName, fin .storageClass(blobStore.getStorageClass()) .acl(blobStore.getCannedACL()) .overrideConfiguration(o -> o.addMetricPublisher(blobStore.getStatsMetricPublisher().putObjectMetricPublisher)); + + if (CollectionUtils.isNotEmpty(metadata)) { + putObjectRequestBuilder = putObjectRequestBuilder.metadata(metadata); + } if (blobStore.serverSideEncryption()) { putObjectRequestBuilder.serverSideEncryption(ServerSideEncryption.AES256); } @@ -583,8 +618,13 @@ void executeSingleUpload(final S3BlobStore blobStore, final String blobName, fin /** * Uploads a blob using multipart upload requests. */ - void executeMultipartUpload(final S3BlobStore blobStore, final String blobName, final InputStream input, final long blobSize) - throws IOException { + void executeMultipartUpload( + final S3BlobStore blobStore, + final String blobName, + final InputStream input, + final long blobSize, + final Map metadata + ) throws IOException { ensureMultiPartUploadSize(blobSize); final long partSize = blobStore.bufferSizeInBytes(); @@ -609,6 +649,10 @@ void executeMultipartUpload(final S3BlobStore blobStore, final String blobName, .acl(blobStore.getCannedACL()) .overrideConfiguration(o -> o.addMetricPublisher(blobStore.getStatsMetricPublisher().multipartUploadMetricCollector)); + if (CollectionUtils.isNotEmpty(metadata)) { + createMultipartUploadRequestBuilder.metadata(metadata); + } + if (blobStore.serverSideEncryption()) { createMultipartUploadRequestBuilder.serverSideEncryption(ServerSideEncryption.AES256); } diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3RetryingInputStream.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3RetryingInputStream.java index d7e47e0ab1bcc..2eb63178b19e0 100644 --- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3RetryingInputStream.java +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3RetryingInputStream.java @@ -48,6 +48,7 @@ import java.nio.file.NoSuchFileException; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; /** @@ -77,6 +78,7 @@ class S3RetryingInputStream extends InputStream { private long currentOffset; private boolean closed; private boolean eof; + private Map metadata; S3RetryingInputStream(S3BlobStore blobStore, String blobKey) throws IOException { this(blobStore, blobKey, 0, Long.MAX_VALUE - 1); @@ -122,6 +124,7 @@ private void openStream() throws IOException { getObjectResponseInputStream.response().contentLength() ); this.currentStream = getObjectResponseInputStream; + this.metadata = getObjectResponseInputStream.response().metadata(); this.isStreamAborted.set(false); } catch (final SdkException e) { if (e instanceof S3Exception) { @@ -265,4 +268,8 @@ boolean isEof() { boolean isAborted() { return isStreamAborted.get(); } + + Map getMetadata() { + return this.metadata; + } } diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/AsyncTransferManager.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/AsyncTransferManager.java index 2259780c95276..80538059d17b8 100644 --- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/AsyncTransferManager.java +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/AsyncTransferManager.java @@ -22,6 +22,7 @@ import software.amazon.awssdk.services.s3.model.DeleteObjectRequest; import software.amazon.awssdk.services.s3.model.PutObjectRequest; import software.amazon.awssdk.services.s3.model.S3Exception; +import software.amazon.awssdk.utils.CollectionUtils; import software.amazon.awssdk.utils.CompletableFutureUtils; import org.apache.logging.log4j.LogManager; @@ -131,6 +132,10 @@ private void uploadInParts( .bucket(uploadRequest.getBucket()) .key(uploadRequest.getKey()) .overrideConfiguration(o -> o.addMetricPublisher(statsMetricPublisher.multipartUploadMetricCollector)); + + if (CollectionUtils.isNotEmpty(uploadRequest.getMetadata())) { + createMultipartUploadRequestBuilder.metadata(uploadRequest.getMetadata()); + } if (uploadRequest.doRemoteDataIntegrityCheck()) { createMultipartUploadRequestBuilder.checksumAlgorithm(ChecksumAlgorithm.CRC32); } @@ -327,6 +332,10 @@ private void uploadInOneChunk( .key(uploadRequest.getKey()) .contentLength(uploadRequest.getContentLength()) .overrideConfiguration(o -> o.addMetricPublisher(statsMetricPublisher.putObjectMetricPublisher)); + + if (CollectionUtils.isNotEmpty(uploadRequest.getMetadata())) { + putObjectRequestBuilder.metadata(uploadRequest.getMetadata()); + } if (uploadRequest.doRemoteDataIntegrityCheck()) { putObjectRequestBuilder.checksumAlgorithm(ChecksumAlgorithm.CRC32); putObjectRequestBuilder.checksumCRC32(base64StringFromLong(uploadRequest.getExpectedChecksum())); diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/UploadRequest.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/UploadRequest.java index a5304dc4a97d6..b944a72225d36 100644 --- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/UploadRequest.java +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/async/UploadRequest.java @@ -9,9 +9,11 @@ package org.opensearch.repositories.s3.async; import org.opensearch.common.CheckedConsumer; +import org.opensearch.common.Nullable; import org.opensearch.common.blobstore.stream.write.WritePriority; import java.io.IOException; +import java.util.Map; /** * A model encapsulating all details for an upload to S3 @@ -24,8 +26,8 @@ public class UploadRequest { private final CheckedConsumer uploadFinalizer; private final boolean doRemoteDataIntegrityCheck; private final Long expectedChecksum; - private boolean uploadRetryEnabled; + private final Map metadata; /** * Construct a new UploadRequest object @@ -37,6 +39,7 @@ public class UploadRequest { * @param uploadFinalizer An upload finalizer to call once all parts are uploaded * @param doRemoteDataIntegrityCheck A boolean to inform vendor plugins whether remote data integrity checks need to be done * @param expectedChecksum Checksum of the file being uploaded for remote data integrity check + * @param metadata Metadata of the file being uploaded */ public UploadRequest( String bucket, @@ -46,7 +49,8 @@ public UploadRequest( CheckedConsumer uploadFinalizer, boolean doRemoteDataIntegrityCheck, Long expectedChecksum, - boolean uploadRetryEnabled + boolean uploadRetryEnabled, + @Nullable Map metadata ) { this.bucket = bucket; this.key = key; @@ -56,6 +60,7 @@ public UploadRequest( this.doRemoteDataIntegrityCheck = doRemoteDataIntegrityCheck; this.expectedChecksum = expectedChecksum; this.uploadRetryEnabled = uploadRetryEnabled; + this.metadata = metadata; } public String getBucket() { @@ -89,4 +94,11 @@ public Long getExpectedChecksum() { public boolean isUploadRetryEnabled() { return uploadRetryEnabled; } + + /** + * @return metadata of the blob to be uploaded + */ + public Map getMetadata() { + return metadata; + } } diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerMockClientTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerMockClientTests.java index 9e830c409a58b..4173f8b66387f 100644 --- a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerMockClientTests.java +++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerMockClientTests.java @@ -57,6 +57,7 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Locale; import java.util.concurrent.CompletableFuture; @@ -75,6 +76,7 @@ import static org.opensearch.repositories.s3.S3Repository.BULK_DELETE_SIZE; import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyMap; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.any; import static org.mockito.Mockito.doAnswer; @@ -659,6 +661,7 @@ private void testLargeFilesRedirectedToSlowSyncClient(boolean expectException) t .writePriority(WritePriority.HIGH) .uploadFinalizer(Assert::assertTrue) .doRemoteDataIntegrityCheck(false) + .metadata(new HashMap<>()) .build(); s3BlobContainer.asyncBlobUpload(writeContext, completionListener); @@ -668,7 +671,13 @@ private void testLargeFilesRedirectedToSlowSyncClient(boolean expectException) t } else { assertNull(exceptionRef.get()); } - verify(s3BlobContainer, times(1)).executeMultipartUpload(any(S3BlobStore.class), anyString(), any(InputStream.class), anyLong()); + verify(s3BlobContainer, times(1)).executeMultipartUpload( + any(S3BlobStore.class), + anyString(), + any(InputStream.class), + anyLong(), + anyMap() + ); if (expectException) { verify(client, times(1)).abortMultipartUpload(any(AbortMultipartUploadRequest.class)); diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerRetriesTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerRetriesTests.java index 8e25ba4d950ef..10578090da75c 100644 --- a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerRetriesTests.java +++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerRetriesTests.java @@ -79,8 +79,10 @@ import java.net.SocketTimeoutException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Locale; +import java.util.Map; import java.util.Objects; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; @@ -240,7 +242,7 @@ public InputStream readBlob(String blobName, long position, long length) throws }; } - public void testWriteBlobWithRetries() throws Exception { + public void writeBlobWithRetriesHelper(Map metadata) throws Exception { final int maxRetries = randomInt(5); final CountDown countDown = new CountDown(maxRetries + 1); @@ -280,11 +282,26 @@ public void testWriteBlobWithRetries() throws Exception { final BlobContainer blobContainer = createBlobContainer(maxRetries, null, true, null); try (InputStream stream = new ByteArrayInputStream(bytes)) { - blobContainer.writeBlob("write_blob_max_retries", stream, bytes.length, false); + if (metadata != null) { + blobContainer.writeBlobWithMetadata("write_blob_max_retries", stream, bytes.length, false, metadata); + } else { + blobContainer.writeBlob("write_blob_max_retries", stream, bytes.length, false); + } } assertThat(countDown.isCountedDown(), is(true)); } + public void testWriteBlobWithMetadataWithRetries() throws Exception { + Map metadata = new HashMap<>(); + metadata.put("key1", "value1"); + metadata.put("key2", "value2"); + writeBlobWithRetriesHelper(metadata); + } + + public void testWriteBlobWithRetries() throws Exception { + writeBlobWithRetriesHelper(null); + } + public void testWriteBlobByStreamsWithRetries() throws Exception { final int maxRetries = randomInt(5); final CountDown countDown = new CountDown(maxRetries + 1); @@ -368,7 +385,7 @@ private int calculateNumberOfParts(long contentLength, long partSize) { return (int) ((contentLength % partSize) == 0 ? contentLength / partSize : (contentLength / partSize) + 1); } - public void testWriteBlobWithReadTimeouts() { + public void writeBlobWithReadTimeoutsHelper(Map metadata) { final byte[] bytes = randomByteArrayOfLength(randomIntBetween(10, 128)); final TimeValue readTimeout = TimeValue.timeValueMillis(randomIntBetween(100, 500)); final BlobContainer blobContainer = createBlobContainer(1, readTimeout, true, null); @@ -386,7 +403,11 @@ public void testWriteBlobWithReadTimeouts() { Exception exception = expectThrows(IOException.class, () -> { try (InputStream stream = new InputStreamIndexInput(new ByteArrayIndexInput("desc", bytes), bytes.length)) { - blobContainer.writeBlob("write_blob_timeout", stream, bytes.length, false); + if (metadata != null) { + blobContainer.writeBlobWithMetadata("write_blob_timeout", stream, bytes.length, false, metadata); + } else { + blobContainer.writeBlob("write_blob_timeout", stream, bytes.length, false); + } } }); assertThat( @@ -401,7 +422,18 @@ public void testWriteBlobWithReadTimeouts() { assertThat(exception.getCause().getCause().getMessage().toLowerCase(Locale.ROOT), containsString("read timed out")); } - public void testWriteLargeBlob() throws Exception { + public void testWriteBlobWithMetadataWithReadTimeouts() throws Exception { + Map metadata = new HashMap<>(); + metadata.put("key1", "value1"); + metadata.put("key2", "value2"); + writeBlobWithReadTimeoutsHelper(metadata); + } + + public void testWriteBlobWithReadTimeouts() throws Exception { + writeBlobWithReadTimeoutsHelper(null); + } + + public void WriteLargeBlobHelper(Map metadata) throws Exception { final boolean useTimeout = rarely(); final TimeValue readTimeout = useTimeout ? TimeValue.timeValueMillis(randomIntBetween(100, 500)) : null; final ByteSizeValue bufferSize = new ByteSizeValue(5, ByteSizeUnit.MB); @@ -487,13 +519,28 @@ public void testWriteLargeBlob() throws Exception { } }); - blobContainer.writeBlob("write_large_blob", new ZeroInputStream(blobSize), blobSize, false); + if (metadata != null) { + blobContainer.writeBlobWithMetadata("write_large_blob", new ZeroInputStream(blobSize), blobSize, false, metadata); + } else { + blobContainer.writeBlob("write_large_blob", new ZeroInputStream(blobSize), blobSize, false); + } assertThat(countDownInitiate.isCountedDown(), is(true)); assertThat(countDownUploads.get(), equalTo(0)); assertThat(countDownComplete.isCountedDown(), is(true)); } + public void testWriteLargeBlobWithMetadata() throws Exception { + Map metadata = new HashMap<>(); + metadata.put("key1", "value1"); + metadata.put("key2", "value2"); + WriteLargeBlobHelper(metadata); + } + + public void testWriteLargeBlob() throws Exception { + WriteLargeBlobHelper(null); + } + /** * Asserts that an InputStream is fully consumed, or aborted, when it is closed */ diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobStoreContainerTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobStoreContainerTests.java index 2b45e9cfe2d4b..654d8a72690c4 100644 --- a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobStoreContainerTests.java +++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobStoreContainerTests.java @@ -90,6 +90,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.Comparator; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -125,7 +126,7 @@ public void testExecuteSingleUploadBlobSizeTooLarge() { final IllegalArgumentException e = expectThrows( IllegalArgumentException.class, - () -> blobContainer.executeSingleUpload(blobStore, randomAlphaOfLengthBetween(1, 10), null, blobSize) + () -> blobContainer.executeSingleUpload(blobStore, randomAlphaOfLengthBetween(1, 10), null, blobSize, null) ); assertEquals("Upload request size [" + blobSize + "] can't be larger than 5gb", e.getMessage()); } @@ -139,7 +140,13 @@ public void testExecuteSingleUploadBlobSizeLargerThanBufferSize() { final IllegalArgumentException e = expectThrows( IllegalArgumentException.class, - () -> blobContainer.executeSingleUpload(blobStore, blobName, new ByteArrayInputStream(new byte[0]), ByteSizeUnit.MB.toBytes(2)) + () -> blobContainer.executeSingleUpload( + blobStore, + blobName, + new ByteArrayInputStream(new byte[0]), + ByteSizeUnit.MB.toBytes(2), + null + ) ); assertEquals("Upload request size [2097152] can't be larger than buffer size", e.getMessage()); } @@ -430,6 +437,10 @@ public void testExecuteSingleUpload() throws IOException { final String bucketName = randomAlphaOfLengthBetween(1, 10); final String blobName = randomAlphaOfLengthBetween(1, 10); + final Map metadata = new HashMap<>(); + metadata.put("key1", "value1"); + metadata.put("key2", "value2"); + final BlobPath blobPath = new BlobPath(); if (randomBoolean()) { IntStream.of(randomIntBetween(1, 5)).forEach(value -> blobPath.add("path_" + value)); @@ -467,7 +478,7 @@ public void testExecuteSingleUpload() throws IOException { ); final ByteArrayInputStream inputStream = new ByteArrayInputStream(new byte[blobSize]); - blobContainer.executeSingleUpload(blobStore, blobName, inputStream, blobSize); + blobContainer.executeSingleUpload(blobStore, blobName, inputStream, blobSize, metadata); final PutObjectRequest request = putObjectRequestArgumentCaptor.getValue(); final RequestBody requestBody = requestBodyArgumentCaptor.getValue(); @@ -480,6 +491,7 @@ public void testExecuteSingleUpload() throws IOException { assertEquals(blobSize, request.contentLength().longValue()); assertEquals(storageClass, request.storageClass()); assertEquals(cannedAccessControlList, request.acl()); + assertEquals(metadata, request.metadata()); if (serverSideEncryption) { assertEquals(ServerSideEncryption.AES256, request.serverSideEncryption()); } @@ -492,7 +504,7 @@ public void testExecuteMultipartUploadBlobSizeTooLarge() { final IllegalArgumentException e = expectThrows( IllegalArgumentException.class, - () -> blobContainer.executeMultipartUpload(blobStore, randomAlphaOfLengthBetween(1, 10), null, blobSize) + () -> blobContainer.executeMultipartUpload(blobStore, randomAlphaOfLengthBetween(1, 10), null, blobSize, null) ); assertEquals("Multipart upload request size [" + blobSize + "] can't be larger than 5tb", e.getMessage()); } @@ -504,7 +516,7 @@ public void testExecuteMultipartUploadBlobSizeTooSmall() { final IllegalArgumentException e = expectThrows( IllegalArgumentException.class, - () -> blobContainer.executeMultipartUpload(blobStore, randomAlphaOfLengthBetween(1, 10), null, blobSize) + () -> blobContainer.executeMultipartUpload(blobStore, randomAlphaOfLengthBetween(1, 10), null, blobSize, null) ); assertEquals("Multipart upload request size [" + blobSize + "] can't be smaller than 5mb", e.getMessage()); } @@ -513,6 +525,10 @@ public void testExecuteMultipartUpload() throws IOException { final String bucketName = randomAlphaOfLengthBetween(1, 10); final String blobName = randomAlphaOfLengthBetween(1, 10); + final Map metadata = new HashMap<>(); + metadata.put("key1", "value1"); + metadata.put("key2", "value2"); + final BlobPath blobPath = new BlobPath(); if (randomBoolean()) { IntStream.of(randomIntBetween(1, 5)).forEach(value -> blobPath.add("path_" + value)); @@ -577,13 +593,15 @@ public void testExecuteMultipartUpload() throws IOException { final ByteArrayInputStream inputStream = new ByteArrayInputStream(new byte[0]); final S3BlobContainer blobContainer = new S3BlobContainer(blobPath, blobStore); - blobContainer.executeMultipartUpload(blobStore, blobName, inputStream, blobSize); + blobContainer.executeMultipartUpload(blobStore, blobName, inputStream, blobSize, metadata); final CreateMultipartUploadRequest initRequest = createMultipartUploadRequestArgumentCaptor.getValue(); assertEquals(bucketName, initRequest.bucket()); assertEquals(blobPath.buildAsString() + blobName, initRequest.key()); assertEquals(storageClass, initRequest.storageClass()); assertEquals(cannedAccessControlList, initRequest.acl()); + assertEquals(metadata, initRequest.metadata()); + if (serverSideEncryption) { assertEquals(ServerSideEncryption.AES256, initRequest.serverSideEncryption()); } @@ -686,7 +704,7 @@ public void testExecuteMultipartUploadAborted() { final IOException e = expectThrows(IOException.class, () -> { final S3BlobContainer blobContainer = new S3BlobContainer(blobPath, blobStore); - blobContainer.executeMultipartUpload(blobStore, blobName, new ByteArrayInputStream(new byte[0]), blobSize); + blobContainer.executeMultipartUpload(blobStore, blobName, new ByteArrayInputStream(new byte[0]), blobSize, null); }); assertEquals("Unable to upload object [" + blobName + "] using multipart upload", e.getMessage()); diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3RetryingInputStreamTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3RetryingInputStreamTests.java index b38d5119b4108..d884a46f3ecc5 100644 --- a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3RetryingInputStreamTests.java +++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3RetryingInputStreamTests.java @@ -43,6 +43,8 @@ import java.io.ByteArrayInputStream; import java.io.IOException; import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; import static org.hamcrest.Matchers.is; import static org.mockito.Mockito.any; @@ -61,6 +63,15 @@ public void testInputStreamFullyConsumed() throws IOException { assertThat(stream.isAborted(), is(false)); } + public void testInputStreamGetMetadata() throws IOException { + final byte[] expectedBytes = randomByteArrayOfLength(randomIntBetween(1, 512)); + + final S3RetryingInputStream stream = createInputStream(expectedBytes, 0L, (long) (Integer.MAX_VALUE - 1)); + + Map metadata = new HashMap<>(); + assertEquals(stream.getMetadata(), metadata); + } + public void testInputStreamIsAborted() throws IOException { final byte[] expectedBytes = randomByteArrayOfLength(randomIntBetween(10, 512)); final byte[] actualBytes = new byte[randomIntBetween(1, Math.max(1, expectedBytes.length - 1))]; diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/AsyncTransferManagerTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/AsyncTransferManagerTests.java index b753b847df869..04d1819bef02b 100644 --- a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/AsyncTransferManagerTests.java +++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/async/AsyncTransferManagerTests.java @@ -40,7 +40,9 @@ import java.io.IOException; import java.io.InputStream; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; @@ -78,11 +80,14 @@ public void testOneChunkUpload() { ); AtomicReference streamRef = new AtomicReference<>(); + Map metadata = new HashMap<>(); + metadata.put("key1", "value1"); + metadata.put("key2", "value2"); CompletableFuture resultFuture = asyncTransferManager.uploadObject( s3AsyncClient, new UploadRequest("bucket", "key", ByteSizeUnit.MB.toBytes(1), WritePriority.HIGH, uploadSuccess -> { // do nothing - }, false, null, true), + }, false, null, true, metadata), new StreamContext((partIdx, partSize, position) -> { streamRef.set(new ZeroInputStream(partSize)); return new InputStreamContainer(streamRef.get(), partSize, position); @@ -123,11 +128,15 @@ public void testOneChunkUploadCorruption() { deleteObjectResponseCompletableFuture.complete(DeleteObjectResponse.builder().build()); when(s3AsyncClient.deleteObject(any(DeleteObjectRequest.class))).thenReturn(deleteObjectResponseCompletableFuture); + Map metadata = new HashMap<>(); + metadata.put("key1", "value1"); + metadata.put("key2", "value2"); + CompletableFuture resultFuture = asyncTransferManager.uploadObject( s3AsyncClient, new UploadRequest("bucket", "key", ByteSizeUnit.MB.toBytes(1), WritePriority.HIGH, uploadSuccess -> { // do nothing - }, false, null, true), + }, false, null, true, metadata), new StreamContext( (partIdx, partSize, position) -> new InputStreamContainer(new ZeroInputStream(partSize), partSize, position), ByteSizeUnit.MB.toBytes(1), @@ -175,12 +184,16 @@ public void testMultipartUpload() { abortMultipartUploadResponseCompletableFuture ); + Map metadata = new HashMap<>(); + metadata.put("key1", "value1"); + metadata.put("key2", "value2"); + List streams = new ArrayList<>(); CompletableFuture resultFuture = asyncTransferManager.uploadObject( s3AsyncClient, new UploadRequest("bucket", "key", ByteSizeUnit.MB.toBytes(5), WritePriority.HIGH, uploadSuccess -> { // do nothing - }, true, 3376132981L, true), + }, true, 3376132981L, true, metadata), new StreamContext((partIdx, partSize, position) -> { InputStream stream = new ZeroInputStream(partSize); streams.add(stream); @@ -236,11 +249,15 @@ public void testMultipartUploadCorruption() { abortMultipartUploadResponseCompletableFuture ); + Map metadata = new HashMap<>(); + metadata.put("key1", "value1"); + metadata.put("key2", "value2"); + CompletableFuture resultFuture = asyncTransferManager.uploadObject( s3AsyncClient, new UploadRequest("bucket", "key", ByteSizeUnit.MB.toBytes(5), WritePriority.HIGH, uploadSuccess -> { // do nothing - }, true, 0L, true), + }, true, 0L, true, metadata), new StreamContext( (partIdx, partSize, position) -> new InputStreamContainer(new ZeroInputStream(partSize), partSize, position), ByteSizeUnit.MB.toBytes(1), diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteStoreMigrationSettingsUpdateIT.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteStoreMigrationSettingsUpdateIT.java new file mode 100644 index 0000000000000..5ae2a976f4066 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteStoreMigrationSettingsUpdateIT.java @@ -0,0 +1,187 @@ +/* + * 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.remotemigration; + +import org.opensearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; +import org.opensearch.client.Client; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.rest.RestStatus; +import org.opensearch.index.IndexSettings; +import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.snapshots.SnapshotInfo; +import org.opensearch.snapshots.SnapshotState; +import org.opensearch.test.OpenSearchIntegTestCase; + +import java.nio.file.Path; +import java.util.Optional; + +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_SEGMENT_STORE_REPOSITORY; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_ENABLED; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE; +import static org.opensearch.index.IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING; +import static org.opensearch.node.remotestore.RemoteStoreNodeService.CompatibilityMode.MIXED; +import static org.opensearch.node.remotestore.RemoteStoreNodeService.Direction.REMOTE_STORE; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; + +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) +public class RemoteStoreMigrationSettingsUpdateIT extends RemoteStoreMigrationShardAllocationBaseTestCase { + + private Client client; + + // remote store backed index setting tests + + public void testNewIndexIsRemoteStoreBackedForRemoteStoreDirectionAndMixedMode() { + logger.info("Initialize cluster: gives non remote cluster manager"); + initializeCluster(false); + + String indexName1 = "test_index_1"; + String indexName2 = "test_index_2"; + + logger.info("Add non-remote node"); + addRemote = false; + String nonRemoteNodeName = internalCluster().startNode(); + internalCluster().validateClusterFormed(); + assertNodeInCluster(nonRemoteNodeName); + + logger.info("Create an index"); + prepareIndexWithoutReplica(Optional.of(indexName1)); + + logger.info("Verify that non remote-backed index is created"); + assertNonRemoteStoreBackedIndex(indexName1); + + logger.info("Set mixed cluster compatibility mode and remote_store direction"); + setClusterMode(MIXED.mode); + setDirection(REMOTE_STORE.direction); + + logger.info("Add remote node"); + addRemote = true; + String remoteNodeName = internalCluster().startNode(); + internalCluster().validateClusterFormed(); + assertNodeInCluster(remoteNodeName); + + logger.info("Create another index"); + prepareIndexWithoutReplica(Optional.of(indexName2)); + + logger.info("Verify that remote backed index is created"); + assertRemoteStoreBackedIndex(indexName2); + } + + public void testNewRestoredIndexIsRemoteStoreBackedForRemoteStoreDirectionAndMixedMode() throws Exception { + logger.info("Initialize cluster: gives non remote cluster manager"); + initializeCluster(false); + + logger.info("Add remote and non-remote nodes"); + setClusterMode(MIXED.mode); + addRemote = false; + String nonRemoteNodeName = internalCluster().startNode(); + addRemote = true; + String remoteNodeName = internalCluster().startNode(); + internalCluster().validateClusterFormed(); + assertNodeInCluster(nonRemoteNodeName); + assertNodeInCluster(remoteNodeName); + + logger.info("Create a non remote-backed index"); + client.admin() + .indices() + .prepareCreate(TEST_INDEX) + .setSettings( + Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0).build() + ) + .get(); + + logger.info("Verify that non remote stored backed index is created"); + assertNonRemoteStoreBackedIndex(TEST_INDEX); + + logger.info("Create repository"); + String snapshotName = "test-snapshot"; + String snapshotRepoName = "test-restore-snapshot-repo"; + Path snapshotRepoNameAbsolutePath = randomRepoPath().toAbsolutePath(); + assertAcked( + clusterAdmin().preparePutRepository(snapshotRepoName) + .setType("fs") + .setSettings(Settings.builder().put("location", snapshotRepoNameAbsolutePath)) + ); + + logger.info("Create snapshot of non remote stored backed index"); + + SnapshotInfo snapshotInfo = client().admin() + .cluster() + .prepareCreateSnapshot(snapshotRepoName, snapshotName) + .setIndices(TEST_INDEX) + .setWaitForCompletion(true) + .get() + .getSnapshotInfo(); + + assertEquals(SnapshotState.SUCCESS, snapshotInfo.state()); + assertTrue(snapshotInfo.successfulShards() > 0); + assertEquals(0, snapshotInfo.failedShards()); + + logger.info("Restore index from snapshot under NONE direction"); + String restoredIndexName1 = TEST_INDEX + "-restored1"; + restoreSnapshot(snapshotRepoName, snapshotName, restoredIndexName1); + + logger.info("Verify that restored index is non remote-backed"); + assertNonRemoteStoreBackedIndex(restoredIndexName1); + + logger.info("Restore index from snapshot under REMOTE_STORE direction"); + setDirection(REMOTE_STORE.direction); + String restoredIndexName2 = TEST_INDEX + "-restored2"; + restoreSnapshot(snapshotRepoName, snapshotName, restoredIndexName2); + + logger.info("Verify that restored index is non remote-backed"); + assertRemoteStoreBackedIndex(restoredIndexName2); + } + + // restore indices from a snapshot + private void restoreSnapshot(String snapshotRepoName, String snapshotName, String restoredIndexName) { + RestoreSnapshotResponse restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName) + .setWaitForCompletion(false) + .setIndices(TEST_INDEX) + .setRenamePattern(TEST_INDEX) + .setRenameReplacement(restoredIndexName) + .get(); + + assertEquals(restoreSnapshotResponse.status(), RestStatus.ACCEPTED); + ensureGreen(restoredIndexName); + } + + // verify that the created index is not remote store backed + private void assertNonRemoteStoreBackedIndex(String indexName) { + Settings indexSettings = client.admin().indices().prepareGetIndex().execute().actionGet().getSettings().get(indexName); + assertEquals(ReplicationType.DOCUMENT.toString(), indexSettings.get(SETTING_REPLICATION_TYPE)); + assertNull(indexSettings.get(SETTING_REMOTE_STORE_ENABLED)); + assertNull(indexSettings.get(SETTING_REMOTE_SEGMENT_STORE_REPOSITORY)); + assertNull(indexSettings.get(SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY)); + } + + // verify that the created index is remote store backed + private void assertRemoteStoreBackedIndex(String indexName) { + Settings indexSettings = client.admin().indices().prepareGetIndex().execute().actionGet().getSettings().get(indexName); + assertEquals(ReplicationType.SEGMENT.toString(), indexSettings.get(SETTING_REPLICATION_TYPE)); + assertEquals("true", indexSettings.get(SETTING_REMOTE_STORE_ENABLED)); + assertEquals(REPOSITORY_NAME, indexSettings.get(SETTING_REMOTE_SEGMENT_STORE_REPOSITORY)); + assertEquals(REPOSITORY_2_NAME, indexSettings.get(SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY)); + assertEquals( + IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL, + INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.get(indexSettings) + ); + } + + // bootstrap a cluster + private void initializeCluster(boolean remoteClusterManager) { + addRemote = remoteClusterManager; + internalCluster().startClusterManagerOnlyNode(); + client = internalCluster().client(); + } + +} diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteStoreMigrationShardAllocationBaseTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteStoreMigrationShardAllocationBaseTestCase.java new file mode 100644 index 0000000000000..ad2302d1ab2e1 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteStoreMigrationShardAllocationBaseTestCase.java @@ -0,0 +1,101 @@ +/* + * 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.remotemigration; + +import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.IndexShardRoutingTable; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.common.settings.Settings; + +import java.util.Map; +import java.util.Optional; + +import static org.opensearch.node.remotestore.RemoteStoreNodeService.MIGRATION_DIRECTION_SETTING; +import static org.opensearch.node.remotestore.RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; + +public class RemoteStoreMigrationShardAllocationBaseTestCase extends MigrationBaseTestCase { + protected static final String TEST_INDEX = "test_index"; + protected static final String NAME = "remote_store_migration"; + + protected final ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest(); + + // set the compatibility mode of cluster [strict, mixed] + protected void setClusterMode(String mode) { + updateSettingsRequest.persistentSettings(Settings.builder().put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), mode)); + assertAcked(internalCluster().client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); + } + + // set the migration direction for cluster [remote_store, docrep, none] + public void setDirection(String direction) { + updateSettingsRequest.persistentSettings(Settings.builder().put(MIGRATION_DIRECTION_SETTING.getKey(), direction)); + assertAcked(internalCluster().client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); + } + + // verify that the given nodeName exists in cluster + protected DiscoveryNode assertNodeInCluster(String nodeName) { + Map nodes = internalCluster().client().admin().cluster().prepareState().get().getState().nodes().getNodes(); + DiscoveryNode discoveryNode = null; + for (Map.Entry entry : nodes.entrySet()) { + DiscoveryNode node = entry.getValue(); + if (node.getName().equals(nodeName)) { + discoveryNode = node; + break; + } + } + assertNotNull(discoveryNode); + return discoveryNode; + } + + // returns a comma-separated list of node names excluding `except` + protected String allNodesExcept(String except) { + StringBuilder exclude = new StringBuilder(); + DiscoveryNodes allNodes = internalCluster().client().admin().cluster().prepareState().get().getState().nodes(); + for (DiscoveryNode node : allNodes) { + if (node.getName().equals(except) == false) { + exclude.append(node.getName()).append(","); + } + } + return exclude.toString(); + } + + // create a new test index + protected void prepareIndexWithoutReplica(Optional name) { + String indexName = name.orElse(TEST_INDEX); + internalCluster().client() + .admin() + .indices() + .prepareCreate(indexName) + .setSettings( + Settings.builder() + .put("index.number_of_shards", 1) + .put("index.number_of_replicas", 0) + .put("index.routing.allocation.exclude._name", allNodesExcept(null)) + ) + .execute() + .actionGet(); + } + + protected ShardRouting getShardRouting(boolean isPrimary) { + IndexShardRoutingTable table = internalCluster().client() + .admin() + .cluster() + .prepareState() + .execute() + .actionGet() + .getState() + .getRoutingTable() + .index(TEST_INDEX) + .shard(0); + return (isPrimary ? table.primaryShard() : table.replicaShards().get(0)); + } + +} diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/ResizeIndexMigrationTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/ResizeIndexMigrationTestCase.java index 0548ce4a7955f..b57bc60c50e8c 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/ResizeIndexMigrationTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/ResizeIndexMigrationTestCase.java @@ -11,25 +11,20 @@ import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; import org.opensearch.action.admin.indices.shrink.ResizeType; import org.opensearch.action.support.ActiveShardCount; -import org.opensearch.client.Client; import org.opensearch.common.settings.Settings; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.test.OpenSearchIntegTestCase; -import java.util.List; - import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE; import static org.opensearch.node.remotestore.RemoteStoreNodeService.MIGRATION_DIRECTION_SETTING; import static org.opensearch.node.remotestore.RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; -@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0, autoManageMasterNodes = false) +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) public class ResizeIndexMigrationTestCase extends MigrationBaseTestCase { private static final String TEST_INDEX = "test_index"; private final static String REMOTE_STORE_DIRECTION = "remote_store"; private final static String DOC_REP_DIRECTION = "docrep"; - private final static String NONE_DIRECTION = "none"; - private final static String STRICT_MODE = "strict"; private final static String MIXED_MODE = "mixed"; /* @@ -37,38 +32,43 @@ public class ResizeIndexMigrationTestCase extends MigrationBaseTestCase { * and index is on DocRep node, and migration to remote store is in progress. * */ public void testFailResizeIndexWhileDocRepToRemoteStoreMigration() throws Exception { - - internalCluster().setBootstrapClusterManagerNodeIndex(0); - List cmNodes = internalCluster().startNodes(1); - Client client = internalCluster().client(cmNodes.get(0)); - ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest(); - updateSettingsRequest.persistentSettings(Settings.builder().put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), MIXED_MODE)); - assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); - - // Adding a non remote and a remote node addRemote = false; - String nonRemoteNodeName = internalCluster().startNode(); + // create a docrep cluster + internalCluster().startClusterManagerOnlyNode(); + internalCluster().validateClusterFormed(); - addRemote = true; - String remoteNodeName = internalCluster().startNode(); + // add a non-remote node + String nonRemoteNodeName = internalCluster().startDataOnlyNode(); + internalCluster().validateClusterFormed(); logger.info("-->Create index on non-remote node and SETTING_REMOTE_STORE_ENABLED is false. Resize should not happen"); Settings.Builder builder = Settings.builder().put(SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT); - client.admin() + internalCluster().client() + .admin() .indices() .prepareCreate(TEST_INDEX) .setSettings( builder.put("index.number_of_shards", 10) .put("index.number_of_replicas", 0) .put("index.routing.allocation.include._name", nonRemoteNodeName) - .put("index.routing.allocation.exclude._name", remoteNodeName) ) .setWaitForActiveShards(ActiveShardCount.ALL) .execute() .actionGet(); + // set mixed mode + ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest(); + updateSettingsRequest.persistentSettings(Settings.builder().put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), MIXED_MODE)); + assertAcked(internalCluster().client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); + + // add a remote node + addRemote = true; + String remoteNodeName = internalCluster().startDataOnlyNode(); + internalCluster().validateClusterFormed(); + + // set remote store migration direction updateSettingsRequest.persistentSettings(Settings.builder().put(MIGRATION_DIRECTION_SETTING.getKey(), REMOTE_STORE_DIRECTION)); - assertAcked(client.admin().cluster().updateSettings(updateSettingsRequest).actionGet()); + assertAcked(internalCluster().client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); ResizeType resizeType; int resizeShardsNum; @@ -90,7 +90,8 @@ public void testFailResizeIndexWhileDocRepToRemoteStoreMigration() throws Except cause = "clone_index"; } - client.admin() + internalCluster().client() + .admin() .indices() .prepareUpdateSettings(TEST_INDEX) .setSettings(Settings.builder().put("index.blocks.write", true)) @@ -106,7 +107,8 @@ public void testFailResizeIndexWhileDocRepToRemoteStoreMigration() throws Except IllegalStateException ex = expectThrows( IllegalStateException.class, - () -> client().admin() + () -> internalCluster().client() + .admin() .indices() .prepareResizeIndex(TEST_INDEX, "first_split") .setResizeType(resizeType) @@ -124,38 +126,43 @@ public void testFailResizeIndexWhileDocRepToRemoteStoreMigration() throws Except * and index is on Remote Store node, and migration to DocRep node is in progress. * */ public void testFailResizeIndexWhileRemoteStoreToDocRepMigration() throws Exception { - + // creates a remote cluster addRemote = true; - internalCluster().setBootstrapClusterManagerNodeIndex(0); - List cmNodes = internalCluster().startNodes(1); - Client client = internalCluster().client(cmNodes.get(0)); - ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest(); - updateSettingsRequest.persistentSettings(Settings.builder().put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), MIXED_MODE)); - assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); + internalCluster().startClusterManagerOnlyNode(); + internalCluster().validateClusterFormed(); - // Adding a non remote and a remote node - String remoteNodeName = internalCluster().startNode(); - - addRemote = false; - String nonRemoteNodeName = internalCluster().startNode(); + // add a remote node + String remoteNodeName = internalCluster().startDataOnlyNode(); + internalCluster().validateClusterFormed(); - logger.info("-->Create index on remote node and SETTING_REMOTE_STORE_ENABLED is true. Resize should not happen"); + logger.info("--> Create index on remote node and SETTING_REMOTE_STORE_ENABLED is true. Resize should not happen"); Settings.Builder builder = Settings.builder().put(SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT); - client.admin() + internalCluster().client() + .admin() .indices() .prepareCreate(TEST_INDEX) .setSettings( builder.put("index.number_of_shards", 10) .put("index.number_of_replicas", 0) .put("index.routing.allocation.include._name", remoteNodeName) - .put("index.routing.allocation.exclude._name", nonRemoteNodeName) ) .setWaitForActiveShards(ActiveShardCount.ALL) .execute() .actionGet(); + // set mixed mode + ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest(); + updateSettingsRequest.persistentSettings(Settings.builder().put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), MIXED_MODE)); + assertAcked(internalCluster().client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); + + // add a non-remote node + addRemote = false; + String nonRemoteNodeName = internalCluster().startDataOnlyNode(); + internalCluster().validateClusterFormed(); + + // set docrep migration direction updateSettingsRequest.persistentSettings(Settings.builder().put(MIGRATION_DIRECTION_SETTING.getKey(), DOC_REP_DIRECTION)); - assertAcked(client.admin().cluster().updateSettings(updateSettingsRequest).actionGet()); + assertAcked(internalCluster().client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); ResizeType resizeType; int resizeShardsNum; @@ -177,7 +184,8 @@ public void testFailResizeIndexWhileRemoteStoreToDocRepMigration() throws Except cause = "clone_index"; } - client.admin() + internalCluster().client() + .admin() .indices() .prepareUpdateSettings(TEST_INDEX) .setSettings(Settings.builder().put("index.blocks.write", true)) @@ -193,7 +201,8 @@ public void testFailResizeIndexWhileRemoteStoreToDocRepMigration() throws Except IllegalStateException ex = expectThrows( IllegalStateException.class, - () -> client().admin() + () -> internalCluster().client() + .admin() .indices() .prepareResizeIndex(TEST_INDEX, "first_split") .setResizeType(resizeType) diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java index a180876685f9c..d55ec3362b01f 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java @@ -52,6 +52,7 @@ import org.opensearch.cluster.block.ClusterBlock; import org.opensearch.cluster.block.ClusterBlockLevel; import org.opensearch.cluster.block.ClusterBlocks; +import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.cluster.routing.IndexRoutingTable; import org.opensearch.cluster.routing.RoutingTable; @@ -100,8 +101,8 @@ import org.opensearch.indices.ShardLimitValidator; import org.opensearch.indices.SystemIndices; import org.opensearch.indices.replication.common.ReplicationType; -import org.opensearch.node.Node; import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; +import org.opensearch.node.remotestore.RemoteStoreNodeService; import org.opensearch.threadpool.ThreadPool; import java.io.IOException; @@ -144,6 +145,8 @@ import static org.opensearch.index.IndexModule.INDEX_STORE_TYPE_SETTING; import static org.opensearch.indices.IndicesService.CLUSTER_REPLICATION_TYPE_SETTING; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemoteDataAttributePresent; +import static org.opensearch.node.remotestore.RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING; +import static org.opensearch.node.remotestore.RemoteStoreNodeService.isMigratingToRemoteStore; /** * Service responsible for submitting create index requests @@ -944,8 +947,8 @@ static Settings aggregateIndexSettings( indexSettingsBuilder.put(IndexMetadata.SETTING_INDEX_PROVIDED_NAME, request.getProvidedName()); indexSettingsBuilder.put(SETTING_INDEX_UUID, UUIDs.randomBase64UUID()); - updateReplicationStrategy(indexSettingsBuilder, request.settings(), settings, combinedTemplateSettings); - updateRemoteStoreSettings(indexSettingsBuilder, settings); + updateReplicationStrategy(indexSettingsBuilder, request.settings(), settings, combinedTemplateSettings, clusterSettings); + updateRemoteStoreSettings(indexSettingsBuilder, currentState, clusterSettings, settings, request.index()); if (sourceMetadata != null) { assert request.resizeType() != null; @@ -996,26 +999,31 @@ static Settings aggregateIndexSettings( public static void updateReplicationStrategy( Settings.Builder settingsBuilder, Settings requestSettings, - Settings clusterSettings, - Settings combinedTemplateSettings + Settings nodeSettings, + Settings combinedTemplateSettings, + ClusterSettings clusterSettings ) { // The replication setting is applied in the following order: - // 1. Explicit index creation request parameter - // 2. Template property for replication type - // 3. Defaults to segment if remote store attributes on the cluster - // 4. Default cluster level setting + // 1. Strictly SEGMENT if cluster is undergoing remote store migration + // 2. Explicit index creation request parameter + // 3. Template property for replication type + // 4. Replication type according to cluster level settings + // 5. Defaults to segment if remote store attributes on the cluster + // 6. Default cluster level setting final ReplicationType indexReplicationType; - if (INDEX_REPLICATION_TYPE_SETTING.exists(requestSettings)) { + if (isMigratingToRemoteStore(clusterSettings)) { + indexReplicationType = ReplicationType.SEGMENT; + } else if (INDEX_REPLICATION_TYPE_SETTING.exists(requestSettings)) { indexReplicationType = INDEX_REPLICATION_TYPE_SETTING.get(requestSettings); } else if (combinedTemplateSettings != null && INDEX_REPLICATION_TYPE_SETTING.exists(combinedTemplateSettings)) { indexReplicationType = INDEX_REPLICATION_TYPE_SETTING.get(combinedTemplateSettings); - } else if (CLUSTER_REPLICATION_TYPE_SETTING.exists(clusterSettings)) { - indexReplicationType = CLUSTER_REPLICATION_TYPE_SETTING.get(clusterSettings); - } else if (isRemoteDataAttributePresent(clusterSettings)) { + } else if (CLUSTER_REPLICATION_TYPE_SETTING.exists(nodeSettings)) { + indexReplicationType = CLUSTER_REPLICATION_TYPE_SETTING.get(nodeSettings); + } else if (isRemoteDataAttributePresent(nodeSettings)) { indexReplicationType = ReplicationType.SEGMENT; } else { - indexReplicationType = CLUSTER_REPLICATION_TYPE_SETTING.getDefault(clusterSettings); + indexReplicationType = CLUSTER_REPLICATION_TYPE_SETTING.getDefault(nodeSettings); } settingsBuilder.put(SETTING_REPLICATION_TYPE, indexReplicationType); } @@ -1023,23 +1031,49 @@ public static void updateReplicationStrategy( /** * Updates index settings to enable remote store by default based on node attributes * @param settingsBuilder index settings builder to be updated with relevant settings - * @param nodeSettings node settings + * @param clusterState state of cluster + * @param clusterSettings cluster level settings + * @param nodeSettings node level settings + * @param indexName name of index */ - public static void updateRemoteStoreSettings(Settings.Builder settingsBuilder, Settings nodeSettings) { - if (RemoteStoreNodeAttribute.isRemoteStoreAttributePresent(nodeSettings)) { - settingsBuilder.put(SETTING_REMOTE_STORE_ENABLED, true) - .put( - SETTING_REMOTE_SEGMENT_STORE_REPOSITORY, - nodeSettings.get( - Node.NODE_ATTRIBUTES.getKey() + RemoteStoreNodeAttribute.REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY - ) - ) - .put( - SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, - nodeSettings.get( - Node.NODE_ATTRIBUTES.getKey() + RemoteStoreNodeAttribute.REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY - ) - ); + public static void updateRemoteStoreSettings( + Settings.Builder settingsBuilder, + ClusterState clusterState, + ClusterSettings clusterSettings, + Settings nodeSettings, + String indexName + ) { + if ((isRemoteDataAttributePresent(nodeSettings) + && clusterSettings.get(REMOTE_STORE_COMPATIBILITY_MODE_SETTING).equals(RemoteStoreNodeService.CompatibilityMode.STRICT)) + || isMigratingToRemoteStore(clusterSettings)) { + String segmentRepo, translogRepo; + + Optional remoteNode = clusterState.nodes() + .getNodes() + .values() + .stream() + .filter(DiscoveryNode::isRemoteStoreNode) + .findFirst(); + + if (remoteNode.isPresent()) { + translogRepo = remoteNode.get() + .getAttributes() + .get(RemoteStoreNodeAttribute.REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY); + segmentRepo = remoteNode.get() + .getAttributes() + .get(RemoteStoreNodeAttribute.REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY); + if (segmentRepo != null && translogRepo != null) { + settingsBuilder.put(SETTING_REMOTE_STORE_ENABLED, true) + .put(SETTING_REMOTE_SEGMENT_STORE_REPOSITORY, segmentRepo) + .put(SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, translogRepo); + } else { + ValidationException validationException = new ValidationException(); + validationException.addValidationErrors( + Collections.singletonList("Cluster is migrating to remote store but no remote node found, failing index creation") + ); + throw new IndexCreationException(indexName, validationException); + } + } } } diff --git a/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java b/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java index e16e75de7f27d..4f5f8d4b1ef5f 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java +++ b/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java @@ -32,6 +32,7 @@ package org.opensearch.common.blobstore; +import org.opensearch.common.Nullable; import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.core.action.ActionListener; @@ -79,16 +80,16 @@ public interface BlobContainer { InputStream readBlob(String blobName) throws IOException; /** - * Creates a new {@link BlobDownloadResponse} for the given blob name. + * Creates a new {@link FetchBlobResult} for the given blob name. * * @param blobName * The name of the blob to get an {@link InputStream} for. - * @return The {@link BlobDownloadResponse} of the blob. + * @return The {@link FetchBlobResult} of the blob. * @throws NoSuchFileException if the blob does not exist * @throws IOException if the blob can not be read. */ @ExperimentalApi - default BlobDownloadResponse readBlobWithMetadata(String blobName) throws IOException { + default FetchBlobResult readBlobWithMetadata(String blobName) throws IOException { throw new UnsupportedOperationException("readBlobWithMetadata is not implemented yet"); }; @@ -166,9 +167,9 @@ default long readBlobPreferredLength() { default void writeBlobWithMetadata( String blobName, InputStream inputStream, - Map metadata, long blobSize, - boolean failIfAlreadyExists + boolean failIfAlreadyExists, + @Nullable Map metadata ) throws IOException { throw new UnsupportedOperationException("writeBlobWithMetadata is not implemented yet"); }; @@ -219,7 +220,7 @@ default void writeBlobWithMetadata( default void writeBlobAtomicWithMetadata( String blobName, InputStream inputStream, - Map metadata, + @Nullable Map metadata, long blobSize, boolean failIfAlreadyExists ) throws IOException { diff --git a/server/src/main/java/org/opensearch/common/blobstore/BlobDownloadResponse.java b/server/src/main/java/org/opensearch/common/blobstore/FetchBlobResult.java similarity index 82% rename from server/src/main/java/org/opensearch/common/blobstore/BlobDownloadResponse.java rename to server/src/main/java/org/opensearch/common/blobstore/FetchBlobResult.java index 97f3e4a16a76c..55aca771b586c 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/BlobDownloadResponse.java +++ b/server/src/main/java/org/opensearch/common/blobstore/FetchBlobResult.java @@ -8,6 +8,8 @@ package org.opensearch.common.blobstore; +import org.opensearch.common.annotation.ExperimentalApi; + import java.io.InputStream; import java.util.Map; @@ -17,7 +19,8 @@ * * @opensearch.experimental */ -public class BlobDownloadResponse { +@ExperimentalApi +public class FetchBlobResult { /** * Downloaded blob InputStream @@ -37,7 +40,7 @@ public Map getMetadata() { return metadata; } - public BlobDownloadResponse(InputStream inputStream, Map metadata) { + public FetchBlobResult(InputStream inputStream, Map metadata) { this.inputStream = inputStream; this.metadata = metadata; } diff --git a/server/src/main/java/org/opensearch/common/blobstore/stream/write/WriteContext.java b/server/src/main/java/org/opensearch/common/blobstore/stream/write/WriteContext.java index d14800e82e495..9a7fc24f7e484 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/stream/write/WriteContext.java +++ b/server/src/main/java/org/opensearch/common/blobstore/stream/write/WriteContext.java @@ -52,7 +52,7 @@ private WriteContext( CheckedConsumer uploadFinalizer, boolean doRemoteDataIntegrityCheck, @Nullable Long expectedChecksum, - Map metadata + @Nullable Map metadata ) { this.fileName = fileName; this.streamContextSupplier = streamContextSupplier; 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 d8aa4e93735e6..6d346de25cadf 100644 --- a/server/src/main/java/org/opensearch/common/cache/Cache.java +++ b/server/src/main/java/org/opensearch/common/cache/Cache.java @@ -896,6 +896,10 @@ private void relinkAtHead(Entry entry) { } } + public ToLongBiFunction getWeigher() { + return weigher; + } + private CacheSegment getCacheSegment(K key) { return segments[key.hashCode() & 0xff]; } diff --git a/server/src/main/java/org/opensearch/common/cache/ICache.java b/server/src/main/java/org/opensearch/common/cache/ICache.java index f7be46a852631..8d8964abf0829 100644 --- a/server/src/main/java/org/opensearch/common/cache/ICache.java +++ b/server/src/main/java/org/opensearch/common/cache/ICache.java @@ -9,6 +9,7 @@ package org.opensearch.common.cache; import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.common.cache.stats.ImmutableCacheStatsHolder; import org.opensearch.common.cache.store.config.CacheConfig; import java.io.Closeable; @@ -23,22 +24,29 @@ */ @ExperimentalApi public interface ICache extends Closeable { - V get(K key); + V get(ICacheKey key); - void put(K key, V value); + void put(ICacheKey key, V value); - V computeIfAbsent(K key, LoadAwareCacheLoader loader) throws Exception; + V computeIfAbsent(ICacheKey key, LoadAwareCacheLoader, V> loader) throws Exception; - void invalidate(K key); + /** + * Invalidates the key. If a dimension in the key has dropStatsOnInvalidation set to true, the cache also completely + * resets stats for that dimension value. It's the caller's responsibility to make sure all keys with that dimension value are + * actually invalidated. + */ + void invalidate(ICacheKey key); void invalidateAll(); - Iterable keys(); + Iterable> keys(); long count(); void refresh(); + ImmutableCacheStatsHolder stats(); + /** * Factory to create objects. */ diff --git a/server/src/main/java/org/opensearch/common/cache/ICacheKey.java b/server/src/main/java/org/opensearch/common/cache/ICacheKey.java new file mode 100644 index 0000000000000..4d93aab933751 --- /dev/null +++ b/server/src/main/java/org/opensearch/common/cache/ICacheKey.java @@ -0,0 +1,96 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.common.cache; + +import org.opensearch.common.annotation.ExperimentalApi; + +import java.util.List; + +/** + * A key wrapper used for ICache implementations, which carries dimensions with it. + * @param the type of the underlying key. K must implement equals(), or else ICacheKey.equals() + * won't work properly and cache behavior may be incorrect! + * + * @opensearch.experimental + */ +@ExperimentalApi +public class ICacheKey { + public final K key; // K must implement equals() + public final List dimensions; // Dimension values. The dimension names are implied. + /** + * If this key is invalidated and dropDimensions is true, the ICache implementation will also drop all stats, + * including hits/misses/evictions, with this combination of dimension values. + */ + private boolean dropStatsForDimensions; + + /** + * Constructor to use when specifying dimensions. + */ + public ICacheKey(K key, List dimensions) { + this.key = key; + this.dimensions = dimensions; + } + + /** + * Constructor to use when no dimensions are needed. + */ + public ICacheKey(K key) { + this.key = key; + this.dimensions = List.of(); + } + + @Override + public boolean equals(Object o) { + if (o == this) { + return true; + } + if (o == null) { + return false; + } + if (o.getClass() != ICacheKey.class) { + return false; + } + ICacheKey other = (ICacheKey) o; + if (!dimensions.equals(other.dimensions)) { + return false; + } + if (this.key == null && other.key == null) { + return true; + } + if (this.key == null || other.key == null) { + return false; + } + return this.key.equals(other.key); + } + + @Override + public int hashCode() { + if (key == null) { + return dimensions.hashCode(); + } + return 31 * key.hashCode() + dimensions.hashCode(); + } + + // As K might not be Accountable, directly pass in its memory usage to be added. + public long ramBytesUsed(long underlyingKeyRamBytes) { + long estimate = underlyingKeyRamBytes; + for (String dim : dimensions) { + estimate += dim.length(); + } + return estimate; + } + + public boolean getDropStatsForDimensions() { + return dropStatsForDimensions; + } + + public void setDropStatsForDimensions(boolean newValue) { + dropStatsForDimensions = newValue; + } +} diff --git a/server/src/main/java/org/opensearch/common/cache/serializer/ICacheKeySerializer.java b/server/src/main/java/org/opensearch/common/cache/serializer/ICacheKeySerializer.java new file mode 100644 index 0000000000000..7521e23091464 --- /dev/null +++ b/server/src/main/java/org/opensearch/common/cache/serializer/ICacheKeySerializer.java @@ -0,0 +1,87 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.common.cache.serializer; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.OpenSearchException; +import org.opensearch.common.cache.ICacheKey; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.common.io.stream.BytesStreamInput; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * A serializer for ICacheKey. + * @param the type of the underlying key in ICacheKey + */ +public class ICacheKeySerializer implements Serializer, byte[]> { + + public final Serializer keySerializer; + private final Logger logger = LogManager.getLogger(ICacheKeySerializer.class); + + public ICacheKeySerializer(Serializer serializer) { + this.keySerializer = serializer; + } + + @Override + public byte[] serialize(ICacheKey object) { + if (object == null || object.key == null || object.dimensions == null) { + return null; + } + byte[] serializedKey = keySerializer.serialize(object.key); + try { + BytesStreamOutput os = new BytesStreamOutput(); + // First write the number of dimensions + os.writeVInt(object.dimensions.size()); + for (String dimValue : object.dimensions) { + os.writeString(dimValue); + } + os.writeVInt(serializedKey.length); // The read byte[] fn seems to not work as expected + os.writeBytes(serializedKey); + byte[] finalBytes = BytesReference.toBytes(os.bytes()); + return finalBytes; + } catch (IOException e) { + logger.debug("Could not write ICacheKey to byte[]"); + throw new OpenSearchException(e); + } + } + + @Override + public ICacheKey deserialize(byte[] bytes) { + if (bytes == null) { + return null; + } + List dimensionList = new ArrayList<>(); + try { + BytesStreamInput is = new BytesStreamInput(bytes, 0, bytes.length); + int numDimensions = is.readVInt(); + for (int i = 0; i < numDimensions; i++) { + dimensionList.add(is.readString()); + } + + int length = is.readVInt(); + byte[] serializedKey = new byte[length]; + is.readBytes(serializedKey, 0, length); + return new ICacheKey<>(keySerializer.deserialize(serializedKey), dimensionList); + } catch (IOException e) { + logger.debug("Could not write byte[] to ICacheKey"); + throw new OpenSearchException(e); + } + } + + @Override + public boolean equals(ICacheKey object, byte[] bytes) { + return Arrays.equals(serialize(object), bytes); + } +} diff --git a/server/src/main/java/org/opensearch/common/cache/stats/CacheStats.java b/server/src/main/java/org/opensearch/common/cache/stats/CacheStats.java new file mode 100644 index 0000000000000..b0cb66b56b70d --- /dev/null +++ b/server/src/main/java/org/opensearch/common/cache/stats/CacheStats.java @@ -0,0 +1,132 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.common.cache.stats; + +import org.opensearch.common.metrics.CounterMetric; + +import java.util.Objects; + +/** + * A mutable class containing the 5 live metrics tracked by a StatsHolder object. + */ +public class CacheStats { + CounterMetric hits; + CounterMetric misses; + CounterMetric evictions; + CounterMetric sizeInBytes; + CounterMetric entries; + + public CacheStats(long hits, long misses, long evictions, long sizeInBytes, long entries) { + this.hits = new CounterMetric(); + this.hits.inc(hits); + this.misses = new CounterMetric(); + this.misses.inc(misses); + this.evictions = new CounterMetric(); + this.evictions.inc(evictions); + this.sizeInBytes = new CounterMetric(); + this.sizeInBytes.inc(sizeInBytes); + this.entries = new CounterMetric(); + this.entries.inc(entries); + } + + public CacheStats() { + this(0, 0, 0, 0, 0); + } + + private void internalAdd(long otherHits, long otherMisses, long otherEvictions, long otherSizeInBytes, long otherEntries) { + this.hits.inc(otherHits); + this.misses.inc(otherMisses); + this.evictions.inc(otherEvictions); + this.sizeInBytes.inc(otherSizeInBytes); + this.entries.inc(otherEntries); + } + + public void add(CacheStats other) { + if (other == null) { + return; + } + internalAdd(other.getHits(), other.getMisses(), other.getEvictions(), other.getSizeInBytes(), other.getEntries()); + } + + public void add(ImmutableCacheStats snapshot) { + if (snapshot == null) { + return; + } + internalAdd(snapshot.getHits(), snapshot.getMisses(), snapshot.getEvictions(), snapshot.getSizeInBytes(), snapshot.getEntries()); + } + + public void subtract(ImmutableCacheStats other) { + if (other == null) { + return; + } + internalAdd(-other.getHits(), -other.getMisses(), -other.getEvictions(), -other.getSizeInBytes(), -other.getEntries()); + } + + @Override + public int hashCode() { + return Objects.hash(hits.count(), misses.count(), evictions.count(), sizeInBytes.count(), entries.count()); + } + + public void incrementHits() { + hits.inc(); + } + + public void incrementMisses() { + misses.inc(); + } + + public void incrementEvictions() { + evictions.inc(); + } + + public void incrementSizeInBytes(long amount) { + sizeInBytes.inc(amount); + } + + public void decrementSizeInBytes(long amount) { + sizeInBytes.dec(amount); + } + + public void incrementEntries() { + entries.inc(); + } + + public void decrementEntries() { + entries.dec(); + } + + public long getHits() { + return hits.count(); + } + + public long getMisses() { + return misses.count(); + } + + public long getEvictions() { + return evictions.count(); + } + + public long getSizeInBytes() { + return sizeInBytes.count(); + } + + public long getEntries() { + return entries.count(); + } + + public void resetSizeAndEntries() { + sizeInBytes = new CounterMetric(); + entries = new CounterMetric(); + } + + public ImmutableCacheStats immutableSnapshot() { + return new ImmutableCacheStats(hits.count(), misses.count(), evictions.count(), sizeInBytes.count(), entries.count()); + } +} diff --git a/server/src/main/java/org/opensearch/common/cache/stats/CacheStatsHolder.java b/server/src/main/java/org/opensearch/common/cache/stats/CacheStatsHolder.java new file mode 100644 index 0000000000000..a8b7c27ef9e79 --- /dev/null +++ b/server/src/main/java/org/opensearch/common/cache/stats/CacheStatsHolder.java @@ -0,0 +1,295 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.common.cache.stats; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.function.Consumer; + +/** + * A class ICache implementations use to internally keep track of their stats across multiple dimensions. + * Not intended to be exposed outside the cache; for this, caches use getImmutableCacheStatsHolder() to create an immutable + * copy of the current state of the stats. + * Currently, in the IRC, the stats tracked in a CacheStatsHolder will not appear for empty shards that have had no cache + * operations done on them yet. This might be changed in the future, by exposing a method to add empty nodes to the + * tree in CacheStatsHolder in the ICache interface. + * + * @opensearch.experimental + */ +public class CacheStatsHolder { + + // The list of permitted dimensions. Should be ordered from "outermost" to "innermost", as you would like to + // aggregate them in an API response. + private final List dimensionNames; + // A tree structure based on dimension values, which stores stats values in its leaf nodes. + // Non-leaf nodes have stats matching the sum of their children. + // We use a tree structure, rather than a map with concatenated keys, to save on memory usage. If there are many leaf + // nodes that share a parent, that parent's dimension value will only be stored once, not many times. + private final Node statsRoot; + // To avoid sync problems, obtain a lock before creating or removing nodes in the stats tree. + // No lock is needed to edit stats on existing nodes. + private final Lock lock = new ReentrantLock(); + + public CacheStatsHolder(List dimensionNames) { + this.dimensionNames = Collections.unmodifiableList(dimensionNames); + this.statsRoot = new Node("", true); // The root node has the empty string as its dimension value + } + + public List getDimensionNames() { + return dimensionNames; + } + + // For all these increment functions, the dimensions list comes from the key, and contains all dimensions present in dimensionNames. + // The order has to match the order given in dimensionNames. + public void incrementHits(List dimensionValues) { + internalIncrement(dimensionValues, Node::incrementHits, true); + } + + public void incrementMisses(List dimensionValues) { + internalIncrement(dimensionValues, Node::incrementMisses, true); + } + + public void incrementEvictions(List dimensionValues) { + internalIncrement(dimensionValues, Node::incrementEvictions, true); + } + + public void incrementSizeInBytes(List dimensionValues, long amountBytes) { + internalIncrement(dimensionValues, (node) -> node.incrementSizeInBytes(amountBytes), true); + } + + // For decrements, we should not create nodes if they are absent. This protects us from erroneously decrementing values for keys + // which have been entirely deleted, for example in an async removal listener. + public void decrementSizeInBytes(List dimensionValues, long amountBytes) { + internalIncrement(dimensionValues, (node) -> node.decrementSizeInBytes(amountBytes), false); + } + + public void incrementEntries(List dimensionValues) { + internalIncrement(dimensionValues, Node::incrementEntries, true); + } + + public void decrementEntries(List dimensionValues) { + internalIncrement(dimensionValues, Node::decrementEntries, false); + } + + /** + * Reset number of entries and memory size when all keys leave the cache, but don't reset hit/miss/eviction numbers. + * This is in line with the behavior of the existing API when caches are cleared. + */ + public void reset() { + resetHelper(statsRoot); + } + + private void resetHelper(Node current) { + current.resetSizeAndEntries(); + for (Node child : current.children.values()) { + resetHelper(child); + } + } + + public long count() { + // Include this here so caches don't have to create an entire CacheStats object to run count(). + return statsRoot.getEntries(); + } + + private void internalIncrement(List dimensionValues, Consumer adder, boolean createNodesIfAbsent) { + assert dimensionValues.size() == dimensionNames.size(); + // First try to increment without creating nodes + boolean didIncrement = internalIncrementHelper(dimensionValues, statsRoot, 0, adder, false); + // If we failed to increment, because nodes had to be created, obtain the lock and run again while creating nodes if needed + if (!didIncrement && createNodesIfAbsent) { + try { + lock.lock(); + internalIncrementHelper(dimensionValues, statsRoot, 0, adder, true); + } finally { + lock.unlock(); + } + } + } + + /** + * Use the incrementer function to increment/decrement a value in the stats for a set of dimensions. + * If createNodesIfAbsent is true, and there is no stats for this set of dimensions, create one. + * Returns true if the increment was applied, false if not. + */ + private boolean internalIncrementHelper( + List dimensionValues, + Node node, + int depth, // Pass in the depth to avoid having to slice the list for each node. + Consumer adder, + boolean createNodesIfAbsent + ) { + if (depth == dimensionValues.size()) { + // This is the leaf node we are trying to reach + adder.accept(node); + return true; + } + + Node child = node.getChild(dimensionValues.get(depth)); + if (child == null) { + if (createNodesIfAbsent) { + boolean createMapInChild = depth < dimensionValues.size() - 1; + child = node.createChild(dimensionValues.get(depth), createMapInChild); + } else { + return false; + } + } + if (internalIncrementHelper(dimensionValues, child, depth + 1, adder, createNodesIfAbsent)) { + // Function returns true if the next node down was incremented + adder.accept(node); + return true; + } + return false; + } + + /** + * Produce an immutable version of these stats. + */ + public ImmutableCacheStatsHolder getImmutableCacheStatsHolder() { + return new ImmutableCacheStatsHolder(statsRoot.snapshot(), dimensionNames); + } + + public void removeDimensions(List dimensionValues) { + assert dimensionValues.size() == dimensionNames.size() : "Must specify a value for every dimension when removing from StatsHolder"; + // As we are removing nodes from the tree, obtain the lock + lock.lock(); + try { + removeDimensionsHelper(dimensionValues, statsRoot, 0); + } finally { + lock.unlock(); + } + } + + // Returns a CacheStatsCounterSnapshot object for the stats to decrement if the removal happened, null otherwise. + private ImmutableCacheStats removeDimensionsHelper(List dimensionValues, Node node, int depth) { + if (depth == dimensionValues.size()) { + // Pass up a snapshot of the original stats to avoid issues when the original is decremented by other fn invocations + return node.getImmutableStats(); + } + Node child = node.getChild(dimensionValues.get(depth)); + if (child == null) { + return null; + } + ImmutableCacheStats statsToDecrement = removeDimensionsHelper(dimensionValues, child, depth + 1); + if (statsToDecrement != null) { + // The removal took place, decrement values and remove this node from its parent if it's now empty + node.decrementBySnapshot(statsToDecrement); + if (child.getChildren().isEmpty()) { + node.children.remove(child.getDimensionValue()); + } + } + return statsToDecrement; + } + + // pkg-private for testing + Node getStatsRoot() { + return statsRoot; + } + + static class Node { + private final String dimensionValue; + // Map from dimensionValue to the DimensionNode for that dimension value. + final Map children; + // The stats for this node. If a leaf node, corresponds to the stats for this combination of dimensions; if not, + // contains the sum of its children's stats. + private CacheStats stats; + + // Used for leaf nodes to avoid allocating many unnecessary maps + private static final Map EMPTY_CHILDREN_MAP = new HashMap<>(); + + Node(String dimensionValue, boolean createChildrenMap) { + this.dimensionValue = dimensionValue; + if (createChildrenMap) { + this.children = new ConcurrentHashMap<>(); + } else { + this.children = EMPTY_CHILDREN_MAP; + } + this.stats = new CacheStats(); + } + + public String getDimensionValue() { + return dimensionValue; + } + + protected Map getChildren() { + // We can safely iterate over ConcurrentHashMap without worrying about thread issues. + return children; + } + + // Functions for modifying internal CacheStatsCounter without callers having to be aware of CacheStatsCounter + + void incrementHits() { + this.stats.incrementHits(); + } + + void incrementMisses() { + this.stats.incrementMisses(); + } + + void incrementEvictions() { + this.stats.incrementEvictions(); + } + + void incrementSizeInBytes(long amountBytes) { + this.stats.incrementSizeInBytes(amountBytes); + } + + void decrementSizeInBytes(long amountBytes) { + this.stats.decrementSizeInBytes(amountBytes); + } + + void incrementEntries() { + this.stats.incrementEntries(); + } + + void decrementEntries() { + this.stats.decrementEntries(); + } + + long getEntries() { + return this.stats.getEntries(); + } + + ImmutableCacheStats getImmutableStats() { + return this.stats.immutableSnapshot(); + } + + void decrementBySnapshot(ImmutableCacheStats snapshot) { + this.stats.subtract(snapshot); + } + + void resetSizeAndEntries() { + this.stats.resetSizeAndEntries(); + } + + Node getChild(String dimensionValue) { + return children.get(dimensionValue); + } + + Node createChild(String dimensionValue, boolean createMapInChild) { + return children.computeIfAbsent(dimensionValue, (key) -> new Node(dimensionValue, createMapInChild)); + } + + ImmutableCacheStatsHolder.Node snapshot() { + TreeMap snapshotChildren = null; + if (!children.isEmpty()) { + snapshotChildren = new TreeMap<>(); + for (Node child : children.values()) { + snapshotChildren.put(child.getDimensionValue(), child.snapshot()); + } + } + return new ImmutableCacheStatsHolder.Node(dimensionValue, snapshotChildren, getImmutableStats()); + } + } +} diff --git a/server/src/main/java/org/opensearch/common/cache/stats/ImmutableCacheStats.java b/server/src/main/java/org/opensearch/common/cache/stats/ImmutableCacheStats.java new file mode 100644 index 0000000000000..7549490fd6b74 --- /dev/null +++ b/server/src/main/java/org/opensearch/common/cache/stats/ImmutableCacheStats.java @@ -0,0 +1,103 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.common.cache.stats; + +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.common.io.stream.Writeable; + +import java.io.IOException; +import java.util.Objects; + +/** + * An immutable snapshot of CacheStats. + * + * @opensearch.experimental + */ +@ExperimentalApi +public class ImmutableCacheStats implements Writeable { // TODO: Make this extend ToXContent (in API PR) + private final long hits; + private final long misses; + private final long evictions; + private final long sizeInBytes; + private final long entries; + + public ImmutableCacheStats(long hits, long misses, long evictions, long sizeInBytes, long entries) { + this.hits = hits; + this.misses = misses; + this.evictions = evictions; + this.sizeInBytes = sizeInBytes; + this.entries = entries; + } + + public ImmutableCacheStats(StreamInput in) throws IOException { + this(in.readVLong(), in.readVLong(), in.readVLong(), in.readVLong(), in.readVLong()); + } + + public static ImmutableCacheStats addSnapshots(ImmutableCacheStats s1, ImmutableCacheStats s2) { + return new ImmutableCacheStats( + s1.hits + s2.hits, + s1.misses + s2.misses, + s1.evictions + s2.evictions, + s1.sizeInBytes + s2.sizeInBytes, + s1.entries + s2.entries + ); + } + + public long getHits() { + return hits; + } + + public long getMisses() { + return misses; + } + + public long getEvictions() { + return evictions; + } + + public long getSizeInBytes() { + return sizeInBytes; + } + + public long getEntries() { + return entries; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVLong(hits); + out.writeVLong(misses); + out.writeVLong(evictions); + out.writeVLong(sizeInBytes); + out.writeVLong(entries); + } + + @Override + public boolean equals(Object o) { + if (o == null) { + return false; + } + if (o.getClass() != ImmutableCacheStats.class) { + return false; + } + ImmutableCacheStats other = (ImmutableCacheStats) o; + return (hits == other.hits) + && (misses == other.misses) + && (evictions == other.evictions) + && (sizeInBytes == other.sizeInBytes) + && (entries == other.entries); + } + + @Override + public int hashCode() { + return Objects.hash(hits, misses, evictions, sizeInBytes, entries); + } +} diff --git a/server/src/main/java/org/opensearch/common/cache/stats/ImmutableCacheStatsHolder.java b/server/src/main/java/org/opensearch/common/cache/stats/ImmutableCacheStatsHolder.java new file mode 100644 index 0000000000000..12e325046d83b --- /dev/null +++ b/server/src/main/java/org/opensearch/common/cache/stats/ImmutableCacheStatsHolder.java @@ -0,0 +1,111 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.common.cache.stats; + +import org.opensearch.common.annotation.ExperimentalApi; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +/** + * An object storing an immutable snapshot of an entire cache's stats. Accessible outside the cache itself. + * + * @opensearch.experimental + */ + +@ExperimentalApi +public class ImmutableCacheStatsHolder { // TODO: extends Writeable, ToXContent + // An immutable snapshot of a stats within a CacheStatsHolder, containing all the stats maintained by the cache. + // Pkg-private for testing. + final Node statsRoot; + final List dimensionNames; + + public ImmutableCacheStatsHolder(Node statsRoot, List dimensionNames) { + this.statsRoot = statsRoot; + this.dimensionNames = dimensionNames; + } + + public ImmutableCacheStats getTotalStats() { + return statsRoot.getStats(); + } + + public long getTotalHits() { + return getTotalStats().getHits(); + } + + public long getTotalMisses() { + return getTotalStats().getMisses(); + } + + public long getTotalEvictions() { + return getTotalStats().getEvictions(); + } + + public long getTotalSizeInBytes() { + return getTotalStats().getSizeInBytes(); + } + + public long getTotalEntries() { + return getTotalStats().getEntries(); + } + + public ImmutableCacheStats getStatsForDimensionValues(List dimensionValues) { + Node current = statsRoot; + for (String dimensionValue : dimensionValues) { + current = current.children.get(dimensionValue); + if (current == null) { + return null; + } + } + return current.stats; + } + + // A similar class to CacheStatsHolder.Node, which uses an ordered TreeMap and holds immutable CacheStatsSnapshot as its stats. + static class Node { + private final String dimensionValue; + final Map children; // Map from dimensionValue to the Node for that dimension value + + // The stats for this node. If a leaf node, corresponds to the stats for this combination of dimensions; if not, + // contains the sum of its children's stats. + private final ImmutableCacheStats stats; + private static final Map EMPTY_CHILDREN_MAP = new HashMap<>(); + + Node(String dimensionValue, TreeMap snapshotChildren, ImmutableCacheStats stats) { + this.dimensionValue = dimensionValue; + this.stats = stats; + if (snapshotChildren == null) { + this.children = EMPTY_CHILDREN_MAP; + } else { + this.children = Collections.unmodifiableMap(snapshotChildren); + } + } + + Map getChildren() { + return children; + } + + public ImmutableCacheStats getStats() { + return stats; + } + + public String getDimensionValue() { + return dimensionValue; + } + } + + // pkg-private for testing + Node getStatsRoot() { + return statsRoot; + } + + // TODO (in API PR): Produce XContent based on aggregateByLevels() +} diff --git a/server/src/main/java/org/opensearch/common/cache/stats/package-info.java b/server/src/main/java/org/opensearch/common/cache/stats/package-info.java new file mode 100644 index 0000000000000..95b5bc8efb510 --- /dev/null +++ b/server/src/main/java/org/opensearch/common/cache/stats/package-info.java @@ -0,0 +1,9 @@ +/* + * 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. + */ +/** A package for cache stats. */ +package org.opensearch.common.cache.stats; 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 c9bec4ba47def..29e5667c9f27d 100644 --- a/server/src/main/java/org/opensearch/common/cache/store/OpenSearchOnHeapCache.java +++ b/server/src/main/java/org/opensearch/common/cache/store/OpenSearchOnHeapCache.java @@ -12,10 +12,14 @@ import org.opensearch.common.cache.CacheBuilder; import org.opensearch.common.cache.CacheType; import org.opensearch.common.cache.ICache; +import org.opensearch.common.cache.ICacheKey; import org.opensearch.common.cache.LoadAwareCacheLoader; import org.opensearch.common.cache.RemovalListener; import org.opensearch.common.cache.RemovalNotification; +import org.opensearch.common.cache.RemovalReason; import org.opensearch.common.cache.settings.CacheSettings; +import org.opensearch.common.cache.stats.CacheStatsHolder; +import org.opensearch.common.cache.stats.ImmutableCacheStatsHolder; import org.opensearch.common.cache.store.builders.ICacheBuilder; import org.opensearch.common.cache.store.config.CacheConfig; import org.opensearch.common.cache.store.settings.OpenSearchOnHeapCacheSettings; @@ -25,7 +29,10 @@ import org.opensearch.common.util.FeatureFlags; import org.opensearch.core.common.unit.ByteSizeValue; +import java.util.List; import java.util.Map; +import java.util.Objects; +import java.util.function.ToLongBiFunction; import static org.opensearch.common.cache.store.settings.OpenSearchOnHeapCacheSettings.EXPIRE_AFTER_ACCESS_KEY; import static org.opensearch.common.cache.store.settings.OpenSearchOnHeapCacheSettings.MAXIMUM_SIZE_IN_BYTES_KEY; @@ -37,13 +44,16 @@ * * @opensearch.experimental */ -public class OpenSearchOnHeapCache implements ICache, RemovalListener { +public class OpenSearchOnHeapCache implements ICache, RemovalListener, V> { - private final Cache cache; - private final RemovalListener removalListener; + private final Cache, V> cache; + private final CacheStatsHolder cacheStatsHolder; + private final RemovalListener, V> removalListener; + private final List dimensionNames; + private final ToLongBiFunction, V> weigher; public OpenSearchOnHeapCache(Builder builder) { - CacheBuilder cacheBuilder = CacheBuilder.builder() + CacheBuilder, V> cacheBuilder = CacheBuilder., V>builder() .setMaximumWeight(builder.getMaxWeightInBytes()) .weigher(builder.getWeigher()) .removalListener(this); @@ -51,44 +61,67 @@ public OpenSearchOnHeapCache(Builder builder) { cacheBuilder.setExpireAfterAccess(builder.getExpireAfterAcess()); } cache = cacheBuilder.build(); + this.dimensionNames = Objects.requireNonNull(builder.dimensionNames, "Dimension names can't be null"); + this.cacheStatsHolder = new CacheStatsHolder(dimensionNames); this.removalListener = builder.getRemovalListener(); + this.weigher = builder.getWeigher(); } @Override - public V get(K key) { + public V get(ICacheKey key) { V value = cache.get(key); + if (value != null) { + cacheStatsHolder.incrementHits(key.dimensions); + } else { + cacheStatsHolder.incrementMisses(key.dimensions); + } return value; } @Override - public void put(K key, V value) { + public void put(ICacheKey key, V value) { cache.put(key, value); + cacheStatsHolder.incrementEntries(key.dimensions); + cacheStatsHolder.incrementSizeInBytes(key.dimensions, weigher.applyAsLong(key, value)); } @Override - public V computeIfAbsent(K key, LoadAwareCacheLoader loader) throws Exception { + public V computeIfAbsent(ICacheKey key, LoadAwareCacheLoader, V> loader) throws Exception { V value = cache.computeIfAbsent(key, key1 -> loader.load(key)); + if (!loader.isLoaded()) { + cacheStatsHolder.incrementHits(key.dimensions); + } else { + cacheStatsHolder.incrementMisses(key.dimensions); + cacheStatsHolder.incrementEntries(key.dimensions); + cacheStatsHolder.incrementSizeInBytes(key.dimensions, cache.getWeigher().applyAsLong(key, value)); + } return value; } @Override - public void invalidate(K key) { - cache.invalidate(key); + public void invalidate(ICacheKey key) { + if (key.getDropStatsForDimensions()) { + cacheStatsHolder.removeDimensions(key.dimensions); + } + if (key.key != null) { + cache.invalidate(key); + } } @Override public void invalidateAll() { cache.invalidateAll(); + cacheStatsHolder.reset(); } @Override - public Iterable keys() { + public Iterable> keys() { return cache.keys(); } @Override public long count() { - return cache.count(); + return cacheStatsHolder.count(); } @Override @@ -100,8 +133,23 @@ public void refresh() { public void close() {} @Override - public void onRemoval(RemovalNotification notification) { - this.removalListener.onRemoval(notification); + public ImmutableCacheStatsHolder stats() { + return cacheStatsHolder.getImmutableCacheStatsHolder(); + } + + @Override + public void onRemoval(RemovalNotification, V> notification) { + removalListener.onRemoval(notification); + cacheStatsHolder.decrementEntries(notification.getKey().dimensions); + cacheStatsHolder.decrementSizeInBytes( + notification.getKey().dimensions, + cache.getWeigher().applyAsLong(notification.getKey(), notification.getValue()) + ); + + if (RemovalReason.EVICTED.equals(notification.getRemovalReason()) + || RemovalReason.CAPACITY.equals(notification.getRemovalReason())) { + cacheStatsHolder.incrementEvictions(notification.getKey().dimensions); + } } /** @@ -115,9 +163,8 @@ public static class OpenSearchOnHeapCacheFactory implements Factory { public ICache create(CacheConfig config, CacheType cacheType, Map cacheFactories) { Map> settingList = OpenSearchOnHeapCacheSettings.getSettingListForCacheType(cacheType); Settings settings = config.getSettings(); - ICacheBuilder builder = new Builder().setMaximumWeightInBytes( - ((ByteSizeValue) settingList.get(MAXIMUM_SIZE_IN_BYTES_KEY).get(settings)).getBytes() - ) + ICacheBuilder builder = new Builder().setDimensionNames(config.getDimensionNames()) + .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()); @@ -145,6 +192,12 @@ public String getCacheName() { * @param Type of value */ public static class Builder extends ICacheBuilder { + private List dimensionNames; + + public Builder setDimensionNames(List dimensionNames) { + this.dimensionNames = dimensionNames; + return this; + } @Override public ICache 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 7ca9080ec1aa6..ac90fcc85ffef 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 @@ -10,6 +10,7 @@ import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.cache.ICache; +import org.opensearch.common.cache.ICacheKey; import org.opensearch.common.cache.RemovalListener; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; @@ -28,13 +29,13 @@ public abstract class ICacheBuilder { private long maxWeightInBytes; - private ToLongBiFunction weigher; + private ToLongBiFunction, V> weigher; private TimeValue expireAfterAcess; private Settings settings; - private RemovalListener removalListener; + private RemovalListener, V> removalListener; public ICacheBuilder() {} @@ -43,7 +44,7 @@ public ICacheBuilder setMaximumWeightInBytes(long sizeInBytes) { return this; } - public ICacheBuilder setWeigher(ToLongBiFunction weigher) { + public ICacheBuilder setWeigher(ToLongBiFunction, V> weigher) { this.weigher = weigher; return this; } @@ -58,7 +59,7 @@ public ICacheBuilder setSettings(Settings settings) { return this; } - public ICacheBuilder setRemovalListener(RemovalListener removalListener) { + public ICacheBuilder setRemovalListener(RemovalListener, V> removalListener) { this.removalListener = removalListener; return this; } @@ -71,11 +72,11 @@ public TimeValue getExpireAfterAcess() { return expireAfterAcess; } - public ToLongBiFunction getWeigher() { + public ToLongBiFunction, V> getWeigher() { return weigher; } - public RemovalListener getRemovalListener() { + public RemovalListener, V> getRemovalListener() { return this.removalListener; } 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 4c9881e845d42..15cbdbd021d71 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 @@ -9,12 +9,15 @@ package org.opensearch.common.cache.store.config; import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.common.cache.ICacheKey; import org.opensearch.common.cache.RemovalListener; import org.opensearch.common.cache.policy.CachedQueryResult; import org.opensearch.common.cache.serializer.Serializer; +import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; +import java.util.List; import java.util.function.Function; import java.util.function.ToLongBiFunction; @@ -41,9 +44,11 @@ public class CacheConfig { /** * Represents a function that calculates the size or weight of a key-value pair. */ - private final ToLongBiFunction weigher; + private final ToLongBiFunction, V> weigher; - private final RemovalListener removalListener; + private final RemovalListener, V> removalListener; + + private final List dimensionNames; // Serializers for keys and values. Not required for all caches. private final Serializer keySerializer; @@ -61,6 +66,8 @@ public class CacheConfig { */ private final TimeValue expireAfterAccess; + private final ClusterSettings clusterSettings; + private CacheConfig(Builder builder) { this.keyType = builder.keyType; this.valueType = builder.valueType; @@ -69,9 +76,11 @@ private CacheConfig(Builder builder) { this.weigher = builder.weigher; this.keySerializer = builder.keySerializer; this.valueSerializer = builder.valueSerializer; + this.dimensionNames = builder.dimensionNames; this.cachedResultParser = builder.cachedResultParser; this.maxSizeInBytes = builder.maxSizeInBytes; this.expireAfterAccess = builder.expireAfterAccess; + this.clusterSettings = builder.clusterSettings; } public Class getKeyType() { @@ -86,7 +95,7 @@ public Settings getSettings() { return settings; } - public RemovalListener getRemovalListener() { + public RemovalListener, V> getRemovalListener() { return removalListener; } @@ -98,7 +107,7 @@ public RemovalListener getRemovalListener() { return valueSerializer; } - public ToLongBiFunction getWeigher() { + public ToLongBiFunction, V> getWeigher() { return weigher; } @@ -106,6 +115,10 @@ public Function getCachedResultParser() { return cachedResultParser; } + public List getDimensionNames() { + return dimensionNames; + } + public Long getMaxSizeInBytes() { return maxSizeInBytes; } @@ -114,6 +127,10 @@ public TimeValue getExpireAfterAccess() { return expireAfterAccess; } + public ClusterSettings getClusterSettings() { + return clusterSettings; + } + /** * Builder class to build Cache config related parameters. * @param Type of key. @@ -127,17 +144,17 @@ public static class Builder { private Class valueType; - private RemovalListener removalListener; - + private RemovalListener, V> removalListener; + private List dimensionNames; private Serializer keySerializer; private Serializer valueSerializer; - - private ToLongBiFunction weigher; + private ToLongBiFunction, V> weigher; private Function cachedResultParser; private long maxSizeInBytes; private TimeValue expireAfterAccess; + private ClusterSettings clusterSettings; public Builder() {} @@ -156,11 +173,16 @@ public Builder setValueType(Class valueType) { return this; } - public Builder setRemovalListener(RemovalListener removalListener) { + public Builder setRemovalListener(RemovalListener, V> removalListener) { this.removalListener = removalListener; return this; } + public Builder setWeigher(ToLongBiFunction, V> weigher) { + this.weigher = weigher; + return this; + } + public Builder setKeySerializer(Serializer keySerializer) { this.keySerializer = keySerializer; return this; @@ -171,8 +193,8 @@ public Builder setValueSerializer(Serializer valueSerializer) { return this; } - public Builder setWeigher(ToLongBiFunction weigher) { - this.weigher = weigher; + public Builder setDimensionNames(List dimensionNames) { + this.dimensionNames = dimensionNames; return this; } @@ -191,6 +213,11 @@ public Builder setExpireAfterAccess(TimeValue expireAfterAccess) { return this; } + public Builder setClusterSettings(ClusterSettings clusterSettings) { + this.clusterSettings = clusterSettings; + return this; + } + public CacheConfig build() { return new CacheConfig<>(this); } diff --git a/server/src/main/java/org/opensearch/common/regex/Regex.java b/server/src/main/java/org/opensearch/common/regex/Regex.java index 323b460af62df..6d8b5c3585c4c 100644 --- a/server/src/main/java/org/opensearch/common/regex/Regex.java +++ b/server/src/main/java/org/opensearch/common/regex/Regex.java @@ -35,6 +35,7 @@ import org.apache.lucene.util.automaton.Automata; import org.apache.lucene.util.automaton.Automaton; import org.apache.lucene.util.automaton.Operations; +import org.opensearch.common.Glob; import org.opensearch.core.common.Strings; import java.util.ArrayList; @@ -125,39 +126,7 @@ public static boolean simpleMatch(String pattern, String str, boolean caseInsens pattern = Strings.toLowercaseAscii(pattern); str = Strings.toLowercaseAscii(str); } - return simpleMatchWithNormalizedStrings(pattern, str); - } - - private static boolean simpleMatchWithNormalizedStrings(String pattern, String str) { - int sIdx = 0, pIdx = 0, match = 0, wildcardIdx = -1; - while (sIdx < str.length()) { - // both chars matching, incrementing both pointers - if (pIdx < pattern.length() && str.charAt(sIdx) == pattern.charAt(pIdx)) { - sIdx++; - pIdx++; - } else if (pIdx < pattern.length() && pattern.charAt(pIdx) == '*') { - // wildcard found, only incrementing pattern pointer - wildcardIdx = pIdx; - match = sIdx; - pIdx++; - } else if (wildcardIdx != -1) { - // last pattern pointer was a wildcard, incrementing string pointer - pIdx = wildcardIdx + 1; - match++; - sIdx = match; - } else { - // current pattern pointer is not a wildcard, last pattern pointer was also not a wildcard - // characters do not match - return false; - } - } - - // check for remaining characters in pattern - while (pIdx < pattern.length() && pattern.charAt(pIdx) == '*') { - pIdx++; - } - - return pIdx == pattern.length(); + return Glob.globMatch(pattern, str); } /** diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index 9d763c970c3e7..004973e50d43a 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -83,6 +83,7 @@ import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.cache.CacheType; import org.opensearch.common.cache.settings.CacheSettings; +import org.opensearch.common.cache.store.settings.OpenSearchOnHeapCacheSettings; import org.opensearch.common.logging.Loggers; import org.opensearch.common.network.NetworkModule; import org.opensearch.common.network.NetworkService; @@ -750,6 +751,14 @@ public void apply(Settings value, Settings current, Settings previous) { TelemetrySettings.METRICS_FEATURE_ENABLED_SETTING ), List.of(FeatureFlags.PLUGGABLE_CACHE), - List.of(CacheSettings.getConcreteStoreNameSettingForCacheType(CacheType.INDICES_REQUEST_CACHE)) + List.of( + CacheSettings.getConcreteStoreNameSettingForCacheType(CacheType.INDICES_REQUEST_CACHE), + OpenSearchOnHeapCacheSettings.MAXIMUM_SIZE_IN_BYTES.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ), + OpenSearchOnHeapCacheSettings.EXPIRE_AFTER_ACCESS_SETTING.getConcreteSettingForNamespace( + CacheType.INDICES_REQUEST_CACHE.getSettingPrefix() + ) + ) ); } diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java index a1d5041ff9aff..bec2d78d9af62 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java @@ -15,10 +15,10 @@ import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.blobstore.AsyncMultiStreamBlobContainer; import org.opensearch.common.blobstore.BlobContainer; -import org.opensearch.common.blobstore.BlobDownloadResponse; import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.blobstore.FetchBlobResult; import org.opensearch.common.blobstore.stream.write.WriteContext; import org.opensearch.common.blobstore.stream.write.WritePriority; import org.opensearch.common.blobstore.transfer.RemoteTransferContainer; @@ -168,7 +168,7 @@ public InputStream downloadBlob(Iterable path, String fileName) throws I @Override @ExperimentalApi - public BlobDownloadResponse downloadBlobWithMetadata(Iterable path, String fileName) throws IOException { + public FetchBlobResult downloadBlobWithMetadata(Iterable path, String fileName) throws IOException { return blobStore.blobContainer((BlobPath) path).readBlobWithMetadata(fileName); } diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/TransferService.java b/server/src/main/java/org/opensearch/index/translog/transfer/TransferService.java index 5fe8b02058383..0894ebf500ebd 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/TransferService.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/TransferService.java @@ -9,9 +9,9 @@ package org.opensearch.index.translog.transfer; import org.opensearch.common.annotation.ExperimentalApi; -import org.opensearch.common.blobstore.BlobDownloadResponse; import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.FetchBlobResult; import org.opensearch.common.blobstore.stream.write.WritePriority; import org.opensearch.core.action.ActionListener; import org.opensearch.index.translog.transfer.FileSnapshot.TransferFileSnapshot; @@ -131,11 +131,11 @@ void uploadBlobs( * * @param path the remote path from where download should be made * @param fileName the name of the file - * @return {@link BlobDownloadResponse} of the remote file + * @return {@link FetchBlobResult} of the remote file * @throws IOException the exception while reading the data */ @ExperimentalApi - BlobDownloadResponse downloadBlobWithMetadata(Iterable path, String fileName) throws IOException; + FetchBlobResult downloadBlobWithMetadata(Iterable path, String fileName) throws IOException; void listAllInSortedOrder(Iterable path, String filenamePrefix, int limit, ActionListener> listener); diff --git a/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java b/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java index 34c8d6cf5e840..eab772cda3213 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java +++ b/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java @@ -39,15 +39,18 @@ import org.apache.lucene.util.Accountable; import org.apache.lucene.util.RamUsageEstimator; import org.opensearch.OpenSearchParseException; +import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.CheckedSupplier; import org.opensearch.common.cache.CacheType; import org.opensearch.common.cache.ICache; +import org.opensearch.common.cache.ICacheKey; import org.opensearch.common.cache.LoadAwareCacheLoader; import org.opensearch.common.cache.RemovalListener; import org.opensearch.common.cache.RemovalNotification; import org.opensearch.common.cache.policy.CachedQueryResult; import org.opensearch.common.cache.serializer.BytesReferenceSerializer; import org.opensearch.common.cache.service.CacheService; +import org.opensearch.common.cache.stats.ImmutableCacheStatsHolder; import org.opensearch.common.cache.store.config.CacheConfig; import org.opensearch.common.lease.Releasable; import org.opensearch.common.lucene.index.OpenSearchDirectoryReader; @@ -73,6 +76,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; +import java.util.List; import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -99,7 +103,7 @@ * * @opensearch.internal */ -public final class IndicesRequestCache implements RemovalListener, Closeable { +public final class IndicesRequestCache implements RemovalListener, BytesReference>, Closeable { private static final Logger logger = LogManager.getLogger(IndicesRequestCache.class); @@ -145,16 +149,21 @@ public final class IndicesRequestCache implements RemovalListener> cacheEntityFunction, CacheService cacheService, - ThreadPool threadPool + ThreadPool threadPool, + ClusterService clusterService ) { this.size = INDICES_CACHE_QUERY_SIZE.get(settings); this.expire = INDICES_CACHE_QUERY_EXPIRE.exists(settings) ? INDICES_CACHE_QUERY_EXPIRE.get(settings) : null; long sizeInBytes = size.getBytes(); - ToLongBiFunction weigher = (k, v) -> k.ramBytesUsed() + v.ramBytesUsed(); + ToLongBiFunction, BytesReference> weigher = (k, v) -> k.ramBytesUsed(k.key.ramBytesUsed()) + v.ramBytesUsed(); this.cacheCleanupManager = new IndicesRequestCacheCleanupManager( threadPool, INDICES_REQUEST_CACHE_CLEAN_INTERVAL_SETTING.get(settings), @@ -169,6 +178,7 @@ public final class IndicesRequestCache implements RemovalListener { try { return CachedQueryResult.getPolicyValues(bytesReference); @@ -179,6 +189,7 @@ public final class IndicesRequestCache implements RemovalListener notification) { + public void onRemoval(RemovalNotification, BytesReference> notification) { // In case this event happens for an old shard, we can safely ignore this as we don't keep track for old // shards as part of request cache. - Key key = notification.getKey(); - cacheEntityLookup.apply(key.shardId).ifPresent(entity -> entity.onRemoval(notification)); + + // Pass a new removal notification containing Key rather than ICacheKey to the CacheEntity for backwards compatibility. + Key key = notification.getKey().key; + RemovalNotification newNotification = new RemovalNotification<>( + key, + notification.getValue(), + notification.getRemovalReason() + ); + + cacheEntityLookup.apply(key.shardId).ifPresent(entity -> entity.onRemoval(newNotification)); cacheCleanupManager.updateCleanupKeyToCountMapOnCacheEviction( new CleanupKey(cacheEntityLookup.apply(key.shardId).orElse(null), key.readerCacheKeyId) ); } + private ICacheKey getICacheKey(Key key) { + String indexDimensionValue = getIndexDimensionName(key); + String shardIdDimensionValue = getShardIdDimensionName(key); + List dimensions = List.of(indexDimensionValue, shardIdDimensionValue); + return new ICacheKey<>(key, dimensions); + } + + private String getShardIdDimensionName(Key key) { + return key.shardId.toString(); + } + + private String getIndexDimensionName(Key key) { + return key.shardId.getIndexName(); + } + BytesReference getOrCompute( IndicesService.IndexShardCacheEntity cacheEntity, CheckedSupplier loader, @@ -227,7 +261,7 @@ BytesReference getOrCompute( assert readerCacheKeyId != null; final Key key = new Key(((IndexShard) cacheEntity.getCacheIdentity()).shardId(), cacheKey, readerCacheKeyId); Loader cacheLoader = new Loader(cacheEntity, loader); - BytesReference value = cache.computeIfAbsent(key, cacheLoader); + BytesReference value = cache.computeIfAbsent(getICacheKey(key), cacheLoader); if (cacheLoader.isLoaded()) { cacheEntity.onMiss(); // see if it's the first time we see this reader, and make sure to register a cleanup key @@ -258,7 +292,7 @@ void invalidate(IndicesService.IndexShardCacheEntity cacheEntity, DirectoryReade IndexReader.CacheHelper cacheHelper = ((OpenSearchDirectoryReader) reader).getDelegatingCacheHelper(); readerCacheKeyId = ((OpenSearchDirectoryReader.DelegatingCacheHelper) cacheHelper).getDelegatingCacheKey().getId(); } - cache.invalidate(new Key(((IndexShard) cacheEntity.getCacheIdentity()).shardId(), cacheKey, readerCacheKeyId)); + cache.invalidate(getICacheKey(new Key(((IndexShard) cacheEntity.getCacheIdentity()).shardId(), cacheKey, readerCacheKeyId))); } /** @@ -266,7 +300,7 @@ void invalidate(IndicesService.IndexShardCacheEntity cacheEntity, DirectoryReade * * @opensearch.internal */ - private static class Loader implements LoadAwareCacheLoader { + private static class Loader implements LoadAwareCacheLoader, BytesReference> { private final CacheEntity entity; private final CheckedSupplier loader; @@ -282,9 +316,9 @@ public boolean isLoaded() { } @Override - public BytesReference load(Key key) throws Exception { + public BytesReference load(ICacheKey key) throws Exception { BytesReference value = loader.get(); - entity.onCached(key, value); + entity.onCached(key.key, value); loaded = true; return value; } @@ -600,7 +634,8 @@ private synchronized void cleanCache(double stalenessThreshold) { iterator.remove(); if (cleanupKey.readerCacheKeyId == null || !cleanupKey.entity.isOpen()) { // null indicates full cleanup, as does a closed shard - cleanupKeysFromClosedShards.add(((IndexShard) cleanupKey.entity.getCacheIdentity()).shardId()); + ShardId shardId = ((IndexShard) cleanupKey.entity.getCacheIdentity()).shardId(); + cleanupKeysFromClosedShards.add(shardId); } else { cleanupKeysFromOutdatedReaders.add(cleanupKey); } @@ -610,17 +645,27 @@ private synchronized void cleanCache(double stalenessThreshold) { return; } - for (Iterator iterator = cache.keys().iterator(); iterator.hasNext();) { - Key key = iterator.next(); - if (cleanupKeysFromClosedShards.contains(key.shardId)) { + Set> dimensionListsToDrop = new HashSet<>(); + + for (Iterator> iterator = cache.keys().iterator(); iterator.hasNext();) { + ICacheKey key = iterator.next(); + if (cleanupKeysFromClosedShards.contains(key.key.shardId)) { + // Since the shard is closed, the cache should drop stats for this shard. + dimensionListsToDrop.add(key.dimensions); iterator.remove(); } else { - CleanupKey cleanupKey = new CleanupKey(cacheEntityLookup.apply(key.shardId).orElse(null), key.readerCacheKeyId); + CleanupKey cleanupKey = new CleanupKey(cacheEntityLookup.apply(key.key.shardId).orElse(null), key.key.readerCacheKeyId); if (cleanupKeysFromOutdatedReaders.contains(cleanupKey)) { iterator.remove(); } } } + for (List closedDimensions : dimensionListsToDrop) { + // Invalidate a dummy key containing the dimensions we need to drop stats for + ICacheKey dummyKey = new ICacheKey<>(null, closedDimensions); + dummyKey.setDropStatsForDimensions(true); + cache.invalidate(dummyKey); + } cache.refresh(); } @@ -711,6 +756,20 @@ long count() { return cache.count(); } + /** + * Returns the current size in bytes of the cache + */ + long getSizeInBytes() { + return cache.stats().getTotalSizeInBytes(); + } + + /** + * Returns the current cache stats. Pkg-private for testing. + */ + ImmutableCacheStatsHolder stats() { + return cache.stats(); + } + int numRegisteredCloseListeners() { // for testing return registeredClosedListeners.size(); } diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index 75130a4041b77..7e2ea5a77cbfa 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -417,7 +417,7 @@ public IndicesService( return Optional.empty(); } return Optional.of(new IndexShardCacheEntity(indexService.getShard(shardId.id()))); - }), cacheService, threadPool); + }), cacheService, threadPool, clusterService); this.indicesQueryCache = new IndicesQueryCache(settings); this.mapperRegistry = mapperRegistry; this.namedWriteableRegistry = namedWriteableRegistry; diff --git a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeService.java b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeService.java index 33b182dd3cc97..94b11086ba865 100644 --- a/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeService.java +++ b/server/src/main/java/org/opensearch/node/remotestore/RemoteStoreNodeService.java @@ -14,6 +14,7 @@ import org.opensearch.cluster.metadata.RepositoriesMetadata; import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; import org.opensearch.common.util.FeatureFlags; import org.opensearch.repositories.RepositoriesService; @@ -223,4 +224,18 @@ public RepositoriesMetadata updateRepositoriesMetadata(DiscoveryNode joiningNode return existingRepositories; } } + + /** + * To check if the cluster is undergoing remote store migration + * @param clusterSettings cluster level settings + * @return + * true For REMOTE_STORE migration direction and MIXED compatibility mode, + * false otherwise + */ + public static boolean isMigratingToRemoteStore(ClusterSettings clusterSettings) { + boolean isMixedMode = clusterSettings.get(REMOTE_STORE_COMPATIBILITY_MODE_SETTING).equals(CompatibilityMode.MIXED); + boolean isRemoteStoreMigrationDirection = clusterSettings.get(MIGRATION_DIRECTION_SETTING).equals(Direction.REMOTE_STORE); + + return (isMixedMode && isRemoteStoreMigrationDirection); + } } diff --git a/server/src/main/java/org/opensearch/search/builder/SearchSourceBuilder.java b/server/src/main/java/org/opensearch/search/builder/SearchSourceBuilder.java index 77d5567b924eb..182350c22f697 100644 --- a/server/src/main/java/org/opensearch/search/builder/SearchSourceBuilder.java +++ b/server/src/main/java/org/opensearch/search/builder/SearchSourceBuilder.java @@ -289,7 +289,7 @@ public SearchSourceBuilder(StreamInput in) throws IOException { if (in.getVersion().onOrAfter(Version.V_2_13_0)) { includeNamedQueriesScore = in.readOptionalBoolean(); } - if (in.getVersion().onOrAfter(Version.V_3_0_0)) { + if (in.getVersion().onOrAfter(Version.V_2_14_0)) { if (in.readBoolean()) { derivedFieldsObject = in.readMap(); } @@ -365,7 +365,7 @@ public void writeTo(StreamOutput out) throws IOException { if (out.getVersion().onOrAfter(Version.V_2_13_0)) { out.writeOptionalBoolean(includeNamedQueriesScore); } - if (out.getVersion().onOrAfter(Version.V_3_0_0)) { + if (out.getVersion().onOrAfter(Version.V_2_14_0)) { boolean hasDerivedFieldsObject = derivedFieldsObject != null; out.writeBoolean(hasDerivedFieldsObject); if (hasDerivedFieldsObject) { diff --git a/server/src/main/java/org/opensearch/snapshots/RestoreService.java b/server/src/main/java/org/opensearch/snapshots/RestoreService.java index 980610d337a63..b79a6a88250f8 100644 --- a/server/src/main/java/org/opensearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/opensearch/snapshots/RestoreService.java @@ -92,6 +92,7 @@ import org.opensearch.indices.IndicesService; import org.opensearch.indices.ShardLimitValidator; import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; +import org.opensearch.node.remotestore.RemoteStoreNodeService; import org.opensearch.repositories.IndexId; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.Repository; @@ -678,19 +679,27 @@ private Settings getOverrideSettingsInternal() { // We will use whatever replication strategy provided by user or from snapshot metadata unless // cluster is remote store enabled or user have restricted a specific replication type in the - // cluster. + // cluster. If cluster is undergoing remote store migration, replication strategy is strictly SEGMENT type if (RemoteStoreNodeAttribute.isRemoteStoreAttributePresent(clusterService.getSettings()) == true - || clusterSettings.get(IndicesService.CLUSTER_INDEX_RESTRICT_REPLICATION_TYPE_SETTING) == true) { + || clusterSettings.get(IndicesService.CLUSTER_INDEX_RESTRICT_REPLICATION_TYPE_SETTING) == true + || RemoteStoreNodeService.isMigratingToRemoteStore(clusterSettings) == true) { MetadataCreateIndexService.updateReplicationStrategy( settingsBuilder, request.indexSettings(), clusterService.getSettings(), - null + null, + clusterSettings ); } // remote store settings needs to be overridden if the remote store feature is enabled in the // cluster where snapshot is being restored. - MetadataCreateIndexService.updateRemoteStoreSettings(settingsBuilder, clusterService.getSettings()); + MetadataCreateIndexService.updateRemoteStoreSettings( + settingsBuilder, + clusterService.state(), + clusterSettings, + clusterService.getSettings(), + request.getDescription() + ); return settingsBuilder.build(); } diff --git a/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java b/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java index fa71b77648d35..d3086de6ec89e 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java @@ -55,6 +55,7 @@ import org.opensearch.cluster.routing.allocation.decider.AwarenessAllocationDecider; import org.opensearch.cluster.routing.allocation.decider.MaxRetryAllocationDecider; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.UUIDs; import org.opensearch.common.compress.CompressedXContent; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.IndexScopedSettings; @@ -62,6 +63,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.BigArrays; +import org.opensearch.common.util.FeatureFlags; import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.core.index.Index; import org.opensearch.core.xcontent.NamedXContentRegistry; @@ -82,6 +84,7 @@ import org.opensearch.indices.SystemIndexDescriptor; import org.opensearch.indices.SystemIndices; import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.node.remotestore.RemoteStoreNodeService; import org.opensearch.snapshots.EmptySnapshotsInfoService; import org.opensearch.test.ClusterServiceUtils; import org.opensearch.test.OpenSearchTestCase; @@ -136,6 +139,7 @@ import static org.opensearch.cluster.metadata.MetadataCreateIndexService.getIndexNumberOfRoutingShards; import static org.opensearch.cluster.metadata.MetadataCreateIndexService.parseV1Mappings; import static org.opensearch.cluster.metadata.MetadataCreateIndexService.resolveAndValidateAliases; +import static org.opensearch.common.util.FeatureFlags.REMOTE_STORE_MIGRATION_EXPERIMENTAL; import static org.opensearch.index.IndexModule.INDEX_STORE_TYPE_SETTING; import static org.opensearch.index.IndexSettings.INDEX_REFRESH_INTERVAL_SETTING; import static org.opensearch.index.IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING; @@ -150,6 +154,8 @@ import static org.opensearch.node.Node.NODE_ATTRIBUTES; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY; +import static org.opensearch.node.remotestore.RemoteStoreNodeService.MIGRATION_DIRECTION_SETTING; +import static org.opensearch.node.remotestore.RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.endsWith; import static org.hamcrest.Matchers.equalTo; @@ -1349,18 +1355,16 @@ public void testClusterForceReplicationTypeInValidateIndexSettings() { } public void testRemoteStoreNoUserOverrideExceptReplicationTypeSegmentIndexSettings() { - Settings settings = Settings.builder() - .put(CLUSTER_REPLICATION_TYPE_SETTING.getKey(), ReplicationType.DOCUMENT) - .put(segmentRepositoryNameAttributeKey, "my-segment-repo-1") - .put(translogRepositoryNameAttributeKey, "my-translog-repo-1") + ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(getRemoteNode()).build()) .build(); - + Settings settings = Settings.builder().put(translogRepositoryNameAttributeKey, "my-translog-repo-1").build(); request = new CreateIndexClusterStateUpdateRequest("create index", "test", "test"); final Settings.Builder requestSettings = Settings.builder(); requestSettings.put(SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT); request.settings(requestSettings.build()); Settings indexSettings = aggregateIndexSettings( - ClusterState.EMPTY_STATE, + clusterState, request, Settings.EMPTY, null, @@ -1376,21 +1380,20 @@ public void testRemoteStoreNoUserOverrideExceptReplicationTypeSegmentIndexSettin "my-segment-repo-1", "my-translog-repo-1", ReplicationType.SEGMENT.toString(), - IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL + null ); } public void testRemoteStoreImplicitOverrideReplicationTypeToSegmentForRemoteStore() { - Settings settings = Settings.builder() - .put(segmentRepositoryNameAttributeKey, "my-segment-repo-1") - .put(translogRepositoryNameAttributeKey, "my-translog-repo-1") + ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(getRemoteNode()).build()) .build(); - + Settings settings = Settings.builder().put(translogRepositoryNameAttributeKey, "my-translog-repo-1").build(); request = new CreateIndexClusterStateUpdateRequest("create index", "test", "test"); final Settings.Builder requestSettings = Settings.builder(); request.settings(requestSettings.build()); Settings indexSettings = aggregateIndexSettings( - ClusterState.EMPTY_STATE, + clusterState, request, Settings.EMPTY, null, @@ -1406,20 +1409,18 @@ public void testRemoteStoreImplicitOverrideReplicationTypeToSegmentForRemoteStor "my-segment-repo-1", "my-translog-repo-1", ReplicationType.SEGMENT.toString(), - IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL + null ); } public void testRemoteStoreNoUserOverrideIndexSettings() { - Settings settings = Settings.builder() - .put(CLUSTER_REPLICATION_TYPE_SETTING.getKey(), ReplicationType.SEGMENT) - .put(segmentRepositoryNameAttributeKey, "my-segment-repo-1") - .put(translogRepositoryNameAttributeKey, "my-translog-repo-1") + ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT) + .nodes(DiscoveryNodes.builder().add(getRemoteNode()).build()) .build(); - + Settings settings = Settings.builder().put(translogRepositoryNameAttributeKey, "my-translog-repo-1").build(); request = new CreateIndexClusterStateUpdateRequest("create index", "test", "test"); Settings indexSettings = aggregateIndexSettings( - ClusterState.EMPTY_STATE, + clusterState, request, Settings.EMPTY, null, @@ -1435,7 +1436,7 @@ public void testRemoteStoreNoUserOverrideIndexSettings() { "my-segment-repo-1", "my-translog-repo-1", ReplicationType.SEGMENT.toString(), - IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL + null ); } @@ -1551,6 +1552,109 @@ public void testRemoteStoreOverrideTranslogRepoIndexSettings() { })); } + public void testNewIndexIsRemoteStoreBackedForRemoteStoreDirectionAndMixedMode() { + FeatureFlags.initializeFeatureFlags(Settings.builder().put(REMOTE_STORE_MIGRATION_EXPERIMENTAL, "true").build()); + + // non-remote cluster manager node + DiscoveryNode nonRemoteClusterManagerNode = new DiscoveryNode(UUIDs.base64UUID(), buildNewFakeTransportAddress(), Version.CURRENT); + + DiscoveryNodes discoveryNodes = DiscoveryNodes.builder() + .add(nonRemoteClusterManagerNode) + .localNodeId(nonRemoteClusterManagerNode.getId()) + .build(); + + ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT).nodes(discoveryNodes).build(); + + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + + request = new CreateIndexClusterStateUpdateRequest("create index", "test", "test"); + + Settings indexSettings = aggregateIndexSettings( + clusterState, + request, + Settings.EMPTY, + null, + Settings.EMPTY, + IndexScopedSettings.DEFAULT_SCOPED_SETTINGS, + randomShardLimitService(), + Collections.emptySet(), + clusterSettings + ); + verifyRemoteStoreIndexSettings(indexSettings, null, null, null, ReplicationType.DOCUMENT.toString(), null); + + // remote data node + DiscoveryNode remoteDataNode = getRemoteNode(); + + discoveryNodes = DiscoveryNodes.builder(discoveryNodes).add(remoteDataNode).localNodeId(remoteDataNode.getId()).build(); + + clusterState = ClusterState.builder(ClusterName.DEFAULT).nodes(discoveryNodes).build(); + + Settings remoteStoreMigrationSettings = Settings.builder() + .put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), RemoteStoreNodeService.CompatibilityMode.MIXED) + .put(MIGRATION_DIRECTION_SETTING.getKey(), RemoteStoreNodeService.Direction.REMOTE_STORE) + .build(); + + clusterSettings = new ClusterSettings(remoteStoreMigrationSettings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + + indexSettings = aggregateIndexSettings( + clusterState, + request, + Settings.EMPTY, + null, + Settings.EMPTY, + IndexScopedSettings.DEFAULT_SCOPED_SETTINGS, + randomShardLimitService(), + Collections.emptySet(), + clusterSettings + ); + + verifyRemoteStoreIndexSettings( + indexSettings, + "true", + "my-segment-repo-1", + "my-translog-repo-1", + ReplicationType.SEGMENT.toString(), + null + ); + + Map missingTranslogAttribute = Map.of(REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY, "my-segment-repo-1"); + + DiscoveryNodes finalDiscoveryNodes = DiscoveryNodes.builder() + .add(nonRemoteClusterManagerNode) + .add( + new DiscoveryNode( + UUIDs.base64UUID(), + buildNewFakeTransportAddress(), + missingTranslogAttribute, + Set.of(DiscoveryNodeRole.INGEST_ROLE, DiscoveryNodeRole.REMOTE_CLUSTER_CLIENT_ROLE, DiscoveryNodeRole.DATA_ROLE), + Version.CURRENT + ) + ) + .build(); + + ClusterState finalClusterState = ClusterState.builder(ClusterName.DEFAULT).nodes(finalDiscoveryNodes).build(); + ClusterSettings finalClusterSettings = clusterSettings; + + final IndexCreationException error = expectThrows(IndexCreationException.class, () -> { + aggregateIndexSettings( + finalClusterState, + request, + Settings.EMPTY, + null, + Settings.EMPTY, + IndexScopedSettings.DEFAULT_SCOPED_SETTINGS, + randomShardLimitService(), + Collections.emptySet(), + finalClusterSettings + ); + }); + + assertThat( + error.getCause().getMessage(), + containsString("Cluster is migrating to remote store but no remote node found, failing index creation") + ); + } + public void testBuildIndexMetadata() { IndexMetadata sourceIndexMetadata = IndexMetadata.builder("parent") .settings(Settings.builder().put("index.version.created", Version.CURRENT).build()) @@ -2118,7 +2222,20 @@ private void verifyRemoteStoreIndexSettings( assertEquals(isRemoteSegmentEnabled, indexSettings.get(SETTING_REMOTE_STORE_ENABLED)); assertEquals(remoteSegmentRepo, indexSettings.get(SETTING_REMOTE_SEGMENT_STORE_REPOSITORY)); assertEquals(remoteTranslogRepo, indexSettings.get(SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY)); - assertEquals(translogBufferInterval, INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.get(indexSettings)); + assertEquals(translogBufferInterval, indexSettings.get(INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey())); + } + + private DiscoveryNode getRemoteNode() { + Map attributes = new HashMap<>(); + attributes.put(REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY, "my-segment-repo-1"); + attributes.put(REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY, "my-translog-repo-1"); + return new DiscoveryNode( + UUIDs.base64UUID(), + buildNewFakeTransportAddress(), + attributes, + DiscoveryNodeRole.BUILT_IN_ROLES, + Version.CURRENT + ); } @After diff --git a/server/src/test/java/org/opensearch/cluster/routing/allocation/RemoteStoreMigrationAllocationDeciderTests.java b/server/src/test/java/org/opensearch/cluster/routing/allocation/RemoteStoreMigrationAllocationDeciderTests.java index 43363407d9249..3e130a42952e4 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/allocation/RemoteStoreMigrationAllocationDeciderTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/allocation/RemoteStoreMigrationAllocationDeciderTests.java @@ -664,7 +664,7 @@ private DiscoveryNode getNonRemoteNode() { } // get a dummy remote node - public DiscoveryNode getRemoteNode() { + private DiscoveryNode getRemoteNode() { Map attributes = new HashMap<>(); attributes.put( REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY, diff --git a/server/src/test/java/org/opensearch/common/GlobTests.java b/server/src/test/java/org/opensearch/common/GlobTests.java new file mode 100644 index 0000000000000..2bbe157be43cc --- /dev/null +++ b/server/src/test/java/org/opensearch/common/GlobTests.java @@ -0,0 +1,67 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.common; + +import org.opensearch.test.OpenSearchTestCase; + +public class GlobTests extends OpenSearchTestCase { + + public void testGlobMatchForNull() { + assertFalse(Glob.globMatch(null, "test")); + assertFalse(Glob.globMatch("test", null)); + assertFalse(Glob.globMatch(null, null)); + } + + public void testGlobMatchNoWildcard() { + assertTrue(Glob.globMatch("abcd", "abcd")); + assertFalse(Glob.globMatch("abcd", "foobar")); + } + + public void testGlobMatchSingleWildcard() { + assertTrue(Glob.globMatch("*foo", "barfoo")); + assertFalse(Glob.globMatch("*foo", "foobar")); + assertTrue(Glob.globMatch("foo*", "foobarfoo")); + assertFalse(Glob.globMatch("foo*", "barfoobar")); + assertTrue(Glob.globMatch("foo*bar", "foobarnfoosbar")); + } + + public void testGlobMatchMultipleWildcards() { + assertTrue(Glob.globMatch("*foo*", "barfoobar")); + assertFalse(Glob.globMatch("*foo*", "baroofbar")); + assertTrue(Glob.globMatch("*foo*bar", "abcdfooefghbar")); + assertFalse(Glob.globMatch("*foo*bar", "foonotbars")); + } + + public void testGlobalMatchDoubleWildcard() { + assertTrue(Glob.globMatch("**foo", "barbarfoo")); + assertFalse(Glob.globMatch("**foo", "barbarfoowoof")); + assertTrue(Glob.globMatch("**bar**", "foobarfoo")); + assertFalse(Glob.globMatch("**bar**", "foobanfoo")); + } + + public void testGlobMatchMultipleCharactersWithSingleWildcard() { + assertTrue(Glob.globMatch("a*b", "acb")); + assertTrue(Glob.globMatch("f*oo", "foo")); + assertTrue(Glob.globMatch("a*b", "aab")); + assertTrue(Glob.globMatch("a*b", "aaab")); + } + + public void testGlobMatchWildcardWithEmptyString() { + assertTrue(Glob.globMatch("*", "")); + assertTrue(Glob.globMatch("a*", "a")); + assertFalse(Glob.globMatch("a*", "")); + } + + public void testGlobMatchMultipleWildcardsWithMultipleCharacters() { + assertTrue(Glob.globMatch("a*b*c", "abc")); + assertTrue(Glob.globMatch("a*b*c", "axxxbxbc")); + assertFalse(Glob.globMatch("a*b*c", "abca")); + assertFalse(Glob.globMatch("a*b*c", "ac")); + } +} diff --git a/server/src/test/java/org/opensearch/common/cache/serializer/ICacheKeySerializerTests.java b/server/src/test/java/org/opensearch/common/cache/serializer/ICacheKeySerializerTests.java new file mode 100644 index 0000000000000..7713fdf1d0adc --- /dev/null +++ b/server/src/test/java/org/opensearch/common/cache/serializer/ICacheKeySerializerTests.java @@ -0,0 +1,107 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.common.cache.serializer; + +import org.opensearch.OpenSearchException; +import org.opensearch.common.Randomness; +import org.opensearch.common.cache.ICacheKey; +import org.opensearch.core.common.bytes.BytesArray; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.UUID; + +public class ICacheKeySerializerTests extends OpenSearchTestCase { + // For these tests, we use BytesReference as K, since we already have a Serializer implementation + public void testEquality() throws Exception { + BytesReferenceSerializer keySer = new BytesReferenceSerializer(); + ICacheKeySerializer serializer = new ICacheKeySerializer<>(keySer); + + int numDimensionsTested = 100; + for (int i = 0; i < numDimensionsTested; i++) { + String dim = getRandomDimValue(); + ICacheKey key = new ICacheKey<>(getRandomBytesReference(), List.of(dim)); + byte[] serialized = serializer.serialize(key); + assertTrue(serializer.equals(key, serialized)); + ICacheKey deserialized = serializer.deserialize(serialized); + assertEquals(key, deserialized); + assertTrue(serializer.equals(deserialized, serialized)); + } + } + + public void testInvalidInput() throws Exception { + BytesReferenceSerializer keySer = new BytesReferenceSerializer(); + ICacheKeySerializer serializer = new ICacheKeySerializer<>(keySer); + + Random rand = Randomness.get(); + byte[] randomInput = new byte[1000]; + rand.nextBytes(randomInput); + + assertThrows(OpenSearchException.class, () -> serializer.deserialize(randomInput)); + } + + public void testDimNumbers() throws Exception { + BytesReferenceSerializer keySer = new BytesReferenceSerializer(); + ICacheKeySerializer serializer = new ICacheKeySerializer<>(keySer); + + for (int numDims : new int[] { 0, 5, 1000 }) { + List dims = new ArrayList<>(); + for (int j = 0; j < numDims; j++) { + dims.add(getRandomDimValue()); + } + ICacheKey key = new ICacheKey<>(getRandomBytesReference(), dims); + byte[] serialized = serializer.serialize(key); + assertTrue(serializer.equals(key, serialized)); + ICacheKey deserialized = serializer.deserialize(serialized); + assertEquals(key, deserialized); + } + } + + public void testHashCodes() throws Exception { + ICacheKey key1 = new ICacheKey<>("key", List.of("dimension_value")); + ICacheKey key2 = new ICacheKey<>("key", List.of("dimension_value")); + + ICacheKey key3 = new ICacheKey<>(null, List.of("dimension_value")); + ICacheKey key4 = new ICacheKey<>(null, List.of("dimension_value")); + + assertEquals(key1, key2); + assertEquals(key1.hashCode(), key2.hashCode()); + + assertEquals(key3, key4); + assertEquals(key3.hashCode(), key4.hashCode()); + + assertNotEquals(key1, key3); + assertNotEquals("string", key3); + } + + public void testNullInputs() throws Exception { + BytesReferenceSerializer keySer = new BytesReferenceSerializer(); + ICacheKeySerializer serializer = new ICacheKeySerializer<>(keySer); + + assertNull(serializer.deserialize(null)); + ICacheKey nullKey = new ICacheKey<>(null, List.of(getRandomDimValue())); + assertNull(serializer.serialize(nullKey)); + assertNull(serializer.serialize(null)); + assertNull(serializer.serialize(new ICacheKey<>(getRandomBytesReference(), null))); + } + + private String getRandomDimValue() { + return UUID.randomUUID().toString(); + } + + private BytesReference getRandomBytesReference() { + byte[] bytesValue = new byte[1000]; + Random rand = Randomness.get(); + rand.nextBytes(bytesValue); + return new BytesArray(bytesValue); + } +} diff --git a/server/src/test/java/org/opensearch/common/cache/stats/CacheStatsHolderTests.java b/server/src/test/java/org/opensearch/common/cache/stats/CacheStatsHolderTests.java new file mode 100644 index 0000000000000..390cd4d601a4b --- /dev/null +++ b/server/src/test/java/org/opensearch/common/cache/stats/CacheStatsHolderTests.java @@ -0,0 +1,287 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.common.cache.stats; + +import org.opensearch.common.Randomness; +import org.opensearch.common.metrics.CounterMetric; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; + +public class CacheStatsHolderTests extends OpenSearchTestCase { + public void testAddAndGet() throws Exception { + List dimensionNames = List.of("dim1", "dim2", "dim3", "dim4"); + CacheStatsHolder cacheStatsHolder = new CacheStatsHolder(dimensionNames); + Map> usedDimensionValues = CacheStatsHolderTests.getUsedDimensionValues(cacheStatsHolder, 10); + Map, CacheStats> expected = CacheStatsHolderTests.populateStats(cacheStatsHolder, usedDimensionValues, 1000, 10); + + // test the value in the map is as expected for each distinct combination of values + for (List dimensionValues : expected.keySet()) { + CacheStats expectedCounter = expected.get(dimensionValues); + + ImmutableCacheStats actualStatsHolder = CacheStatsHolderTests.getNode(dimensionValues, cacheStatsHolder.getStatsRoot()) + .getImmutableStats(); + ImmutableCacheStats actualCacheStats = getNode(dimensionValues, cacheStatsHolder.getStatsRoot()).getImmutableStats(); + + assertEquals(expectedCounter.immutableSnapshot(), actualStatsHolder); + assertEquals(expectedCounter.immutableSnapshot(), actualCacheStats); + } + + // Check overall total matches + CacheStats expectedTotal = new CacheStats(); + for (List dims : expected.keySet()) { + expectedTotal.add(expected.get(dims)); + } + assertEquals(expectedTotal.immutableSnapshot(), cacheStatsHolder.getStatsRoot().getImmutableStats()); + + // Check sum of children stats are correct + assertSumOfChildrenStats(cacheStatsHolder.getStatsRoot()); + } + + public void testReset() throws Exception { + List dimensionNames = List.of("dim1", "dim2"); + CacheStatsHolder cacheStatsHolder = new CacheStatsHolder(dimensionNames); + Map> usedDimensionValues = getUsedDimensionValues(cacheStatsHolder, 10); + Map, CacheStats> expected = populateStats(cacheStatsHolder, usedDimensionValues, 100, 10); + + cacheStatsHolder.reset(); + + for (List dimensionValues : expected.keySet()) { + CacheStats originalCounter = expected.get(dimensionValues); + originalCounter.sizeInBytes = new CounterMetric(); + originalCounter.entries = new CounterMetric(); + + CacheStatsHolder.Node node = getNode(dimensionValues, cacheStatsHolder.getStatsRoot()); + ImmutableCacheStats actual = node.getImmutableStats(); + assertEquals(originalCounter.immutableSnapshot(), actual); + } + } + + public void testDropStatsForDimensions() throws Exception { + List dimensionNames = List.of("dim1", "dim2"); + CacheStatsHolder cacheStatsHolder = new CacheStatsHolder(dimensionNames); + + // Create stats for the following dimension sets + List> populatedStats = List.of(List.of("A1", "B1"), List.of("A2", "B2"), List.of("A2", "B3")); + for (List dims : populatedStats) { + cacheStatsHolder.incrementHits(dims); + } + + assertEquals(3, cacheStatsHolder.getStatsRoot().getImmutableStats().getHits()); + + // When we invalidate A2, B2, we should lose the node for B2, but not B3 or A2. + + cacheStatsHolder.removeDimensions(List.of("A2", "B2")); + + assertEquals(2, cacheStatsHolder.getStatsRoot().getImmutableStats().getHits()); + assertNull(getNode(List.of("A2", "B2"), cacheStatsHolder.getStatsRoot())); + assertNotNull(getNode(List.of("A2"), cacheStatsHolder.getStatsRoot())); + assertNotNull(getNode(List.of("A2", "B3"), cacheStatsHolder.getStatsRoot())); + + // When we invalidate A1, B1, we should lose the nodes for B1 and also A1, as it has no more children. + + cacheStatsHolder.removeDimensions(List.of("A1", "B1")); + + assertEquals(1, cacheStatsHolder.getStatsRoot().getImmutableStats().getHits()); + assertNull(getNode(List.of("A1", "B1"), cacheStatsHolder.getStatsRoot())); + assertNull(getNode(List.of("A1"), cacheStatsHolder.getStatsRoot())); + + // When we invalidate the last node, all nodes should be deleted except the root node + + cacheStatsHolder.removeDimensions(List.of("A2", "B3")); + assertEquals(0, cacheStatsHolder.getStatsRoot().getImmutableStats().getHits()); + assertEquals(0, cacheStatsHolder.getStatsRoot().children.size()); + } + + public void testCount() throws Exception { + List dimensionNames = List.of("dim1", "dim2"); + CacheStatsHolder cacheStatsHolder = new CacheStatsHolder(dimensionNames); + Map> usedDimensionValues = getUsedDimensionValues(cacheStatsHolder, 10); + Map, CacheStats> expected = populateStats(cacheStatsHolder, usedDimensionValues, 100, 10); + + long expectedCount = 0L; + for (CacheStats counter : expected.values()) { + expectedCount += counter.getEntries(); + } + assertEquals(expectedCount, cacheStatsHolder.count()); + } + + public void testConcurrentRemoval() throws Exception { + List dimensionNames = List.of("dim1", "dim2"); + CacheStatsHolder cacheStatsHolder = new CacheStatsHolder(dimensionNames); + + // Create stats for the following dimension sets + List> populatedStats = List.of(List.of("A1", "B1"), List.of("A2", "B2"), List.of("A2", "B3")); + for (List dims : populatedStats) { + cacheStatsHolder.incrementHits(dims); + } + + // Remove (A2, B2) and (A1, B1), before re-adding (A2, B2). At the end we should have stats for (A2, B2) but not (A1, B1). + + Thread[] threads = new Thread[3]; + CountDownLatch countDownLatch = new CountDownLatch(3); + threads[0] = new Thread(() -> { + cacheStatsHolder.removeDimensions(List.of("A2", "B2")); + countDownLatch.countDown(); + }); + threads[1] = new Thread(() -> { + cacheStatsHolder.removeDimensions(List.of("A1", "B1")); + countDownLatch.countDown(); + }); + threads[2] = new Thread(() -> { + cacheStatsHolder.incrementMisses(List.of("A2", "B2")); + cacheStatsHolder.incrementMisses(List.of("A2", "B3")); + countDownLatch.countDown(); + }); + for (Thread thread : threads) { + thread.start(); + // Add short sleep to ensure threads start their functions in order (so that incrementing doesn't happen before removal) + Thread.sleep(1); + } + countDownLatch.await(); + assertNull(getNode(List.of("A1", "B1"), cacheStatsHolder.getStatsRoot())); + assertNull(getNode(List.of("A1"), cacheStatsHolder.getStatsRoot())); + assertNotNull(getNode(List.of("A2", "B2"), cacheStatsHolder.getStatsRoot())); + assertEquals( + new ImmutableCacheStats(0, 1, 0, 0, 0), + getNode(List.of("A2", "B2"), cacheStatsHolder.getStatsRoot()).getImmutableStats() + ); + assertEquals( + new ImmutableCacheStats(1, 1, 0, 0, 0), + getNode(List.of("A2", "B3"), cacheStatsHolder.getStatsRoot()).getImmutableStats() + ); + } + + /** + * Returns the node found by following these dimension values down from the root node. + * Returns null if no such node exists. + */ + static CacheStatsHolder.Node getNode(List dimensionValues, CacheStatsHolder.Node root) { + CacheStatsHolder.Node current = root; + for (String dimensionValue : dimensionValues) { + current = current.getChildren().get(dimensionValue); + if (current == null) { + return null; + } + } + return current; + } + + static Map, CacheStats> populateStats( + CacheStatsHolder cacheStatsHolder, + Map> usedDimensionValues, + int numDistinctValuePairs, + int numRepetitionsPerValue + ) throws InterruptedException { + Map, CacheStats> expected = new ConcurrentHashMap<>(); + Thread[] threads = new Thread[numDistinctValuePairs]; + CountDownLatch countDownLatch = new CountDownLatch(numDistinctValuePairs); + Random rand = Randomness.get(); + List> dimensionsForThreads = new ArrayList<>(); + for (int i = 0; i < numDistinctValuePairs; i++) { + dimensionsForThreads.add(getRandomDimList(cacheStatsHolder.getDimensionNames(), usedDimensionValues, true, rand)); + int finalI = i; + threads[i] = new Thread(() -> { + Random threadRand = Randomness.get(); + List dimensions = dimensionsForThreads.get(finalI); + expected.computeIfAbsent(dimensions, (key) -> new CacheStats()); + for (int j = 0; j < numRepetitionsPerValue; j++) { + CacheStats statsToInc = new CacheStats( + threadRand.nextInt(10), + threadRand.nextInt(10), + threadRand.nextInt(10), + threadRand.nextInt(5000), + threadRand.nextInt(10) + ); + expected.get(dimensions).hits.inc(statsToInc.getHits()); + expected.get(dimensions).misses.inc(statsToInc.getMisses()); + expected.get(dimensions).evictions.inc(statsToInc.getEvictions()); + expected.get(dimensions).sizeInBytes.inc(statsToInc.getSizeInBytes()); + expected.get(dimensions).entries.inc(statsToInc.getEntries()); + CacheStatsHolderTests.populateStatsHolderFromStatsValueMap(cacheStatsHolder, Map.of(dimensions, statsToInc)); + } + countDownLatch.countDown(); + }); + } + for (Thread thread : threads) { + thread.start(); + } + countDownLatch.await(); + return expected; + } + + private static List getRandomDimList( + List dimensionNames, + Map> usedDimensionValues, + boolean pickValueForAllDims, + Random rand + ) { + List result = new ArrayList<>(); + for (String dimName : dimensionNames) { + if (pickValueForAllDims || rand.nextBoolean()) { // if pickValueForAllDims, always pick a value for each dimension, otherwise do + // so 50% of the time + int index = between(0, usedDimensionValues.get(dimName).size() - 1); + result.add(usedDimensionValues.get(dimName).get(index)); + } + } + return result; + } + + static Map> getUsedDimensionValues(CacheStatsHolder cacheStatsHolder, int numValuesPerDim) { + Map> usedDimensionValues = new HashMap<>(); + for (int i = 0; i < cacheStatsHolder.getDimensionNames().size(); i++) { + List values = new ArrayList<>(); + for (int j = 0; j < numValuesPerDim; j++) { + values.add(UUID.randomUUID().toString()); + } + usedDimensionValues.put(cacheStatsHolder.getDimensionNames().get(i), values); + } + return usedDimensionValues; + } + + private void assertSumOfChildrenStats(CacheStatsHolder.Node current) { + if (!current.children.isEmpty()) { + CacheStats expectedTotal = new CacheStats(); + for (CacheStatsHolder.Node child : current.children.values()) { + expectedTotal.add(child.getImmutableStats()); + } + assertEquals(expectedTotal.immutableSnapshot(), current.getImmutableStats()); + for (CacheStatsHolder.Node child : current.children.values()) { + assertSumOfChildrenStats(child); + } + } + } + + static void populateStatsHolderFromStatsValueMap(CacheStatsHolder cacheStatsHolder, Map, CacheStats> statsMap) { + for (Map.Entry, CacheStats> entry : statsMap.entrySet()) { + CacheStats stats = entry.getValue(); + List dims = entry.getKey(); + for (int i = 0; i < stats.getHits(); i++) { + cacheStatsHolder.incrementHits(dims); + } + for (int i = 0; i < stats.getMisses(); i++) { + cacheStatsHolder.incrementMisses(dims); + } + for (int i = 0; i < stats.getEvictions(); i++) { + cacheStatsHolder.incrementEvictions(dims); + } + cacheStatsHolder.incrementSizeInBytes(dims, stats.getSizeInBytes()); + for (int i = 0; i < stats.getEntries(); i++) { + cacheStatsHolder.incrementEntries(dims); + } + } + } +} diff --git a/server/src/test/java/org/opensearch/common/cache/stats/ImmutableCacheStatsHolderTests.java b/server/src/test/java/org/opensearch/common/cache/stats/ImmutableCacheStatsHolderTests.java new file mode 100644 index 0000000000000..933b8abd6e392 --- /dev/null +++ b/server/src/test/java/org/opensearch/common/cache/stats/ImmutableCacheStatsHolderTests.java @@ -0,0 +1,88 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.common.cache.stats; + +import org.opensearch.test.OpenSearchTestCase; + +import java.util.List; +import java.util.Map; + +public class ImmutableCacheStatsHolderTests extends OpenSearchTestCase { + + public void testGet() throws Exception { + List dimensionNames = List.of("dim1", "dim2", "dim3", "dim4"); + CacheStatsHolder cacheStatsHolder = new CacheStatsHolder(dimensionNames); + Map> usedDimensionValues = CacheStatsHolderTests.getUsedDimensionValues(cacheStatsHolder, 10); + Map, CacheStats> expected = CacheStatsHolderTests.populateStats(cacheStatsHolder, usedDimensionValues, 1000, 10); + ImmutableCacheStatsHolder stats = cacheStatsHolder.getImmutableCacheStatsHolder(); + + // test the value in the map is as expected for each distinct combination of values + for (List dimensionValues : expected.keySet()) { + CacheStats expectedCounter = expected.get(dimensionValues); + + ImmutableCacheStats actualCacheStatsHolder = CacheStatsHolderTests.getNode(dimensionValues, cacheStatsHolder.getStatsRoot()) + .getImmutableStats(); + ImmutableCacheStats actualImmutableCacheStatsHolder = getNode(dimensionValues, stats.getStatsRoot()).getStats(); + + assertEquals(expectedCounter.immutableSnapshot(), actualCacheStatsHolder); + assertEquals(expectedCounter.immutableSnapshot(), actualImmutableCacheStatsHolder); + } + + // test gets for total (this also checks sum-of-children logic) + CacheStats expectedTotal = new CacheStats(); + for (List dims : expected.keySet()) { + expectedTotal.add(expected.get(dims)); + } + assertEquals(expectedTotal.immutableSnapshot(), stats.getTotalStats()); + + assertEquals(expectedTotal.getHits(), stats.getTotalHits()); + assertEquals(expectedTotal.getMisses(), stats.getTotalMisses()); + assertEquals(expectedTotal.getEvictions(), stats.getTotalEvictions()); + assertEquals(expectedTotal.getSizeInBytes(), stats.getTotalSizeInBytes()); + assertEquals(expectedTotal.getEntries(), stats.getTotalEntries()); + + assertSumOfChildrenStats(stats.getStatsRoot()); + } + + public void testEmptyDimsList() throws Exception { + // If the dimension list is empty, the tree should have only the root node containing the total stats. + CacheStatsHolder cacheStatsHolder = new CacheStatsHolder(List.of()); + Map> usedDimensionValues = CacheStatsHolderTests.getUsedDimensionValues(cacheStatsHolder, 100); + CacheStatsHolderTests.populateStats(cacheStatsHolder, usedDimensionValues, 10, 100); + ImmutableCacheStatsHolder stats = cacheStatsHolder.getImmutableCacheStatsHolder(); + + ImmutableCacheStatsHolder.Node statsRoot = stats.getStatsRoot(); + assertEquals(0, statsRoot.children.size()); + assertEquals(stats.getTotalStats(), statsRoot.getStats()); + } + + private ImmutableCacheStatsHolder.Node getNode(List dimensionValues, ImmutableCacheStatsHolder.Node root) { + ImmutableCacheStatsHolder.Node current = root; + for (String dimensionValue : dimensionValues) { + current = current.getChildren().get(dimensionValue); + if (current == null) { + return null; + } + } + return current; + } + + private void assertSumOfChildrenStats(ImmutableCacheStatsHolder.Node current) { + if (!current.children.isEmpty()) { + CacheStats expectedTotal = new CacheStats(); + for (ImmutableCacheStatsHolder.Node child : current.children.values()) { + expectedTotal.add(child.getStats()); + } + assertEquals(expectedTotal.immutableSnapshot(), current.getStats()); + for (ImmutableCacheStatsHolder.Node child : current.children.values()) { + assertSumOfChildrenStats(child); + } + } + } +} diff --git a/server/src/test/java/org/opensearch/common/cache/stats/ImmutableCacheStatsTests.java b/server/src/test/java/org/opensearch/common/cache/stats/ImmutableCacheStatsTests.java new file mode 100644 index 0000000000000..50ddd81943c3b --- /dev/null +++ b/server/src/test/java/org/opensearch/common/cache/stats/ImmutableCacheStatsTests.java @@ -0,0 +1,47 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.common.cache.stats; + +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.common.io.stream.BytesStreamInput; +import org.opensearch.test.OpenSearchTestCase; + +public class ImmutableCacheStatsTests extends OpenSearchTestCase { + public void testSerialization() throws Exception { + ImmutableCacheStats immutableCacheStats = new ImmutableCacheStats(1, 2, 3, 4, 5); + BytesStreamOutput os = new BytesStreamOutput(); + immutableCacheStats.writeTo(os); + BytesStreamInput is = new BytesStreamInput(BytesReference.toBytes(os.bytes())); + ImmutableCacheStats deserialized = new ImmutableCacheStats(is); + + assertEquals(immutableCacheStats, deserialized); + } + + public void testAddSnapshots() throws Exception { + ImmutableCacheStats ics1 = new ImmutableCacheStats(1, 2, 3, 4, 5); + ImmutableCacheStats ics2 = new ImmutableCacheStats(6, 7, 8, 9, 10); + ImmutableCacheStats expected = new ImmutableCacheStats(7, 9, 11, 13, 15); + assertEquals(expected, ImmutableCacheStats.addSnapshots(ics1, ics2)); + } + + public void testEqualsAndHash() throws Exception { + ImmutableCacheStats ics1 = new ImmutableCacheStats(1, 2, 3, 4, 5); + ImmutableCacheStats ics2 = new ImmutableCacheStats(1, 2, 3, 4, 5); + ImmutableCacheStats ics3 = new ImmutableCacheStats(0, 2, 3, 4, 5); + + assertEquals(ics1, ics2); + assertNotEquals(ics1, ics3); + assertNotEquals(ics1, null); + assertNotEquals(ics1, "string"); + + assertEquals(ics1.hashCode(), ics2.hashCode()); + assertNotEquals(ics1.hashCode(), ics3.hashCode()); + } +} 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 new file mode 100644 index 0000000000000..008dc7c2e0902 --- /dev/null +++ b/server/src/test/java/org/opensearch/common/cache/store/OpenSearchOnHeapCacheTests.java @@ -0,0 +1,181 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.common.cache.store; + +import org.opensearch.common.Randomness; +import org.opensearch.common.cache.CacheType; +import org.opensearch.common.cache.ICache; +import org.opensearch.common.cache.ICacheKey; +import org.opensearch.common.cache.LoadAwareCacheLoader; +import org.opensearch.common.cache.RemovalListener; +import org.opensearch.common.cache.RemovalNotification; +import org.opensearch.common.cache.stats.ImmutableCacheStats; +import org.opensearch.common.cache.store.config.CacheConfig; +import org.opensearch.common.cache.store.settings.OpenSearchOnHeapCacheSettings; +import org.opensearch.common.metrics.CounterMetric; +import org.opensearch.common.settings.Settings; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.UUID; + +import static org.opensearch.common.cache.store.settings.OpenSearchOnHeapCacheSettings.MAXIMUM_SIZE_IN_BYTES_KEY; + +public class OpenSearchOnHeapCacheTests extends OpenSearchTestCase { + private final static long keyValueSize = 50; + private final static List dimensionNames = List.of("dim1", "dim2", "dim3"); + + public void testStats() throws Exception { + MockRemovalListener listener = new MockRemovalListener<>(); + int maxKeys = between(10, 50); + int numEvicted = between(10, 20); + OpenSearchOnHeapCache cache = getCache(maxKeys, listener); + + List> keysAdded = new ArrayList<>(); + int numAdded = maxKeys + numEvicted; + for (int i = 0; i < numAdded; i++) { + ICacheKey key = getICacheKey(UUID.randomUUID().toString()); + keysAdded.add(key); + cache.computeIfAbsent(key, getLoadAwareCacheLoader()); + + assertEquals(i + 1, cache.stats().getTotalMisses()); + assertEquals(0, cache.stats().getTotalHits()); + assertEquals(Math.min(maxKeys, i + 1), cache.stats().getTotalEntries()); + assertEquals(Math.min(maxKeys, i + 1) * keyValueSize, cache.stats().getTotalSizeInBytes()); + assertEquals(Math.max(0, i + 1 - maxKeys), cache.stats().getTotalEvictions()); + } + // do gets from the last part of the list, which should be hits + for (int i = numAdded - maxKeys; i < numAdded; i++) { + cache.computeIfAbsent(keysAdded.get(i), getLoadAwareCacheLoader()); + int numHits = i + 1 - (numAdded - maxKeys); + + assertEquals(numAdded, cache.stats().getTotalMisses()); + assertEquals(numHits, cache.stats().getTotalHits()); + assertEquals(maxKeys, cache.stats().getTotalEntries()); + assertEquals(maxKeys * keyValueSize, cache.stats().getTotalSizeInBytes()); + assertEquals(numEvicted, cache.stats().getTotalEvictions()); + } + + // invalidate keys + for (int i = numAdded - maxKeys; i < numAdded; i++) { + cache.invalidate(keysAdded.get(i)); + int numInvalidated = i + 1 - (numAdded - maxKeys); + + assertEquals(numAdded, cache.stats().getTotalMisses()); + assertEquals(maxKeys, cache.stats().getTotalHits()); + assertEquals(maxKeys - numInvalidated, cache.stats().getTotalEntries()); + assertEquals((maxKeys - numInvalidated) * keyValueSize, cache.stats().getTotalSizeInBytes()); + assertEquals(numEvicted, cache.stats().getTotalEvictions()); + } + } + + private OpenSearchOnHeapCache getCache(int maxSizeKeys, MockRemovalListener listener) { + ICache.Factory onHeapCacheFactory = new OpenSearchOnHeapCache.OpenSearchOnHeapCacheFactory(); + Settings settings = Settings.builder() + .put( + OpenSearchOnHeapCacheSettings.getSettingListForCacheType(CacheType.INDICES_REQUEST_CACHE) + .get(MAXIMUM_SIZE_IN_BYTES_KEY) + .getKey(), + maxSizeKeys * keyValueSize + "b" + ) + .build(); + + CacheConfig cacheConfig = new CacheConfig.Builder().setKeyType(String.class) + .setValueType(String.class) + .setWeigher((k, v) -> keyValueSize) + .setRemovalListener(listener) + .setSettings(settings) + .setDimensionNames(dimensionNames) + .setMaxSizeInBytes(maxSizeKeys * keyValueSize) + .build(); + return (OpenSearchOnHeapCache) onHeapCacheFactory.create(cacheConfig, CacheType.INDICES_REQUEST_CACHE, null); + } + + public void testInvalidateWithDropDimensions() throws Exception { + MockRemovalListener listener = new MockRemovalListener<>(); + int maxKeys = 50; + OpenSearchOnHeapCache cache = getCache(maxKeys, listener); + + List> keysAdded = new ArrayList<>(); + + for (int i = 0; i < maxKeys - 5; i++) { + ICacheKey key = new ICacheKey<>(UUID.randomUUID().toString(), getRandomDimensions()); + keysAdded.add(key); + cache.computeIfAbsent(key, getLoadAwareCacheLoader()); + } + + ICacheKey keyToDrop = keysAdded.get(0); + + ImmutableCacheStats snapshot = cache.stats().getStatsForDimensionValues(keyToDrop.dimensions); + assertNotNull(snapshot); + + keyToDrop.setDropStatsForDimensions(true); + cache.invalidate(keyToDrop); + + // Now assert the stats are gone for any key that has this combination of dimensions, but still there otherwise + for (ICacheKey keyAdded : keysAdded) { + snapshot = cache.stats().getStatsForDimensionValues(keyAdded.dimensions); + if (keyAdded.dimensions.equals(keyToDrop.dimensions)) { + assertNull(snapshot); + } else { + assertNotNull(snapshot); + } + } + } + + private List getRandomDimensions() { + Random rand = Randomness.get(); + int bound = 3; + List result = new ArrayList<>(); + for (String dimName : dimensionNames) { + result.add(String.valueOf(rand.nextInt(bound))); + } + return result; + } + + private static class MockRemovalListener implements RemovalListener, V> { + CounterMetric numRemovals; + + MockRemovalListener() { + numRemovals = new CounterMetric(); + } + + @Override + public void onRemoval(RemovalNotification, V> notification) { + numRemovals.inc(); + } + } + + private ICacheKey getICacheKey(String key) { + List dims = new ArrayList<>(); + for (String dimName : dimensionNames) { + dims.add("0"); + } + return new ICacheKey<>(key, dims); + } + + private LoadAwareCacheLoader, String> getLoadAwareCacheLoader() { + return new LoadAwareCacheLoader<>() { + boolean isLoaded = false; + + @Override + public String load(ICacheKey key) { + isLoaded = true; + return UUID.randomUUID().toString(); + } + + @Override + public boolean isLoaded() { + return isLoaded; + } + }; + } +} diff --git a/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java b/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java index 594b9aac971b7..e3dca1b7bfda2 100644 --- a/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java +++ b/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java @@ -45,11 +45,14 @@ import org.apache.lucene.search.TopDocs; import org.apache.lucene.store.Directory; import org.apache.lucene.util.BytesRef; +import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.CheckedSupplier; +import org.opensearch.common.cache.ICacheKey; import org.opensearch.common.cache.RemovalNotification; import org.opensearch.common.cache.RemovalReason; import org.opensearch.common.cache.module.CacheModule; import org.opensearch.common.cache.service.CacheService; +import org.opensearch.common.cache.stats.ImmutableCacheStats; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.lucene.index.OpenSearchDirectoryReader; import org.opensearch.common.settings.Settings; @@ -69,13 +72,16 @@ import org.opensearch.index.query.TermQueryBuilder; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.IndexShardState; +import org.opensearch.index.shard.ShardNotFoundException; import org.opensearch.node.Node; +import org.opensearch.test.ClusterServiceUtils; import org.opensearch.test.OpenSearchSingleNodeTestCase; import org.opensearch.threadpool.ThreadPool; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.List; import java.util.Optional; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; @@ -96,7 +102,8 @@ public void testBasicOperationsCache() throws Exception { Settings.EMPTY, (shardId -> Optional.of(new IndicesService.IndexShardCacheEntity(indexShard))), new CacheModule(new ArrayList<>(), Settings.EMPTY).getCacheService(), - threadPool + threadPool, + ClusterServiceUtils.createClusterService(threadPool) ); Directory dir = newDirectory(); IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); @@ -160,7 +167,8 @@ public void testBasicOperationsCacheWithFeatureFlag() throws Exception { Settings.builder().put(super.featureFlagSettings()).put(FeatureFlags.PLUGGABLE_CACHE, "true").build(), (shardId -> Optional.of(new IndicesService.IndexShardCacheEntity(indexShard))), cacheService, - threadPool + threadPool, + ClusterServiceUtils.createClusterService(threadPool) ); Directory dir = newDirectory(); IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); @@ -228,7 +236,11 @@ public void testCacheDifferentReaders() throws Exception { return Optional.empty(); } return Optional.of(new IndicesService.IndexShardCacheEntity(indexService.getShard(shardId.id()))); - }), new CacheModule(new ArrayList<>(), Settings.EMPTY).getCacheService(), threadPool); + }), + new CacheModule(new ArrayList<>(), Settings.EMPTY).getCacheService(), + threadPool, + ClusterServiceUtils.createClusterService(threadPool) + ); Directory dir = newDirectory(); IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); @@ -360,7 +372,11 @@ public void testCacheCleanupBasedOnZeroThreshold() throws Exception { return Optional.empty(); } return Optional.of(new IndicesService.IndexShardCacheEntity(indexService.getShard(shardId.id()))); - }), new CacheModule(new ArrayList<>(), settings).getCacheService(), threadPool); + }), + new CacheModule(new ArrayList<>(), settings).getCacheService(), + threadPool, + ClusterServiceUtils.createClusterService(threadPool) + ); Directory dir = newDirectory(); IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); @@ -421,7 +437,11 @@ public void testCacheCleanupBasedOnStaleThreshold_StalenessEqualToThreshold() th return Optional.empty(); } return Optional.of(new IndicesService.IndexShardCacheEntity(indexService.getShard(shardId.id()))); - }), new CacheModule(new ArrayList<>(), settings).getCacheService(), threadPool); + }), + new CacheModule(new ArrayList<>(), settings).getCacheService(), + threadPool, + ClusterServiceUtils.createClusterService(threadPool) + ); Directory dir = newDirectory(); IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); @@ -483,7 +503,11 @@ public void testStaleCount_OnRemovalNotificationOfStaleKey_DecrementsStaleCount( return Optional.empty(); } return Optional.of(new IndicesService.IndexShardCacheEntity(indexService.getShard(shardId.id()))); - }), new CacheModule(new ArrayList<>(), settings).getCacheService(), threadPool); + }), + new CacheModule(new ArrayList<>(), settings).getCacheService(), + threadPool, + ClusterServiceUtils.createClusterService(threadPool) + ); Directory dir = newDirectory(); IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); @@ -534,7 +558,13 @@ public void testStaleCount_OnRemovalNotificationOfStaleKey_DecrementsStaleCount( readerCacheKeyId ); - cache.onRemoval(new RemovalNotification(key, termBytes, RemovalReason.EVICTED)); + cache.onRemoval( + new RemovalNotification, BytesReference>( + new ICacheKey<>(key), + termBytes, + RemovalReason.EVICTED + ) + ); staleKeysCount = cache.cacheCleanupManager.getStaleKeysCount(); // eviction of previous stale key from the cache should decrement staleKeysCount in iRC assertEquals(0, staleKeysCount.get()); @@ -556,7 +586,11 @@ public void testStaleCount_OnRemovalNotificationOfStaleKey_DoesNotDecrementsStal return Optional.empty(); } return Optional.of(new IndicesService.IndexShardCacheEntity(indexService.getShard(shardId.id()))); - }), new CacheModule(new ArrayList<>(), settings).getCacheService(), threadPool); + }), + new CacheModule(new ArrayList<>(), settings).getCacheService(), + threadPool, + ClusterServiceUtils.createClusterService(threadPool) + ); Directory dir = newDirectory(); IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); @@ -607,7 +641,13 @@ public void testStaleCount_OnRemovalNotificationOfStaleKey_DoesNotDecrementsStal readerCacheKeyId ); - cache.onRemoval(new RemovalNotification(key, termBytes, RemovalReason.EVICTED)); + cache.onRemoval( + new RemovalNotification, BytesReference>( + new ICacheKey<>(key), + termBytes, + RemovalReason.EVICTED + ) + ); staleKeysCount = cache.cacheCleanupManager.getStaleKeysCount(); // eviction of NON-stale key from the cache should NOT decrement staleKeysCount in iRC assertEquals(1, staleKeysCount.get()); @@ -629,7 +669,11 @@ public void testCacheCleanupBasedOnStaleThreshold_StalenessGreaterThanThreshold( return Optional.empty(); } return Optional.of(new IndicesService.IndexShardCacheEntity(indexService.getShard(shardId.id()))); - }), new CacheModule(new ArrayList<>(), settings).getCacheService(), threadPool); + }), + new CacheModule(new ArrayList<>(), settings).getCacheService(), + threadPool, + ClusterServiceUtils.createClusterService(threadPool) + ); Directory dir = newDirectory(); IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); @@ -691,7 +735,11 @@ public void testCacheCleanupBasedOnStaleThreshold_StalenessLesserThanThreshold() return Optional.empty(); } return Optional.of(new IndicesService.IndexShardCacheEntity(indexService.getShard(shardId.id()))); - }), new CacheModule(new ArrayList<>(), Settings.EMPTY).getCacheService(), threadPool); + }), + new CacheModule(new ArrayList<>(), Settings.EMPTY).getCacheService(), + threadPool, + ClusterServiceUtils.createClusterService(threadPool) + ); Directory dir = newDirectory(); IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); @@ -740,6 +788,117 @@ public void testCacheCleanupBasedOnStaleThreshold_StalenessLesserThanThreshold() terminate(threadPool); } + public void testClosingIndexWipesStats() throws Exception { + IndicesService indicesService = getInstanceFromNode(IndicesService.class); + // Create two indices each with multiple shards + int numShards = 3; + Settings indexSettings = Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, numShards).build(); + String indexToKeepName = "test"; + String indexToCloseName = "test2"; + IndexService indexToKeep = createIndex(indexToKeepName, indexSettings); + IndexService indexToClose = createIndex(indexToCloseName, indexSettings); + for (int i = 0; i < numShards; i++) { + // Check we can get all the shards we expect + assertNotNull(indexToKeep.getShard(i)); + assertNotNull(indexToClose.getShard(i)); + } + ThreadPool threadPool = getThreadPool(); + Settings settings = Settings.builder().put(INDICES_REQUEST_CACHE_STALENESS_THRESHOLD_SETTING.getKey(), "0.001%").build(); + IndicesRequestCache cache = new IndicesRequestCache(settings, (shardId -> { + IndexService indexService = null; + try { + indexService = indicesService.indexServiceSafe(shardId.getIndex()); + } catch (IndexNotFoundException ex) { + return Optional.empty(); + } + try { + return Optional.of(new IndicesService.IndexShardCacheEntity(indexService.getShard(shardId.id()))); + } catch (ShardNotFoundException ex) { + return Optional.empty(); + } + }), + new CacheModule(new ArrayList<>(), Settings.EMPTY).getCacheService(), + threadPool, + ClusterServiceUtils.createClusterService(threadPool) + ); + Directory dir = newDirectory(); + IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); + + writer.addDocument(newDoc(0, "foo")); + TermQueryBuilder termQuery = new TermQueryBuilder("id", "0"); + BytesReference termBytes = XContentHelper.toXContent(termQuery, MediaTypeRegistry.JSON, false); + if (randomBoolean()) { + writer.flush(); + IOUtils.close(writer); + writer = new IndexWriter(dir, newIndexWriterConfig()); + } + writer.updateDocument(new Term("id", "0"), newDoc(0, "bar")); + DirectoryReader secondReader = OpenSearchDirectoryReader.wrap(DirectoryReader.open(writer), new ShardId("foo", "bar", 1)); + + List readersToClose = new ArrayList<>(); + List readersToKeep = new ArrayList<>(); + // Put entries into the cache for each shard + for (IndexService indexService : new IndexService[] { indexToKeep, indexToClose }) { + for (int i = 0; i < numShards; i++) { + IndexShard indexShard = indexService.getShard(i); + IndicesService.IndexShardCacheEntity entity = new IndicesService.IndexShardCacheEntity(indexShard); + DirectoryReader reader = OpenSearchDirectoryReader.wrap(DirectoryReader.open(writer), indexShard.shardId()); + if (indexService == indexToClose) { + readersToClose.add(reader); + } else { + readersToKeep.add(reader); + } + Loader loader = new Loader(reader, 0); + cache.getOrCompute(entity, loader, reader, termBytes); + } + } + + // Check resulting stats + List> initialDimensionValues = new ArrayList<>(); + for (IndexService indexService : new IndexService[] { indexToKeep, indexToClose }) { + for (int i = 0; i < numShards; i++) { + ShardId shardId = indexService.getShard(i).shardId(); + List dimensionValues = List.of(shardId.getIndexName(), shardId.toString()); + initialDimensionValues.add(dimensionValues); + ImmutableCacheStats snapshot = cache.stats().getStatsForDimensionValues(dimensionValues); + assertNotNull(snapshot); + // check the values are not empty by confirming entries != 0, this should always be true since the missed value is loaded + // into the cache + assertNotEquals(0, snapshot.getEntries()); + } + } + + // Delete an index + indexToClose.close("test_deletion", true); + // This actually closes the shards associated with the readers, which is necessary for cache cleanup logic + // In this UT, manually close the readers as well; could not figure out how to connect all this up in a UT so that + // we could get readers that were properly connected to an index's directory + for (DirectoryReader reader : readersToClose) { + IOUtils.close(reader); + } + // Trigger cache cleanup + cache.cacheCleanupManager.cleanCache(); + + // Now stats for the closed index should be gone + for (List dimensionValues : initialDimensionValues) { + ImmutableCacheStats snapshot = cache.stats().getStatsForDimensionValues(dimensionValues); + if (dimensionValues.get(0).equals(indexToCloseName)) { + assertNull(snapshot); + } else { + assertNotNull(snapshot); + // check the values are not empty by confirming entries != 0, this should always be true since the missed value is loaded + // into the cache + assertNotEquals(0, snapshot.getEntries()); + } + } + + for (DirectoryReader reader : readersToKeep) { + IOUtils.close(reader); + } + IOUtils.close(secondReader, writer, dir, cache); + terminate(threadPool); + } + public void testEviction() throws Exception { final ByteSizeValue size; { @@ -749,7 +908,8 @@ public void testEviction() throws Exception { Settings.EMPTY, (shardId -> Optional.of(new IndicesService.IndexShardCacheEntity(indexShard))), new CacheModule(new ArrayList<>(), Settings.EMPTY).getCacheService(), - threadPool + threadPool, + ClusterServiceUtils.createClusterService(threadPool) ); Directory dir = newDirectory(); IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); @@ -770,17 +930,19 @@ public void testEviction() throws Exception { assertEquals("foo", value1.streamInput().readString()); BytesReference value2 = cache.getOrCompute(secondEntity, secondLoader, secondReader, termBytes); assertEquals("bar", value2.streamInput().readString()); - size = indexShard.requestCache().stats().getMemorySize(); + size = new ByteSizeValue(cache.getSizeInBytes()); IOUtils.close(reader, secondReader, writer, dir, cache); terminate(threadPool); } IndexShard indexShard = createIndex("test1").getShard(0); ThreadPool threadPool = getThreadPool(); IndicesRequestCache cache = new IndicesRequestCache( - Settings.builder().put(IndicesRequestCache.INDICES_CACHE_QUERY_SIZE.getKey(), size.getBytes() + 1 + "b").build(), + // Add 5 instead of 1; the key size now depends on the length of dimension names and values so there's more variation + Settings.builder().put(IndicesRequestCache.INDICES_CACHE_QUERY_SIZE.getKey(), size.getBytes() + 5 + "b").build(), (shardId -> Optional.of(new IndicesService.IndexShardCacheEntity(indexShard))), new CacheModule(new ArrayList<>(), Settings.EMPTY).getCacheService(), - threadPool + threadPool, + ClusterServiceUtils.createClusterService(threadPool) ); Directory dir = newDirectory(); IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); @@ -827,7 +989,11 @@ public void testClearAllEntityIdentity() throws Exception { return Optional.empty(); } return Optional.of(new IndicesService.IndexShardCacheEntity(indexService.getShard(shardId.id()))); - }), new CacheModule(new ArrayList<>(), Settings.EMPTY).getCacheService(), threadPool); + }), + new CacheModule(new ArrayList<>(), Settings.EMPTY).getCacheService(), + threadPool, + ClusterServiceUtils.createClusterService(threadPool) + ); Directory dir = newDirectory(); IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); @@ -923,7 +1089,11 @@ public void testInvalidate() throws Exception { return Optional.empty(); } return Optional.of(new IndicesService.IndexShardCacheEntity(indexService.getShard(shardId.id()))); - }), new CacheModule(new ArrayList<>(), Settings.EMPTY).getCacheService(), threadPool); + }), + new CacheModule(new ArrayList<>(), Settings.EMPTY).getCacheService(), + threadPool, + ClusterServiceUtils.createClusterService(threadPool) + ); Directory dir = newDirectory(); IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig());