From 435af893d14b23685aa450169626379c455d1548 Mon Sep 17 00:00:00 2001 From: Rishabh Maurya Date: Wed, 12 Jun 2024 18:19:34 -0700 Subject: [PATCH 01/84] Fix the rewrite method for MatchOnlyText field query (#14154) Signed-off-by: Rishabh Maurya --- .../index/query/SourceFieldMatchQuery.java | 2 +- .../search/query/QueryProfilePhaseTests.java | 96 +++++++++++++++++++ 2 files changed, 97 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/index/query/SourceFieldMatchQuery.java b/server/src/main/java/org/opensearch/index/query/SourceFieldMatchQuery.java index b0be20e417efe..e9abcb698f68f 100644 --- a/server/src/main/java/org/opensearch/index/query/SourceFieldMatchQuery.java +++ b/server/src/main/java/org/opensearch/index/query/SourceFieldMatchQuery.java @@ -73,7 +73,7 @@ public void visit(QueryVisitor visitor) { @Override public Query rewrite(IndexSearcher indexSearcher) throws IOException { - Query rewritten = indexSearcher.rewrite(delegateQuery); + Query rewritten = delegateQuery.rewrite(indexSearcher); if (rewritten == delegateQuery) { return this; } diff --git a/server/src/test/java/org/opensearch/search/query/QueryProfilePhaseTests.java b/server/src/test/java/org/opensearch/search/query/QueryProfilePhaseTests.java index 6af04e15acef0..1d545cea67207 100644 --- a/server/src/test/java/org/opensearch/search/query/QueryProfilePhaseTests.java +++ b/server/src/test/java/org/opensearch/search/query/QueryProfilePhaseTests.java @@ -35,6 +35,7 @@ import org.apache.lucene.search.LeafCollector; import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.MatchNoDocsQuery; +import org.apache.lucene.search.PhraseQuery; import org.apache.lucene.search.Pruning; import org.apache.lucene.search.Query; import org.apache.lucene.search.Sort; @@ -48,12 +49,18 @@ import org.opensearch.action.search.SearchShardTask; import org.opensearch.common.settings.Settings; import org.opensearch.common.xcontent.json.JsonXContent; +import org.opensearch.core.index.Index; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.index.mapper.DocumentMapper; +import org.opensearch.index.mapper.MatchOnlyTextFieldMapper; import org.opensearch.index.mapper.NumberFieldMapper.NumberFieldType; import org.opensearch.index.mapper.NumberFieldMapper.NumberType; +import org.opensearch.index.mapper.SourceFieldMapper; +import org.opensearch.index.mapper.TextSearchInfo; import org.opensearch.index.query.ParsedQuery; import org.opensearch.index.query.QueryShardContext; +import org.opensearch.index.query.SourceFieldMatchQuery; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.IndexShardTestCase; import org.opensearch.lucene.queries.MinDocQuery; @@ -62,6 +69,9 @@ import org.opensearch.search.internal.ContextIndexSearcher; import org.opensearch.search.internal.ScrollContext; import org.opensearch.search.internal.SearchContext; +import org.opensearch.search.lookup.LeafSearchLookup; +import org.opensearch.search.lookup.SearchLookup; +import org.opensearch.search.lookup.SourceLookup; import org.opensearch.search.profile.ProfileResult; import org.opensearch.search.profile.ProfileShardResult; import org.opensearch.search.profile.SearchProfileShardResults; @@ -80,6 +90,7 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; @@ -94,6 +105,7 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.instanceOf; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -1514,6 +1526,90 @@ public void testCollapseQuerySearchResults() throws Exception { dir.close(); } + public void testSourceFieldMatchQueryWithProfile() throws Exception { + Directory dir = newDirectory(); + IndexWriterConfig iwc = newIndexWriterConfig(); + RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc); + w.close(); + IndexReader reader = DirectoryReader.open(dir); + QueryShardContext queryShardContext = mock(QueryShardContext.class); + DocumentMapper mockDocumentMapper = mock(DocumentMapper.class); + SourceFieldMapper mockSourceMapper = mock(SourceFieldMapper.class); + SearchLookup searchLookup = mock(SearchLookup.class); + LeafSearchLookup leafSearchLookup = mock(LeafSearchLookup.class); + + when(queryShardContext.sourcePath("foo")).thenReturn(Set.of("bar")); + when(queryShardContext.index()).thenReturn(new Index("test_index", "uuid")); + when(searchLookup.getLeafSearchLookup(any())).thenReturn(leafSearchLookup); + when(leafSearchLookup.source()).thenReturn(new SourceLookup()); + when(mockSourceMapper.enabled()).thenReturn(true); + when(mockDocumentMapper.sourceMapper()).thenReturn(mockSourceMapper); + when(queryShardContext.documentMapper(any())).thenReturn(mockDocumentMapper); + when(queryShardContext.lookup()).thenReturn(searchLookup); + + TestSearchContext context = new TestSearchContext(queryShardContext, indexShard, newContextSearcher(reader, executor)); + context.parsedQuery( + new ParsedQuery( + new SourceFieldMatchQuery( + new TermQuery(new Term("foo", "bar")), + new PhraseQuery("foo", "bar", "baz"), + new MatchOnlyTextFieldMapper.MatchOnlyTextFieldType( + "user", + true, + true, + TextSearchInfo.WHITESPACE_MATCH_ONLY, + Collections.emptyMap() + ), + queryShardContext + ) + ) + ); + + context.setTask(new SearchShardTask(123L, "", "", "", null, Collections.emptyMap())); + context.setSize(1); + context.trackTotalHitsUpTo(5); + QueryPhase.executeInternal(context.withCleanQueryResult().withProfilers(), queryPhaseSearcher); + assertProfileData(context, "SourceFieldMatchQuery", query -> { + assertThat(query.getTimeBreakdown().keySet(), not(empty())); + assertThat(query.getTimeBreakdown().get("score"), equalTo(0L)); + assertThat(query.getTimeBreakdown().get("score_count"), equalTo(0L)); + if (executor != null) { + long maxScore = query.getTimeBreakdown().get("max_score"); + long minScore = query.getTimeBreakdown().get("min_score"); + long avgScore = query.getTimeBreakdown().get("avg_score"); + long maxScoreCount = query.getTimeBreakdown().get("max_score_count"); + long minScoreCount = query.getTimeBreakdown().get("min_score_count"); + long avgScoreCount = query.getTimeBreakdown().get("avg_score_count"); + assertThat(maxScore, equalTo(0L)); + assertThat(minScore, equalTo(0L)); + assertThat(avgScore, equalTo(0L)); + assertThat(maxScore, equalTo(avgScore)); + assertThat(avgScore, equalTo(minScore)); + assertThat(maxScoreCount, equalTo(0L)); + assertThat(minScoreCount, equalTo(0L)); + assertThat(avgScoreCount, equalTo(0L)); + assertThat(maxScoreCount, equalTo(avgScoreCount)); + assertThat(avgScoreCount, equalTo(minScoreCount)); + } + assertThat(query.getTimeBreakdown().get("create_weight"), greaterThan(0L)); + assertThat(query.getTimeBreakdown().get("create_weight_count"), equalTo(1L)); + assertThat(query.getProfiledChildren(), empty()); + }, collector -> { + assertThat(collector.getReason(), equalTo("search_top_hits")); + assertThat(collector.getTime(), greaterThan(0L)); + if (collector.getName().contains("CollectorManager")) { + assertThat(collector.getReduceTime(), greaterThan(0L)); + } + assertThat(collector.getMaxSliceTime(), greaterThan(0L)); + assertThat(collector.getMinSliceTime(), greaterThan(0L)); + assertThat(collector.getAvgSliceTime(), greaterThan(0L)); + assertThat(collector.getSliceCount(), greaterThanOrEqualTo(1)); + assertThat(collector.getProfiledChildren(), empty()); + }); + reader.close(); + dir.close(); + } + private void assertProfileData(SearchContext context, String type, Consumer query, Consumer collector) throws IOException { assertProfileData(context, collector, (profileResult) -> { From ccf528973ef9487adcc5bc4e8d38c6632b42b6fb Mon Sep 17 00:00:00 2001 From: Kiran Prakash Date: Wed, 12 Jun 2024 18:30:06 -0700 Subject: [PATCH 02/84] [Tiered Caching] [Bug Fix] Use concurrentMap instead of HashMap to fix Concurrent Modification Exception (#14221) * use concurrentmap Signed-off-by: Kiran Prakash * Update IndicesRequestCacheTests.java Signed-off-by: Kiran Prakash * Update IndicesRequestCacheTests.java Signed-off-by: Kiran Prakash * Update CHANGELOG.md Signed-off-by: Kiran Prakash * Update IndicesRequestCache.java Signed-off-by: Kiran Prakash * Update IndicesRequestCacheTests.java Signed-off-by: Kiran Prakash * Update IndicesRequestCacheTests.java Signed-off-by: Kiran Prakash * revert feature flags Signed-off-by: Kiran Prakash * changelog to releaselog Signed-off-by: Kiran Prakash * use concurrentmap Signed-off-by: Kiran Prakash * Update IndicesRequestCacheTests.java Signed-off-by: Kiran Prakash * Update IndicesRequestCacheTests.java Signed-off-by: Kiran Prakash * Update CHANGELOG.md Signed-off-by: Kiran Prakash * Update IndicesRequestCache.java Signed-off-by: Kiran Prakash * Update IndicesRequestCacheTests.java Signed-off-by: Kiran Prakash * Update IndicesRequestCacheTests.java Signed-off-by: Kiran Prakash * revert feature flags Signed-off-by: Kiran Prakash * changelog to releaselog Signed-off-by: Kiran Prakash * revert the test removal Signed-off-by: Kiran Prakash * revert the conflict resolutions Signed-off-by: Kiran Prakash --------- Signed-off-by: Kiran Prakash --- .../opensearch.release-notes-2.15.0.md | 3 +- .../indices/IndicesRequestCache.java | 13 ++-- .../indices/IndicesRequestCacheTests.java | 64 +++++++++++++++++-- 3 files changed, 69 insertions(+), 11 deletions(-) diff --git a/release-notes/opensearch.release-notes-2.15.0.md b/release-notes/opensearch.release-notes-2.15.0.md index 4e95173abd700..02458b0c89b7d 100644 --- a/release-notes/opensearch.release-notes-2.15.0.md +++ b/release-notes/opensearch.release-notes-2.15.0.md @@ -70,4 +70,5 @@ - Fix double invocation of postCollection when MultiBucketCollector is present ([#14015](https://github.com/opensearch-project/OpenSearch/pull/14015)) - Fix ReplicaShardBatchAllocator to batch shards without duplicates ([#13710](https://github.com/opensearch-project/OpenSearch/pull/13710)) - Java high-level REST client bulk() is not respecting the bulkRequest.requireAlias(true) method call ([#14146](https://github.com/opensearch-project/OpenSearch/pull/14146)) -- Fix ShardNotFoundException during request cache clean up ([#14219](https://github.com/opensearch-project/OpenSearch/pull/14219)) \ No newline at end of file +- Fix ShardNotFoundException during request cache clean up ([#14219](https://github.com/opensearch-project/OpenSearch/pull/14219)) +- Fix Concurrent Modification Exception in Indices Request Cache([#14032](https://github.com/opensearch-project/OpenSearch/pull/14221)) \ No newline at end of file diff --git a/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java b/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java index 5c82e5e9639f7..06cd77a34fe0b 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java +++ b/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java @@ -75,7 +75,6 @@ import java.io.IOException; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; @@ -507,7 +506,7 @@ public int hashCode() { * */ class IndicesRequestCacheCleanupManager implements Closeable { private final Set keysToClean; - private final ConcurrentMap> cleanupKeyToCountMap; + private final ConcurrentMap> cleanupKeyToCountMap; private final AtomicInteger staleKeysCount; private volatile double stalenessThreshold; private final IndicesRequestCacheCleaner cacheCleaner; @@ -568,7 +567,13 @@ private void updateStaleCountOnCacheInsert(CleanupKey cleanupKey) { // If the key doesn't exist, it's added with a value of 1. // If the key exists, its value is incremented by 1. - cleanupKeyToCountMap.computeIfAbsent(shardId, k -> new HashMap<>()).merge(cleanupKey.readerCacheKeyId, 1, Integer::sum); + addToCleanupKeyToCountMap(shardId, cleanupKey.readerCacheKeyId); + } + + // pkg-private for testing + void addToCleanupKeyToCountMap(ShardId shardId, String readerCacheKeyId) { + cleanupKeyToCountMap.computeIfAbsent(shardId, k -> ConcurrentCollections.newConcurrentMap()) + .merge(readerCacheKeyId, 1, Integer::sum); } /** @@ -826,7 +831,7 @@ public void close() { } // for testing - ConcurrentMap> getCleanupKeyToCountMap() { + ConcurrentMap> getCleanupKeyToCountMap() { return cleanupKeyToCountMap; } diff --git a/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java b/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java index 9dbdddb76ea24..205712d388cd1 100644 --- a/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java +++ b/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java @@ -95,7 +95,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; +import java.util.ConcurrentModificationException; import java.util.List; import java.util.Map; import java.util.Optional; @@ -105,7 +105,9 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.Phaser; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import static java.util.Collections.emptyMap; @@ -489,7 +491,7 @@ public void testStaleCount_OnRemovalNotificationOfStaleKey_DecrementsStaleCount( indexShard.hashCode() ); // test the mapping - ConcurrentMap> cleanupKeyToCountMap = cache.cacheCleanupManager.getCleanupKeyToCountMap(); + ConcurrentMap> cleanupKeyToCountMap = cache.cacheCleanupManager.getCleanupKeyToCountMap(); // shard id should exist assertTrue(cleanupKeyToCountMap.containsKey(shardId)); // reader CacheKeyId should NOT exist @@ -552,7 +554,7 @@ public void testStaleCount_OnRemovalNotificationOfNonStaleKey_DoesNotDecrementsS ); // test the mapping - ConcurrentMap> cleanupKeyToCountMap = cache.cacheCleanupManager.getCleanupKeyToCountMap(); + ConcurrentMap> cleanupKeyToCountMap = cache.cacheCleanupManager.getCleanupKeyToCountMap(); // shard id should exist assertTrue(cleanupKeyToCountMap.containsKey(shardId)); // reader CacheKeyId should NOT exist @@ -720,7 +722,7 @@ public void testCleanupKeyToCountMapAreSetAppropriately() throws Exception { cache.getOrCompute(getEntity(indexShard), getLoader(reader), reader, getTermBytes()); assertEquals(1, cache.count()); // test the mappings - ConcurrentMap> cleanupKeyToCountMap = cache.cacheCleanupManager.getCleanupKeyToCountMap(); + ConcurrentMap> cleanupKeyToCountMap = cache.cacheCleanupManager.getCleanupKeyToCountMap(); assertEquals(1, (int) cleanupKeyToCountMap.get(shardId).get(getReaderCacheKeyId(reader))); cache.getOrCompute(getEntity(indexShard), getLoader(secondReader), secondReader, getTermBytes()); @@ -793,8 +795,54 @@ public void testCleanupKeyToCountMapAreSetAppropriately() throws Exception { IOUtils.close(secondReader); } - private DirectoryReader getReader(IndexWriter writer, ShardId shardId) throws IOException { - return OpenSearchDirectoryReader.wrap(DirectoryReader.open(writer), shardId); + // test adding to cleanupKeyToCountMap with multiple threads + public void testAddToCleanupKeyToCountMap() throws Exception { + threadPool = getThreadPool(); + Settings settings = Settings.builder().put(INDICES_REQUEST_CACHE_STALENESS_THRESHOLD_SETTING.getKey(), "51%").build(); + cache = getIndicesRequestCache(settings); + + int numberOfThreads = 10; + int numberOfIterations = 1000; + Phaser phaser = new Phaser(numberOfThreads + 1); // +1 for the main thread + AtomicBoolean exceptionDetected = new AtomicBoolean(false); + + ExecutorService executorService = Executors.newFixedThreadPool(numberOfThreads); + + for (int i = 0; i < numberOfThreads; i++) { + executorService.submit(() -> { + phaser.arriveAndAwaitAdvance(); // Ensure all threads start at the same time + try { + for (int j = 0; j < numberOfIterations; j++) { + cache.cacheCleanupManager.addToCleanupKeyToCountMap(indexShard.shardId(), UUID.randomUUID().toString()); + } + } catch (ConcurrentModificationException e) { + logger.error("ConcurrentModificationException detected in thread : " + e.getMessage()); + exceptionDetected.set(true); // Set flag if exception is detected + } + }); + } + phaser.arriveAndAwaitAdvance(); // Start all threads + + // Main thread iterates over the map + executorService.submit(() -> { + try { + for (int j = 0; j < numberOfIterations; j++) { + cache.cacheCleanupManager.getCleanupKeyToCountMap().forEach((k, v) -> { + v.forEach((k1, v1) -> { + // Accessing the map to create contention + v.get(k1); + }); + }); + } + } catch (ConcurrentModificationException e) { + logger.error("ConcurrentModificationException detected in main thread : " + e.getMessage()); + exceptionDetected.set(true); // Set flag if exception is detected + } + }); + + executorService.shutdown(); + executorService.awaitTermination(60, TimeUnit.SECONDS); + assertFalse(exceptionDetected.get()); } private IndicesRequestCache getIndicesRequestCache(Settings settings) { @@ -808,6 +856,10 @@ private IndicesRequestCache getIndicesRequestCache(Settings settings) { ); } + private DirectoryReader getReader(IndexWriter writer, ShardId shardId) throws IOException { + return OpenSearchDirectoryReader.wrap(DirectoryReader.open(writer), shardId); + } + private Loader getLoader(DirectoryReader reader) { return new Loader(reader, 0); } From afeddc228ba7791a549fb7c6ef94349d432c0824 Mon Sep 17 00:00:00 2001 From: Gaurav Bafna <85113518+gbbafna@users.noreply.github.com> Date: Thu, 13 Jun 2024 15:12:47 +0530 Subject: [PATCH 03/84] [Remote Store] Fix sleep time bug during remote store sync (#14037) --------- Signed-off-by: Gaurav Bafna --- .../MigrationBaseTestCase.java | 32 +++++++++++++++++-- .../RemotePrimaryRelocationIT.java | 23 ++----------- .../opensearch/index/shard/IndexShard.java | 2 +- 3 files changed, 32 insertions(+), 25 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java index 901b36f872622..9dcbe380477dc 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java @@ -9,6 +9,8 @@ package org.opensearch.remotemigration; import org.opensearch.action.DocWriteResponse; +import org.opensearch.action.admin.cluster.health.ClusterHealthRequest; +import org.opensearch.action.admin.cluster.health.ClusterHealthResponse; import org.opensearch.action.admin.cluster.repositories.get.GetRepositoriesRequest; import org.opensearch.action.admin.cluster.repositories.get.GetRepositoriesResponse; import org.opensearch.action.bulk.BulkRequest; @@ -16,11 +18,15 @@ import org.opensearch.action.delete.DeleteResponse; import org.opensearch.action.index.IndexRequest; import org.opensearch.action.index.IndexResponse; +import org.opensearch.client.Requests; import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.health.ClusterHealthStatus; import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.cluster.routing.RoutingNode; +import org.opensearch.common.Priority; import org.opensearch.common.UUIDs; import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; import org.opensearch.repositories.fs.ReloadableFsRepository; import org.opensearch.test.OpenSearchIntegTestCase; import org.junit.Before; @@ -39,6 +45,7 @@ import static org.opensearch.node.remotestore.RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING; import static org.opensearch.repositories.fs.ReloadableFsRepository.REPOSITORIES_FAILRATE_SETTING; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; +import static org.hamcrest.Matchers.equalTo; public class MigrationBaseTestCase extends OpenSearchIntegTestCase { protected static final String REPOSITORY_NAME = "test-remote-store-repo"; @@ -114,6 +121,10 @@ public void initDocRepToRemoteMigration() { ); } + public ClusterHealthStatus ensureGreen(String... indices) { + return ensureGreen(TimeValue.timeValueSeconds(60), indices); + } + public BulkResponse indexBulk(String indexName, int numDocs) { BulkRequest bulkRequest = new BulkRequest(); for (int i = 0; i < numDocs; i++) { @@ -181,14 +192,12 @@ private Thread getIndexingThread() { long currentDocCount = indexedDocs.incrementAndGet(); if (currentDocCount > 0 && currentDocCount % refreshFrequency == 0) { if (rarely()) { - logger.info("--> [iteration {}] flushing index", currentDocCount); client().admin().indices().prepareFlush(indexName).get(); + logger.info("Completed ingestion of {} docs. Flushing now", currentDocCount); } else { - logger.info("--> [iteration {}] refreshing index", currentDocCount); client().admin().indices().prepareRefresh(indexName).get(); } } - logger.info("Completed ingestion of {} docs", currentDocCount); } }); } @@ -218,4 +227,21 @@ public void stopShardRebalancing() { .get() ); } + + public ClusterHealthStatus waitForRelocation() { + ClusterHealthRequest request = Requests.clusterHealthRequest() + .waitForNoRelocatingShards(true) + .timeout(TimeValue.timeValueSeconds(60)) + .waitForEvents(Priority.LANGUID); + ClusterHealthResponse actionGet = client().admin().cluster().health(request).actionGet(); + if (actionGet.isTimedOut()) { + logger.info( + "waitForRelocation timed out, cluster state:\n{}\n{}", + client().admin().cluster().prepareState().get().getState(), + client().admin().cluster().preparePendingClusterTasks().get() + ); + assertThat("timed out waiting for relocation", actionGet.isTimedOut(), equalTo(false)); + } + return actionGet.getStatus(); + } } diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemotePrimaryRelocationIT.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemotePrimaryRelocationIT.java index cea653c0ead4b..fa3b9368ded47 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemotePrimaryRelocationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemotePrimaryRelocationIT.java @@ -99,16 +99,7 @@ public void testRemotePrimaryRelocation() throws Exception { .add(new MoveAllocationCommand("test", 0, primaryNodeName("test"), remoteNode)) .execute() .actionGet(); - ClusterHealthResponse clusterHealthResponse = client().admin() - .cluster() - .prepareHealth() - .setTimeout(TimeValue.timeValueSeconds(60)) - .setWaitForEvents(Priority.LANGUID) - .setWaitForNoRelocatingShards(true) - .execute() - .actionGet(); - - assertEquals(0, clusterHealthResponse.getRelocatingShards()); + waitForRelocation(); assertEquals(remoteNode, primaryNodeName("test")); logger.info("--> relocation from docrep to remote complete"); @@ -123,16 +114,7 @@ public void testRemotePrimaryRelocation() throws Exception { .add(new MoveAllocationCommand("test", 0, remoteNode, remoteNode2)) .execute() .actionGet(); - clusterHealthResponse = client().admin() - .cluster() - .prepareHealth() - .setTimeout(TimeValue.timeValueSeconds(60)) - .setWaitForEvents(Priority.LANGUID) - .setWaitForNoRelocatingShards(true) - .execute() - .actionGet(); - - assertEquals(0, clusterHealthResponse.getRelocatingShards()); + waitForRelocation(); assertEquals(remoteNode2, primaryNodeName("test")); logger.info("--> relocation from remote to remote complete"); @@ -155,7 +137,6 @@ public void testRemotePrimaryRelocation() throws Exception { public void testMixedModeRelocation_RemoteSeedingFail() throws Exception { String docRepNode = internalCluster().startNode(); - Client client = internalCluster().client(docRepNode); ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest(); updateSettingsRequest.persistentSettings(Settings.builder().put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), "mixed")); assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index 49cb710c915fc..82b68b32f3bf8 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -2146,7 +2146,7 @@ public void waitForRemoteStoreSync(Runnable onProgress) throws IOException { segmentUploadeCount = directory.getSegmentsUploadedToRemoteStore().size(); } try { - Thread.sleep(TimeValue.timeValueSeconds(30).seconds()); + Thread.sleep(TimeValue.timeValueSeconds(30).millis()); } catch (InterruptedException ie) { throw new OpenSearchException("Interrupted waiting for completion of [{}]", ie); } From 18c5bb6cd57192ea1bb5cbc772814fa0443659fe Mon Sep 17 00:00:00 2001 From: SwethaGuptha <156877431+SwethaGuptha@users.noreply.github.com> Date: Thu, 13 Jun 2024 18:58:09 +0530 Subject: [PATCH 04/84] Fix unassigned batch allocation (#13748) (#13748) Signed-off-by: Swetha Guptha --- .../gateway/RecoveryFromGatewayIT.java | 98 ++++++++ .../gateway/BaseGatewayShardAllocator.java | 53 +---- .../gateway/PrimaryShardBatchAllocator.java | 60 +++-- .../gateway/ReplicaShardBatchAllocator.java | 128 ++++++---- .../PrimaryShardBatchAllocatorTests.java | 124 +++++++--- .../ReplicaShardBatchAllocatorTests.java | 223 ++++++++++++++++-- 6 files changed, 505 insertions(+), 181 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index bc0557ddc2afa..fc0a574c191b1 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -34,6 +34,7 @@ import org.apache.lucene.index.CorruptIndexException; import org.opensearch.Version; +import org.opensearch.action.admin.cluster.allocation.ClusterAllocationExplainResponse; import org.opensearch.action.admin.cluster.configuration.AddVotingConfigExclusionsAction; import org.opensearch.action.admin.cluster.configuration.AddVotingConfigExclusionsRequest; import org.opensearch.action.admin.cluster.configuration.ClearVotingConfigExclusionsAction; @@ -55,7 +56,9 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.ShardRoutingState; import org.opensearch.cluster.routing.UnassignedInfo; +import org.opensearch.cluster.routing.allocation.AllocationDecision; import org.opensearch.cluster.routing.allocation.ExistingShardsAllocator; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.Settings; @@ -98,6 +101,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; import java.util.stream.IntStream; import static java.util.Collections.emptyMap; @@ -105,8 +109,10 @@ import static org.opensearch.cluster.coordination.ClusterBootstrapService.INITIAL_CLUSTER_MANAGER_NODES_SETTING; import static org.opensearch.cluster.health.ClusterHealthStatus.GREEN; import static org.opensearch.cluster.health.ClusterHealthStatus.RED; +import static org.opensearch.cluster.health.ClusterHealthStatus.YELLOW; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS; +import static org.opensearch.cluster.routing.UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING; import static org.opensearch.common.xcontent.XContentFactory.jsonBuilder; import static org.opensearch.gateway.GatewayRecoveryTestUtils.corruptShard; import static org.opensearch.gateway.GatewayRecoveryTestUtils.getDiscoveryNodes; @@ -753,6 +759,7 @@ public void testMessyElectionsStillMakeClusterGoGreen() throws Exception { Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), "100ms") .put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), "100ms") .build() ); @@ -843,6 +850,87 @@ public void testBatchModeDisabled() throws Exception { ensureGreen("test"); } + public void testMultipleReplicaShardAssignmentWithDelayedAllocationAndDifferentNodeStartTimeInBatchMode() throws Exception { + internalCluster().startClusterManagerOnlyNodes( + 1, + Settings.builder().put(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.getKey(), true).build() + ); + internalCluster().startDataOnlyNodes(6); + createIndex( + "test", + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 3) + .put(INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), "60m") + .build() + ); + ensureGreen("test"); + + List nodesWithReplicaShards = findNodesWithShard(false); + Settings replicaNode0DataPathSettings = internalCluster().dataPathSettings(nodesWithReplicaShards.get(0)); + Settings replicaNode1DataPathSettings = internalCluster().dataPathSettings(nodesWithReplicaShards.get(1)); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(nodesWithReplicaShards.get(0))); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(nodesWithReplicaShards.get(1))); + + ensureStableCluster(5); + + logger.info("--> explicitly triggering reroute"); + ClusterRerouteResponse clusterRerouteResponse = client().admin().cluster().prepareReroute().setRetryFailed(true).get(); + assertTrue(clusterRerouteResponse.isAcknowledged()); + + ClusterHealthResponse health = client().admin().cluster().health(Requests.clusterHealthRequest().timeout("5m")).actionGet(); + assertFalse(health.isTimedOut()); + assertEquals(YELLOW, health.getStatus()); + assertEquals(2, health.getUnassignedShards()); + // shard should be unassigned because of Allocation_Delayed + ClusterAllocationExplainResponse allocationExplainResponse = client().admin() + .cluster() + .prepareAllocationExplain() + .setIndex("test") + .setShard(0) + .setPrimary(false) + .get(); + assertEquals( + AllocationDecision.ALLOCATION_DELAYED, + allocationExplainResponse.getExplanation().getShardAllocationDecision().getAllocateDecision().getAllocationDecision() + ); + + logger.info("--> restarting the node 1"); + internalCluster().startDataOnlyNode( + Settings.builder().put("node.name", nodesWithReplicaShards.get(0)).put(replicaNode0DataPathSettings).build() + ); + clusterRerouteResponse = client().admin().cluster().prepareReroute().setRetryFailed(true).get(); + assertTrue(clusterRerouteResponse.isAcknowledged()); + ensureStableCluster(6); + waitUntil( + () -> client().admin().cluster().health(Requests.clusterHealthRequest().timeout("5m")).actionGet().getInitializingShards() == 0 + ); + + health = client().admin().cluster().health(Requests.clusterHealthRequest().timeout("5m")).actionGet(); + assertFalse(health.isTimedOut()); + assertEquals(YELLOW, health.getStatus()); + assertEquals(1, health.getUnassignedShards()); + assertEquals(1, health.getDelayedUnassignedShards()); + allocationExplainResponse = client().admin() + .cluster() + .prepareAllocationExplain() + .setIndex("test") + .setShard(0) + .setPrimary(false) + .get(); + assertEquals( + AllocationDecision.ALLOCATION_DELAYED, + allocationExplainResponse.getExplanation().getShardAllocationDecision().getAllocateDecision().getAllocationDecision() + ); + + logger.info("--> restarting the node 0"); + internalCluster().startDataOnlyNode( + Settings.builder().put("node.name", nodesWithReplicaShards.get(1)).put(replicaNode1DataPathSettings).build() + ); + ensureStableCluster(7); + ensureGreen("test"); + } + public void testNBatchesCreationAndAssignment() throws Exception { // we will reduce batch size to 5 to make sure we have enough batches to test assignment // Total number of primary shards = 50 (50 indices*1) @@ -1293,4 +1381,14 @@ private void prepareIndex(String indexName, int numberOfPrimaryShards) { index(indexName, "type", "1", Collections.emptyMap()); flush(indexName); } + + private List findNodesWithShard(final boolean primary) { + ClusterState state = client().admin().cluster().prepareState().get().getState(); + List startedShards = state.routingTable().shardsWithState(ShardRoutingState.STARTED); + List requiredStartedShards = startedShards.stream() + .filter(startedShard -> startedShard.primary() == primary) + .collect(Collectors.toList()); + Collections.shuffle(requiredStartedShards, random()); + return requiredStartedShards.stream().map(shard -> state.nodes().get(shard.currentNodeId()).getName()).collect(Collectors.toList()); + } } diff --git a/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java b/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java index eed5de65258fc..58982e869794f 100644 --- a/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java @@ -36,7 +36,6 @@ import org.apache.logging.log4j.Logger; import org.opensearch.cluster.routing.RecoverySource; import org.opensearch.cluster.routing.RoutingNode; -import org.opensearch.cluster.routing.RoutingNodes; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; import org.opensearch.cluster.routing.allocation.AllocationDecision; @@ -46,9 +45,7 @@ import org.opensearch.cluster.routing.allocation.decider.Decision; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; -import java.util.stream.Collectors; /** * An abstract class that implements basic functionality for allocating @@ -81,38 +78,7 @@ public void allocateUnassigned( executeDecision(shardRouting, allocateUnassignedDecision, allocation, unassignedAllocationHandler); } - /** - * Allocate Batch of unassigned shard to nodes where valid copies of the shard already exists - * @param shardRoutings the shards to allocate - * @param allocation the allocation state container object - */ - public void allocateUnassignedBatch(List shardRoutings, RoutingAllocation allocation) { - // make Allocation Decisions for all shards - HashMap decisionMap = makeAllocationDecision(shardRoutings, allocation, logger); - assert shardRoutings.size() == decisionMap.size() : "make allocation decision didn't return allocation decision for " - + "some shards"; - // get all unassigned shards iterator - RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); - - while (iterator.hasNext()) { - ShardRouting shard = iterator.next(); - try { - if (decisionMap.isEmpty() == false) { - if (decisionMap.containsKey(shard)) { - executeDecision(shard, decisionMap.remove(shard), allocation, iterator); - } - } else { - // no need to keep iterating the unassigned shards, if we don't have anything in decision map - break; - } - } catch (Exception e) { - logger.error("Failed to execute decision for shard {} while initializing {}", shard, e); - throw e; - } - } - } - - private void executeDecision( + protected void executeDecision( ShardRouting shardRouting, AllocateUnassignedDecision allocateUnassignedDecision, RoutingAllocation allocation, @@ -135,8 +101,6 @@ private void executeDecision( } } - public void allocateUnassignedBatch(String batchId, RoutingAllocation allocation) {} - protected long getExpectedShardSize(ShardRouting shardRouting, RoutingAllocation allocation) { if (shardRouting.primary()) { if (shardRouting.recoverySource().getType() == RecoverySource.Type.SNAPSHOT) { @@ -165,21 +129,6 @@ public abstract AllocateUnassignedDecision makeAllocationDecision( Logger logger ); - public HashMap makeAllocationDecision( - List unassignedShardBatch, - RoutingAllocation allocation, - Logger logger - ) { - - return (HashMap) unassignedShardBatch.stream() - .collect( - Collectors.toMap( - unassignedShard -> unassignedShard, - unassignedShard -> makeAllocationDecision(unassignedShard, allocation, logger) - ) - ); - } - /** * Builds decisions for all nodes in the cluster, so that the explain API can provide information on * allocation decisions for each node, while still waiting to allocate the shard (e.g. due to fetching shard data). diff --git a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java index 1979f33484d49..27f9bedc4e495 100644 --- a/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/PrimaryShardBatchAllocator.java @@ -14,6 +14,7 @@ import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; import org.opensearch.cluster.routing.allocation.RoutingAllocation; +import org.opensearch.core.index.shard.ShardId; import org.opensearch.gateway.AsyncShardFetch.FetchResult; import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.GatewayStartedShard; import org.opensearch.gateway.TransportNodesGatewayStartedShardHelper.NodeGatewayStartedShard; @@ -61,50 +62,59 @@ protected FetchResult shardsState = fetchData( + List.of(unassignedShard), + Collections.emptyList(), + allocation + ); + List nodeGatewayStartedShards = adaptToNodeShardStates(unassignedShard, shardsState); + return getAllocationDecision(unassignedShard, allocation, nodeGatewayStartedShards, logger); } /** - * Build allocation decisions for all the shards present in the batch identified by batchId. + * Allocate Batch of unassigned shard to nodes where valid copies of the shard already exists * - * @param shards set of shards given for allocation - * @param allocation current allocation of all the shards - * @param logger logger used for logging - * @return shard to allocation decision map + * @param shardRoutings the shards to allocate + * @param allocation the allocation state container object */ - @Override - public HashMap makeAllocationDecision( - List shards, - RoutingAllocation allocation, - Logger logger - ) { - HashMap shardAllocationDecisions = new HashMap<>(); + public void allocateUnassignedBatch(List shardRoutings, RoutingAllocation allocation) { + HashMap ineligibleShardAllocationDecisions = new HashMap<>(); List eligibleShards = new ArrayList<>(); List inEligibleShards = new ArrayList<>(); // identify ineligible shards - for (ShardRouting shard : shards) { + for (ShardRouting shard : shardRoutings) { AllocateUnassignedDecision decision = getInEligibleShardDecision(shard, allocation); if (decision != null) { + ineligibleShardAllocationDecisions.put(shard.shardId(), decision); inEligibleShards.add(shard); - shardAllocationDecisions.put(shard, decision); } else { eligibleShards.add(shard); } } - // Do not call fetchData if there are no eligible shards - if (eligibleShards.isEmpty()) { - return shardAllocationDecisions; - } + // only fetch data for eligible shards final FetchResult shardsState = fetchData(eligibleShards, inEligibleShards, allocation); - // process the received data - for (ShardRouting unassignedShard : eligibleShards) { - List nodeShardStates = adaptToNodeShardStates(unassignedShard, shardsState); - // get allocation decision for this shard - shardAllocationDecisions.put(unassignedShard, getAllocationDecision(unassignedShard, allocation, nodeShardStates, logger)); + RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); + while (iterator.hasNext()) { + ShardRouting unassignedShard = iterator.next(); + AllocateUnassignedDecision allocationDecision; + + if (shardRoutings.contains(unassignedShard)) { + assert unassignedShard.primary(); + if (ineligibleShardAllocationDecisions.containsKey(unassignedShard.shardId())) { + allocationDecision = ineligibleShardAllocationDecisions.get(unassignedShard.shardId()); + } else { + List nodeShardStates = adaptToNodeShardStates(unassignedShard, shardsState); + allocationDecision = getAllocationDecision(unassignedShard, allocation, nodeShardStates, logger); + } + executeDecision(unassignedShard, allocationDecision, allocation, iterator); + } } - return shardAllocationDecisions; } /** diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java index be7867b7823f6..f2cb3d053440d 100644 --- a/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardBatchAllocator.java @@ -10,6 +10,7 @@ import org.apache.logging.log4j.Logger; import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.routing.RoutingNodes; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.UnassignedInfo; import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; @@ -29,6 +30,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.function.Supplier; +import java.util.stream.Collectors; /** * Allocates replica shards in a batch mode @@ -42,7 +45,7 @@ public abstract class ReplicaShardBatchAllocator extends ReplicaShardAllocator { * match. Today, a better match is one that can perform a no-op recovery while the previous recovery * has to copy segment files. * - * @param allocation the overall routing allocation + * @param allocation the overall routing allocation * @param shardBatches a list of shard batches to check for existing recoveries */ public void processExistingRecoveries(RoutingAllocation allocation, List> shardBatches) { @@ -98,71 +101,92 @@ protected FetchResult> fetchDataResultSupplier = () -> { + return convertToNodeStoreFilesMetadataMap( + unassignedShard, + fetchData(List.of(unassignedShard), Collections.emptyList(), allocation) + ); + }; + return getUnassignedShardAllocationDecision(unassignedShard, allocation, fetchDataResultSupplier); } - @Override - public HashMap makeAllocationDecision( - List shards, - RoutingAllocation allocation, - Logger logger - ) { - HashMap shardAllocationDecisions = new HashMap<>(); - final boolean explain = allocation.debugDecision(); + /** + * Allocate Batch of unassigned shard to nodes where valid copies of the shard already exists + * + * @param shardRoutings the shards to allocate + * @param allocation the allocation state container object + */ + public void allocateUnassignedBatch(List shardRoutings, RoutingAllocation allocation) { List eligibleShards = new ArrayList<>(); List ineligibleShards = new ArrayList<>(); - HashMap>> nodeAllocationDecisions = new HashMap<>(); - for (ShardRouting shard : shards) { - if (!isResponsibleFor(shard)) { - // this allocator n is not responsible for allocating this shard + Map ineligibleShardAllocationDecisions = new HashMap<>(); + + for (ShardRouting shard : shardRoutings) { + AllocateUnassignedDecision shardDecisionWithoutFetch = getUnassignedShardAllocationDecision(shard, allocation, null); + // Without fetchData, decision for in-eligible shards is non-null from our preliminary checks and null for eligible shards. + if (shardDecisionWithoutFetch != null) { ineligibleShards.add(shard); - shardAllocationDecisions.put(shard, AllocateUnassignedDecision.NOT_TAKEN); - continue; + ineligibleShardAllocationDecisions.put(shard, shardDecisionWithoutFetch); + } else { + eligibleShards.add(shard); } + } - Tuple> result = canBeAllocatedToAtLeastOneNode(shard, allocation); - Decision allocationDecision = result.v1(); - if (allocationDecision.type() != Decision.Type.YES && (!explain || !hasInitiatedFetching(shard))) { - // only return early if we are not in explain mode, or we are in explain mode but we have not - // yet attempted to fetch any shard data - logger.trace("{}: ignoring allocation, can't be allocated on any node", shard); - shardAllocationDecisions.put( - shard, - AllocateUnassignedDecision.no( - UnassignedInfo.AllocationStatus.fromDecision(allocationDecision.type()), - result.v2() != null ? new ArrayList<>(result.v2().values()) : null - ) - ); - continue; - } - // storing the nodeDecisions in nodeAllocationDecisions if the decision is not YES - // so that we don't have to compute the decisions again - nodeAllocationDecisions.put(shard, result); + // only fetch data for eligible shards + final FetchResult shardsState = fetchData(eligibleShards, ineligibleShards, allocation); - eligibleShards.add(shard); + List shardIdsFromBatch = shardRoutings.stream().map(shardRouting -> shardRouting.shardId()).collect(Collectors.toList()); + RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); + while (iterator.hasNext()) { + ShardRouting unassignedShard = iterator.next(); + // There will be only one entry for the shard in the unassigned shards batch + // for a shard with multiple unassigned replicas, hence we are comparing the shard ids + // instead of ShardRouting in-order to evaluate shard assignment for all unassigned replicas of a shard. + if (!unassignedShard.primary() && shardIdsFromBatch.contains(unassignedShard.shardId())) { + AllocateUnassignedDecision allocateUnassignedDecision; + if (ineligibleShardAllocationDecisions.containsKey(unassignedShard)) { + allocateUnassignedDecision = ineligibleShardAllocationDecisions.get(unassignedShard); + } else { + // The shard's eligibility is being recomputed again as + // the routing allocation state is updated during shard allocation decision execution + // because of which allocation eligibility of other unassigned shards can change. + allocateUnassignedDecision = getUnassignedShardAllocationDecision( + unassignedShard, + allocation, + () -> convertToNodeStoreFilesMetadataMap(unassignedShard, shardsState) + ); + } + executeDecision(unassignedShard, allocateUnassignedDecision, allocation, iterator); + } } + } - // Do not call fetchData if there are no eligible shards - if (eligibleShards.isEmpty()) { - return shardAllocationDecisions; + private AllocateUnassignedDecision getUnassignedShardAllocationDecision( + ShardRouting shardRouting, + RoutingAllocation allocation, + Supplier> nodeStoreFileMetaDataMapSupplier + ) { + if (!isResponsibleFor(shardRouting)) { + return AllocateUnassignedDecision.NOT_TAKEN; } - // only fetch data for eligible shards - final FetchResult shardsState = fetchData(eligibleShards, ineligibleShards, allocation); + Tuple> result = canBeAllocatedToAtLeastOneNode(shardRouting, allocation); - for (ShardRouting unassignedShard : eligibleShards) { - Tuple> result = nodeAllocationDecisions.get(unassignedShard); - shardAllocationDecisions.put( - unassignedShard, - getAllocationDecision( - unassignedShard, - allocation, - convertToNodeStoreFilesMetadataMap(unassignedShard, shardsState), - result, - logger - ) + final boolean explain = allocation.debugDecision(); + Decision allocationDecision = result.v1(); + if (allocationDecision.type() != Decision.Type.YES && (!explain || !hasInitiatedFetching(shardRouting))) { + // only return early if we are not in explain mode, or we are in explain mode but we have not + // yet attempted to fetch any shard data + logger.trace("{}: ignoring allocation, can't be allocated on any node", shardRouting); + return AllocateUnassignedDecision.no( + UnassignedInfo.AllocationStatus.fromDecision(allocationDecision.type()), + result.v2() != null ? new ArrayList<>(result.v2().values()) : null ); } - return shardAllocationDecisions; + if (nodeStoreFileMetaDataMapSupplier != null) { + Map discoveryNodeStoreFilesMetadataMap = nodeStoreFileMetaDataMapSupplier.get(); + return getAllocationDecision(shardRouting, allocation, discoveryNodeStoreFilesMetadataMap, result, logger); + } + return null; } private Map convertToNodeStoreFilesMetadataMap( diff --git a/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java index 522ad2a64ea5d..e90850de3fe33 100644 --- a/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java @@ -9,6 +9,7 @@ import org.apache.lucene.codecs.Codec; import org.opensearch.Version; +import org.opensearch.cluster.ClusterInfo; import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.OpenSearchAllocationTestCase; @@ -19,12 +20,15 @@ import org.opensearch.cluster.routing.RoutingNodes; import org.opensearch.cluster.routing.RoutingTable; import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.ShardRoutingState; import org.opensearch.cluster.routing.UnassignedInfo; import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; import org.opensearch.cluster.routing.allocation.AllocationDecision; import org.opensearch.cluster.routing.allocation.RoutingAllocation; import org.opensearch.cluster.routing.allocation.decider.AllocationDeciders; +import org.opensearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider; import org.opensearch.common.Nullable; +import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.set.Sets; import org.opensearch.core.index.shard.ShardId; @@ -44,6 +48,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; import static org.opensearch.cluster.routing.UnassignedInfo.Reason.CLUSTER_RECOVERED; @@ -87,42 +92,28 @@ private void allocateAllUnassignedBatch(final RoutingAllocation allocation) { public void testMakeAllocationDecisionDataFetching() { final RoutingAllocation allocation = routingAllocationWithOnePrimary(noAllocationDeciders(), CLUSTER_RECOVERED, "allocId1"); - - List shards = new ArrayList<>(); - allocateAllUnassignedBatch(allocation); ShardRouting shard = allocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).primaryShard(); - shards.add(shard); - HashMap allDecisions = batchAllocator.makeAllocationDecision(shards, allocation, logger); - // verify we get decisions for all the shards - assertEquals(shards.size(), allDecisions.size()); - assertEquals(shards, new ArrayList<>(allDecisions.keySet())); - assertEquals(AllocationDecision.AWAITING_INFO, allDecisions.get(shard).getAllocationDecision()); + AllocateUnassignedDecision allocateUnassignedDecision = batchAllocator.makeAllocationDecision(shard, allocation, logger); + assertEquals(AllocationDecision.AWAITING_INFO, allocateUnassignedDecision.getAllocationDecision()); } public void testMakeAllocationDecisionForReplicaShard() { final RoutingAllocation allocation = routingAllocationWithOnePrimary(noAllocationDeciders(), CLUSTER_RECOVERED, "allocId1"); List replicaShards = allocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).replicaShards(); - List shards = new ArrayList<>(replicaShards); - HashMap allDecisions = batchAllocator.makeAllocationDecision(shards, allocation, logger); - // verify we get decisions for all the shards - assertEquals(shards.size(), allDecisions.size()); - assertEquals(shards, new ArrayList<>(allDecisions.keySet())); - assertFalse(allDecisions.get(replicaShards.get(0)).isDecisionTaken()); + for (ShardRouting shardRouting : replicaShards) { + AllocateUnassignedDecision allocateUnassignedDecision = batchAllocator.makeAllocationDecision(shardRouting, allocation, logger); + assertFalse(allocateUnassignedDecision.isDecisionTaken()); + } } public void testMakeAllocationDecisionDataFetched() { final RoutingAllocation allocation = routingAllocationWithOnePrimary(noAllocationDeciders(), CLUSTER_RECOVERED, "allocId1"); - List shards = new ArrayList<>(); ShardRouting shard = allocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).primaryShard(); - shards.add(shard); batchAllocator.addData(node1, "allocId1", true, new ReplicationCheckpoint(shardId, 20, 101, 1, Codec.getDefault().getName())); - HashMap allDecisions = batchAllocator.makeAllocationDecision(shards, allocation, logger); - // verify we get decisions for all the shards - assertEquals(shards.size(), allDecisions.size()); - assertEquals(shards, new ArrayList<>(allDecisions.keySet())); - assertEquals(AllocationDecision.YES, allDecisions.get(shard).getAllocationDecision()); + AllocateUnassignedDecision allocateUnassignedDecision = batchAllocator.makeAllocationDecision(shard, allocation, logger); + assertEquals(AllocationDecision.YES, allocateUnassignedDecision.getAllocationDecision()); } public void testMakeAllocationDecisionDataFetchedMultipleShards() { @@ -149,13 +140,88 @@ public void testMakeAllocationDecisionDataFetchedMultipleShards() { null ); } - HashMap allDecisions = batchAllocator.makeAllocationDecision(shards, allocation, logger); - // verify we get decisions for all the shards - assertEquals(shards.size(), allDecisions.size()); - assertEquals(new HashSet<>(shards), allDecisions.keySet()); - for (ShardRouting shard : shards) { - assertEquals(AllocationDecision.YES, allDecisions.get(shard).getAllocationDecision()); + for (ShardRouting shardRouting : shards) { + AllocateUnassignedDecision allocateUnassignedDecision = batchAllocator.makeAllocationDecision(shardRouting, allocation, logger); + assertEquals(AllocationDecision.YES, allocateUnassignedDecision.getAllocationDecision()); + } + } + + public void testInitializePrimaryShards() { + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + AllocationDeciders allocationDeciders = randomAllocationDeciders(Settings.builder().build(), clusterSettings, random()); + setUpShards(2); + final RoutingAllocation routingAllocation = routingAllocationWithMultiplePrimaries( + allocationDeciders, + CLUSTER_RECOVERED, + 2, + 0, + "allocId-0", + "allocId-1" + ); + + for (ShardId shardId : shardsInBatch) { + batchAllocator.addShardData( + node1, + "allocId-" + shardId.id(), + shardId, + true, + new ReplicationCheckpoint(shardId, 20, 101, 1, Codec.getDefault().getName()), + null + ); + } + + allocateAllUnassignedBatch(routingAllocation); + + assertEquals(0, routingAllocation.routingNodes().unassigned().size()); + List initializingShards = routingAllocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING); + assertEquals(2, initializingShards.size()); + assertTrue(shardsInBatch.contains(initializingShards.get(0).shardId())); + assertTrue(shardsInBatch.contains(initializingShards.get(1).shardId())); + assertEquals(2, routingAllocation.routingNodes().getInitialPrimariesIncomingRecoveries(node1.getId())); + } + + public void testAllocateUnassignedBatchThrottlingAllocationDeciderIsHonoured() { + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + AllocationDeciders allocationDeciders = randomAllocationDeciders( + Settings.builder() + .put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES_SETTING.getKey(), 1) + .build(), + clusterSettings, + random() + ); + setUpShards(2); + final RoutingAllocation routingAllocation = routingAllocationWithMultiplePrimaries( + allocationDeciders, + CLUSTER_RECOVERED, + 2, + 0, + "allocId-0", + "allocId-1" + ); + + for (ShardId shardId : shardsInBatch) { + batchAllocator.addShardData( + node1, + "allocId-" + shardId.id(), + shardId, + true, + new ReplicationCheckpoint(shardId, 20, 101, 1, Codec.getDefault().getName()), + null + ); } + + allocateAllUnassignedBatch(routingAllocation); + + // Verify the throttling decider was not throttled, recovering shards on node greater than initial concurrent recovery setting + assertEquals(1, routingAllocation.routingNodes().getInitialPrimariesIncomingRecoveries(node1.getId())); + List initializingShards = routingAllocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING); + assertEquals(1, initializingShards.size()); + Set nodesWithInitialisingShards = initializingShards.stream().map(ShardRouting::currentNodeId).collect(Collectors.toSet()); + assertEquals(1, nodesWithInitialisingShards.size()); + assertEquals(Collections.singleton(node1.getId()), nodesWithInitialisingShards); + List ignoredShards = routingAllocation.routingNodes().unassigned().ignored(); + assertEquals(1, ignoredShards.size()); + assertEquals(UnassignedInfo.AllocationStatus.DECIDERS_THROTTLED, ignoredShards.get(0).unassignedInfo().getLastAllocationStatus()); } private RoutingAllocation routingAllocationWithOnePrimary( @@ -235,7 +301,7 @@ private RoutingAllocation routingAllocationWithMultiplePrimaries( .routingTable(routingTableBuilder.build()) .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) .build(); - return new RoutingAllocation(deciders, new RoutingNodes(state, false), state, null, null, System.nanoTime()); + return new RoutingAllocation(deciders, new RoutingNodes(state, false), state, ClusterInfo.EMPTY, null, System.nanoTime()); } class TestBatchAllocator extends PrimaryShardBatchAllocator { diff --git a/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java index 464038c93228b..2e148c2bc8130 100644 --- a/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java @@ -28,16 +28,19 @@ import org.opensearch.cluster.routing.ShardRoutingState; import org.opensearch.cluster.routing.TestShardRouting; import org.opensearch.cluster.routing.UnassignedInfo; +import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; import org.opensearch.cluster.routing.allocation.RoutingAllocation; import org.opensearch.cluster.routing.allocation.decider.AllocationDecider; import org.opensearch.cluster.routing.allocation.decider.AllocationDeciders; import org.opensearch.cluster.routing.allocation.decider.Decision; import org.opensearch.cluster.routing.allocation.decider.SameShardAllocationDecider; +import org.opensearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider; import org.opensearch.common.Nullable; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.set.Sets; +import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; import org.opensearch.index.engine.Engine; import org.opensearch.index.seqno.ReplicationTracker; @@ -72,6 +75,7 @@ public class ReplicaShardBatchAllocatorTests extends OpenSearchAllocationTestCas private static final org.apache.lucene.util.Version MIN_SUPPORTED_LUCENE_VERSION = org.opensearch.Version.CURRENT .minimumIndexCompatibilityVersion().luceneVersion; private final ShardId shardId = new ShardId("test", "_na_", 0); + private static Set shardsInBatch; private final DiscoveryNode node1 = newNode("node1"); private final DiscoveryNode node2 = newNode("node2"); private final DiscoveryNode node3 = newNode("node3"); @@ -83,6 +87,14 @@ public void buildTestAllocator() { this.testBatchAllocator = new TestBatchAllocator(); } + public static void setUpShards(int numberOfShards) { + shardsInBatch = new HashSet<>(); + for (int shardNumber = 0; shardNumber < numberOfShards; shardNumber++) { + ShardId shardId = new ShardId("test", "_na_", shardNumber); + shardsInBatch.add(shardId); + } + } + private void allocateAllUnassignedBatch(final RoutingAllocation allocation) { final RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); List shardToBatch = new ArrayList<>(); @@ -115,8 +127,6 @@ public void testAsyncFetchWithNoShardOnIndexCreation() { ); testBatchAllocator.clean(); allocateAllUnassignedBatch(allocation); - assertThat(testBatchAllocator.getFetchDataCalledAndClean(), equalTo(false)); - assertThat(testBatchAllocator.getShardEligibleFetchDataCountAndClean(), equalTo(0)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(1)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).get(0).shardId(), equalTo(shardId)); } @@ -634,6 +644,60 @@ public void testDoNotCancelForBrokenNode() { assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED), empty()); } + public void testAllocateUnassignedBatchThrottlingAllocationDeciderIsHonoured() throws InterruptedException { + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + AllocationDeciders allocationDeciders = randomAllocationDeciders( + Settings.builder() + .put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES_SETTING.getKey(), 1) + .put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING.getKey(), 1) + .build(), + clusterSettings, + random() + ); + setUpShards(2); + final RoutingAllocation routingAllocation = twoPrimaryAndOneUnAssignedReplica(allocationDeciders); + for (ShardId shardIdFromBatch : shardsInBatch) { + testBatchAllocator.addShardData( + node1, + shardIdFromBatch, + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ) + .addShardData( + node2, + shardIdFromBatch, + "NO_MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ) + .addShardData( + node3, + shardIdFromBatch, + "MATCH", + null, + new StoreFileMetadata("file1", 10, "MATCH_CHECKSUM", MIN_SUPPORTED_LUCENE_VERSION) + ); + } + allocateAllUnassignedBatch(routingAllocation); + // Verify the throttling decider was throttled, incoming recoveries on a node should be + // lesser than or equal to allowed concurrent recoveries + assertEquals(0, routingAllocation.routingNodes().getIncomingRecoveries(node2.getId())); + assertEquals(1, routingAllocation.routingNodes().getIncomingRecoveries(node3.getId())); + List initializingShards = routingAllocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING); + assertEquals(1, initializingShards.size()); + List ignoredShardRoutings = routingAllocation.routingNodes().unassigned().ignored(); + assertEquals(1, ignoredShardRoutings.size()); + // Allocation status for ignored replicas shards is not updated after running the deciders they just get marked as ignored. + assertEquals(UnassignedInfo.AllocationStatus.NO_ATTEMPT, ignoredShardRoutings.get(0).unassignedInfo().getLastAllocationStatus()); + AllocateUnassignedDecision allocateUnassignedDecision = testBatchAllocator.makeAllocationDecision( + ignoredShardRoutings.get(0), + routingAllocation, + logger + ); + assertEquals(UnassignedInfo.AllocationStatus.DECIDERS_THROTTLED, allocateUnassignedDecision.getAllocationStatus()); + } + private RoutingAllocation onePrimaryOnNode1And1Replica(AllocationDeciders deciders) { return onePrimaryOnNode1And1Replica(deciders, Settings.EMPTY, UnassignedInfo.Reason.CLUSTER_RECOVERED); } @@ -692,6 +756,77 @@ private RoutingAllocation onePrimaryOnNode1And1Replica(AllocationDeciders decide ); } + private RoutingAllocation twoPrimaryAndOneUnAssignedReplica(AllocationDeciders deciders) throws InterruptedException { + Map shardIdShardRoutingMap = new HashMap<>(); + Index index = shardId.getIndex(); + + // Created started ShardRouting for each primary shards + for (ShardId shardIdFromBatch : shardsInBatch) { + shardIdShardRoutingMap.put( + shardIdFromBatch, + TestShardRouting.newShardRouting(shardIdFromBatch, node1.getId(), true, ShardRoutingState.STARTED) + ); + } + + // Create Index Metadata + IndexMetadata.Builder indexMetadata = IndexMetadata.builder(index.getName()) + .settings(settings(Version.CURRENT).put(Settings.EMPTY)) + .numberOfShards(2) + .numberOfReplicas(1); + for (ShardId shardIdFromBatch : shardsInBatch) { + indexMetadata.putInSyncAllocationIds( + shardIdFromBatch.id(), + Sets.newHashSet(shardIdShardRoutingMap.get(shardIdFromBatch).allocationId().getId()) + ); + } + Metadata metadata = Metadata.builder().put(indexMetadata).build(); + + // Create Index Routing table + IndexRoutingTable.Builder indexRoutingTableBuilder = IndexRoutingTable.builder(index); + for (ShardId shardIdFromBatch : shardsInBatch) { + IndexShardRoutingTable.Builder indexShardRoutingTableBuilder = new IndexShardRoutingTable.Builder(shardIdFromBatch); + // Add a primary shard in started state + indexShardRoutingTableBuilder.addShard(shardIdShardRoutingMap.get(shardIdFromBatch)); + // Add replicas of primary shard in un-assigned state. + for (int i = 0; i < 1; i++) { + indexShardRoutingTableBuilder.addShard( + ShardRouting.newUnassigned( + shardIdFromBatch, + false, + RecoverySource.PeerRecoverySource.INSTANCE, + new UnassignedInfo( + UnassignedInfo.Reason.CLUSTER_RECOVERED, + null, + null, + 0, + System.nanoTime(), + System.currentTimeMillis(), + false, + UnassignedInfo.AllocationStatus.NO_ATTEMPT, + Collections.emptySet() + ) + ) + ); + } + indexRoutingTableBuilder.addIndexShard(indexShardRoutingTableBuilder.build()); + } + + RoutingTable routingTable = RoutingTable.builder().add(indexRoutingTableBuilder.build()).build(); + ClusterState state = ClusterState.builder(org.opensearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(routingTable) + .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)) + .build(); + return new RoutingAllocation( + deciders, + new RoutingNodes(state, false), + state, + ClusterInfo.EMPTY, + SnapshotShardSizeInfo.EMPTY, + System.nanoTime() + ); + } + private RoutingAllocation onePrimaryOnNode1And1ReplicaRecovering(AllocationDeciders deciders, UnassignedInfo unassignedInfo) { ShardRouting primaryShard = TestShardRouting.newShardRouting(shardId, node1.getId(), true, ShardRoutingState.STARTED); Metadata metadata = Metadata.builder() @@ -755,7 +890,7 @@ static String randomSyncId() { } class TestBatchAllocator extends ReplicaShardBatchAllocator { - private Map data = null; + private Map data = null; private AtomicBoolean fetchDataCalled = new AtomicBoolean(false); private AtomicInteger eligibleShardFetchDataCount = new AtomicInteger(0); @@ -800,6 +935,55 @@ public TestBatchAllocator addData( } data.put( node, + new TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch( + node, + Map.of( + shardId, + new TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata( + new TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata( + shardId, + new Store.MetadataSnapshot(unmodifiableMap(filesAsMap), unmodifiableMap(commitData), randomInt()), + peerRecoveryRetentionLeases + ), + storeFileFetchException + ) + ) + ) + ); + return this; + } + + public TestBatchAllocator addShardData( + DiscoveryNode node, + ShardId shardId, + String syncId, + @Nullable Exception storeFileFetchException, + StoreFileMetadata... files + ) { + return addShardData(node, Collections.emptyList(), shardId, syncId, storeFileFetchException, files); + } + + public TestBatchAllocator addShardData( + DiscoveryNode node, + List peerRecoveryRetentionLeases, + ShardId shardId, + String syncId, + @Nullable Exception storeFileFetchException, + StoreFileMetadata... files + ) { + if (data == null) { + data = new HashMap<>(); + } + Map filesAsMap = new HashMap<>(); + for (StoreFileMetadata file : files) { + filesAsMap.put(file.name(), file); + } + Map commitData = new HashMap<>(); + if (syncId != null) { + commitData.put(Engine.SYNC_COMMIT_ID, syncId); + } + + TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata nodeStoreFilesMetadata = new TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadata( new TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata( shardId, @@ -807,8 +991,19 @@ public TestBatchAllocator addData( peerRecoveryRetentionLeases ), storeFileFetchException - ) + ); + Map shardIdNodeStoreFilesMetadataHashMap = + new HashMap<>(); + if (data.containsKey(node)) { + NodeStoreFilesMetadataBatch nodeStoreFilesMetadataBatch = data.get(node); + shardIdNodeStoreFilesMetadataHashMap.putAll(nodeStoreFilesMetadataBatch.getNodeStoreFilesMetadataBatch()); + } + shardIdNodeStoreFilesMetadataHashMap.put(shardId, nodeStoreFilesMetadata); + data.put( + node, + new TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch(node, shardIdNodeStoreFilesMetadataHashMap) ); + return this; } @@ -820,25 +1015,7 @@ protected AsyncShardFetch.FetchResult fetchData( ) { fetchDataCalled.set(true); eligibleShardFetchDataCount.set(eligibleShards.size()); - Map tData = null; - if (data != null) { - tData = new HashMap<>(); - for (Map.Entry entry : data.entrySet()) { - Map shardData = Map.of( - shardId, - entry.getValue() - ); - tData.put( - entry.getKey(), - new TransportNodesListShardStoreMetadataBatch.NodeStoreFilesMetadataBatch(entry.getKey(), shardData) - ); - } - } - return new AsyncShardFetch.FetchResult<>(tData, new HashMap<>() { - { - put(shardId, Collections.emptySet()); - } - }); + return new AsyncShardFetch.FetchResult<>(data, Collections.>emptyMap()); } @Override From 8af4647fd5957dd2a95835dd5d7a21922808de5a Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Thu, 13 Jun 2024 12:09:59 -0400 Subject: [PATCH 05/84] Update to Gradle 8.8 (#13584) (#14182) Signed-off-by: Andriy Redko --- .../gradle/info/GlobalBuildInfoPlugin.java | 23 ++- ...nternalDistributionArchiveSetupPlugin.java | 4 +- .../DependencyLicensesPrecommitPlugin.java | 20 +- .../precommit/DependencyLicensesTask.java | 22 ++- .../gradle/precommit/ThirdPartyAuditTask.java | 13 +- .../gradle/precommit/UpdateShasTask.java | 2 +- .../gradle/tar/SymbolicLinkPreservingTar.java | 6 +- .../opensearch/gradle/util/GradleUtils.java | 26 +++ .../DependencyLicensesTaskTests.java | 2 +- .../gradle/precommit/UpdateShasTaskTests.java | 4 +- distribution/archives/build.gradle | 20 +- distribution/build.gradle | 12 +- distribution/packages/build.gradle | 60 ++++-- gradle/wrapper/gradle-wrapper.properties | 4 +- gradlew.bat | 20 +- server/build.gradle | 16 +- server/licenses/jackson-LICENSE | 8 + server/licenses/jackson-NOTICE | 20 ++ server/licenses/jackson-core-2.17.1.jar.sha1 | 1 + .../jackson-dataformat-cbor-2.17.1.jar.sha1 | 1 + .../jackson-dataformat-smile-2.17.1.jar.sha1 | 1 + .../jackson-dataformat-yaml-2.17.1.jar.sha1 | 1 + server/licenses/jopt-simple-5.0.4.jar.sha1 | 1 + server/licenses/jopt-simple-LICENSE.txt | 24 +++ server/licenses/jopt-simple-NOTICE.txt | 0 server/licenses/snakeyaml-2.1.jar.sha1 | 1 + server/licenses/snakeyaml-LICENSE.txt | 176 ++++++++++++++++++ server/licenses/snakeyaml-NOTICE.txt | 24 +++ server/licenses/zstd-jni-1.5.5-5.jar.sha1 | 1 + server/licenses/zstd-jni-LICENSE.txt | 29 +++ server/licenses/zstd-jni-NOTICE.txt | 1 + settings.gradle | 1 - 32 files changed, 467 insertions(+), 77 deletions(-) create mode 100644 server/licenses/jackson-LICENSE create mode 100644 server/licenses/jackson-NOTICE create mode 100644 server/licenses/jackson-core-2.17.1.jar.sha1 create mode 100644 server/licenses/jackson-dataformat-cbor-2.17.1.jar.sha1 create mode 100644 server/licenses/jackson-dataformat-smile-2.17.1.jar.sha1 create mode 100644 server/licenses/jackson-dataformat-yaml-2.17.1.jar.sha1 create mode 100644 server/licenses/jopt-simple-5.0.4.jar.sha1 create mode 100644 server/licenses/jopt-simple-LICENSE.txt create mode 100644 server/licenses/jopt-simple-NOTICE.txt create mode 100644 server/licenses/snakeyaml-2.1.jar.sha1 create mode 100644 server/licenses/snakeyaml-LICENSE.txt create mode 100644 server/licenses/snakeyaml-NOTICE.txt create mode 100644 server/licenses/zstd-jni-1.5.5-5.jar.sha1 create mode 100644 server/licenses/zstd-jni-LICENSE.txt create mode 100644 server/licenses/zstd-jni-NOTICE.txt diff --git a/buildSrc/src/main/java/org/opensearch/gradle/info/GlobalBuildInfoPlugin.java b/buildSrc/src/main/java/org/opensearch/gradle/info/GlobalBuildInfoPlugin.java index 448ba8a96ef02..570ab4a9f70e1 100644 --- a/buildSrc/src/main/java/org/opensearch/gradle/info/GlobalBuildInfoPlugin.java +++ b/buildSrc/src/main/java/org/opensearch/gradle/info/GlobalBuildInfoPlugin.java @@ -199,7 +199,28 @@ private JavaVersion determineJavaVersion(String description, File javaHome, Java } private JvmInstallationMetadata getJavaInstallation(File javaHome) { - final InstallationLocation location = new InstallationLocation(javaHome, "Java home"); + InstallationLocation location = null; + + try { + try { + // The InstallationLocation(File, String) is used by Gradle pre-8.8 + location = (InstallationLocation) MethodHandles.publicLookup() + .findConstructor(InstallationLocation.class, MethodType.methodType(void.class, File.class, String.class)) + .invokeExact(javaHome, "Java home"); + } catch (Throwable ex) { + // The InstallationLocation::userDefined is used by Gradle post-8.7 + location = (InstallationLocation) MethodHandles.publicLookup() + .findStatic( + InstallationLocation.class, + "userDefined", + MethodType.methodType(InstallationLocation.class, File.class, String.class) + ) + .invokeExact(javaHome, "Java home"); + + } + } catch (Throwable ex) { + throw new IllegalStateException("Unable to find suitable InstallationLocation constructor / factory method", ex); + } try { try { diff --git a/buildSrc/src/main/java/org/opensearch/gradle/internal/InternalDistributionArchiveSetupPlugin.java b/buildSrc/src/main/java/org/opensearch/gradle/internal/InternalDistributionArchiveSetupPlugin.java index 7ab91448252f2..a7f720855951a 100644 --- a/buildSrc/src/main/java/org/opensearch/gradle/internal/InternalDistributionArchiveSetupPlugin.java +++ b/buildSrc/src/main/java/org/opensearch/gradle/internal/InternalDistributionArchiveSetupPlugin.java @@ -148,8 +148,8 @@ private void configureGeneralTaskDefaults(Project project) { project.getTasks().withType(AbstractCopyTask.class).configureEach(t -> { t.dependsOn(project.getTasks().withType(EmptyDirTask.class)); t.setIncludeEmptyDirs(true); - t.setDirMode(0755); - t.setFileMode(0644); + t.dirPermissions(perms -> perms.unix(0755)); + t.filePermissions(perms -> perms.unix(0644)); }); // common config across all archives diff --git a/buildSrc/src/main/java/org/opensearch/gradle/precommit/DependencyLicensesPrecommitPlugin.java b/buildSrc/src/main/java/org/opensearch/gradle/precommit/DependencyLicensesPrecommitPlugin.java index d4dcde9d63087..28a344de31ddb 100644 --- a/buildSrc/src/main/java/org/opensearch/gradle/precommit/DependencyLicensesPrecommitPlugin.java +++ b/buildSrc/src/main/java/org/opensearch/gradle/precommit/DependencyLicensesPrecommitPlugin.java @@ -33,11 +33,14 @@ package org.opensearch.gradle.precommit; import org.opensearch.gradle.dependencies.CompileOnlyResolvePlugin; +import org.opensearch.gradle.util.GradleUtils; import org.gradle.api.Project; import org.gradle.api.Task; import org.gradle.api.artifacts.Configuration; import org.gradle.api.artifacts.ProjectDependency; +import org.gradle.api.file.FileCollection; import org.gradle.api.plugins.JavaPlugin; +import org.gradle.api.provider.Provider; import org.gradle.api.tasks.TaskProvider; public class DependencyLicensesPrecommitPlugin extends PrecommitPlugin { @@ -48,15 +51,16 @@ public TaskProvider createTask(Project project) { TaskProvider dependencyLicenses = project.getTasks() .register("dependencyLicenses", DependencyLicensesTask.class); + final Configuration runtimeClasspath = project.getConfigurations().getByName(JavaPlugin.RUNTIME_CLASSPATH_CONFIGURATION_NAME); + final Configuration compileOnly = project.getConfigurations() + .getByName(CompileOnlyResolvePlugin.RESOLVEABLE_COMPILE_ONLY_CONFIGURATION_NAME); + final Provider provider = project.provider( + () -> GradleUtils.getFiles(project, runtimeClasspath, dependency -> dependency instanceof ProjectDependency == false) + .minus(compileOnly) + ); + // only require dependency licenses for non-opensearch deps - dependencyLicenses.configure(t -> { - Configuration runtimeClasspath = project.getConfigurations().getByName(JavaPlugin.RUNTIME_CLASSPATH_CONFIGURATION_NAME); - Configuration compileOnly = project.getConfigurations() - .getByName(CompileOnlyResolvePlugin.RESOLVEABLE_COMPILE_ONLY_CONFIGURATION_NAME); - t.setDependencies( - runtimeClasspath.fileCollection(dependency -> dependency instanceof ProjectDependency == false).minus(compileOnly) - ); - }); + dependencyLicenses.configure(t -> t.getDependencies().set(provider)); // we also create the updateShas helper task that is associated with dependencyLicenses project.getTasks().register("updateShas", UpdateShasTask.class, t -> t.setParentTask(dependencyLicenses)); diff --git a/buildSrc/src/main/java/org/opensearch/gradle/precommit/DependencyLicensesTask.java b/buildSrc/src/main/java/org/opensearch/gradle/precommit/DependencyLicensesTask.java index e801681c5c386..7248e0bc14431 100644 --- a/buildSrc/src/main/java/org/opensearch/gradle/precommit/DependencyLicensesTask.java +++ b/buildSrc/src/main/java/org/opensearch/gradle/precommit/DependencyLicensesTask.java @@ -39,6 +39,7 @@ import org.gradle.api.file.FileCollection; import org.gradle.api.logging.Logger; import org.gradle.api.logging.Logging; +import org.gradle.api.provider.Property; import org.gradle.api.tasks.Input; import org.gradle.api.tasks.InputDirectory; import org.gradle.api.tasks.InputFiles; @@ -121,7 +122,7 @@ public class DependencyLicensesTask extends DefaultTask { /** * A collection of jar files that should be checked. */ - private FileCollection dependencies; + private Property dependenciesProvider; /** * The directory to find the license and sha files in. @@ -158,12 +159,11 @@ public void mapping(Map props) { } @InputFiles - public FileCollection getDependencies() { - return dependencies; - } - - public void setDependencies(FileCollection dependencies) { - this.dependencies = dependencies; + public Property getDependencies() { + if (dependenciesProvider == null) { + dependenciesProvider = getProject().getObjects().property(FileCollection.class); + } + return dependenciesProvider; } @Optional @@ -190,6 +190,11 @@ public void ignoreSha(String dep) { @TaskAction public void checkDependencies() throws IOException, NoSuchAlgorithmException { + if (dependenciesProvider == null) { + throw new GradleException("No dependencies variable defined."); + } + + final FileCollection dependencies = dependenciesProvider.get(); if (dependencies == null) { throw new GradleException("No dependencies variable defined."); } @@ -226,7 +231,7 @@ public void checkDependencies() throws IOException, NoSuchAlgorithmException { } } - checkDependencies(licenses, notices, sources, shaFiles); + checkDependencies(dependencies, licenses, notices, sources, shaFiles); licenses.forEach((item, exists) -> failIfAnyMissing(item, exists, "license")); @@ -255,6 +260,7 @@ private void failIfAnyMissing(String item, Boolean exists, String type) { } private void checkDependencies( + FileCollection dependencies, Map licenses, Map notices, Map sources, diff --git a/buildSrc/src/main/java/org/opensearch/gradle/precommit/ThirdPartyAuditTask.java b/buildSrc/src/main/java/org/opensearch/gradle/precommit/ThirdPartyAuditTask.java index efcd01f163089..f7bb708933803 100644 --- a/buildSrc/src/main/java/org/opensearch/gradle/precommit/ThirdPartyAuditTask.java +++ b/buildSrc/src/main/java/org/opensearch/gradle/precommit/ThirdPartyAuditTask.java @@ -37,6 +37,7 @@ import org.opensearch.gradle.LoggedExec; import org.opensearch.gradle.OS; import org.opensearch.gradle.dependencies.CompileOnlyResolvePlugin; +import org.opensearch.gradle.util.GradleUtils; import org.gradle.api.DefaultTask; import org.gradle.api.JavaVersion; import org.gradle.api.artifacts.Configuration; @@ -203,11 +204,13 @@ public Set getJarsToScan() { // or dependencies added as `files(...)`, we can't be sure if those are third party or not. // err on the side of scanning these to make sure we don't miss anything Spec reallyThirdParty = dep -> dep.getGroup() != null && dep.getGroup().startsWith("org.opensearch") == false; - Set jars = getRuntimeConfiguration().getResolvedConfiguration().getFiles(reallyThirdParty); - Set compileOnlyConfiguration = getProject().getConfigurations() - .getByName(CompileOnlyResolvePlugin.RESOLVEABLE_COMPILE_ONLY_CONFIGURATION_NAME) - .getResolvedConfiguration() - .getFiles(reallyThirdParty); + + Set jars = GradleUtils.getFiles(getProject(), getRuntimeConfiguration(), reallyThirdParty).getFiles(); + Set compileOnlyConfiguration = GradleUtils.getFiles( + getProject(), + getProject().getConfigurations().getByName(CompileOnlyResolvePlugin.RESOLVEABLE_COMPILE_ONLY_CONFIGURATION_NAME), + reallyThirdParty + ).getFiles(); // don't scan provided dependencies that we already scanned, e.x. don't scan cores dependencies for every plugin if (compileOnlyConfiguration != null) { jars.removeAll(compileOnlyConfiguration); diff --git a/buildSrc/src/main/java/org/opensearch/gradle/precommit/UpdateShasTask.java b/buildSrc/src/main/java/org/opensearch/gradle/precommit/UpdateShasTask.java index 3fe08888afb09..de479f3b560b6 100644 --- a/buildSrc/src/main/java/org/opensearch/gradle/precommit/UpdateShasTask.java +++ b/buildSrc/src/main/java/org/opensearch/gradle/precommit/UpdateShasTask.java @@ -66,7 +66,7 @@ public UpdateShasTask() { public void updateShas() throws NoSuchAlgorithmException, IOException { Set shaFiles = parentTask.get().getShaFiles(); - for (File dependency : parentTask.get().getDependencies()) { + for (File dependency : parentTask.get().getDependencies().get()) { String jarName = dependency.getName(); File shaFile = parentTask.get().getShaFile(jarName); diff --git a/buildSrc/src/main/java/org/opensearch/gradle/tar/SymbolicLinkPreservingTar.java b/buildSrc/src/main/java/org/opensearch/gradle/tar/SymbolicLinkPreservingTar.java index e82d8ed73ced2..3352dda98ef66 100644 --- a/buildSrc/src/main/java/org/opensearch/gradle/tar/SymbolicLinkPreservingTar.java +++ b/buildSrc/src/main/java/org/opensearch/gradle/tar/SymbolicLinkPreservingTar.java @@ -184,7 +184,7 @@ private void visitSymbolicLink(final FileCopyDetailsInternal details) { visitedSymbolicLinks.add(details.getFile()); final TarArchiveEntry entry = new TarArchiveEntry(details.getRelativePath().getPathString(), TarConstants.LF_SYMLINK); entry.setModTime(getModTime(details)); - entry.setMode(UnixStat.LINK_FLAG | details.getMode()); + entry.setMode(UnixStat.LINK_FLAG | details.getPermissions().toUnixNumeric()); try { entry.setLinkName(Files.readSymbolicLink(details.getFile().toPath()).toString()); tar.putArchiveEntry(entry); @@ -197,7 +197,7 @@ private void visitSymbolicLink(final FileCopyDetailsInternal details) { private void visitDirectory(final FileCopyDetailsInternal details) { final TarArchiveEntry entry = new TarArchiveEntry(details.getRelativePath().getPathString() + "/"); entry.setModTime(getModTime(details)); - entry.setMode(UnixStat.DIR_FLAG | details.getMode()); + entry.setMode(UnixStat.DIR_FLAG | details.getPermissions().toUnixNumeric()); try { tar.putArchiveEntry(entry); tar.closeArchiveEntry(); @@ -209,7 +209,7 @@ private void visitDirectory(final FileCopyDetailsInternal details) { private void visitFile(final FileCopyDetailsInternal details) { final TarArchiveEntry entry = new TarArchiveEntry(details.getRelativePath().getPathString()); entry.setModTime(getModTime(details)); - entry.setMode(UnixStat.FILE_FLAG | details.getMode()); + entry.setMode(UnixStat.FILE_FLAG | details.getPermissions().toUnixNumeric()); entry.setSize(details.getSize()); try { tar.putArchiveEntry(entry); diff --git a/buildSrc/src/main/java/org/opensearch/gradle/util/GradleUtils.java b/buildSrc/src/main/java/org/opensearch/gradle/util/GradleUtils.java index 031fee2d1127f..428b4a16748e1 100644 --- a/buildSrc/src/main/java/org/opensearch/gradle/util/GradleUtils.java +++ b/buildSrc/src/main/java/org/opensearch/gradle/util/GradleUtils.java @@ -39,12 +39,17 @@ import org.gradle.api.UnknownTaskException; import org.gradle.api.artifacts.Configuration; import org.gradle.api.artifacts.Dependency; +import org.gradle.api.artifacts.LenientConfiguration; +import org.gradle.api.file.FileCollection; +import org.gradle.api.internal.artifacts.ivyservice.ResolvedFilesCollectingVisitor; +import org.gradle.api.internal.artifacts.ivyservice.resolveengine.artifact.SelectedArtifactSet; import org.gradle.api.plugins.JavaBasePlugin; import org.gradle.api.plugins.JavaPluginExtension; import org.gradle.api.provider.Provider; import org.gradle.api.services.BuildService; import org.gradle.api.services.BuildServiceRegistration; import org.gradle.api.services.BuildServiceRegistry; +import org.gradle.api.specs.Spec; import org.gradle.api.tasks.SourceSet; import org.gradle.api.tasks.SourceSetContainer; import org.gradle.api.tasks.TaskContainer; @@ -53,6 +58,9 @@ import org.gradle.plugins.ide.eclipse.model.EclipseModel; import org.gradle.plugins.ide.idea.model.IdeaModel; +import java.lang.invoke.MethodHandle; +import java.lang.invoke.MethodHandles; +import java.lang.invoke.MethodType; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -245,4 +253,22 @@ public static String getProjectPathFromTask(String taskPath) { int lastDelimiterIndex = taskPath.lastIndexOf(":"); return lastDelimiterIndex == 0 ? ":" : taskPath.substring(0, lastDelimiterIndex); } + + public static FileCollection getFiles(Project project, Configuration cfg, Spec spec) { + final LenientConfiguration configuration = cfg.getResolvedConfiguration().getLenientConfiguration(); + try { + // Using reflection here to cover the pre 8.7 releases (since those have no such APIs), the + // ResolverResults.LegacyResolverResults.LegacyVisitedArtifactSet::select(...) is not available + // on older versions. + final MethodHandle mh = MethodHandles.lookup() + .findVirtual(configuration.getClass(), "select", MethodType.methodType(SelectedArtifactSet.class, Spec.class)) + .bindTo(configuration); + + final ResolvedFilesCollectingVisitor visitor = new ResolvedFilesCollectingVisitor(); + ((SelectedArtifactSet) mh.invoke(spec)).visitArtifacts(visitor, false); + return project.files(visitor.getFiles()); + } catch (Throwable ex) { + return project.files(configuration.getFiles(spec)); + } + } } diff --git a/buildSrc/src/test/java/org/opensearch/gradle/precommit/DependencyLicensesTaskTests.java b/buildSrc/src/test/java/org/opensearch/gradle/precommit/DependencyLicensesTaskTests.java index bb216b27128e1..28513710470af 100644 --- a/buildSrc/src/test/java/org/opensearch/gradle/precommit/DependencyLicensesTaskTests.java +++ b/buildSrc/src/test/java/org/opensearch/gradle/precommit/DependencyLicensesTaskTests.java @@ -344,7 +344,7 @@ private TaskProvider createDependencyLicensesTask(Projec .register("dependencyLicenses", DependencyLicensesTask.class, new Action() { @Override public void execute(DependencyLicensesTask dependencyLicensesTask) { - dependencyLicensesTask.setDependencies(getDependencies(project)); + dependencyLicensesTask.getDependencies().set(getDependencies(project)); final Map mappings = new HashMap<>(); mappings.put("from", "groovy-.*"); diff --git a/buildSrc/src/test/java/org/opensearch/gradle/precommit/UpdateShasTaskTests.java b/buildSrc/src/test/java/org/opensearch/gradle/precommit/UpdateShasTaskTests.java index 2deabb752017a..15d6d6cd4c31c 100644 --- a/buildSrc/src/test/java/org/opensearch/gradle/precommit/UpdateShasTaskTests.java +++ b/buildSrc/src/test/java/org/opensearch/gradle/precommit/UpdateShasTaskTests.java @@ -102,7 +102,7 @@ public void whenDependencyExistsButShaNotThenShouldCreateNewShaFile() throws IOE public void whenDependencyAndWrongShaExistsThenShouldNotOverwriteShaFile() throws IOException, NoSuchAlgorithmException { project.getDependencies().add("someCompileConfiguration", dependency); - File groovyJar = task.getParentTask().getDependencies().getFiles().iterator().next(); + File groovyJar = task.getParentTask().getDependencies().get().getFiles().iterator().next(); String groovyShaName = groovyJar.getName() + ".sha1"; File groovySha = createFileIn(getLicensesDir(project), groovyShaName, "content"); @@ -162,7 +162,7 @@ private TaskProvider createDependencyLicensesTask(Projec .register("dependencyLicenses", DependencyLicensesTask.class, new Action() { @Override public void execute(DependencyLicensesTask dependencyLicensesTask) { - dependencyLicensesTask.setDependencies(getDependencies(project)); + dependencyLicensesTask.getDependencies().set(getDependencies(project)); } }); diff --git a/distribution/archives/build.gradle b/distribution/archives/build.gradle index 161b8008525b4..792b1ab57ddbc 100644 --- a/distribution/archives/build.gradle +++ b/distribution/archives/build.gradle @@ -39,11 +39,17 @@ CopySpec archiveFiles(CopySpec modulesFiles, String distributionType, String pla with libFiles() } into('config') { - dirMode 0750 - fileMode 0660 + dirPermissions { + unix 0750 + } + filePermissions { + unix 0660 + } with configFiles(distributionType, java) from { - dirMode 0750 + dirPermissions { + unix 0750 + } jvmOptionsDir.getParent() } } @@ -61,13 +67,17 @@ CopySpec archiveFiles(CopySpec modulesFiles, String distributionType, String pla } into('') { from { - dirMode 0755 + dirPermissions { + unix 0755 + } logsDir.getParent() } } into('') { from { - dirMode 0755 + dirPermissions { + unix 0755 + } pluginsDir.getParent() } } diff --git a/distribution/build.gradle b/distribution/build.gradle index 35ca84ca66dba..36efe2e0d45e8 100644 --- a/distribution/build.gradle +++ b/distribution/build.gradle @@ -363,9 +363,9 @@ configure(subprojects.findAll { ['archives', 'packages'].contains(it.name) }) { if (it.relativePath.segments[-2] == 'bin' || ((platform == 'darwin-x64' || platform == 'darwin-arm64') && it.relativePath.segments[-2] == 'MacOS')) { // bin files, wherever they are within modules (eg platform specific) should be executable // and MacOS is an alternative to bin on macOS - it.mode = 0755 + it.permissions(perm -> perm.unix(0755)) } else { - it.mode = 0644 + it.permissions(perm -> perm.unix(0644)) } } def buildModules = buildModulesTaskProvider @@ -413,7 +413,7 @@ configure(subprojects.findAll { ['archives', 'packages'].contains(it.name) }) { from '../src/bin' exclude '*.exe' exclude '*.bat' - eachFile { it.setMode(0755) } + eachFile { it.permissions(perm -> perm.unix(0755)) } MavenFilteringHack.filter(it, expansionsForDistribution(distributionType, java)) } // windows files, only for zip @@ -431,7 +431,7 @@ configure(subprojects.findAll { ['archives', 'packages'].contains(it.name) }) { } // module provided bin files with copySpec { - eachFile { it.setMode(0755) } + eachFile { it.permissions(perm -> perm.unix(0755)) } from project(':distribution').buildBin if (distributionType != 'zip') { exclude '*.bat' @@ -473,7 +473,7 @@ configure(subprojects.findAll { ['archives', 'packages'].contains(it.name) }) { } eachFile { FileCopyDetails details -> if (details.relativePath.segments[-2] == 'bin' || details.relativePath.segments[-1] == 'jspawnhelper') { - details.mode = 0755 + details.permissions(perm -> perm.unix(0755)) } if (details.name == 'src.zip') { details.exclude() @@ -501,7 +501,7 @@ configure(subprojects.findAll { ['archives', 'packages'].contains(it.name) }) { } eachFile { FileCopyDetails details -> if (details.relativePath.segments[-2] == 'bin' || details.relativePath.segments[-1] == 'jspawnhelper') { - details.mode = 0755 + details.permissions(perm -> perm.unix(0755)) } } } diff --git a/distribution/packages/build.gradle b/distribution/packages/build.gradle index 211b3bd55da60..621620eef9d71 100644 --- a/distribution/packages/build.gradle +++ b/distribution/packages/build.gradle @@ -160,7 +160,9 @@ Closure commonPackageConfig(String type, boolean jdk, String architecture) { } from(rootProject.projectDir) { include 'README.md' - fileMode 0644 + filePermissions { + unix 0644 + } } into('lib') { with libFiles() @@ -183,9 +185,9 @@ Closure commonPackageConfig(String type, boolean jdk, String architecture) { directory('/' + segments[0..i].join('/'), 0755) } if (segments[-2] == 'bin' || segments[-1] == 'jspawnhelper') { - fcp.mode = 0755 + fcp.permissions(perm -> perm.unix(0755)) } else { - fcp.mode = 0644 + fcp.permissions(perm -> perm.unix(0644)) } } } @@ -195,7 +197,9 @@ Closure commonPackageConfig(String type, boolean jdk, String architecture) { if (type == 'deb') { into("/usr/share/doc/${packageName}") { from "${packagingFiles}/copyright" - fileMode 0644 + filePermissions { + unix 0644 + } } } else { assert type == 'rpm' @@ -204,7 +208,9 @@ Closure commonPackageConfig(String type, boolean jdk, String architecture) { include 'APACHE-LICENSE-2.0.txt' rename { 'LICENSE.txt' } } - fileMode 0644 + filePermissions { + unix 0644 + } } } @@ -213,7 +219,9 @@ Closure commonPackageConfig(String type, boolean jdk, String architecture) { configurationFile '/etc/opensearch/jvm.options' configurationFile '/etc/opensearch/log4j2.properties' from("${packagingFiles}") { - dirMode 0750 + dirPermissions { + unix 0750 + } into('/etc') permissionGroup 'opensearch' includeEmptyDirs true @@ -223,8 +231,12 @@ Closure commonPackageConfig(String type, boolean jdk, String architecture) { } from("${packagingFiles}/etc/opensearch") { into('/etc/opensearch') - dirMode 0750 - fileMode 0660 + dirPermissions { + unix 0750 + } + filePermissions{ + unix 0660 + } permissionGroup 'opensearch' includeEmptyDirs true createDirectoryEntry true @@ -235,34 +247,46 @@ Closure commonPackageConfig(String type, boolean jdk, String architecture) { into(new File(envFile).getParent()) { fileType CONFIG | NOREPLACE permissionGroup 'opensearch' - fileMode 0660 + filePermissions { + unix 0660 + } from "${packagingFiles}/env/opensearch" } // ========= systemd ========= into('/usr/lib/tmpfiles.d') { from "${packagingFiles}/systemd/opensearch.conf" - fileMode 0644 + filePermissions { + unix 0644 + } } into('/usr/lib/systemd/system') { fileType CONFIG | NOREPLACE from "${packagingFiles}/systemd/opensearch.service" - fileMode 0644 + filePermissions { + unix 0644 + } } into('/usr/lib/sysctl.d') { fileType CONFIG | NOREPLACE from "${packagingFiles}/systemd/sysctl/opensearch.conf" - fileMode 0644 + filePermissions { + unix 0644 + } } into('/usr/share/opensearch/bin') { from "${packagingFiles}/systemd/systemd-entrypoint" - fileMode 0755 + filePermissions { + unix 0755 + } } // ========= sysV init ========= configurationFile '/etc/init.d/opensearch' into('/etc/init.d') { - fileMode 0750 + filePermissions { + unix 0750 + } fileType CONFIG | NOREPLACE from "${packagingFiles}/init.d/opensearch" } @@ -278,7 +302,9 @@ Closure commonPackageConfig(String type, boolean jdk, String architecture) { createDirectoryEntry true user u permissionGroup g - dirMode mode + dirPermissions { + unix mode + } } } copyEmptyDir('/var/log/opensearch', 'opensearch', 'opensearch', 0750) @@ -341,7 +367,9 @@ Closure commonDebConfig(boolean jdk, String architecture) { into('/usr/share/lintian/overrides') { from('src/deb/lintian/opensearch') - fileMode 0644 + filePermissions { + unix 0644 + } } } } diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index 9b0d73222260e..f5bdef81deb70 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -11,7 +11,7 @@ distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists -distributionUrl=https\://services.gradle.org/distributions/gradle-8.7-all.zip +distributionUrl=https\://services.gradle.org/distributions/gradle-8.8-all.zip zipStoreBase=GRADLE_USER_HOME zipStorePath=wrapper/dists -distributionSha256Sum=194717442575a6f96e1c1befa2c30e9a4fc90f701d7aee33eb879b79e7ff05c0 +distributionSha256Sum=f8b4f4772d302c8ff580bc40d0f56e715de69b163546944f787c87abf209c961 diff --git a/gradlew.bat b/gradlew.bat index 6689b85beecde..7101f8e4676fc 100644 --- a/gradlew.bat +++ b/gradlew.bat @@ -43,11 +43,11 @@ set JAVA_EXE=java.exe %JAVA_EXE% -version >NUL 2>&1 if %ERRORLEVEL% equ 0 goto execute -echo. -echo ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. -echo. -echo Please set the JAVA_HOME variable in your environment to match the -echo location of your Java installation. +echo. 1>&2 +echo ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. 1>&2 +echo. 1>&2 +echo Please set the JAVA_HOME variable in your environment to match the 1>&2 +echo location of your Java installation. 1>&2 goto fail @@ -57,11 +57,11 @@ set JAVA_EXE=%JAVA_HOME%/bin/java.exe if exist "%JAVA_EXE%" goto execute -echo. -echo ERROR: JAVA_HOME is set to an invalid directory: %JAVA_HOME% -echo. -echo Please set the JAVA_HOME variable in your environment to match the -echo location of your Java installation. +echo. 1>&2 +echo ERROR: JAVA_HOME is set to an invalid directory: %JAVA_HOME% 1>&2 +echo. 1>&2 +echo Please set the JAVA_HOME variable in your environment to match the 1>&2 +echo location of your Java installation. 1>&2 goto fail diff --git a/server/build.gradle b/server/build.gradle index 624e5fe332662..b8a99facbf964 100644 --- a/server/build.gradle +++ b/server/build.gradle @@ -356,14 +356,18 @@ tasks.named("thirdPartyAudit").configure { } tasks.named("dependencyLicenses").configure { + mapping from: /jackson-.*/, to: 'jackson' mapping from: /reactor-.*/, to: 'reactor' mapping from: /lucene-.*/, to: 'lucene' - dependencies = project.configurations.runtimeClasspath.fileCollection { - it.group.startsWith('org.opensearch') == false || - // keep the following org.opensearch jars in - (it.name == 'jna' || - it.name == 'securesm') - } + dependencies = project.configurations.runtimeClasspath.incoming.artifactView { + componentFilter { + it instanceof ModuleComponentIdentifier && + (it.group.startsWith('org.opensearch') == false || + // keep the following org.opensearch jars in + (it.name == 'jna' || + it.name == 'securesm')) + } + }.files } tasks.named("filepermissions").configure { diff --git a/server/licenses/jackson-LICENSE b/server/licenses/jackson-LICENSE new file mode 100644 index 0000000000000..f5f45d26a49d6 --- /dev/null +++ b/server/licenses/jackson-LICENSE @@ -0,0 +1,8 @@ +This copy of Jackson JSON processor streaming parser/generator is licensed under the +Apache (Software) License, version 2.0 ("the License"). +See the License for details about distribution rights, and the +specific rights regarding derivate works. + +You may obtain a copy of the License at: + +http://www.apache.org/licenses/LICENSE-2.0 diff --git a/server/licenses/jackson-NOTICE b/server/licenses/jackson-NOTICE new file mode 100644 index 0000000000000..4c976b7b4cc58 --- /dev/null +++ b/server/licenses/jackson-NOTICE @@ -0,0 +1,20 @@ +# Jackson JSON processor + +Jackson is a high-performance, Free/Open Source JSON processing library. +It was originally written by Tatu Saloranta (tatu.saloranta@iki.fi), and has +been in development since 2007. +It is currently developed by a community of developers, as well as supported +commercially by FasterXML.com. + +## Licensing + +Jackson core and extension components may licensed under different licenses. +To find the details that apply to this artifact see the accompanying LICENSE file. +For more information, including possible other licensing options, contact +FasterXML.com (http://fasterxml.com). + +## Credits + +A list of contributors may be found from CREDITS file, which is included +in some artifacts (usually source distributions); but is always available +from the source code management (SCM) system project uses. diff --git a/server/licenses/jackson-core-2.17.1.jar.sha1 b/server/licenses/jackson-core-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..82dab5981e652 --- /dev/null +++ b/server/licenses/jackson-core-2.17.1.jar.sha1 @@ -0,0 +1 @@ +5e52a11644cd59a28ef79f02bddc2cc3bab45edb \ No newline at end of file diff --git a/server/licenses/jackson-dataformat-cbor-2.17.1.jar.sha1 b/server/licenses/jackson-dataformat-cbor-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..ff42ed1f92cfe --- /dev/null +++ b/server/licenses/jackson-dataformat-cbor-2.17.1.jar.sha1 @@ -0,0 +1 @@ +ba5d8e6ecc62aa0e49c0ce935b8696352dbebc71 \ No newline at end of file diff --git a/server/licenses/jackson-dataformat-smile-2.17.1.jar.sha1 b/server/licenses/jackson-dataformat-smile-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..47d19067cf2a6 --- /dev/null +++ b/server/licenses/jackson-dataformat-smile-2.17.1.jar.sha1 @@ -0,0 +1 @@ +89683ac4f0a0c2c4f69ea56b90480ed40266dac8 \ No newline at end of file diff --git a/server/licenses/jackson-dataformat-yaml-2.17.1.jar.sha1 b/server/licenses/jackson-dataformat-yaml-2.17.1.jar.sha1 new file mode 100644 index 0000000000000..7946e994c7104 --- /dev/null +++ b/server/licenses/jackson-dataformat-yaml-2.17.1.jar.sha1 @@ -0,0 +1 @@ +b4c7b8a9ea3f398116a75c146b982b22afebc4ee \ No newline at end of file diff --git a/server/licenses/jopt-simple-5.0.4.jar.sha1 b/server/licenses/jopt-simple-5.0.4.jar.sha1 new file mode 100644 index 0000000000000..7ade81efe4d0d --- /dev/null +++ b/server/licenses/jopt-simple-5.0.4.jar.sha1 @@ -0,0 +1 @@ +4fdac2fbe92dfad86aa6e9301736f6b4342a3f5c \ No newline at end of file diff --git a/server/licenses/jopt-simple-LICENSE.txt b/server/licenses/jopt-simple-LICENSE.txt new file mode 100644 index 0000000000000..85f923a95268a --- /dev/null +++ b/server/licenses/jopt-simple-LICENSE.txt @@ -0,0 +1,24 @@ +/* + The MIT License + + Copyright (c) 2004-2015 Paul R. Holser, Jr. + + Permission is hereby granted, free of charge, to any person obtaining + a copy of this software and associated documentation files (the + "Software"), to deal in the Software without restriction, including + without limitation the rights to use, copy, modify, merge, publish, + distribute, sublicense, and/or sell copies of the Software, and to + permit persons to whom the Software is furnished to do so, subject to + the following conditions: + + The above copyright notice and this permission notice shall be + included in all copies or substantial portions of the Software. + + THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, + EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF + MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND + NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE + LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION + OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION + WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. +*/ diff --git a/server/licenses/jopt-simple-NOTICE.txt b/server/licenses/jopt-simple-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/server/licenses/snakeyaml-2.1.jar.sha1 b/server/licenses/snakeyaml-2.1.jar.sha1 new file mode 100644 index 0000000000000..5586b210a9736 --- /dev/null +++ b/server/licenses/snakeyaml-2.1.jar.sha1 @@ -0,0 +1 @@ +c79f47315517560b5bd6a62376ee385e48105437 \ No newline at end of file diff --git a/server/licenses/snakeyaml-LICENSE.txt b/server/licenses/snakeyaml-LICENSE.txt new file mode 100644 index 0000000000000..d9a10c0d8e868 --- /dev/null +++ b/server/licenses/snakeyaml-LICENSE.txt @@ -0,0 +1,176 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS diff --git a/server/licenses/snakeyaml-NOTICE.txt b/server/licenses/snakeyaml-NOTICE.txt new file mode 100644 index 0000000000000..b51464eee1f00 --- /dev/null +++ b/server/licenses/snakeyaml-NOTICE.txt @@ -0,0 +1,24 @@ +***The art of simplicity is a puzzle of complexity.*** + +## Overview ## +[YAML](http://yaml.org) is a data serialization format designed for human readability and interaction with scripting languages. + +SnakeYAML is a YAML processor for the Java Virtual Machine. + +## SnakeYAML features ## + +* a **complete** [YAML 1.1 processor](http://yaml.org/spec/1.1/current.html). In particular, SnakeYAML can parse all examples from the specification. +* Unicode support including UTF-8/UTF-16 input/output. +* high-level API for serializing and deserializing native Java objects. +* support for all types from the [YAML types repository](http://yaml.org/type/index.html). +* relatively sensible error messages. + +## Info ## + * [Changes](https://bitbucket.org/asomov/snakeyaml/wiki/Changes) + * [Documentation](https://bitbucket.org/asomov/snakeyaml/wiki/Documentation) + +## Contribute ## +* Mercurial DVCS is used to dance with the [source code](https://bitbucket.org/asomov/snakeyaml/src). +* If you find a bug in SnakeYAML, please [file a bug report](https://bitbucket.org/asomov/snakeyaml/issues?status=new&status=open). +* You may discuss SnakeYAML at +[the mailing list](http://groups.google.com/group/snakeyaml-core). \ No newline at end of file diff --git a/server/licenses/zstd-jni-1.5.5-5.jar.sha1 b/server/licenses/zstd-jni-1.5.5-5.jar.sha1 new file mode 100644 index 0000000000000..498c60c34e3da --- /dev/null +++ b/server/licenses/zstd-jni-1.5.5-5.jar.sha1 @@ -0,0 +1 @@ +74ffdc5f140080adacf5278287aadd950179f848 \ No newline at end of file diff --git a/server/licenses/zstd-jni-LICENSE.txt b/server/licenses/zstd-jni-LICENSE.txt new file mode 100644 index 0000000000000..c4dd507c1c72f --- /dev/null +++ b/server/licenses/zstd-jni-LICENSE.txt @@ -0,0 +1,29 @@ +----------------------------------------------------------------------------- +** Beginning of "BSD License" text. ** + +Zstd-jni: JNI bindings to Zstd Library + +Copyright (c) 2015-present, Luben Karavelov/ All rights reserved. + +BSD License + +Redistribution and use in source and binary forms, with or without modification, +are permitted provided that the following conditions are met: + +1. Redistributions of source code must retain the above copyright notice, this + list of conditions and the following disclaimer. + +2. Redistributions in binary form must reproduce the above copyright notice, this + list of conditions and the following disclaimer in the documentation and/or + other materials provided with the distribution. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" AND +ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED +WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE +DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE LIABLE FOR +ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES +(INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON +ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/server/licenses/zstd-jni-NOTICE.txt b/server/licenses/zstd-jni-NOTICE.txt new file mode 100644 index 0000000000000..389c97cbc892d --- /dev/null +++ b/server/licenses/zstd-jni-NOTICE.txt @@ -0,0 +1 @@ +The code for the JNI bindings to Zstd library was originally authored by Luben Karavelov diff --git a/settings.gradle b/settings.gradle index ca8538a967ef7..888ecc62aac7c 100644 --- a/settings.gradle +++ b/settings.gradle @@ -18,7 +18,6 @@ ext.disableBuildCache = hasProperty('DISABLE_BUILD_CACHE') || System.getenv().co buildCache { local { enabled = !disableBuildCache - removeUnusedEntriesAfterDays = 14 } } From 71dbc22da2bde2af9786eb47943400e98bad7cbe Mon Sep 17 00:00:00 2001 From: Gaurav Bafna <85113518+gbbafna@users.noreply.github.com> Date: Thu, 13 Jun 2024 22:59:07 +0530 Subject: [PATCH 06/84] Revert "[Remote Store] Fix sleep time bug during remote store sync (#14037)" (#14274) This reverts commit afeddc228ba7791a549fb7c6ef94349d432c0824. Signed-off-by: Gaurav Bafna --- .../MigrationBaseTestCase.java | 32 ++----------------- .../RemotePrimaryRelocationIT.java | 23 +++++++++++-- .../opensearch/index/shard/IndexShard.java | 2 +- 3 files changed, 25 insertions(+), 32 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java index 9dcbe380477dc..901b36f872622 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java @@ -9,8 +9,6 @@ package org.opensearch.remotemigration; import org.opensearch.action.DocWriteResponse; -import org.opensearch.action.admin.cluster.health.ClusterHealthRequest; -import org.opensearch.action.admin.cluster.health.ClusterHealthResponse; import org.opensearch.action.admin.cluster.repositories.get.GetRepositoriesRequest; import org.opensearch.action.admin.cluster.repositories.get.GetRepositoriesResponse; import org.opensearch.action.bulk.BulkRequest; @@ -18,15 +16,11 @@ import org.opensearch.action.delete.DeleteResponse; import org.opensearch.action.index.IndexRequest; import org.opensearch.action.index.IndexResponse; -import org.opensearch.client.Requests; import org.opensearch.cluster.ClusterState; -import org.opensearch.cluster.health.ClusterHealthStatus; import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.cluster.routing.RoutingNode; -import org.opensearch.common.Priority; import org.opensearch.common.UUIDs; import org.opensearch.common.settings.Settings; -import org.opensearch.common.unit.TimeValue; import org.opensearch.repositories.fs.ReloadableFsRepository; import org.opensearch.test.OpenSearchIntegTestCase; import org.junit.Before; @@ -45,7 +39,6 @@ import static org.opensearch.node.remotestore.RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING; import static org.opensearch.repositories.fs.ReloadableFsRepository.REPOSITORIES_FAILRATE_SETTING; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; -import static org.hamcrest.Matchers.equalTo; public class MigrationBaseTestCase extends OpenSearchIntegTestCase { protected static final String REPOSITORY_NAME = "test-remote-store-repo"; @@ -121,10 +114,6 @@ public void initDocRepToRemoteMigration() { ); } - public ClusterHealthStatus ensureGreen(String... indices) { - return ensureGreen(TimeValue.timeValueSeconds(60), indices); - } - public BulkResponse indexBulk(String indexName, int numDocs) { BulkRequest bulkRequest = new BulkRequest(); for (int i = 0; i < numDocs; i++) { @@ -192,12 +181,14 @@ private Thread getIndexingThread() { long currentDocCount = indexedDocs.incrementAndGet(); if (currentDocCount > 0 && currentDocCount % refreshFrequency == 0) { if (rarely()) { + logger.info("--> [iteration {}] flushing index", currentDocCount); client().admin().indices().prepareFlush(indexName).get(); - logger.info("Completed ingestion of {} docs. Flushing now", currentDocCount); } else { + logger.info("--> [iteration {}] refreshing index", currentDocCount); client().admin().indices().prepareRefresh(indexName).get(); } } + logger.info("Completed ingestion of {} docs", currentDocCount); } }); } @@ -227,21 +218,4 @@ public void stopShardRebalancing() { .get() ); } - - public ClusterHealthStatus waitForRelocation() { - ClusterHealthRequest request = Requests.clusterHealthRequest() - .waitForNoRelocatingShards(true) - .timeout(TimeValue.timeValueSeconds(60)) - .waitForEvents(Priority.LANGUID); - ClusterHealthResponse actionGet = client().admin().cluster().health(request).actionGet(); - if (actionGet.isTimedOut()) { - logger.info( - "waitForRelocation timed out, cluster state:\n{}\n{}", - client().admin().cluster().prepareState().get().getState(), - client().admin().cluster().preparePendingClusterTasks().get() - ); - assertThat("timed out waiting for relocation", actionGet.isTimedOut(), equalTo(false)); - } - return actionGet.getStatus(); - } } diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemotePrimaryRelocationIT.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemotePrimaryRelocationIT.java index fa3b9368ded47..cea653c0ead4b 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemotePrimaryRelocationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemotePrimaryRelocationIT.java @@ -99,7 +99,16 @@ public void testRemotePrimaryRelocation() throws Exception { .add(new MoveAllocationCommand("test", 0, primaryNodeName("test"), remoteNode)) .execute() .actionGet(); - waitForRelocation(); + ClusterHealthResponse clusterHealthResponse = client().admin() + .cluster() + .prepareHealth() + .setTimeout(TimeValue.timeValueSeconds(60)) + .setWaitForEvents(Priority.LANGUID) + .setWaitForNoRelocatingShards(true) + .execute() + .actionGet(); + + assertEquals(0, clusterHealthResponse.getRelocatingShards()); assertEquals(remoteNode, primaryNodeName("test")); logger.info("--> relocation from docrep to remote complete"); @@ -114,7 +123,16 @@ public void testRemotePrimaryRelocation() throws Exception { .add(new MoveAllocationCommand("test", 0, remoteNode, remoteNode2)) .execute() .actionGet(); - waitForRelocation(); + clusterHealthResponse = client().admin() + .cluster() + .prepareHealth() + .setTimeout(TimeValue.timeValueSeconds(60)) + .setWaitForEvents(Priority.LANGUID) + .setWaitForNoRelocatingShards(true) + .execute() + .actionGet(); + + assertEquals(0, clusterHealthResponse.getRelocatingShards()); assertEquals(remoteNode2, primaryNodeName("test")); logger.info("--> relocation from remote to remote complete"); @@ -137,6 +155,7 @@ public void testRemotePrimaryRelocation() throws Exception { public void testMixedModeRelocation_RemoteSeedingFail() throws Exception { String docRepNode = internalCluster().startNode(); + Client client = internalCluster().client(docRepNode); ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest(); updateSettingsRequest.persistentSettings(Settings.builder().put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), "mixed")); assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index 82b68b32f3bf8..49cb710c915fc 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -2146,7 +2146,7 @@ public void waitForRemoteStoreSync(Runnable onProgress) throws IOException { segmentUploadeCount = directory.getSegmentsUploadedToRemoteStore().size(); } try { - Thread.sleep(TimeValue.timeValueSeconds(30).millis()); + Thread.sleep(TimeValue.timeValueSeconds(30).seconds()); } catch (InterruptedException ie) { throw new OpenSearchException("Interrupted waiting for completion of [{}]", ie); } From 79405edcdb9d814528a7b33253942a134377b774 Mon Sep 17 00:00:00 2001 From: Rishabh Maurya Date: Thu, 13 Jun 2024 12:39:09 -0700 Subject: [PATCH 07/84] Add the match_only_text rewrite fix to release notes (#14286) Signed-off-by: Rishabh Maurya --- release-notes/opensearch.release-notes-2.15.0.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/release-notes/opensearch.release-notes-2.15.0.md b/release-notes/opensearch.release-notes-2.15.0.md index 02458b0c89b7d..e3b7cfc0558f3 100644 --- a/release-notes/opensearch.release-notes-2.15.0.md +++ b/release-notes/opensearch.release-notes-2.15.0.md @@ -71,4 +71,5 @@ - Fix ReplicaShardBatchAllocator to batch shards without duplicates ([#13710](https://github.com/opensearch-project/OpenSearch/pull/13710)) - Java high-level REST client bulk() is not respecting the bulkRequest.requireAlias(true) method call ([#14146](https://github.com/opensearch-project/OpenSearch/pull/14146)) - Fix ShardNotFoundException during request cache clean up ([#14219](https://github.com/opensearch-project/OpenSearch/pull/14219)) -- Fix Concurrent Modification Exception in Indices Request Cache([#14032](https://github.com/opensearch-project/OpenSearch/pull/14221)) \ No newline at end of file +- Fix Concurrent Modification Exception in Indices Request Cache([#14032](https://github.com/opensearch-project/OpenSearch/pull/14221)) +- Fix the rewrite method for MatchOnlyText field query ([#14248](https://github.com/opensearch-project/OpenSearch/pull/14248)) From 5f2bff3521b17b003e86df47b363b5ca0b8ce045 Mon Sep 17 00:00:00 2001 From: Gaurav Bafna <85113518+gbbafna@users.noreply.github.com> Date: Fri, 14 Jun 2024 17:01:02 +0530 Subject: [PATCH 08/84] [Remote Store] Fix sleep time bug during remote store sync (#14342) * [Remote Store] Fix sleep time bug during remote store sync (#14037) --------- Signed-off-by: Gaurav Bafna * Fix remote migration ITs Signed-off-by: Gaurav Bafna --------- Signed-off-by: Gaurav Bafna --- .../MigrationBaseTestCase.java | 49 +++++++++++++++++-- .../RemotePrimaryRelocationIT.java | 23 +-------- .../RemoteReplicaRecoveryIT.java | 25 +--------- .../RemoteStoreMigrationTestCase.java | 24 +++++---- .../opensearch/index/shard/IndexShard.java | 2 +- 5 files changed, 62 insertions(+), 61 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java index 901b36f872622..5be9b25512704 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java @@ -9,6 +9,8 @@ package org.opensearch.remotemigration; import org.opensearch.action.DocWriteResponse; +import org.opensearch.action.admin.cluster.health.ClusterHealthRequest; +import org.opensearch.action.admin.cluster.health.ClusterHealthResponse; import org.opensearch.action.admin.cluster.repositories.get.GetRepositoriesRequest; import org.opensearch.action.admin.cluster.repositories.get.GetRepositoriesResponse; import org.opensearch.action.bulk.BulkRequest; @@ -16,11 +18,15 @@ import org.opensearch.action.delete.DeleteResponse; import org.opensearch.action.index.IndexRequest; import org.opensearch.action.index.IndexResponse; +import org.opensearch.client.Requests; import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.health.ClusterHealthStatus; import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.cluster.routing.RoutingNode; +import org.opensearch.common.Priority; import org.opensearch.common.UUIDs; import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; import org.opensearch.repositories.fs.ReloadableFsRepository; import org.opensearch.test.OpenSearchIntegTestCase; import org.junit.Before; @@ -39,6 +45,7 @@ import static org.opensearch.node.remotestore.RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING; import static org.opensearch.repositories.fs.ReloadableFsRepository.REPOSITORIES_FAILRATE_SETTING; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; +import static org.hamcrest.Matchers.equalTo; public class MigrationBaseTestCase extends OpenSearchIntegTestCase { protected static final String REPOSITORY_NAME = "test-remote-store-repo"; @@ -114,6 +121,10 @@ public void initDocRepToRemoteMigration() { ); } + public ClusterHealthStatus ensureGreen(String... indices) { + return ensureGreen(TimeValue.timeValueSeconds(60), indices); + } + public BulkResponse indexBulk(String indexName, int numDocs) { BulkRequest bulkRequest = new BulkRequest(); for (int i = 0; i < numDocs; i++) { @@ -181,14 +192,12 @@ private Thread getIndexingThread() { long currentDocCount = indexedDocs.incrementAndGet(); if (currentDocCount > 0 && currentDocCount % refreshFrequency == 0) { if (rarely()) { - logger.info("--> [iteration {}] flushing index", currentDocCount); client().admin().indices().prepareFlush(indexName).get(); + logger.info("Completed ingestion of {} docs. Flushing now", currentDocCount); } else { - logger.info("--> [iteration {}] refreshing index", currentDocCount); client().admin().indices().prepareRefresh(indexName).get(); } } - logger.info("Completed ingestion of {} docs", currentDocCount); } }); } @@ -218,4 +227,38 @@ public void stopShardRebalancing() { .get() ); } + + public ClusterHealthStatus waitForRelocation() { + ClusterHealthRequest request = Requests.clusterHealthRequest() + .waitForNoRelocatingShards(true) + .timeout(TimeValue.timeValueSeconds(60)) + .waitForEvents(Priority.LANGUID); + ClusterHealthResponse actionGet = client().admin().cluster().health(request).actionGet(); + if (actionGet.isTimedOut()) { + logger.info( + "waitForRelocation timed out, cluster state:\n{}\n{}", + client().admin().cluster().prepareState().get().getState(), + client().admin().cluster().preparePendingClusterTasks().get() + ); + assertThat("timed out waiting for relocation", actionGet.isTimedOut(), equalTo(false)); + } + return actionGet.getStatus(); + } + + public ClusterHealthStatus waitForRelocation(TimeValue t) { + ClusterHealthRequest request = Requests.clusterHealthRequest() + .waitForNoRelocatingShards(true) + .timeout(t) + .waitForEvents(Priority.LANGUID); + ClusterHealthResponse actionGet = client().admin().cluster().health(request).actionGet(); + if (actionGet.isTimedOut()) { + logger.info( + "waitForRelocation timed out, cluster state:\n{}\n{}", + client().admin().cluster().prepareState().get().getState(), + client().admin().cluster().preparePendingClusterTasks().get() + ); + assertThat("timed out waiting for relocation", actionGet.isTimedOut(), equalTo(false)); + } + return actionGet.getStatus(); + } } diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemotePrimaryRelocationIT.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemotePrimaryRelocationIT.java index cea653c0ead4b..fa3b9368ded47 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemotePrimaryRelocationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemotePrimaryRelocationIT.java @@ -99,16 +99,7 @@ public void testRemotePrimaryRelocation() throws Exception { .add(new MoveAllocationCommand("test", 0, primaryNodeName("test"), remoteNode)) .execute() .actionGet(); - ClusterHealthResponse clusterHealthResponse = client().admin() - .cluster() - .prepareHealth() - .setTimeout(TimeValue.timeValueSeconds(60)) - .setWaitForEvents(Priority.LANGUID) - .setWaitForNoRelocatingShards(true) - .execute() - .actionGet(); - - assertEquals(0, clusterHealthResponse.getRelocatingShards()); + waitForRelocation(); assertEquals(remoteNode, primaryNodeName("test")); logger.info("--> relocation from docrep to remote complete"); @@ -123,16 +114,7 @@ public void testRemotePrimaryRelocation() throws Exception { .add(new MoveAllocationCommand("test", 0, remoteNode, remoteNode2)) .execute() .actionGet(); - clusterHealthResponse = client().admin() - .cluster() - .prepareHealth() - .setTimeout(TimeValue.timeValueSeconds(60)) - .setWaitForEvents(Priority.LANGUID) - .setWaitForNoRelocatingShards(true) - .execute() - .actionGet(); - - assertEquals(0, clusterHealthResponse.getRelocatingShards()); + waitForRelocation(); assertEquals(remoteNode2, primaryNodeName("test")); logger.info("--> relocation from remote to remote complete"); @@ -155,7 +137,6 @@ public void testRemotePrimaryRelocation() throws Exception { public void testMixedModeRelocation_RemoteSeedingFail() throws Exception { String docRepNode = internalCluster().startNode(); - Client client = internalCluster().client(docRepNode); ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest(); updateSettingsRequest.persistentSettings(Settings.builder().put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), "mixed")); assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteReplicaRecoveryIT.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteReplicaRecoveryIT.java index aae726fe2a6bc..d6e25c0cab3ac 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteReplicaRecoveryIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteReplicaRecoveryIT.java @@ -8,15 +8,12 @@ package org.opensearch.remotemigration; -import org.opensearch.action.admin.cluster.health.ClusterHealthResponse; import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; import org.opensearch.action.admin.indices.replication.SegmentReplicationStatsResponse; import org.opensearch.action.admin.indices.settings.put.UpdateSettingsRequest; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.routing.allocation.command.MoveAllocationCommand; -import org.opensearch.common.Priority; import org.opensearch.common.settings.Settings; -import org.opensearch.common.unit.TimeValue; import org.opensearch.index.SegmentReplicationPerGroupStats; import org.opensearch.index.query.QueryBuilders; import org.opensearch.test.OpenSearchIntegTestCase; @@ -83,16 +80,8 @@ public void testReplicaRecovery() throws Exception { .add(new MoveAllocationCommand("test", 0, primaryNode, remoteNode)) .execute() .actionGet(); - ClusterHealthResponse clusterHealthResponse = client().admin() - .cluster() - .prepareHealth() - .setTimeout(TimeValue.timeValueSeconds(60)) - .setWaitForEvents(Priority.LANGUID) - .setWaitForNoRelocatingShards(true) - .execute() - .actionGet(); - assertEquals(0, clusterHealthResponse.getRelocatingShards()); + waitForRelocation(); logger.info("--> relocation of primary from docrep to remote complete"); logger.info("--> getting up the new replicas now to doc rep node as well as remote node "); @@ -109,17 +98,7 @@ public void testReplicaRecovery() throws Exception { ) .get(); - client().admin() - .cluster() - .prepareHealth() - .setTimeout(TimeValue.timeValueSeconds(60)) - .setWaitForEvents(Priority.LANGUID) - .setWaitForGreenStatus() - .execute() - .actionGet(); - logger.info("--> replica is up now on another docrep now as well as remote node"); - - assertEquals(0, clusterHealthResponse.getRelocatingShards()); + waitForRelocation(); asyncIndexingService.stopIndexing(); refresh("test"); diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteStoreMigrationTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteStoreMigrationTestCase.java index 4e4f6da56d622..e0e25db4ca722 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteStoreMigrationTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteStoreMigrationTestCase.java @@ -8,13 +8,11 @@ package org.opensearch.remotemigration; -import org.opensearch.action.admin.cluster.health.ClusterHealthResponse; import org.opensearch.action.admin.cluster.repositories.get.GetRepositoriesRequest; import org.opensearch.action.admin.cluster.repositories.get.GetRepositoriesResponse; import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; import org.opensearch.client.Client; import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.common.Priority; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.FeatureFlags; @@ -28,6 +26,7 @@ import java.util.List; import java.util.Map; +import static org.opensearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES_SETTING; 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; @@ -48,6 +47,10 @@ protected Settings featureFlagSettings() { return Settings.builder().put(super.featureFlagSettings()).put(FeatureFlags.REMOTE_STORE_MIGRATION_EXPERIMENTAL, "true").build(); } + protected int maximumNumberOfShards() { + return 5; + } + public void testMixedModeAddRemoteNodes() throws Exception { internalCluster().setBootstrapClusterManagerNodeIndex(0); List cmNodes = internalCluster().startNodes(1); @@ -155,7 +158,11 @@ public void testEndToEndRemoteMigration() throws Exception { internalCluster().setBootstrapClusterManagerNodeIndex(0); List docRepNodes = internalCluster().startNodes(2); ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest(); - updateSettingsRequest.persistentSettings(Settings.builder().put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), "mixed")); + updateSettingsRequest.persistentSettings( + Settings.builder() + .put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), "mixed") + .put(CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES_SETTING.getKey(), maximumNumberOfShards()) + ); assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); client().admin().indices().prepareCreate("test").setSettings(indexSettings()).setMapping("field", "type=text").get(); ensureGreen("test"); @@ -189,16 +196,7 @@ public void testEndToEndRemoteMigration() throws Exception { ) .get() ); - - ClusterHealthResponse clusterHealthResponse = client().admin() - .cluster() - .prepareHealth() - .setTimeout(TimeValue.timeValueSeconds(45)) - .setWaitForEvents(Priority.LANGUID) - .setWaitForNoRelocatingShards(true) - .execute() - .actionGet(); - assertTrue(clusterHealthResponse.getRelocatingShards() == 0); + waitForRelocation(TimeValue.timeValueSeconds(90)); logger.info("---> Stopping indexing thread"); asyncIndexingService.stopIndexing(); Map shardCountByNodeId = getShardCountByNodeId(); diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index 49cb710c915fc..82b68b32f3bf8 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -2146,7 +2146,7 @@ public void waitForRemoteStoreSync(Runnable onProgress) throws IOException { segmentUploadeCount = directory.getSegmentsUploadedToRemoteStore().size(); } try { - Thread.sleep(TimeValue.timeValueSeconds(30).seconds()); + Thread.sleep(TimeValue.timeValueSeconds(30).millis()); } catch (InterruptedException ie) { throw new OpenSearchException("Interrupted waiting for completion of [{}]", ie); } From 0731548cedce04f60914d5b32f902bfa7c10297c Mon Sep 17 00:00:00 2001 From: Prudhvi Godithi Date: Fri, 14 Jun 2024 07:02:44 -0700 Subject: [PATCH 09/84] Update gradle check workflow (#14334) Signed-off-by: Prudhvi Godithi --- .github/ISSUE_TEMPLATE/failed_check.md | 8 -------- .github/workflows/gradle-check.yml | 15 +++------------ 2 files changed, 3 insertions(+), 20 deletions(-) delete mode 100644 .github/ISSUE_TEMPLATE/failed_check.md diff --git a/.github/ISSUE_TEMPLATE/failed_check.md b/.github/ISSUE_TEMPLATE/failed_check.md deleted file mode 100644 index 71508c9f5bd43..0000000000000 --- a/.github/ISSUE_TEMPLATE/failed_check.md +++ /dev/null @@ -1,8 +0,0 @@ ---- -title: '[AUTOCUT] Gradle Check Failure on push to {{ env.branch_name }}' -labels: '>test-failure, bug, autocut' ---- - -Gradle check has failed on push of your commit {{ env.pr_from_sha }}. -Please examine the workflow log {{ env.workflow_url }}. -Is the failure [a flaky test](https://github.com/opensearch-project/OpenSearch/blob/main/DEVELOPER_GUIDE.md#flaky-tests) unrelated to your change? diff --git a/.github/workflows/gradle-check.yml b/.github/workflows/gradle-check.yml index 07185ef4c65e3..2909ee95349ce 100644 --- a/.github/workflows/gradle-check.yml +++ b/.github/workflows/gradle-check.yml @@ -14,7 +14,7 @@ permissions: jobs: check-files: runs-on: ubuntu-latest - outputs: + outputs: RUN_GRADLE_CHECK: ${{ steps.changed-files-specific.outputs.any_changed }} steps: - uses: actions/checkout@v4 @@ -26,7 +26,7 @@ jobs: release-notes/*.md .github/** *.md - + gradle-check: needs: check-files if: github.repository == 'opensearch-project/OpenSearch' && needs.check-files.outputs.RUN_GRADLE_CHECK == 'true' @@ -158,15 +158,6 @@ jobs: Please examine the workflow log, locate, and copy-paste the failure(s) below, then iterate to green. Is the failure [a flaky test](https://github.com/opensearch-project/OpenSearch/blob/main/DEVELOPER_GUIDE.md#flaky-tests) unrelated to your change? - - name: Create Issue On Push Failure - if: ${{ github.event_name == 'push' && failure() }} - uses: dblock/create-a-github-issue@v3 - env: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - with: - assignees: ${{ github.event.head_commit.author.username }}, ${{ github.triggering_actor }} - filename: .github/ISSUE_TEMPLATE/failed_check.md - check-result: needs: [check-files, gradle-check] if: always() @@ -174,4 +165,4 @@ jobs: steps: - name: Fail if gradle-check fails if: ${{ needs.check-files.outputs.RUN_GRADLE_CHECK && needs.gradle-check.result == 'failure' }} - run: exit 1 \ No newline at end of file + run: exit 1 From 0c2ff039890c9e891da068ba401a7a77683c4a5b Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Fri, 14 Jun 2024 19:33:42 +0530 Subject: [PATCH 10/84] Fix flakiness of testRemoteCleanupDeleteStale, bug fix in RemoteMetadataManifest and RemoteReadResult (#14230) * Address flakiness of testRemoteCleanupDeleteStale - Make RemoteReadResult have Object rather than ToXContent - Fix getManifestCodecVersion in RemoteClusterMetadataManifest Signed-off-by: Shivansh Arora --- .../RemoteClusterStateCleanupManagerIT.java | 6 +- .../RemoteClusterStateAttributesManager.java | 3 +- .../remote/RemoteClusterStateService.java | 4 +- .../remote/RemoteGlobalMetadataManager.java | 3 +- .../model/RemoteClusterMetadataManifest.java | 7 +- .../remote/model/RemoteReadResult.java | 8 +- ...oteClusterStateAttributesManagerTests.java | 141 ++++++++++++++++++ .../model/RemoteClusterBlocksTests.java | 2 +- .../RemoteClusterMetadataManifestTests.java | 24 +++ .../model/RemoteClusterStateCustomsTests.java | 6 +- .../model/RemoteDiscoveryNodesTests.java | 2 +- 11 files changed, 186 insertions(+), 20 deletions(-) create mode 100644 server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManagerTests.java diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java index e96dedaa3e6a0..5074971ab1a1f 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java @@ -108,6 +108,9 @@ public void testRemoteCleanupDeleteStale() throws Exception { .add("cluster-state") .add(getClusterState().metadata().clusterUUID()); BlobPath manifestContainerPath = baseMetadataPath.add("manifest"); + RemoteClusterStateCleanupManager remoteClusterStateCleanupManager = internalCluster().getClusterManagerNodeInstance( + RemoteClusterStateCleanupManager.class + ); // set cleanup interval to 100 ms to make the test faster ClusterUpdateSettingsResponse response = client().admin() @@ -117,6 +120,7 @@ public void testRemoteCleanupDeleteStale() throws Exception { .get(); assertTrue(response.isAcknowledged()); + assertBusy(() -> assertEquals(100, remoteClusterStateCleanupManager.getStaleFileDeletionTask().getInterval().getMillis())); assertBusy(() -> { int manifestFiles = repository.blobStore().blobContainer(manifestContainerPath).listBlobsByPrefix("manifest").size(); @@ -128,7 +132,7 @@ public void testRemoteCleanupDeleteStale() throws Exception { "Current number of manifest files: " + manifestFiles, manifestFiles >= RETAINED_MANIFESTS && manifestFiles < RETAINED_MANIFESTS + 2 * SKIP_CLEANUP_STATE_CHANGES ); - }, 500, TimeUnit.MILLISECONDS); + }); // disable the clean up to avoid race condition during shutdown response = client().admin() diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManager.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManager.java index 4098993246073..b052b6e1a613d 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManager.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManager.java @@ -15,7 +15,6 @@ import org.opensearch.common.remote.RemoteWritableEntityStore; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; -import org.opensearch.core.xcontent.ToXContent; import org.opensearch.gateway.remote.model.RemoteClusterBlocks; import org.opensearch.gateway.remote.model.RemoteClusterStateBlobStore; import org.opensearch.gateway.remote.model.RemoteClusterStateCustoms; @@ -121,7 +120,7 @@ public CheckedRunnable getAsyncMetadataReadAction( LatchedActionListener listener ) { final ActionListener actionListener = ActionListener.wrap( - response -> listener.onResponse(new RemoteReadResult((ToXContent) response, CLUSTER_STATE_ATTRIBUTE, component)), + response -> listener.onResponse(new RemoteReadResult(response, CLUSTER_STATE_ATTRIBUTE, component)), listener::onFailure ); return () -> getStore(blobEntity).readAsync(blobEntity, actionListener); diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java index bd371ae671cf4..ada29fdb57c57 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java @@ -276,7 +276,7 @@ public RemoteClusterStateManifestInfo writeIncrementalMetadata( ClusterState clusterState, ClusterMetadataManifest previousManifest ) throws IOException { - logger.info("WRITING INCREMENTAL STATE"); + logger.trace("WRITING INCREMENTAL STATE"); final long startTimeNanos = relativeTimeNanosSupplier.getAsLong(); if (clusterState.nodes().isLocalNodeElectedClusterManager() == false) { @@ -766,7 +766,7 @@ private UploadedMetadataResults writeMetadataInParallel( throw new IllegalStateException("Unknown metadata component name " + name); } }); - logger.info("response {}", response.uploadedIndicesRoutingMetadata.toString()); + logger.trace("response {}", response.uploadedIndicesRoutingMetadata.toString()); return response; } diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManager.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManager.java index cd29114e05684..3053095368972 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManager.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManager.java @@ -25,7 +25,6 @@ import org.opensearch.core.common.io.stream.NamedWriteableRegistry; import org.opensearch.core.compress.Compressor; import org.opensearch.core.xcontent.NamedXContentRegistry; -import org.opensearch.core.xcontent.ToXContent; import org.opensearch.gateway.remote.model.RemoteClusterStateBlobStore; import org.opensearch.gateway.remote.model.RemoteCoordinationMetadata; import org.opensearch.gateway.remote.model.RemoteCustomMetadata; @@ -194,7 +193,7 @@ CheckedRunnable getAsyncMetadataReadAction( LatchedActionListener listener ) { ActionListener actionListener = ActionListener.wrap( - response -> listener.onResponse(new RemoteReadResult((ToXContent) response, readEntity.getType(), componentName)), + response -> listener.onResponse(new RemoteReadResult(response, readEntity.getType(), componentName)), listener::onFailure ); return () -> getStore(readEntity).readAsync(readEntity, actionListener); diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterMetadataManifest.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterMetadataManifest.java index 328601139c150..1dc56712d4ab5 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterMetadataManifest.java +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterMetadataManifest.java @@ -131,16 +131,17 @@ public ClusterMetadataManifest deserialize(final InputStream inputStream) throws return blobStoreFormat.deserialize(blobName, getNamedXContentRegistry(), Streams.readFully(inputStream)); } - private int getManifestCodecVersion() { + // package private for testing + int getManifestCodecVersion() { assert blobName != null; - String[] splitName = blobName.split(DELIMITER); + String[] splitName = getBlobFileName().split(DELIMITER); if (splitName.length == SPLITTED_MANIFEST_FILE_LENGTH) { return Integer.parseInt(splitName[splitName.length - 1]); // Last value would be codec version. } else if (splitName.length < SPLITTED_MANIFEST_FILE_LENGTH) { // Where codec is not part of file name, i.e. default codec version 0 // is used. return ClusterMetadataManifest.CODEC_V0; } else { - throw new IllegalArgumentException("Manifest file name is corrupted"); + throw new IllegalArgumentException("Manifest file name is corrupted : " + blobName); } } diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteReadResult.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteReadResult.java index adee09eaeffef..06d3b88ae1ecf 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteReadResult.java +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteReadResult.java @@ -8,24 +8,22 @@ package org.opensearch.gateway.remote.model; -import org.opensearch.core.xcontent.ToXContent; - /** * Container class for entity read from remote store */ public class RemoteReadResult { - ToXContent obj; + Object obj; String component; String componentName; - public RemoteReadResult(ToXContent obj, String component, String componentName) { + public RemoteReadResult(Object obj, String component, String componentName) { this.obj = obj; this.component = component; this.componentName = componentName; } - public ToXContent getObj() { + public Object getObj() { return obj; } diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManagerTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManagerTests.java new file mode 100644 index 0000000000000..0aff1c4b0e5e2 --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManagerTests.java @@ -0,0 +1,141 @@ +/* + * 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.gateway.remote; + +import org.opensearch.action.LatchedActionListener; +import org.opensearch.cluster.block.ClusterBlocks; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.common.CheckedRunnable; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.common.io.stream.NamedWriteableRegistry; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.compress.NoneCompressor; +import org.opensearch.gateway.remote.model.RemoteClusterBlocks; +import org.opensearch.gateway.remote.model.RemoteDiscoveryNodes; +import org.opensearch.gateway.remote.model.RemoteReadResult; +import org.opensearch.index.translog.transfer.BlobStoreTransferService; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; + +import java.io.IOException; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicReference; + +import static java.util.Collections.emptyList; +import static org.opensearch.gateway.remote.RemoteClusterStateAttributesManager.DISCOVERY_NODES; +import static org.opensearch.gateway.remote.model.RemoteClusterBlocks.CLUSTER_BLOCKS; +import static org.opensearch.gateway.remote.model.RemoteClusterBlocks.CLUSTER_BLOCKS_FORMAT; +import static org.opensearch.gateway.remote.model.RemoteClusterBlocksTests.randomClusterBlocks; +import static org.opensearch.gateway.remote.model.RemoteDiscoveryNodes.DISCOVERY_NODES_FORMAT; +import static org.opensearch.gateway.remote.model.RemoteDiscoveryNodesTests.getDiscoveryNodes; +import static org.mockito.ArgumentMatchers.anyIterable; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class RemoteClusterStateAttributesManagerTests extends OpenSearchTestCase { + private RemoteClusterStateAttributesManager remoteClusterStateAttributesManager; + private BlobStoreTransferService blobStoreTransferService; + private BlobStoreRepository blobStoreRepository; + private Compressor compressor; + private ThreadPool threadpool = new TestThreadPool(RemoteClusterStateAttributesManagerTests.class.getName()); + + @Before + public void setup() throws Exception { + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(emptyList()); + blobStoreRepository = mock(BlobStoreRepository.class); + blobStoreTransferService = mock(BlobStoreTransferService.class); + compressor = new NoneCompressor(); + + remoteClusterStateAttributesManager = new RemoteClusterStateAttributesManager( + "test-cluster", + blobStoreRepository, + blobStoreTransferService, + namedWriteableRegistry, + threadpool + ); + } + + @After + public void tearDown() throws Exception { + super.tearDown(); + threadpool.shutdown(); + } + + public void testGetAsyncMetadataReadAction_DiscoveryNodes() throws IOException { + DiscoveryNodes discoveryNodes = getDiscoveryNodes(); + String fileName = randomAlphaOfLength(10); + when(blobStoreTransferService.downloadBlob(anyIterable(), anyString())).thenReturn( + DISCOVERY_NODES_FORMAT.serialize(discoveryNodes, fileName, compressor).streamInput() + ); + RemoteDiscoveryNodes remoteObjForDownload = new RemoteDiscoveryNodes(fileName, "cluster-uuid", compressor); + CountDownLatch latch = new CountDownLatch(1); + AtomicReference readDiscoveryNodes = new AtomicReference<>(); + LatchedActionListener assertingListener = new LatchedActionListener<>( + ActionListener.wrap(response -> readDiscoveryNodes.set((DiscoveryNodes) response.getObj()), Assert::assertNull), + latch + ); + CheckedRunnable runnable = remoteClusterStateAttributesManager.getAsyncMetadataReadAction( + DISCOVERY_NODES, + remoteObjForDownload, + assertingListener + ); + + try { + runnable.run(); + latch.await(); + assertEquals(discoveryNodes.getSize(), readDiscoveryNodes.get().getSize()); + discoveryNodes.getNodes().forEach((nodeId, node) -> assertEquals(readDiscoveryNodes.get().get(nodeId), node)); + assertEquals(discoveryNodes.getClusterManagerNodeId(), readDiscoveryNodes.get().getClusterManagerNodeId()); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public void testGetAsyncMetadataReadAction_ClusterBlocks() throws IOException { + ClusterBlocks clusterBlocks = randomClusterBlocks(); + String fileName = randomAlphaOfLength(10); + when(blobStoreTransferService.downloadBlob(anyIterable(), anyString())).thenReturn( + CLUSTER_BLOCKS_FORMAT.serialize(clusterBlocks, fileName, compressor).streamInput() + ); + RemoteClusterBlocks remoteClusterBlocks = new RemoteClusterBlocks(fileName, "cluster-uuid", compressor); + CountDownLatch latch = new CountDownLatch(1); + AtomicReference readClusterBlocks = new AtomicReference<>(); + LatchedActionListener assertingListener = new LatchedActionListener<>( + ActionListener.wrap(response -> readClusterBlocks.set((ClusterBlocks) response.getObj()), Assert::assertNull), + latch + ); + + CheckedRunnable runnable = remoteClusterStateAttributesManager.getAsyncMetadataReadAction( + CLUSTER_BLOCKS, + remoteClusterBlocks, + assertingListener + ); + + try { + runnable.run(); + latch.await(); + assertEquals(clusterBlocks.global(), readClusterBlocks.get().global()); + assertEquals(clusterBlocks.indices().keySet(), readClusterBlocks.get().indices().keySet()); + for (String index : clusterBlocks.indices().keySet()) { + assertEquals(clusterBlocks.indices().get(index), readClusterBlocks.get().indices().get(index)); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteClusterBlocksTests.java b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteClusterBlocksTests.java index fe273c73c651b..3c1e141b81360 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteClusterBlocksTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteClusterBlocksTests.java @@ -136,7 +136,7 @@ public void testSerDe() throws IOException { } } - static ClusterBlocks randomClusterBlocks() { + public static ClusterBlocks randomClusterBlocks() { ClusterBlocks.Builder builder = ClusterBlocks.builder(); int randomGlobalBlocks = randomIntBetween(1, 10); for (int i = 0; i < randomGlobalBlocks; i++) { diff --git a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteClusterMetadataManifestTests.java b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteClusterMetadataManifestTests.java index 7cb80a1600c03..de1befbecd924 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteClusterMetadataManifestTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteClusterMetadataManifestTests.java @@ -41,6 +41,8 @@ import java.util.stream.Stream; import static java.util.stream.Collectors.toList; +import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V0; +import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V2; import static org.opensearch.gateway.remote.model.RemoteClusterMetadataManifest.MANIFEST; import static org.opensearch.gateway.remote.model.RemoteClusterMetadataManifest.MANIFEST_CURRENT_CODEC_VERSION; import static org.hamcrest.Matchers.greaterThan; @@ -236,6 +238,28 @@ public void testSerDe() throws IOException { assertThrows(IllegalArgumentException.class, () -> invalidRemoteObject.deserialize(new ByteArrayInputStream(new byte[0]))); } + public void testGetManifestCodecVersion() { + String manifestFileWithDelimiterInPath = + "123456789012_test-cluster/cluster-state/dsgYj10__Nkso7/manifest/manifest__9223372036854775806__9223372036854775804__C__9223370319103329556__2"; + RemoteClusterMetadataManifest remoteManifestForDownload = new RemoteClusterMetadataManifest( + manifestFileWithDelimiterInPath, + clusterUUID, + compressor, + namedXContentRegistry + ); + assertEquals(CODEC_V2, remoteManifestForDownload.getManifestCodecVersion()); + + String v0ManifestFileWithDelimiterInPath = + "123456789012_test-cluster/cluster-state/dsgYj10__Nkso7/manifest/manifest__9223372036854775806__9223372036854775804__C__9223370319103329556"; + RemoteClusterMetadataManifest remoteManifestV0ForDownload = new RemoteClusterMetadataManifest( + v0ManifestFileWithDelimiterInPath, + clusterUUID, + compressor, + namedXContentRegistry + ); + assertEquals(CODEC_V0, remoteManifestV0ForDownload.getManifestCodecVersion()); + } + private ClusterMetadataManifest getClusterMetadataManifest() { return ClusterMetadataManifest.builder() .opensearchVersion(Version.CURRENT) diff --git a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteClusterStateCustomsTests.java b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteClusterStateCustomsTests.java index 1f7a5e8bfffb1..1b020e13324a4 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteClusterStateCustomsTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteClusterStateCustomsTests.java @@ -232,12 +232,12 @@ public void testSerDe() throws IOException { try (InputStream inputStream = remoteObjectForUpload.serialize()) { remoteObjectForUpload.setFullBlobName(BlobPath.cleanPath()); assertThat(inputStream.available(), greaterThan(0)); - Custom readclusterStateCustoms = remoteObjectForUpload.deserialize(inputStream); - assertThat(readclusterStateCustoms, is(clusterStateCustoms)); + Custom readClusterStateCustoms = remoteObjectForUpload.deserialize(inputStream); + assertThat(readClusterStateCustoms, is(clusterStateCustoms)); } } - private Custom getClusterStateCustom() { + public static SnapshotsInProgress getClusterStateCustom() { return SnapshotsInProgress.of( List.of( new SnapshotsInProgress.Entry( diff --git a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteDiscoveryNodesTests.java b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteDiscoveryNodesTests.java index b9b6e02e8274f..f1bced2bdf855 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteDiscoveryNodesTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteDiscoveryNodesTests.java @@ -156,7 +156,7 @@ public void testExceptionDuringDeserialize() throws IOException { IOException ioe = assertThrows(IOException.class, () -> remoteObjectForDownload.deserialize(in)); } - private DiscoveryNodes getDiscoveryNodes() { + public static DiscoveryNodes getDiscoveryNodes() { return DiscoveryNodes.builder() .add( new DiscoveryNode( From d480027b7ada659a383fc4722bb01dcbac320311 Mon Sep 17 00:00:00 2001 From: Liyun Xiu Date: Fri, 14 Jun 2024 09:57:02 -0700 Subject: [PATCH 11/84] [Bugfix] Fix incorrect document order when there's exception during batch ingest (#14341) Signed-off-by: Liyun Xiu --- .../org/opensearch/ingest/IngestClientIT.java | 111 ++++++++++++++++++ .../org/opensearch/ingest/IngestService.java | 18 +-- .../opensearch/ingest/IngestServiceTests.java | 22 +++- 3 files changed, 136 insertions(+), 15 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/ingest/IngestClientIT.java b/server/src/internalClusterTest/java/org/opensearch/ingest/IngestClientIT.java index 9481a6116cdbc..dbde31ef1eb65 100644 --- a/server/src/internalClusterTest/java/org/opensearch/ingest/IngestClientIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/ingest/IngestClientIT.java @@ -60,15 +60,18 @@ import org.opensearch.plugins.Plugin; import org.opensearch.test.OpenSearchIntegTestCase; import org.opensearch.test.ParameterizedStaticSettingsOpenSearchIntegTestCase; +import org.hamcrest.MatcherAssert; import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.stream.Collectors; import static org.opensearch.common.xcontent.XContentFactory.jsonBuilder; import static org.opensearch.test.NodeRoles.nonIngestNode; +import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.notNullValue; @@ -159,6 +162,14 @@ public void testSimulate() throws Exception { } public void testBulkWithIngestFailures() throws Exception { + runBulkTestWithRandomDocs(false); + } + + public void testBulkWithIngestFailuresWithBatchSize() throws Exception { + runBulkTestWithRandomDocs(true); + } + + private void runBulkTestWithRandomDocs(boolean shouldSetBatchSize) throws Exception { createIndex("index"); BytesReference source = BytesReference.bytes( @@ -177,6 +188,9 @@ public void testBulkWithIngestFailures() throws Exception { int numRequests = scaledRandomIntBetween(32, 128); BulkRequest bulkRequest = new BulkRequest(); + if (shouldSetBatchSize) { + bulkRequest.batchSize(numRequests); + } for (int i = 0; i < numRequests; i++) { IndexRequest indexRequest = new IndexRequest("index").id(Integer.toString(i)).setPipeline("_id"); indexRequest.source(Requests.INDEX_CONTENT_TYPE, "field", "value", "fail", i % 2 == 0); @@ -209,6 +223,103 @@ public void testBulkWithIngestFailures() throws Exception { assertTrue(deletePipelineResponse.isAcknowledged()); } + public void testBulkWithIngestFailuresBatch() throws Exception { + createIndex("index"); + + BytesReference source = BytesReference.bytes( + jsonBuilder().startObject() + .field("description", "my_pipeline") + .startArray("processors") + .startObject() + .startObject("test") + .endObject() + .endObject() + .endArray() + .endObject() + ); + PutPipelineRequest putPipelineRequest = new PutPipelineRequest("_id", source, MediaTypeRegistry.JSON); + client().admin().cluster().putPipeline(putPipelineRequest).get(); + + BulkRequest bulkRequest = new BulkRequest(); + bulkRequest.batchSize(2); + bulkRequest.add( + new IndexRequest("index").id("_fail").setPipeline("_id").source(Requests.INDEX_CONTENT_TYPE, "field", "value", "fail", true) + ); + bulkRequest.add( + new IndexRequest("index").id("_success").setPipeline("_id").source(Requests.INDEX_CONTENT_TYPE, "field", "value", "fail", false) + ); + + BulkResponse response = client().bulk(bulkRequest).actionGet(); + MatcherAssert.assertThat(response.getItems().length, equalTo(bulkRequest.requests().size())); + + Map results = Arrays.stream(response.getItems()) + .collect(Collectors.toMap(BulkItemResponse::getId, r -> r)); + + MatcherAssert.assertThat(results.keySet(), containsInAnyOrder("_fail", "_success")); + assertNotNull(results.get("_fail").getFailure()); + assertNull(results.get("_success").getFailure()); + + // verify field of successful doc + Map successDoc = client().prepareGet("index", "_success").get().getSourceAsMap(); + assertThat(successDoc.get("processed"), equalTo(true)); + + // cleanup + AcknowledgedResponse deletePipelineResponse = client().admin().cluster().prepareDeletePipeline("_id").get(); + assertTrue(deletePipelineResponse.isAcknowledged()); + } + + public void testBulkWithIngestFailuresAndDropBatch() throws Exception { + createIndex("index"); + + BytesReference source = BytesReference.bytes( + jsonBuilder().startObject() + .field("description", "my_pipeline") + .startArray("processors") + .startObject() + .startObject("test") + .endObject() + .endObject() + .endArray() + .endObject() + ); + PutPipelineRequest putPipelineRequest = new PutPipelineRequest("_id", source, MediaTypeRegistry.JSON); + client().admin().cluster().putPipeline(putPipelineRequest).get(); + + BulkRequest bulkRequest = new BulkRequest(); + bulkRequest.batchSize(3); + bulkRequest.add( + new IndexRequest("index").id("_fail").setPipeline("_id").source(Requests.INDEX_CONTENT_TYPE, "field", "value", "fail", true) + ); + bulkRequest.add( + new IndexRequest("index").id("_success").setPipeline("_id").source(Requests.INDEX_CONTENT_TYPE, "field", "value", "fail", false) + ); + bulkRequest.add( + new IndexRequest("index").id("_drop").setPipeline("_id").source(Requests.INDEX_CONTENT_TYPE, "field", "value", "drop", true) + ); + + BulkResponse response = client().bulk(bulkRequest).actionGet(); + MatcherAssert.assertThat(response.getItems().length, equalTo(bulkRequest.requests().size())); + + Map results = Arrays.stream(response.getItems()) + .collect(Collectors.toMap(BulkItemResponse::getId, r -> r)); + + MatcherAssert.assertThat(results.keySet(), containsInAnyOrder("_fail", "_success", "_drop")); + assertNotNull(results.get("_fail").getFailure()); + assertNull(results.get("_success").getFailure()); + assertNull(results.get("_drop").getFailure()); + + // verify dropped doc not in index + assertNull(client().prepareGet("index", "_drop").get().getSourceAsMap()); + + // verify field of successful doc + Map successDoc = client().prepareGet("index", "_success").get().getSourceAsMap(); + assertThat(successDoc.get("processed"), equalTo(true)); + + // cleanup + AcknowledgedResponse deletePipelineResponse = client().admin().cluster().prepareDeletePipeline("_id").get(); + assertTrue(deletePipelineResponse.isAcknowledged()); + } + public void testBulkWithUpsert() throws Exception { createIndex("index"); diff --git a/server/src/main/java/org/opensearch/ingest/IngestService.java b/server/src/main/java/org/opensearch/ingest/IngestService.java index ab8e823199447..2281ccd4c0382 100644 --- a/server/src/main/java/org/opensearch/ingest/IngestService.java +++ b/server/src/main/java/org/opensearch/ingest/IngestService.java @@ -775,7 +775,7 @@ private void executePipelinesInBatchRequests( ), results.get(i).getException() ); - onFailure.accept(slots.get(i), results.get(i).getException()); + onFailure.accept(results.get(i).getSlot(), results.get(i).getException()); } } @@ -1092,15 +1092,15 @@ private void innerBatchExecute( } if (!exceptions.isEmpty()) { totalMetrics.failedN(exceptions.size()); - } else if (!dropped.isEmpty()) { + } + if (!dropped.isEmpty()) { dropped.forEach(t -> itemDroppedHandler.accept(t.getSlot())); - } else { - for (IngestDocumentWrapper ingestDocumentWrapper : succeeded) { - updateIndexRequestWithIngestDocument( - slotToindexRequestMap.get(ingestDocumentWrapper.getSlot()), - ingestDocumentWrapper.getIngestDocument() - ); - } + } + for (IngestDocumentWrapper ingestDocumentWrapper : succeeded) { + updateIndexRequestWithIngestDocument( + slotToindexRequestMap.get(ingestDocumentWrapper.getSlot()), + ingestDocumentWrapper.getIngestDocument() + ); } handler.accept(allResults); } diff --git a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java index 6d216370bae9a..a32cd2c3cad3f 100644 --- a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java +++ b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java @@ -97,6 +97,7 @@ import java.util.function.LongSupplier; import java.util.stream.Collectors; +import org.mockito.ArgumentCaptor; import org.mockito.ArgumentMatcher; import org.mockito.invocation.InvocationOnMock; @@ -1894,7 +1895,7 @@ public void testExecuteBulkRequestInBatchWithException() { verify(mockCompoundProcessor, never()).execute(any(), any()); } - public void testExecuteBulkRequestInBatchWithExceptionInCallback() { + public void testExecuteBulkRequestInBatchWithExceptionAndDropInCallback() { CompoundProcessor mockCompoundProcessor = mockCompoundProcessor(); IngestService ingestService = createWithProcessors( Collections.singletonMap("mock", (factories, tag, description, config) -> mockCompoundProcessor) @@ -1906,11 +1907,14 @@ public void testExecuteBulkRequestInBatchWithExceptionInCallback() { bulkRequest.add(indexRequest1); IndexRequest indexRequest2 = new IndexRequest("_index").id("_id2").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); bulkRequest.add(indexRequest2); - bulkRequest.batchSize(2); + IndexRequest indexRequest3 = new IndexRequest("_index").id("_id3").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); + bulkRequest.add(indexRequest3); + bulkRequest.batchSize(3); List results = Arrays.asList( new IngestDocumentWrapper(0, IngestService.toIngestDocument(indexRequest1), null), - new IngestDocumentWrapper(1, null, new RuntimeException()) + new IngestDocumentWrapper(1, null, new RuntimeException()), + new IngestDocumentWrapper(2, null, null) ); doAnswer(args -> { @SuppressWarnings("unchecked") @@ -1923,16 +1927,22 @@ public void testExecuteBulkRequestInBatchWithExceptionInCallback() { final BiConsumer failureHandler = mock(BiConsumer.class); @SuppressWarnings("unchecked") final BiConsumer completionHandler = mock(BiConsumer.class); + final IntConsumer dropHandler = mock(IntConsumer.class); ingestService.executeBulkRequest( - 2, + 3, bulkRequest.requests(), failureHandler, completionHandler, - indexReq -> {}, + dropHandler, Names.WRITE, bulkRequest ); - verify(failureHandler, times(1)).accept(any(), any()); + ArgumentCaptor failureSlotCaptor = ArgumentCaptor.forClass(Integer.class); + verify(failureHandler, times(1)).accept(failureSlotCaptor.capture(), any()); + assertEquals(1, failureSlotCaptor.getValue().intValue()); + ArgumentCaptor dropSlotCaptor = ArgumentCaptor.forClass(Integer.class); + verify(dropHandler, times(1)).accept(dropSlotCaptor.capture()); + assertEquals(2, dropSlotCaptor.getValue().intValue()); verify(completionHandler, times(1)).accept(Thread.currentThread(), null); verify(mockCompoundProcessor, times(1)).batchExecute(any(), any()); verify(mockCompoundProcessor, never()).execute(any(), any()); From 57cb2f90b9af3755777bf48d8789cccae080feda Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Fri, 14 Jun 2024 13:40:17 -0400 Subject: [PATCH 12/84] Bump netty from 4.1.110.Final to 4.1.111.Final (#14356) Signed-off-by: Andriy Redko --- CHANGELOG.md | 1 + buildSrc/version.properties | 2 +- .../licenses/netty-buffer-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-buffer-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-codec-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-codec-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-codec-http-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-codec-http-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-codec-http2-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-codec-http2-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-common-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-common-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-handler-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-handler-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-resolver-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-resolver-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-transport-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-transport-4.1.111.Final.jar.sha1 | 1 + .../netty-transport-native-unix-common-4.1.110.Final.jar.sha1 | 1 - .../netty-transport-native-unix-common-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-codec-dns-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-codec-dns-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-codec-http2-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-codec-http2-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-codec-socks-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-codec-socks-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-handler-proxy-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-handler-proxy-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-resolver-dns-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-resolver-dns-4.1.111.Final.jar.sha1 | 1 + .../netty-transport-native-unix-common-4.1.110.Final.jar.sha1 | 1 - .../netty-transport-native-unix-common-4.1.111.Final.jar.sha1 | 1 + .../repository-hdfs/licenses/netty-all-4.1.110.Final.jar.sha1 | 1 - .../repository-hdfs/licenses/netty-all-4.1.111.Final.jar.sha1 | 1 + .../repository-s3/licenses/netty-buffer-4.1.110.Final.jar.sha1 | 1 - .../repository-s3/licenses/netty-buffer-4.1.111.Final.jar.sha1 | 1 + .../repository-s3/licenses/netty-codec-4.1.110.Final.jar.sha1 | 1 - .../repository-s3/licenses/netty-codec-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-codec-http-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-codec-http-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-codec-http2-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-codec-http2-4.1.111.Final.jar.sha1 | 1 + .../repository-s3/licenses/netty-common-4.1.110.Final.jar.sha1 | 1 - .../repository-s3/licenses/netty-common-4.1.111.Final.jar.sha1 | 1 + .../repository-s3/licenses/netty-handler-4.1.110.Final.jar.sha1 | 1 - .../repository-s3/licenses/netty-handler-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-resolver-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-resolver-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-transport-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-transport-4.1.111.Final.jar.sha1 | 1 + .../netty-transport-classes-epoll-4.1.110.Final.jar.sha1 | 1 - .../netty-transport-classes-epoll-4.1.111.Final.jar.sha1 | 1 + .../netty-transport-native-unix-common-4.1.110.Final.jar.sha1 | 1 - .../netty-transport-native-unix-common-4.1.111.Final.jar.sha1 | 1 + .../transport-nio/licenses/netty-buffer-4.1.110.Final.jar.sha1 | 1 - .../transport-nio/licenses/netty-buffer-4.1.111.Final.jar.sha1 | 1 + .../transport-nio/licenses/netty-codec-4.1.110.Final.jar.sha1 | 1 - .../transport-nio/licenses/netty-codec-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-codec-http-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-codec-http-4.1.111.Final.jar.sha1 | 1 + .../transport-nio/licenses/netty-common-4.1.110.Final.jar.sha1 | 1 - .../transport-nio/licenses/netty-common-4.1.111.Final.jar.sha1 | 1 + .../transport-nio/licenses/netty-handler-4.1.110.Final.jar.sha1 | 1 - .../transport-nio/licenses/netty-handler-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-resolver-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-resolver-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-transport-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-transport-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-buffer-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-buffer-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-codec-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-codec-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-codec-dns-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-codec-dns-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-codec-http-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-codec-http-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-codec-http2-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-codec-http2-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-common-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-common-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-handler-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-handler-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-resolver-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-resolver-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-resolver-dns-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-resolver-dns-4.1.111.Final.jar.sha1 | 1 + .../licenses/netty-transport-4.1.110.Final.jar.sha1 | 1 - .../licenses/netty-transport-4.1.111.Final.jar.sha1 | 1 + .../netty-transport-native-unix-common-4.1.110.Final.jar.sha1 | 1 - .../netty-transport-native-unix-common-4.1.111.Final.jar.sha1 | 1 + 90 files changed, 46 insertions(+), 45 deletions(-) delete mode 100644 modules/transport-netty4/licenses/netty-buffer-4.1.110.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-buffer-4.1.111.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-codec-4.1.110.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-codec-4.1.111.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-codec-http-4.1.110.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-codec-http-4.1.111.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-codec-http2-4.1.111.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-common-4.1.110.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-common-4.1.111.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-handler-4.1.110.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-handler-4.1.111.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-resolver-4.1.110.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-resolver-4.1.111.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-transport-4.1.110.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-transport-4.1.111.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-transport-native-unix-common-4.1.111.Final.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/netty-codec-dns-4.1.110.Final.jar.sha1 create mode 100644 plugins/repository-azure/licenses/netty-codec-dns-4.1.111.Final.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 create mode 100644 plugins/repository-azure/licenses/netty-codec-http2-4.1.111.Final.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/netty-codec-socks-4.1.110.Final.jar.sha1 create mode 100644 plugins/repository-azure/licenses/netty-codec-socks-4.1.111.Final.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/netty-handler-proxy-4.1.110.Final.jar.sha1 create mode 100644 plugins/repository-azure/licenses/netty-handler-proxy-4.1.111.Final.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/netty-resolver-dns-4.1.110.Final.jar.sha1 create mode 100644 plugins/repository-azure/licenses/netty-resolver-dns-4.1.111.Final.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 create mode 100644 plugins/repository-azure/licenses/netty-transport-native-unix-common-4.1.111.Final.jar.sha1 delete mode 100644 plugins/repository-hdfs/licenses/netty-all-4.1.110.Final.jar.sha1 create mode 100644 plugins/repository-hdfs/licenses/netty-all-4.1.111.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-buffer-4.1.110.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-buffer-4.1.111.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-codec-4.1.110.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-codec-4.1.111.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-codec-http-4.1.110.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-codec-http-4.1.111.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-codec-http2-4.1.111.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-common-4.1.110.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-common-4.1.111.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-handler-4.1.110.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-handler-4.1.111.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-resolver-4.1.110.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-resolver-4.1.111.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-transport-4.1.110.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-transport-4.1.111.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.110.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.111.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.111.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-buffer-4.1.110.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-buffer-4.1.111.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-codec-4.1.110.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-codec-4.1.111.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-codec-http-4.1.110.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-codec-http-4.1.111.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-common-4.1.110.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-common-4.1.111.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-handler-4.1.110.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-handler-4.1.111.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-resolver-4.1.110.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-resolver-4.1.111.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-transport-4.1.110.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-transport-4.1.111.Final.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/netty-buffer-4.1.110.Final.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/netty-buffer-4.1.111.Final.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/netty-codec-4.1.110.Final.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/netty-codec-4.1.111.Final.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/netty-codec-dns-4.1.110.Final.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/netty-codec-dns-4.1.111.Final.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/netty-codec-http-4.1.110.Final.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/netty-codec-http-4.1.111.Final.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/netty-codec-http2-4.1.111.Final.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/netty-common-4.1.110.Final.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/netty-common-4.1.111.Final.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/netty-handler-4.1.110.Final.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/netty-handler-4.1.111.Final.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/netty-resolver-4.1.110.Final.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/netty-resolver-4.1.111.Final.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/netty-resolver-dns-4.1.110.Final.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/netty-resolver-dns-4.1.111.Final.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/netty-transport-4.1.110.Final.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/netty-transport-4.1.111.Final.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/netty-transport-native-unix-common-4.1.111.Final.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index 0066e077b16a2..0dfc09310e040 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -9,6 +9,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ### Dependencies - Bump `org.gradle.test-retry` from 1.5.8 to 1.5.9 ([#13442](https://github.com/opensearch-project/OpenSearch/pull/13442)) - Update to Apache Lucene 9.11.0 ([#14042](https://github.com/opensearch-project/OpenSearch/pull/14042)) +- Bump `netty` from 4.1.110.Final to 4.1.111.Final ([#14356](https://github.com/opensearch-project/OpenSearch/pull/14356)) ### Changed diff --git a/buildSrc/version.properties b/buildSrc/version.properties index c34409053b915..d20c3bd46f304 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -29,7 +29,7 @@ hdrhistogram = 2.2.2 # when updating the JNA version, also update the version in buildSrc/build.gradle jna = 5.13.0 -netty = 4.1.110.Final +netty = 4.1.111.Final joda = 2.12.7 # project reactor diff --git a/modules/transport-netty4/licenses/netty-buffer-4.1.110.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-buffer-4.1.110.Final.jar.sha1 deleted file mode 100644 index faaf70c858a6e..0000000000000 --- a/modules/transport-netty4/licenses/netty-buffer-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3d918a9ee057d995c362902b54634fc307132aac \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-buffer-4.1.111.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-buffer-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..6784ac6c3b64f --- /dev/null +++ b/modules/transport-netty4/licenses/netty-buffer-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +b54863f578939e135d3b3aea610284ae57c188cf \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-codec-4.1.110.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-codec-4.1.110.Final.jar.sha1 deleted file mode 100644 index 7affbc14fa93a..0000000000000 --- a/modules/transport-netty4/licenses/netty-codec-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -f1fa43b03e93ab88e805b6a4e3e83780c80b47d2 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-codec-4.1.111.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-codec-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..3d86194de9213 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-codec-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +a6762ec00a6d268f9980741f5b755838bcd658bf \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-codec-http-4.1.110.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-codec-http-4.1.110.Final.jar.sha1 deleted file mode 100644 index 07730a5606ce2..0000000000000 --- a/modules/transport-netty4/licenses/netty-codec-http-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -9d05cd927209ea25bbf342962c00b8e5a828c2a4 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-codec-http-4.1.111.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-codec-http-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..4ef1adb818300 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-codec-http-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +c6ecbc452321e632bf3cea0f9758839b650455c7 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 deleted file mode 100644 index ebd1e0d52efb2..0000000000000 --- a/modules/transport-netty4/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e0849843eb5b1c036b12551baca98a9f7ff847a0 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-codec-http2-4.1.111.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-codec-http2-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..06c86b8fda557 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-codec-http2-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +f0cca5df75bfb4f858d0435f601d8b1cae1de054 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-common-4.1.110.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-common-4.1.110.Final.jar.sha1 deleted file mode 100644 index 568c0aa2a2c03..0000000000000 --- a/modules/transport-netty4/licenses/netty-common-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -ec361e7e025c029be50c55c8480080cabcbc01e7 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-common-4.1.111.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-common-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..16cb1cce7f504 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-common-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +58210befcb31adbcadd5724966a061444db91863 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-handler-4.1.110.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-handler-4.1.110.Final.jar.sha1 deleted file mode 100644 index 2d6050dd1e3a5..0000000000000 --- a/modules/transport-netty4/licenses/netty-handler-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -168db749c22652ee7fed1ebf7ec46ce856d75e51 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-handler-4.1.111.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-handler-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..2f70f791f65ed --- /dev/null +++ b/modules/transport-netty4/licenses/netty-handler-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +2bc6a58ad2e9e279634b6e55022e8dcd3c175cc4 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-resolver-4.1.110.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-resolver-4.1.110.Final.jar.sha1 deleted file mode 100644 index c3ee8087a8b5d..0000000000000 --- a/modules/transport-netty4/licenses/netty-resolver-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -66c15921104cda0159b34e316541bc765dfaf3c0 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-resolver-4.1.111.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-resolver-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..621cbf58f3133 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-resolver-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +3493179999f211dc49714319f81da2be86523a3b \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-transport-4.1.110.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-transport-4.1.110.Final.jar.sha1 deleted file mode 100644 index 32c8fa2b876a2..0000000000000 --- a/modules/transport-netty4/licenses/netty-transport-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -b91f04c39ac14d6a29d07184ef305953ee6e0348 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-transport-4.1.111.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-transport-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..ac96e7545ed58 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-transport-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +24e97cf14ea9d80afe4c5ab69066b587fccc154a \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 deleted file mode 100644 index 2c468962b1b64..0000000000000 --- a/modules/transport-netty4/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -a7096e7c0a25a983647909d7513f5d4943d589c0 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-transport-native-unix-common-4.1.111.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-transport-native-unix-common-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..0847ac3034db7 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-transport-native-unix-common-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +acafc128cddafa021bc0b48b0788eb0e118add5e \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-codec-dns-4.1.110.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-codec-dns-4.1.110.Final.jar.sha1 deleted file mode 100644 index c4ca8f15e85c5..0000000000000 --- a/plugins/repository-azure/licenses/netty-codec-dns-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -381c5bf8b7570c163fa7893a26d02b7ac36ff6eb \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-codec-dns-4.1.111.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-codec-dns-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..5e3f819012811 --- /dev/null +++ b/plugins/repository-azure/licenses/netty-codec-dns-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +f988dbb527efb0e7cf7d444cc50b0fc3f5f380ec \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 deleted file mode 100644 index ebd1e0d52efb2..0000000000000 --- a/plugins/repository-azure/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e0849843eb5b1c036b12551baca98a9f7ff847a0 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-codec-http2-4.1.111.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-codec-http2-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..06c86b8fda557 --- /dev/null +++ b/plugins/repository-azure/licenses/netty-codec-http2-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +f0cca5df75bfb4f858d0435f601d8b1cae1de054 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-codec-socks-4.1.110.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-codec-socks-4.1.110.Final.jar.sha1 deleted file mode 100644 index 9f6e95ba38d2e..0000000000000 --- a/plugins/repository-azure/licenses/netty-codec-socks-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -4d54c8d5b95b14756043efb59b8c3e62ec67aa43 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-codec-socks-4.1.111.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-codec-socks-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..226ee06d39d6c --- /dev/null +++ b/plugins/repository-azure/licenses/netty-codec-socks-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +ea52ef6617a9b69b0baaebb7f0b80373527f9607 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-handler-proxy-4.1.110.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-handler-proxy-4.1.110.Final.jar.sha1 deleted file mode 100644 index f31396d94c2ec..0000000000000 --- a/plugins/repository-azure/licenses/netty-handler-proxy-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -b7fb401dd47c79e6b99f2319ac3b561c50c31c30 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-handler-proxy-4.1.111.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-handler-proxy-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..dcc2b0c7ca923 --- /dev/null +++ b/plugins/repository-azure/licenses/netty-handler-proxy-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +1e459c8630bb7c942b79a97e62dd728798de6a8c \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-resolver-dns-4.1.110.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-resolver-dns-4.1.110.Final.jar.sha1 deleted file mode 100644 index 18d122acd2c44..0000000000000 --- a/plugins/repository-azure/licenses/netty-resolver-dns-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3e687cdc4ecdbbad07508a11b715bdf95fa20939 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-resolver-dns-4.1.111.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-resolver-dns-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..b22ad6784809b --- /dev/null +++ b/plugins/repository-azure/licenses/netty-resolver-dns-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +5ac6a3d96935129ba45ea768ad30e31cad0d8c4d \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 deleted file mode 100644 index 2c468962b1b64..0000000000000 --- a/plugins/repository-azure/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -a7096e7c0a25a983647909d7513f5d4943d589c0 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-transport-native-unix-common-4.1.111.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-transport-native-unix-common-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..0847ac3034db7 --- /dev/null +++ b/plugins/repository-azure/licenses/netty-transport-native-unix-common-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +acafc128cddafa021bc0b48b0788eb0e118add5e \ No newline at end of file diff --git a/plugins/repository-hdfs/licenses/netty-all-4.1.110.Final.jar.sha1 b/plugins/repository-hdfs/licenses/netty-all-4.1.110.Final.jar.sha1 deleted file mode 100644 index 8f8d86e6065b2..0000000000000 --- a/plugins/repository-hdfs/licenses/netty-all-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -db3f4d3ad3d16e26991a64d50b749ae09e0e0c8e \ No newline at end of file diff --git a/plugins/repository-hdfs/licenses/netty-all-4.1.111.Final.jar.sha1 b/plugins/repository-hdfs/licenses/netty-all-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..076124a7d1f89 --- /dev/null +++ b/plugins/repository-hdfs/licenses/netty-all-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +8fba10bb4911517eb1bdcc05ef392499dda4d5ac \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-buffer-4.1.110.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-buffer-4.1.110.Final.jar.sha1 deleted file mode 100644 index faaf70c858a6e..0000000000000 --- a/plugins/repository-s3/licenses/netty-buffer-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3d918a9ee057d995c362902b54634fc307132aac \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-buffer-4.1.111.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-buffer-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..6784ac6c3b64f --- /dev/null +++ b/plugins/repository-s3/licenses/netty-buffer-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +b54863f578939e135d3b3aea610284ae57c188cf \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-codec-4.1.110.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-codec-4.1.110.Final.jar.sha1 deleted file mode 100644 index 7affbc14fa93a..0000000000000 --- a/plugins/repository-s3/licenses/netty-codec-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -f1fa43b03e93ab88e805b6a4e3e83780c80b47d2 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-codec-4.1.111.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-codec-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..3d86194de9213 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-codec-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +a6762ec00a6d268f9980741f5b755838bcd658bf \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-codec-http-4.1.110.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-codec-http-4.1.110.Final.jar.sha1 deleted file mode 100644 index 07730a5606ce2..0000000000000 --- a/plugins/repository-s3/licenses/netty-codec-http-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -9d05cd927209ea25bbf342962c00b8e5a828c2a4 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-codec-http-4.1.111.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-codec-http-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..4ef1adb818300 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-codec-http-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +c6ecbc452321e632bf3cea0f9758839b650455c7 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 deleted file mode 100644 index ebd1e0d52efb2..0000000000000 --- a/plugins/repository-s3/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e0849843eb5b1c036b12551baca98a9f7ff847a0 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-codec-http2-4.1.111.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-codec-http2-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..06c86b8fda557 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-codec-http2-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +f0cca5df75bfb4f858d0435f601d8b1cae1de054 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-common-4.1.110.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-common-4.1.110.Final.jar.sha1 deleted file mode 100644 index 568c0aa2a2c03..0000000000000 --- a/plugins/repository-s3/licenses/netty-common-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -ec361e7e025c029be50c55c8480080cabcbc01e7 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-common-4.1.111.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-common-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..16cb1cce7f504 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-common-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +58210befcb31adbcadd5724966a061444db91863 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-handler-4.1.110.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-handler-4.1.110.Final.jar.sha1 deleted file mode 100644 index 2d6050dd1e3a5..0000000000000 --- a/plugins/repository-s3/licenses/netty-handler-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -168db749c22652ee7fed1ebf7ec46ce856d75e51 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-handler-4.1.111.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-handler-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..2f70f791f65ed --- /dev/null +++ b/plugins/repository-s3/licenses/netty-handler-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +2bc6a58ad2e9e279634b6e55022e8dcd3c175cc4 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-resolver-4.1.110.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-resolver-4.1.110.Final.jar.sha1 deleted file mode 100644 index c3ee8087a8b5d..0000000000000 --- a/plugins/repository-s3/licenses/netty-resolver-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -66c15921104cda0159b34e316541bc765dfaf3c0 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-resolver-4.1.111.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-resolver-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..621cbf58f3133 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-resolver-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +3493179999f211dc49714319f81da2be86523a3b \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-transport-4.1.110.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-transport-4.1.110.Final.jar.sha1 deleted file mode 100644 index 32c8fa2b876a2..0000000000000 --- a/plugins/repository-s3/licenses/netty-transport-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -b91f04c39ac14d6a29d07184ef305953ee6e0348 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-transport-4.1.111.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-transport-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..ac96e7545ed58 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-transport-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +24e97cf14ea9d80afe4c5ab69066b587fccc154a \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.110.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.110.Final.jar.sha1 deleted file mode 100644 index 408f3aa5d1339..0000000000000 --- a/plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3ca1cff0bf82bfd38e89f6946e54f24cbb3424a2 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.111.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..97001777eadf5 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +8b97d32eb1489043e478deea99bd93ce487b82f6 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 deleted file mode 100644 index 2c468962b1b64..0000000000000 --- a/plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -a7096e7c0a25a983647909d7513f5d4943d589c0 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.111.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..0847ac3034db7 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +acafc128cddafa021bc0b48b0788eb0e118add5e \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-buffer-4.1.110.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-buffer-4.1.110.Final.jar.sha1 deleted file mode 100644 index faaf70c858a6e..0000000000000 --- a/plugins/transport-nio/licenses/netty-buffer-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3d918a9ee057d995c362902b54634fc307132aac \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-buffer-4.1.111.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-buffer-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..6784ac6c3b64f --- /dev/null +++ b/plugins/transport-nio/licenses/netty-buffer-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +b54863f578939e135d3b3aea610284ae57c188cf \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-codec-4.1.110.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-codec-4.1.110.Final.jar.sha1 deleted file mode 100644 index 7affbc14fa93a..0000000000000 --- a/plugins/transport-nio/licenses/netty-codec-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -f1fa43b03e93ab88e805b6a4e3e83780c80b47d2 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-codec-4.1.111.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-codec-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..3d86194de9213 --- /dev/null +++ b/plugins/transport-nio/licenses/netty-codec-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +a6762ec00a6d268f9980741f5b755838bcd658bf \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-codec-http-4.1.110.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-codec-http-4.1.110.Final.jar.sha1 deleted file mode 100644 index 07730a5606ce2..0000000000000 --- a/plugins/transport-nio/licenses/netty-codec-http-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -9d05cd927209ea25bbf342962c00b8e5a828c2a4 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-codec-http-4.1.111.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-codec-http-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..4ef1adb818300 --- /dev/null +++ b/plugins/transport-nio/licenses/netty-codec-http-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +c6ecbc452321e632bf3cea0f9758839b650455c7 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-common-4.1.110.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-common-4.1.110.Final.jar.sha1 deleted file mode 100644 index 568c0aa2a2c03..0000000000000 --- a/plugins/transport-nio/licenses/netty-common-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -ec361e7e025c029be50c55c8480080cabcbc01e7 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-common-4.1.111.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-common-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..16cb1cce7f504 --- /dev/null +++ b/plugins/transport-nio/licenses/netty-common-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +58210befcb31adbcadd5724966a061444db91863 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-handler-4.1.110.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-handler-4.1.110.Final.jar.sha1 deleted file mode 100644 index 2d6050dd1e3a5..0000000000000 --- a/plugins/transport-nio/licenses/netty-handler-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -168db749c22652ee7fed1ebf7ec46ce856d75e51 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-handler-4.1.111.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-handler-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..2f70f791f65ed --- /dev/null +++ b/plugins/transport-nio/licenses/netty-handler-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +2bc6a58ad2e9e279634b6e55022e8dcd3c175cc4 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-resolver-4.1.110.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-resolver-4.1.110.Final.jar.sha1 deleted file mode 100644 index c3ee8087a8b5d..0000000000000 --- a/plugins/transport-nio/licenses/netty-resolver-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -66c15921104cda0159b34e316541bc765dfaf3c0 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-resolver-4.1.111.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-resolver-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..621cbf58f3133 --- /dev/null +++ b/plugins/transport-nio/licenses/netty-resolver-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +3493179999f211dc49714319f81da2be86523a3b \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-transport-4.1.110.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-transport-4.1.110.Final.jar.sha1 deleted file mode 100644 index 32c8fa2b876a2..0000000000000 --- a/plugins/transport-nio/licenses/netty-transport-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -b91f04c39ac14d6a29d07184ef305953ee6e0348 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-transport-4.1.111.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-transport-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..ac96e7545ed58 --- /dev/null +++ b/plugins/transport-nio/licenses/netty-transport-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +24e97cf14ea9d80afe4c5ab69066b587fccc154a \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-buffer-4.1.110.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-buffer-4.1.110.Final.jar.sha1 deleted file mode 100644 index faaf70c858a6e..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/netty-buffer-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3d918a9ee057d995c362902b54634fc307132aac \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-buffer-4.1.111.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-buffer-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..6784ac6c3b64f --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/netty-buffer-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +b54863f578939e135d3b3aea610284ae57c188cf \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-codec-4.1.110.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-codec-4.1.110.Final.jar.sha1 deleted file mode 100644 index 7affbc14fa93a..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/netty-codec-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -f1fa43b03e93ab88e805b6a4e3e83780c80b47d2 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-codec-4.1.111.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-codec-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..3d86194de9213 --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/netty-codec-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +a6762ec00a6d268f9980741f5b755838bcd658bf \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-codec-dns-4.1.110.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-codec-dns-4.1.110.Final.jar.sha1 deleted file mode 100644 index c4ca8f15e85c5..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/netty-codec-dns-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -381c5bf8b7570c163fa7893a26d02b7ac36ff6eb \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-codec-dns-4.1.111.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-codec-dns-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..5e3f819012811 --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/netty-codec-dns-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +f988dbb527efb0e7cf7d444cc50b0fc3f5f380ec \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-codec-http-4.1.110.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-codec-http-4.1.110.Final.jar.sha1 deleted file mode 100644 index 07730a5606ce2..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/netty-codec-http-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -9d05cd927209ea25bbf342962c00b8e5a828c2a4 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-codec-http-4.1.111.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-codec-http-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..4ef1adb818300 --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/netty-codec-http-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +c6ecbc452321e632bf3cea0f9758839b650455c7 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 deleted file mode 100644 index ebd1e0d52efb2..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/netty-codec-http2-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e0849843eb5b1c036b12551baca98a9f7ff847a0 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-codec-http2-4.1.111.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-codec-http2-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..06c86b8fda557 --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/netty-codec-http2-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +f0cca5df75bfb4f858d0435f601d8b1cae1de054 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-common-4.1.110.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-common-4.1.110.Final.jar.sha1 deleted file mode 100644 index 568c0aa2a2c03..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/netty-common-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -ec361e7e025c029be50c55c8480080cabcbc01e7 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-common-4.1.111.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-common-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..16cb1cce7f504 --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/netty-common-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +58210befcb31adbcadd5724966a061444db91863 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-handler-4.1.110.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-handler-4.1.110.Final.jar.sha1 deleted file mode 100644 index 2d6050dd1e3a5..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/netty-handler-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -168db749c22652ee7fed1ebf7ec46ce856d75e51 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-handler-4.1.111.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-handler-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..2f70f791f65ed --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/netty-handler-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +2bc6a58ad2e9e279634b6e55022e8dcd3c175cc4 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-resolver-4.1.110.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-resolver-4.1.110.Final.jar.sha1 deleted file mode 100644 index c3ee8087a8b5d..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/netty-resolver-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -66c15921104cda0159b34e316541bc765dfaf3c0 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-resolver-4.1.111.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-resolver-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..621cbf58f3133 --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/netty-resolver-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +3493179999f211dc49714319f81da2be86523a3b \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-resolver-dns-4.1.110.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-resolver-dns-4.1.110.Final.jar.sha1 deleted file mode 100644 index 18d122acd2c44..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/netty-resolver-dns-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3e687cdc4ecdbbad07508a11b715bdf95fa20939 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-resolver-dns-4.1.111.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-resolver-dns-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..b22ad6784809b --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/netty-resolver-dns-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +5ac6a3d96935129ba45ea768ad30e31cad0d8c4d \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-transport-4.1.110.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-transport-4.1.110.Final.jar.sha1 deleted file mode 100644 index 32c8fa2b876a2..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/netty-transport-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -b91f04c39ac14d6a29d07184ef305953ee6e0348 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-transport-4.1.111.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-transport-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..ac96e7545ed58 --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/netty-transport-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +24e97cf14ea9d80afe4c5ab69066b587fccc154a \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 deleted file mode 100644 index 2c468962b1b64..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/netty-transport-native-unix-common-4.1.110.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -a7096e7c0a25a983647909d7513f5d4943d589c0 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/netty-transport-native-unix-common-4.1.111.Final.jar.sha1 b/plugins/transport-reactor-netty4/licenses/netty-transport-native-unix-common-4.1.111.Final.jar.sha1 new file mode 100644 index 0000000000000..0847ac3034db7 --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/netty-transport-native-unix-common-4.1.111.Final.jar.sha1 @@ -0,0 +1 @@ +acafc128cddafa021bc0b48b0788eb0e118add5e \ No newline at end of file From c92e1253b9330fef5b45df36faf18675f21ea06a Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Fri, 14 Jun 2024 16:15:08 -0400 Subject: [PATCH 13/84] Bump org.wiremock:wiremock-standalone from 3.3.1 to 3.6.0 (#14361) Signed-off-by: Andriy Redko --- CHANGELOG.md | 1 + buildSrc/build.gradle | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 0dfc09310e040..9b7e333174354 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -10,6 +10,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `org.gradle.test-retry` from 1.5.8 to 1.5.9 ([#13442](https://github.com/opensearch-project/OpenSearch/pull/13442)) - Update to Apache Lucene 9.11.0 ([#14042](https://github.com/opensearch-project/OpenSearch/pull/14042)) - Bump `netty` from 4.1.110.Final to 4.1.111.Final ([#14356](https://github.com/opensearch-project/OpenSearch/pull/14356)) +- Bump `org.wiremock:wiremock-standalone` from 3.3.1 to 3.6.0 ([#14361](https://github.com/opensearch-project/OpenSearch/pull/14361)) ### Changed diff --git a/buildSrc/build.gradle b/buildSrc/build.gradle index c68cc0406d3a6..b984ef3800490 100644 --- a/buildSrc/build.gradle +++ b/buildSrc/build.gradle @@ -128,7 +128,7 @@ dependencies { testFixturesApi "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${props.getProperty('randomizedrunner')}" testFixturesApi gradleApi() testFixturesApi gradleTestKit() - testImplementation 'org.wiremock:wiremock-standalone:3.3.1' + testImplementation 'org.wiremock:wiremock-standalone:3.6.0' testImplementation "org.mockito:mockito-core:${props.getProperty('mockito')}" integTestImplementation('org.spockframework:spock-core:2.3-groovy-3.0') { exclude module: "groovy" From 56c5dcc1da9e3129f36b854e09198b8f4dadec4b Mon Sep 17 00:00:00 2001 From: Peter Alfonsi Date: Fri, 14 Jun 2024 13:20:51 -0700 Subject: [PATCH 14/84] [Bugfix] Fix TieredSpilloverCache flaky tests (#14333) * Fix flaky TSC stats tests Signed-off-by: Peter Alfonsi * Addressed andrross's comment Signed-off-by: Peter Alfonsi * fix forbidden API Signed-off-by: Peter Alfonsi --------- Signed-off-by: Peter Alfonsi Co-authored-by: Peter Alfonsi --- .../tier/TieredSpilloverCacheStatsIT.java | 6 +++++ .../CacheStatsAPIIndicesRequestCacheIT.java | 7 ++++++ .../cache/stats/ImmutableCacheStats.java | 23 +++++++++++++++++++ 3 files changed, 36 insertions(+) diff --git a/modules/cache-common/src/internalClusterTest/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsIT.java b/modules/cache-common/src/internalClusterTest/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsIT.java index 537caccbac652..783b6083e9226 100644 --- a/modules/cache-common/src/internalClusterTest/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsIT.java +++ b/modules/cache-common/src/internalClusterTest/java/org/opensearch/cache/common/tier/TieredSpilloverCacheStatsIT.java @@ -10,6 +10,7 @@ import org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest; import org.opensearch.action.admin.cluster.node.stats.NodesStatsResponse; +import org.opensearch.action.admin.indices.forcemerge.ForceMergeResponse; import org.opensearch.action.admin.indices.stats.CommonStatsFlags; import org.opensearch.action.search.SearchResponse; import org.opensearch.client.Client; @@ -20,6 +21,7 @@ import org.opensearch.common.cache.stats.ImmutableCacheStatsHolder; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.index.IndexSettings; import org.opensearch.index.cache.request.RequestCacheStats; import org.opensearch.index.query.QueryBuilders; import org.opensearch.indices.IndicesRequestCache; @@ -351,11 +353,15 @@ private void startIndex(Client client, String indexName) throws InterruptedExcep .put(IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING.getKey(), true) .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + // Disable index refreshing to avoid cache being invalidated mid-test + .put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), TimeValue.timeValueMillis(-1)) .build() ) .get() ); indexRandom(true, client.prepareIndex(indexName).setSource("k", "hello")); + // 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(indexName).setFlush(true).get(); ensureSearchable(indexName); } diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/CacheStatsAPIIndicesRequestCacheIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/CacheStatsAPIIndicesRequestCacheIT.java index 0539f96e429c1..28bac3c7441b6 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/CacheStatsAPIIndicesRequestCacheIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/CacheStatsAPIIndicesRequestCacheIT.java @@ -13,6 +13,7 @@ import org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest; import org.opensearch.action.admin.cluster.node.stats.NodesStatsResponse; import org.opensearch.action.admin.indices.cache.clear.ClearIndicesCacheRequest; +import org.opensearch.action.admin.indices.forcemerge.ForceMergeResponse; import org.opensearch.action.admin.indices.stats.CommonStatsFlags; import org.opensearch.action.search.SearchResponse; import org.opensearch.client.Client; @@ -23,12 +24,14 @@ import org.opensearch.common.cache.stats.ImmutableCacheStats; import org.opensearch.common.cache.stats.ImmutableCacheStatsHolder; import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.FeatureFlags; import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.core.xcontent.MediaTypeRegistry; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.index.IndexSettings; import org.opensearch.index.cache.request.RequestCacheStats; import org.opensearch.index.query.QueryBuilders; import org.opensearch.test.OpenSearchIntegTestCase; @@ -266,10 +269,14 @@ private void startIndex(Client client, String indexName) throws InterruptedExcep .put(IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING.getKey(), true) .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + // Disable index refreshing to avoid cache being invalidated mid-test + .put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), TimeValue.timeValueMillis(-1)) ) .get() ); indexRandom(true, client.prepareIndex(indexName).setSource("k", "hello")); + // 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(indexName).setFlush(true).get(); ensureSearchable(indexName); } 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 index dbd78a2584f9c..db23e7b877596 100644 --- a/server/src/main/java/org/opensearch/common/cache/stats/ImmutableCacheStats.java +++ b/server/src/main/java/org/opensearch/common/cache/stats/ImmutableCacheStats.java @@ -115,6 +115,29 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws return builder; } + @Override + public String toString() { + return Fields.HIT_COUNT + + "=" + + hits + + ", " + + Fields.MISS_COUNT + + "=" + + misses + + ", " + + Fields.EVICTIONS + + "=" + + evictions + + ", " + + Fields.SIZE_IN_BYTES + + "=" + + sizeInBytes + + ", " + + Fields.ITEM_COUNT + + "=" + + items; + } + /** * Field names used to write the values in this object to XContent. */ From 16c88062126447b51ab15022a76aaa32de600225 Mon Sep 17 00:00:00 2001 From: gaobinlong Date: Sat, 15 Jun 2024 04:38:26 +0800 Subject: [PATCH 15/84] Add fingerprint ingest processor (#13724) * Add fingerprint ingest processor Signed-off-by: Gao Binlong * Ignore metadata fields Signed-off-by: Gao Binlong * Add sha3-256 hash method Signed-off-by: Gao Binlong * Remove unused code Signed-off-by: Gao Binlong * Add exclude_fields and remove include_all_fields Signed-off-by: Gao Binlong * Modify processor description Signed-off-by: Gao Binlong * Make FingerprintProcessor being final Signed-off-by: Gao Binlong * Optimize error message and check if field name is empty string Signed-off-by: Gao Binlong * Fix yaml test failure Signed-off-by: Gao Binlong * Prepend string length to the field value Signed-off-by: Gao Binlong * Append hash method with version number Signed-off-by: Gao Binlong * Update supported version in yml test file Signed-off-by: Gao Binlong * Add more comment Signed-off-by: Gao Binlong * Prepend hash method to the hash value and add more test cases Signed-off-by: Gao Binlong --------- Signed-off-by: Gao Binlong --- CHANGELOG.md | 1 + .../ingest/common/FingerprintProcessor.java | 279 +++++++ .../common/IngestCommonModulePlugin.java | 1 + .../FingerprintProcessorFactoryTests.java | 119 +++ .../common/FingerprintProcessorTests.java | 176 ++++ .../rest-api-spec/test/ingest/10_basic.yml | 16 + .../test/ingest/340_fingerprint_processor.yml | 786 ++++++++++++++++++ .../common/hash/MessageDigests.java | 13 + .../common/hash/MessageDigestsTests.java | 25 + 9 files changed, 1416 insertions(+) create mode 100644 modules/ingest-common/src/main/java/org/opensearch/ingest/common/FingerprintProcessor.java create mode 100644 modules/ingest-common/src/test/java/org/opensearch/ingest/common/FingerprintProcessorFactoryTests.java create mode 100644 modules/ingest-common/src/test/java/org/opensearch/ingest/common/FingerprintProcessorTests.java create mode 100644 modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/340_fingerprint_processor.yml diff --git a/CHANGELOG.md b/CHANGELOG.md index 9b7e333174354..4e1d41ec6b152 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -5,6 +5,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ## [Unreleased 2.x] ### Added +- Add fingerprint ingest processor ([#13724](https://github.com/opensearch-project/OpenSearch/pull/13724)) ### Dependencies - Bump `org.gradle.test-retry` from 1.5.8 to 1.5.9 ([#13442](https://github.com/opensearch-project/OpenSearch/pull/13442)) diff --git a/modules/ingest-common/src/main/java/org/opensearch/ingest/common/FingerprintProcessor.java b/modules/ingest-common/src/main/java/org/opensearch/ingest/common/FingerprintProcessor.java new file mode 100644 index 0000000000000..c2f59bf586c81 --- /dev/null +++ b/modules/ingest-common/src/main/java/org/opensearch/ingest/common/FingerprintProcessor.java @@ -0,0 +1,279 @@ +/* + * 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.ingest.common; + +import org.opensearch.common.Nullable; +import org.opensearch.common.hash.MessageDigests; +import org.opensearch.core.common.Strings; +import org.opensearch.ingest.AbstractProcessor; +import org.opensearch.ingest.ConfigurationUtils; +import org.opensearch.ingest.IngestDocument; +import org.opensearch.ingest.Processor; + +import java.nio.charset.StandardCharsets; +import java.security.MessageDigest; +import java.util.Base64; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.opensearch.ingest.ConfigurationUtils.newConfigurationException; + +/** + * Processor that generating hash value for the specified fields or fields not in the specified excluded list + */ +public final class FingerprintProcessor extends AbstractProcessor { + public static final String TYPE = "fingerprint"; + // this processor is introduced in 2.16.0, we append the OpenSearch version to the hash method name to ensure + // that this processor always generates same hash value based on a specific hash method, if the processing logic + // of this processor changes in future version, the version number in the hash method should be increased correspondingly. + private static final Set HASH_METHODS = Set.of("MD5@2.16.0", "SHA-1@2.16.0", "SHA-256@2.16.0", "SHA3-256@2.16.0"); + + // fields used to generate hash value + private final List fields; + // all fields other than the excluded fields are used to generate hash value + private final List excludeFields; + // the target field to store the hash value, defaults to fingerprint + private final String targetField; + // hash method used to generate the hash value, defaults to SHA-1 + private final String hashMethod; + private final boolean ignoreMissing; + + FingerprintProcessor( + String tag, + String description, + @Nullable List fields, + @Nullable List excludeFields, + String targetField, + String hashMethod, + boolean ignoreMissing + ) { + super(tag, description); + if (fields != null && !fields.isEmpty()) { + if (fields.stream().anyMatch(Strings::isNullOrEmpty)) { + throw new IllegalArgumentException("field name in [fields] cannot be null nor empty"); + } + if (excludeFields != null && !excludeFields.isEmpty()) { + throw new IllegalArgumentException("either fields or exclude_fields can be set"); + } + } + if (excludeFields != null && !excludeFields.isEmpty() && excludeFields.stream().anyMatch(Strings::isNullOrEmpty)) { + throw new IllegalArgumentException("field name in [exclude_fields] cannot be null nor empty"); + } + + if (!HASH_METHODS.contains(hashMethod.toUpperCase(Locale.ROOT))) { + throw new IllegalArgumentException("hash method must be MD5@2.16.0, SHA-1@2.16.0 or SHA-256@2.16.0 or SHA3-256@2.16.0"); + } + this.fields = fields; + this.excludeFields = excludeFields; + this.targetField = targetField; + this.hashMethod = hashMethod; + this.ignoreMissing = ignoreMissing; + } + + public List getFields() { + return fields; + } + + public List getExcludeFields() { + return excludeFields; + } + + public String getTargetField() { + return targetField; + } + + public String getHashMethod() { + return hashMethod; + } + + public boolean isIgnoreMissing() { + return ignoreMissing; + } + + @Override + public IngestDocument execute(IngestDocument document) { + // we should deduplicate and sort the field names to make sure we can get consistent hash value + final List sortedFields; + Set existingFields = new HashSet<>(document.getSourceAndMetadata().keySet()); + Set metadataFields = document.getMetadata() + .keySet() + .stream() + .map(IngestDocument.Metadata::getFieldName) + .collect(Collectors.toSet()); + // metadata fields such as _index, _id and _routing are ignored + if (fields != null && !fields.isEmpty()) { + sortedFields = fields.stream() + .distinct() + .filter(field -> !metadataFields.contains(field)) + .sorted() + .collect(Collectors.toList()); + } else if (excludeFields != null && !excludeFields.isEmpty()) { + sortedFields = existingFields.stream() + .filter(field -> !metadataFields.contains(field) && !excludeFields.contains(field)) + .sorted() + .collect(Collectors.toList()); + } else { + sortedFields = existingFields.stream().filter(field -> !metadataFields.contains(field)).sorted().collect(Collectors.toList()); + } + assert (!sortedFields.isEmpty()); + + final StringBuilder concatenatedFields = new StringBuilder(); + sortedFields.forEach(field -> { + if (!document.hasField(field)) { + if (ignoreMissing) { + return; + } else { + throw new IllegalArgumentException("field [" + field + "] doesn't exist"); + } + } + + final Object value = document.getFieldValue(field, Object.class); + if (value instanceof Map) { + @SuppressWarnings("unchecked") + Map flattenedMap = toFlattenedMap((Map) value); + flattenedMap.entrySet().stream().sorted(Map.Entry.comparingByKey()).forEach(entry -> { + String fieldValue = String.valueOf(entry.getValue()); + concatenatedFields.append("|") + .append(field) + .append(".") + .append(entry.getKey()) + .append("|") + .append(fieldValue.length()) + .append(":") + .append(fieldValue); + }); + } else { + String fieldValue = String.valueOf(value); + concatenatedFields.append("|").append(field).append("|").append(fieldValue.length()).append(":").append(fieldValue); + } + }); + // if all specified fields don't exist and ignore_missing is true, then do nothing + if (concatenatedFields.length() == 0) { + return document; + } + concatenatedFields.append("|"); + + MessageDigest messageDigest = HashMethod.fromMethodName(hashMethod); + assert (messageDigest != null); + messageDigest.update(concatenatedFields.toString().getBytes(StandardCharsets.UTF_8)); + document.setFieldValue(targetField, hashMethod + ":" + Base64.getEncoder().encodeToString(messageDigest.digest())); + + return document; + } + + @Override + public String getType() { + return TYPE; + } + + /** + * Convert a map containing nested fields to a flattened map, + * for example, if the original map is + * { + * "a": { + * "b": 1, + * "c": 2 + * } + * }, then the converted map is + * { + * "a.b": 1, + * "a.c": 2 + * } + * @param map the original map which may contain nested fields + * @return a flattened map which has only one level fields + */ + @SuppressWarnings("unchecked") + private Map toFlattenedMap(Map map) { + Map flattenedMap = new HashMap<>(); + for (Map.Entry entry : map.entrySet()) { + if (entry.getValue() instanceof Map) { + toFlattenedMap((Map) entry.getValue()).forEach( + (key, value) -> flattenedMap.put(entry.getKey() + "." + key, value) + ); + } else { + flattenedMap.put(entry.getKey(), entry.getValue()); + } + } + return flattenedMap; + } + + /** + * The supported hash methods used to generate hash value + */ + enum HashMethod { + MD5(MessageDigests.md5()), + SHA1(MessageDigests.sha1()), + SHA256(MessageDigests.sha256()), + SHA3256(MessageDigests.sha3256()); + + private final MessageDigest messageDigest; + + HashMethod(MessageDigest messageDigest) { + this.messageDigest = messageDigest; + } + + public static MessageDigest fromMethodName(String methodName) { + String name = methodName.toUpperCase(Locale.ROOT); + switch (name) { + case "MD5@2.16.0": + return MD5.messageDigest; + case "SHA-1@2.16.0": + return SHA1.messageDigest; + case "SHA-256@2.16.0": + return SHA256.messageDigest; + case "SHA3-256@2.16.0": + return SHA3256.messageDigest; + default: + return null; + } + } + } + + public static final class Factory implements Processor.Factory { + @Override + public FingerprintProcessor create( + Map registry, + String processorTag, + String description, + Map config + ) throws Exception { + List fields = ConfigurationUtils.readOptionalList(TYPE, processorTag, config, "fields"); + List excludeFields = ConfigurationUtils.readOptionalList(TYPE, processorTag, config, "exclude_fields"); + if (fields != null && !fields.isEmpty()) { + if (fields.stream().anyMatch(Strings::isNullOrEmpty)) { + throw newConfigurationException(TYPE, processorTag, "fields", "field name cannot be null nor empty"); + } + if (excludeFields != null && !excludeFields.isEmpty()) { + throw newConfigurationException(TYPE, processorTag, "fields", "either fields or exclude_fields can be set"); + } + } + if (excludeFields != null && !excludeFields.isEmpty() && excludeFields.stream().anyMatch(Strings::isNullOrEmpty)) { + throw newConfigurationException(TYPE, processorTag, "exclude_fields", "field name cannot be null nor empty"); + } + + String targetField = ConfigurationUtils.readStringProperty(TYPE, processorTag, config, "target_field", "fingerprint"); + String hashMethod = ConfigurationUtils.readStringProperty(TYPE, processorTag, config, "hash_method", "SHA-1@2.16.0"); + if (!HASH_METHODS.contains(hashMethod.toUpperCase(Locale.ROOT))) { + throw newConfigurationException( + TYPE, + processorTag, + "hash_method", + "hash method must be MD5@2.16.0, SHA-1@2.16.0, SHA-256@2.16.0 or SHA3-256@2.16.0" + ); + } + boolean ignoreMissing = ConfigurationUtils.readBooleanProperty(TYPE, processorTag, config, "ignore_missing", false); + return new FingerprintProcessor(processorTag, description, fields, excludeFields, targetField, hashMethod, ignoreMissing); + } + } +} diff --git a/modules/ingest-common/src/main/java/org/opensearch/ingest/common/IngestCommonModulePlugin.java b/modules/ingest-common/src/main/java/org/opensearch/ingest/common/IngestCommonModulePlugin.java index 0f8b248fd5af8..162934efa6778 100644 --- a/modules/ingest-common/src/main/java/org/opensearch/ingest/common/IngestCommonModulePlugin.java +++ b/modules/ingest-common/src/main/java/org/opensearch/ingest/common/IngestCommonModulePlugin.java @@ -109,6 +109,7 @@ public Map getProcessors(Processor.Parameters paramet processors.put(CopyProcessor.TYPE, new CopyProcessor.Factory(parameters.scriptService)); processors.put(RemoveByPatternProcessor.TYPE, new RemoveByPatternProcessor.Factory()); processors.put(CommunityIdProcessor.TYPE, new CommunityIdProcessor.Factory()); + processors.put(FingerprintProcessor.TYPE, new FingerprintProcessor.Factory()); return Collections.unmodifiableMap(processors); } diff --git a/modules/ingest-common/src/test/java/org/opensearch/ingest/common/FingerprintProcessorFactoryTests.java b/modules/ingest-common/src/test/java/org/opensearch/ingest/common/FingerprintProcessorFactoryTests.java new file mode 100644 index 0000000000000..74ad4cade7b37 --- /dev/null +++ b/modules/ingest-common/src/test/java/org/opensearch/ingest/common/FingerprintProcessorFactoryTests.java @@ -0,0 +1,119 @@ +/* + * 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.ingest.common; + +import org.opensearch.OpenSearchParseException; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import static org.hamcrest.CoreMatchers.equalTo; + +public class FingerprintProcessorFactoryTests extends OpenSearchTestCase { + + private FingerprintProcessor.Factory factory; + + @Before + public void init() { + factory = new FingerprintProcessor.Factory(); + } + + public void testCreate() throws Exception { + Map config = new HashMap<>(); + + List fields = null; + List excludeFields = null; + if (randomBoolean()) { + fields = List.of(randomAlphaOfLength(10)); + config.put("fields", fields); + } else { + excludeFields = List.of(randomAlphaOfLength(10)); + config.put("exclude_fields", excludeFields); + } + + String targetField = null; + if (randomBoolean()) { + targetField = randomAlphaOfLength(10); + } + config.put("target_field", targetField); + + boolean ignoreMissing = randomBoolean(); + config.put("ignore_missing", ignoreMissing); + String processorTag = randomAlphaOfLength(10); + FingerprintProcessor fingerprintProcessor = factory.create(null, processorTag, null, config); + assertThat(fingerprintProcessor.getTag(), equalTo(processorTag)); + assertThat(fingerprintProcessor.getFields(), equalTo(fields)); + assertThat(fingerprintProcessor.getExcludeFields(), equalTo(excludeFields)); + assertThat(fingerprintProcessor.getTargetField(), equalTo(Objects.requireNonNullElse(targetField, "fingerprint"))); + assertThat(fingerprintProcessor.isIgnoreMissing(), equalTo(ignoreMissing)); + } + + public void testCreateWithFields() throws Exception { + Map config = new HashMap<>(); + config.put("fields", List.of(randomAlphaOfLength(10))); + config.put("exclude_fields", List.of(randomAlphaOfLength(10))); + try { + factory.create(null, null, null, config); + fail("factory create should have failed"); + } catch (OpenSearchParseException e) { + assertThat(e.getMessage(), equalTo("[fields] either fields or exclude_fields can be set")); + } + + config = new HashMap<>(); + List fields = new ArrayList<>(); + if (randomBoolean()) { + fields.add(null); + } else { + fields.add(""); + } + config.put("fields", fields); + try { + factory.create(null, null, null, config); + fail("factory create should have failed"); + } catch (OpenSearchParseException e) { + assertThat(e.getMessage(), equalTo("[fields] field name cannot be null nor empty")); + } + + config = new HashMap<>(); + List excludeFields = new ArrayList<>(); + if (randomBoolean()) { + excludeFields.add(null); + } else { + excludeFields.add(""); + } + config.put("exclude_fields", excludeFields); + try { + factory.create(null, null, null, config); + fail("factory create should have failed"); + } catch (OpenSearchParseException e) { + assertThat(e.getMessage(), equalTo("[exclude_fields] field name cannot be null nor empty")); + } + } + + public void testCreateWithHashMethod() throws Exception { + Map config = new HashMap<>(); + List fields = List.of(randomAlphaOfLength(10)); + config.put("fields", fields); + config.put("hash_method", randomAlphaOfLength(10)); + try { + factory.create(null, null, null, config); + fail("factory create should have failed"); + } catch (OpenSearchParseException e) { + assertThat( + e.getMessage(), + equalTo("[hash_method] hash method must be MD5@2.16.0, SHA-1@2.16.0, SHA-256@2.16.0 or SHA3-256@2.16.0") + ); + } + } +} diff --git a/modules/ingest-common/src/test/java/org/opensearch/ingest/common/FingerprintProcessorTests.java b/modules/ingest-common/src/test/java/org/opensearch/ingest/common/FingerprintProcessorTests.java new file mode 100644 index 0000000000000..67a82f28fb763 --- /dev/null +++ b/modules/ingest-common/src/test/java/org/opensearch/ingest/common/FingerprintProcessorTests.java @@ -0,0 +1,176 @@ +/* + * 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.ingest.common; + +import org.opensearch.ingest.IngestDocument; +import org.opensearch.ingest.Processor; +import org.opensearch.ingest.RandomDocumentPicks; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +import static org.hamcrest.Matchers.equalTo; + +public class FingerprintProcessorTests extends OpenSearchTestCase { + private final List hashMethods = List.of("MD5@2.16.0", "SHA-1@2.16.0", "SHA-256@2.16.0", "SHA3-256@2.16.0"); + + public void testGenerateFingerprint() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + List fields = null; + List excludeFields = null; + if (randomBoolean()) { + fields = new ArrayList<>(); + for (int i = 0; i < randomIntBetween(1, 10); i++) { + fields.add(RandomDocumentPicks.addRandomField(random(), ingestDocument, randomAlphaOfLength(10))); + } + } else { + excludeFields = new ArrayList<>(); + for (int i = 0; i < randomIntBetween(1, 10); i++) { + excludeFields.add(RandomDocumentPicks.addRandomField(random(), ingestDocument, randomAlphaOfLength(10))); + } + } + + String targetField = "fingerprint"; + if (randomBoolean()) { + targetField = randomAlphaOfLength(10); + } + + String hashMethod = randomFrom(hashMethods); + Processor processor = createFingerprintProcessor(fields, excludeFields, targetField, hashMethod, false); + processor.execute(ingestDocument); + assertThat(ingestDocument.hasField(targetField), equalTo(true)); + } + + public void testCreateFingerprintProcessorFailed() { + List fields = new ArrayList<>(); + if (randomBoolean()) { + fields.add(null); + } else { + fields.add(""); + } + fields.add(randomAlphaOfLength(10)); + + assertThrows( + "field name in [fields] cannot be null nor empty", + IllegalArgumentException.class, + () -> createFingerprintProcessor(fields, null, null, randomFrom(hashMethods), false) + ); + + List excludeFields = new ArrayList<>(); + if (randomBoolean()) { + excludeFields.add(null); + } else { + excludeFields.add(""); + } + excludeFields.add(randomAlphaOfLength(10)); + + assertThrows( + "field name in [exclude_fields] cannot be null nor empty", + IllegalArgumentException.class, + () -> createFingerprintProcessor(null, excludeFields, null, randomFrom(hashMethods), false) + ); + + assertThrows( + "either fields or exclude_fields can be set", + IllegalArgumentException.class, + () -> createFingerprintProcessor( + List.of(randomAlphaOfLength(10)), + List.of(randomAlphaOfLength(10)), + null, + randomFrom(hashMethods), + false + ) + ); + + assertThrows( + "hash method must be MD5@2.16.0, SHA-1@2.16.0, SHA-256@2.16.0 or SHA3-256@2.16.0", + IllegalArgumentException.class, + () -> createFingerprintProcessor(Collections.emptyList(), null, "fingerprint", randomAlphaOfLength(10), false) + ); + } + + public void testEmptyFieldAndExcludeFields() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + List fields = null; + List excludeFields = null; + if (randomBoolean()) { + fields = new ArrayList<>(); + } else { + excludeFields = new ArrayList<>(); + } + String targetField = "fingerprint"; + if (randomBoolean()) { + targetField = randomAlphaOfLength(10); + } + + String hashMethod = randomFrom(hashMethods); + Processor processor = createFingerprintProcessor(fields, excludeFields, targetField, hashMethod, false); + processor.execute(ingestDocument); + assertThat(ingestDocument.hasField(targetField), equalTo(true)); + } + + public void testIgnoreMissing() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + String nonExistingFieldName = RandomDocumentPicks.randomNonExistingFieldName(random(), ingestDocument); + List nonExistingFields = List.of(nonExistingFieldName); + Processor processor = createFingerprintProcessor(nonExistingFields, null, "fingerprint", randomFrom(hashMethods), false); + assertThrows( + "field [" + nonExistingFieldName + "] doesn't exist", + IllegalArgumentException.class, + () -> processor.execute(ingestDocument) + ); + + String targetField = "fingerprint"; + Processor processorWithIgnoreMissing = createFingerprintProcessor( + nonExistingFields, + null, + "fingerprint", + randomFrom(hashMethods), + true + ); + processorWithIgnoreMissing.execute(ingestDocument); + assertThat(ingestDocument.hasField(targetField), equalTo(false)); + } + + public void testIgnoreMetadataFields() throws Exception { + IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); + List metadataFields = ingestDocument.getMetadata() + .keySet() + .stream() + .map(IngestDocument.Metadata::getFieldName) + .collect(Collectors.toList()); + + String existingFieldName = RandomDocumentPicks.addRandomField(random(), ingestDocument, randomAlphaOfLength(10)); + List fields = List.of(existingFieldName, metadataFields.get(randomIntBetween(0, metadataFields.size() - 1))); + + String targetField = "fingerprint"; + String algorithm = randomFrom(hashMethods); + Processor processor = createFingerprintProcessor(fields, null, targetField, algorithm, false); + + processor.execute(ingestDocument); + String fingerprint = ingestDocument.getFieldValue(targetField, String.class); + + processor = createFingerprintProcessor(List.of(existingFieldName), null, targetField, algorithm, false); + processor.execute(ingestDocument); + assertThat(ingestDocument.getFieldValue(targetField, String.class), equalTo(fingerprint)); + } + + private FingerprintProcessor createFingerprintProcessor( + List fields, + List excludeFields, + String targetField, + String hashMethod, + boolean ignoreMissing + ) { + return new FingerprintProcessor(randomAlphaOfLength(10), null, fields, excludeFields, targetField, hashMethod, ignoreMissing); + } +} diff --git a/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/10_basic.yml b/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/10_basic.yml index 2a816f0386667..9bf4faf53a999 100644 --- a/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/10_basic.yml +++ b/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/10_basic.yml @@ -86,3 +86,19 @@ - do: nodes.info: {} - contains: { nodes.$cluster_manager.ingest.processors: { type: community_id } } + +--- +"Fingerprint processor exists": + - skip: + version: " - 2.15.99" + features: contains + reason: "fingerprint processor was introduced in 2.16.0 and contains is a newly added assertion" + - do: + cluster.state: {} + + # Get cluster-manager node id + - set: { cluster_manager_node: cluster_manager } + + - do: + nodes.info: {} + - contains: { nodes.$cluster_manager.ingest.processors: { type: fingerprint } } diff --git a/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/340_fingerprint_processor.yml b/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/340_fingerprint_processor.yml new file mode 100644 index 0000000000000..04568916239f4 --- /dev/null +++ b/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/340_fingerprint_processor.yml @@ -0,0 +1,786 @@ +--- +teardown: + - do: + ingest.delete_pipeline: + id: "1" + ignore: 404 + +--- +"Test creat fingerprint processor": + - skip: + version: " - 2.15.99" + reason: "introduced in 2.16.0" + - do: + catch: /field name cannot be null nor empty/ + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + "fields": [null] + } + } + ] + } + - do: + catch: /field name cannot be null nor empty/ + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + "exclude_fields": [""] + } + } + ] + } + - do: + catch: /either fields or exclude\_fields can be set/ + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + "fields": ["foo"], + "exclude_fields": ["bar"] + } + } + ] + } + + - do: + catch: /hash method must be MD5@2.16.0\, SHA\-1@2.16.0, SHA\-256@2.16.0 or SHA3\-256@2.16.0/ + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + "fields": ["foo"], + "hash_method": "non-existing" + } + } + ] + } + + - do: + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + "fields" : ["foo"], + "target_field" : "fingerprint_field", + "hash_method": "SHA-256@2.16.0" + } + } + ] + } + - match: { acknowledged: true } + + - do: + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + } + } + ] + } + - match: { acknowledged: true } + +--- +"Test fingerprint processor with ignore_missing": + - skip: + version: " - 2.15.99" + reason: "introduced in 2.16.0" + - do: + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + "fields" : ["foo"] + } + } + ] + } + - match: { acknowledged: true } + + - do: + catch: /field \[foo\] doesn't exist/ + index: + index: test + id: 1 + pipeline: "1" + body: { + bar: "bar" + } + + - do: + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + "fields" : ["foo", "bar"], + "ignore_missing" : true + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + id: 1 + pipeline: "1" + body: { + foo: "foo" + } + - do: + get: + index: test + id: 1 + - match: { _source.fingerprint: "SHA-1@2.16.0:YqpBTuHXCPV04j/7lGfWeUl8Tyo=" } + +--- +"Test fingerprint processor with custom target field": + - skip: + version: " - 2.15.99" + reason: "introduced in 2.16.0" + - do: + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + "fields" : ["foo"], + "target_field" : "target" + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + id: 1 + pipeline: "1" + body: { + foo: "foo" + } + - do: + get: + index: test + id: 1 + - match: { _source.target: "SHA-1@2.16.0:YqpBTuHXCPV04j/7lGfWeUl8Tyo=" } + +--- +"Test fingerprint processor with non-primitive fields and SHA-1": + - skip: + version: " - 2.15.99" + reason: "introduced in 2.16.0" + - do: + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + "fields" : ["foo", "bar", "zoo"] + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + id: 1 + pipeline: "1" + body: { + foo: [1, 2, 3], + bar: { + field: { + innerField: "inner" + } + }, + zoo: null + } + - do: + get: + index: test + id: 1 + - match: { _source.fingerprint: "SHA-1@2.16.0:KYJ4pc4ouFmAbgZGp7CfNoykZeo=" } + + - do: + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + id: 2 + pipeline: "1" + body: { + foo: [1, 2, 3], + bar: { + field: { + innerField: "inner" + } + }, + zoo: null + } + - do: + get: + index: test + id: 2 + - match: { _source.fingerprint: "SHA-1@2.16.0:KYJ4pc4ouFmAbgZGp7CfNoykZeo=" } + + - do: + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + "fields":[] + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + id: 3 + pipeline: "1" + body: { + foo: [1, 2, 3], + bar: { + field: { + innerField: "inner" + } + }, + zoo: null + } + - do: + get: + index: test + id: 3 + - match: { _source.fingerprint: "SHA-1@2.16.0:KYJ4pc4ouFmAbgZGp7CfNoykZeo=" } + + - do: + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + "exclude_fields":[] + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + id: 4 + pipeline: "1" + body: { + foo: [1, 2, 3], + bar: { + field: { + innerField: "inner" + } + }, + zoo: null + } + - do: + get: + index: test + id: 4 + - match: { _source.fingerprint: "SHA-1@2.16.0:KYJ4pc4ouFmAbgZGp7CfNoykZeo=" } + +--- +"Test fingerprint processor with non-primitive fields and MD5": + - skip: + version: " - 2.15.99" + reason: "introduced in 2.16.0" + - do: + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + "fields" : ["foo", "bar", "zoo"], + "hash_method" : "MD5@2.16.0" + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + id: 1 + pipeline: "1" + body: { + foo: [1, 2, 3], + bar: { + field: { + innerField: "inner" + } + }, + zoo: null + } + - do: + get: + index: test + id: 1 + - match: { _source.fingerprint: "MD5@2.16.0:NovpcJ+MYHzEZtCewcDPTQ==" } + + - do: + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + "hash_method" : "MD5@2.16.0" + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + id: 2 + pipeline: "1" + body: { + foo: [1, 2, 3], + bar: { + field: { + innerField: "inner" + } + }, + zoo: null + } + - do: + get: + index: test + id: 2 + - match: { _source.fingerprint: "MD5@2.16.0:NovpcJ+MYHzEZtCewcDPTQ==" } + + - do: + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + "fields":[], + "hash_method" : "MD5@2.16.0" + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + id: 3 + pipeline: "1" + body: { + foo: [1, 2, 3], + bar: { + field: { + innerField: "inner" + } + }, + zoo: null + } + - do: + get: + index: test + id: 3 + - match: { _source.fingerprint: "MD5@2.16.0:NovpcJ+MYHzEZtCewcDPTQ==" } + + - do: + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + "exclude_fields":[], + "hash_method" : "MD5@2.16.0" + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + id: 4 + pipeline: "1" + body: { + foo: [1, 2, 3], + bar: { + field: { + innerField: "inner" + } + }, + zoo: null + } + - do: + get: + index: test + id: 4 + - match: { _source.fingerprint: "MD5@2.16.0:NovpcJ+MYHzEZtCewcDPTQ==" } + + +--- +"Test fingerprint processor with non-primitive fields and SHA-256": + - skip: + version: " - 2.15.99" + reason: "introduced in 2.16.0" + - do: + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + "fields" : ["foo", "bar", "zoo"], + "hash_method" : "SHA-256@2.16.0" + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + id: 1 + pipeline: "1" + body: { + foo: [1, 2, 3], + bar: { + field: { + innerField: "inner" + } + }, + zoo: null + } + - do: + get: + index: test + id: 1 + - match: { _source.fingerprint: "SHA-256@2.16.0:Sdlg0BodM3n1my4BvaTfJCPrvHxfrxno0kCLfMaC+XY=" } + + - do: + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + "hash_method" : "SHA-256@2.16.0" + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + id: 2 + pipeline: "1" + body: { + foo: [1, 2, 3], + bar: { + field: { + innerField: "inner" + } + }, + zoo: null + } + - do: + get: + index: test + id: 2 + - match: { _source.fingerprint: "SHA-256@2.16.0:Sdlg0BodM3n1my4BvaTfJCPrvHxfrxno0kCLfMaC+XY=" } + + - do: + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + "fields":[], + "hash_method" : "SHA-256@2.16.0" + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + id: 3 + pipeline: "1" + body: { + foo: [1, 2, 3], + bar: { + field: { + innerField: "inner" + } + }, + zoo: null + } + - do: + get: + index: test + id: 3 + - match: { _source.fingerprint: "SHA-256@2.16.0:Sdlg0BodM3n1my4BvaTfJCPrvHxfrxno0kCLfMaC+XY=" } + + - do: + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + "exclude_fields":[], + "hash_method" : "SHA-256@2.16.0" + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + id: 4 + pipeline: "1" + body: { + foo: [1, 2, 3], + bar: { + field: { + innerField: "inner" + } + }, + zoo: null + } + - do: + get: + index: test + id: 4 + - match: { _source.fingerprint: "SHA-256@2.16.0:Sdlg0BodM3n1my4BvaTfJCPrvHxfrxno0kCLfMaC+XY=" } + +--- +"Test fingerprint processor with non-primitive fields and SHA3-256": + - skip: + version: " - 2.15.99" + reason: "introduced in 2.16.0" + - do: + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + "fields" : ["foo", "bar", "zoo"], + "hash_method" : "SHA3-256@2.16.0" + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + id: 1 + pipeline: "1" + body: { + foo: [1, 2, 3], + bar: { + field: { + innerField: "inner" + } + }, + zoo: null + } + - do: + get: + index: test + id: 1 + - match: { _source.fingerprint: "SHA3-256@2.16.0:+GZCkMLEMkUA/4IrEZEZZYsVMbZdpJ92ppN3wUsFYOI=" } + + - do: + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + "hash_method" : "SHA3-256@2.16.0" + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + id: 2 + pipeline: "1" + body: { + foo: [1, 2, 3], + bar: { + field: { + innerField: "inner" + } + }, + zoo: null + } + - do: + get: + index: test + id: 2 + - match: { _source.fingerprint: "SHA3-256@2.16.0:+GZCkMLEMkUA/4IrEZEZZYsVMbZdpJ92ppN3wUsFYOI=" } + + - do: + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + "fields":[], + "hash_method" : "SHA3-256@2.16.0" + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + id: 3 + pipeline: "1" + body: { + foo: [1, 2, 3], + bar: { + field: { + innerField: "inner" + } + }, + zoo: null + } + - do: + get: + index: test + id: 3 + - match: { _source.fingerprint: "SHA3-256@2.16.0:+GZCkMLEMkUA/4IrEZEZZYsVMbZdpJ92ppN3wUsFYOI=" } + + - do: + ingest.put_pipeline: + id: "1" + body: > + { + "processors": [ + { + "fingerprint" : { + "exclude_fields":[], + "hash_method" : "SHA3-256@2.16.0" + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + id: 4 + pipeline: "1" + body: { + foo: [1, 2, 3], + bar: { + field: { + innerField: "inner" + } + }, + zoo: null + } + - do: + get: + index: test + id: 4 + - match: { _source.fingerprint: "SHA3-256@2.16.0:+GZCkMLEMkUA/4IrEZEZZYsVMbZdpJ92ppN3wUsFYOI=" } diff --git a/server/src/main/java/org/opensearch/common/hash/MessageDigests.java b/server/src/main/java/org/opensearch/common/hash/MessageDigests.java index f53f60a3a97a3..123bd3489bedb 100644 --- a/server/src/main/java/org/opensearch/common/hash/MessageDigests.java +++ b/server/src/main/java/org/opensearch/common/hash/MessageDigests.java @@ -58,6 +58,7 @@ private static ThreadLocal createThreadLocalMessageDigest(String private static final ThreadLocal MD5_DIGEST = createThreadLocalMessageDigest("MD5"); private static final ThreadLocal SHA_1_DIGEST = createThreadLocalMessageDigest("SHA-1"); private static final ThreadLocal SHA_256_DIGEST = createThreadLocalMessageDigest("SHA-256"); + private static final ThreadLocal SHA3_256_DIGEST = createThreadLocalMessageDigest("SHA3-256"); /** * Returns a {@link MessageDigest} instance for MD5 digests; note @@ -95,6 +96,18 @@ public static MessageDigest sha256() { return get(SHA_256_DIGEST); } + /** + * Returns a {@link MessageDigest} instance for SHA3-256 digests; + * note that the instance returned is thread local and must not be + * shared amongst threads. + * + * @return a thread local {@link MessageDigest} instance that + * provides SHA3-256 message digest functionality. + */ + public static MessageDigest sha3256() { + return get(SHA3_256_DIGEST); + } + private static MessageDigest get(ThreadLocal messageDigest) { MessageDigest instance = messageDigest.get(); instance.reset(); diff --git a/server/src/test/java/org/opensearch/common/hash/MessageDigestsTests.java b/server/src/test/java/org/opensearch/common/hash/MessageDigestsTests.java index 9e793e5487eb8..6b7cfb4c8932c 100644 --- a/server/src/test/java/org/opensearch/common/hash/MessageDigestsTests.java +++ b/server/src/test/java/org/opensearch/common/hash/MessageDigestsTests.java @@ -91,6 +91,31 @@ public void testSha256() throws Exception { ); } + public void testSha3256() throws Exception { + assertHash("a7ffc6f8bf1ed76651c14756a061d662f580ff4de43b49fa82d80a4b80f8434a", "", MessageDigests.sha3256()); + assertHash("3a985da74fe225b2045c172d6bd390bd855f086e3e9d525b46bfe24511431532", "abc", MessageDigests.sha3256()); + assertHash( + "41c0dba2a9d6240849100376a8235e2c82e1b9998a999e21db32dd97496d3376", + "abcdbcdecdefdefgefghfghighijhijkijkljklmklmnlmnomnopnopq", + MessageDigests.sha3256() + ); + assertHash( + "5c8875ae474a3634ba4fd55ec85bffd661f32aca75c6d699d0cdcb6c115891c1", + new String(new char[1000000]).replace("\0", "a"), + MessageDigests.sha3256() + ); + assertHash( + "69070dda01975c8c120c3aada1b282394e7f032fa9cf32f4cb2259a0897dfc04", + "The quick brown fox jumps over the lazy dog", + MessageDigests.sha3256() + ); + assertHash( + "cc80b0b13ba89613d93f02ee7ccbe72ee26c6edfe577f22e63a1380221caedbc", + "The quick brown fox jumps over the lazy cog", + MessageDigests.sha3256() + ); + } + public void testToHexString() throws Exception { BigInteger expected = BigInteger.probablePrime(256, random()); byte[] bytes = expected.toByteArray(); From 7650e6412056f0b06e069ae6b2936f9ea2da4a7f Mon Sep 17 00:00:00 2001 From: Muneer Kolarkunnu <33829651+akolarkunnu@users.noreply.github.com> Date: Sat, 15 Jun 2024 06:11:02 +0530 Subject: [PATCH 16/84] COMPAT locale provider will be removed in a future release (#14345) * COMPAT locale provider will be removed in a future release Description: From JDK21 onwards a new warning has started to come, "WARNING: COMPAT locale provider will be removed in a future release". So, we have to avoid usage of COMPAT provider. We were setting exlpicitly to COMPAT locale provider in couple of places, this change is to convert COMPAT to CLDR locale provider. After this change, couple of tests started to fail becasue some locale data has minor changes in CLDR compared to COMPAT. For example, day and month short names of GERMAN "de" locale are different in CLDR and COMPAT, just need to add a . in the end for CLDR. Resolves #11550 Signed-off-by: Abdul Muneer Kolarkunnu * COMPAT locale provider will be removed in a future release Description: From JDK21 onwards a new warning has started to come, "WARNING: COMPAT locale provider will be removed in a future release". So, we have to avoid usage of COMPAT provider. We were setting exlpicitly to COMPAT locale provider in couple of places, this change is to convert COMPAT to CLDR locale provider. After this change, couple of tests started to fail becasue some locale data has minor changes in CLDR compared to COMPAT. For example, day and month short names of GERMAN "de" locale are different in CLDR and COMPAT, just need to add a . in the end for CLDR. Resolves #11550 Signed-off-by: Abdul Muneer Kolarkunnu --------- Signed-off-by: Abdul Muneer Kolarkunnu --- CHANGELOG-3.0.md | 1 + .../gradle/OpenSearchTestBasePlugin.java | 2 +- .../tools/launchers/SystemJvmOptions.java | 7 +------ gradle/ide.gradle | 2 +- .../opensearch/search/query/SearchQueryIT.java | 18 ++++++++---------- .../common/joda/JavaJodaTimeDuellingTests.java | 3 ++- .../index/mapper/DateFieldMapperTests.java | 2 +- 7 files changed, 15 insertions(+), 20 deletions(-) diff --git a/CHANGELOG-3.0.md b/CHANGELOG-3.0.md index 964383078c38d..06b761b1df8bd 100644 --- a/CHANGELOG-3.0.md +++ b/CHANGELOG-3.0.md @@ -17,6 +17,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ### Dependencies ### Changed +- Changed locale provider from COMPAT to CLDR ([#14345](https://github.com/opensearch-project/OpenSearch/pull/14345)) - Migrate client transports to Apache HttpClient / Core 5.x ([#4459](https://github.com/opensearch-project/OpenSearch/pull/4459)) - Change http code on create index API with bad input raising NotXContentException from 500 to 400 ([#4773](https://github.com/opensearch-project/OpenSearch/pull/4773)) - Improve summary error message for invalid setting updates ([#4792](https://github.com/opensearch-project/OpenSearch/pull/4792)) diff --git a/buildSrc/src/main/java/org/opensearch/gradle/OpenSearchTestBasePlugin.java b/buildSrc/src/main/java/org/opensearch/gradle/OpenSearchTestBasePlugin.java index 2ea8c2d015ecc..d0cb2da9c1dd3 100644 --- a/buildSrc/src/main/java/org/opensearch/gradle/OpenSearchTestBasePlugin.java +++ b/buildSrc/src/main/java/org/opensearch/gradle/OpenSearchTestBasePlugin.java @@ -110,7 +110,7 @@ public void execute(Task t) { if (BuildParams.getRuntimeJavaVersion() == JavaVersion.VERSION_1_8) { test.systemProperty("java.locale.providers", "SPI,JRE"); } else { - test.systemProperty("java.locale.providers", "SPI,COMPAT"); + test.systemProperty("java.locale.providers", "SPI,CLDR"); if (test.getJavaVersion().compareTo(JavaVersion.VERSION_17) < 0) { test.jvmArgs("--illegal-access=warn"); } diff --git a/distribution/tools/launchers/src/main/java/org/opensearch/tools/launchers/SystemJvmOptions.java b/distribution/tools/launchers/src/main/java/org/opensearch/tools/launchers/SystemJvmOptions.java index 726c381db09f6..af7138569972a 100644 --- a/distribution/tools/launchers/src/main/java/org/opensearch/tools/launchers/SystemJvmOptions.java +++ b/distribution/tools/launchers/src/main/java/org/opensearch/tools/launchers/SystemJvmOptions.java @@ -105,13 +105,8 @@ private static String javaLocaleProviders() { SPI setting is used to allow loading custom CalendarDataProvider in jdk8 it has to be loaded from jre/lib/ext, in jdk9+ it is already within ES project and on a classpath - - Due to internationalization enhancements in JDK 9 OpenSearch need to set the provider to COMPAT otherwise time/date - parsing will break in an incompatible way for some date patterns and locales. - //TODO COMPAT will be deprecated in at some point, see please https://bugs.openjdk.java.net/browse/JDK-8232906 - See also: documentation in server/org.opensearch.common.time.IsoCalendarDataProvider */ - return "-Djava.locale.providers=SPI,COMPAT"; + return "-Djava.locale.providers=SPI,CLDR"; } } diff --git a/gradle/ide.gradle b/gradle/ide.gradle index ea353f8d92bdd..e266d9add172d 100644 --- a/gradle/ide.gradle +++ b/gradle/ide.gradle @@ -81,7 +81,7 @@ if (System.getProperty('idea.active') == 'true') { } runConfigurations { defaults(JUnit) { - vmParameters = '-ea -Djava.locale.providers=SPI,COMPAT' + vmParameters = '-ea -Djava.locale.providers=SPI,CLDR' if (BuildParams.runtimeJavaVersion > JavaVersion.VERSION_17) { vmParameters += ' -Djava.security.manager=allow' } diff --git a/server/src/internalClusterTest/java/org/opensearch/search/query/SearchQueryIT.java b/server/src/internalClusterTest/java/org/opensearch/search/query/SearchQueryIT.java index a58db51780826..01ad06757640c 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/query/SearchQueryIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/query/SearchQueryIT.java @@ -1914,14 +1914,8 @@ public void testRangeQueryWithTimeZone() throws Exception { * Test range with a custom locale, e.g. "de" in this case. Documents here mention the day of week * as "Mi" for "Mittwoch (Wednesday" and "Do" for "Donnerstag (Thursday)" and the month in the query * as "Dez" for "Dezember (December)". - * Note: this test currently needs the JVM arg `-Djava.locale.providers=SPI,COMPAT` to be set. - * When running with gradle this is done implicitly through the BuildPlugin, but when running from - * an IDE this might need to be set manually in the run configuration. See also CONTRIBUTING.md section - * on "Configuring IDEs And Running Tests". */ public void testRangeQueryWithLocaleMapping() throws Exception { - assert ("SPI,COMPAT".equals(System.getProperty("java.locale.providers"))) : "`-Djava.locale.providers=SPI,COMPAT` needs to be set"; - assertAcked( prepareCreate("test").setMapping( jsonBuilder().startObject() @@ -1938,17 +1932,21 @@ public void testRangeQueryWithLocaleMapping() throws Exception { indexRandom( true, - client().prepareIndex("test").setId("1").setSource("date_field", "Mi, 06 Dez 2000 02:55:00 -0800"), - client().prepareIndex("test").setId("2").setSource("date_field", "Do, 07 Dez 2000 02:55:00 -0800") + client().prepareIndex("test").setId("1").setSource("date_field", "Mi., 06 Dez. 2000 02:55:00 -0800"), + client().prepareIndex("test").setId("2").setSource("date_field", "Do., 07 Dez. 2000 02:55:00 -0800") ); SearchResponse searchResponse = client().prepareSearch("test") - .setQuery(QueryBuilders.rangeQuery("date_field").gte("Di, 05 Dez 2000 02:55:00 -0800").lte("Do, 07 Dez 2000 00:00:00 -0800")) + .setQuery( + QueryBuilders.rangeQuery("date_field").gte("Di., 05 Dez. 2000 02:55:00 -0800").lte("Do., 07 Dez. 2000 00:00:00 -0800") + ) .get(); assertHitCount(searchResponse, 1L); searchResponse = client().prepareSearch("test") - .setQuery(QueryBuilders.rangeQuery("date_field").gte("Di, 05 Dez 2000 02:55:00 -0800").lte("Fr, 08 Dez 2000 00:00:00 -0800")) + .setQuery( + QueryBuilders.rangeQuery("date_field").gte("Di., 05 Dez. 2000 02:55:00 -0800").lte("Fr., 08 Dez. 2000 00:00:00 -0800") + ) .get(); assertHitCount(searchResponse, 2L); } diff --git a/server/src/test/java/org/opensearch/common/joda/JavaJodaTimeDuellingTests.java b/server/src/test/java/org/opensearch/common/joda/JavaJodaTimeDuellingTests.java index 4fd8986d0b428..28edace8433e6 100644 --- a/server/src/test/java/org/opensearch/common/joda/JavaJodaTimeDuellingTests.java +++ b/server/src/test/java/org/opensearch/common/joda/JavaJodaTimeDuellingTests.java @@ -51,6 +51,7 @@ import java.util.Locale; import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.startsWith; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.is; @@ -186,7 +187,7 @@ public void testIncompatiblePatterns() { .parseDateTime("2019-01-01T01:01:01.001+0000"); String jodaZoneId = DateTimeFormat.forPattern("YYYY-MM-dd'T'HH:mm:ss.SSSz").print(dateTime); assertThat(javaZoneId, equalTo("2019-01-01T01:01:01.001Z")); - assertThat(jodaZoneId, equalTo("2019-01-01T01:01:01.001UTC")); + assertThat(jodaZoneId, startsWith("2019-01-01T01:01:01.001")); } private void assertSameMillis(String input, String jodaFormat, String javaFormat) { diff --git a/server/src/test/java/org/opensearch/index/mapper/DateFieldMapperTests.java b/server/src/test/java/org/opensearch/index/mapper/DateFieldMapperTests.java index 2aa310ae959d9..98bcaa3a1a46b 100644 --- a/server/src/test/java/org/opensearch/index/mapper/DateFieldMapperTests.java +++ b/server/src/test/java/org/opensearch/index/mapper/DateFieldMapperTests.java @@ -208,7 +208,7 @@ public void testChangeLocale() throws IOException { fieldMapping(b -> b.field("type", "date").field("format", "E, d MMM yyyy HH:mm:ss Z").field("locale", "de")) ); - mapper.parse(source(b -> b.field("field", "Mi, 06 Dez 2000 02:55:00 -0800"))); + mapper.parse(source(b -> b.field("field", "Mi., 06 Dez. 2000 02:55:00 -0800"))); } public void testNullValue() throws IOException { From 0d38d1498bc6c4b87461378ce91447dca46b2b58 Mon Sep 17 00:00:00 2001 From: Andrew Ross Date: Sat, 15 Jun 2024 09:17:38 -0700 Subject: [PATCH 17/84] Fix AwarenessAttributeDecommissionIT.testConcurrentDecommissionAction (#14372) The problem is that this test would decommission one of six nodes. The tear down logic of the test would attempt to assert on the health of the cluster by randomly selecting a node and requesting the cluster health. If this random check happened to select the node that was decommissioned, then the test would fail. The fix is to recommission the node at the end of the test. Also, the "recommission node and assert cluster health" logic was used in multiple places and could be refactored out to a helper method. Resolves #14290 Resolves #12197 Signed-off-by: Andrew Ross --- .../AwarenessAttributeDecommissionIT.java | 74 +++++++------------ 1 file changed, 25 insertions(+), 49 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/cluster/coordination/AwarenessAttributeDecommissionIT.java b/server/src/internalClusterTest/java/org/opensearch/cluster/coordination/AwarenessAttributeDecommissionIT.java index b33d57ed43189..beed6e6846b46 100644 --- a/server/src/internalClusterTest/java/org/opensearch/cluster/coordination/AwarenessAttributeDecommissionIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/cluster/coordination/AwarenessAttributeDecommissionIT.java @@ -539,18 +539,7 @@ private void assertNodesRemovedAfterZoneDecommission(boolean originalClusterMana assertEquals(originalClusterManager, currentClusterManager); } - // Will wait for all events to complete - client(activeNode).admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).get(); - - // Recommissioning the zone back to gracefully succeed the test once above tests succeeds - DeleteDecommissionStateResponse deleteDecommissionStateResponse = client(currentClusterManager).execute( - DeleteDecommissionStateAction.INSTANCE, - new DeleteDecommissionStateRequest() - ).get(); - assertTrue(deleteDecommissionStateResponse.isAcknowledged()); - - // will wait for cluster to stabilise with a timeout of 2 min as by then all nodes should have joined the cluster - ensureStableCluster(15, TimeValue.timeValueMinutes(2)); + deleteDecommissionStateAndWaitForStableCluster(currentClusterManager, 15); } public void testDecommissionFailedWhenDifferentAttributeAlreadyDecommissioned() throws Exception { @@ -617,18 +606,7 @@ public void testDecommissionFailedWhenDifferentAttributeAlreadyDecommissioned() ) ); - // Will wait for all events to complete - client(node_in_c).admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).get(); - - // Recommissioning the zone back to gracefully succeed the test once above tests succeeds - DeleteDecommissionStateResponse deleteDecommissionStateResponse = client(node_in_c).execute( - DeleteDecommissionStateAction.INSTANCE, - new DeleteDecommissionStateRequest() - ).get(); - assertTrue(deleteDecommissionStateResponse.isAcknowledged()); - - // will wait for cluster to stabilise with a timeout of 2 min as by then all nodes should have joined the cluster - ensureStableCluster(6, TimeValue.timeValueMinutes(2)); + deleteDecommissionStateAndWaitForStableCluster(node_in_c, 6); } public void testDecommissionStatusUpdatePublishedToAllNodes() throws ExecutionException, InterruptedException { @@ -748,20 +726,7 @@ public void testDecommissionStatusUpdatePublishedToAllNodes() throws ExecutionEx ); logger.info("--> Verified the decommissioned node has in_progress state."); - // Will wait for all events to complete - client(activeNode).admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).get(); - logger.info("--> Got LANGUID event"); - // Recommissioning the zone back to gracefully succeed the test once above tests succeeds - DeleteDecommissionStateResponse deleteDecommissionStateResponse = client(activeNode).execute( - DeleteDecommissionStateAction.INSTANCE, - new DeleteDecommissionStateRequest() - ).get(); - assertTrue(deleteDecommissionStateResponse.isAcknowledged()); - logger.info("--> Deleting decommission done."); - - // will wait for cluster to stabilise with a timeout of 2 min (findPeerInterval for decommissioned nodes) - // as by then all nodes should have joined the cluster - ensureStableCluster(6, TimeValue.timeValueSeconds(121)); + deleteDecommissionStateAndWaitForStableCluster(activeNode, 6); } public void testDecommissionFailedWhenAttributeNotWeighedAway() throws Exception { @@ -983,15 +948,7 @@ public void testDecommissionAcknowledgedIfWeightsNotSetForNonRoutingNode() throw assertEquals(clusterState.nodes().getDataNodes().size(), 3); assertEquals(clusterState.nodes().getClusterManagerNodes().size(), 2); - // Recommissioning the zone back to gracefully succeed the test once above tests succeeds - DeleteDecommissionStateResponse deleteDecommissionStateResponse = client(dataNodes.get(0)).execute( - DeleteDecommissionStateAction.INSTANCE, - new DeleteDecommissionStateRequest() - ).get(); - assertTrue(deleteDecommissionStateResponse.isAcknowledged()); - - // will wait for cluster to stabilise with a timeout of 2 min as by then all nodes should have joined the cluster - ensureStableCluster(6, TimeValue.timeValueMinutes(2)); + deleteDecommissionStateAndWaitForStableCluster(dataNodes.get(0), 6); } public void testConcurrentDecommissionAction() throws Exception { @@ -1019,7 +976,7 @@ public void testConcurrentDecommissionAction() throws Exception { .build() ); logger.info("--> start 3 data nodes on zones 'a' & 'b' & 'c'"); - internalCluster().startNodes( + final String bZoneDataNode = internalCluster().startNodes( Settings.builder() .put(commonSettings) .put("node.attr.zone", "a") @@ -1035,7 +992,7 @@ public void testConcurrentDecommissionAction() throws Exception { .put("node.attr.zone", "c") .put(onlyRole(commonSettings, DiscoveryNodeRole.DATA_ROLE)) .build() - ); + ).get(1); ensureStableCluster(6); ClusterHealthResponse health = client().admin() @@ -1100,6 +1057,25 @@ public void testConcurrentDecommissionAction() throws Exception { assertEquals(concurrentRuns, numRequestAcknowledged.get() + numRequestUnAcknowledged.get() + numRequestFailed.get()); assertEquals(concurrentRuns - 1, numRequestFailed.get()); assertEquals(1, numRequestAcknowledged.get() + numRequestUnAcknowledged.get()); + + deleteDecommissionStateAndWaitForStableCluster(bZoneDataNode, 6); + } + + private void deleteDecommissionStateAndWaitForStableCluster(String activeNodeName, int expectedClusterSize) throws ExecutionException, + InterruptedException { + client(activeNodeName).admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).get(); + + // Recommissioning the zone back to gracefully succeed the test once above tests succeeds + DeleteDecommissionStateResponse deleteDecommissionStateResponse = client(activeNodeName).execute( + DeleteDecommissionStateAction.INSTANCE, + new DeleteDecommissionStateRequest() + ).get(); + assertTrue(deleteDecommissionStateResponse.isAcknowledged()); + logger.info("--> Deleting decommission done."); + + // will wait for cluster to stabilise with a timeout of 2 min (findPeerInterval for decommissioned nodes) + // as by then all nodes should have joined the cluster + ensureStableCluster(expectedClusterSize, TimeValue.timeValueSeconds(121)); } private static class WaitForFailedDecommissionState implements ClusterStateObserver.Listener { From a3402d1201ce4b04111a4f3a7a76210c9ded9fd2 Mon Sep 17 00:00:00 2001 From: Sooraj Sinha <81695996+soosinha@users.noreply.github.com> Date: Mon, 17 Jun 2024 14:25:05 +0530 Subject: [PATCH 18/84] Use remote publication flag to decide which custom objects to upload (#14338) * Simplify updated customs (ClusterState.Custom & Metadata.Custom) persistence logic to remote store Signed-off-by: Sooraj Sinha --- .../RemoteClusterStateAttributesManager.java | 44 ++-- .../remote/RemoteClusterStateService.java | 97 ++++---- .../remote/RemoteGlobalMetadataManager.java | 58 +++-- .../model/RemoteClusterStateBlobStore.java | 4 +- ...oteClusterStateAttributesManagerTests.java | 186 +++++++++++++++- .../RemoteClusterStateServiceTests.java | 91 +++++++- .../RemoteGlobalMetadataManagerTests.java | 209 ++++++++++++++++++ 7 files changed, 591 insertions(+), 98 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManager.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManager.java index b052b6e1a613d..8f986423587d7 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManager.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManager.java @@ -10,6 +10,8 @@ import org.opensearch.action.LatchedActionListener; import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.DiffableUtils; +import org.opensearch.cluster.DiffableUtils.NonDiffableValueSerializer; import org.opensearch.common.CheckedRunnable; import org.opensearch.common.remote.AbstractRemoteWritableBlobEntity; import org.opensearch.common.remote.RemoteWritableEntityStore; @@ -25,10 +27,9 @@ import org.opensearch.threadpool.ThreadPool; import java.io.IOException; +import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.Map; -import java.util.Set; /** * A Manager which provides APIs to upload and download attributes of ClusterState to the {@link RemoteClusterStateBlobStore} @@ -126,18 +127,35 @@ public CheckedRunnable getAsyncMetadataReadAction( return () -> getStore(blobEntity).readAsync(blobEntity, actionListener); } - public Map getUpdatedCustoms(ClusterState clusterState, ClusterState previousClusterState) { - Map updatedCustoms = new HashMap<>(); - Set currentCustoms = new HashSet<>(clusterState.customs().keySet()); - for (Map.Entry entry : previousClusterState.customs().entrySet()) { - if (currentCustoms.contains(entry.getKey()) && !entry.getValue().equals(clusterState.customs().get(entry.getKey()))) { - updatedCustoms.put(entry.getKey(), clusterState.customs().get(entry.getKey())); - } - currentCustoms.remove(entry.getKey()); + public DiffableUtils.MapDiff> getUpdatedCustoms( + ClusterState clusterState, + ClusterState previousClusterState, + boolean isRemotePublicationEnabled, + boolean isFirstUpload + ) { + if (!isRemotePublicationEnabled) { + // When isRemotePublicationEnabled is false, we do not want store any custom objects + return DiffableUtils.diff( + Collections.emptyMap(), + Collections.emptyMap(), + DiffableUtils.getStringKeySerializer(), + NonDiffableValueSerializer.getAbstractInstance() + ); } - for (String custom : currentCustoms) { - updatedCustoms.put(custom, clusterState.customs().get(custom)); + if (isFirstUpload) { + // For first upload of ephemeral metadata, we want to upload all customs + return DiffableUtils.diff( + Collections.emptyMap(), + clusterState.customs(), + DiffableUtils.getStringKeySerializer(), + NonDiffableValueSerializer.getAbstractInstance() + ); } - return updatedCustoms; + return DiffableUtils.diff( + previousClusterState.customs(), + clusterState.customs(), + DiffableUtils.getStringKeySerializer(), + NonDiffableValueSerializer.getAbstractInstance() + ); } } diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java index ada29fdb57c57..4a1c9c8615e39 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java @@ -39,6 +39,7 @@ import org.opensearch.common.settings.Setting.Property; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.FeatureFlags; import org.opensearch.common.util.io.IOUtils; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; @@ -88,6 +89,7 @@ import static java.util.Collections.emptyList; import static java.util.Collections.emptyMap; +import static org.opensearch.common.util.FeatureFlags.REMOTE_PUBLICATION_EXPERIMENTAL; import static org.opensearch.gateway.PersistedClusterStateService.SLOW_WRITE_LOGGING_THRESHOLD; import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V2; import static org.opensearch.gateway.remote.RemoteClusterStateAttributesManager.CLUSTER_BLOCKS; @@ -159,6 +161,7 @@ public class RemoteClusterStateService implements Closeable { private final String METADATA_UPDATE_LOG_STRING = "wrote metadata for [{}] indices and skipped [{}] unchanged " + "indices, coordination metadata updated : [{}], settings metadata updated : [{}], templates metadata " + "updated : [{}], custom metadata updated : [{}], indices routing updated : [{}]"; + private final boolean isPublicationEnabled; // ToXContent Params with gateway mode. // We are using gateway context mode to persist all custom metadata. @@ -201,6 +204,9 @@ public RemoteClusterStateService( threadPool ); this.remoteClusterStateCleanupManager = new RemoteClusterStateCleanupManager(this, clusterService, remoteRoutingTableService); + this.isPublicationEnabled = FeatureFlags.isEnabled(REMOTE_PUBLICATION_EXPERIMENTAL) + && RemoteStoreNodeAttribute.isRemoteStoreClusterStateEnabled(settings) + && RemoteStoreNodeAttribute.isRemoteRoutingTableEnabled(settings); } /** @@ -221,15 +227,15 @@ public RemoteClusterStateManifestInfo writeFullMetadata(ClusterState clusterStat clusterState, new ArrayList<>(clusterState.metadata().indices().values()), emptyMap(), - clusterState.metadata().customs(), + RemoteGlobalMetadataManager.filterCustoms(clusterState.metadata().customs(), isPublicationEnabled), true, true, true, - true, - true, - true, - clusterState.customs(), - true, + isPublicationEnabled, + isPublicationEnabled, + isPublicationEnabled, + isPublicationEnabled ? clusterState.customs() : Collections.emptyMap(), + isPublicationEnabled, remoteRoutingTableService.getIndicesRouting(clusterState.getRoutingTable()) ); final RemoteClusterStateManifestInfo manifestDetails = remoteManifestManager.uploadManifest( @@ -285,28 +291,17 @@ public RemoteClusterStateManifestInfo writeIncrementalMetadata( } assert previousClusterState.metadata().coordinationMetadata().term() == clusterState.metadata().coordinationMetadata().term(); - final Map customsToBeDeletedFromRemote = new HashMap<>(previousManifest.getCustomMetadataMap()); - final Map customsToUpload = remoteGlobalMetadataManager.getUpdatedCustoms( - clusterState, - previousClusterState - ); - final Map clusterStateCustomsToBeDeleted = new HashMap<>( + boolean firstUploadForSplitGlobalMetadata = !previousManifest.hasMetadataAttributesFiles(); + + final DiffableUtils.MapDiff> customsDiff = remoteGlobalMetadataManager + .getCustomsDiff(clusterState, previousClusterState, firstUploadForSplitGlobalMetadata, isPublicationEnabled); + final DiffableUtils.MapDiff> clusterStateCustomsDiff = + remoteClusterStateAttributesManager.getUpdatedCustoms(clusterState, previousClusterState, isPublicationEnabled, false); + final Map allUploadedCustomMap = new HashMap<>(previousManifest.getCustomMetadataMap()); + final Map allUploadedClusterStateCustomsMap = new HashMap<>( previousManifest.getClusterStateCustomMap() ); - final Map clusterStateCustomsToUpload = remoteClusterStateAttributesManager.getUpdatedCustoms( - clusterState, - previousClusterState - ); - final Map allUploadedCustomMap = new HashMap<>(previousManifest.getCustomMetadataMap()); - for (final String custom : clusterState.metadata().customs().keySet()) { - // remove all the customs which are present currently - customsToBeDeletedFromRemote.remove(custom); - } final Map indicesToBeDeletedFromRemote = new HashMap<>(previousClusterState.metadata().indices()); - for (final String custom : clusterState.customs().keySet()) { - // remove all the custom which are present currently - clusterStateCustomsToBeDeleted.remove(custom); - } int numIndicesUpdated = 0; int numIndicesUnchanged = 0; final Map allUploadedIndexMetadata = previousManifest.getIndices() @@ -337,42 +332,44 @@ public RemoteClusterStateManifestInfo writeIncrementalMetadata( indicesToBeDeletedFromRemote.remove(indexMetadata.getIndex().getName()); } - DiffableUtils.MapDiff> routingTableDiff = remoteRoutingTableService + final DiffableUtils.MapDiff> routingTableDiff = remoteRoutingTableService .getIndicesRoutingMapDiff(previousClusterState.getRoutingTable(), clusterState.getRoutingTable()); - List indicesRoutingToUpload = new ArrayList<>(); + final List indicesRoutingToUpload = new ArrayList<>(); routingTableDiff.getUpserts().forEach((k, v) -> indicesRoutingToUpload.add(v)); UploadedMetadataResults uploadedMetadataResults; // For migration case from codec V0 or V1 to V2, we have added null check on metadata attribute files, // If file is empty and codec is 1 then write global metadata. - boolean firstUploadForSplitGlobalMetadata = !previousManifest.hasMetadataAttributesFiles(); boolean updateCoordinationMetadata = firstUploadForSplitGlobalMetadata || Metadata.isCoordinationMetadataEqual(previousClusterState.metadata(), clusterState.metadata()) == false; ; boolean updateSettingsMetadata = firstUploadForSplitGlobalMetadata || Metadata.isSettingsMetadataEqual(previousClusterState.metadata(), clusterState.metadata()) == false; - boolean updateTransientSettingsMetadata = firstUploadForSplitGlobalMetadata - || Metadata.isTransientSettingsMetadataEqual(previousClusterState.metadata(), clusterState.metadata()) == false; + boolean updateTransientSettingsMetadata = Metadata.isTransientSettingsMetadataEqual( + previousClusterState.metadata(), + clusterState.metadata() + ) == false; boolean updateTemplatesMetadata = firstUploadForSplitGlobalMetadata || Metadata.isTemplatesMetadataEqual(previousClusterState.metadata(), clusterState.metadata()) == false; - // ToDo: check if these needs to be updated or not - final boolean updateDiscoveryNodes = clusterState.getNodes().delta(previousClusterState.getNodes()).hasChanges(); - final boolean updateClusterBlocks = !clusterState.blocks().equals(previousClusterState.blocks()); - final boolean updateHashesOfConsistentSettings = firstUploadForSplitGlobalMetadata + + final boolean updateDiscoveryNodes = isPublicationEnabled + && clusterState.getNodes().delta(previousClusterState.getNodes()).hasChanges(); + final boolean updateClusterBlocks = isPublicationEnabled && !clusterState.blocks().equals(previousClusterState.blocks()); + final boolean updateHashesOfConsistentSettings = isPublicationEnabled || Metadata.isHashesOfConsistentSettingsEqual(previousClusterState.metadata(), clusterState.metadata()) == false; uploadedMetadataResults = writeMetadataInParallel( clusterState, toUpload, prevIndexMetadataByName, - firstUploadForSplitGlobalMetadata ? clusterState.metadata().customs() : customsToUpload, + customsDiff.getUpserts(), updateCoordinationMetadata, updateSettingsMetadata, updateTemplatesMetadata, updateDiscoveryNodes, updateClusterBlocks, updateTransientSettingsMetadata, - clusterStateCustomsToUpload, + clusterStateCustomsDiff.getUpserts(), updateHashesOfConsistentSettings, indicesRoutingToUpload ); @@ -382,10 +379,11 @@ public RemoteClusterStateManifestInfo writeIncrementalMetadata( uploadedIndexMetadata -> allUploadedIndexMetadata.put(uploadedIndexMetadata.getIndexName(), uploadedIndexMetadata) ); allUploadedCustomMap.putAll(uploadedMetadataResults.uploadedCustomMetadataMap); + allUploadedClusterStateCustomsMap.putAll(uploadedMetadataResults.uploadedClusterStateCustomMetadataMap); // remove the data for removed custom/indices - customsToBeDeletedFromRemote.keySet().forEach(allUploadedCustomMap::remove); + customsDiff.getDeletes().forEach(allUploadedCustomMap::remove); indicesToBeDeletedFromRemote.keySet().forEach(allUploadedIndexMetadata::remove); - clusterStateCustomsToBeDeleted.keySet().forEach(allUploadedCustomMap::remove); + clusterStateCustomsDiff.getDeletes().forEach(allUploadedClusterStateCustomsMap::remove); if (!updateCoordinationMetadata) { uploadedMetadataResults.uploadedCoordinationMetadata = previousManifest.getCoordinationMetadata(); @@ -399,31 +397,24 @@ public RemoteClusterStateManifestInfo writeIncrementalMetadata( if (!updateTemplatesMetadata) { uploadedMetadataResults.uploadedTemplatesMetadata = previousManifest.getTemplatesMetadata(); } - if (!updateDiscoveryNodes && !firstUploadForSplitGlobalMetadata) { + if (!updateDiscoveryNodes) { uploadedMetadataResults.uploadedDiscoveryNodes = previousManifest.getDiscoveryNodesMetadata(); } - if (!updateClusterBlocks && !firstUploadForSplitGlobalMetadata) { + if (!updateClusterBlocks) { uploadedMetadataResults.uploadedClusterBlocks = previousManifest.getClusterBlocksMetadata(); } - if (!updateHashesOfConsistentSettings && !firstUploadForSplitGlobalMetadata) { + if (!updateHashesOfConsistentSettings) { uploadedMetadataResults.uploadedHashesOfConsistentSettings = previousManifest.getHashesOfConsistentSettings(); } - if (!firstUploadForSplitGlobalMetadata && customsToUpload.isEmpty()) { - uploadedMetadataResults.uploadedCustomMetadataMap = previousManifest.getCustomMetadataMap(); - } - if (!firstUploadForSplitGlobalMetadata && clusterStateCustomsToUpload.isEmpty()) { - uploadedMetadataResults.uploadedClusterStateCustomMetadataMap = previousManifest.getClusterStateCustomMap(); - } uploadedMetadataResults.uploadedCustomMetadataMap = allUploadedCustomMap; + uploadedMetadataResults.uploadedClusterStateCustomMetadataMap = allUploadedClusterStateCustomsMap; uploadedMetadataResults.uploadedIndexMetadata = new ArrayList<>(allUploadedIndexMetadata.values()); - List allUploadedIndicesRouting = new ArrayList<>(); - allUploadedIndicesRouting = remoteRoutingTableService.getAllUploadedIndicesRouting( + uploadedMetadataResults.uploadedIndicesRoutingMetadata = remoteRoutingTableService.getAllUploadedIndicesRouting( previousManifest, uploadedMetadataResults.uploadedIndicesRoutingMetadata, routingTableDiff.getDeletes() ); - uploadedMetadataResults.uploadedIndicesRoutingMetadata = allUploadedIndicesRouting; final RemoteClusterStateManifestInfo manifestDetails = remoteManifestManager.uploadManifest( clusterState, @@ -448,7 +439,7 @@ public RemoteClusterStateManifestInfo writeIncrementalMetadata( updateCoordinationMetadata, updateSettingsMetadata, updateTemplatesMetadata, - customsToUpload.size(), + customsDiff.getUpserts().size(), indicesRoutingToUpload.size() ); if (durationMillis >= slowWriteLoggingThreshold.getMillis()) { @@ -464,7 +455,7 @@ public RemoteClusterStateManifestInfo writeIncrementalMetadata( updateCoordinationMetadata, updateSettingsMetadata, updateTemplatesMetadata, - customsToUpload.size() + customsDiff.getUpserts().size() ); } else { logger.info("{}; {}", clusterStateUploadTimeMessage, metadataUpdateMessage); @@ -479,7 +470,7 @@ public RemoteClusterStateManifestInfo writeIncrementalMetadata( updateCoordinationMetadata, updateSettingsMetadata, updateTemplatesMetadata, - customsToUpload.size() + customsDiff.getUpserts().size() ); } return manifestDetails; diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManager.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManager.java index 3053095368972..2c5aad99adc0c 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManager.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManager.java @@ -10,9 +10,12 @@ import org.opensearch.action.LatchedActionListener; import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.DiffableUtils; +import org.opensearch.cluster.DiffableUtils.NonDiffableValueSerializer; import org.opensearch.cluster.coordination.CoordinationMetadata; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.metadata.Metadata.Custom; +import org.opensearch.cluster.metadata.Metadata.XContentContext; import org.opensearch.cluster.metadata.TemplatesMetadata; import org.opensearch.common.CheckedRunnable; import org.opensearch.common.remote.AbstractRemoteWritableBlobEntity; @@ -39,11 +42,12 @@ import org.opensearch.threadpool.ThreadPool; import java.io.IOException; +import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.Locale; import java.util.Map; -import java.util.Set; +import java.util.Map.Entry; +import java.util.stream.Collectors; import static org.opensearch.gateway.remote.RemoteClusterStateUtils.METADATA_NAME_FORMAT; @@ -276,29 +280,37 @@ Metadata getGlobalMetadata(String clusterUUID, ClusterMetadataManifest clusterMe } } - Map getUpdatedCustoms(ClusterState currentState, ClusterState previousState) { - if (Metadata.isCustomMetadataEqual(previousState.metadata(), currentState.metadata())) { - return new HashMap<>(); - } - Map updatedCustom = new HashMap<>(); - Set currentCustoms = new HashSet<>(currentState.metadata().customs().keySet()); - for (Map.Entry cursor : previousState.metadata().customs().entrySet()) { - if (cursor.getValue().context().contains(Metadata.XContentContext.GATEWAY)) { - if (currentCustoms.contains(cursor.getKey()) - && !cursor.getValue().equals(currentState.metadata().custom(cursor.getKey()))) { - // If the custom metadata is updated, we need to upload the new version. - updatedCustom.put(cursor.getKey(), currentState.metadata().custom(cursor.getKey())); - } - currentCustoms.remove(cursor.getKey()); - } + DiffableUtils.MapDiff> getCustomsDiff( + ClusterState currentState, + ClusterState previousState, + boolean firstUploadForSplitGlobalMetadata, + boolean isRemotePublicationEnabled + ) { + if (firstUploadForSplitGlobalMetadata) { + // For first split global metadata upload, we want to upload all customs + return DiffableUtils.diff( + Collections.emptyMap(), + filterCustoms(currentState.metadata().customs(), isRemotePublicationEnabled), + DiffableUtils.getStringKeySerializer(), + NonDiffableValueSerializer.getAbstractInstance() + ); } - for (String custom : currentCustoms) { - Metadata.Custom cursor = currentState.metadata().custom(custom); - if (cursor.context().contains(Metadata.XContentContext.GATEWAY)) { - updatedCustom.put(custom, cursor); - } + return DiffableUtils.diff( + filterCustoms(previousState.metadata().customs(), isRemotePublicationEnabled), + filterCustoms(currentState.metadata().customs(), isRemotePublicationEnabled), + DiffableUtils.getStringKeySerializer(), + NonDiffableValueSerializer.getAbstractInstance() + ); + } + + public static Map filterCustoms(Map customs, boolean isRemotePublicationEnabled) { + if (isRemotePublicationEnabled) { + return customs; } - return updatedCustom; + return customs.entrySet() + .stream() + .filter(e -> e.getValue().context().contains(XContentContext.GATEWAY)) + .collect(Collectors.toMap(Entry::getKey, Entry::getValue)); } boolean isGlobalMetadataEqual(ClusterMetadataManifest first, ClusterMetadataManifest second, String clusterName) { diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterStateBlobStore.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterStateBlobStore.java index 83326f65f0d43..1dd23443f1252 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterStateBlobStore.java +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterStateBlobStore.java @@ -72,7 +72,9 @@ public void writeAsync(final U entity, final ActionListener listener) { public T read(final U entity) throws IOException { // TODO Add timing logs and tracing assert entity.getFullBlobName() != null; - return entity.deserialize(transferService.downloadBlob(getBlobPathForDownload(entity), entity.getBlobFileName())); + try (InputStream inputStream = transferService.downloadBlob(getBlobPathForDownload(entity), entity.getBlobFileName())) { + return entity.deserialize(inputStream); + } } @Override diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManagerTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManagerTests.java index 0aff1c4b0e5e2..41e1546ead164 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManagerTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManagerTests.java @@ -8,7 +8,13 @@ package org.opensearch.gateway.remote; +import org.opensearch.Version; import org.opensearch.action.LatchedActionListener; +import org.opensearch.cluster.AbstractNamedDiffable; +import org.opensearch.cluster.ClusterName; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.ClusterState.Custom; +import org.opensearch.cluster.DiffableUtils; import org.opensearch.cluster.block.ClusterBlocks; import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.common.CheckedRunnable; @@ -16,8 +22,11 @@ import org.opensearch.common.settings.Settings; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.compress.Compressor; import org.opensearch.core.compress.NoneCompressor; +import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.gateway.remote.model.RemoteClusterBlocks; import org.opensearch.gateway.remote.model.RemoteDiscoveryNodes; import org.opensearch.gateway.remote.model.RemoteReadResult; @@ -31,6 +40,9 @@ import org.junit.Before; import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicReference; @@ -41,6 +53,7 @@ import static org.opensearch.gateway.remote.model.RemoteClusterBlocksTests.randomClusterBlocks; import static org.opensearch.gateway.remote.model.RemoteDiscoveryNodes.DISCOVERY_NODES_FORMAT; import static org.opensearch.gateway.remote.model.RemoteDiscoveryNodesTests.getDiscoveryNodes; +import static org.hamcrest.Matchers.is; import static org.mockito.ArgumentMatchers.anyIterable; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.mock; @@ -51,7 +64,7 @@ public class RemoteClusterStateAttributesManagerTests extends OpenSearchTestCase private BlobStoreTransferService blobStoreTransferService; private BlobStoreRepository blobStoreRepository; private Compressor compressor; - private ThreadPool threadpool = new TestThreadPool(RemoteClusterStateAttributesManagerTests.class.getName()); + private ThreadPool threadPool = new TestThreadPool(RemoteClusterStateAttributesManagerTests.class.getName()); @Before public void setup() throws Exception { @@ -65,15 +78,15 @@ public void setup() throws Exception { "test-cluster", blobStoreRepository, blobStoreTransferService, - namedWriteableRegistry, - threadpool + writableRegistry(), + threadPool ); } @After public void tearDown() throws Exception { super.tearDown(); - threadpool.shutdown(); + threadPool.shutdown(); } public void testGetAsyncMetadataReadAction_DiscoveryNodes() throws IOException { @@ -138,4 +151,169 @@ public void testGetAsyncMetadataReadAction_ClusterBlocks() throws IOException { throw new RuntimeException(e); } } + + public void testGetUpdatedCustoms() { + Map previousCustoms = Map.of( + TestCustom1.TYPE, + new TestCustom1("data1"), + TestCustom2.TYPE, + new TestCustom2("data2"), + TestCustom3.TYPE, + new TestCustom3("data3") + ); + ClusterState previousState = ClusterState.builder(new ClusterName("test-cluster")).customs(previousCustoms).build(); + + Map currentCustoms = Map.of( + TestCustom2.TYPE, + new TestCustom2("data2"), + TestCustom3.TYPE, + new TestCustom3("data3-changed"), + TestCustom4.TYPE, + new TestCustom4("data4") + ); + + ClusterState currentState = ClusterState.builder(new ClusterName("test-cluster")).customs(currentCustoms).build(); + + DiffableUtils.MapDiff> customsDiff = + remoteClusterStateAttributesManager.getUpdatedCustoms(currentState, previousState, false, randomBoolean()); + assertThat(customsDiff.getUpserts(), is(Collections.emptyMap())); + assertThat(customsDiff.getDeletes(), is(Collections.emptyList())); + + customsDiff = remoteClusterStateAttributesManager.getUpdatedCustoms(currentState, previousState, true, true); + assertThat(customsDiff.getUpserts(), is(currentCustoms)); + assertThat(customsDiff.getDeletes(), is(Collections.emptyList())); + + Map expectedCustoms = Map.of( + TestCustom3.TYPE, + new TestCustom3("data3-changed"), + TestCustom4.TYPE, + new TestCustom4("data4") + ); + + customsDiff = remoteClusterStateAttributesManager.getUpdatedCustoms(currentState, previousState, true, false); + assertThat(customsDiff.getUpserts(), is(expectedCustoms)); + assertThat(customsDiff.getDeletes(), is(List.of(TestCustom1.TYPE))); + } + + private static abstract class AbstractTestCustom extends AbstractNamedDiffable implements ClusterState.Custom { + + private final String value; + + AbstractTestCustom(String value) { + this.value = value; + } + + AbstractTestCustom(StreamInput in) throws IOException { + this.value = in.readString(); + } + + @Override + public Version getMinimalSupportedVersion() { + return Version.CURRENT; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(value); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return builder; + } + + @Override + public boolean isPrivate() { + return true; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + AbstractTestCustom that = (AbstractTestCustom) o; + + if (!value.equals(that.value)) return false; + + return true; + } + + @Override + public int hashCode() { + return value.hashCode(); + } + } + + private static class TestCustom1 extends AbstractTestCustom { + + private static final String TYPE = "custom_1"; + + TestCustom1(String value) { + super(value); + } + + TestCustom1(StreamInput in) throws IOException { + super(in); + } + + @Override + public String getWriteableName() { + return TYPE; + } + } + + private static class TestCustom2 extends AbstractTestCustom { + + private static final String TYPE = "custom_2"; + + TestCustom2(String value) { + super(value); + } + + TestCustom2(StreamInput in) throws IOException { + super(in); + } + + @Override + public String getWriteableName() { + return TYPE; + } + } + + private static class TestCustom3 extends AbstractTestCustom { + + private static final String TYPE = "custom_3"; + + TestCustom3(String value) { + super(value); + } + + TestCustom3(StreamInput in) throws IOException { + super(in); + } + + @Override + public String getWriteableName() { + return TYPE; + } + } + + private static class TestCustom4 extends AbstractTestCustom { + + private static final String TYPE = "custom_4"; + + TestCustom4(String value) { + super(value); + } + + TestCustom4(StreamInput in) throws IOException { + super(in); + } + + @Override + public String getWriteableName() { + return TYPE; + } + } } diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java index feae97bae48e9..c8fd982fec1e1 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java @@ -12,6 +12,8 @@ import org.opensearch.cluster.ClusterModule; import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.RepositoryCleanupInProgress; +import org.opensearch.cluster.RepositoryCleanupInProgress.Entry; import org.opensearch.cluster.coordination.CoordinationMetadata; import org.opensearch.cluster.metadata.IndexGraveyard; import org.opensearch.cluster.metadata.IndexMetadata; @@ -74,6 +76,7 @@ import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Locale; import java.util.Map; @@ -130,6 +133,7 @@ public class RemoteClusterStateServiceTests extends OpenSearchTestCase { private BlobStoreRepository blobStoreRepository; private BlobStore blobStore; private Settings settings; + private boolean publicationEnabled; private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); @Before @@ -154,6 +158,7 @@ public void setup() { .put(stateRepoTypeAttributeKey, FsRepository.TYPE) .put(stateRepoSettingsAttributeKeyPrefix + "location", "randomRepoPath") .put(RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true) + .put("node.attr." + REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY, "routing_repository") .build(); clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); @@ -190,6 +195,9 @@ public void setup() { public void teardown() throws Exception { super.tearDown(); remoteClusterStateService.close(); + publicationEnabled = false; + Settings nodeSettings = Settings.builder().build(); + FeatureFlags.initializeFeatureFlags(nodeSettings); threadPool.shutdown(); } @@ -263,9 +271,67 @@ public void testWriteFullMetadataSuccess() throws IOException { assertThat(manifest.getSettingsMetadata(), notNullValue()); assertThat(manifest.getTemplatesMetadata(), notNullValue()); assertFalse(manifest.getCustomMetadataMap().isEmpty()); + assertThat(manifest.getClusterBlocksMetadata(), nullValue()); + assertThat(manifest.getDiscoveryNodesMetadata(), nullValue()); + assertThat(manifest.getTransientSettingsMetadata(), nullValue()); + assertThat(manifest.getHashesOfConsistentSettings(), nullValue()); + assertThat(manifest.getClusterStateCustomMap().size(), is(0)); + } + + public void testWriteFullMetadataSuccessPublicationEnabled() throws IOException { + // TODO Make the publication flag parameterized + publicationEnabled = true; + Settings nodeSettings = Settings.builder().put(REMOTE_PUBLICATION_EXPERIMENTAL, publicationEnabled).build(); + FeatureFlags.initializeFeatureFlags(nodeSettings); + remoteClusterStateService = new RemoteClusterStateService( + "test-node-id", + repositoriesServiceSupplier, + settings, + clusterService, + () -> 0L, + threadPool, + List.of(new RemoteIndexPathUploader(threadPool, settings, repositoriesServiceSupplier, clusterSettings)), + writableRegistry() + ); + final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()) + .customs(Map.of(RepositoryCleanupInProgress.TYPE, new RepositoryCleanupInProgress(List.of(new Entry("test-repo", 10L))))) + .build(); + mockBlobStoreObjects(); + remoteClusterStateService.start(); + final ClusterMetadataManifest manifest = remoteClusterStateService.writeFullMetadata(clusterState, "prev-cluster-uuid") + .getClusterMetadataManifest(); + final UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "index-uuid", "metadata-filename"); + List indices = List.of(uploadedIndexMetadata); + + final ClusterMetadataManifest expectedManifest = ClusterMetadataManifest.builder() + .indices(indices) + .clusterTerm(1L) + .stateVersion(1L) + .stateUUID("state-uuid") + .clusterUUID("cluster-uuid") + .previousClusterUUID("prev-cluster-uuid") + .build(); + + assertThat(manifest.getIndices().size(), is(1)); + assertThat(manifest.getIndices().get(0).getIndexName(), is(uploadedIndexMetadata.getIndexName())); + assertThat(manifest.getIndices().get(0).getIndexUUID(), is(uploadedIndexMetadata.getIndexUUID())); + assertThat(manifest.getIndices().get(0).getUploadedFilename(), notNullValue()); + assertThat(manifest.getClusterTerm(), is(expectedManifest.getClusterTerm())); + assertThat(manifest.getStateVersion(), is(expectedManifest.getStateVersion())); + assertThat(manifest.getClusterUUID(), is(expectedManifest.getClusterUUID())); + assertThat(manifest.getStateUUID(), is(expectedManifest.getStateUUID())); + assertThat(manifest.getPreviousClusterUUID(), is(expectedManifest.getPreviousClusterUUID())); + assertThat(manifest.getGlobalMetadataFileName(), nullValue()); + assertThat(manifest.getCoordinationMetadata(), notNullValue()); + assertThat(manifest.getSettingsMetadata(), notNullValue()); + assertThat(manifest.getTemplatesMetadata(), notNullValue()); + assertFalse(manifest.getCustomMetadataMap().isEmpty()); + assertThat(manifest.getClusterStateCustomMap().size(), is(1)); + assertThat(manifest.getClusterStateCustomMap().containsKey(RepositoryCleanupInProgress.TYPE), is(true)); } public void testWriteFullMetadataInParallelSuccess() throws IOException { + // TODO Add test with publication flag enabled final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); AsyncMultiStreamBlobContainer container = (AsyncMultiStreamBlobContainer) mockBlobStoreObjects(AsyncMultiStreamBlobContainer.class); @@ -310,8 +376,8 @@ public void testWriteFullMetadataInParallelSuccess() throws IOException { assertThat(manifest.getStateUUID(), is(expectedManifest.getStateUUID())); assertThat(manifest.getPreviousClusterUUID(), is(expectedManifest.getPreviousClusterUUID())); - assertEquals(11, actionListenerArgumentCaptor.getAllValues().size()); - assertEquals(11, writeContextArgumentCaptor.getAllValues().size()); + assertEquals(7, actionListenerArgumentCaptor.getAllValues().size()); + assertEquals(7, writeContextArgumentCaptor.getAllValues().size()); byte[] writtenBytes = capturedWriteContext.get("metadata") .getStreamProvider(Integer.MAX_VALUE) @@ -584,7 +650,7 @@ private void verifyWriteIncrementalGlobalMetadataFromOlderCodecSuccess(ClusterMe assertNotNull(manifest.getCoordinationMetadata()); assertNotNull(manifest.getSettingsMetadata()); assertNotNull(manifest.getTemplatesMetadata()); - assertNotEquals(0, manifest.getCustomMetadataMap().size()); + assertNotNull(manifest.getCustomMetadataMap()); assertEquals(expectedManifest.getClusterTerm(), manifest.getClusterTerm()); assertEquals(expectedManifest.getStateVersion(), manifest.getStateVersion()); @@ -769,6 +835,7 @@ public void testCustomMetadataDeletedUpdatedAndAdded() throws IOException { .putCustom("custom1", new CustomMetadata1("mock_custom_metadata1")) .putCustom("custom2", new CustomMetadata1("mock_custom_metadata2")) .putCustom("custom3", new CustomMetadata1("mock_custom_metadata3")) + .version(initialClusterState.metadata().version() + 1) ) .build(); @@ -784,6 +851,7 @@ public void testCustomMetadataDeletedUpdatedAndAdded() throws IOException { .putCustom("custom2", new CustomMetadata1("mock_updated_custom_metadata")) .putCustom("custom3", new CustomMetadata1("mock_custom_metadata3")) .putCustom("custom4", new CustomMetadata1("mock_custom_metadata4")) + .version(clusterState1.metadata().version() + 1) ) .build(); ClusterMetadataManifest manifest2 = remoteClusterStateService.writeIncrementalMetadata(clusterState1, clusterState2, manifest1) @@ -1313,7 +1381,11 @@ public void testRemoteStateStats() throws IOException { } public void testRemoteRoutingTableNotInitializedWhenDisabled() { - assertTrue(remoteClusterStateService.getRemoteRoutingTableService() instanceof NoopRemoteRoutingTableService); + if (publicationEnabled) { + assertTrue(remoteClusterStateService.getRemoteRoutingTableService() instanceof InternalRemoteRoutingTableService); + } else { + assertTrue(remoteClusterStateService.getRemoteRoutingTableService() instanceof NoopRemoteRoutingTableService); + } } public void testRemoteRoutingTableInitializedWhenEnabled() { @@ -1737,6 +1809,17 @@ private BlobContainer mockBlobStoreObjects(Class blobCo final BlobPath blobPath = mock(BlobPath.class); when((blobStoreRepository.basePath())).thenReturn(blobPath); when(blobPath.add(anyString())).thenReturn(blobPath); + when(blobPath.iterator()).thenReturn(new Iterator() { + @Override + public boolean hasNext() { + return false; + } + + @Override + public String next() { + return null; + } + }); when(blobPath.buildAsString()).thenReturn("/blob/path/"); final BlobContainer blobContainer = mock(blobContainerClazz); when(blobContainer.path()).thenReturn(blobPath); diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManagerTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManagerTests.java index f24f8ddeb1959..bd01bc1ab0cdb 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManagerTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManagerTests.java @@ -8,7 +8,14 @@ package org.opensearch.gateway.remote; +import org.opensearch.Version; import org.opensearch.cluster.ClusterModule; +import org.opensearch.cluster.ClusterName; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.DiffableUtils; +import org.opensearch.cluster.metadata.IndexGraveyard; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.metadata.Metadata.XContentContext; import org.opensearch.common.network.NetworkModule; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; @@ -19,15 +26,20 @@ import org.opensearch.indices.IndicesModule; import org.opensearch.repositories.blobstore.BlobStoreRepository; import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.test.TestCustomMetadata; import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; import org.junit.After; import org.junit.Before; +import java.util.EnumSet; +import java.util.List; +import java.util.Map; import java.util.function.Function; import java.util.stream.Stream; import static java.util.stream.Collectors.toList; +import static org.hamcrest.Matchers.is; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -83,4 +95,201 @@ public void testGlobalMetadataUploadWaitTimeSetting() { clusterSettings.applySettings(newSettings); assertEquals(globalMetadataUploadTimeout, remoteGlobalMetadataManager.getGlobalMetadataUploadTimeout().seconds()); } + + public void testGetUpdatedCustoms() { + Map previousCustoms = Map.of( + CustomMetadata1.TYPE, + new CustomMetadata1("data1"), + CustomMetadata2.TYPE, + new CustomMetadata2("data2"), + CustomMetadata3.TYPE, + new CustomMetadata3("data3") + ); + ClusterState previousState = ClusterState.builder(new ClusterName("test-cluster")) + .metadata(Metadata.builder().customs(previousCustoms)) + .build(); + + Map currentCustoms = Map.of( + CustomMetadata2.TYPE, + new CustomMetadata2("data2"), + CustomMetadata3.TYPE, + new CustomMetadata3("data3-changed"), + CustomMetadata4.TYPE, + new CustomMetadata4("data4"), + CustomMetadata5.TYPE, + new CustomMetadata5("data5") + ); + ClusterState currentState = ClusterState.builder(new ClusterName("test-cluster")) + .metadata(Metadata.builder().customs(currentCustoms)) + .build(); + + DiffableUtils.MapDiff> customsDiff = remoteGlobalMetadataManager + .getCustomsDiff(currentState, previousState, true, false); + Map expectedUpserts = Map.of( + CustomMetadata2.TYPE, + new CustomMetadata2("data2"), + CustomMetadata3.TYPE, + new CustomMetadata3("data3-changed"), + CustomMetadata4.TYPE, + new CustomMetadata4("data4"), + IndexGraveyard.TYPE, + IndexGraveyard.builder().build() + ); + assertThat(customsDiff.getUpserts(), is(expectedUpserts)); + assertThat(customsDiff.getDeletes(), is(List.of())); + + customsDiff = remoteGlobalMetadataManager.getCustomsDiff(currentState, previousState, false, false); + expectedUpserts = Map.of( + CustomMetadata3.TYPE, + new CustomMetadata3("data3-changed"), + CustomMetadata4.TYPE, + new CustomMetadata4("data4") + ); + assertThat(customsDiff.getUpserts(), is(expectedUpserts)); + assertThat(customsDiff.getDeletes(), is(List.of(CustomMetadata1.TYPE))); + + customsDiff = remoteGlobalMetadataManager.getCustomsDiff(currentState, previousState, true, true); + expectedUpserts = Map.of( + CustomMetadata2.TYPE, + new CustomMetadata2("data2"), + CustomMetadata3.TYPE, + new CustomMetadata3("data3-changed"), + CustomMetadata4.TYPE, + new CustomMetadata4("data4"), + CustomMetadata5.TYPE, + new CustomMetadata5("data5"), + IndexGraveyard.TYPE, + IndexGraveyard.builder().build() + ); + assertThat(customsDiff.getUpserts(), is(expectedUpserts)); + assertThat(customsDiff.getDeletes(), is(List.of())); + + customsDiff = remoteGlobalMetadataManager.getCustomsDiff(currentState, previousState, false, true); + expectedUpserts = Map.of( + CustomMetadata3.TYPE, + new CustomMetadata3("data3-changed"), + CustomMetadata4.TYPE, + new CustomMetadata4("data4"), + CustomMetadata5.TYPE, + new CustomMetadata5("data5") + ); + assertThat(customsDiff.getUpserts(), is(expectedUpserts)); + assertThat(customsDiff.getDeletes(), is(List.of(CustomMetadata1.TYPE))); + + } + + private static class CustomMetadata1 extends TestCustomMetadata { + public static final String TYPE = "custom_md_1"; + + CustomMetadata1(String data) { + super(data); + } + + @Override + public String getWriteableName() { + return TYPE; + } + + @Override + public Version getMinimalSupportedVersion() { + return Version.CURRENT; + } + + @Override + public EnumSet context() { + return EnumSet.of(Metadata.XContentContext.GATEWAY); + } + } + + private static class CustomMetadata2 extends TestCustomMetadata { + public static final String TYPE = "custom_md_2"; + + CustomMetadata2(String data) { + super(data); + } + + @Override + public String getWriteableName() { + return TYPE; + } + + @Override + public Version getMinimalSupportedVersion() { + return Version.CURRENT; + } + + @Override + public EnumSet context() { + return EnumSet.of(Metadata.XContentContext.GATEWAY); + } + } + + private static class CustomMetadata3 extends TestCustomMetadata { + public static final String TYPE = "custom_md_3"; + + CustomMetadata3(String data) { + super(data); + } + + @Override + public String getWriteableName() { + return TYPE; + } + + @Override + public Version getMinimalSupportedVersion() { + return Version.CURRENT; + } + + @Override + public EnumSet context() { + return EnumSet.of(Metadata.XContentContext.GATEWAY); + } + } + + private static class CustomMetadata4 extends TestCustomMetadata { + public static final String TYPE = "custom_md_4"; + + CustomMetadata4(String data) { + super(data); + } + + @Override + public String getWriteableName() { + return TYPE; + } + + @Override + public Version getMinimalSupportedVersion() { + return Version.CURRENT; + } + + @Override + public EnumSet context() { + return EnumSet.of(Metadata.XContentContext.GATEWAY); + } + } + + private static class CustomMetadata5 extends TestCustomMetadata { + public static final String TYPE = "custom_md_5"; + + CustomMetadata5(String data) { + super(data); + } + + @Override + public String getWriteableName() { + return TYPE; + } + + @Override + public Version getMinimalSupportedVersion() { + return Version.CURRENT; + } + + @Override + public EnumSet context() { + return EnumSet.of(XContentContext.API); + } + } } From 47425a11fc7c3ab3e72679c0e22033ea86cbb7bf Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 17 Jun 2024 10:18:58 -0400 Subject: [PATCH 19/84] Bump commons-net:commons-net from 3.10.0 to 3.11.1 in /test/fixtures/hdfs-fixture (#14396) * Bump commons-net:commons-net in /test/fixtures/hdfs-fixture Bumps commons-net:commons-net from 3.10.0 to 3.11.1. --- updated-dependencies: - dependency-name: commons-net:commons-net dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] * Update changelog Signed-off-by: dependabot[bot] --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] --- CHANGELOG.md | 1 + test/fixtures/hdfs-fixture/build.gradle | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 4e1d41ec6b152..fa387e19c3d77 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -12,6 +12,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Update to Apache Lucene 9.11.0 ([#14042](https://github.com/opensearch-project/OpenSearch/pull/14042)) - Bump `netty` from 4.1.110.Final to 4.1.111.Final ([#14356](https://github.com/opensearch-project/OpenSearch/pull/14356)) - Bump `org.wiremock:wiremock-standalone` from 3.3.1 to 3.6.0 ([#14361](https://github.com/opensearch-project/OpenSearch/pull/14361)) +- Bump `commons-net:commons-net` from 3.10.0 to 3.11.1 ([#14396](https://github.com/opensearch-project/OpenSearch/pull/14396)) ### Changed diff --git a/test/fixtures/hdfs-fixture/build.gradle b/test/fixtures/hdfs-fixture/build.gradle index ddb876b46fd1c..a532bf0c6287b 100644 --- a/test/fixtures/hdfs-fixture/build.gradle +++ b/test/fixtures/hdfs-fixture/build.gradle @@ -70,7 +70,7 @@ dependencies { api "org.eclipse.jetty.websocket:javax-websocket-server-impl:${versions.jetty}" api 'org.apache.zookeeper:zookeeper:3.9.2' api "org.apache.commons:commons-text:1.12.0" - api "commons-net:commons-net:3.10.0" + api "commons-net:commons-net:3.11.1" api "ch.qos.logback:logback-core:1.5.6" api "ch.qos.logback:logback-classic:1.2.13" api 'org.apache.kerby:kerb-admin:2.0.3' From 5ec2abeb8200bf60f1f706e66f4364f03e756573 Mon Sep 17 00:00:00 2001 From: gaobinlong Date: Mon, 17 Jun 2024 22:28:03 +0800 Subject: [PATCH 20/84] Fix flaky tests in org.opensearch.cluster.routing.remote.RemoteRoutingTableServiceTests (#14264) Signed-off-by: Gao Binlong --- .../remote/RemoteRoutingTableServiceTests.java | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/server/src/test/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableServiceTests.java b/server/src/test/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableServiceTests.java index cc31c1a6e8fd1..839ebe1ff8301 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableServiceTests.java @@ -227,11 +227,11 @@ public void testGetIndicesRoutingMapDiffIndexAdded() { .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetadata.SETTING_INDEX_UUID, "uuid") .build() - ).numberOfShards(randomInt(1000)).numberOfReplicas(randomInt(10)).build(); + ).numberOfShards(between(1, 1000)).numberOfReplicas(randomInt(10)).build(); RoutingTable routingTable = RoutingTable.builder().addAsNew(indexMetadata).build(); String indexName2 = randomAlphaOfLength(randomIntBetween(1, 50)); - int noOfShards = randomInt(1000); + int noOfShards = between(1, 1000); int noOfReplicas = randomInt(10); final IndexMetadata indexMetadata2 = new IndexMetadata.Builder(indexName2).settings( Settings.builder() @@ -252,8 +252,7 @@ public void testGetIndicesRoutingMapDiffIndexAdded() { public void testGetIndicesRoutingMapDiffShardChanged() { String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); - final Index index = new Index(indexName, "uuid"); - int noOfShards = randomInt(1000); + int noOfShards = between(1, 1000); int noOfReplicas = randomInt(10); final IndexMetadata indexMetadata = new IndexMetadata.Builder(indexName).settings( Settings.builder() @@ -299,8 +298,7 @@ public void testGetIndicesRoutingMapDiffShardChanged() { public void testGetIndicesRoutingMapDiffShardDetailChanged() { String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); - final Index index = new Index(indexName, "uuid"); - int noOfShards = randomInt(1000); + int noOfShards = between(1, 1000); int noOfReplicas = randomInt(10); final IndexMetadata indexMetadata = new IndexMetadata.Builder(indexName).settings( Settings.builder() @@ -328,7 +326,7 @@ public void testGetIndicesRoutingMapDiffIndexDeleted() { .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetadata.SETTING_INDEX_UUID, "uuid") .build() - ).numberOfShards(randomInt(1000)).numberOfReplicas(randomInt(10)).build(); + ).numberOfShards(between(1, 1000)).numberOfReplicas(randomInt(10)).build(); RoutingTable routingTable = RoutingTable.builder().addAsNew(indexMetadata).build(); String indexName2 = randomAlphaOfLength(randomIntBetween(1, 50)); @@ -337,7 +335,7 @@ public void testGetIndicesRoutingMapDiffIndexDeleted() { .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetadata.SETTING_INDEX_UUID, "uuid2") .build() - ).numberOfShards(randomInt(1000)).numberOfReplicas(randomInt(10)).build(); + ).numberOfShards(between(1, 1000)).numberOfReplicas(randomInt(10)).build(); RoutingTable routingTable2 = RoutingTable.builder().addAsNew(indexMetadata2).build(); DiffableUtils.MapDiff> diff = remoteRoutingTableService @@ -732,7 +730,6 @@ private ClusterState createIndices(int numberOfIndices) { RoutingTable.Builder routingTableBuilder = RoutingTable.builder(); for (int i = 0; i < numberOfIndices; i++) { String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); - final Index index = new Index(indexName, "uuid"); final IndexMetadata indexMetadata = new IndexMetadata.Builder(indexName).settings( Settings.builder() .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) @@ -751,7 +748,7 @@ private ClusterState createClusterState(String indexName) { .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetadata.SETTING_INDEX_UUID, "uuid") .build() - ).numberOfShards(randomInt(1000)).numberOfReplicas(randomInt(10)).build(); + ).numberOfShards(between(1, 1000)).numberOfReplicas(randomInt(10)).build(); RoutingTable routingTable = RoutingTable.builder().addAsNew(indexMetadata).build(); return ClusterState.builder(ClusterName.DEFAULT) .routingTable(routingTable) From f99a6f8f682a79d052b68eb6610b631dd9ce0e41 Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Mon, 17 Jun 2024 11:21:36 -0400 Subject: [PATCH 21/84] Bump reactor from 3.5.17 to 3.5.18 and reactor-netty from 1.1.19 to 1.1.20 (#14395) Signed-off-by: Andriy Redko --- CHANGELOG.md | 2 ++ buildSrc/version.properties | 4 ++-- .../licenses/reactor-netty-core-1.1.19.jar.sha1 | 1 - .../licenses/reactor-netty-core-1.1.20.jar.sha1 | 1 + .../licenses/reactor-netty-http-1.1.19.jar.sha1 | 1 - .../licenses/reactor-netty-http-1.1.20.jar.sha1 | 1 + .../licenses/reactor-netty-core-1.1.19.jar.sha1 | 1 - .../licenses/reactor-netty-core-1.1.20.jar.sha1 | 1 + .../licenses/reactor-netty-http-1.1.19.jar.sha1 | 1 - .../licenses/reactor-netty-http-1.1.20.jar.sha1 | 1 + server/licenses/reactor-core-3.5.17.jar.sha1 | 1 - server/licenses/reactor-core-3.5.18.jar.sha1 | 1 + 12 files changed, 9 insertions(+), 7 deletions(-) delete mode 100644 plugins/repository-azure/licenses/reactor-netty-core-1.1.19.jar.sha1 create mode 100644 plugins/repository-azure/licenses/reactor-netty-core-1.1.20.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/reactor-netty-http-1.1.19.jar.sha1 create mode 100644 plugins/repository-azure/licenses/reactor-netty-http-1.1.20.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/reactor-netty-core-1.1.19.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/reactor-netty-core-1.1.20.jar.sha1 delete mode 100644 plugins/transport-reactor-netty4/licenses/reactor-netty-http-1.1.19.jar.sha1 create mode 100644 plugins/transport-reactor-netty4/licenses/reactor-netty-http-1.1.20.jar.sha1 delete mode 100644 server/licenses/reactor-core-3.5.17.jar.sha1 create mode 100644 server/licenses/reactor-core-3.5.18.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index fa387e19c3d77..b6beaa40e6d5a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -12,6 +12,8 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Update to Apache Lucene 9.11.0 ([#14042](https://github.com/opensearch-project/OpenSearch/pull/14042)) - Bump `netty` from 4.1.110.Final to 4.1.111.Final ([#14356](https://github.com/opensearch-project/OpenSearch/pull/14356)) - Bump `org.wiremock:wiremock-standalone` from 3.3.1 to 3.6.0 ([#14361](https://github.com/opensearch-project/OpenSearch/pull/14361)) +- Bump `reactor` from 3.5.17 to 3.5.18 ([#14395](https://github.com/opensearch-project/OpenSearch/pull/14395)) +- Bump `reactor-netty` from 1.1.19 to 1.1.20 ([#14395](https://github.com/opensearch-project/OpenSearch/pull/14395)) - Bump `commons-net:commons-net` from 3.10.0 to 3.11.1 ([#14396](https://github.com/opensearch-project/OpenSearch/pull/14396)) ### Changed diff --git a/buildSrc/version.properties b/buildSrc/version.properties index d20c3bd46f304..eb96261b056e3 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -33,8 +33,8 @@ netty = 4.1.111.Final joda = 2.12.7 # project reactor -reactor_netty = 1.1.19 -reactor = 3.5.17 +reactor_netty = 1.1.20 +reactor = 3.5.18 # client dependencies httpclient5 = 5.2.1 diff --git a/plugins/repository-azure/licenses/reactor-netty-core-1.1.19.jar.sha1 b/plugins/repository-azure/licenses/reactor-netty-core-1.1.19.jar.sha1 deleted file mode 100644 index cbcbfcd87d682..0000000000000 --- a/plugins/repository-azure/licenses/reactor-netty-core-1.1.19.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -639e2c63ade6f2a49d7e501ca2264b74d240b448 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/reactor-netty-core-1.1.20.jar.sha1 b/plugins/repository-azure/licenses/reactor-netty-core-1.1.20.jar.sha1 new file mode 100644 index 0000000000000..2f4d023c88c80 --- /dev/null +++ b/plugins/repository-azure/licenses/reactor-netty-core-1.1.20.jar.sha1 @@ -0,0 +1 @@ +1a5ef52a470a82d9313e2e1ad8ba064bdbd38948 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/reactor-netty-http-1.1.19.jar.sha1 b/plugins/repository-azure/licenses/reactor-netty-http-1.1.19.jar.sha1 deleted file mode 100644 index 1eeedfc0926f5..0000000000000 --- a/plugins/repository-azure/licenses/reactor-netty-http-1.1.19.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -b4bbb1aeb64ecb2b3949c38983032a7f0b0ebd07 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/reactor-netty-http-1.1.20.jar.sha1 b/plugins/repository-azure/licenses/reactor-netty-http-1.1.20.jar.sha1 new file mode 100644 index 0000000000000..6c031e00e39c1 --- /dev/null +++ b/plugins/repository-azure/licenses/reactor-netty-http-1.1.20.jar.sha1 @@ -0,0 +1 @@ +8d4ee98405a5856cf0c9d7c1a70f3f14631e3c46 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/reactor-netty-core-1.1.19.jar.sha1 b/plugins/transport-reactor-netty4/licenses/reactor-netty-core-1.1.19.jar.sha1 deleted file mode 100644 index cbcbfcd87d682..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/reactor-netty-core-1.1.19.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -639e2c63ade6f2a49d7e501ca2264b74d240b448 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/reactor-netty-core-1.1.20.jar.sha1 b/plugins/transport-reactor-netty4/licenses/reactor-netty-core-1.1.20.jar.sha1 new file mode 100644 index 0000000000000..2f4d023c88c80 --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/reactor-netty-core-1.1.20.jar.sha1 @@ -0,0 +1 @@ +1a5ef52a470a82d9313e2e1ad8ba064bdbd38948 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/reactor-netty-http-1.1.19.jar.sha1 b/plugins/transport-reactor-netty4/licenses/reactor-netty-http-1.1.19.jar.sha1 deleted file mode 100644 index 1eeedfc0926f5..0000000000000 --- a/plugins/transport-reactor-netty4/licenses/reactor-netty-http-1.1.19.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -b4bbb1aeb64ecb2b3949c38983032a7f0b0ebd07 \ No newline at end of file diff --git a/plugins/transport-reactor-netty4/licenses/reactor-netty-http-1.1.20.jar.sha1 b/plugins/transport-reactor-netty4/licenses/reactor-netty-http-1.1.20.jar.sha1 new file mode 100644 index 0000000000000..6c031e00e39c1 --- /dev/null +++ b/plugins/transport-reactor-netty4/licenses/reactor-netty-http-1.1.20.jar.sha1 @@ -0,0 +1 @@ +8d4ee98405a5856cf0c9d7c1a70f3f14631e3c46 \ No newline at end of file diff --git a/server/licenses/reactor-core-3.5.17.jar.sha1 b/server/licenses/reactor-core-3.5.17.jar.sha1 deleted file mode 100644 index 6663356bab047..0000000000000 --- a/server/licenses/reactor-core-3.5.17.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -2cf9b080e3a2d8a5a39948260db5fd1dae54c3ac \ No newline at end of file diff --git a/server/licenses/reactor-core-3.5.18.jar.sha1 b/server/licenses/reactor-core-3.5.18.jar.sha1 new file mode 100644 index 0000000000000..c503f768beafa --- /dev/null +++ b/server/licenses/reactor-core-3.5.18.jar.sha1 @@ -0,0 +1 @@ +3a8157f7d66d71a407eb77ba12bce72a38c5b4da \ No newline at end of file From 21d3aaa8d2235b0e6fc2e9ce69f98b333c5aa293 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 17 Jun 2024 12:06:02 -0400 Subject: [PATCH 22/84] Bump org.apache.commons:commons-configuration2 from 2.10.1 to 2.11.0 in /plugins/repository-hdfs (#14399) * Bump org.apache.commons:commons-configuration2 Bumps org.apache.commons:commons-configuration2 from 2.10.1 to 2.11.0. --- updated-dependencies: - dependency-name: org.apache.commons:commons-configuration2 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] * Updating SHAs Signed-off-by: dependabot[bot] * Update changelog Signed-off-by: dependabot[bot] --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] --- CHANGELOG.md | 1 + plugins/repository-hdfs/build.gradle | 2 +- .../licenses/commons-configuration2-2.10.1.jar.sha1 | 1 - .../licenses/commons-configuration2-2.11.0.jar.sha1 | 1 + 4 files changed, 3 insertions(+), 2 deletions(-) delete mode 100644 plugins/repository-hdfs/licenses/commons-configuration2-2.10.1.jar.sha1 create mode 100644 plugins/repository-hdfs/licenses/commons-configuration2-2.11.0.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index b6beaa40e6d5a..b9413d379f88b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -15,6 +15,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `reactor` from 3.5.17 to 3.5.18 ([#14395](https://github.com/opensearch-project/OpenSearch/pull/14395)) - Bump `reactor-netty` from 1.1.19 to 1.1.20 ([#14395](https://github.com/opensearch-project/OpenSearch/pull/14395)) - Bump `commons-net:commons-net` from 3.10.0 to 3.11.1 ([#14396](https://github.com/opensearch-project/OpenSearch/pull/14396)) +- Bump `org.apache.commons:commons-configuration2` from 2.10.1 to 2.11.0 ([#14399](https://github.com/opensearch-project/OpenSearch/pull/14399)) ### Changed diff --git a/plugins/repository-hdfs/build.gradle b/plugins/repository-hdfs/build.gradle index eb50bd2d0615a..63eb783649884 100644 --- a/plugins/repository-hdfs/build.gradle +++ b/plugins/repository-hdfs/build.gradle @@ -74,7 +74,7 @@ dependencies { api "commons-codec:commons-codec:${versions.commonscodec}" api 'commons-collections:commons-collections:3.2.2' api "org.apache.commons:commons-compress:${versions.commonscompress}" - api 'org.apache.commons:commons-configuration2:2.10.1' + api 'org.apache.commons:commons-configuration2:2.11.0' api "commons-io:commons-io:${versions.commonsio}" api 'org.apache.commons:commons-lang3:3.14.0' implementation 'com.google.re2j:re2j:1.7' diff --git a/plugins/repository-hdfs/licenses/commons-configuration2-2.10.1.jar.sha1 b/plugins/repository-hdfs/licenses/commons-configuration2-2.10.1.jar.sha1 deleted file mode 100644 index d4c0f8417d357..0000000000000 --- a/plugins/repository-hdfs/licenses/commons-configuration2-2.10.1.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -2b681b3bcddeaa5bf5c2a2939cd77e2f9ad6efda \ No newline at end of file diff --git a/plugins/repository-hdfs/licenses/commons-configuration2-2.11.0.jar.sha1 b/plugins/repository-hdfs/licenses/commons-configuration2-2.11.0.jar.sha1 new file mode 100644 index 0000000000000..eea24804c5228 --- /dev/null +++ b/plugins/repository-hdfs/licenses/commons-configuration2-2.11.0.jar.sha1 @@ -0,0 +1 @@ +af5a2c6abe587074c0be1107fcb27fa2fad91304 \ No newline at end of file From a32035bff597359316760a6861ac02cafb4f2397 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 17 Jun 2024 15:37:00 -0400 Subject: [PATCH 23/84] Bump com.gradle.develocity from 3.17.4 to 3.17.5 (#14397) * Bump com.gradle.develocity from 3.17.4 to 3.17.5 Bumps com.gradle.develocity from 3.17.4 to 3.17.5. --- updated-dependencies: - dependency-name: com.gradle.develocity dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] * Update changelog Signed-off-by: dependabot[bot] --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] --- CHANGELOG.md | 1 + settings.gradle | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index b9413d379f88b..347c28792b35b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -16,6 +16,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `reactor-netty` from 1.1.19 to 1.1.20 ([#14395](https://github.com/opensearch-project/OpenSearch/pull/14395)) - Bump `commons-net:commons-net` from 3.10.0 to 3.11.1 ([#14396](https://github.com/opensearch-project/OpenSearch/pull/14396)) - Bump `org.apache.commons:commons-configuration2` from 2.10.1 to 2.11.0 ([#14399](https://github.com/opensearch-project/OpenSearch/pull/14399)) +- Bump `com.gradle.develocity` from 3.17.4 to 3.17.5 ([#14397](https://github.com/opensearch-project/OpenSearch/pull/14397)) ### Changed diff --git a/settings.gradle b/settings.gradle index 888ecc62aac7c..a96d00a4ab863 100644 --- a/settings.gradle +++ b/settings.gradle @@ -10,7 +10,7 @@ */ plugins { - id "com.gradle.develocity" version "3.17.4" + id "com.gradle.develocity" version "3.17.5" } ext.disableBuildCache = hasProperty('DISABLE_BUILD_CACHE') || System.getenv().containsKey('DISABLE_BUILD_CACHE') From 1d14569622f36fb163ee45e0e52620553a050a7d Mon Sep 17 00:00:00 2001 From: Liyun Xiu Date: Mon, 17 Jun 2024 13:11:20 -0700 Subject: [PATCH 24/84] Remove one unnecesary test and simply some code in a test (#14360) Signed-off-by: Liyun Xiu --- .../org/opensearch/ingest/IngestClientIT.java | 50 ++----------------- .../opensearch/ingest/IngestServiceTests.java | 27 ++++------ 2 files changed, 15 insertions(+), 62 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/ingest/IngestClientIT.java b/server/src/internalClusterTest/java/org/opensearch/ingest/IngestClientIT.java index dbde31ef1eb65..657d0f178e096 100644 --- a/server/src/internalClusterTest/java/org/opensearch/ingest/IngestClientIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/ingest/IngestClientIT.java @@ -189,7 +189,7 @@ private void runBulkTestWithRandomDocs(boolean shouldSetBatchSize) throws Except int numRequests = scaledRandomIntBetween(32, 128); BulkRequest bulkRequest = new BulkRequest(); if (shouldSetBatchSize) { - bulkRequest.batchSize(numRequests); + bulkRequest.batchSize(scaledRandomIntBetween(2, numRequests)); } for (int i = 0; i < numRequests; i++) { IndexRequest indexRequest = new IndexRequest("index").id(Integer.toString(i)).setPipeline("_id"); @@ -214,6 +214,9 @@ private void runBulkTestWithRandomDocs(boolean shouldSetBatchSize) throws Except ); assertThat(indexResponse, notNullValue()); assertThat(indexResponse.getId(), equalTo(Integer.toString(i))); + // verify field of successful doc + Map successDoc = client().prepareGet("index", indexResponse.getId()).get().getSourceAsMap(); + assertThat(successDoc.get("processed"), equalTo(true)); assertEquals(DocWriteResponse.Result.CREATED, indexResponse.getResult()); } } @@ -223,51 +226,6 @@ private void runBulkTestWithRandomDocs(boolean shouldSetBatchSize) throws Except assertTrue(deletePipelineResponse.isAcknowledged()); } - public void testBulkWithIngestFailuresBatch() throws Exception { - createIndex("index"); - - BytesReference source = BytesReference.bytes( - jsonBuilder().startObject() - .field("description", "my_pipeline") - .startArray("processors") - .startObject() - .startObject("test") - .endObject() - .endObject() - .endArray() - .endObject() - ); - PutPipelineRequest putPipelineRequest = new PutPipelineRequest("_id", source, MediaTypeRegistry.JSON); - client().admin().cluster().putPipeline(putPipelineRequest).get(); - - BulkRequest bulkRequest = new BulkRequest(); - bulkRequest.batchSize(2); - bulkRequest.add( - new IndexRequest("index").id("_fail").setPipeline("_id").source(Requests.INDEX_CONTENT_TYPE, "field", "value", "fail", true) - ); - bulkRequest.add( - new IndexRequest("index").id("_success").setPipeline("_id").source(Requests.INDEX_CONTENT_TYPE, "field", "value", "fail", false) - ); - - BulkResponse response = client().bulk(bulkRequest).actionGet(); - MatcherAssert.assertThat(response.getItems().length, equalTo(bulkRequest.requests().size())); - - Map results = Arrays.stream(response.getItems()) - .collect(Collectors.toMap(BulkItemResponse::getId, r -> r)); - - MatcherAssert.assertThat(results.keySet(), containsInAnyOrder("_fail", "_success")); - assertNotNull(results.get("_fail").getFailure()); - assertNull(results.get("_success").getFailure()); - - // verify field of successful doc - Map successDoc = client().prepareGet("index", "_success").get().getSourceAsMap(); - assertThat(successDoc.get("processed"), equalTo(true)); - - // cleanup - AcknowledgedResponse deletePipelineResponse = client().admin().cluster().prepareDeletePipeline("_id").get(); - assertTrue(deletePipelineResponse.isAcknowledged()); - } - public void testBulkWithIngestFailuresAndDropBatch() throws Exception { createIndex("index"); diff --git a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java index a32cd2c3cad3f..684297c11c140 100644 --- a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java +++ b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java @@ -81,6 +81,7 @@ import org.junit.Before; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.Comparator; @@ -88,6 +89,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; @@ -97,7 +99,6 @@ import java.util.function.LongSupplier; import java.util.stream.Collectors; -import org.mockito.ArgumentCaptor; import org.mockito.ArgumentMatcher; import org.mockito.invocation.InvocationOnMock; @@ -1923,27 +1924,21 @@ public void testExecuteBulkRequestInBatchWithExceptionAndDropInCallback() { return null; }).when(mockCompoundProcessor).batchExecute(any(), any()); - @SuppressWarnings("unchecked") - final BiConsumer failureHandler = mock(BiConsumer.class); - @SuppressWarnings("unchecked") - final BiConsumer completionHandler = mock(BiConsumer.class); - final IntConsumer dropHandler = mock(IntConsumer.class); + final Map failureHandler = new HashMap<>(); + final Map completionHandler = new HashMap<>(); + final List dropHandler = new ArrayList<>(); ingestService.executeBulkRequest( 3, bulkRequest.requests(), - failureHandler, - completionHandler, - dropHandler, + failureHandler::put, + completionHandler::put, + dropHandler::add, Names.WRITE, bulkRequest ); - ArgumentCaptor failureSlotCaptor = ArgumentCaptor.forClass(Integer.class); - verify(failureHandler, times(1)).accept(failureSlotCaptor.capture(), any()); - assertEquals(1, failureSlotCaptor.getValue().intValue()); - ArgumentCaptor dropSlotCaptor = ArgumentCaptor.forClass(Integer.class); - verify(dropHandler, times(1)).accept(dropSlotCaptor.capture()); - assertEquals(2, dropSlotCaptor.getValue().intValue()); - verify(completionHandler, times(1)).accept(Thread.currentThread(), null); + assertEquals(Set.of(1), failureHandler.keySet()); + assertEquals(List.of(2), dropHandler); + assertEquals(Set.of(Thread.currentThread()), completionHandler.keySet()); verify(mockCompoundProcessor, times(1)).batchExecute(any(), any()); verify(mockCompoundProcessor, never()).execute(any(), any()); } From 112704bf2976db8b0dd71980173896029a490cd9 Mon Sep 17 00:00:00 2001 From: Andrew Ross Date: Mon, 17 Jun 2024 13:33:50 -0700 Subject: [PATCH 25/84] Add missing data types to IngestDocument deep copy (#14380) PR #11725 added a new deep copy in the ScriptProcessor flow. If a script uses a Short or Byte data type then this new deep copy introduced a regression. This commit fixes that regression. However, it appears there has been an existing bug where using a Character type in the same way will fail (this failed before PR 11725). The failure is different, and appears to be related to something deeping in the XContent serialization layer. For now, I have fixed the regression but not yet dug into the failure with the Character data type. I have added a test that expects this failure. Resolves #14379 Signed-off-by: Andrew Ross --- CHANGELOG.md | 1 + .../test/ingest/190_script_processor.yml | 75 +++++++++++++++++++ .../org/opensearch/ingest/IngestDocument.java | 3 + .../ingest/IngestDocumentTests.java | 47 ++++++++---- 4 files changed, 111 insertions(+), 15 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 347c28792b35b..a43c0acf3219a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -25,6 +25,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ### Removed ### Fixed +- Fix handling of Short and Byte data types in ScriptProcessor ingest pipeline ([#14379](https://github.com/opensearch-project/OpenSearch/issues/14379)) ### Security diff --git a/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/190_script_processor.yml b/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/190_script_processor.yml index a66f02d6b6a6d..984c67d39757d 100644 --- a/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/190_script_processor.yml +++ b/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/190_script_processor.yml @@ -278,3 +278,78 @@ teardown: body: {source_field: "fooBar", foo: {foo: "bar"}} - match: { error.root_cause.0.type: "illegal_argument_exception" } - match: { error.root_cause.0.reason: "Iterable object is self-referencing itself (ingest script)" } + +--- +"Test painless data types": + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "script" : { + "source" : "ctx.byte = (byte)127;ctx.short = (short)32767;ctx.int = (int)2147483647;ctx.long = (long)9223372036854775807L;ctx.float = (float)0.1;ctx.double = (double)0.1;ctx.boolean = (boolean)true" + } + }, + { + "script" : { + "source" : "ctx.other_field = 'other_field'" + } + } + ] + } + - match: { acknowledged: true } + + - do: + index: + index: test + id: 1 + pipeline: "my_pipeline" + body: {source_field: "FooBar"} + + - do: + get: + index: test + id: 1 + - match: { _source.byte: 127 } + - match: { _source.int: 2147483647 } + - match: { _source.long: 9223372036854775807 } + - gt: { _source.float: 0.0 } + - lt: { _source.float: 0.2 } + - gt: { _source.double: 0.0 } + - lt: { _source.double: 0.2 } + - match: { _source.boolean: true } + +--- +"Test char type fails": + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "script" : { + "source" : "ctx.char = (char)'a'" + } + }, + { + "script" : { + "source" : "ctx.other_field = 'other_field'" + } + } + ] + } + - match: { acknowledged: true } + + - do: + catch: bad_request + index: + index: test + id: 1 + pipeline: "my_pipeline" + body: {source_field: "FooBar"} + - match: { error.root_cause.0.type: "illegal_argument_exception" } diff --git a/server/src/main/java/org/opensearch/ingest/IngestDocument.java b/server/src/main/java/org/opensearch/ingest/IngestDocument.java index d975b0014de1f..9ec59e4c275a8 100644 --- a/server/src/main/java/org/opensearch/ingest/IngestDocument.java +++ b/server/src/main/java/org/opensearch/ingest/IngestDocument.java @@ -776,6 +776,9 @@ public static Object deepCopy(Object value) { byte[] bytes = (byte[]) value; return Arrays.copyOf(bytes, bytes.length); } else if (value == null + || value instanceof Byte + || value instanceof Character + || value instanceof Short || value instanceof String || value instanceof Integer || value instanceof Long diff --git a/server/src/test/java/org/opensearch/ingest/IngestDocumentTests.java b/server/src/test/java/org/opensearch/ingest/IngestDocumentTests.java index be035bc6ef7ea..8b78689acd6d2 100644 --- a/server/src/test/java/org/opensearch/ingest/IngestDocumentTests.java +++ b/server/src/test/java/org/opensearch/ingest/IngestDocumentTests.java @@ -62,6 +62,7 @@ public class IngestDocumentTests extends OpenSearchTestCase { private static final ZonedDateTime BOGUS_TIMESTAMP = ZonedDateTime.of(2016, 10, 23, 0, 0, 0, 0, ZoneOffset.UTC); private IngestDocument ingestDocument; + private int initialSourceAndMetadataSize; @Before public void setTestIngestDocument() { @@ -70,7 +71,6 @@ public void setTestIngestDocument() { ingestMap.put("timestamp", BOGUS_TIMESTAMP); document.put("_ingest", ingestMap); document.put("foo", "bar"); - document.put("int", 123); Map innerObject = new HashMap<>(); innerObject.put("buzz", "hello world"); innerObject.put("foo_null", null); @@ -92,7 +92,17 @@ public void setTestIngestDocument() { list2.add("bar"); list2.add("baz"); document.put("list2", list2); + document.put("byte", (byte) 1); + document.put("short", (short) 2); + document.put("int", Integer.MAX_VALUE); + document.put("long", Long.MAX_VALUE); + document.put("float", 0.1f); + document.put("double", 0.1d); + document.put("char", 'a'); + document.put("string", "A test string \uD83C\uDF89"); + document.put("datetime", ZonedDateTime.parse("2007-12-03T10:15:30+01:00[Europe/Paris]")); ingestDocument = new IngestDocument("index", "id", null, null, null, document); + initialSourceAndMetadataSize = 16; // i.e. ingestDocument.getSourceAndMetadata().size() } public void testSelfReferencingSource() { @@ -101,11 +111,18 @@ public void testSelfReferencingSource() { expectThrows(IllegalArgumentException.class, () -> IngestDocument.deepCopyMap(value)); } + public void testCopy() { + final IngestDocument copy = new IngestDocument(ingestDocument); + assertThat(copy, equalTo(ingestDocument)); + assertThat(copy.getSourceAndMetadata(), not(sameInstance(ingestDocument.getSourceAndMetadata()))); + assertThat(copy.getIngestMetadata(), not(sameInstance(ingestDocument.getIngestMetadata()))); + } + public void testSimpleGetFieldValue() { assertThat(ingestDocument.getFieldValue("foo", String.class), equalTo("bar")); - assertThat(ingestDocument.getFieldValue("int", Integer.class), equalTo(123)); + assertThat(ingestDocument.getFieldValue("int", Integer.class), equalTo(Integer.MAX_VALUE)); assertThat(ingestDocument.getFieldValue("_source.foo", String.class), equalTo("bar")); - assertThat(ingestDocument.getFieldValue("_source.int", Integer.class), equalTo(123)); + assertThat(ingestDocument.getFieldValue("_source.int", Integer.class), equalTo(Integer.MAX_VALUE)); assertThat(ingestDocument.getFieldValue("_index", String.class), equalTo("index")); assertThat(ingestDocument.getFieldValue("_id", String.class), equalTo("id")); assertThat( @@ -578,7 +595,7 @@ public void testAppendFieldValueConvertIntegerToList() { @SuppressWarnings("unchecked") List list = (List) object; assertThat(list.size(), equalTo(2)); - assertThat(list.get(0), equalTo(123)); + assertThat(list.get(0), equalTo(Integer.MAX_VALUE)); assertThat(list.get(1), equalTo(456)); } @@ -589,7 +606,7 @@ public void testAppendFieldValuesConvertIntegerToList() { @SuppressWarnings("unchecked") List list = (List) object; assertThat(list.size(), equalTo(3)); - assertThat(list.get(0), equalTo(123)); + assertThat(list.get(0), equalTo(Integer.MAX_VALUE)); assertThat(list.get(1), equalTo(456)); assertThat(list.get(2), equalTo(789)); } @@ -812,23 +829,23 @@ public void testSetFieldValueEmptyName() { public void testRemoveField() { ingestDocument.removeField("foo"); - assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(7)); + assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(initialSourceAndMetadataSize - 1)); assertThat(ingestDocument.getSourceAndMetadata().containsKey("foo"), equalTo(false)); ingestDocument.removeField("_index"); - assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(6)); + assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(initialSourceAndMetadataSize - 2)); assertThat(ingestDocument.getSourceAndMetadata().containsKey("_index"), equalTo(false)); ingestDocument.removeField("_source.fizz"); - assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(5)); + assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(initialSourceAndMetadataSize - 3)); assertThat(ingestDocument.getSourceAndMetadata().containsKey("fizz"), equalTo(false)); assertThat(ingestDocument.getIngestMetadata().size(), equalTo(1)); ingestDocument.removeField("_ingest.timestamp"); - assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(5)); + assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(initialSourceAndMetadataSize - 3)); assertThat(ingestDocument.getIngestMetadata().size(), equalTo(0)); } public void testRemoveInnerField() { ingestDocument.removeField("fizz.buzz"); - assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(8)); + assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(initialSourceAndMetadataSize)); assertThat(ingestDocument.getSourceAndMetadata().get("fizz"), instanceOf(Map.class)); @SuppressWarnings("unchecked") Map map = (Map) ingestDocument.getSourceAndMetadata().get("fizz"); @@ -837,17 +854,17 @@ public void testRemoveInnerField() { ingestDocument.removeField("fizz.foo_null"); assertThat(map.size(), equalTo(2)); - assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(8)); + assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(initialSourceAndMetadataSize)); assertThat(ingestDocument.getSourceAndMetadata().containsKey("fizz"), equalTo(true)); ingestDocument.removeField("fizz.1"); assertThat(map.size(), equalTo(1)); - assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(8)); + assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(initialSourceAndMetadataSize)); assertThat(ingestDocument.getSourceAndMetadata().containsKey("fizz"), equalTo(true)); ingestDocument.removeField("fizz.list"); assertThat(map.size(), equalTo(0)); - assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(8)); + assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(initialSourceAndMetadataSize)); assertThat(ingestDocument.getSourceAndMetadata().containsKey("fizz"), equalTo(true)); } @@ -883,7 +900,7 @@ public void testRemoveSourceObject() { public void testRemoveIngestObject() { ingestDocument.removeField("_ingest"); - assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(7)); + assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(initialSourceAndMetadataSize - 1)); assertThat(ingestDocument.getSourceAndMetadata().containsKey("_ingest"), equalTo(false)); } @@ -905,7 +922,7 @@ public void testRemoveEmptyPathAfterStrippingOutPrefix() { public void testListRemoveField() { ingestDocument.removeField("list.0.field"); - assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(8)); + assertThat(ingestDocument.getSourceAndMetadata().size(), equalTo(initialSourceAndMetadataSize)); assertThat(ingestDocument.getSourceAndMetadata().containsKey("list"), equalTo(true)); Object object = ingestDocument.getSourceAndMetadata().get("list"); assertThat(object, instanceOf(List.class)); From e22b651ed06fb93165a4c029302adb1c504ef722 Mon Sep 17 00:00:00 2001 From: Gaurav Bafna <85113518+gbbafna@users.noreply.github.com> Date: Tue, 18 Jun 2024 16:17:31 +0530 Subject: [PATCH 26/84] [Remote Store] Rate limiter for low priority uploads (#14374) --------- Signed-off-by: Gaurav Bafna --- CHANGELOG.md | 1 + .../indices/create/RemoteCloneIndexIT.java | 86 +++++++++++++++++++ .../MockFsMetadataSupportedRepository.java | 4 +- .../index/store/RemoteDirectory.java | 18 +++- .../RemoteSegmentStoreDirectoryFactory.java | 1 + .../repositories/FilterRepository.java | 5 ++ .../opensearch/repositories/Repository.java | 9 +- .../blobstore/BlobStoreRepository.java | 28 ++++++ 8 files changed, 147 insertions(+), 5 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a43c0acf3219a..6654b478c74f8 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -6,6 +6,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ## [Unreleased 2.x] ### Added - Add fingerprint ingest processor ([#13724](https://github.com/opensearch-project/OpenSearch/pull/13724)) +- [Remote Store] Rate limiter for remote store low priority uploads ([#14374](https://github.com/opensearch-project/OpenSearch/pull/14374/)) ### Dependencies - Bump `org.gradle.test-retry` from 1.5.8 to 1.5.9 ([#13442](https://github.com/opensearch-project/OpenSearch/pull/13442)) diff --git a/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteCloneIndexIT.java b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteCloneIndexIT.java index a1122f279c7e4..acbd68fff6dd0 100644 --- a/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteCloneIndexIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/RemoteCloneIndexIT.java @@ -42,24 +42,32 @@ import org.opensearch.Version; import org.opensearch.action.admin.cluster.health.ClusterHealthRequest; import org.opensearch.action.admin.cluster.health.ClusterHealthResponse; +import org.opensearch.action.admin.cluster.repositories.get.GetRepositoriesRequest; +import org.opensearch.action.admin.cluster.repositories.get.GetRepositoriesResponse; import org.opensearch.action.admin.indices.settings.get.GetSettingsResponse; import org.opensearch.action.admin.indices.shrink.ResizeType; import org.opensearch.action.admin.indices.stats.IndicesStatsResponse; import org.opensearch.client.Requests; +import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.cluster.routing.allocation.decider.EnableAllocationDecider; import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.unit.ByteSizeValue; import org.opensearch.core.xcontent.MediaTypeRegistry; import org.opensearch.index.query.TermsQueryBuilder; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.remotestore.RemoteStoreBaseIntegTestCase; +import org.opensearch.repositories.RepositoriesService; import org.opensearch.test.OpenSearchIntegTestCase; import org.opensearch.test.VersionUtils; +import org.junit.Before; import java.util.concurrent.ExecutionException; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; @OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) public class RemoteCloneIndexIT extends RemoteStoreBaseIntegTestCase { @@ -69,6 +77,11 @@ protected boolean forbidPrivateIndexSettings() { return false; } + @Before + public void setup() { + asyncUploadMockFsRepo = true; + } + public void testCreateCloneIndex() { Version version = VersionUtils.randomIndexCompatibleVersion(random()); int numPrimaryShards = randomIntBetween(1, 5); @@ -140,6 +153,79 @@ public void testCreateCloneIndex() { } + public void testCreateCloneIndexLowPriorityRateLimit() { + Version version = VersionUtils.randomIndexCompatibleVersion(random()); + int numPrimaryShards = 1; + prepareCreate("source").setSettings( + Settings.builder().put(indexSettings()).put("number_of_shards", numPrimaryShards).put("index.version.created", version) + ).get(); + final int docs = randomIntBetween(0, 128); + for (int i = 0; i < docs; i++) { + client().prepareIndex("source").setSource("{\"foo\" : \"bar\", \"i\" : " + i + "}", MediaTypeRegistry.JSON).get(); + } + ByteSizeValue shardSize = client().admin().indices().prepareStats("source").execute().actionGet().getShards()[0].getStats() + .getStore() + .size(); + logger.info("Shard size is {}", shardSize); + internalCluster().ensureAtLeastNumDataNodes(2); + // ensure all shards are allocated otherwise the ensure green below might not succeed since we require the merge node + // if we change the setting too quickly we will end up with one replica unassigned which can't be assigned anymore due + // to the require._name below. + ensureGreen(); + // relocate all shards to one node such that we can merge it. + client().admin().indices().prepareUpdateSettings("source").setSettings(Settings.builder().put("index.blocks.write", true)).get(); + ensureGreen(); + + // disable rebalancing to be able to capture the right stats. balancing can move the target primary + // making it hard to pin point the source shards. + client().admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings(Settings.builder().put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), "none")) + .get(); + try { + // apply rate limiter + setLowPriorityUploadRate(REPOSITORY_NAME, "1kb"); + assertAcked( + client().admin() + .indices() + .prepareResizeIndex("source", "target") + .setResizeType(ResizeType.CLONE) + .setSettings(Settings.builder().put("index.number_of_replicas", 0).putNull("index.blocks.write").build()) + .get() + ); + ensureGreen(); + long uploadPauseTime = 0L; + for (RepositoriesService repositoriesService : internalCluster().getDataNodeInstances(RepositoriesService.class)) { + uploadPauseTime += repositoriesService.repository(REPOSITORY_NAME).getLowPriorityRemoteUploadThrottleTimeInNanos(); + } + assertThat(uploadPauseTime, greaterThan(TimeValue.timeValueSeconds(randomIntBetween(5, 10)).nanos())); + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + // clean up + client().admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings( + Settings.builder() + .put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), (String) null) + .put(RecoverySettings.INDICES_INTERNAL_REMOTE_UPLOAD_TIMEOUT.getKey(), (String) null) + ) + .get(); + } + } + + protected void setLowPriorityUploadRate(String repoName, String value) throws ExecutionException, InterruptedException { + GetRepositoriesRequest gr = new GetRepositoriesRequest(new String[] { repoName }); + GetRepositoriesResponse res = client().admin().cluster().getRepositories(gr).get(); + RepositoryMetadata rmd = res.repositories().get(0); + Settings.Builder settings = Settings.builder() + .put("location", rmd.settings().get("location")) + .put("max_remote_low_priority_upload_bytes_per_sec", value); + assertAcked(client().admin().cluster().preparePutRepository(repoName).setType(rmd.type()).setSettings(settings).get()); + } + public void testCreateCloneIndexFailure() throws ExecutionException, InterruptedException { asyncUploadMockFsRepo = false; Version version = VersionUtils.randomIndexCompatibleVersion(random()); diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/translogmetadata/mocks/MockFsMetadataSupportedRepository.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/translogmetadata/mocks/MockFsMetadataSupportedRepository.java index 333fba413ce4e..1abacbe5091dd 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/translogmetadata/mocks/MockFsMetadataSupportedRepository.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/translogmetadata/mocks/MockFsMetadataSupportedRepository.java @@ -16,9 +16,9 @@ import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.Environment; import org.opensearch.indices.recovery.RecoverySettings; -import org.opensearch.repositories.fs.FsRepository; +import org.opensearch.repositories.fs.ReloadableFsRepository; -public class MockFsMetadataSupportedRepository extends FsRepository { +public class MockFsMetadataSupportedRepository extends ReloadableFsRepository { public static Setting TRIGGER_DATA_INTEGRITY_FAILURE = Setting.boolSetting( "mock_fs_repository.trigger_data_integrity_failure", diff --git a/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java b/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java index ab76150f8f83d..99f78130ad3ef 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java @@ -64,6 +64,8 @@ public class RemoteDirectory extends Directory { private final UnaryOperator uploadRateLimiter; + private final UnaryOperator lowPriorityUploadRateLimiter; + private final UnaryOperator downloadRateLimiter; /** @@ -76,15 +78,17 @@ public BlobContainer getBlobContainer() { } public RemoteDirectory(BlobContainer blobContainer) { - this(blobContainer, UnaryOperator.identity(), UnaryOperator.identity()); + this(blobContainer, UnaryOperator.identity(), UnaryOperator.identity(), UnaryOperator.identity()); } public RemoteDirectory( BlobContainer blobContainer, UnaryOperator uploadRateLimiter, + UnaryOperator lowPriorityUploadRateLimiter, UnaryOperator downloadRateLimiter ) { this.blobContainer = blobContainer; + this.lowPriorityUploadRateLimiter = lowPriorityUploadRateLimiter; this.uploadRateLimiter = uploadRateLimiter; this.downloadRateLimiter = downloadRateLimiter; } @@ -357,13 +361,23 @@ private void uploadBlob( remoteIntegrityEnabled = ((AsyncMultiStreamBlobContainer) getBlobContainer()).remoteIntegrityCheckSupported(); } lowPriorityUpload = lowPriorityUpload || contentLength > ByteSizeUnit.GB.toBytes(15); + RemoteTransferContainer.OffsetRangeInputStreamSupplier offsetRangeInputStreamSupplier; + if (lowPriorityUpload) { + offsetRangeInputStreamSupplier = (size, position) -> lowPriorityUploadRateLimiter.apply( + new OffsetRangeIndexInputStream(from.openInput(src, ioContext), size, position) + ); + } else { + offsetRangeInputStreamSupplier = (size, position) -> uploadRateLimiter.apply( + new OffsetRangeIndexInputStream(from.openInput(src, ioContext), size, position) + ); + } RemoteTransferContainer remoteTransferContainer = new RemoteTransferContainer( src, remoteFileName, contentLength, true, lowPriorityUpload ? WritePriority.LOW : WritePriority.NORMAL, - (size, position) -> uploadRateLimiter.apply(new OffsetRangeIndexInputStream(from.openInput(src, ioContext), size, position)), + offsetRangeInputStreamSupplier, expectedChecksum, remoteIntegrityEnabled ); diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java index b965d7ce73ae6..3f6f4eeeef87b 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java @@ -77,6 +77,7 @@ public Directory newDirectory(String repositoryName, String indexUUID, ShardId s RemoteDirectory dataDirectory = new RemoteDirectory( blobStoreRepository.blobStore().blobContainer(dataPath), blobStoreRepository::maybeRateLimitRemoteUploadTransfers, + blobStoreRepository::maybeRateLimitLowPriorityRemoteUploadTransfers, blobStoreRepository::maybeRateLimitRemoteDownloadTransfers ); diff --git a/server/src/main/java/org/opensearch/repositories/FilterRepository.java b/server/src/main/java/org/opensearch/repositories/FilterRepository.java index 697ac37c4a175..d700a92ed4bad 100644 --- a/server/src/main/java/org/opensearch/repositories/FilterRepository.java +++ b/server/src/main/java/org/opensearch/repositories/FilterRepository.java @@ -142,6 +142,11 @@ public long getRemoteUploadThrottleTimeInNanos() { return in.getRemoteUploadThrottleTimeInNanos(); } + @Override + public long getLowPriorityRemoteUploadThrottleTimeInNanos() { + return in.getRemoteUploadThrottleTimeInNanos(); + } + @Override public long getRemoteDownloadThrottleTimeInNanos() { return in.getRemoteDownloadThrottleTimeInNanos(); diff --git a/server/src/main/java/org/opensearch/repositories/Repository.java b/server/src/main/java/org/opensearch/repositories/Repository.java index b3f1e9ce2eed9..ed30aad7b4dd2 100644 --- a/server/src/main/java/org/opensearch/repositories/Repository.java +++ b/server/src/main/java/org/opensearch/repositories/Repository.java @@ -207,10 +207,17 @@ default void deleteSnapshotsAndReleaseLockFiles( long getRestoreThrottleTimeInNanos(); /** - * Returns restore throttle time in nanoseconds + * Returns upload throttle time in nanoseconds */ long getRemoteUploadThrottleTimeInNanos(); + /** + * Returns low priority upload throttle time in nanoseconds + */ + default long getLowPriorityRemoteUploadThrottleTimeInNanos() { + return 0; + } + /** * Returns restore throttle time in nanoseconds */ diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java index 1a5701d9204ef..c41e97d278dd5 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -316,6 +316,8 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp private volatile RateLimiter remoteUploadRateLimiter; + private volatile RateLimiter remoteUploadLowPriorityRateLimiter; + private volatile RateLimiter remoteDownloadRateLimiter; private final CounterMetric snapshotRateLimitingTimeInNanos = new CounterMetric(); @@ -326,6 +328,8 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp private final CounterMetric remoteUploadRateLimitingTimeInNanos = new CounterMetric(); + private final CounterMetric remoteUploadLowPriorityRateLimitingTimeInNanos = new CounterMetric(); + public static final ChecksumBlobStoreFormat GLOBAL_METADATA_FORMAT = new ChecksumBlobStoreFormat<>( "metadata", METADATA_NAME_FORMAT, @@ -445,6 +449,11 @@ private void readRepositoryMetadata(RepositoryMetadata repositoryMetadata) { snapshotRateLimiter = getRateLimiter(metadata.settings(), "max_snapshot_bytes_per_sec", new ByteSizeValue(40, ByteSizeUnit.MB)); restoreRateLimiter = getRateLimiter(metadata.settings(), "max_restore_bytes_per_sec", ByteSizeValue.ZERO); remoteUploadRateLimiter = getRateLimiter(metadata.settings(), "max_remote_upload_bytes_per_sec", ByteSizeValue.ZERO); + remoteUploadLowPriorityRateLimiter = getRateLimiter( + metadata.settings(), + "max_remote_low_priority_upload_bytes_per_sec", + ByteSizeValue.ZERO + ); remoteDownloadRateLimiter = getRateLimiter(metadata.settings(), "max_remote_download_bytes_per_sec", ByteSizeValue.ZERO); readOnly = READONLY_SETTING.get(metadata.settings()); cacheRepositoryData = CACHE_REPOSITORY_DATA.get(metadata.settings()); @@ -1882,6 +1891,11 @@ public long getRemoteUploadThrottleTimeInNanos() { return remoteUploadRateLimitingTimeInNanos.count(); } + @Override + public long getLowPriorityRemoteUploadThrottleTimeInNanos() { + return remoteUploadLowPriorityRateLimitingTimeInNanos.count(); + } + @Override public long getRemoteDownloadThrottleTimeInNanos() { return remoteDownloadRateLimitingTimeInNanos.count(); @@ -3177,6 +3191,20 @@ public OffsetRangeInputStream maybeRateLimitRemoteUploadTransfers(OffsetRangeInp ); } + public OffsetRangeInputStream maybeRateLimitLowPriorityRemoteUploadTransfers(OffsetRangeInputStream offsetRangeInputStream) { + return maybeRateLimitRemoteTransfers( + maybeRateLimitRemoteTransfers( + offsetRangeInputStream, + () -> remoteUploadRateLimiter, + remoteUploadRateLimitingTimeInNanos, + BlobStoreTransferContext.REMOTE_UPLOAD + ), + () -> remoteUploadLowPriorityRateLimiter, + remoteUploadLowPriorityRateLimitingTimeInNanos, + BlobStoreTransferContext.REMOTE_UPLOAD + ); + } + public InputStream maybeRateLimitRemoteDownloadTransfers(InputStream inputStream) { return maybeRateLimit( maybeRateLimit( From 3a0c0c0b38c0b42bc519c3673d5cd4a1e3379550 Mon Sep 17 00:00:00 2001 From: Sachin Kale Date: Tue, 18 Jun 2024 16:28:44 +0530 Subject: [PATCH 27/84] Fix flaky test IndexShardTests.testCommitLevelRestoreShardFromRemoteStore (#14418) Signed-off-by: Sachin Kale --- .../java/org/opensearch/index/shard/IndexShardTests.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java index e5bfa8caee79a..3188de13bb00b 100644 --- a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java @@ -2956,6 +2956,14 @@ public void testRestoreShardFromRemoteStore(boolean performFlush) throws IOExcep ) ); + // Make sure to drain refreshes from the shard. Otherwise, if the refresh is in-progress, it overlaps with + // deletion of segment files in the subsequent code block. + for (ReferenceManager.RefreshListener refreshListener : target.getEngine().config().getInternalRefreshListener()) { + if (refreshListener instanceof ReleasableRetryableRefreshListener) { + ((ReleasableRetryableRefreshListener) refreshListener).drainRefreshes(); + } + } + // Delete files in store directory to restore from remote directory Directory storeDirectory = target.store().directory(); From 2a55a2a0c61e264d4d68759af9f82ea5e5e0e7d5 Mon Sep 17 00:00:00 2001 From: Prudhvi Godithi Date: Tue, 18 Jun 2024 09:14:06 -0700 Subject: [PATCH 28/84] Update DEVELOPER_GUIDE.md to add gradle-check-flaky-test-detector automation information (#14417) Signed-off-by: Prudhvi Godithi --- DEVELOPER_GUIDE.md | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/DEVELOPER_GUIDE.md b/DEVELOPER_GUIDE.md index bc11e7335af49..03cd189aa911e 100644 --- a/DEVELOPER_GUIDE.md +++ b/DEVELOPER_GUIDE.md @@ -651,16 +651,18 @@ Note that these snapshots do not follow the Maven [naming convention](https://ma ### Flaky Tests -OpenSearch has a very large test suite with long running, often failing (flaky), integration tests. Such individual tests are labelled as [Flaky Random Test Failure](https://github.com/opensearch-project/OpenSearch/issues?q=is%3Aopen+is%3Aissue+label%3A%22flaky-test%22). Your help is wanted fixing these! +If you encounter a test failure locally or in CI that is seemingly unrelated to the change in your pull request, it may be a known flaky test or a new test failure. OpenSearch has a very large test suite with long running, often failing (flaky), integration tests. Such individual tests are labelled as [Flaky Random Test Failure](https://github.com/opensearch-project/OpenSearch/issues?q=is%3Aopen+is%3Aissue+label%3A%22flaky-test%22). Your help is wanted fixing these! -If you encounter a build/test failure in CI that is unrelated to the change in your pull request, it may be a known flaky test, or a new test failure. +The automation [gradle-check-flaky-test-detector](https://build.ci.opensearch.org/job/gradle-check-flaky-test-detector/), which runs in OpenSearch public Jenkins, identifies failing flaky issues that are part of post-merge actions. Once a flaky test is identified, the automation creates an issue with detailed report that includes links to all relevant commits, the Gradle check build log, the test report, and pull requests that are impacted with the flaky test failures. This automation leverages data from the [OpenSearch Metrics Project](https://github.com/opensearch-project/opensearch-metrics) to establish a baseline for creating the issue and updating the flaky test report. For all flaky test issues created by automation, visit this [link](https://github.com/opensearch-project/OpenSearch/issues?q=is%3Aissue+is%3Aopen+label%3A%3Etest-failure+author%3Aopensearch-ci-bot). + +If you still see a failing test that is not part of the post merge actions, please do: + +* Follow failed CI links, and locate the failing test(s) or use the [Gradle Check Metrics Dashboard](#gradle-check-metrics-dashboard). +* Copy-paste the failure into a comment of your PR. +* Search through issues using the name of the failed test for whether this is a known flaky test. +* If no existing issue is found, open one. +* Retry CI via the GitHub UX or by pushing an update to your PR. -1. Follow failed CI links, and locate the failing test(s). -2. Copy-paste the failure into a comment of your PR. -3. Search through [issues](https://github.com/opensearch-project/OpenSearch/issues?q=is%3Aopen+is%3Aissue+label%3A%22flaky-test%22) using the name of the failed test for whether this is a known flaky test. -4. If an existing issue is found, paste a link to the known issue in a comment to your PR. -5. If no existing issue is found, open one. -6. Retry CI via the GitHub UX or by pushing an update to your PR. ### Gradle Check Metrics Dashboard From daf4ac7e826edfb7b60ef93371c30935b29916d4 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 18 Jun 2024 12:15:05 -0400 Subject: [PATCH 29/84] Bump com.nimbusds:nimbus-jose-jwt from 9.37.3 to 9.40 in /plugins/repository-azure (#14398) * Bump com.nimbusds:nimbus-jose-jwt in /plugins/repository-azure Bumps [com.nimbusds:nimbus-jose-jwt](https://bitbucket.org/connect2id/nimbus-jose-jwt) from 9.37.3 to 9.40. - [Changelog](https://bitbucket.org/connect2id/nimbus-jose-jwt/src/master/CHANGELOG.txt) - [Commits](https://bitbucket.org/connect2id/nimbus-jose-jwt/branches/compare/9.40..9.37.3) --- updated-dependencies: - dependency-name: com.nimbusds:nimbus-jose-jwt dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] * Updating SHAs Signed-off-by: dependabot[bot] * Update changelog Signed-off-by: dependabot[bot] Signed-off-by: Andriy Redko --------- Signed-off-by: dependabot[bot] Signed-off-by: Andriy Redko Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] --- CHANGELOG.md | 1 + plugins/repository-azure/build.gradle | 7 +------ .../licenses/nimbus-jose-jwt-9.37.3.jar.sha1 | 1 - .../licenses/nimbus-jose-jwt-9.40.jar.sha1 | 1 + 4 files changed, 3 insertions(+), 7 deletions(-) delete mode 100644 plugins/repository-azure/licenses/nimbus-jose-jwt-9.37.3.jar.sha1 create mode 100644 plugins/repository-azure/licenses/nimbus-jose-jwt-9.40.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index 6654b478c74f8..529a5ce57ddf3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -16,6 +16,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `reactor` from 3.5.17 to 3.5.18 ([#14395](https://github.com/opensearch-project/OpenSearch/pull/14395)) - Bump `reactor-netty` from 1.1.19 to 1.1.20 ([#14395](https://github.com/opensearch-project/OpenSearch/pull/14395)) - Bump `commons-net:commons-net` from 3.10.0 to 3.11.1 ([#14396](https://github.com/opensearch-project/OpenSearch/pull/14396)) +- Bump `com.nimbusds:nimbus-jose-jwt` from 9.37.3 to 9.40 ([#14398](https://github.com/opensearch-project/OpenSearch/pull/14398)) - Bump `org.apache.commons:commons-configuration2` from 2.10.1 to 2.11.0 ([#14399](https://github.com/opensearch-project/OpenSearch/pull/14399)) - Bump `com.gradle.develocity` from 3.17.4 to 3.17.5 ([#14397](https://github.com/opensearch-project/OpenSearch/pull/14397)) diff --git a/plugins/repository-azure/build.gradle b/plugins/repository-azure/build.gradle index 61e9f71712eaf..0fd30af71dd0a 100644 --- a/plugins/repository-azure/build.gradle +++ b/plugins/repository-azure/build.gradle @@ -63,7 +63,7 @@ dependencies { api "net.java.dev.jna:jna-platform:${versions.jna}" api 'com.microsoft.azure:msal4j:1.14.3' api 'com.nimbusds:oauth2-oidc-sdk:11.9.1' - api 'com.nimbusds:nimbus-jose-jwt:9.37.3' + api 'com.nimbusds:nimbus-jose-jwt:9.40' api 'com.nimbusds:content-type:2.3' api 'com.nimbusds:lang-tag:1.7' // Both msal4j:1.14.3 and oauth2-oidc-sdk:11.9.1 has compile dependency on different versions of json-smart, @@ -219,11 +219,6 @@ thirdPartyAudit { 'org.bouncycastle.cert.X509CertificateHolder', 'org.bouncycastle.cert.jcajce.JcaX509CertificateHolder', 'org.bouncycastle.cert.jcajce.JcaX509v3CertificateBuilder', - 'org.bouncycastle.crypto.InvalidCipherTextException', - 'org.bouncycastle.crypto.engines.AESEngine', - 'org.bouncycastle.crypto.modes.GCMBlockCipher', - 'org.bouncycastle.jcajce.provider.BouncyCastleFipsProvider', - 'org.bouncycastle.jce.provider.BouncyCastleProvider', 'org.bouncycastle.openssl.PEMKeyPair', 'org.bouncycastle.openssl.PEMParser', 'org.bouncycastle.openssl.jcajce.JcaPEMKeyConverter', diff --git a/plugins/repository-azure/licenses/nimbus-jose-jwt-9.37.3.jar.sha1 b/plugins/repository-azure/licenses/nimbus-jose-jwt-9.37.3.jar.sha1 deleted file mode 100644 index 7278cd8994f71..0000000000000 --- a/plugins/repository-azure/licenses/nimbus-jose-jwt-9.37.3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -700f71ffefd60c16bd8ce711a956967ea9071cec \ No newline at end of file diff --git a/plugins/repository-azure/licenses/nimbus-jose-jwt-9.40.jar.sha1 b/plugins/repository-azure/licenses/nimbus-jose-jwt-9.40.jar.sha1 new file mode 100644 index 0000000000000..83228caf233cc --- /dev/null +++ b/plugins/repository-azure/licenses/nimbus-jose-jwt-9.40.jar.sha1 @@ -0,0 +1 @@ +42b1dfa0360e4062951b070bac52dd8d96fd7b38 \ No newline at end of file From 903784b0afe756ee9f3e5eed7120f2289b207682 Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Tue, 18 Jun 2024 12:15:34 -0400 Subject: [PATCH 30/84] Bump Apache Lucene to 9.12.0-snapshot-c896995 (#14414) Signed-off-by: Andriy Redko --- buildSrc/version.properties | 2 +- libs/core/licenses/lucene-core-9.11.0.jar.sha1 | 1 - libs/core/licenses/lucene-core-9.12.0-snapshot-c896995.jar.sha1 | 1 + libs/core/src/main/java/org/opensearch/Version.java | 2 +- .../lang-expression/licenses/lucene-expressions-9.11.0.jar.sha1 | 1 - .../lucene-expressions-9.12.0-snapshot-c896995.jar.sha1 | 1 + .../analysis-icu/licenses/lucene-analysis-icu-9.11.0.jar.sha1 | 1 - .../lucene-analysis-icu-9.12.0-snapshot-c896995.jar.sha1 | 1 + .../licenses/lucene-analysis-kuromoji-9.11.0.jar.sha1 | 1 - .../lucene-analysis-kuromoji-9.12.0-snapshot-c896995.jar.sha1 | 1 + .../analysis-nori/licenses/lucene-analysis-nori-9.11.0.jar.sha1 | 1 - .../lucene-analysis-nori-9.12.0-snapshot-c896995.jar.sha1 | 1 + .../licenses/lucene-analysis-phonetic-9.11.0.jar.sha1 | 1 - .../lucene-analysis-phonetic-9.12.0-snapshot-c896995.jar.sha1 | 1 + .../licenses/lucene-analysis-smartcn-9.11.0.jar.sha1 | 1 - .../lucene-analysis-smartcn-9.12.0-snapshot-c896995.jar.sha1 | 1 + .../licenses/lucene-analysis-stempel-9.11.0.jar.sha1 | 1 - .../lucene-analysis-stempel-9.12.0-snapshot-c896995.jar.sha1 | 1 + .../licenses/lucene-analysis-morfologik-9.11.0.jar.sha1 | 1 - .../lucene-analysis-morfologik-9.12.0-snapshot-c896995.jar.sha1 | 1 + server/licenses/lucene-analysis-common-9.11.0.jar.sha1 | 1 - .../lucene-analysis-common-9.12.0-snapshot-c896995.jar.sha1 | 1 + server/licenses/lucene-backward-codecs-9.11.0.jar.sha1 | 1 - .../lucene-backward-codecs-9.12.0-snapshot-c896995.jar.sha1 | 1 + server/licenses/lucene-core-9.11.0.jar.sha1 | 1 - server/licenses/lucene-core-9.12.0-snapshot-c896995.jar.sha1 | 1 + server/licenses/lucene-grouping-9.11.0.jar.sha1 | 1 - .../licenses/lucene-grouping-9.12.0-snapshot-c896995.jar.sha1 | 1 + server/licenses/lucene-highlighter-9.11.0.jar.sha1 | 1 - .../lucene-highlighter-9.12.0-snapshot-c896995.jar.sha1 | 1 + server/licenses/lucene-join-9.11.0.jar.sha1 | 1 - server/licenses/lucene-join-9.12.0-snapshot-c896995.jar.sha1 | 1 + server/licenses/lucene-memory-9.11.0.jar.sha1 | 1 - server/licenses/lucene-memory-9.12.0-snapshot-c896995.jar.sha1 | 1 + server/licenses/lucene-misc-9.11.0.jar.sha1 | 1 - server/licenses/lucene-misc-9.12.0-snapshot-c896995.jar.sha1 | 1 + server/licenses/lucene-queries-9.11.0.jar.sha1 | 1 - server/licenses/lucene-queries-9.12.0-snapshot-c896995.jar.sha1 | 1 + server/licenses/lucene-queryparser-9.11.0.jar.sha1 | 1 - .../lucene-queryparser-9.12.0-snapshot-c896995.jar.sha1 | 1 + server/licenses/lucene-sandbox-9.11.0.jar.sha1 | 1 - server/licenses/lucene-sandbox-9.12.0-snapshot-c896995.jar.sha1 | 1 + server/licenses/lucene-spatial-extras-9.11.0.jar.sha1 | 1 - .../lucene-spatial-extras-9.12.0-snapshot-c896995.jar.sha1 | 1 + server/licenses/lucene-spatial3d-9.11.0.jar.sha1 | 1 - .../licenses/lucene-spatial3d-9.12.0-snapshot-c896995.jar.sha1 | 1 + server/licenses/lucene-suggest-9.11.0.jar.sha1 | 1 - server/licenses/lucene-suggest-9.12.0-snapshot-c896995.jar.sha1 | 1 + 48 files changed, 25 insertions(+), 25 deletions(-) delete mode 100644 libs/core/licenses/lucene-core-9.11.0.jar.sha1 create mode 100644 libs/core/licenses/lucene-core-9.12.0-snapshot-c896995.jar.sha1 delete mode 100644 modules/lang-expression/licenses/lucene-expressions-9.11.0.jar.sha1 create mode 100644 modules/lang-expression/licenses/lucene-expressions-9.12.0-snapshot-c896995.jar.sha1 delete mode 100644 plugins/analysis-icu/licenses/lucene-analysis-icu-9.11.0.jar.sha1 create mode 100644 plugins/analysis-icu/licenses/lucene-analysis-icu-9.12.0-snapshot-c896995.jar.sha1 delete mode 100644 plugins/analysis-kuromoji/licenses/lucene-analysis-kuromoji-9.11.0.jar.sha1 create mode 100644 plugins/analysis-kuromoji/licenses/lucene-analysis-kuromoji-9.12.0-snapshot-c896995.jar.sha1 delete mode 100644 plugins/analysis-nori/licenses/lucene-analysis-nori-9.11.0.jar.sha1 create mode 100644 plugins/analysis-nori/licenses/lucene-analysis-nori-9.12.0-snapshot-c896995.jar.sha1 delete mode 100644 plugins/analysis-phonetic/licenses/lucene-analysis-phonetic-9.11.0.jar.sha1 create mode 100644 plugins/analysis-phonetic/licenses/lucene-analysis-phonetic-9.12.0-snapshot-c896995.jar.sha1 delete mode 100644 plugins/analysis-smartcn/licenses/lucene-analysis-smartcn-9.11.0.jar.sha1 create mode 100644 plugins/analysis-smartcn/licenses/lucene-analysis-smartcn-9.12.0-snapshot-c896995.jar.sha1 delete mode 100644 plugins/analysis-stempel/licenses/lucene-analysis-stempel-9.11.0.jar.sha1 create mode 100644 plugins/analysis-stempel/licenses/lucene-analysis-stempel-9.12.0-snapshot-c896995.jar.sha1 delete mode 100644 plugins/analysis-ukrainian/licenses/lucene-analysis-morfologik-9.11.0.jar.sha1 create mode 100644 plugins/analysis-ukrainian/licenses/lucene-analysis-morfologik-9.12.0-snapshot-c896995.jar.sha1 delete mode 100644 server/licenses/lucene-analysis-common-9.11.0.jar.sha1 create mode 100644 server/licenses/lucene-analysis-common-9.12.0-snapshot-c896995.jar.sha1 delete mode 100644 server/licenses/lucene-backward-codecs-9.11.0.jar.sha1 create mode 100644 server/licenses/lucene-backward-codecs-9.12.0-snapshot-c896995.jar.sha1 delete mode 100644 server/licenses/lucene-core-9.11.0.jar.sha1 create mode 100644 server/licenses/lucene-core-9.12.0-snapshot-c896995.jar.sha1 delete mode 100644 server/licenses/lucene-grouping-9.11.0.jar.sha1 create mode 100644 server/licenses/lucene-grouping-9.12.0-snapshot-c896995.jar.sha1 delete mode 100644 server/licenses/lucene-highlighter-9.11.0.jar.sha1 create mode 100644 server/licenses/lucene-highlighter-9.12.0-snapshot-c896995.jar.sha1 delete mode 100644 server/licenses/lucene-join-9.11.0.jar.sha1 create mode 100644 server/licenses/lucene-join-9.12.0-snapshot-c896995.jar.sha1 delete mode 100644 server/licenses/lucene-memory-9.11.0.jar.sha1 create mode 100644 server/licenses/lucene-memory-9.12.0-snapshot-c896995.jar.sha1 delete mode 100644 server/licenses/lucene-misc-9.11.0.jar.sha1 create mode 100644 server/licenses/lucene-misc-9.12.0-snapshot-c896995.jar.sha1 delete mode 100644 server/licenses/lucene-queries-9.11.0.jar.sha1 create mode 100644 server/licenses/lucene-queries-9.12.0-snapshot-c896995.jar.sha1 delete mode 100644 server/licenses/lucene-queryparser-9.11.0.jar.sha1 create mode 100644 server/licenses/lucene-queryparser-9.12.0-snapshot-c896995.jar.sha1 delete mode 100644 server/licenses/lucene-sandbox-9.11.0.jar.sha1 create mode 100644 server/licenses/lucene-sandbox-9.12.0-snapshot-c896995.jar.sha1 delete mode 100644 server/licenses/lucene-spatial-extras-9.11.0.jar.sha1 create mode 100644 server/licenses/lucene-spatial-extras-9.12.0-snapshot-c896995.jar.sha1 delete mode 100644 server/licenses/lucene-spatial3d-9.11.0.jar.sha1 create mode 100644 server/licenses/lucene-spatial3d-9.12.0-snapshot-c896995.jar.sha1 delete mode 100644 server/licenses/lucene-suggest-9.11.0.jar.sha1 create mode 100644 server/licenses/lucene-suggest-9.12.0-snapshot-c896995.jar.sha1 diff --git a/buildSrc/version.properties b/buildSrc/version.properties index eb96261b056e3..af421b97e12e4 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -1,5 +1,5 @@ opensearch = 3.0.0 -lucene = 9.11.0 +lucene = 9.12.0-snapshot-c896995 bundled_jdk_vendor = adoptium bundled_jdk = 21.0.3+9 diff --git a/libs/core/licenses/lucene-core-9.11.0.jar.sha1 b/libs/core/licenses/lucene-core-9.11.0.jar.sha1 deleted file mode 100644 index b0d38c4165581..0000000000000 --- a/libs/core/licenses/lucene-core-9.11.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -2e487755a6814b2a1bc770c26569dcba86873dcf \ No newline at end of file diff --git a/libs/core/licenses/lucene-core-9.12.0-snapshot-c896995.jar.sha1 b/libs/core/licenses/lucene-core-9.12.0-snapshot-c896995.jar.sha1 new file mode 100644 index 0000000000000..299283562fddc --- /dev/null +++ b/libs/core/licenses/lucene-core-9.12.0-snapshot-c896995.jar.sha1 @@ -0,0 +1 @@ +826b328c37ea7f27c05d685db03bf8d2b00457ff \ No newline at end of file diff --git a/libs/core/src/main/java/org/opensearch/Version.java b/libs/core/src/main/java/org/opensearch/Version.java index d99dae2a5e64b..0cb2d4f867c12 100644 --- a/libs/core/src/main/java/org/opensearch/Version.java +++ b/libs/core/src/main/java/org/opensearch/Version.java @@ -106,7 +106,7 @@ public class Version implements Comparable, ToXContentFragment { public static final Version V_2_14_1 = new Version(2140199, org.apache.lucene.util.Version.LUCENE_9_10_0); public static final Version V_2_15_0 = new Version(2150099, org.apache.lucene.util.Version.LUCENE_9_10_0); public static final Version V_2_16_0 = new Version(2160099, org.apache.lucene.util.Version.LUCENE_9_11_0); - public static final Version V_3_0_0 = new Version(3000099, org.apache.lucene.util.Version.LUCENE_9_11_0); + public static final Version V_3_0_0 = new Version(3000099, org.apache.lucene.util.Version.LUCENE_9_12_0); public static final Version CURRENT = V_3_0_0; public static Version fromId(int id) { diff --git a/modules/lang-expression/licenses/lucene-expressions-9.11.0.jar.sha1 b/modules/lang-expression/licenses/lucene-expressions-9.11.0.jar.sha1 deleted file mode 100644 index 29aade3ad4298..0000000000000 --- a/modules/lang-expression/licenses/lucene-expressions-9.11.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -5e21d20edee0712472e7c6f605c9d97aeecf16c0 \ No newline at end of file diff --git a/modules/lang-expression/licenses/lucene-expressions-9.12.0-snapshot-c896995.jar.sha1 b/modules/lang-expression/licenses/lucene-expressions-9.12.0-snapshot-c896995.jar.sha1 new file mode 100644 index 0000000000000..6d8d3be59f945 --- /dev/null +++ b/modules/lang-expression/licenses/lucene-expressions-9.12.0-snapshot-c896995.jar.sha1 @@ -0,0 +1 @@ +9f0321cf2d34fca3f1f9334fdfee2b79d9d27444 \ No newline at end of file diff --git a/plugins/analysis-icu/licenses/lucene-analysis-icu-9.11.0.jar.sha1 b/plugins/analysis-icu/licenses/lucene-analysis-icu-9.11.0.jar.sha1 deleted file mode 100644 index 6f0501d3312ae..0000000000000 --- a/plugins/analysis-icu/licenses/lucene-analysis-icu-9.11.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -5c7f2d8eab0fca3fdc3d3e57a7f48a335dc7ac33 \ No newline at end of file diff --git a/plugins/analysis-icu/licenses/lucene-analysis-icu-9.12.0-snapshot-c896995.jar.sha1 b/plugins/analysis-icu/licenses/lucene-analysis-icu-9.12.0-snapshot-c896995.jar.sha1 new file mode 100644 index 0000000000000..696803bf63b46 --- /dev/null +++ b/plugins/analysis-icu/licenses/lucene-analysis-icu-9.12.0-snapshot-c896995.jar.sha1 @@ -0,0 +1 @@ +e6314f36fb29e208d58c0470f14269c9c36996ba \ No newline at end of file diff --git a/plugins/analysis-kuromoji/licenses/lucene-analysis-kuromoji-9.11.0.jar.sha1 b/plugins/analysis-kuromoji/licenses/lucene-analysis-kuromoji-9.11.0.jar.sha1 deleted file mode 100644 index 25031381c9cb3..0000000000000 --- a/plugins/analysis-kuromoji/licenses/lucene-analysis-kuromoji-9.11.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -efcf65dda1b4e9d7e83926fd5895a47e491cbf29 \ No newline at end of file diff --git a/plugins/analysis-kuromoji/licenses/lucene-analysis-kuromoji-9.12.0-snapshot-c896995.jar.sha1 b/plugins/analysis-kuromoji/licenses/lucene-analysis-kuromoji-9.12.0-snapshot-c896995.jar.sha1 new file mode 100644 index 0000000000000..7a12077d7fc62 --- /dev/null +++ b/plugins/analysis-kuromoji/licenses/lucene-analysis-kuromoji-9.12.0-snapshot-c896995.jar.sha1 @@ -0,0 +1 @@ +77fbf1e37af79715f28f66d8cc5b50af2982fc54 \ No newline at end of file diff --git a/plugins/analysis-nori/licenses/lucene-analysis-nori-9.11.0.jar.sha1 b/plugins/analysis-nori/licenses/lucene-analysis-nori-9.11.0.jar.sha1 deleted file mode 100644 index e27d45b217dad..0000000000000 --- a/plugins/analysis-nori/licenses/lucene-analysis-nori-9.11.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -59599d7b8bed2e6bd27d0dad7935c078b98c39cc \ No newline at end of file diff --git a/plugins/analysis-nori/licenses/lucene-analysis-nori-9.12.0-snapshot-c896995.jar.sha1 b/plugins/analysis-nori/licenses/lucene-analysis-nori-9.12.0-snapshot-c896995.jar.sha1 new file mode 100644 index 0000000000000..efed62c7e5e5b --- /dev/null +++ b/plugins/analysis-nori/licenses/lucene-analysis-nori-9.12.0-snapshot-c896995.jar.sha1 @@ -0,0 +1 @@ +a7a4e9c6004c72782e1002e1dcfaf4fbab7887d8 \ No newline at end of file diff --git a/plugins/analysis-phonetic/licenses/lucene-analysis-phonetic-9.11.0.jar.sha1 b/plugins/analysis-phonetic/licenses/lucene-analysis-phonetic-9.11.0.jar.sha1 deleted file mode 100644 index ad5473865537d..0000000000000 --- a/plugins/analysis-phonetic/licenses/lucene-analysis-phonetic-9.11.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e55f83bb373ac139e313f64e80afe1eb0a75b8c0 \ No newline at end of file diff --git a/plugins/analysis-phonetic/licenses/lucene-analysis-phonetic-9.12.0-snapshot-c896995.jar.sha1 b/plugins/analysis-phonetic/licenses/lucene-analysis-phonetic-9.12.0-snapshot-c896995.jar.sha1 new file mode 100644 index 0000000000000..f2020abcb8ef7 --- /dev/null +++ b/plugins/analysis-phonetic/licenses/lucene-analysis-phonetic-9.12.0-snapshot-c896995.jar.sha1 @@ -0,0 +1 @@ +42ac148a3769d6eb880d7f184d1917bad48ca303 \ No newline at end of file diff --git a/plugins/analysis-smartcn/licenses/lucene-analysis-smartcn-9.11.0.jar.sha1 b/plugins/analysis-smartcn/licenses/lucene-analysis-smartcn-9.11.0.jar.sha1 deleted file mode 100644 index 68abd162e7266..0000000000000 --- a/plugins/analysis-smartcn/licenses/lucene-analysis-smartcn-9.11.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -1be59d91c45a4de069611fb7f8aa3e8fd26020ec \ No newline at end of file diff --git a/plugins/analysis-smartcn/licenses/lucene-analysis-smartcn-9.12.0-snapshot-c896995.jar.sha1 b/plugins/analysis-smartcn/licenses/lucene-analysis-smartcn-9.12.0-snapshot-c896995.jar.sha1 new file mode 100644 index 0000000000000..b64e4061311e5 --- /dev/null +++ b/plugins/analysis-smartcn/licenses/lucene-analysis-smartcn-9.12.0-snapshot-c896995.jar.sha1 @@ -0,0 +1 @@ +adf2a25339ac8722647f8196288c1f5056bbf0de \ No newline at end of file diff --git a/plugins/analysis-stempel/licenses/lucene-analysis-stempel-9.11.0.jar.sha1 b/plugins/analysis-stempel/licenses/lucene-analysis-stempel-9.11.0.jar.sha1 deleted file mode 100644 index c5f1521ec3769..0000000000000 --- a/plugins/analysis-stempel/licenses/lucene-analysis-stempel-9.11.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -d5b5922acf3743b5a0c542959dd93fca8be333a7 \ No newline at end of file diff --git a/plugins/analysis-stempel/licenses/lucene-analysis-stempel-9.12.0-snapshot-c896995.jar.sha1 b/plugins/analysis-stempel/licenses/lucene-analysis-stempel-9.12.0-snapshot-c896995.jar.sha1 new file mode 100644 index 0000000000000..f56e7fc5df766 --- /dev/null +++ b/plugins/analysis-stempel/licenses/lucene-analysis-stempel-9.12.0-snapshot-c896995.jar.sha1 @@ -0,0 +1 @@ +a689e3af2015b21b7b4f41a1206b50c44519b6f7 \ No newline at end of file diff --git a/plugins/analysis-ukrainian/licenses/lucene-analysis-morfologik-9.11.0.jar.sha1 b/plugins/analysis-ukrainian/licenses/lucene-analysis-morfologik-9.11.0.jar.sha1 deleted file mode 100644 index b676ca507467a..0000000000000 --- a/plugins/analysis-ukrainian/licenses/lucene-analysis-morfologik-9.11.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -50fd7b471cbdd6648c4972169f3fc67fae9db7f6 \ No newline at end of file diff --git a/plugins/analysis-ukrainian/licenses/lucene-analysis-morfologik-9.12.0-snapshot-c896995.jar.sha1 b/plugins/analysis-ukrainian/licenses/lucene-analysis-morfologik-9.12.0-snapshot-c896995.jar.sha1 new file mode 100644 index 0000000000000..30732e3c4a688 --- /dev/null +++ b/plugins/analysis-ukrainian/licenses/lucene-analysis-morfologik-9.12.0-snapshot-c896995.jar.sha1 @@ -0,0 +1 @@ +c875f7706ee81b1fb0b3443767a8c9c52f30abc5 \ No newline at end of file diff --git a/server/licenses/lucene-analysis-common-9.11.0.jar.sha1 b/server/licenses/lucene-analysis-common-9.11.0.jar.sha1 deleted file mode 100644 index 7139f6a43a15a..0000000000000 --- a/server/licenses/lucene-analysis-common-9.11.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -75a0a333cf1e043102743066c929e65fe51cbcda \ No newline at end of file diff --git a/server/licenses/lucene-analysis-common-9.12.0-snapshot-c896995.jar.sha1 b/server/licenses/lucene-analysis-common-9.12.0-snapshot-c896995.jar.sha1 new file mode 100644 index 0000000000000..4b545e061c52f --- /dev/null +++ b/server/licenses/lucene-analysis-common-9.12.0-snapshot-c896995.jar.sha1 @@ -0,0 +1 @@ +73696492c6e59972974cd91e03ad9464e6b5bfcd \ No newline at end of file diff --git a/server/licenses/lucene-backward-codecs-9.11.0.jar.sha1 b/server/licenses/lucene-backward-codecs-9.11.0.jar.sha1 deleted file mode 100644 index 735e80b60b001..0000000000000 --- a/server/licenses/lucene-backward-codecs-9.11.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -db385446bc3fd70e7c6a744276c0a157bd60ee0a \ No newline at end of file diff --git a/server/licenses/lucene-backward-codecs-9.12.0-snapshot-c896995.jar.sha1 b/server/licenses/lucene-backward-codecs-9.12.0-snapshot-c896995.jar.sha1 new file mode 100644 index 0000000000000..ae4ffb2b1800b --- /dev/null +++ b/server/licenses/lucene-backward-codecs-9.12.0-snapshot-c896995.jar.sha1 @@ -0,0 +1 @@ +3cbb29ecc873e8c880a6f32e739655551708dbcf \ No newline at end of file diff --git a/server/licenses/lucene-core-9.11.0.jar.sha1 b/server/licenses/lucene-core-9.11.0.jar.sha1 deleted file mode 100644 index b0d38c4165581..0000000000000 --- a/server/licenses/lucene-core-9.11.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -2e487755a6814b2a1bc770c26569dcba86873dcf \ No newline at end of file diff --git a/server/licenses/lucene-core-9.12.0-snapshot-c896995.jar.sha1 b/server/licenses/lucene-core-9.12.0-snapshot-c896995.jar.sha1 new file mode 100644 index 0000000000000..299283562fddc --- /dev/null +++ b/server/licenses/lucene-core-9.12.0-snapshot-c896995.jar.sha1 @@ -0,0 +1 @@ +826b328c37ea7f27c05d685db03bf8d2b00457ff \ No newline at end of file diff --git a/server/licenses/lucene-grouping-9.11.0.jar.sha1 b/server/licenses/lucene-grouping-9.11.0.jar.sha1 deleted file mode 100644 index 562de95605b60..0000000000000 --- a/server/licenses/lucene-grouping-9.11.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -882bdaf209b0acb332aa34836616424bcbecf462 \ No newline at end of file diff --git a/server/licenses/lucene-grouping-9.12.0-snapshot-c896995.jar.sha1 b/server/licenses/lucene-grouping-9.12.0-snapshot-c896995.jar.sha1 new file mode 100644 index 0000000000000..b0268c98167d3 --- /dev/null +++ b/server/licenses/lucene-grouping-9.12.0-snapshot-c896995.jar.sha1 @@ -0,0 +1 @@ +a3a7003dc83197523e830f058a3748dbea96cab7 \ No newline at end of file diff --git a/server/licenses/lucene-highlighter-9.11.0.jar.sha1 b/server/licenses/lucene-highlighter-9.11.0.jar.sha1 deleted file mode 100644 index e0ef36d321c9d..0000000000000 --- a/server/licenses/lucene-highlighter-9.11.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -44accdc03c5482e602718f7bf91e5940ba4e4870 \ No newline at end of file diff --git a/server/licenses/lucene-highlighter-9.12.0-snapshot-c896995.jar.sha1 b/server/licenses/lucene-highlighter-9.12.0-snapshot-c896995.jar.sha1 new file mode 100644 index 0000000000000..d87927364b5a8 --- /dev/null +++ b/server/licenses/lucene-highlighter-9.12.0-snapshot-c896995.jar.sha1 @@ -0,0 +1 @@ +00eb386915c3cffa9efcef2dc4c406f8a6776afe \ No newline at end of file diff --git a/server/licenses/lucene-join-9.11.0.jar.sha1 b/server/licenses/lucene-join-9.11.0.jar.sha1 deleted file mode 100644 index 34c618ccfbcc7..0000000000000 --- a/server/licenses/lucene-join-9.11.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -32a30ee03ed4f3e43bf63250270b2d4d53050045 \ No newline at end of file diff --git a/server/licenses/lucene-join-9.12.0-snapshot-c896995.jar.sha1 b/server/licenses/lucene-join-9.12.0-snapshot-c896995.jar.sha1 new file mode 100644 index 0000000000000..25a95546ab544 --- /dev/null +++ b/server/licenses/lucene-join-9.12.0-snapshot-c896995.jar.sha1 @@ -0,0 +1 @@ +bb1fc572da7d473bf39672fd8ac323b15a1ffff0 \ No newline at end of file diff --git a/server/licenses/lucene-memory-9.11.0.jar.sha1 b/server/licenses/lucene-memory-9.11.0.jar.sha1 deleted file mode 100644 index d730cfb4b7660..0000000000000 --- a/server/licenses/lucene-memory-9.11.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -b3e80aa6aa3299118e76a23edc23b58f3ba5a515 \ No newline at end of file diff --git a/server/licenses/lucene-memory-9.12.0-snapshot-c896995.jar.sha1 b/server/licenses/lucene-memory-9.12.0-snapshot-c896995.jar.sha1 new file mode 100644 index 0000000000000..a0b3fd812561c --- /dev/null +++ b/server/licenses/lucene-memory-9.12.0-snapshot-c896995.jar.sha1 @@ -0,0 +1 @@ +05ebfcef0435f4870859a19c93020e24398bb939 \ No newline at end of file diff --git a/server/licenses/lucene-misc-9.11.0.jar.sha1 b/server/licenses/lucene-misc-9.11.0.jar.sha1 deleted file mode 100644 index 9be27f004435b..0000000000000 --- a/server/licenses/lucene-misc-9.11.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -54fe308908194e1b0697a1157a45c5998c9e1083 \ No newline at end of file diff --git a/server/licenses/lucene-misc-9.12.0-snapshot-c896995.jar.sha1 b/server/licenses/lucene-misc-9.12.0-snapshot-c896995.jar.sha1 new file mode 100644 index 0000000000000..1e2cc97c37257 --- /dev/null +++ b/server/licenses/lucene-misc-9.12.0-snapshot-c896995.jar.sha1 @@ -0,0 +1 @@ +d5747ed1be242b59aa36b0c32b0d3bd26b1d8fb8 \ No newline at end of file diff --git a/server/licenses/lucene-queries-9.11.0.jar.sha1 b/server/licenses/lucene-queries-9.11.0.jar.sha1 deleted file mode 100644 index b445610c25858..0000000000000 --- a/server/licenses/lucene-queries-9.11.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -987d1286949ddf514b8405fd453ed47bebdfb12d \ No newline at end of file diff --git a/server/licenses/lucene-queries-9.12.0-snapshot-c896995.jar.sha1 b/server/licenses/lucene-queries-9.12.0-snapshot-c896995.jar.sha1 new file mode 100644 index 0000000000000..31d4fe2886fc1 --- /dev/null +++ b/server/licenses/lucene-queries-9.12.0-snapshot-c896995.jar.sha1 @@ -0,0 +1 @@ +fb6678d7fe035e55c545450682b67be49457ef1b \ No newline at end of file diff --git a/server/licenses/lucene-queryparser-9.11.0.jar.sha1 b/server/licenses/lucene-queryparser-9.11.0.jar.sha1 deleted file mode 100644 index a1620ba9c7708..0000000000000 --- a/server/licenses/lucene-queryparser-9.11.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e97fe1c0d102edb8d6e1c01454992fd2b8d80ae0 \ No newline at end of file diff --git a/server/licenses/lucene-queryparser-9.12.0-snapshot-c896995.jar.sha1 b/server/licenses/lucene-queryparser-9.12.0-snapshot-c896995.jar.sha1 new file mode 100644 index 0000000000000..754e4ea20765f --- /dev/null +++ b/server/licenses/lucene-queryparser-9.12.0-snapshot-c896995.jar.sha1 @@ -0,0 +1 @@ +a11d7f56a9e78dc8e61f85b9b54ad94d73583bb3 \ No newline at end of file diff --git a/server/licenses/lucene-sandbox-9.11.0.jar.sha1 b/server/licenses/lucene-sandbox-9.11.0.jar.sha1 deleted file mode 100644 index 0dc193f054973..0000000000000 --- a/server/licenses/lucene-sandbox-9.11.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -5e46b790744bd9118ccc053f70235364213312a5 \ No newline at end of file diff --git a/server/licenses/lucene-sandbox-9.12.0-snapshot-c896995.jar.sha1 b/server/licenses/lucene-sandbox-9.12.0-snapshot-c896995.jar.sha1 new file mode 100644 index 0000000000000..08c2bc48ae85b --- /dev/null +++ b/server/licenses/lucene-sandbox-9.12.0-snapshot-c896995.jar.sha1 @@ -0,0 +1 @@ +75352855bcc052abfba821f878a27fd2b328fb1c \ No newline at end of file diff --git a/server/licenses/lucene-spatial-extras-9.11.0.jar.sha1 b/server/licenses/lucene-spatial-extras-9.11.0.jar.sha1 deleted file mode 100644 index 9d3a8d2857db6..0000000000000 --- a/server/licenses/lucene-spatial-extras-9.11.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -079ca5aaf544a3acde84b8b88423ace6dedc23eb \ No newline at end of file diff --git a/server/licenses/lucene-spatial-extras-9.12.0-snapshot-c896995.jar.sha1 b/server/licenses/lucene-spatial-extras-9.12.0-snapshot-c896995.jar.sha1 new file mode 100644 index 0000000000000..5e0b7196f48c2 --- /dev/null +++ b/server/licenses/lucene-spatial-extras-9.12.0-snapshot-c896995.jar.sha1 @@ -0,0 +1 @@ +299be103216d67ca092bef177642b275224e77a6 \ No newline at end of file diff --git a/server/licenses/lucene-spatial3d-9.11.0.jar.sha1 b/server/licenses/lucene-spatial3d-9.11.0.jar.sha1 deleted file mode 100644 index fd5ff875a0113..0000000000000 --- a/server/licenses/lucene-spatial3d-9.11.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -564558818d70fc384db5b36fbc8a0ab27b107609 \ No newline at end of file diff --git a/server/licenses/lucene-spatial3d-9.12.0-snapshot-c896995.jar.sha1 b/server/licenses/lucene-spatial3d-9.12.0-snapshot-c896995.jar.sha1 new file mode 100644 index 0000000000000..c79b34adea5e2 --- /dev/null +++ b/server/licenses/lucene-spatial3d-9.12.0-snapshot-c896995.jar.sha1 @@ -0,0 +1 @@ +29b4a76cd0bdabe0e067063831e661dedac6e503 \ No newline at end of file diff --git a/server/licenses/lucene-suggest-9.11.0.jar.sha1 b/server/licenses/lucene-suggest-9.11.0.jar.sha1 deleted file mode 100644 index 2fa96e97f307a..0000000000000 --- a/server/licenses/lucene-suggest-9.11.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -aa345db9b6caaf881e7890ea5b8911357d592167 \ No newline at end of file diff --git a/server/licenses/lucene-suggest-9.12.0-snapshot-c896995.jar.sha1 b/server/licenses/lucene-suggest-9.12.0-snapshot-c896995.jar.sha1 new file mode 100644 index 0000000000000..8d5334f0c4619 --- /dev/null +++ b/server/licenses/lucene-suggest-9.12.0-snapshot-c896995.jar.sha1 @@ -0,0 +1 @@ +597edb659e9ea93398a816e6837da7d47ef53873 \ No newline at end of file From 823ce6819ffe6b6ca4451c5bb28f44b95990d569 Mon Sep 17 00:00:00 2001 From: Kiran Prakash Date: Tue, 18 Jun 2024 10:18:25 -0700 Subject: [PATCH 31/84] [Tiered Cache] Use ConcurrentHashMap explicitly in IndicesRequestCache (#14409) Signed-off-by: Kiran Prakash --- .../indices/IndicesRequestCacheIT.java | 2 +- .../indices/IndicesRequestCache.java | 10 +++---- .../indices/IndicesRequestCacheTests.java | 26 ++++++++++++------- 3 files changed, 22 insertions(+), 16 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/IndicesRequestCacheIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/IndicesRequestCacheIT.java index 299652e4f07a9..0383aca2de33f 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/IndicesRequestCacheIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/IndicesRequestCacheIT.java @@ -1168,7 +1168,7 @@ public void testCacheCleanupAfterIndexDeletion() throws Exception { }, cacheCleanIntervalInMillis * 2, TimeUnit.MILLISECONDS); } - // when staleness threshold is lower than staleness, it should clean the cache from all indices having stale keys + // when staleness threshold is lower than staleness, it should clean cache from all indices having stale keys public void testStaleKeysCleanupWithMultipleIndices() throws Exception { int cacheCleanIntervalInMillis = 10; String node = internalCluster().startNode( diff --git a/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java b/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java index 06cd77a34fe0b..93946fa11de13 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java +++ b/server/src/main/java/org/opensearch/indices/IndicesRequestCache.java @@ -81,6 +81,7 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -506,7 +507,7 @@ public int hashCode() { * */ class IndicesRequestCacheCleanupManager implements Closeable { private final Set keysToClean; - private final ConcurrentMap> cleanupKeyToCountMap; + private final ConcurrentHashMap> cleanupKeyToCountMap; private final AtomicInteger staleKeysCount; private volatile double stalenessThreshold; private final IndicesRequestCacheCleaner cacheCleaner; @@ -514,7 +515,7 @@ class IndicesRequestCacheCleanupManager implements Closeable { IndicesRequestCacheCleanupManager(ThreadPool threadpool, TimeValue cleanInterval, double stalenessThreshold) { this.stalenessThreshold = stalenessThreshold; this.keysToClean = ConcurrentCollections.newConcurrentSet(); - this.cleanupKeyToCountMap = ConcurrentCollections.newConcurrentMap(); + this.cleanupKeyToCountMap = new ConcurrentHashMap<>(); this.staleKeysCount = new AtomicInteger(0); this.cacheCleaner = new IndicesRequestCacheCleaner(this, threadpool, cleanInterval); threadpool.schedule(cacheCleaner, cleanInterval, ThreadPool.Names.SAME); @@ -572,8 +573,7 @@ private void updateStaleCountOnCacheInsert(CleanupKey cleanupKey) { // pkg-private for testing void addToCleanupKeyToCountMap(ShardId shardId, String readerCacheKeyId) { - cleanupKeyToCountMap.computeIfAbsent(shardId, k -> ConcurrentCollections.newConcurrentMap()) - .merge(readerCacheKeyId, 1, Integer::sum); + cleanupKeyToCountMap.computeIfAbsent(shardId, k -> new ConcurrentHashMap<>()).merge(readerCacheKeyId, 1, Integer::sum); } /** @@ -831,7 +831,7 @@ public void close() { } // for testing - ConcurrentMap> getCleanupKeyToCountMap() { + ConcurrentHashMap> getCleanupKeyToCountMap() { return cleanupKeyToCountMap; } diff --git a/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java b/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java index 205712d388cd1..10688de3ab0ae 100644 --- a/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java +++ b/server/src/test/java/org/opensearch/indices/IndicesRequestCacheTests.java @@ -101,7 +101,6 @@ import java.util.Optional; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -491,7 +490,8 @@ public void testStaleCount_OnRemovalNotificationOfStaleKey_DecrementsStaleCount( indexShard.hashCode() ); // test the mapping - ConcurrentMap> cleanupKeyToCountMap = cache.cacheCleanupManager.getCleanupKeyToCountMap(); + ConcurrentHashMap> cleanupKeyToCountMap = cache.cacheCleanupManager + .getCleanupKeyToCountMap(); // shard id should exist assertTrue(cleanupKeyToCountMap.containsKey(shardId)); // reader CacheKeyId should NOT exist @@ -554,7 +554,8 @@ public void testStaleCount_OnRemovalNotificationOfNonStaleKey_DoesNotDecrementsS ); // test the mapping - ConcurrentMap> cleanupKeyToCountMap = cache.cacheCleanupManager.getCleanupKeyToCountMap(); + ConcurrentHashMap> cleanupKeyToCountMap = cache.cacheCleanupManager + .getCleanupKeyToCountMap(); // shard id should exist assertTrue(cleanupKeyToCountMap.containsKey(shardId)); // reader CacheKeyId should NOT exist @@ -722,7 +723,8 @@ public void testCleanupKeyToCountMapAreSetAppropriately() throws Exception { cache.getOrCompute(getEntity(indexShard), getLoader(reader), reader, getTermBytes()); assertEquals(1, cache.count()); // test the mappings - ConcurrentMap> cleanupKeyToCountMap = cache.cacheCleanupManager.getCleanupKeyToCountMap(); + ConcurrentHashMap> cleanupKeyToCountMap = cache.cacheCleanupManager + .getCleanupKeyToCountMap(); assertEquals(1, (int) cleanupKeyToCountMap.get(shardId).get(getReaderCacheKeyId(reader))); cache.getOrCompute(getEntity(indexShard), getLoader(secondReader), secondReader, getTermBytes()); @@ -796,7 +798,7 @@ public void testCleanupKeyToCountMapAreSetAppropriately() throws Exception { } // test adding to cleanupKeyToCountMap with multiple threads - public void testAddToCleanupKeyToCountMap() throws Exception { + public void testAddingToCleanupKeyToCountMapWorksAppropriatelyWithMultipleThreads() throws Exception { threadPool = getThreadPool(); Settings settings = Settings.builder().put(INDICES_REQUEST_CACHE_STALENESS_THRESHOLD_SETTING.getKey(), "51%").build(); cache = getIndicesRequestCache(settings); @@ -804,7 +806,7 @@ public void testAddToCleanupKeyToCountMap() throws Exception { int numberOfThreads = 10; int numberOfIterations = 1000; Phaser phaser = new Phaser(numberOfThreads + 1); // +1 for the main thread - AtomicBoolean exceptionDetected = new AtomicBoolean(false); + AtomicBoolean concurrentModificationExceptionDetected = new AtomicBoolean(false); ExecutorService executorService = Executors.newFixedThreadPool(numberOfThreads); @@ -817,7 +819,7 @@ public void testAddToCleanupKeyToCountMap() throws Exception { } } catch (ConcurrentModificationException e) { logger.error("ConcurrentModificationException detected in thread : " + e.getMessage()); - exceptionDetected.set(true); // Set flag if exception is detected + concurrentModificationExceptionDetected.set(true); // Set flag if exception is detected } }); } @@ -836,13 +838,17 @@ public void testAddToCleanupKeyToCountMap() throws Exception { } } catch (ConcurrentModificationException e) { logger.error("ConcurrentModificationException detected in main thread : " + e.getMessage()); - exceptionDetected.set(true); // Set flag if exception is detected + concurrentModificationExceptionDetected.set(true); // Set flag if exception is detected } }); executorService.shutdown(); - executorService.awaitTermination(60, TimeUnit.SECONDS); - assertFalse(exceptionDetected.get()); + assertTrue(executorService.awaitTermination(60, TimeUnit.SECONDS)); + assertEquals( + numberOfThreads * numberOfIterations, + cache.cacheCleanupManager.getCleanupKeyToCountMap().get(indexShard.shardId()).size() + ); + assertFalse(concurrentModificationExceptionDetected.get()); } private IndicesRequestCache getIndicesRequestCache(Settings settings) { From 120678d9b5e1ec611303f4dc5b3ce9b96fe21531 Mon Sep 17 00:00:00 2001 From: Heemin Kim Date: Tue, 18 Jun 2024 10:31:53 -0700 Subject: [PATCH 32/84] Switch to iterative version of WKT format parser (#14086) Signed-off-by: Heemin Kim --- CHANGELOG.md | 1 + .../geometry/utils/WellKnownText.java | 69 +++++++++++++++++-- .../geometry/GeometryCollectionTests.java | 30 ++++++++ 3 files changed, 94 insertions(+), 6 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 529a5ce57ddf3..dc85bd4f85ffd 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -28,6 +28,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ### Fixed - Fix handling of Short and Byte data types in ScriptProcessor ingest pipeline ([#14379](https://github.com/opensearch-project/OpenSearch/issues/14379)) +- Switch to iterative version of WKT format parser ([#14086](https://github.com/opensearch-project/OpenSearch/pull/14086)) ### Security diff --git a/libs/geo/src/main/java/org/opensearch/geometry/utils/WellKnownText.java b/libs/geo/src/main/java/org/opensearch/geometry/utils/WellKnownText.java index ed1d63e6d4fef..8ad135b8bc1ca 100644 --- a/libs/geo/src/main/java/org/opensearch/geometry/utils/WellKnownText.java +++ b/libs/geo/src/main/java/org/opensearch/geometry/utils/WellKnownText.java @@ -49,8 +49,10 @@ import java.io.StreamTokenizer; import java.io.StringReader; import java.text.ParseException; +import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collections; +import java.util.Deque; import java.util.List; import java.util.Locale; @@ -67,6 +69,7 @@ public class WellKnownText { public static final String RPAREN = ")"; public static final String COMMA = ","; public static final String NAN = "NaN"; + public static final int MAX_DEPTH_OF_GEO_COLLECTION = 1000; private final String NUMBER = ""; private final String EOF = "END-OF-STREAM"; @@ -278,6 +281,16 @@ public Geometry fromWKT(String wkt) throws IOException, ParseException { */ private Geometry parseGeometry(StreamTokenizer stream) throws IOException, ParseException { final String type = nextWord(stream).toLowerCase(Locale.ROOT); + switch (type) { + case "geometrycollection": + return parseGeometryCollection(stream); + default: + return parseSimpleGeometry(stream, type); + } + } + + private Geometry parseSimpleGeometry(StreamTokenizer stream, String type) throws IOException, ParseException { + assert "geometrycollection".equals(type) == false; switch (type) { case "point": return parsePoint(stream); @@ -294,7 +307,7 @@ private Geometry parseGeometry(StreamTokenizer stream) throws IOException, Parse case "bbox": return parseBBox(stream); case "geometrycollection": - return parseGeometryCollection(stream); + throw new IllegalStateException("Unexpected type: geometrycollection"); case "circle": // Not part of the standard, but we need it for internal serialization return parseCircle(stream); } @@ -305,12 +318,56 @@ private GeometryCollection parseGeometryCollection(StreamTokenizer str if (nextEmptyOrOpen(stream).equals(EMPTY)) { return GeometryCollection.EMPTY; } - List shapes = new ArrayList<>(); - shapes.add(parseGeometry(stream)); - while (nextCloserOrComma(stream).equals(COMMA)) { - shapes.add(parseGeometry(stream)); + + List topLevelShapes = new ArrayList<>(); + Deque> deque = new ArrayDeque<>(); + deque.push(topLevelShapes); + boolean isFirstIteration = true; + List currentLevelShapes = null; + while (!deque.isEmpty()) { + List previousShapes = deque.pop(); + if (currentLevelShapes != null) { + previousShapes.add(new GeometryCollection<>(currentLevelShapes)); + } + currentLevelShapes = previousShapes; + + if (isFirstIteration == true) { + isFirstIteration = false; + } else { + if (nextCloserOrComma(stream).equals(COMMA) == false) { + // Done with current level, continue with parent level + continue; + } + } + while (true) { + final String type = nextWord(stream).toLowerCase(Locale.ROOT); + if (type.equals("geometrycollection")) { + if (nextEmptyOrOpen(stream).equals(EMPTY) == false) { + // GEOMETRYCOLLECTION() -> 1 depth, GEOMETRYCOLLECTION(GEOMETRYCOLLECTION()) -> 2 depth + // When parsing the top level geometry collection, the queue size is zero. + // When max depth is 1, we don't want to push any sub geometry collection in the queue. + // Therefore, we subtract 2 from max depth. + if (deque.size() >= MAX_DEPTH_OF_GEO_COLLECTION - 2) { + throw new IllegalArgumentException( + "a geometry collection with a depth greater than " + MAX_DEPTH_OF_GEO_COLLECTION + " is not supported" + ); + } + deque.push(currentLevelShapes); + currentLevelShapes = new ArrayList<>(); + continue; + } + currentLevelShapes.add(GeometryCollection.EMPTY); + } else { + currentLevelShapes.add(parseSimpleGeometry(stream, type)); + } + + if (nextCloserOrComma(stream).equals(COMMA) == false) { + break; + } + } } - return new GeometryCollection<>(shapes); + + return new GeometryCollection<>(topLevelShapes); } private Point parsePoint(StreamTokenizer stream) throws IOException, ParseException { diff --git a/libs/geo/src/test/java/org/opensearch/geometry/GeometryCollectionTests.java b/libs/geo/src/test/java/org/opensearch/geometry/GeometryCollectionTests.java index 631b6456a77da..cd8bb8f585966 100644 --- a/libs/geo/src/test/java/org/opensearch/geometry/GeometryCollectionTests.java +++ b/libs/geo/src/test/java/org/opensearch/geometry/GeometryCollectionTests.java @@ -62,6 +62,11 @@ public void testBasicSerialization() throws IOException, ParseException { assertEquals("GEOMETRYCOLLECTION EMPTY", wkt.toWKT(GeometryCollection.EMPTY)); assertEquals(GeometryCollection.EMPTY, wkt.fromWKT("GEOMETRYCOLLECTION EMPTY)")); + + assertEquals( + new GeometryCollection(Arrays.asList(GeometryCollection.EMPTY)), + wkt.fromWKT("GEOMETRYCOLLECTION (GEOMETRYCOLLECTION EMPTY)") + ); } @SuppressWarnings("ConstantConditions") @@ -86,4 +91,29 @@ public void testInitValidation() { new StandardValidator(true).validate(new GeometryCollection(Collections.singletonList(new Point(20, 10, 30)))); } + + public void testDeeplyNestedGeometryCollection() throws IOException, ParseException { + WellKnownText wkt = new WellKnownText(true, new GeographyValidator(true)); + StringBuilder validGeometryCollectionHead = new StringBuilder("GEOMETRYCOLLECTION"); + StringBuilder validGeometryCollectionTail = new StringBuilder(" EMPTY"); + for (int i = 0; i < WellKnownText.MAX_DEPTH_OF_GEO_COLLECTION - 1; i++) { + validGeometryCollectionHead.append(" (GEOMETRYCOLLECTION"); + validGeometryCollectionTail.append(")"); + } + // Expect no exception + wkt.fromWKT(validGeometryCollectionHead.append(validGeometryCollectionTail).toString()); + + StringBuilder invalidGeometryCollectionHead = new StringBuilder("GEOMETRYCOLLECTION"); + StringBuilder invalidGeometryCollectionTail = new StringBuilder(" EMPTY"); + for (int i = 0; i < WellKnownText.MAX_DEPTH_OF_GEO_COLLECTION; i++) { + invalidGeometryCollectionHead.append(" (GEOMETRYCOLLECTION"); + invalidGeometryCollectionTail.append(")"); + } + + IllegalArgumentException ex = expectThrows( + IllegalArgumentException.class, + () -> wkt.fromWKT(invalidGeometryCollectionHead.append(invalidGeometryCollectionTail).toString()) + ); + assertEquals("a geometry collection with a depth greater than 1000 is not supported", ex.getMessage()); + } } From 802f2e6e4b21f27ddc6c01e7fc6f6cdcd69138d3 Mon Sep 17 00:00:00 2001 From: SwethaGuptha <156877431+SwethaGuptha@users.noreply.github.com> Date: Tue, 18 Jun 2024 23:11:42 +0530 Subject: [PATCH 33/84] Fix flaky test RecoveryFromGatewayIT.testMultipleReplicaShardAssignmentWithDelayedAllocationAndDifferentNodeStartTimeInBatchMode (#14424) Signed-off-by: Swetha Guptha Co-authored-by: Swetha Guptha --- .../gateway/RecoveryFromGatewayIT.java | 44 ++++++++----------- 1 file changed, 19 insertions(+), 25 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index fc0a574c191b1..6296608c64d37 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -34,7 +34,6 @@ import org.apache.lucene.index.CorruptIndexException; import org.opensearch.Version; -import org.opensearch.action.admin.cluster.allocation.ClusterAllocationExplainResponse; import org.opensearch.action.admin.cluster.configuration.AddVotingConfigExclusionsAction; import org.opensearch.action.admin.cluster.configuration.AddVotingConfigExclusionsRequest; import org.opensearch.action.admin.cluster.configuration.ClearVotingConfigExclusionsAction; @@ -101,6 +100,8 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.function.BooleanSupplier; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -883,17 +884,20 @@ public void testMultipleReplicaShardAssignmentWithDelayedAllocationAndDifferentN assertEquals(YELLOW, health.getStatus()); assertEquals(2, health.getUnassignedShards()); // shard should be unassigned because of Allocation_Delayed - ClusterAllocationExplainResponse allocationExplainResponse = client().admin() - .cluster() - .prepareAllocationExplain() - .setIndex("test") - .setShard(0) - .setPrimary(false) - .get(); - assertEquals( - AllocationDecision.ALLOCATION_DELAYED, - allocationExplainResponse.getExplanation().getShardAllocationDecision().getAllocateDecision().getAllocationDecision() + BooleanSupplier delayedShardAllocationStatusVerificationSupplier = () -> AllocationDecision.ALLOCATION_DELAYED.equals( + client().admin() + .cluster() + .prepareAllocationExplain() + .setIndex("test") + .setShard(0) + .setPrimary(false) + .get() + .getExplanation() + .getShardAllocationDecision() + .getAllocateDecision() + .getAllocationDecision() ); + waitUntil(delayedShardAllocationStatusVerificationSupplier, 2, TimeUnit.MINUTES); logger.info("--> restarting the node 1"); internalCluster().startDataOnlyNode( @@ -903,26 +907,16 @@ public void testMultipleReplicaShardAssignmentWithDelayedAllocationAndDifferentN assertTrue(clusterRerouteResponse.isAcknowledged()); ensureStableCluster(6); waitUntil( - () -> client().admin().cluster().health(Requests.clusterHealthRequest().timeout("5m")).actionGet().getInitializingShards() == 0 + () -> client().admin().cluster().health(Requests.clusterHealthRequest().timeout("5m")).actionGet().getActiveShards() == 3, + 2, + TimeUnit.MINUTES ); - health = client().admin().cluster().health(Requests.clusterHealthRequest().timeout("5m")).actionGet(); assertFalse(health.isTimedOut()); assertEquals(YELLOW, health.getStatus()); assertEquals(1, health.getUnassignedShards()); assertEquals(1, health.getDelayedUnassignedShards()); - allocationExplainResponse = client().admin() - .cluster() - .prepareAllocationExplain() - .setIndex("test") - .setShard(0) - .setPrimary(false) - .get(); - assertEquals( - AllocationDecision.ALLOCATION_DELAYED, - allocationExplainResponse.getExplanation().getShardAllocationDecision().getAllocateDecision().getAllocationDecision() - ); - + waitUntil(delayedShardAllocationStatusVerificationSupplier, 2, TimeUnit.MINUTES); logger.info("--> restarting the node 0"); internalCluster().startDataOnlyNode( Settings.builder().put("node.name", nodesWithReplicaShards.get(1)).put(replicaNode1DataPathSettings).build() From c5c6024da6d53c145189ad898515ae4874fe7fce Mon Sep 17 00:00:00 2001 From: Harsha Vamsi Kalluri Date: Tue, 18 Jun 2024 11:36:12 -0700 Subject: [PATCH 34/84] Set INDICES_MAX_CLAUSE_COUNT dynamically (#13568) --------- Signed-off-by: Harsha Vamsi Kalluri --- CHANGELOG.md | 1 + .../search/query/QueryStringIT.java | 4 +- .../search/query/SimpleQueryStringIT.java | 51 ++++++++++++++++++- .../common/settings/ClusterSettings.java | 3 +- .../index/search/QueryParserHelper.java | 4 +- .../org/opensearch/search/SearchModule.java | 9 ---- .../org/opensearch/search/SearchService.java | 13 +++++ 7 files changed, 68 insertions(+), 17 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index dc85bd4f85ffd..2daacf507c469 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -21,6 +21,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `com.gradle.develocity` from 3.17.4 to 3.17.5 ([#14397](https://github.com/opensearch-project/OpenSearch/pull/14397)) ### Changed +- Updated the `indices.query.bool.max_clause_count` setting from being static to dynamically updateable ([#13568](https://github.com/opensearch-project/OpenSearch/pull/13568)) ### Deprecated diff --git a/server/src/internalClusterTest/java/org/opensearch/search/query/QueryStringIT.java b/server/src/internalClusterTest/java/org/opensearch/search/query/QueryStringIT.java index c43a9c23661ea..8841638328ea4 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/query/QueryStringIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/query/QueryStringIT.java @@ -45,7 +45,7 @@ import org.opensearch.index.query.QueryStringQueryBuilder; import org.opensearch.search.SearchHit; import org.opensearch.search.SearchHits; -import org.opensearch.search.SearchModule; +import org.opensearch.search.SearchService; import org.opensearch.test.ParameterizedStaticSettingsOpenSearchIntegTestCase; import org.junit.Before; import org.junit.BeforeClass; @@ -101,7 +101,7 @@ public void setup() throws Exception { protected Settings nodeSettings(int nodeOrdinal) { return Settings.builder() .put(super.nodeSettings(nodeOrdinal)) - .put(SearchModule.INDICES_MAX_CLAUSE_COUNT_SETTING.getKey(), CLUSTER_MAX_CLAUSE_COUNT) + .put(SearchService.INDICES_MAX_CLAUSE_COUNT_SETTING.getKey(), CLUSTER_MAX_CLAUSE_COUNT) .build(); } diff --git a/server/src/internalClusterTest/java/org/opensearch/search/query/SimpleQueryStringIT.java b/server/src/internalClusterTest/java/org/opensearch/search/query/SimpleQueryStringIT.java index cae543506f919..f9ccdbd62de1c 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/query/SimpleQueryStringIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/query/SimpleQueryStringIT.java @@ -57,7 +57,7 @@ import org.opensearch.plugins.Plugin; import org.opensearch.search.SearchHit; import org.opensearch.search.SearchHits; -import org.opensearch.search.SearchModule; +import org.opensearch.search.SearchService; import org.opensearch.search.builder.SearchSourceBuilder; import org.opensearch.test.ParameterizedStaticSettingsOpenSearchIntegTestCase; import org.junit.BeforeClass; @@ -79,6 +79,7 @@ import static org.opensearch.index.query.QueryBuilders.simpleQueryStringQuery; import static org.opensearch.index.query.QueryBuilders.termQuery; import static org.opensearch.search.SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING; +import static org.opensearch.search.SearchService.INDICES_MAX_CLAUSE_COUNT_SETTING; import static org.opensearch.test.StreamsUtils.copyToStringFromClasspath; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertFailures; @@ -122,7 +123,7 @@ public static void createRandomClusterSetting() { protected Settings nodeSettings(int nodeOrdinal) { return Settings.builder() .put(super.nodeSettings(nodeOrdinal)) - .put(SearchModule.INDICES_MAX_CLAUSE_COUNT_SETTING.getKey(), CLUSTER_MAX_CLAUSE_COUNT) + .put(SearchService.INDICES_MAX_CLAUSE_COUNT_SETTING.getKey(), CLUSTER_MAX_CLAUSE_COUNT) .build(); } @@ -720,6 +721,52 @@ public void testFieldAliasOnDisallowedFieldType() throws Exception { assertHits(response.getHits(), "1"); } + public void testDynamicClauseCountUpdate() throws Exception { + client().prepareIndex("testdynamic").setId("1").setSource("field", "foo bar baz").get(); + assertAcked( + client().admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings(Settings.builder().put(INDICES_MAX_CLAUSE_COUNT_SETTING.getKey(), CLUSTER_MAX_CLAUSE_COUNT - 1)) + ); + refresh(); + StringBuilder sb = new StringBuilder("foo"); + + // create clause_count + 1 clauses to hit error + for (int i = 0; i <= CLUSTER_MAX_CLAUSE_COUNT; i++) { + sb.append(" OR foo" + i); + } + + QueryStringQueryBuilder qb = queryStringQuery(sb.toString()).field("field"); + + SearchPhaseExecutionException e = expectThrows(SearchPhaseExecutionException.class, () -> { + client().prepareSearch("testdynamic").setQuery(qb).get(); + }); + + assert (e.getDetailedMessage().contains("maxClauseCount is set to " + (CLUSTER_MAX_CLAUSE_COUNT - 1))); + + // increase clause count by 2 + assertAcked( + client().admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings(Settings.builder().put(INDICES_MAX_CLAUSE_COUNT_SETTING.getKey(), CLUSTER_MAX_CLAUSE_COUNT + 2)) + ); + + Thread.sleep(1); + + SearchResponse response = client().prepareSearch("testdynamic").setQuery(qb).get(); + assertHitCount(response, 1); + assertHits(response.getHits(), "1"); + + assertAcked( + client().admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings(Settings.builder().putNull(INDICES_MAX_CLAUSE_COUNT_SETTING.getKey())) + ); + } + private void assertHits(SearchHits hits, String... ids) { assertThat(hits.getTotalHits().value, equalTo((long) ids.length)); Set hitIds = new HashSet<>(); 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 7ea04acf00415..233a8d732d178 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -150,7 +150,6 @@ import org.opensearch.repositories.fs.FsRepository; import org.opensearch.rest.BaseRestHandler; import org.opensearch.script.ScriptService; -import org.opensearch.search.SearchModule; import org.opensearch.search.SearchService; import org.opensearch.search.aggregations.MultiBucketConsumerService; import org.opensearch.search.backpressure.settings.NodeDuressSettings; @@ -540,6 +539,7 @@ public void apply(Settings value, Settings current, Settings previous) { SearchService.MAX_OPEN_PIT_CONTEXT, SearchService.MAX_PIT_KEEPALIVE_SETTING, SearchService.MAX_AGGREGATION_REWRITE_FILTERS, + SearchService.INDICES_MAX_CLAUSE_COUNT_SETTING, SearchService.CARDINALITY_AGGREGATION_PRUNING_THRESHOLD, CreatePitController.PIT_INIT_KEEP_ALIVE, Node.WRITE_PORTS_FILE_SETTING, @@ -590,7 +590,6 @@ public void apply(Settings value, Settings current, Settings previous) { ResourceWatcherService.RELOAD_INTERVAL_HIGH, ResourceWatcherService.RELOAD_INTERVAL_MEDIUM, ResourceWatcherService.RELOAD_INTERVAL_LOW, - SearchModule.INDICES_MAX_CLAUSE_COUNT_SETTING, ThreadPool.ESTIMATED_TIME_INTERVAL_SETTING, FastVectorHighlighter.SETTING_TV_HIGHLIGHT_MULTI_VALUE, Node.BREAKER_TYPE_KEY, diff --git a/server/src/main/java/org/opensearch/index/search/QueryParserHelper.java b/server/src/main/java/org/opensearch/index/search/QueryParserHelper.java index bae58c0ce1ebf..06f450f090e63 100644 --- a/server/src/main/java/org/opensearch/index/search/QueryParserHelper.java +++ b/server/src/main/java/org/opensearch/index/search/QueryParserHelper.java @@ -38,7 +38,7 @@ import org.opensearch.index.mapper.MappedFieldType; import org.opensearch.index.query.QueryShardContext; import org.opensearch.index.query.QueryShardException; -import org.opensearch.search.SearchModule; +import org.opensearch.search.SearchService; import java.util.Collection; import java.util.HashMap; @@ -180,7 +180,7 @@ static Map resolveMappingField( } static void checkForTooManyFields(int numberOfFields, QueryShardContext context, @Nullable String inputPattern) { - Integer limit = SearchModule.INDICES_MAX_CLAUSE_COUNT_SETTING.get(context.getIndexSettings().getSettings()); + int limit = SearchService.INDICES_MAX_CLAUSE_COUNT_SETTING.get(context.getIndexSettings().getSettings()); if (numberOfFields > limit) { StringBuilder errorMsg = new StringBuilder("field expansion "); if (inputPattern != null) { diff --git a/server/src/main/java/org/opensearch/search/SearchModule.java b/server/src/main/java/org/opensearch/search/SearchModule.java index 88218896dceae..b463458847a88 100644 --- a/server/src/main/java/org/opensearch/search/SearchModule.java +++ b/server/src/main/java/org/opensearch/search/SearchModule.java @@ -37,7 +37,6 @@ import org.opensearch.common.Nullable; import org.opensearch.common.geo.GeoShapeType; import org.opensearch.common.geo.ShapesAvailability; -import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.common.xcontent.ParseFieldRegistry; import org.opensearch.core.ParseField; @@ -302,13 +301,6 @@ * @opensearch.internal */ public class SearchModule { - public static final Setting INDICES_MAX_CLAUSE_COUNT_SETTING = Setting.intSetting( - "indices.query.bool.max_clause_count", - 1024, - 1, - Integer.MAX_VALUE, - Setting.Property.NodeScope - ); private final Map highlighters; private final ParseFieldRegistry movingAverageModelParserRegistry = new ParseFieldRegistry<>( @@ -1094,7 +1086,6 @@ private void registerQueryParsers(List plugins) { registerQuery(new QuerySpec<>(MatchAllQueryBuilder.NAME, MatchAllQueryBuilder::new, MatchAllQueryBuilder::fromXContent)); registerQuery(new QuerySpec<>(QueryStringQueryBuilder.NAME, QueryStringQueryBuilder::new, QueryStringQueryBuilder::fromXContent)); registerQuery(new QuerySpec<>(BoostingQueryBuilder.NAME, BoostingQueryBuilder::new, BoostingQueryBuilder::fromXContent)); - BooleanQuery.setMaxClauseCount(INDICES_MAX_CLAUSE_COUNT_SETTING.get(settings)); registerQuery(new QuerySpec<>(BoolQueryBuilder.NAME, BoolQueryBuilder::new, BoolQueryBuilder::fromXContent)); registerQuery(new QuerySpec<>(TermQueryBuilder.NAME, TermQueryBuilder::new, TermQueryBuilder::fromXContent)); registerQuery(new QuerySpec<>(TermsQueryBuilder.NAME, TermsQueryBuilder::new, TermsQueryBuilder::fromXContent)); diff --git a/server/src/main/java/org/opensearch/search/SearchService.java b/server/src/main/java/org/opensearch/search/SearchService.java index 135af91912e5d..a53a7198c366f 100644 --- a/server/src/main/java/org/opensearch/search/SearchService.java +++ b/server/src/main/java/org/opensearch/search/SearchService.java @@ -35,6 +35,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.lucene.search.FieldDoc; +import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.TopDocs; import org.opensearch.OpenSearchException; import org.opensearch.action.ActionRunnable; @@ -281,6 +282,15 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv Property.NodeScope ); + public static final Setting INDICES_MAX_CLAUSE_COUNT_SETTING = Setting.intSetting( + "indices.query.bool.max_clause_count", + 1024, + 1, + Integer.MAX_VALUE, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + public static final Setting CLUSTER_ALLOW_DERIVED_FIELD_SETTING = Setting.boolSetting( "search.derived_field.enabled", true, @@ -411,6 +421,9 @@ public SearchService( lowLevelCancellation = LOW_LEVEL_CANCELLATION_SETTING.get(settings); clusterService.getClusterSettings().addSettingsUpdateConsumer(LOW_LEVEL_CANCELLATION_SETTING, this::setLowLevelCancellation); + IndexSearcher.setMaxClauseCount(INDICES_MAX_CLAUSE_COUNT_SETTING.get(settings)); + clusterService.getClusterSettings().addSettingsUpdateConsumer(INDICES_MAX_CLAUSE_COUNT_SETTING, IndexSearcher::setMaxClauseCount); + allowDerivedField = CLUSTER_ALLOW_DERIVED_FIELD_SETTING.get(settings); clusterService.getClusterSettings().addSettingsUpdateConsumer(CLUSTER_ALLOW_DERIVED_FIELD_SETTING, this::setAllowDerivedField); } From f8213b8492b213922bea35d22c4317a79786a74f Mon Sep 17 00:00:00 2001 From: Finn Date: Tue, 18 Jun 2024 13:48:05 -0700 Subject: [PATCH 35/84] Skip ComposeBuild task when docker cli not found (#14357) Signed-off-by: Finn Carroll --- .../org/opensearch/gradle/testfixtures/TestFixturesPlugin.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/buildSrc/src/main/java/org/opensearch/gradle/testfixtures/TestFixturesPlugin.java b/buildSrc/src/main/java/org/opensearch/gradle/testfixtures/TestFixturesPlugin.java index c9e18426966f9..e8772522b19a4 100644 --- a/buildSrc/src/main/java/org/opensearch/gradle/testfixtures/TestFixturesPlugin.java +++ b/buildSrc/src/main/java/org/opensearch/gradle/testfixtures/TestFixturesPlugin.java @@ -34,6 +34,7 @@ import com.avast.gradle.dockercompose.ComposeExtension; import com.avast.gradle.dockercompose.DockerComposePlugin; import com.avast.gradle.dockercompose.ServiceInfo; +import com.avast.gradle.dockercompose.tasks.ComposeBuild; import com.avast.gradle.dockercompose.tasks.ComposeDown; import com.avast.gradle.dockercompose.tasks.ComposePull; import com.avast.gradle.dockercompose.tasks.ComposeUp; @@ -200,6 +201,7 @@ public void execute(Task task) { maybeSkipTasks(tasks, dockerSupport, getTaskClass("org.opensearch.gradle.test.RestIntegTestTask")); maybeSkipTasks(tasks, dockerSupport, TestingConventionsTasks.class); maybeSkipTasks(tasks, dockerSupport, getTaskClass("org.opensearch.gradle.test.AntFixture")); + maybeSkipTasks(tasks, dockerSupport, ComposeBuild.class); maybeSkipTasks(tasks, dockerSupport, ComposeUp.class); maybeSkipTasks(tasks, dockerSupport, ComposePull.class); maybeSkipTasks(tasks, dockerSupport, ComposeDown.class); From f86990db1a1792465e747ffe456ef549e9963496 Mon Sep 17 00:00:00 2001 From: Shubh Sahu Date: Wed, 19 Jun 2024 11:40:45 +0530 Subject: [PATCH 36/84] [Remote Migration IT] Using voting config exclusion instead of stopping master node (#14433) Signed-off-by: Shubh Sahu --- .../RemoteMigrationIndexMetadataUpdateIT.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataUpdateIT.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataUpdateIT.java index 793adef0594fc..6885d37c4aab0 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataUpdateIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataUpdateIT.java @@ -8,6 +8,8 @@ package org.opensearch.remotemigration; +import org.opensearch.action.admin.cluster.configuration.AddVotingConfigExclusionsAction; +import org.opensearch.action.admin.cluster.configuration.AddVotingConfigExclusionsRequest; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.health.ClusterHealthStatus; import org.opensearch.cluster.metadata.IndexMetadata; @@ -471,7 +473,6 @@ public void testRemotePathMetadataAddedWithFirstPrimaryMovingToRemote() throws E * exclude docrep nodes, assert that remote index path file exists * when shards start relocating to the remote nodes. */ - @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/13939") public void testRemoteIndexPathFileExistsAfterMigration() throws Exception { String docrepClusterManager = internalCluster().startClusterManagerOnlyNode(); @@ -518,7 +519,11 @@ public void testRemoteIndexPathFileExistsAfterMigration() throws Exception { .isAcknowledged() ); - internalCluster().stopRandomNode(InternalTestCluster.nameFilter(docrepClusterManager)); + // elect cluster manager with remote-cluster state enabled + internalCluster().client() + .execute(AddVotingConfigExclusionsAction.INSTANCE, new AddVotingConfigExclusionsRequest(docrepClusterManager)) + .get(); + internalCluster().validateClusterFormed(); logger.info("---> Excluding docrep nodes from allocation"); From 8aed62e4c56914be1ddc802a118e7fa6039d9dc1 Mon Sep 17 00:00:00 2001 From: gaobinlong Date: Wed, 19 Jun 2024 22:19:20 +0800 Subject: [PATCH 37/84] Make the class CommunityIdProcessor final (#14448) * Make the class CommunityIdProcessor final Signed-off-by: Gao Binlong * Update change log Signed-off-by: Gao Binlong --------- Signed-off-by: Gao Binlong --- CHANGELOG.md | 1 + .../java/org/opensearch/ingest/common/CommunityIdProcessor.java | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 2daacf507c469..fae8686d1e45d 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -22,6 +22,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ### Changed - Updated the `indices.query.bool.max_clause_count` setting from being static to dynamically updateable ([#13568](https://github.com/opensearch-project/OpenSearch/pull/13568)) +- Make the class CommunityIdProcessor final ([#14448](https://github.com/opensearch-project/OpenSearch/pull/14448)) ### Deprecated diff --git a/modules/ingest-common/src/main/java/org/opensearch/ingest/common/CommunityIdProcessor.java b/modules/ingest-common/src/main/java/org/opensearch/ingest/common/CommunityIdProcessor.java index c968fb2f6c2da..c84892971c87e 100644 --- a/modules/ingest-common/src/main/java/org/opensearch/ingest/common/CommunityIdProcessor.java +++ b/modules/ingest-common/src/main/java/org/opensearch/ingest/common/CommunityIdProcessor.java @@ -29,7 +29,7 @@ * Processor that generating community id flow hash for the network flow tuples, the algorithm is defined in * Community ID Flow Hashing. */ -public class CommunityIdProcessor extends AbstractProcessor { +public final class CommunityIdProcessor extends AbstractProcessor { public static final String TYPE = "community_id"; // the version of the community id flow hashing algorithm private static final String COMMUNITY_ID_HASH_VERSION = "1"; From 8e32ed736372aa90db4c0ce3b85888b7b473a337 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Thu, 20 Jun 2024 00:21:52 +0530 Subject: [PATCH 38/84] Add tests for RemoteGlobalMetadataManager (#14394) * Add tests for RemoteGlobalMetadataManager Signed-off-by: Shivansh Arora * Add TestCapturingListener Signed-off-by: Shivansh Arora * Move TestCapturingListener to test/framework Signed-off-by: Shivansh Arora * Added javadoc Signed-off-by: Shivansh Arora --------- Signed-off-by: Shivansh Arora --- .../RemoteGlobalMetadataManagerTests.java | 532 +++++++++++++++++- .../RemoteCoordinationMetadataTests.java | 2 +- .../model/RemoteCustomMetadataTests.java | 2 +- .../model/RemoteGlobalMetadataTests.java | 2 +- ...RemoteHashesOfConsistentSettingsTests.java | 2 +- ...RemotePersistentSettingsMetadataTests.java | 2 +- .../model/RemoteTemplatesMetadataTests.java | 2 +- .../common/util/TestCapturingListener.java | 39 ++ 8 files changed, 573 insertions(+), 10 deletions(-) create mode 100644 test/framework/src/main/java/org/opensearch/common/util/TestCapturingListener.java diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManagerTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManagerTests.java index bd01bc1ab0cdb..c543f986b3e86 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManagerTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManagerTests.java @@ -9,19 +9,36 @@ package org.opensearch.gateway.remote; import org.opensearch.Version; +import org.opensearch.action.LatchedActionListener; import org.opensearch.cluster.ClusterModule; import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.DiffableUtils; +import org.opensearch.cluster.coordination.CoordinationMetadata; +import org.opensearch.cluster.metadata.DiffableStringMap; import org.opensearch.cluster.metadata.IndexGraveyard; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.metadata.Metadata.XContentContext; +import org.opensearch.cluster.metadata.TemplatesMetadata; +import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.network.NetworkModule; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.TestCapturingListener; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.common.io.stream.NamedWriteableRegistry; import org.opensearch.core.compress.Compressor; import org.opensearch.core.compress.NoneCompressor; import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.gateway.remote.model.RemoteCoordinationMetadata; +import org.opensearch.gateway.remote.model.RemoteCustomMetadata; +import org.opensearch.gateway.remote.model.RemoteGlobalMetadata; +import org.opensearch.gateway.remote.model.RemoteHashesOfConsistentSettings; +import org.opensearch.gateway.remote.model.RemotePersistentSettingsMetadata; +import org.opensearch.gateway.remote.model.RemoteReadResult; +import org.opensearch.gateway.remote.model.RemoteTemplatesMetadata; +import org.opensearch.gateway.remote.model.RemoteTransientSettingsMetadata; +import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.index.translog.transfer.BlobStoreTransferService; import org.opensearch.indices.IndicesModule; import org.opensearch.repositories.blobstore.BlobStoreRepository; @@ -32,14 +49,48 @@ import org.junit.After; import org.junit.Before; +import java.io.IOException; +import java.io.InputStream; import java.util.EnumSet; import java.util.List; import java.util.Map; +import java.util.concurrent.CountDownLatch; import java.util.function.Function; import java.util.stream.Stream; import static java.util.stream.Collectors.toList; +import static org.opensearch.cluster.metadata.Metadata.isGlobalStateEquals; +import static org.opensearch.common.blobstore.stream.write.WritePriority.URGENT; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.CLUSTER_STATE_PATH_TOKEN; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.CUSTOM_DELIMITER; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.DELIMITER; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.FORMAT_PARAMS; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.GLOBAL_METADATA_CURRENT_CODEC_VERSION; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.GLOBAL_METADATA_PATH_TOKEN; +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.PATH_DELIMITER; +import static org.opensearch.gateway.remote.model.RemoteCoordinationMetadata.COORDINATION_METADATA; +import static org.opensearch.gateway.remote.model.RemoteCoordinationMetadata.COORDINATION_METADATA_FORMAT; +import static org.opensearch.gateway.remote.model.RemoteCoordinationMetadataTests.getCoordinationMetadata; +import static org.opensearch.gateway.remote.model.RemoteCustomMetadata.CUSTOM_METADATA; +import static org.opensearch.gateway.remote.model.RemoteCustomMetadataTests.getCustomMetadata; +import static org.opensearch.gateway.remote.model.RemoteGlobalMetadata.GLOBAL_METADATA; +import static org.opensearch.gateway.remote.model.RemoteGlobalMetadata.GLOBAL_METADATA_FORMAT; +import static org.opensearch.gateway.remote.model.RemoteGlobalMetadataTests.getGlobalMetadata; +import static org.opensearch.gateway.remote.model.RemoteHashesOfConsistentSettings.HASHES_OF_CONSISTENT_SETTINGS; +import static org.opensearch.gateway.remote.model.RemoteHashesOfConsistentSettings.HASHES_OF_CONSISTENT_SETTINGS_FORMAT; +import static org.opensearch.gateway.remote.model.RemoteHashesOfConsistentSettingsTests.getHashesOfConsistentSettings; +import static org.opensearch.gateway.remote.model.RemotePersistentSettingsMetadata.SETTING_METADATA; +import static org.opensearch.gateway.remote.model.RemotePersistentSettingsMetadataTests.getSettings; +import static org.opensearch.gateway.remote.model.RemoteTemplatesMetadata.TEMPLATES_METADATA; +import static org.opensearch.gateway.remote.model.RemoteTemplatesMetadata.TEMPLATES_METADATA_FORMAT; +import static org.opensearch.gateway.remote.model.RemoteTemplatesMetadataTests.getTemplatesMetadata; +import static org.opensearch.gateway.remote.model.RemoteTransientSettingsMetadata.TRANSIENT_SETTING_METADATA; import static org.hamcrest.Matchers.is; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyIterable; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -47,26 +98,36 @@ public class RemoteGlobalMetadataManagerTests extends OpenSearchTestCase { private RemoteGlobalMetadataManager remoteGlobalMetadataManager; private ClusterSettings clusterSettings; private BlobStoreRepository blobStoreRepository; + private BlobStoreTransferService blobStoreTransferService; + private Compressor compressor; + private NamedXContentRegistry xContentRegistry; + private NamedWriteableRegistry namedWriteableRegistry; private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); + private final long METADATA_VERSION = 7331L; + private final String CLUSTER_NAME = "test-cluster"; + private final String CLUSTER_UUID = "test-cluster-uuid"; @Before public void setup() { clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); blobStoreRepository = mock(BlobStoreRepository.class); - BlobStoreTransferService blobStoreTransferService = mock(BlobStoreTransferService.class); - NamedXContentRegistry xContentRegistry = new NamedXContentRegistry( + blobStoreTransferService = mock(BlobStoreTransferService.class); + compressor = new NoneCompressor(); + xContentRegistry = new NamedXContentRegistry( Stream.of( NetworkModule.getNamedXContents().stream(), IndicesModule.getNamedXContents().stream(), ClusterModule.getNamedXWriteables().stream() ).flatMap(Function.identity()).collect(toList()) ); - Compressor compressor = new NoneCompressor(); + namedWriteableRegistry = writableRegistry(); + BlobPath blobPath = new BlobPath(); when(blobStoreRepository.getCompressor()).thenReturn(compressor); when(blobStoreRepository.getNamedXContentRegistry()).thenReturn(xContentRegistry); + when(blobStoreRepository.basePath()).thenReturn(blobPath); remoteGlobalMetadataManager = new RemoteGlobalMetadataManager( clusterSettings, - "test-cluster", + CLUSTER_NAME, blobStoreRepository, blobStoreTransferService, writableRegistry(), @@ -96,6 +157,469 @@ public void testGlobalMetadataUploadWaitTimeSetting() { assertEquals(globalMetadataUploadTimeout, remoteGlobalMetadataManager.getGlobalMetadataUploadTimeout().seconds()); } + public void testGetReadMetadataAsyncAction_CoordinationMetadata() throws Exception { + CoordinationMetadata coordinationMetadata = getCoordinationMetadata(); + String fileName = randomAlphaOfLength(10); + RemoteCoordinationMetadata coordinationMetadataForDownload = new RemoteCoordinationMetadata( + fileName, + CLUSTER_UUID, + compressor, + xContentRegistry + ); + when(blobStoreTransferService.downloadBlob(anyIterable(), anyString())).thenReturn( + COORDINATION_METADATA_FORMAT.serialize(coordinationMetadata, fileName, compressor, FORMAT_PARAMS).streamInput() + ); + TestCapturingListener listener = new TestCapturingListener<>(); + CountDownLatch latch = new CountDownLatch(1); + + remoteGlobalMetadataManager.getAsyncMetadataReadAction( + coordinationMetadataForDownload, + COORDINATION_METADATA, + new LatchedActionListener<>(listener, latch) + ).run(); + latch.await(); + assertNull(listener.getFailure()); + assertNotNull(listener.getResult()); + assertEquals(coordinationMetadata, listener.getResult().getObj()); + assertEquals(COORDINATION_METADATA, listener.getResult().getComponent()); + assertEquals(COORDINATION_METADATA, listener.getResult().getComponentName()); + } + + public void testGetAsyncMetadataWriteAction_CoordinationMetadata() throws Exception { + CoordinationMetadata coordinationMetadata = getCoordinationMetadata(); + RemoteCoordinationMetadata remoteCoordinationMetadata = new RemoteCoordinationMetadata( + coordinationMetadata, + METADATA_VERSION, + CLUSTER_UUID, + compressor, + xContentRegistry + ); + doAnswer(invocationOnMock -> { + invocationOnMock.getArgument(4, ActionListener.class).onResponse(null); + return null; + }).when(blobStoreTransferService) + .uploadBlob(any(InputStream.class), anyIterable(), anyString(), eq(URGENT), any(ActionListener.class)); + TestCapturingListener listener = new TestCapturingListener<>(); + CountDownLatch latch = new CountDownLatch(1); + + remoteGlobalMetadataManager.getAsyncMetadataWriteAction(remoteCoordinationMetadata, new LatchedActionListener<>(listener, latch)) + .run(); + latch.await(); + assertNull(listener.getFailure()); + assertNotNull(listener.getResult()); + ClusterMetadataManifest.UploadedMetadata uploadedMetadata = listener.getResult(); + assertEquals(COORDINATION_METADATA, uploadedMetadata.getComponent()); + String uploadedFileName = uploadedMetadata.getUploadedFilename(); + String[] pathTokens = uploadedFileName.split(PATH_DELIMITER); + assertEquals(5, pathTokens.length); + assertEquals(RemoteClusterStateUtils.encodeString(CLUSTER_NAME), pathTokens[0]); + assertEquals(CLUSTER_STATE_PATH_TOKEN, pathTokens[1]); + assertEquals(CLUSTER_UUID, pathTokens[2]); + assertEquals(GLOBAL_METADATA_PATH_TOKEN, pathTokens[3]); + String[] splitFileName = pathTokens[4].split(DELIMITER); + assertEquals(4, splitFileName.length); + assertEquals(COORDINATION_METADATA, splitFileName[0]); + assertEquals(RemoteStoreUtils.invertLong(METADATA_VERSION), splitFileName[1]); + assertEquals(GLOBAL_METADATA_CURRENT_CODEC_VERSION, Integer.parseInt(splitFileName[3])); + } + + public void testGetReadMetadataAsyncAction_PersistentSettings() throws Exception { + Settings settingsMetadata = getSettings(); + String fileName = randomAlphaOfLength(10); + RemotePersistentSettingsMetadata persistentSettings = new RemotePersistentSettingsMetadata( + fileName, + CLUSTER_UUID, + compressor, + xContentRegistry + ); + when(blobStoreTransferService.downloadBlob(anyIterable(), anyString())).thenReturn( + RemotePersistentSettingsMetadata.SETTINGS_METADATA_FORMAT.serialize(settingsMetadata, fileName, compressor, FORMAT_PARAMS) + .streamInput() + ); + TestCapturingListener listener = new TestCapturingListener<>(); + CountDownLatch latch = new CountDownLatch(1); + + remoteGlobalMetadataManager.getAsyncMetadataReadAction( + persistentSettings, + SETTING_METADATA, + new LatchedActionListener<>(listener, latch) + ).run(); + latch.await(); + assertNull(listener.getFailure()); + assertNotNull(listener.getResult()); + assertEquals(settingsMetadata, listener.getResult().getObj()); + assertEquals(SETTING_METADATA, listener.getResult().getComponent()); + assertEquals(SETTING_METADATA, listener.getResult().getComponentName()); + } + + public void testGetAsyncMetadataWriteAction_PersistentSettings() throws Exception { + Settings settingsMetadata = getSettings(); + RemotePersistentSettingsMetadata persistentSettings = new RemotePersistentSettingsMetadata( + settingsMetadata, + METADATA_VERSION, + CLUSTER_UUID, + compressor, + xContentRegistry + ); + doAnswer(invocationOnMock -> { + invocationOnMock.getArgument(4, ActionListener.class).onResponse(null); + return null; + }).when(blobStoreTransferService) + .uploadBlob(any(InputStream.class), anyIterable(), anyString(), eq(URGENT), any(ActionListener.class)); + TestCapturingListener listener = new TestCapturingListener<>(); + CountDownLatch latch = new CountDownLatch(1); + remoteGlobalMetadataManager.getAsyncMetadataWriteAction(persistentSettings, new LatchedActionListener<>(listener, latch)).run(); + + latch.await(); + assertNull(listener.getFailure()); + assertNotNull(listener.getResult()); + ClusterMetadataManifest.UploadedMetadata uploadedMetadata = listener.getResult(); + assertEquals(SETTING_METADATA, uploadedMetadata.getComponent()); + String uploadedFileName = uploadedMetadata.getUploadedFilename(); + String[] pathTokens = uploadedFileName.split(PATH_DELIMITER); + assertEquals(5, pathTokens.length); + assertEquals(RemoteClusterStateUtils.encodeString(CLUSTER_NAME), pathTokens[0]); + assertEquals(CLUSTER_STATE_PATH_TOKEN, pathTokens[1]); + assertEquals(CLUSTER_UUID, pathTokens[2]); + assertEquals(GLOBAL_METADATA_PATH_TOKEN, pathTokens[3]); + String[] splitFileName = pathTokens[4].split(DELIMITER); + assertEquals(4, splitFileName.length); + assertEquals(SETTING_METADATA, splitFileName[0]); + assertEquals(RemoteStoreUtils.invertLong(METADATA_VERSION), splitFileName[1]); + assertEquals(GLOBAL_METADATA_CURRENT_CODEC_VERSION, Integer.parseInt(splitFileName[3])); + } + + public void testGetReadMetadataAsyncAction_TransientSettings() throws Exception { + Settings settingsMetadata = getSettings(); + String fileName = randomAlphaOfLength(10); + RemoteTransientSettingsMetadata transientSettings = new RemoteTransientSettingsMetadata( + fileName, + CLUSTER_UUID, + compressor, + xContentRegistry + ); + when(blobStoreTransferService.downloadBlob(anyIterable(), anyString())).thenReturn( + RemoteTransientSettingsMetadata.SETTINGS_METADATA_FORMAT.serialize(settingsMetadata, fileName, compressor, FORMAT_PARAMS) + .streamInput() + ); + TestCapturingListener listener = new TestCapturingListener<>(); + CountDownLatch latch = new CountDownLatch(1); + + remoteGlobalMetadataManager.getAsyncMetadataReadAction( + transientSettings, + TRANSIENT_SETTING_METADATA, + new LatchedActionListener<>(listener, latch) + ).run(); + latch.await(); + assertNull(listener.getFailure()); + assertNotNull(listener.getResult()); + assertEquals(settingsMetadata, listener.getResult().getObj()); + assertEquals(TRANSIENT_SETTING_METADATA, listener.getResult().getComponent()); + assertEquals(TRANSIENT_SETTING_METADATA, listener.getResult().getComponentName()); + } + + public void testGetAsyncMetadataWriteAction_TransientSettings() throws Exception { + Settings settingsMetadata = getSettings(); + RemoteTransientSettingsMetadata transientSettings = new RemoteTransientSettingsMetadata( + settingsMetadata, + METADATA_VERSION, + CLUSTER_UUID, + compressor, + xContentRegistry + ); + doAnswer(invocationOnMock -> { + invocationOnMock.getArgument(4, ActionListener.class).onResponse(null); + return null; + }).when(blobStoreTransferService) + .uploadBlob(any(InputStream.class), anyIterable(), anyString(), eq(URGENT), any(ActionListener.class)); + TestCapturingListener listener = new TestCapturingListener<>(); + CountDownLatch latch = new CountDownLatch(1); + remoteGlobalMetadataManager.getAsyncMetadataWriteAction(transientSettings, new LatchedActionListener<>(listener, latch)).run(); + latch.await(); + assertNull(listener.getFailure()); + assertNotNull(listener.getResult()); + ClusterMetadataManifest.UploadedMetadata uploadedMetadata = listener.getResult(); + assertEquals(TRANSIENT_SETTING_METADATA, uploadedMetadata.getComponent()); + String uploadedFileName = uploadedMetadata.getUploadedFilename(); + String[] pathTokens = uploadedFileName.split(PATH_DELIMITER); + assertEquals(5, pathTokens.length); + assertEquals(RemoteClusterStateUtils.encodeString(CLUSTER_NAME), pathTokens[0]); + assertEquals(CLUSTER_STATE_PATH_TOKEN, pathTokens[1]); + assertEquals(CLUSTER_UUID, pathTokens[2]); + assertEquals(GLOBAL_METADATA_PATH_TOKEN, pathTokens[3]); + String[] splitFileName = pathTokens[4].split(DELIMITER); + assertEquals(4, splitFileName.length); + assertEquals(TRANSIENT_SETTING_METADATA, splitFileName[0]); + assertEquals(RemoteStoreUtils.invertLong(METADATA_VERSION), splitFileName[1]); + assertEquals(GLOBAL_METADATA_CURRENT_CODEC_VERSION, Integer.parseInt(splitFileName[3])); + } + + public void testGetReadMetadataAsyncAction_HashesOfConsistentSettings() throws Exception { + DiffableStringMap hashesOfConsistentSettings = getHashesOfConsistentSettings(); + String fileName = randomAlphaOfLength(10); + RemoteHashesOfConsistentSettings hashesOfConsistentSettingsForDownload = new RemoteHashesOfConsistentSettings( + fileName, + CLUSTER_UUID, + compressor + ); + when(blobStoreTransferService.downloadBlob(anyIterable(), anyString())).thenReturn( + HASHES_OF_CONSISTENT_SETTINGS_FORMAT.serialize(hashesOfConsistentSettings, fileName, compressor).streamInput() + ); + TestCapturingListener listener = new TestCapturingListener<>(); + CountDownLatch latch = new CountDownLatch(1); + + remoteGlobalMetadataManager.getAsyncMetadataReadAction( + hashesOfConsistentSettingsForDownload, + HASHES_OF_CONSISTENT_SETTINGS, + new LatchedActionListener<>(listener, latch) + ).run(); + latch.await(); + assertNull(listener.getFailure()); + assertNotNull(listener.getResult()); + assertEquals(hashesOfConsistentSettings, listener.getResult().getObj()); + assertEquals(HASHES_OF_CONSISTENT_SETTINGS, listener.getResult().getComponent()); + assertEquals(HASHES_OF_CONSISTENT_SETTINGS, listener.getResult().getComponentName()); + } + + public void testGetAsyncMetadataWriteAction_HashesOfConsistentSettings() throws Exception { + DiffableStringMap hashesOfConsistentSettings = getHashesOfConsistentSettings(); + RemoteHashesOfConsistentSettings hashesOfConsistentSettingsForUpload = new RemoteHashesOfConsistentSettings( + hashesOfConsistentSettings, + METADATA_VERSION, + CLUSTER_UUID, + compressor + ); + doAnswer(invocationOnMock -> { + invocationOnMock.getArgument(4, ActionListener.class).onResponse(null); + return null; + }).when(blobStoreTransferService) + .uploadBlob(any(InputStream.class), anyIterable(), anyString(), eq(URGENT), any(ActionListener.class)); + TestCapturingListener listener = new TestCapturingListener<>(); + CountDownLatch latch = new CountDownLatch(1); + remoteGlobalMetadataManager.getAsyncMetadataWriteAction( + hashesOfConsistentSettingsForUpload, + new LatchedActionListener<>(listener, latch) + ).run(); + latch.await(); + assertNull(listener.getFailure()); + assertNotNull(listener.getResult()); + ClusterMetadataManifest.UploadedMetadata uploadedMetadata = listener.getResult(); + assertEquals(HASHES_OF_CONSISTENT_SETTINGS, uploadedMetadata.getComponent()); + String uploadedFileName = uploadedMetadata.getUploadedFilename(); + String[] pathTokens = uploadedFileName.split(PATH_DELIMITER); + assertEquals(5, pathTokens.length); + assertEquals(RemoteClusterStateUtils.encodeString(CLUSTER_NAME), pathTokens[0]); + assertEquals(CLUSTER_STATE_PATH_TOKEN, pathTokens[1]); + assertEquals(CLUSTER_UUID, pathTokens[2]); + assertEquals(GLOBAL_METADATA_PATH_TOKEN, pathTokens[3]); + String[] splitFileName = pathTokens[4].split(DELIMITER); + assertEquals(4, splitFileName.length); + assertEquals(HASHES_OF_CONSISTENT_SETTINGS, splitFileName[0]); + assertEquals(RemoteStoreUtils.invertLong(METADATA_VERSION), splitFileName[1]); + assertEquals(GLOBAL_METADATA_CURRENT_CODEC_VERSION, Integer.parseInt(splitFileName[3])); + } + + public void testGetReadMetadataAsyncAction_TemplatesMetadata() throws Exception { + TemplatesMetadata templatesMetadata = getTemplatesMetadata(); + String fileName = randomAlphaOfLength(10); + RemoteTemplatesMetadata templatesMetadataForDownload = new RemoteTemplatesMetadata( + fileName, + CLUSTER_UUID, + compressor, + xContentRegistry + ); + when(blobStoreTransferService.downloadBlob(anyIterable(), anyString())).thenReturn( + TEMPLATES_METADATA_FORMAT.serialize(templatesMetadata, fileName, compressor, FORMAT_PARAMS).streamInput() + ); + TestCapturingListener listener = new TestCapturingListener<>(); + CountDownLatch latch = new CountDownLatch(1); + remoteGlobalMetadataManager.getAsyncMetadataReadAction( + templatesMetadataForDownload, + TEMPLATES_METADATA, + new LatchedActionListener<>(listener, latch) + ).run(); + latch.await(); + assertNull(listener.getFailure()); + assertNotNull(listener.getResult()); + assertEquals(templatesMetadata, listener.getResult().getObj()); + assertEquals(TEMPLATES_METADATA, listener.getResult().getComponent()); + assertEquals(TEMPLATES_METADATA, listener.getResult().getComponentName()); + } + + public void testGetAsyncMetadataWriteAction_TemplatesMetadata() throws Exception { + TemplatesMetadata templatesMetadata = getTemplatesMetadata(); + RemoteTemplatesMetadata templateMetadataForUpload = new RemoteTemplatesMetadata( + templatesMetadata, + METADATA_VERSION, + CLUSTER_UUID, + compressor, + xContentRegistry + ); + doAnswer(invocationOnMock -> { + invocationOnMock.getArgument(4, ActionListener.class).onResponse(null); + return null; + }).when(blobStoreTransferService) + .uploadBlob(any(InputStream.class), anyIterable(), anyString(), eq(URGENT), any(ActionListener.class)); + TestCapturingListener listener = new TestCapturingListener<>(); + CountDownLatch latch = new CountDownLatch(1); + remoteGlobalMetadataManager.getAsyncMetadataWriteAction(templateMetadataForUpload, new LatchedActionListener<>(listener, latch)) + .run(); + latch.await(); + assertNull(listener.getFailure()); + assertNotNull(listener.getResult()); + ClusterMetadataManifest.UploadedMetadata uploadedMetadata = listener.getResult(); + assertEquals(TEMPLATES_METADATA, uploadedMetadata.getComponent()); + String uploadedFileName = uploadedMetadata.getUploadedFilename(); + String[] pathTokens = uploadedFileName.split(PATH_DELIMITER); + assertEquals(5, pathTokens.length); + assertEquals(RemoteClusterStateUtils.encodeString(CLUSTER_NAME), pathTokens[0]); + assertEquals(CLUSTER_STATE_PATH_TOKEN, pathTokens[1]); + assertEquals(CLUSTER_UUID, pathTokens[2]); + assertEquals(GLOBAL_METADATA_PATH_TOKEN, pathTokens[3]); + String[] splitFileName = pathTokens[4].split(DELIMITER); + assertEquals(4, splitFileName.length); + assertEquals(TEMPLATES_METADATA, splitFileName[0]); + assertEquals(RemoteStoreUtils.invertLong(METADATA_VERSION), splitFileName[1]); + assertEquals(GLOBAL_METADATA_CURRENT_CODEC_VERSION, Integer.parseInt(splitFileName[3])); + } + + public void testGetReadMetadataAsyncAction_CustomMetadata() throws Exception { + Metadata.Custom customMetadata = getCustomMetadata(); + String fileName = randomAlphaOfLength(10); + RemoteCustomMetadata customMetadataForDownload = new RemoteCustomMetadata( + fileName, + IndexGraveyard.TYPE, + CLUSTER_UUID, + compressor, + namedWriteableRegistry + ); + when(blobStoreTransferService.downloadBlob(anyIterable(), anyString())).thenReturn( + customMetadataForDownload.customBlobStoreFormat.serialize(customMetadata, fileName, compressor).streamInput() + ); + TestCapturingListener listener = new TestCapturingListener<>(); + CountDownLatch latch = new CountDownLatch(1); + remoteGlobalMetadataManager.getAsyncMetadataReadAction( + customMetadataForDownload, + IndexGraveyard.TYPE, + new LatchedActionListener<>(listener, latch) + ).run(); + latch.await(); + assertNull(listener.getFailure()); + assertNotNull(listener.getResult()); + assertEquals(customMetadata, listener.getResult().getObj()); + assertEquals(CUSTOM_METADATA, listener.getResult().getComponent()); + assertEquals(IndexGraveyard.TYPE, listener.getResult().getComponentName()); + } + + public void testGetAsyncMetadataWriteAction_CustomMetadata() throws Exception { + Metadata.Custom customMetadata = getCustomMetadata(); + RemoteCustomMetadata customMetadataForUpload = new RemoteCustomMetadata( + customMetadata, + IndexGraveyard.TYPE, + METADATA_VERSION, + CLUSTER_UUID, + compressor, + namedWriteableRegistry + ); + doAnswer(invocationOnMock -> { + invocationOnMock.getArgument(4, ActionListener.class).onResponse(null); + return null; + }).when(blobStoreTransferService) + .uploadBlob(any(InputStream.class), anyIterable(), anyString(), eq(URGENT), any(ActionListener.class)); + TestCapturingListener listener = new TestCapturingListener<>(); + CountDownLatch latch = new CountDownLatch(1); + remoteGlobalMetadataManager.getAsyncMetadataWriteAction(customMetadataForUpload, new LatchedActionListener<>(listener, latch)) + .run(); + latch.await(); + assertNull(listener.getFailure()); + assertNotNull(listener.getResult()); + ClusterMetadataManifest.UploadedMetadata uploadedMetadata = listener.getResult(); + assertEquals(String.join(CUSTOM_DELIMITER, CUSTOM_METADATA, IndexGraveyard.TYPE), uploadedMetadata.getComponent()); + String uploadedFileName = uploadedMetadata.getUploadedFilename(); + String[] pathTokens = uploadedFileName.split(PATH_DELIMITER); + assertEquals(5, pathTokens.length); + assertEquals(RemoteClusterStateUtils.encodeString(CLUSTER_NAME), pathTokens[0]); + assertEquals(CLUSTER_STATE_PATH_TOKEN, pathTokens[1]); + assertEquals(CLUSTER_UUID, pathTokens[2]); + assertEquals(GLOBAL_METADATA_PATH_TOKEN, pathTokens[3]); + String[] splitFileName = pathTokens[4].split(DELIMITER); + assertEquals(4, splitFileName.length); + assertEquals(String.join(CUSTOM_DELIMITER, CUSTOM_METADATA, IndexGraveyard.TYPE), splitFileName[0]); + assertEquals(RemoteStoreUtils.invertLong(METADATA_VERSION), splitFileName[1]); + assertEquals(GLOBAL_METADATA_CURRENT_CODEC_VERSION, Integer.parseInt(splitFileName[3])); + } + + public void testGetReadMetadataAsyncAction_GlobalMetadata() throws Exception { + Metadata metadata = getGlobalMetadata(); + String fileName = randomAlphaOfLength(10); + RemoteGlobalMetadata globalMetadataForDownload = new RemoteGlobalMetadata(fileName, CLUSTER_UUID, compressor, xContentRegistry); + when(blobStoreTransferService.downloadBlob(anyIterable(), anyString())).thenReturn( + GLOBAL_METADATA_FORMAT.serialize(metadata, fileName, compressor, FORMAT_PARAMS).streamInput() + ); + TestCapturingListener listener = new TestCapturingListener<>(); + CountDownLatch latch = new CountDownLatch(1); + remoteGlobalMetadataManager.getAsyncMetadataReadAction( + globalMetadataForDownload, + GLOBAL_METADATA, + new LatchedActionListener<>(listener, latch) + ).run(); + latch.await(); + assertNull(listener.getFailure()); + assertNotNull(listener.getResult()); + assertTrue(isGlobalStateEquals(metadata, (Metadata) listener.getResult().getObj())); + assertEquals(GLOBAL_METADATA, listener.getResult().getComponent()); + assertEquals(GLOBAL_METADATA, listener.getResult().getComponentName()); + } + + public void testGetReadMetadataAsyncAction_IOException() throws Exception { + String fileName = randomAlphaOfLength(10); + RemoteCoordinationMetadata coordinationMetadataForDownload = new RemoteCoordinationMetadata( + fileName, + CLUSTER_UUID, + compressor, + xContentRegistry + ); + IOException ioException = new IOException("mock test exception"); + when(blobStoreTransferService.downloadBlob(anyIterable(), anyString())).thenThrow(ioException); + TestCapturingListener listener = new TestCapturingListener<>(); + CountDownLatch latch = new CountDownLatch(1); + remoteGlobalMetadataManager.getAsyncMetadataReadAction( + coordinationMetadataForDownload, + COORDINATION_METADATA, + new LatchedActionListener<>(listener, latch) + ).run(); + latch.await(); + assertNull(listener.getResult()); + assertNotNull(listener.getFailure()); + assertEquals(ioException, listener.getFailure()); + } + + public void testGetAsyncMetadataWriteAction_IOException() throws Exception { + CoordinationMetadata coordinationMetadata = getCoordinationMetadata(); + RemoteCoordinationMetadata remoteCoordinationMetadata = new RemoteCoordinationMetadata( + coordinationMetadata, + METADATA_VERSION, + CLUSTER_UUID, + compressor, + xContentRegistry + ); + IOException ioException = new IOException("mock test exception"); + doAnswer(invocationOnMock -> { + invocationOnMock.getArgument(4, ActionListener.class).onFailure(ioException); + return null; + }).when(blobStoreTransferService) + .uploadBlob(any(InputStream.class), anyIterable(), anyString(), eq(URGENT), any(ActionListener.class)); + + TestCapturingListener listener = new TestCapturingListener<>(); + CountDownLatch latch = new CountDownLatch(1); + remoteGlobalMetadataManager.getAsyncMetadataWriteAction(remoteCoordinationMetadata, new LatchedActionListener<>(listener, latch)) + .run(); + assertNull(listener.getResult()); + assertNotNull(listener.getFailure()); + assertTrue(listener.getFailure() instanceof RemoteStateTransferException); + assertEquals(ioException, listener.getFailure().getCause()); + } + public void testGetUpdatedCustoms() { Map previousCustoms = Map.of( CustomMetadata1.TYPE, diff --git a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteCoordinationMetadataTests.java b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteCoordinationMetadataTests.java index 9484afe6b7d6c..63d6de05a737c 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteCoordinationMetadataTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteCoordinationMetadataTests.java @@ -229,7 +229,7 @@ public void testSerDe() throws IOException { } } - private CoordinationMetadata getCoordinationMetadata() { + public static CoordinationMetadata getCoordinationMetadata() { return CoordinationMetadata.builder() .term(TERM) .lastAcceptedConfiguration(new VotingConfiguration(Set.of("node1"))) diff --git a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteCustomMetadataTests.java b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteCustomMetadataTests.java index 1bce176273270..1e28817be79f2 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteCustomMetadataTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteCustomMetadataTests.java @@ -232,7 +232,7 @@ public void testSerDe() throws IOException { } } - private Custom getCustomMetadata() { + public static Custom getCustomMetadata() { return IndexGraveyard.builder().addTombstone(new Index("test-index", "3q2423")).build(); } diff --git a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteGlobalMetadataTests.java b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteGlobalMetadataTests.java index 02ddc8ba93071..23de485357547 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteGlobalMetadataTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteGlobalMetadataTests.java @@ -180,7 +180,7 @@ public void testSerDe() throws IOException { } } - private Metadata getGlobalMetadata() { + public static Metadata getGlobalMetadata() { return Metadata.builder() .templates( TemplatesMetadata.builder() diff --git a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteHashesOfConsistentSettingsTests.java b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteHashesOfConsistentSettingsTests.java index d883eabf9fbc9..b931f24f98631 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteHashesOfConsistentSettingsTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteHashesOfConsistentSettingsTests.java @@ -185,7 +185,7 @@ public void testSerDe() throws IOException { } } - private DiffableStringMap getHashesOfConsistentSettings() { + public static DiffableStringMap getHashesOfConsistentSettings() { Map hashesOfConsistentSettings = new HashMap<>(); hashesOfConsistentSettings.put("secure-setting-key", "secure-setting-value"); return new DiffableStringMap(hashesOfConsistentSettings); diff --git a/server/src/test/java/org/opensearch/gateway/remote/model/RemotePersistentSettingsMetadataTests.java b/server/src/test/java/org/opensearch/gateway/remote/model/RemotePersistentSettingsMetadataTests.java index 850c18f03fa49..5e4d5d66ca1b7 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/model/RemotePersistentSettingsMetadataTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/model/RemotePersistentSettingsMetadataTests.java @@ -224,7 +224,7 @@ public void testSerDe() throws IOException { } } - private Settings getSettings() { + public static Settings getSettings() { return Settings.builder().put("random_index_setting_" + randomAlphaOfLength(3), randomAlphaOfLength(5)).build(); } } diff --git a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteTemplatesMetadataTests.java b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteTemplatesMetadataTests.java index b86044003aa55..d7ecd2ad3f44a 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/model/RemoteTemplatesMetadataTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/model/RemoteTemplatesMetadataTests.java @@ -227,7 +227,7 @@ public void testSerDe() throws IOException { } } - private TemplatesMetadata getTemplatesMetadata() { + public static TemplatesMetadata getTemplatesMetadata() { return TemplatesMetadata.builder() .put( IndexTemplateMetadata.builder("template" + randomAlphaOfLength(3)) diff --git a/test/framework/src/main/java/org/opensearch/common/util/TestCapturingListener.java b/test/framework/src/main/java/org/opensearch/common/util/TestCapturingListener.java new file mode 100644 index 0000000000000..a3c8cc15de927 --- /dev/null +++ b/test/framework/src/main/java/org/opensearch/common/util/TestCapturingListener.java @@ -0,0 +1,39 @@ +/* + * 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.util; + +import org.opensearch.core.action.ActionListener; + +/** + * A simple implementation of {@link ActionListener} that captures the response and failures used for testing purposes. + * + * @param the result type + */ +public class TestCapturingListener implements ActionListener { + private T result; + private Exception failure; + + @Override + public void onResponse(T result) { + this.result = result; + } + + @Override + public void onFailure(Exception e) { + this.failure = e; + } + + public T getResult() { + return result; + } + + public Exception getFailure() { + return failure; + } +} From a8e40448dca97428ec9241a85289c581c33e1e4c Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Wed, 19 Jun 2024 16:20:26 -0400 Subject: [PATCH 39/84] Bump OpenTelemetry to 1.39.0 (#14457) Signed-off-by: Andriy Redko --- CHANGELOG.md | 1 + buildSrc/version.properties | 4 ++-- plugins/telemetry-otel/build.gradle | 10 ++++------ .../licenses/opentelemetry-api-1.36.0.jar.sha1 | 1 - .../licenses/opentelemetry-api-1.39.0.jar.sha1 | 1 + .../opentelemetry-api-incubator-1.39.0-alpha.jar.sha1 | 1 + ...NSE.txt => opentelemetry-api-incubator-LICENSE.txt} | 0 ...TICE.txt => opentelemetry-api-incubator-NOTICE.txt} | 0 .../licenses/opentelemetry-context-1.36.0.jar.sha1 | 1 - .../licenses/opentelemetry-context-1.39.0.jar.sha1 | 1 + .../opentelemetry-exporter-common-1.36.0.jar.sha1 | 1 - .../opentelemetry-exporter-common-1.39.0.jar.sha1 | 1 + .../opentelemetry-exporter-logging-1.36.0.jar.sha1 | 1 - .../opentelemetry-exporter-logging-1.39.0.jar.sha1 | 1 + .../opentelemetry-exporter-otlp-1.36.0.jar.sha1 | 1 - .../opentelemetry-exporter-otlp-1.39.0.jar.sha1 | 1 + .../opentelemetry-exporter-otlp-common-1.36.0.jar.sha1 | 1 - .../opentelemetry-exporter-otlp-common-1.39.0.jar.sha1 | 1 + ...pentelemetry-exporter-sender-okhttp-1.36.0.jar.sha1 | 1 - ...pentelemetry-exporter-sender-okhttp-1.39.0.jar.sha1 | 1 + ...telemetry-extension-incubator-1.36.0-alpha.jar.sha1 | 1 - .../licenses/opentelemetry-sdk-1.36.0.jar.sha1 | 1 - .../licenses/opentelemetry-sdk-1.39.0.jar.sha1 | 1 + .../licenses/opentelemetry-sdk-common-1.36.0.jar.sha1 | 1 - .../licenses/opentelemetry-sdk-common-1.39.0.jar.sha1 | 1 + .../licenses/opentelemetry-sdk-logs-1.36.0.jar.sha1 | 1 - .../licenses/opentelemetry-sdk-logs-1.39.0.jar.sha1 | 1 + .../licenses/opentelemetry-sdk-metrics-1.36.0.jar.sha1 | 1 - .../licenses/opentelemetry-sdk-metrics-1.39.0.jar.sha1 | 1 + .../licenses/opentelemetry-sdk-trace-1.36.0.jar.sha1 | 1 - .../licenses/opentelemetry-sdk-trace-1.39.0.jar.sha1 | 1 + .../opentelemetry-semconv-1.23.1-alpha.jar.sha1 | 1 - .../opentelemetry-semconv-1.25.0-alpha.jar.sha1 | 1 + 33 files changed, 21 insertions(+), 22 deletions(-) delete mode 100644 plugins/telemetry-otel/licenses/opentelemetry-api-1.36.0.jar.sha1 create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-api-1.39.0.jar.sha1 create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-api-incubator-1.39.0-alpha.jar.sha1 rename plugins/telemetry-otel/licenses/{opentelemetry-extension-incubator-LICENSE.txt => opentelemetry-api-incubator-LICENSE.txt} (100%) rename plugins/telemetry-otel/licenses/{opentelemetry-extension-incubator-NOTICE.txt => opentelemetry-api-incubator-NOTICE.txt} (100%) delete mode 100644 plugins/telemetry-otel/licenses/opentelemetry-context-1.36.0.jar.sha1 create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-context-1.39.0.jar.sha1 delete mode 100644 plugins/telemetry-otel/licenses/opentelemetry-exporter-common-1.36.0.jar.sha1 create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-exporter-common-1.39.0.jar.sha1 delete mode 100644 plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-1.36.0.jar.sha1 create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-1.39.0.jar.sha1 delete mode 100644 plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-1.36.0.jar.sha1 create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-1.39.0.jar.sha1 delete mode 100644 plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-common-1.36.0.jar.sha1 create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-common-1.39.0.jar.sha1 delete mode 100644 plugins/telemetry-otel/licenses/opentelemetry-exporter-sender-okhttp-1.36.0.jar.sha1 create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-exporter-sender-okhttp-1.39.0.jar.sha1 delete mode 100644 plugins/telemetry-otel/licenses/opentelemetry-extension-incubator-1.36.0-alpha.jar.sha1 delete mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-1.36.0.jar.sha1 create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-1.39.0.jar.sha1 delete mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-common-1.36.0.jar.sha1 create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-common-1.39.0.jar.sha1 delete mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-1.36.0.jar.sha1 create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-1.39.0.jar.sha1 delete mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-1.36.0.jar.sha1 create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-1.39.0.jar.sha1 delete mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-1.36.0.jar.sha1 create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-1.39.0.jar.sha1 delete mode 100644 plugins/telemetry-otel/licenses/opentelemetry-semconv-1.23.1-alpha.jar.sha1 create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-semconv-1.25.0-alpha.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index fae8686d1e45d..24a4f6fd1b1f1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -19,6 +19,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `com.nimbusds:nimbus-jose-jwt` from 9.37.3 to 9.40 ([#14398](https://github.com/opensearch-project/OpenSearch/pull/14398)) - Bump `org.apache.commons:commons-configuration2` from 2.10.1 to 2.11.0 ([#14399](https://github.com/opensearch-project/OpenSearch/pull/14399)) - Bump `com.gradle.develocity` from 3.17.4 to 3.17.5 ([#14397](https://github.com/opensearch-project/OpenSearch/pull/14397)) +- Bump `opentelemetry` from 1.36.0 to 1.39.0 ([#14457](https://github.com/opensearch-project/OpenSearch/pull/14457)) ### Changed - Updated the `indices.query.bool.max_clause_count` setting from being static to dynamically updateable ([#13568](https://github.com/opensearch-project/OpenSearch/pull/13568)) diff --git a/buildSrc/version.properties b/buildSrc/version.properties index af421b97e12e4..e9aa32ea9a4f5 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -74,5 +74,5 @@ jzlib = 1.1.3 resteasy = 6.2.4.Final # opentelemetry dependencies -opentelemetry = 1.36.0 -opentelemetrysemconv = 1.23.1-alpha +opentelemetry = 1.39.0 +opentelemetrysemconv = 1.25.0-alpha diff --git a/plugins/telemetry-otel/build.gradle b/plugins/telemetry-otel/build.gradle index 735cbd92b691a..66d172e3dc7f3 100644 --- a/plugins/telemetry-otel/build.gradle +++ b/plugins/telemetry-otel/build.gradle @@ -37,7 +37,7 @@ dependencies { runtimeOnly "com.squareup.okhttp3:okhttp:4.11.0" runtimeOnly "com.squareup.okio:okio-jvm:3.5.0" runtimeOnly "io.opentelemetry:opentelemetry-exporter-sender-okhttp:${versions.opentelemetry}" - api "io.opentelemetry:opentelemetry-extension-incubator:${versions.opentelemetry}-alpha" + api "io.opentelemetry:opentelemetry-api-incubator:${versions.opentelemetry}-alpha" testImplementation "io.opentelemetry:opentelemetry-sdk-testing:${versions.opentelemetry}" } @@ -48,7 +48,9 @@ thirdPartyAudit { 'io.opentelemetry.internal.shaded.jctools.queues.MpscArrayQueueProducerIndexField', 'io.opentelemetry.internal.shaded.jctools.queues.MpscArrayQueueProducerLimitField', 'io.opentelemetry.internal.shaded.jctools.util.UnsafeAccess', - 'io.opentelemetry.internal.shaded.jctools.util.UnsafeRefArrayAccess' + 'io.opentelemetry.internal.shaded.jctools.util.UnsafeRefArrayAccess', + 'io.opentelemetry.exporter.internal.marshal.UnsafeAccess', + 'io.opentelemetry.exporter.internal.marshal.UnsafeAccess$UnsafeHolder' ) ignoreMissingClasses( @@ -78,10 +80,6 @@ thirdPartyAudit { 'org.conscrypt.ConscryptHostnameVerifier', 'org.openjsse.javax.net.ssl.SSLParameters', 'org.openjsse.javax.net.ssl.SSLSocket', - 'io.opentelemetry.api.events.EventBuilder', - 'io.opentelemetry.api.events.EventEmitter', - 'io.opentelemetry.api.events.EventEmitterBuilder', - 'io.opentelemetry.api.events.EventEmitterProvider', 'io.opentelemetry.sdk.autoconfigure.spi.ConfigProperties', 'io.opentelemetry.sdk.autoconfigure.spi.logs.ConfigurableLogRecordExporterProvider', 'io.opentelemetry.sdk.autoconfigure.spi.metrics.ConfigurableMetricExporterProvider', diff --git a/plugins/telemetry-otel/licenses/opentelemetry-api-1.36.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-api-1.36.0.jar.sha1 deleted file mode 100644 index b577500d71e1d..0000000000000 --- a/plugins/telemetry-otel/licenses/opentelemetry-api-1.36.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -59470f4aa3a9207f21936461b8fdcb36d46455ab \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-api-1.39.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-api-1.39.0.jar.sha1 new file mode 100644 index 0000000000000..415fe8f3d8aaa --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-api-1.39.0.jar.sha1 @@ -0,0 +1 @@ +396b89a66526bd5694ad3bef4604b876177e0b44 \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-api-incubator-1.39.0-alpha.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-api-incubator-1.39.0-alpha.jar.sha1 new file mode 100644 index 0000000000000..9c3c9f43d153c --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-api-incubator-1.39.0-alpha.jar.sha1 @@ -0,0 +1 @@ +1a1fd96155e1b58726300bbf8457630713035e51 \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-extension-incubator-LICENSE.txt b/plugins/telemetry-otel/licenses/opentelemetry-api-incubator-LICENSE.txt similarity index 100% rename from plugins/telemetry-otel/licenses/opentelemetry-extension-incubator-LICENSE.txt rename to plugins/telemetry-otel/licenses/opentelemetry-api-incubator-LICENSE.txt diff --git a/plugins/telemetry-otel/licenses/opentelemetry-extension-incubator-NOTICE.txt b/plugins/telemetry-otel/licenses/opentelemetry-api-incubator-NOTICE.txt similarity index 100% rename from plugins/telemetry-otel/licenses/opentelemetry-extension-incubator-NOTICE.txt rename to plugins/telemetry-otel/licenses/opentelemetry-api-incubator-NOTICE.txt diff --git a/plugins/telemetry-otel/licenses/opentelemetry-context-1.36.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-context-1.36.0.jar.sha1 deleted file mode 100644 index d3156577248d5..0000000000000 --- a/plugins/telemetry-otel/licenses/opentelemetry-context-1.36.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -8850bc4c65d0fd22ff987b4683206ec4e69f2689 \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-context-1.39.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-context-1.39.0.jar.sha1 new file mode 100644 index 0000000000000..115d4ccb1f34b --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-context-1.39.0.jar.sha1 @@ -0,0 +1 @@ +f0601fb1c06f661afeffbc73a1dbe29797b2f13b \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-exporter-common-1.36.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-exporter-common-1.36.0.jar.sha1 deleted file mode 100644 index f176b21d12dc4..0000000000000 --- a/plugins/telemetry-otel/licenses/opentelemetry-exporter-common-1.36.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -8d1cb823ab18fa871a1549e7c522bf28f2b3d8fe \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-exporter-common-1.39.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-exporter-common-1.39.0.jar.sha1 new file mode 100644 index 0000000000000..a10b92995becd --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-exporter-common-1.39.0.jar.sha1 @@ -0,0 +1 @@ +570d71e39e36fe2caad142557bde0c11fcdb3b92 \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-1.36.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-1.36.0.jar.sha1 deleted file mode 100644 index cd25e0ab9f294..0000000000000 --- a/plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-1.36.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -bc045cae89ff6f18071760f6e4659dd880e88a1b \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-1.39.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-1.39.0.jar.sha1 new file mode 100644 index 0000000000000..f43393104296a --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-1.39.0.jar.sha1 @@ -0,0 +1 @@ +f5b528f8d6f8531836eabba698979516964b24ed \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-1.36.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-1.36.0.jar.sha1 deleted file mode 100644 index fabb394f9c2e0..0000000000000 --- a/plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-1.36.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -5ee49902ba884d6c3e48499a9311a624396d9630 \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-1.39.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-1.39.0.jar.sha1 new file mode 100644 index 0000000000000..5adba2ba0f342 --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-1.39.0.jar.sha1 @@ -0,0 +1 @@ +04fc0e4983253ea58430c3d24b6b3c5c95f84dc9 \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-common-1.36.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-common-1.36.0.jar.sha1 deleted file mode 100644 index 378ba4d43dcd1..0000000000000 --- a/plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-common-1.36.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -2706e3b883d2bcd1a6b3e0bb4118ffbd7820550b \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-common-1.39.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-common-1.39.0.jar.sha1 new file mode 100644 index 0000000000000..ea9c293f25025 --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-exporter-otlp-common-1.39.0.jar.sha1 @@ -0,0 +1 @@ +a2b8571e36b11c3153d31ec87ec69cc168af8036 \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-exporter-sender-okhttp-1.36.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-exporter-sender-okhttp-1.36.0.jar.sha1 deleted file mode 100644 index a3d7e15e1a624..0000000000000 --- a/plugins/telemetry-otel/licenses/opentelemetry-exporter-sender-okhttp-1.36.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -dcc924787b559278697b74dbc5bb6d046b236ef6 \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-exporter-sender-okhttp-1.39.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-exporter-sender-okhttp-1.39.0.jar.sha1 new file mode 100644 index 0000000000000..dcf23f16ac89f --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-exporter-sender-okhttp-1.39.0.jar.sha1 @@ -0,0 +1 @@ +1a8947a2e28924ad9374e319150a23837926ca4b \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-extension-incubator-1.36.0-alpha.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-extension-incubator-1.36.0-alpha.jar.sha1 deleted file mode 100644 index 71ab3e184db9e..0000000000000 --- a/plugins/telemetry-otel/licenses/opentelemetry-extension-incubator-1.36.0-alpha.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -d58f7c669e371f6ff61b705770af9a3c1f31df52 \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-1.36.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-1.36.0.jar.sha1 deleted file mode 100644 index c9a75d1b4350a..0000000000000 --- a/plugins/telemetry-otel/licenses/opentelemetry-sdk-1.36.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -4056d1b562b4da7720817d8af15d1d3ccdf4b776 \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-1.39.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-1.39.0.jar.sha1 new file mode 100644 index 0000000000000..f603af04d8012 --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-sdk-1.39.0.jar.sha1 @@ -0,0 +1 @@ +ba9afdf3ef1ea51e42999fd68c959e3ceb219399 \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-common-1.36.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-common-1.36.0.jar.sha1 deleted file mode 100644 index c31584f59c0d8..0000000000000 --- a/plugins/telemetry-otel/licenses/opentelemetry-sdk-common-1.36.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -11d6f8c7b029efcb5c6c449cadef155b781afb78 \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-common-1.39.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-common-1.39.0.jar.sha1 new file mode 100644 index 0000000000000..f9419f6ccfbee --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-sdk-common-1.39.0.jar.sha1 @@ -0,0 +1 @@ +fb8168627bf0059445f61081eaa47c4ab787fc2e \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-1.36.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-1.36.0.jar.sha1 deleted file mode 100644 index a134bb06ec635..0000000000000 --- a/plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-1.36.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -98e94479db1e68c4779efc44bf6b4fca83e98b54 \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-1.39.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-1.39.0.jar.sha1 new file mode 100644 index 0000000000000..63269f239eacd --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-1.39.0.jar.sha1 @@ -0,0 +1 @@ +b6b45155399bc9fa563945f3e3a77416d7165948 \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-1.36.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-1.36.0.jar.sha1 deleted file mode 100644 index d146241f52f29..0000000000000 --- a/plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-1.36.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -4f8f5d30c3eeede7b2260d979d9f403cfa381c3d \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-1.39.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-1.39.0.jar.sha1 new file mode 100644 index 0000000000000..f18c8259c1adc --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-1.39.0.jar.sha1 @@ -0,0 +1 @@ +522d46926cc06a4c18829da7e4c4340bdf5673c3 \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-1.36.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-1.36.0.jar.sha1 deleted file mode 100644 index 802761e38846c..0000000000000 --- a/plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-1.36.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e3068cbaedfac6a28c6483923982b2efb861d3f4 \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-1.39.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-1.39.0.jar.sha1 new file mode 100644 index 0000000000000..03b81424f46d5 --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-1.39.0.jar.sha1 @@ -0,0 +1 @@ +0b72722a5bbea5f46319bf08b2caed5b8f987a92 \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-semconv-1.23.1-alpha.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-semconv-1.23.1-alpha.jar.sha1 deleted file mode 100644 index e730c83af905e..0000000000000 --- a/plugins/telemetry-otel/licenses/opentelemetry-semconv-1.23.1-alpha.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -218e361772670212a46be5940010222d68e66f2a \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-semconv-1.25.0-alpha.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-semconv-1.25.0-alpha.jar.sha1 new file mode 100644 index 0000000000000..7cf8e7e8ede28 --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-semconv-1.25.0-alpha.jar.sha1 @@ -0,0 +1 @@ +76b3d4ca0a8f20b27c1590ceece54f0c7fb5857e \ No newline at end of file From bd56456baab283570da9b423152125124d482db3 Mon Sep 17 00:00:00 2001 From: Kiran Prakash Date: Wed, 19 Jun 2024 15:36:59 -0700 Subject: [PATCH 40/84] Fix Indices Request Cache ITs from flaking and timing out (#14369) Signed-off-by: Kiran Prakash --- .../indices/IndicesRequestCacheCleanupIT.java | 746 ++++++++++++++++++ .../indices/IndicesRequestCacheIT.java | 631 +-------------- 2 files changed, 781 insertions(+), 596 deletions(-) create mode 100644 server/src/internalClusterTest/java/org/opensearch/indices/IndicesRequestCacheCleanupIT.java diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/IndicesRequestCacheCleanupIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/IndicesRequestCacheCleanupIT.java new file mode 100644 index 0000000000000..6f698c5816278 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/indices/IndicesRequestCacheCleanupIT.java @@ -0,0 +1,746 @@ +/* + * 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. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.indices; + +import org.opensearch.action.admin.cluster.node.stats.NodeStats; +import org.opensearch.action.admin.cluster.node.stats.NodesStatsResponse; +import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; +import org.opensearch.action.admin.indices.cache.clear.ClearIndicesCacheRequest; +import org.opensearch.action.admin.indices.forcemerge.ForceMergeResponse; +import org.opensearch.action.admin.indices.segments.IndicesSegmentsRequest; +import org.opensearch.action.search.SearchResponse; +import org.opensearch.client.Client; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.index.IndexNotFoundException; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.MergePolicyProvider; +import org.opensearch.index.cache.request.RequestCacheStats; +import org.opensearch.index.query.QueryBuilders; +import org.opensearch.plugins.Plugin; +import org.opensearch.test.InternalSettingsPlugin; +import org.opensearch.test.OpenSearchIntegTestCase; +import org.opensearch.test.hamcrest.OpenSearchAssertions; + +import java.util.Arrays; +import java.util.Collection; +import java.util.concurrent.TimeUnit; + +import static org.opensearch.indices.IndicesRequestCache.INDICES_REQUEST_CACHE_STALENESS_THRESHOLD_SETTING; +import static org.opensearch.indices.IndicesService.INDICES_CACHE_CLEANUP_INTERVAL_SETTING_KEY; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertSearchResponse; +import static org.hamcrest.Matchers.equalTo; + +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0, supportsDedicatedMasters = false) +public class IndicesRequestCacheCleanupIT extends OpenSearchIntegTestCase { + + @Override + protected Collection> nodePlugins() { + return Arrays.asList(InternalSettingsPlugin.class); + } + + public void testCacheWithInvalidation() throws Exception { + Client client = client(); + String index = "index"; + 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) + // Disable index refreshing to avoid cache being invalidated mid-test + .put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), TimeValue.timeValueMillis(-1)) + ) + .get() + ); + indexRandom(false, client.prepareIndex(index).setSource("k", "hello")); + ensureSearchable(index); + // Force merge the index to ensure there can be no background merges during the subsequent searches that would invalidate the cache + forceMerge(client, index); + SearchResponse resp = client.prepareSearch(index).setRequestCache(true).setQuery(QueryBuilders.termQuery("k", "hello")).get(); + assertSearchResponse(resp); + OpenSearchAssertions.assertAllSuccessful(resp); + assertThat(resp.getHits().getTotalHits().value, equalTo(1L)); + + assertCacheState(client, index, 0, 1); + // Index but don't refresh + indexRandom(false, client.prepareIndex(index).setSource("k", "hello2")); + resp = client.prepareSearch(index).setRequestCache(true).setQuery(QueryBuilders.termQuery("k", "hello")).get(); + assertSearchResponse(resp); + // Should expect hit as here as refresh didn't happen + assertCacheState(client, index, 1, 1); + + // assert segment counts stay the same + assertEquals(1, getSegmentCount(client, index)); + // Explicit refresh would invalidate cache + refreshAndWaitForReplication(); + // Hit same query again + resp = client.prepareSearch(index).setRequestCache(true).setQuery(QueryBuilders.termQuery("k", "hello")).get(); + assertSearchResponse(resp); + // Should expect miss as key has changed due to change in IndexReader.CacheKey (due to refresh) + assertCacheState(client, index, 1, 2); + } + + // calling cache clear api, when staleness threshold is lower than staleness, it should clean the stale keys from cache + public void testCacheClearAPIRemovesStaleKeysWhenStalenessThresholdIsLow() throws Exception { + String node = internalCluster().startNode( + Settings.builder() + .put(IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_STALENESS_THRESHOLD_SETTING_KEY, 0.10) + .put( + IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_INTERVAL_SETTING_KEY, + // setting intentionally high to avoid cache cleaner interfering + TimeValue.timeValueMillis(300) + ) + ); + Client client = client(node); + String index1 = "index1"; + String index2 = "index2"; + setupIndex(client, index1); + setupIndex(client, index2); + + // create first cache entry in index1 + createCacheEntry(client, index1, "hello"); + assertCacheState(client, index1, 0, 1); + long memorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); + assertTrue(memorySizeForIndex1 > 0); + + // create second cache entry in index1 + createCacheEntry(client, index1, "there"); + assertCacheState(client, index1, 0, 2); + long finalMemorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); + assertTrue(finalMemorySizeForIndex1 > memorySizeForIndex1); + + // create first cache entry in index2 + createCacheEntry(client, index2, "hello"); + assertCacheState(client, index2, 0, 1); + assertTrue(getRequestCacheStats(client, index2).getMemorySizeInBytes() > 0); + + ClearIndicesCacheRequest clearIndicesCacheRequest = new ClearIndicesCacheRequest(index2); + client.admin().indices().clearCache(clearIndicesCacheRequest).actionGet(); + + // assert segment counts stay the same + assertEquals(1, getSegmentCount(client, index1)); + assertEquals(1, getSegmentCount(client, index2)); + // cache cleaner should have cleaned up the stale key from index 2 + assertEquals(0, getRequestCacheStats(client, index2).getMemorySizeInBytes()); + // cache cleaner should NOT have cleaned from index 1 + assertEquals(finalMemorySizeForIndex1, getRequestCacheStats(client, index1).getMemorySizeInBytes()); + } + + // when staleness threshold is lower than staleness, it should clean the stale keys from cache + public void testStaleKeysCleanupWithLowThreshold() throws Exception { + int cacheCleanIntervalInMillis = 1; + String node = internalCluster().startNode( + Settings.builder() + .put(IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_STALENESS_THRESHOLD_SETTING_KEY, 0.10) + .put( + IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_INTERVAL_SETTING_KEY, + TimeValue.timeValueMillis(cacheCleanIntervalInMillis) + ) + ); + Client client = client(node); + String index1 = "index1"; + String index2 = "index2"; + setupIndex(client, index1); + setupIndex(client, index2); + + // create first cache entry in index1 + createCacheEntry(client, index1, "hello"); + assertCacheState(client, index1, 0, 1); + long memorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); + assertTrue(memorySizeForIndex1 > 0); + + // create second cache entry in index1 + createCacheEntry(client, index1, "there"); + assertCacheState(client, index1, 0, 2); + long finalMemorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); + assertTrue(finalMemorySizeForIndex1 > memorySizeForIndex1); + + // create first cache entry in index2 + createCacheEntry(client, index2, "hello"); + assertCacheState(client, index2, 0, 1); + assertTrue(getRequestCacheStats(client, index2).getMemorySizeInBytes() > 0); + + // create 1 stale key + indexRandom(false, client.prepareIndex(index2).setId("1").setSource("d", "hello")); + forceMerge(client, index2); + // sleep until cache cleaner would have cleaned up the stale key from index 2 + assertBusy(() -> { + // assert segment counts stay the same + assertEquals(1, getSegmentCount(client, index1)); + assertEquals(2, getSegmentCount(client, index2)); + // cache cleaner should have cleaned up the stale key from index 2 + assertEquals(0, getRequestCacheStats(client, index2).getMemorySizeInBytes()); + // cache cleaner should NOT have cleaned from index 1 + assertEquals(finalMemorySizeForIndex1, getRequestCacheStats(client, index1).getMemorySizeInBytes()); + }, cacheCleanIntervalInMillis * 2, TimeUnit.MILLISECONDS); + // sleep until cache cleaner would have cleaned up the stale key from index 2 + } + + // when staleness threshold is equal to staleness, it should clean the stale keys from cache + public void testCacheCleanupOnEqualStalenessAndThreshold() throws Exception { + int cacheCleanIntervalInMillis = 1; + String node = internalCluster().startNode( + Settings.builder() + .put(IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_STALENESS_THRESHOLD_SETTING_KEY, 0.33) + .put( + IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_INTERVAL_SETTING_KEY, + TimeValue.timeValueMillis(cacheCleanIntervalInMillis) + ) + ); + Client client = client(node); + String index1 = "index1"; + String index2 = "index2"; + setupIndex(client, index1); + setupIndex(client, index2); + + // create first cache entry in index1 + createCacheEntry(client, index1, "hello"); + assertCacheState(client, index1, 0, 1); + long memorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); + assertTrue(memorySizeForIndex1 > 0); + + // create second cache entry in index1 + createCacheEntry(client, index1, "there"); + assertCacheState(client, index1, 0, 2); + long finalMemorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); + assertTrue(finalMemorySizeForIndex1 > memorySizeForIndex1); + + // create first cache entry in index2 + createCacheEntry(client, index2, "hello"); + assertCacheState(client, index2, 0, 1); + assertTrue(getRequestCacheStats(client, index2).getMemorySizeInBytes() > 0); + + // create 1 stale key + indexRandom(false, client.prepareIndex(index2).setId("1").setSource("d", "hello")); + forceMerge(client, index2); + // sleep until cache cleaner would have cleaned up the stale key from index 2 + assertBusy(() -> { + // assert segment counts stay the same + assertEquals(1, getSegmentCount(client, index1)); + assertEquals(2, getSegmentCount(client, index2)); + // cache cleaner should have cleaned up the stale key from index 2 + assertEquals(0, getRequestCacheStats(client, index2).getMemorySizeInBytes()); + // cache cleaner should NOT have cleaned from index 1 + assertEquals(finalMemorySizeForIndex1, getRequestCacheStats(client, index1).getMemorySizeInBytes()); + }, cacheCleanIntervalInMillis * 2, TimeUnit.MILLISECONDS); + } + + // when staleness threshold is higher than staleness, it should NOT clean the cache + public void testCacheCleanupSkipsWithHighStalenessThreshold() throws Exception { + int cacheCleanIntervalInMillis = 1; + String node = internalCluster().startNode( + Settings.builder() + .put(IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_STALENESS_THRESHOLD_SETTING_KEY, 0.90) + .put( + IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_INTERVAL_SETTING_KEY, + TimeValue.timeValueMillis(cacheCleanIntervalInMillis) + ) + ); + Client client = client(node); + String index1 = "index1"; + String index2 = "index2"; + setupIndex(client, index1); + setupIndex(client, index2); + + // create first cache entry in index1 + createCacheEntry(client, index1, "hello"); + assertCacheState(client, index1, 0, 1); + long memorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); + assertTrue(memorySizeForIndex1 > 0); + + // create second cache entry in index1 + createCacheEntry(client, index1, "there"); + assertCacheState(client, index1, 0, 2); + long finalMemorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); + assertTrue(finalMemorySizeForIndex1 > memorySizeForIndex1); + + // create first cache entry in index2 + createCacheEntry(client, index2, "hello"); + assertCacheState(client, index2, 0, 1); + assertTrue(getRequestCacheStats(client, index2).getMemorySizeInBytes() > 0); + + // force refresh so that it creates 1 stale key + flushAndRefresh(index2); + // sleep until cache cleaner would have cleaned up the stale key from index 2 + assertBusy(() -> { + // assert segment counts stay the same + assertEquals(1, getSegmentCount(client, index1)); + assertEquals(1, getSegmentCount(client, index2)); + // cache cleaner should NOT have cleaned up the stale key from index 2 + assertTrue(getRequestCacheStats(client, index2).getMemorySizeInBytes() > 0); + // cache cleaner should NOT have cleaned from index 1 + assertEquals(finalMemorySizeForIndex1, getRequestCacheStats(client, index1).getMemorySizeInBytes()); + }, cacheCleanIntervalInMillis * 2, TimeUnit.MILLISECONDS); + } + + // when staleness threshold is explicitly set to 0, cache cleaner regularly cleans up stale keys. + public void testCacheCleanupOnZeroStalenessThreshold() throws Exception { + int cacheCleanIntervalInMillis = 50; + String node = internalCluster().startNode( + Settings.builder() + .put(IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_STALENESS_THRESHOLD_SETTING_KEY, 0) + .put( + IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_INTERVAL_SETTING_KEY, + TimeValue.timeValueMillis(cacheCleanIntervalInMillis) + ) + ); + Client client = client(node); + String index1 = "index1"; + String index2 = "index2"; + setupIndex(client, index1); + setupIndex(client, index2); + + // create 10 index1 cache entries + for (int i = 1; i <= 10; i++) { + long cacheSizeBefore = getRequestCacheStats(client, index1).getMemorySizeInBytes(); + createCacheEntry(client, index1, "hello" + i); + assertCacheState(client, index1, 0, i); + long cacheSizeAfter = getRequestCacheStats(client, index1).getMemorySizeInBytes(); + assertTrue(cacheSizeAfter > cacheSizeBefore); + } + + long finalMemorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); + + // create first cache entry in index2 + createCacheEntry(client, index2, "hello"); + assertCacheState(client, index2, 0, 1); + assertTrue(getRequestCacheStats(client, index2).getMemorySizeInBytes() > 0); + + // create 1 stale key + indexRandom(false, client.prepareIndex(index2).setId("1").setSource("d", "hello")); + forceMerge(client, index2); + // sleep until cache cleaner would have cleaned up the stale key from index 2 + assertBusy(() -> { + // assert segment counts stay the same + assertEquals(1, getSegmentCount(client, index1)); + assertEquals(2, getSegmentCount(client, index2)); + // cache cleaner should have cleaned up the stale key from index 2 + assertEquals(0, getRequestCacheStats(client, index2).getMemorySizeInBytes()); + // cache cleaner should NOT have cleaned from index 1 + assertEquals(finalMemorySizeForIndex1, getRequestCacheStats(client, index1).getMemorySizeInBytes()); + }, cacheCleanIntervalInMillis * 2, TimeUnit.MILLISECONDS); + } + + // when staleness threshold is not explicitly set, cache cleaner regularly cleans up stale keys + public void testStaleKeysRemovalWithoutExplicitThreshold() throws Exception { + int cacheCleanIntervalInMillis = 1; + String node = internalCluster().startNode( + Settings.builder() + .put( + IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_INTERVAL_SETTING_KEY, + TimeValue.timeValueMillis(cacheCleanIntervalInMillis) + ) + ); + String index1 = "index1"; + String index2 = "index2"; + Client client = client(node); + setupIndex(client, index1); + setupIndex(client, index2); + + // create first cache entry in index1 + createCacheEntry(client, index1, "hello"); + assertCacheState(client, index1, 0, 1); + long memorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); + assertTrue(memorySizeForIndex1 > 0); + + // create second cache entry in index1 + createCacheEntry(client, index1, "there"); + assertCacheState(client, index1, 0, 2); + long finalMemorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); + assertTrue(finalMemorySizeForIndex1 > memorySizeForIndex1); + + // create first cache entry in index2 + createCacheEntry(client, index2, "hello"); + assertCacheState(client, index2, 0, 1); + assertTrue(getRequestCacheStats(client, index2).getMemorySizeInBytes() > 0); + + // force refresh so that it creates 1 stale key + indexRandom(false, client.prepareIndex(index2).setId("1").setSource("d", "hello")); + forceMerge(client, index2); + // sleep until cache cleaner would have cleaned up the stale key from index 2 + assertBusy(() -> { + assertEquals(1, getSegmentCount(client, index1)); + assertEquals(2, getSegmentCount(client, index2)); + // cache cleaner should have cleaned up the stale key from index 2 + assertEquals(0, getRequestCacheStats(client, index2).getMemorySizeInBytes()); + // cache cleaner should NOT have cleaned from index 1 + assertEquals(finalMemorySizeForIndex1, getRequestCacheStats(client, index1).getMemorySizeInBytes()); + }, cacheCleanIntervalInMillis * 2, TimeUnit.MILLISECONDS); + } + + // when cache cleaner interval setting is not set, cache cleaner is configured appropriately with the fall-back setting + public void testCacheCleanupWithDefaultSettings() throws Exception { + int cacheCleanIntervalInMillis = 1; + String node = internalCluster().startNode( + Settings.builder().put(INDICES_CACHE_CLEANUP_INTERVAL_SETTING_KEY, TimeValue.timeValueMillis(cacheCleanIntervalInMillis)) + ); + Client client = client(node); + String index1 = "index1"; + String index2 = "index2"; + setupIndex(client, index1); + setupIndex(client, index2); + + // create first cache entry in index1 + createCacheEntry(client, index1, "hello"); + assertCacheState(client, index1, 0, 1); + long memorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); + assertTrue(memorySizeForIndex1 > 0); + + // create second cache entry in index1 + createCacheEntry(client, index1, "there"); + assertCacheState(client, index1, 0, 2); + long finalMemorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); + assertTrue(finalMemorySizeForIndex1 > memorySizeForIndex1); + + // create first cache entry in index2 + createCacheEntry(client, index2, "hello"); + assertCacheState(client, index2, 0, 1); + assertTrue(getRequestCacheStats(client, index2).getMemorySizeInBytes() > 0); + + // create 1 stale key + indexRandom(false, client.prepareIndex(index2).setId("1").setSource("d", "hello")); + forceMerge(client, index2); + // sleep until cache cleaner would have cleaned up the stale key from index 2 + assertBusy(() -> { + // assert segment counts stay the same + assertEquals(1, getSegmentCount(client, index1)); + assertEquals(2, getSegmentCount(client, index2)); + // cache cleaner should have cleaned up the stale key from index 2 + assertEquals(0, getRequestCacheStats(client, index2).getMemorySizeInBytes()); + // cache cleaner should NOT have cleaned from index 1 + assertEquals(finalMemorySizeForIndex1, getRequestCacheStats(client, index1).getMemorySizeInBytes()); + }, cacheCleanIntervalInMillis * 2, TimeUnit.MILLISECONDS); + } + + // staleness threshold updates flows through to the cache cleaner + public void testDynamicStalenessThresholdUpdate() throws Exception { + int cacheCleanIntervalInMillis = 1; + String node = internalCluster().startNode( + Settings.builder() + .put(IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_STALENESS_THRESHOLD_SETTING_KEY, 0.90) + .put( + IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_INTERVAL_SETTING_KEY, + TimeValue.timeValueMillis(cacheCleanIntervalInMillis) + ) + ); + Client client = client(node); + String index1 = "index1"; + String index2 = "index2"; + setupIndex(client, index1); + setupIndex(client, index2); + + // create first cache entry in index1 + createCacheEntry(client, index1, "hello"); + assertCacheState(client, index1, 0, 1); + long memorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); + assertTrue(memorySizeForIndex1 > 0); + + // create second cache entry in index1 + createCacheEntry(client, index1, "there"); + assertCacheState(client, index1, 0, 2); + assertTrue(getRequestCacheStats(client, index1).getMemorySizeInBytes() > memorySizeForIndex1); + + // create first cache entry in index2 + createCacheEntry(client, index2, "hello"); + assertCacheState(client, index2, 0, 1); + long finalMemorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); + assertTrue(finalMemorySizeForIndex1 > 0); + + // create 1 stale key + indexRandom(false, client.prepareIndex(index2).setId("1").setSource("d", "hello")); + forceMerge(client, index2); + assertBusy(() -> { + // cache cleaner should NOT have cleaned up the stale key from index 2 + assertTrue(getRequestCacheStats(client, index2).getMemorySizeInBytes() > 0); + }, cacheCleanIntervalInMillis * 2, TimeUnit.MILLISECONDS); + + // Update indices.requests.cache.cleanup.staleness_threshold to "10%" + ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest(); + updateSettingsRequest.persistentSettings(Settings.builder().put(INDICES_REQUEST_CACHE_STALENESS_THRESHOLD_SETTING.getKey(), 0.10)); + assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); + + assertBusy(() -> { + // assert segment counts stay the same + assertEquals(1, getSegmentCount(client, index1)); + assertEquals(2, getSegmentCount(client, index2)); + // cache cleaner should have cleaned up the stale key from index 2 + assertEquals(0, getRequestCacheStats(client, index2).getMemorySizeInBytes()); + // cache cleaner should NOT have cleaned from index 1 + assertEquals(finalMemorySizeForIndex1, getRequestCacheStats(client, index1).getMemorySizeInBytes()); + }, cacheCleanIntervalInMillis * 2, TimeUnit.MILLISECONDS); + } + + // staleness threshold dynamic updates should throw exceptions on invalid input + public void testInvalidStalenessThresholdUpdateThrowsException() throws Exception { + // Update indices.requests.cache.cleanup.staleness_threshold to "10%" with illegal argument + assertThrows("Ratio should be in [0-1.0]", IllegalArgumentException.class, () -> { + ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest(); + updateSettingsRequest.persistentSettings( + Settings.builder().put(IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_STALENESS_THRESHOLD_SETTING_KEY, 10) + ); + client().admin().cluster().updateSettings(updateSettingsRequest).actionGet(); + }); + } + + // closing the Index after caching will clean up from Indices Request Cache + public void testCacheClearanceAfterIndexClosure() throws Exception { + int cacheCleanIntervalInMillis = 100; + String node = internalCluster().startNode( + Settings.builder() + .put(IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_STALENESS_THRESHOLD_SETTING_KEY, 0.10) + .put( + IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_INTERVAL_SETTING_KEY, + TimeValue.timeValueMillis(cacheCleanIntervalInMillis) + ) + ); + Client client = client(node); + String index = "index"; + setupIndex(client, index); + + // assert there are no entries in the cache for index + assertEquals(0, getRequestCacheStats(client, index).getMemorySizeInBytes()); + // assert there are no entries in the cache from other indices in the node + assertEquals(0, getNodeCacheStats(client).getMemorySizeInBytes()); + // create first cache entry in index + createCacheEntry(client, index, "hello"); + assertCacheState(client, index, 0, 1); + assertTrue(getRequestCacheStats(client, index).getMemorySizeInBytes() > 0); + assertTrue(getNodeCacheStats(client).getMemorySizeInBytes() > 0); + + // close index + assertAcked(client.admin().indices().prepareClose(index)); + // request cache stats cannot be access since Index should be closed + try { + getRequestCacheStats(client, index); + } catch (Exception e) { + assert (e instanceof IndexClosedException); + } + // sleep until cache cleaner would have cleaned up the stale key from index + assertBusy(() -> { + // cache cleaner should have cleaned up the stale keys from index + assertEquals(0, getNodeCacheStats(client).getMemorySizeInBytes()); + }, cacheCleanIntervalInMillis * 2, TimeUnit.MILLISECONDS); + } + + // deleting the Index after caching will clean up from Indices Request Cache + public void testCacheCleanupAfterIndexDeletion() throws Exception { + int cacheCleanIntervalInMillis = 100; + String node = internalCluster().startNode( + Settings.builder() + .put(IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_STALENESS_THRESHOLD_SETTING_KEY, 0.10) + .put( + IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_INTERVAL_SETTING_KEY, + TimeValue.timeValueMillis(cacheCleanIntervalInMillis) + ) + ); + Client client = client(node); + String index = "index"; + setupIndex(client, index); + + // assert there are no entries in the cache for index + assertEquals(0, getRequestCacheStats(client, index).getMemorySizeInBytes()); + // assert there are no entries in the cache from other indices in the node + assertEquals(0, getNodeCacheStats(client).getMemorySizeInBytes()); + // create first cache entry in index + createCacheEntry(client, index, "hello"); + assertCacheState(client, index, 0, 1); + assertTrue(getRequestCacheStats(client, index).getMemorySizeInBytes() > 0); + assertTrue(getNodeCacheStats(client).getMemorySizeInBytes() > 0); + + // delete index + assertAcked(client.admin().indices().prepareDelete(index)); + // request cache stats cannot be access since Index should be deleted + try { + getRequestCacheStats(client, index); + } catch (Exception e) { + assert (e instanceof IndexNotFoundException); + } + + // sleep until cache cleaner would have cleaned up the stale key from index + assertBusy(() -> { + // cache cleaner should have cleaned up the stale keys from index + assertEquals(0, getNodeCacheStats(client).getMemorySizeInBytes()); + }, cacheCleanIntervalInMillis * 2, TimeUnit.MILLISECONDS); + } + + // when staleness threshold is lower than staleness, it should clean the cache from all indices having stale keys + public void testStaleKeysCleanupWithMultipleIndices() throws Exception { + int cacheCleanIntervalInMillis = 10; + String node = internalCluster().startNode( + Settings.builder() + .put(IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_STALENESS_THRESHOLD_SETTING_KEY, 0.10) + .put( + IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_INTERVAL_SETTING_KEY, + TimeValue.timeValueMillis(cacheCleanIntervalInMillis) + ) + ); + Client client = client(node); + String index1 = "index1"; + String index2 = "index2"; + setupIndex(client, index1); + setupIndex(client, index2); + + // assert cache is empty for index1 + assertEquals(0, getRequestCacheStats(client, index1).getMemorySizeInBytes()); + // create first cache entry in index1 + createCacheEntry(client, index1, "hello"); + assertCacheState(client, index1, 0, 1); + long memorySizeForIndex1With1Entries = getRequestCacheStats(client, index1).getMemorySizeInBytes(); + assertTrue(memorySizeForIndex1With1Entries > 0); + + // create second cache entry in index1 + createCacheEntry(client, index1, "there"); + assertCacheState(client, index1, 0, 2); + long memorySizeForIndex1With2Entries = getRequestCacheStats(client, index1).getMemorySizeInBytes(); + assertTrue(memorySizeForIndex1With2Entries > memorySizeForIndex1With1Entries); + + // assert cache is empty for index2 + assertEquals(0, getRequestCacheStats(client, index2).getMemorySizeInBytes()); + // create first cache entry in index2 + createCacheEntry(client, index2, "hello"); + assertCacheState(client, index2, 0, 1); + assertTrue(getRequestCacheStats(client, index2).getMemorySizeInBytes() > 0); + + // invalidate the cache for index1 + indexRandom(false, client.prepareIndex(index1).setId("1").setSource("d", "hello")); + forceMerge(client, index1); + // Assert cache is cleared up + assertBusy( + () -> { assertEquals(0, getRequestCacheStats(client, index1).getMemorySizeInBytes()); }, + cacheCleanIntervalInMillis * 2, + TimeUnit.MILLISECONDS + ); + + // invalidate the cache for index2 + indexRandom(false, client.prepareIndex(index2).setId("1").setSource("d", "hello")); + forceMerge(client, index2); + + // create another cache entry in index 1 same as memorySizeForIndex1With1Entries, this should not be cleaned up. + createCacheEntry(client, index1, "hello"); + + // sleep until cache cleaner would have cleaned up the stale key from index2 + assertBusy(() -> { + // assert segment counts stay the same + assertEquals(2, getSegmentCount(client, index1)); + assertEquals(2, getSegmentCount(client, index2)); + // cache cleaner should have cleaned up the stale key from index2 and hence cache should be empty + assertEquals(0, getRequestCacheStats(client, index2).getMemorySizeInBytes()); + // cache cleaner should have only cleaned up the stale entities for index1 + long currentMemorySizeInBytesForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); + // assert the memory size of index1 to only contain 1 entry added after flushAndRefresh + assertEquals(memorySizeForIndex1With1Entries, currentMemorySizeInBytesForIndex1); + }, cacheCleanIntervalInMillis * 2, TimeUnit.MILLISECONDS); + } + + private void setupIndex(Client client, String index) throws Exception { + 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) + // Disable index refreshing to avoid cache being invalidated mid-test + .put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), TimeValue.timeValueMillis(-1)) + // Disable background segment merges invalidating the cache + .put(MergePolicyProvider.INDEX_MERGE_ENABLED, false) + ) + .get() + ); + indexRandom(false, client.prepareIndex(index).setSource("k", "hello")); + indexRandom(false, client.prepareIndex(index).setSource("k", "there")); + ensureSearchable(index); + forceMerge(client, index); + } + + private int getSegmentCount(Client client, String indexName) { + return client.admin() + .indices() + .segments(new IndicesSegmentsRequest(indexName)) + .actionGet() + .getIndices() + .get(indexName) + .getShards() + .get(0) + .getShards()[0].getSegments() + .size(); + } + + private void forceMerge(Client client, String index) { + ForceMergeResponse forceMergeResponse = client.admin().indices().prepareForceMerge(index).setFlush(true).get(); + OpenSearchAssertions.assertAllSuccessful(forceMergeResponse); + refreshAndWaitForReplication(); + } + + private void createCacheEntry(Client client, String index, String value) { + SearchResponse resp = client.prepareSearch(index).setRequestCache(true).setQuery(QueryBuilders.termQuery("k", value)).get(); + assertSearchResponse(resp); + OpenSearchAssertions.assertAllSuccessful(resp); + } + + private static void assertCacheState(Client client, String index, long expectedHits, long expectedMisses) { + RequestCacheStats requestCacheStats = getRequestCacheStats(client, index); + // Check the hit count and miss count together so if they are not + // correct we can see both values + assertEquals( + Arrays.asList(expectedHits, expectedMisses, 0L), + Arrays.asList(requestCacheStats.getHitCount(), requestCacheStats.getMissCount(), requestCacheStats.getEvictions()) + ); + + } + + private static RequestCacheStats getRequestCacheStats(Client client, String index) { + return client.admin().indices().prepareStats(index).setRequestCache(true).get().getTotal().getRequestCache(); + } + + private static RequestCacheStats getNodeCacheStats(Client client) { + NodesStatsResponse stats = client.admin().cluster().prepareNodesStats().execute().actionGet(); + for (NodeStats stat : stats.getNodes()) { + if (stat.getNode().isDataNode()) { + return stat.getIndices().getRequestCache(); + } + } + return null; + } +} diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/IndicesRequestCacheIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/IndicesRequestCacheIT.java index 0383aca2de33f..09d5c208a8756 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/IndicesRequestCacheIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/IndicesRequestCacheIT.java @@ -37,7 +37,6 @@ import org.opensearch.action.admin.cluster.health.ClusterHealthResponse; import org.opensearch.action.admin.cluster.node.stats.NodeStats; import org.opensearch.action.admin.cluster.node.stats.NodesStatsResponse; -import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; import org.opensearch.action.admin.indices.alias.Alias; import org.opensearch.action.admin.indices.cache.clear.ClearIndicesCacheRequest; import org.opensearch.action.admin.indices.forcemerge.ForceMergeResponse; @@ -55,7 +54,7 @@ import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; import org.opensearch.env.NodeEnvironment; -import org.opensearch.index.IndexNotFoundException; +import org.opensearch.index.IndexSettings; import org.opensearch.index.cache.request.RequestCacheStats; import org.opensearch.index.query.QueryBuilders; import org.opensearch.search.aggregations.bucket.global.GlobalAggregationBuilder; @@ -75,13 +74,10 @@ import java.util.Arrays; import java.util.Collection; import java.util.List; -import java.util.concurrent.TimeUnit; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS; import static org.opensearch.cluster.routing.allocation.decider.EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING; -import static org.opensearch.indices.IndicesRequestCache.INDICES_REQUEST_CACHE_STALENESS_THRESHOLD_SETTING; -import static org.opensearch.indices.IndicesService.INDICES_CACHE_CLEANUP_INTERVAL_SETTING_KEY; import static org.opensearch.search.SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING; import static org.opensearch.search.aggregations.AggregationBuilders.dateHistogram; import static org.opensearch.search.aggregations.AggregationBuilders.dateRange; @@ -126,6 +122,8 @@ public void testCacheAggs() throws Exception { .put(IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING.getKey(), true) .put(SETTING_NUMBER_OF_SHARDS, 1) .put(SETTING_NUMBER_OF_REPLICAS, 0) + // Disable index refreshing to avoid cache being invalidated mid-test + .put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), TimeValue.timeValueMillis(-1)) ) .get() ); @@ -135,6 +133,8 @@ public void testCacheAggs() throws Exception { client.prepareIndex(index).setSource("f", "2014-05-13T00:00:00.000Z") ); ensureSearchable(index); + // Force merge the index to ensure there can be no background merges during the subsequent searches that would invalidate the cache + forceMerge(client, index); // This is not a random example: serialization with time zones writes shared strings // which used to not work well with the query cache because of the handles stream output @@ -197,6 +197,8 @@ public void testQueryRewrite() throws Exception { .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 5) .put("index.number_of_routing_shards", 5) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + // Disable index refreshing to avoid cache being invalidated mid-test + .put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), TimeValue.timeValueMillis(-1)) ) .get() ); @@ -214,12 +216,8 @@ public void testQueryRewrite() throws Exception { ); ensureSearchable(index); assertCacheState(client, index, 0, 0); - // 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); - refreshAndWaitForReplication(); - ensureSearchable(index); + forceMerge(client, index); assertCacheState(client, index, 0, 0); @@ -268,6 +266,8 @@ public void testQueryRewriteMissingValues() throws Exception { .put(IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING.getKey(), true) .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + // Disable index refreshing to avoid cache being invalidated mid-test + .put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), TimeValue.timeValueMillis(-1)) ) .get() ); @@ -287,10 +287,7 @@ public void testQueryRewriteMissingValues() throws Exception { assertCacheState(client, index, 0, 0); // 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); - refreshAndWaitForReplication(); - ensureSearchable(index); + forceMerge(client, index); assertCacheState(client, index, 0, 0); @@ -335,6 +332,8 @@ public void testQueryRewriteDates() throws Exception { .put(IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING.getKey(), true) .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + // Disable index refreshing to avoid cache being invalidated mid-test + .put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), TimeValue.timeValueMillis(-1)) ) .get() ); @@ -354,10 +353,7 @@ public void testQueryRewriteDates() throws Exception { assertCacheState(client, index, 0, 0); // 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); - refreshAndWaitForReplication(); - ensureSearchable(index); + forceMerge(client, index); assertCacheState(client, index, 0, 0); @@ -399,6 +395,8 @@ public void testQueryRewriteDatesWithNow() throws Exception { .put(IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING.getKey(), true) .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + // Disable index refreshing to avoid cache being invalidated mid-test + .put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), TimeValue.timeValueMillis(-1)) .build(); assertAcked(client.admin().indices().prepareCreate("index-1").setMapping("d", "type=date").setSettings(settings).get()); assertAcked(client.admin().indices().prepareCreate("index-2").setMapping("d", "type=date").setSettings(settings).get()); @@ -480,6 +478,7 @@ public void testCanCache() throws Exception { .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 2) .put("index.number_of_routing_shards", 2) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), TimeValue.timeValueMillis(-1)) .build(); String index = "index"; assertAcked(client.admin().indices().prepareCreate(index).setMapping("s", "type=date").setSettings(settings).get()); @@ -499,10 +498,7 @@ public void testCanCache() throws Exception { assertCacheState(client, index, 0, 0); // 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); - refreshAndWaitForReplication(); - ensureSearchable(index); + forceMerge(client, index); assertCacheState(client, index, 0, 0); @@ -644,11 +640,15 @@ public void testProfileDisableCache() throws Exception { .put(IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING.getKey(), true) .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + // Disable index refreshing to avoid cache being invalidated mid-test + .put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), TimeValue.timeValueMillis(-1)) ) .get() ); indexRandom(true, client.prepareIndex(index).setSource("k", "hello")); ensureSearchable(index); + // Force merge the index to ensure there can be no background merges during the subsequent searches that would invalidate the cache + forceMerge(client, index); int expectedHits = 0; int expectedMisses = 0; @@ -673,556 +673,6 @@ public void testProfileDisableCache() throws Exception { } } - public void testCacheWithInvalidation() throws Exception { - Client client = client(); - String index = "index"; - 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() - ); - indexRandom(true, client.prepareIndex(index).setSource("k", "hello")); - ensureSearchable(index); - SearchResponse resp = client.prepareSearch(index).setRequestCache(true).setQuery(QueryBuilders.termQuery("k", "hello")).get(); - assertSearchResponse(resp); - OpenSearchAssertions.assertAllSuccessful(resp); - assertThat(resp.getHits().getTotalHits().value, equalTo(1L)); - - assertCacheState(client, index, 0, 1); - // Index but don't refresh - indexRandom(false, client.prepareIndex(index).setSource("k", "hello2")); - resp = client.prepareSearch(index).setRequestCache(true).setQuery(QueryBuilders.termQuery("k", "hello")).get(); - assertSearchResponse(resp); - // Should expect hit as here as refresh didn't happen - assertCacheState(client, index, 1, 1); - - // Explicit refresh would invalidate cache - refreshAndWaitForReplication(); - // Hit same query again - resp = client.prepareSearch(index).setRequestCache(true).setQuery(QueryBuilders.termQuery("k", "hello")).get(); - assertSearchResponse(resp); - // Should expect miss as key has changed due to change in IndexReader.CacheKey (due to refresh) - assertCacheState(client, index, 1, 2); - } - - // calling cache clear api, when staleness threshold is lower than staleness, it should clean the stale keys from cache - public void testCacheClearAPIRemovesStaleKeysWhenStalenessThresholdIsLow() throws Exception { - String node = internalCluster().startNode( - Settings.builder() - .put(IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_STALENESS_THRESHOLD_SETTING_KEY, 0.10) - .put( - IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_INTERVAL_SETTING_KEY, - // setting intentionally high to avoid cache cleaner interfering - TimeValue.timeValueMillis(300) - ) - ); - Client client = client(node); - String index1 = "index1"; - String index2 = "index2"; - setupIndex(client, index1); - setupIndex(client, index2); - - // create first cache entry in index1 - createCacheEntry(client, index1, "hello"); - assertCacheState(client, index1, 0, 1); - long memorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); - assertTrue(memorySizeForIndex1 > 0); - - // create second cache entry in index1 - createCacheEntry(client, index1, "there"); - assertCacheState(client, index1, 0, 2); - long finalMemorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); - assertTrue(finalMemorySizeForIndex1 > memorySizeForIndex1); - - // create first cache entry in index2 - createCacheEntry(client, index2, "hello"); - assertCacheState(client, index2, 0, 1); - assertTrue(getRequestCacheStats(client, index2).getMemorySizeInBytes() > 0); - - ClearIndicesCacheRequest clearIndicesCacheRequest = new ClearIndicesCacheRequest(index2); - client.admin().indices().clearCache(clearIndicesCacheRequest).actionGet(); - - // cache cleaner should have cleaned up the stale key from index 2 - assertEquals(0, getRequestCacheStats(client, index2).getMemorySizeInBytes()); - // cache cleaner should NOT have cleaned from index 1 - assertEquals(finalMemorySizeForIndex1, getRequestCacheStats(client, index1).getMemorySizeInBytes()); - } - - // when staleness threshold is lower than staleness, it should clean the stale keys from cache - public void testStaleKeysCleanupWithLowThreshold() throws Exception { - int cacheCleanIntervalInMillis = 1; - String node = internalCluster().startNode( - Settings.builder() - .put(IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_STALENESS_THRESHOLD_SETTING_KEY, 0.10) - .put( - IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_INTERVAL_SETTING_KEY, - TimeValue.timeValueMillis(cacheCleanIntervalInMillis) - ) - ); - Client client = client(node); - String index1 = "index1"; - String index2 = "index2"; - setupIndex(client, index1); - setupIndex(client, index2); - - // create first cache entry in index1 - createCacheEntry(client, index1, "hello"); - assertCacheState(client, index1, 0, 1); - long memorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); - assertTrue(memorySizeForIndex1 > 0); - - // create second cache entry in index1 - createCacheEntry(client, index1, "there"); - assertCacheState(client, index1, 0, 2); - long finalMemorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); - assertTrue(finalMemorySizeForIndex1 > memorySizeForIndex1); - - // create first cache entry in index2 - createCacheEntry(client, index2, "hello"); - assertCacheState(client, index2, 0, 1); - assertTrue(getRequestCacheStats(client, index2).getMemorySizeInBytes() > 0); - - // force refresh so that it creates 1 stale key - flushAndRefresh(index2); - // sleep until cache cleaner would have cleaned up the stale key from index 2 - assertBusy(() -> { - // cache cleaner should have cleaned up the stale key from index 2 - assertEquals(0, getRequestCacheStats(client, index2).getMemorySizeInBytes()); - // cache cleaner should NOT have cleaned from index 1 - assertEquals(finalMemorySizeForIndex1, getRequestCacheStats(client, index1).getMemorySizeInBytes()); - }, cacheCleanIntervalInMillis * 2, TimeUnit.MILLISECONDS); - // sleep until cache cleaner would have cleaned up the stale key from index 2 - } - - // when staleness threshold is equal to staleness, it should clean the stale keys from cache - public void testCacheCleanupOnEqualStalenessAndThreshold() throws Exception { - int cacheCleanIntervalInMillis = 1; - String node = internalCluster().startNode( - Settings.builder() - .put(IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_STALENESS_THRESHOLD_SETTING_KEY, 0.33) - .put( - IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_INTERVAL_SETTING_KEY, - TimeValue.timeValueMillis(cacheCleanIntervalInMillis) - ) - ); - Client client = client(node); - String index1 = "index1"; - String index2 = "index2"; - setupIndex(client, index1); - setupIndex(client, index2); - - // create first cache entry in index1 - createCacheEntry(client, index1, "hello"); - assertCacheState(client, index1, 0, 1); - long memorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); - assertTrue(memorySizeForIndex1 > 0); - - // create second cache entry in index1 - createCacheEntry(client, index1, "there"); - assertCacheState(client, index1, 0, 2); - long finalMemorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); - assertTrue(finalMemorySizeForIndex1 > memorySizeForIndex1); - - // create first cache entry in index2 - createCacheEntry(client, index2, "hello"); - assertCacheState(client, index2, 0, 1); - assertTrue(getRequestCacheStats(client, index2).getMemorySizeInBytes() > 0); - - // force refresh so that it creates 1 stale key - flushAndRefresh(index2); - // sleep until cache cleaner would have cleaned up the stale key from index 2 - assertBusy(() -> { - // cache cleaner should have cleaned up the stale key from index 2 - assertEquals(0, getRequestCacheStats(client, index2).getMemorySizeInBytes()); - // cache cleaner should NOT have cleaned from index 1 - assertEquals(finalMemorySizeForIndex1, getRequestCacheStats(client, index1).getMemorySizeInBytes()); - }, cacheCleanIntervalInMillis * 2, TimeUnit.MILLISECONDS); - } - - // when staleness threshold is higher than staleness, it should NOT clean the cache - public void testCacheCleanupSkipsWithHighStalenessThreshold() throws Exception { - int cacheCleanIntervalInMillis = 1; - String node = internalCluster().startNode( - Settings.builder() - .put(IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_STALENESS_THRESHOLD_SETTING_KEY, 0.90) - .put( - IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_INTERVAL_SETTING_KEY, - TimeValue.timeValueMillis(cacheCleanIntervalInMillis) - ) - ); - Client client = client(node); - String index1 = "index1"; - String index2 = "index2"; - setupIndex(client, index1); - setupIndex(client, index2); - - // create first cache entry in index1 - createCacheEntry(client, index1, "hello"); - assertCacheState(client, index1, 0, 1); - long memorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); - assertTrue(memorySizeForIndex1 > 0); - - // create second cache entry in index1 - createCacheEntry(client, index1, "there"); - assertCacheState(client, index1, 0, 2); - long finalMemorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); - assertTrue(finalMemorySizeForIndex1 > memorySizeForIndex1); - - // create first cache entry in index2 - createCacheEntry(client, index2, "hello"); - assertCacheState(client, index2, 0, 1); - assertTrue(getRequestCacheStats(client, index2).getMemorySizeInBytes() > 0); - - // force refresh so that it creates 1 stale key - flushAndRefresh(index2); - // sleep until cache cleaner would have cleaned up the stale key from index 2 - assertBusy(() -> { - // cache cleaner should NOT have cleaned up the stale key from index 2 - assertTrue(getRequestCacheStats(client, index2).getMemorySizeInBytes() > 0); - // cache cleaner should NOT have cleaned from index 1 - assertEquals(finalMemorySizeForIndex1, getRequestCacheStats(client, index1).getMemorySizeInBytes()); - }, cacheCleanIntervalInMillis * 2, TimeUnit.MILLISECONDS); - } - - // when staleness threshold is explicitly set to 0, cache cleaner regularly cleans up stale keys. - public void testCacheCleanupOnZeroStalenessThreshold() throws Exception { - int cacheCleanIntervalInMillis = 50; - String node = internalCluster().startNode( - Settings.builder() - .put(IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_STALENESS_THRESHOLD_SETTING_KEY, 0) - .put( - IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_INTERVAL_SETTING_KEY, - TimeValue.timeValueMillis(cacheCleanIntervalInMillis) - ) - ); - Client client = client(node); - String index1 = "index1"; - String index2 = "index2"; - setupIndex(client, index1); - setupIndex(client, index2); - - // create 10 index1 cache entries - for (int i = 1; i <= 10; i++) { - long cacheSizeBefore = getRequestCacheStats(client, index1).getMemorySizeInBytes(); - createCacheEntry(client, index1, "hello" + i); - assertCacheState(client, index1, 0, i); - long cacheSizeAfter = getRequestCacheStats(client, index1).getMemorySizeInBytes(); - assertTrue(cacheSizeAfter > cacheSizeBefore); - } - - long finalMemorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); - - // create first cache entry in index2 - createCacheEntry(client, index2, "hello"); - assertCacheState(client, index2, 0, 1); - assertTrue(getRequestCacheStats(client, index2).getMemorySizeInBytes() > 0); - - // force refresh so that it creates 1 stale key - flushAndRefresh(index2); - // sleep until cache cleaner would have cleaned up the stale key from index 2 - assertBusy(() -> { - // cache cleaner should have cleaned up the stale key from index 2 - assertEquals(0, getRequestCacheStats(client, index2).getMemorySizeInBytes()); - // cache cleaner should NOT have cleaned from index 1 - assertEquals(finalMemorySizeForIndex1, getRequestCacheStats(client, index1).getMemorySizeInBytes()); - }, cacheCleanIntervalInMillis * 2, TimeUnit.MILLISECONDS); - } - - // when staleness threshold is not explicitly set, cache cleaner regularly cleans up stale keys - public void testStaleKeysRemovalWithoutExplicitThreshold() throws Exception { - int cacheCleanIntervalInMillis = 1; - String node = internalCluster().startNode( - Settings.builder() - .put( - IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_INTERVAL_SETTING_KEY, - TimeValue.timeValueMillis(cacheCleanIntervalInMillis) - ) - ); - String index1 = "index1"; - String index2 = "index2"; - Client client = client(node); - setupIndex(client, index1); - setupIndex(client, index2); - - // create first cache entry in index1 - createCacheEntry(client, index1, "hello"); - assertCacheState(client, index1, 0, 1); - long memorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); - assertTrue(memorySizeForIndex1 > 0); - - // create second cache entry in index1 - createCacheEntry(client, index1, "there"); - assertCacheState(client, index1, 0, 2); - long finalMemorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); - assertTrue(finalMemorySizeForIndex1 > memorySizeForIndex1); - - // create first cache entry in index2 - createCacheEntry(client, index2, "hello"); - assertCacheState(client, index2, 0, 1); - assertTrue(getRequestCacheStats(client, index2).getMemorySizeInBytes() > 0); - - // force refresh so that it creates 1 stale key - flushAndRefresh(index2); - // sleep until cache cleaner would have cleaned up the stale key from index 2 - assertBusy(() -> { - // cache cleaner should have cleaned up the stale key from index 2 - assertEquals(0, getRequestCacheStats(client, index2).getMemorySizeInBytes()); - // cache cleaner should NOT have cleaned from index 1 - assertEquals(finalMemorySizeForIndex1, getRequestCacheStats(client, index1).getMemorySizeInBytes()); - }, cacheCleanIntervalInMillis * 2, TimeUnit.MILLISECONDS); - } - - // when cache cleaner interval setting is not set, cache cleaner is configured appropriately with the fall-back setting - public void testCacheCleanupWithDefaultSettings() throws Exception { - int cacheCleanIntervalInMillis = 1; - String node = internalCluster().startNode( - Settings.builder().put(INDICES_CACHE_CLEANUP_INTERVAL_SETTING_KEY, TimeValue.timeValueMillis(cacheCleanIntervalInMillis)) - ); - Client client = client(node); - String index1 = "index1"; - String index2 = "index2"; - setupIndex(client, index1); - setupIndex(client, index2); - - // create first cache entry in index1 - createCacheEntry(client, index1, "hello"); - assertCacheState(client, index1, 0, 1); - long memorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); - assertTrue(memorySizeForIndex1 > 0); - - // create second cache entry in index1 - createCacheEntry(client, index1, "there"); - assertCacheState(client, index1, 0, 2); - long finalMemorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); - assertTrue(finalMemorySizeForIndex1 > memorySizeForIndex1); - - // create first cache entry in index2 - createCacheEntry(client, index2, "hello"); - assertCacheState(client, index2, 0, 1); - assertTrue(getRequestCacheStats(client, index2).getMemorySizeInBytes() > 0); - - // force refresh so that it creates 1 stale key - flushAndRefresh(index2); - // sleep until cache cleaner would have cleaned up the stale key from index 2 - assertBusy(() -> { - // cache cleaner should have cleaned up the stale key from index 2 - assertEquals(0, getRequestCacheStats(client, index2).getMemorySizeInBytes()); - // cache cleaner should NOT have cleaned from index 1 - assertEquals(finalMemorySizeForIndex1, getRequestCacheStats(client, index1).getMemorySizeInBytes()); - }, cacheCleanIntervalInMillis * 2, TimeUnit.MILLISECONDS); - } - - // staleness threshold updates flows through to the cache cleaner - public void testDynamicStalenessThresholdUpdate() throws Exception { - int cacheCleanIntervalInMillis = 1; - String node = internalCluster().startNode( - Settings.builder() - .put(IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_STALENESS_THRESHOLD_SETTING_KEY, 0.90) - .put( - IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_INTERVAL_SETTING_KEY, - TimeValue.timeValueMillis(cacheCleanIntervalInMillis) - ) - ); - Client client = client(node); - String index1 = "index1"; - String index2 = "index2"; - setupIndex(client, index1); - setupIndex(client, index2); - - // create first cache entry in index1 - createCacheEntry(client, index1, "hello"); - assertCacheState(client, index1, 0, 1); - long memorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); - assertTrue(memorySizeForIndex1 > 0); - - // create second cache entry in index1 - createCacheEntry(client, index1, "there"); - assertCacheState(client, index1, 0, 2); - assertTrue(getRequestCacheStats(client, index1).getMemorySizeInBytes() > memorySizeForIndex1); - - // create first cache entry in index2 - createCacheEntry(client, index2, "hello"); - assertCacheState(client, index2, 0, 1); - long finalMemorySizeForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); - assertTrue(finalMemorySizeForIndex1 > 0); - - // force refresh so that it creates 1 stale key - flushAndRefresh(index2); - assertBusy(() -> { - // cache cleaner should NOT have cleaned up the stale key from index 2 - assertTrue(getRequestCacheStats(client, index2).getMemorySizeInBytes() > 0); - }, cacheCleanIntervalInMillis * 2, TimeUnit.MILLISECONDS); - - // Update indices.requests.cache.cleanup.staleness_threshold to "10%" - ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest(); - updateSettingsRequest.persistentSettings(Settings.builder().put(INDICES_REQUEST_CACHE_STALENESS_THRESHOLD_SETTING.getKey(), 0.10)); - assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); - - assertBusy(() -> { - // cache cleaner should have cleaned up the stale key from index 2 - assertEquals(0, getRequestCacheStats(client, index2).getMemorySizeInBytes()); - // cache cleaner should NOT have cleaned from index 1 - assertEquals(finalMemorySizeForIndex1, getRequestCacheStats(client, index1).getMemorySizeInBytes()); - }, cacheCleanIntervalInMillis * 2, TimeUnit.MILLISECONDS); - } - - // staleness threshold dynamic updates should throw exceptions on invalid input - public void testInvalidStalenessThresholdUpdateThrowsException() throws Exception { - // Update indices.requests.cache.cleanup.staleness_threshold to "10%" with illegal argument - assertThrows("Ratio should be in [0-1.0]", IllegalArgumentException.class, () -> { - ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest(); - updateSettingsRequest.persistentSettings( - Settings.builder().put(IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_STALENESS_THRESHOLD_SETTING_KEY, 10) - ); - client().admin().cluster().updateSettings(updateSettingsRequest).actionGet(); - }); - } - - // closing the Index after caching will clean up from Indices Request Cache - public void testCacheClearanceAfterIndexClosure() throws Exception { - int cacheCleanIntervalInMillis = 100; - String node = internalCluster().startNode( - Settings.builder() - .put(IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_STALENESS_THRESHOLD_SETTING_KEY, 0.10) - .put( - IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_INTERVAL_SETTING_KEY, - TimeValue.timeValueMillis(cacheCleanIntervalInMillis) - ) - ); - Client client = client(node); - String index = "index"; - setupIndex(client, index); - - // assert there are no entries in the cache for index - assertEquals(0, getRequestCacheStats(client, index).getMemorySizeInBytes()); - // assert there are no entries in the cache from other indices in the node - assertEquals(0, getNodeCacheStats(client).getMemorySizeInBytes()); - // create first cache entry in index - createCacheEntry(client, index, "hello"); - assertCacheState(client, index, 0, 1); - assertTrue(getRequestCacheStats(client, index).getMemorySizeInBytes() > 0); - assertTrue(getNodeCacheStats(client).getMemorySizeInBytes() > 0); - - // close index - assertAcked(client.admin().indices().prepareClose(index)); - // request cache stats cannot be access since Index should be closed - try { - getRequestCacheStats(client, index); - } catch (Exception e) { - assert (e instanceof IndexClosedException); - } - // sleep until cache cleaner would have cleaned up the stale key from index - assertBusy(() -> { - // cache cleaner should have cleaned up the stale keys from index - assertEquals(0, getNodeCacheStats(client).getMemorySizeInBytes()); - }, cacheCleanIntervalInMillis * 2, TimeUnit.MILLISECONDS); - } - - // deleting the Index after caching will clean up from Indices Request Cache - public void testCacheCleanupAfterIndexDeletion() throws Exception { - int cacheCleanIntervalInMillis = 100; - String node = internalCluster().startNode( - Settings.builder() - .put(IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_STALENESS_THRESHOLD_SETTING_KEY, 0.10) - .put( - IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_INTERVAL_SETTING_KEY, - TimeValue.timeValueMillis(cacheCleanIntervalInMillis) - ) - ); - Client client = client(node); - String index = "index"; - setupIndex(client, index); - - // assert there are no entries in the cache for index - assertEquals(0, getRequestCacheStats(client, index).getMemorySizeInBytes()); - // assert there are no entries in the cache from other indices in the node - assertEquals(0, getNodeCacheStats(client).getMemorySizeInBytes()); - // create first cache entry in index - createCacheEntry(client, index, "hello"); - assertCacheState(client, index, 0, 1); - assertTrue(getRequestCacheStats(client, index).getMemorySizeInBytes() > 0); - assertTrue(getNodeCacheStats(client).getMemorySizeInBytes() > 0); - - // delete index - assertAcked(client.admin().indices().prepareDelete(index)); - // request cache stats cannot be access since Index should be deleted - try { - getRequestCacheStats(client, index); - } catch (Exception e) { - assert (e instanceof IndexNotFoundException); - } - - // sleep until cache cleaner would have cleaned up the stale key from index - assertBusy(() -> { - // cache cleaner should have cleaned up the stale keys from index - assertEquals(0, getNodeCacheStats(client).getMemorySizeInBytes()); - }, cacheCleanIntervalInMillis * 2, TimeUnit.MILLISECONDS); - } - - // when staleness threshold is lower than staleness, it should clean cache from all indices having stale keys - public void testStaleKeysCleanupWithMultipleIndices() throws Exception { - int cacheCleanIntervalInMillis = 10; - String node = internalCluster().startNode( - Settings.builder() - .put(IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_STALENESS_THRESHOLD_SETTING_KEY, 0.10) - .put( - IndicesRequestCache.INDICES_REQUEST_CACHE_CLEANUP_INTERVAL_SETTING_KEY, - TimeValue.timeValueMillis(cacheCleanIntervalInMillis) - ) - ); - Client client = client(node); - String index1 = "index1"; - String index2 = "index2"; - setupIndex(client, index1); - setupIndex(client, index2); - - // assert cache is empty for index1 - assertEquals(0, getRequestCacheStats(client, index1).getMemorySizeInBytes()); - // create first cache entry in index1 - createCacheEntry(client, index1, "hello"); - assertCacheState(client, index1, 0, 1); - long memorySizeForIndex1With1Entries = getRequestCacheStats(client, index1).getMemorySizeInBytes(); - assertTrue(memorySizeForIndex1With1Entries > 0); - - // create second cache entry in index1 - createCacheEntry(client, index1, "there"); - assertCacheState(client, index1, 0, 2); - long memorySizeForIndex1With2Entries = getRequestCacheStats(client, index1).getMemorySizeInBytes(); - assertTrue(memorySizeForIndex1With2Entries > memorySizeForIndex1With1Entries); - - // assert cache is empty for index2 - assertEquals(0, getRequestCacheStats(client, index2).getMemorySizeInBytes()); - // create first cache entry in index2 - createCacheEntry(client, index2, "hello"); - assertCacheState(client, index2, 0, 1); - assertTrue(getRequestCacheStats(client, index2).getMemorySizeInBytes() > 0); - - // force refresh both index1 and index2 - flushAndRefresh(index1, index2); - // create another cache entry in index 1 same as memorySizeForIndex1With1Entries, this should not be cleaned up. - createCacheEntry(client, index1, "hello"); - // sleep until cache cleaner would have cleaned up the stale key from index2 - assertBusy(() -> { - // cache cleaner should have cleaned up the stale key from index2 and hence cache should be empty - assertEquals(0, getRequestCacheStats(client, index2).getMemorySizeInBytes()); - // cache cleaner should have only cleaned up the stale entities for index1 - long currentMemorySizeInBytesForIndex1 = getRequestCacheStats(client, index1).getMemorySizeInBytes(); - // assert the memory size of index1 to only contain 1 entry added after flushAndRefresh - assertEquals(memorySizeForIndex1With1Entries, currentMemorySizeInBytesForIndex1); - // cache for index1 should not be empty since there was an item cached after flushAndRefresh - assertTrue(currentMemorySizeInBytesForIndex1 > 0); - }, cacheCleanIntervalInMillis * 2, TimeUnit.MILLISECONDS); - } - public void testDeleteAndCreateSameIndexShardOnSameNode() throws Exception { String node_1 = internalCluster().startNode(Settings.builder().build()); Client client = client(node_1); @@ -1238,7 +688,12 @@ public void testDeleteAndCreateSameIndexShardOnSameNode() throws Exception { logger.info("Creating an index: {} with 2 shards", indexName); createIndex( indexName, - Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 2).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0).build() + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 2) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + // Disable index refreshing to avoid cache being invalidated mid-test + .put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), TimeValue.timeValueMillis(-1)) + .build() ); ensureGreen(indexName); @@ -1246,6 +701,9 @@ public void testDeleteAndCreateSameIndexShardOnSameNode() throws Exception { logger.info("Writing few docs and searching those which will cache items in RequestCache"); indexRandom(true, client.prepareIndex(indexName).setSource("k", "hello")); indexRandom(true, client.prepareIndex(indexName).setSource("y", "hello again")); + ensureSearchable(indexName); + // Force merge the index to ensure there can be no background merges during the subsequent searches that would invalidate the cache + forceMerge(client, indexName); SearchResponse resp = client.prepareSearch(indexName).setRequestCache(true).setQuery(QueryBuilders.termQuery("k", "hello")).get(); assertSearchResponse(resp); resp = client.prepareSearch(indexName).setRequestCache(true).setQuery(QueryBuilders.termQuery("y", "hello")).get(); @@ -1318,29 +776,10 @@ private Path[] shardDirectory(String server, Index index, int shard) { return paths; } - private void setupIndex(Client client, String index) throws Exception { - assertAcked( - client.admin() - .indices() - .prepareCreate(index) - .setMapping("k", "type=keyword") - .setSettings( - Settings.builder() - .put(IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING.getKey(), true) - .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) - ) - .get() - ); - indexRandom(true, client.prepareIndex(index).setSource("k", "hello")); - indexRandom(true, client.prepareIndex(index).setSource("k", "there")); - ensureSearchable(index); - } - - private void createCacheEntry(Client client, String index, String value) { - SearchResponse resp = client.prepareSearch(index).setRequestCache(true).setQuery(QueryBuilders.termQuery("k", value)).get(); - assertSearchResponse(resp); - OpenSearchAssertions.assertAllSuccessful(resp); + private void forceMerge(Client client, String index) { + ForceMergeResponse forceMergeResponse = client.admin().indices().prepareForceMerge(index).setFlush(true).get(); + OpenSearchAssertions.assertAllSuccessful(forceMergeResponse); + refreshAndWaitForReplication(); } private static void assertCacheState(Client client, String index, long expectedHits, long expectedMisses) { From 57fb50b22bf30148a632bd4c5e78bde53116f00f Mon Sep 17 00:00:00 2001 From: bowenlan-amzn Date: Wed, 19 Jun 2024 15:48:48 -0700 Subject: [PATCH 41/84] Apply the date histogram rewrite optimization to range aggregation (#13865) * Refactor the ranges representation Signed-off-by: bowenlan-amzn * Refactor try fast filter Signed-off-by: bowenlan-amzn * Main work finished; left the handling of different numeric data types Signed-off-by: bowenlan-amzn * buildRanges accepts field type Signed-off-by: bowenlan-amzn * first working draft probably Signed-off-by: bowenlan-amzn * add change log Signed-off-by: bowenlan-amzn * accommodate geo distance agg Signed-off-by: bowenlan-amzn * Fix test support all numeric types minus one on the upper range Signed-off-by: bowenlan-amzn * [Refactor] range is lower inclusive, right exclusive Signed-off-by: bowenlan-amzn * adding test Signed-off-by: bowenlan-amzn * Adding test and refactor Signed-off-by: bowenlan-amzn * refactor Signed-off-by: bowenlan-amzn * add test Signed-off-by: bowenlan-amzn * add test and update the compare logic in tree traversal Signed-off-by: bowenlan-amzn * fix test, add random test Signed-off-by: bowenlan-amzn * refactor to address comments Signed-off-by: bowenlan-amzn * small potential performance update Signed-off-by: bowenlan-amzn * fix precommit Signed-off-by: bowenlan-amzn * refactor Signed-off-by: bowenlan-amzn * refactor Signed-off-by: bowenlan-amzn * set refresh_interval to -1 Signed-off-by: bowenlan-amzn * address comment Signed-off-by: bowenlan-amzn * address comment Signed-off-by: bowenlan-amzn * address comment Signed-off-by: bowenlan-amzn * Fix test To understand fully about the double and bigdecimal usage in scaled float field will take more time. Signed-off-by: bowenlan-amzn --------- Signed-off-by: bowenlan-amzn --- CHANGELOG.md | 1 + .../index/mapper/ScaledFloatFieldMapper.java | 18 +- .../test/search.aggregation/40_range.yml | 139 ++++++ .../index/mapper/DateFieldMapper.java | 9 +- .../index/mapper/NumberFieldMapper.java | 87 +++- .../index/mapper/NumericPointEncoder.java | 16 + .../bucket/FastFilterRewriteHelper.java | 470 +++++++++++------- .../bucket/composite/CompositeAggregator.java | 19 +- .../AutoDateHistogramAggregator.java | 17 +- .../histogram/DateHistogramAggregator.java | 17 +- .../range/AbstractRangeAggregatorFactory.java | 3 +- .../range/GeoDistanceAggregatorSupplier.java | 4 +- .../GeoDistanceRangeAggregatorFactory.java | 9 +- .../bucket/range/RangeAggregator.java | 37 +- .../bucket/range/RangeAggregatorSupplier.java | 4 +- .../DateHistogramAggregatorTests.java | 2 +- .../bucket/range/RangeAggregatorTests.java | 282 ++++++++++- 17 files changed, 902 insertions(+), 232 deletions(-) create mode 100644 server/src/main/java/org/opensearch/index/mapper/NumericPointEncoder.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 24a4f6fd1b1f1..8c3e63e36bc82 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,6 +7,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ### Added - Add fingerprint ingest processor ([#13724](https://github.com/opensearch-project/OpenSearch/pull/13724)) - [Remote Store] Rate limiter for remote store low priority uploads ([#14374](https://github.com/opensearch-project/OpenSearch/pull/14374/)) +- Apply the date histogram rewrite optimization to range aggregation ([#13865](https://github.com/opensearch-project/OpenSearch/pull/13865)) ### Dependencies - Bump `org.gradle.test-retry` from 1.5.8 to 1.5.9 ([#13442](https://github.com/opensearch-project/OpenSearch/pull/13442)) diff --git a/modules/mapper-extras/src/main/java/org/opensearch/index/mapper/ScaledFloatFieldMapper.java b/modules/mapper-extras/src/main/java/org/opensearch/index/mapper/ScaledFloatFieldMapper.java index 400d867296e5f..3115dce6c10a5 100644 --- a/modules/mapper-extras/src/main/java/org/opensearch/index/mapper/ScaledFloatFieldMapper.java +++ b/modules/mapper-extras/src/main/java/org/opensearch/index/mapper/ScaledFloatFieldMapper.java @@ -35,6 +35,7 @@ import com.fasterxml.jackson.core.JsonParseException; import org.apache.lucene.document.Field; +import org.apache.lucene.document.LongPoint; import org.apache.lucene.index.DocValues; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.NumericDocValues; @@ -165,7 +166,7 @@ public ScaledFloatFieldMapper build(BuilderContext context) { public static final TypeParser PARSER = new TypeParser((n, c) -> new Builder(n, c.getSettings())); - public static final class ScaledFloatFieldType extends SimpleMappedFieldType { + public static final class ScaledFloatFieldType extends SimpleMappedFieldType implements NumericPointEncoder { private final double scalingFactor; private final Double nullValue; @@ -188,6 +189,21 @@ public ScaledFloatFieldType(String name, double scalingFactor) { this(name, true, false, true, Collections.emptyMap(), scalingFactor, null); } + @Override + public byte[] encodePoint(Number value) { + assert value instanceof Double; + double doubleValue = (Double) value; + byte[] point = new byte[Long.BYTES]; + if (doubleValue == Double.POSITIVE_INFINITY) { + LongPoint.encodeDimension(Long.MAX_VALUE, point, 0); + } else if (doubleValue == Double.NEGATIVE_INFINITY) { + LongPoint.encodeDimension(Long.MIN_VALUE, point, 0); + } else { + LongPoint.encodeDimension(Math.round(scale(value)), point, 0); + } + return point; + } + public double getScalingFactor() { return scalingFactor; } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/40_range.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/40_range.yml index 7d887d56ae8fe..2fd926276d0b4 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/40_range.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/40_range.yml @@ -14,6 +14,9 @@ setup: date: type: date format: epoch_second + scaled_field: + type: scaled_float + scaling_factor: 100 - do: cluster.health: @@ -528,3 +531,139 @@ setup: - is_false: aggregations.unsigned_long_range.buckets.2.to - match: { aggregations.unsigned_long_range.buckets.2.doc_count: 0 } + +--- +"Double range profiler shows filter rewrite info": + - skip: + version: " - 2.99.99" + reason: debug info for filter rewrite added in 3.0.0 (to be backported to 2.15.0) + + - do: + indices.create: + index: test_profile + body: + settings: + number_of_replicas: 0 + refresh_interval: -1 + mappings: + properties: + ip: + type: ip + double: + type: double + date: + type: date + format: epoch_second + + - do: + bulk: + index: test_profile + refresh: true + body: + - '{"index": {}}' + - '{"double" : 42}' + - '{"index": {}}' + - '{"double" : 100}' + - '{"index": {}}' + - '{"double" : 50}' + + - do: + search: + index: test_profile + body: + size: 0 + profile: true + aggs: + double_range: + range: + field: double + ranges: + - to: 50 + - from: 50 + to: 150 + - from: 150 + + - length: { aggregations.double_range.buckets: 3 } + + - match: { aggregations.double_range.buckets.0.key: "*-50.0" } + - is_false: aggregations.double_range.buckets.0.from + - match: { aggregations.double_range.buckets.0.to: 50.0 } + - match: { aggregations.double_range.buckets.0.doc_count: 1 } + - match: { aggregations.double_range.buckets.1.key: "50.0-150.0" } + - match: { aggregations.double_range.buckets.1.from: 50.0 } + - match: { aggregations.double_range.buckets.1.to: 150.0 } + - match: { aggregations.double_range.buckets.1.doc_count: 2 } + - match: { aggregations.double_range.buckets.2.key: "150.0-*" } + - match: { aggregations.double_range.buckets.2.from: 150.0 } + - is_false: aggregations.double_range.buckets.2.to + - match: { aggregations.double_range.buckets.2.doc_count: 0 } + + - match: { profile.shards.0.aggregations.0.debug.optimized_segments: 1 } + - match: { profile.shards.0.aggregations.0.debug.unoptimized_segments: 0 } + - match: { profile.shards.0.aggregations.0.debug.leaf_visited: 1 } + - match: { profile.shards.0.aggregations.0.debug.inner_visited: 0 } + +--- +"Scaled Float Range Aggregation": + - do: + index: + index: test + id: 1 + body: { "scaled_field": 1 } + + - do: + index: + index: test + id: 2 + body: { "scaled_field": 1.53 } + + - do: + index: + index: test + id: 3 + body: { "scaled_field": -2.1 } + + - do: + index: + index: test + id: 4 + body: { "scaled_field": 1.53 } + + - do: + indices.refresh: { } + + - do: + search: + index: test + body: + size: 0 + aggs: + my_range: + range: + field: scaled_field + ranges: + - to: 0 + - from: 0 + to: 1 + - from: 1 + to: 1.5 + - from: 1.5 + + - length: { aggregations.my_range.buckets: 4 } + + - match: { aggregations.my_range.buckets.0.key: "*-0.0" } + - is_false: aggregations.my_range.buckets.0.from + - match: { aggregations.my_range.buckets.0.to: 0.0 } + - match: { aggregations.my_range.buckets.0.doc_count: 1 } + - match: { aggregations.my_range.buckets.1.key: "0.0-1.0" } + - match: { aggregations.my_range.buckets.1.from: 0.0 } + - match: { aggregations.my_range.buckets.1.to: 1.0 } + - match: { aggregations.my_range.buckets.1.doc_count: 0 } + - match: { aggregations.my_range.buckets.2.key: "1.0-1.5" } + - match: { aggregations.my_range.buckets.2.from: 1.0 } + - match: { aggregations.my_range.buckets.2.to: 1.5 } + - match: { aggregations.my_range.buckets.2.doc_count: 1 } + - match: { aggregations.my_range.buckets.3.key: "1.5-*" } + - match: { aggregations.my_range.buckets.3.from: 1.5 } + - is_false: aggregations.my_range.buckets.3.to + - match: { aggregations.my_range.buckets.3.doc_count: 2 } diff --git a/server/src/main/java/org/opensearch/index/mapper/DateFieldMapper.java b/server/src/main/java/org/opensearch/index/mapper/DateFieldMapper.java index b7ee3bb8ca3e3..cf8703209fb37 100644 --- a/server/src/main/java/org/opensearch/index/mapper/DateFieldMapper.java +++ b/server/src/main/java/org/opensearch/index/mapper/DateFieldMapper.java @@ -348,7 +348,7 @@ public DateFieldMapper build(BuilderContext context) { * * @opensearch.internal */ - public static final class DateFieldType extends MappedFieldType { + public static final class DateFieldType extends MappedFieldType implements NumericPointEncoder { protected final DateFormatter dateTimeFormatter; protected final DateMathParser dateMathParser; protected final Resolution resolution; @@ -549,6 +549,13 @@ public static long parseToLong( return resolution.convert(dateParser.parse(BytesRefs.toString(value), now, roundUp, zone)); } + @Override + public byte[] encodePoint(Number value) { + byte[] point = new byte[Long.BYTES]; + LongPoint.encodeDimension(value.longValue(), point, 0); + return point; + } + @Override public Query distanceFeatureQuery(Object origin, String pivot, float boost, QueryShardContext context) { failIfNotIndexedAndNoDocValues(); diff --git a/server/src/main/java/org/opensearch/index/mapper/NumberFieldMapper.java b/server/src/main/java/org/opensearch/index/mapper/NumberFieldMapper.java index eb3a99b0e0388..25e5f9970795f 100644 --- a/server/src/main/java/org/opensearch/index/mapper/NumberFieldMapper.java +++ b/server/src/main/java/org/opensearch/index/mapper/NumberFieldMapper.java @@ -171,7 +171,7 @@ public NumberFieldMapper build(BuilderContext context) { * * @opensearch.internal */ - public enum NumberType { + public enum NumberType implements NumericPointEncoder { HALF_FLOAT("half_float", NumericType.HALF_FLOAT) { @Override public Float parse(Object value, boolean coerce) { @@ -194,6 +194,13 @@ public Number parsePoint(byte[] value) { return HalfFloatPoint.decodeDimension(value, 0); } + @Override + public byte[] encodePoint(Number value) { + byte[] point = new byte[HalfFloatPoint.BYTES]; + HalfFloatPoint.encodeDimension(value.floatValue(), point, 0); + return point; + } + @Override public Float parse(XContentParser parser, boolean coerce) throws IOException { float parsed = parser.floatValue(coerce); @@ -331,6 +338,13 @@ public Number parsePoint(byte[] value) { return FloatPoint.decodeDimension(value, 0); } + @Override + public byte[] encodePoint(Number value) { + byte[] point = new byte[Float.BYTES]; + FloatPoint.encodeDimension(value.floatValue(), point, 0); + return point; + } + @Override public Float parse(XContentParser parser, boolean coerce) throws IOException { float parsed = parser.floatValue(coerce); @@ -457,6 +471,13 @@ public Number parsePoint(byte[] value) { return DoublePoint.decodeDimension(value, 0); } + @Override + public byte[] encodePoint(Number value) { + byte[] point = new byte[Double.BYTES]; + DoublePoint.encodeDimension(value.doubleValue(), point, 0); + return point; + } + @Override public Double parse(XContentParser parser, boolean coerce) throws IOException { double parsed = parser.doubleValue(coerce); @@ -582,6 +603,13 @@ public Number parsePoint(byte[] value) { return INTEGER.parsePoint(value).byteValue(); } + @Override + public byte[] encodePoint(Number value) { + byte[] point = new byte[Integer.BYTES]; + IntPoint.encodeDimension(value.intValue(), point, 0); + return point; + } + @Override public Short parse(XContentParser parser, boolean coerce) throws IOException { int value = parser.intValue(coerce); @@ -654,6 +682,13 @@ public Number parsePoint(byte[] value) { return INTEGER.parsePoint(value).shortValue(); } + @Override + public byte[] encodePoint(Number value) { + byte[] point = new byte[Integer.BYTES]; + IntPoint.encodeDimension(value.intValue(), point, 0); + return point; + } + @Override public Short parse(XContentParser parser, boolean coerce) throws IOException { return parser.shortValue(coerce); @@ -722,6 +757,13 @@ public Number parsePoint(byte[] value) { return IntPoint.decodeDimension(value, 0); } + @Override + public byte[] encodePoint(Number value) { + byte[] point = new byte[Integer.BYTES]; + IntPoint.encodeDimension(value.intValue(), point, 0); + return point; + } + @Override public Integer parse(XContentParser parser, boolean coerce) throws IOException { return parser.intValue(coerce); @@ -868,6 +910,13 @@ public Number parsePoint(byte[] value) { return LongPoint.decodeDimension(value, 0); } + @Override + public byte[] encodePoint(Number value) { + byte[] point = new byte[Long.BYTES]; + LongPoint.encodeDimension(value.longValue(), point, 0); + return point; + } + @Override public Long parse(XContentParser parser, boolean coerce) throws IOException { return parser.longValue(coerce); @@ -988,6 +1037,13 @@ public Number parsePoint(byte[] value) { return BigIntegerPoint.decodeDimension(value, 0); } + @Override + public byte[] encodePoint(Number value) { + byte[] point = new byte[BigIntegerPoint.BYTES]; + BigIntegerPoint.encodeDimension(objectToUnsignedLong(value, false, true), point, 0); + return point; + } + @Override public BigInteger parse(XContentParser parser, boolean coerce) throws IOException { return parser.bigIntegerValue(coerce); @@ -1215,16 +1271,30 @@ public static long objectToLong(Object value, boolean coerce) { return Numbers.toLong(stringValue, coerce); } + public static BigInteger objectToUnsignedLong(Object value, boolean coerce) { + return objectToUnsignedLong(value, coerce, false); + } + /** - * Converts and Object to a {@code long} by checking it against known + * Converts an Object to a {@code BigInteger} by checking it against known * types and checking its range. + * + * @param lenientBound if true, use MIN or MAX if the value is out of bound */ - public static BigInteger objectToUnsignedLong(Object value, boolean coerce) { + public static BigInteger objectToUnsignedLong(Object value, boolean coerce, boolean lenientBound) { if (value instanceof Long) { return Numbers.toUnsignedBigInteger(((Long) value).longValue()); } double doubleValue = objectToDouble(value); + if (lenientBound) { + if (doubleValue < Numbers.MIN_UNSIGNED_LONG_VALUE.doubleValue()) { + return Numbers.MIN_UNSIGNED_LONG_VALUE; + } + if (doubleValue > Numbers.MAX_UNSIGNED_LONG_VALUE.doubleValue()) { + return Numbers.MAX_UNSIGNED_LONG_VALUE; + } + } if (doubleValue < Numbers.MIN_UNSIGNED_LONG_VALUE.doubleValue() || doubleValue > Numbers.MAX_UNSIGNED_LONG_VALUE.doubleValue()) { throw new IllegalArgumentException("Value [" + value + "] is out of range for an unsigned long"); @@ -1349,7 +1419,7 @@ public static Query unsignedLongRangeQuery( * * @opensearch.internal */ - public static class NumberFieldType extends SimpleMappedFieldType { + public static class NumberFieldType extends SimpleMappedFieldType implements NumericPointEncoder { private final NumberType type; private final boolean coerce; @@ -1394,6 +1464,10 @@ public String typeName() { return type.name; } + public NumberType numberType() { + return type; + } + public NumericType numericType() { return type.numericType(); } @@ -1501,6 +1575,11 @@ public DocValueFormat docValueFormat(String format, ZoneId timeZone) { public Number parsePoint(byte[] value) { return type.parsePoint(value); } + + @Override + public byte[] encodePoint(Number value) { + return type.encodePoint(value); + } } private final NumberType type; diff --git a/server/src/main/java/org/opensearch/index/mapper/NumericPointEncoder.java b/server/src/main/java/org/opensearch/index/mapper/NumericPointEncoder.java new file mode 100644 index 0000000000000..be746a5526594 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/mapper/NumericPointEncoder.java @@ -0,0 +1,16 @@ +/* + * 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.index.mapper; + +/** + * Interface for encoding a point value + */ +public interface NumericPointEncoder { + byte[] encodePoint(Number value); +} diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/FastFilterRewriteHelper.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/FastFilterRewriteHelper.java index c8ce39a52f869..2ab003fb94e33 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/FastFilterRewriteHelper.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/FastFilterRewriteHelper.java @@ -10,6 +10,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.lucene.document.LongPoint; import org.apache.lucene.index.DocValues; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.NumericDocValues; @@ -32,24 +33,26 @@ import org.opensearch.index.mapper.DateFieldMapper; import org.opensearch.index.mapper.DocCountFieldMapper; import org.opensearch.index.mapper.MappedFieldType; +import org.opensearch.index.mapper.NumericPointEncoder; import org.opensearch.index.query.DateRangeIncludingNowQuery; import org.opensearch.search.aggregations.bucket.composite.CompositeAggregator; import org.opensearch.search.aggregations.bucket.composite.CompositeValuesSourceConfig; import org.opensearch.search.aggregations.bucket.composite.RoundingValuesSource; import org.opensearch.search.aggregations.bucket.histogram.LongBounds; +import org.opensearch.search.aggregations.bucket.range.RangeAggregator.Range; +import org.opensearch.search.aggregations.support.ValuesSource; +import org.opensearch.search.aggregations.support.ValuesSourceConfig; import org.opensearch.search.internal.SearchContext; import java.io.IOException; -import java.util.Arrays; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.OptionalLong; import java.util.function.BiConsumer; -import java.util.function.BiFunction; import java.util.function.Function; +import static org.opensearch.index.mapper.NumberFieldMapper.NumberType.LONG; import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS; /** @@ -187,8 +190,8 @@ public static class FastFilterContext { private AggregationType aggregationType; private final SearchContext context; - private String fieldName; - private long[][] ranges; + private MappedFieldType fieldType; + private Ranges ranges; // debug info related fields public int leaf; @@ -196,12 +199,13 @@ public static class FastFilterContext { public int segments; public int optimizedSegments; - public void setFieldName(String fieldName) { - this.fieldName = fieldName; + public FastFilterContext(SearchContext context) { + this.context = context; } - public FastFilterContext(SearchContext context) { + public FastFilterContext(SearchContext context, AggregationType aggregationType) { this.context = context; + this.aggregationType = aggregationType; } public AggregationType getAggregationType() { @@ -221,23 +225,87 @@ public boolean isRewriteable(final Object parent, final int subAggLength) { return rewriteable; } - public void buildRanges() throws IOException { + public void buildRanges(MappedFieldType fieldType) throws IOException { assert ranges == null : "Ranges should only be built once at shard level, but they are already built"; - this.ranges = this.aggregationType.buildRanges(context); + this.fieldType = fieldType; + this.ranges = this.aggregationType.buildRanges(context, fieldType); if (ranges != null) { logger.debug("Ranges built for shard {}", context.indexShard().shardId()); rangesBuiltAtShardLevel = true; } } - public long[][] buildRanges(LeafReaderContext leaf) throws IOException { - long[][] ranges = this.aggregationType.buildRanges(leaf, context); + private Ranges buildRanges(LeafReaderContext leaf) throws IOException { + Ranges ranges = this.aggregationType.buildRanges(leaf, context, fieldType); if (ranges != null) { logger.debug("Ranges built for shard {} segment {}", context.indexShard().shardId(), leaf.ord); } return ranges; } + /** + * Try to populate the bucket doc counts for aggregation + *

+ * Usage: invoked at segment level — in getLeafCollector of aggregator + * + * @param bucketOrd bucket ordinal producer + * @param incrementDocCount consume the doc_count results for certain ordinal + */ + public boolean tryFastFilterAggregation( + final LeafReaderContext ctx, + final BiConsumer incrementDocCount, + final Function bucketOrd + ) throws IOException { + this.segments++; + if (!this.rewriteable) { + return false; + } + + if (ctx.reader().hasDeletions()) return false; + + PointValues values = ctx.reader().getPointValues(this.fieldType.name()); + if (values == null) return false; + // only proceed if every document corresponds to exactly one point + if (values.getDocCount() != values.size()) return false; + + NumericDocValues docCountValues = DocValues.getNumeric(ctx.reader(), DocCountFieldMapper.NAME); + if (docCountValues.nextDoc() != NO_MORE_DOCS) { + logger.debug( + "Shard {} segment {} has at least one document with _doc_count field, skip fast filter optimization", + this.context.indexShard().shardId(), + ctx.ord + ); + return false; + } + + // even if no ranges built at shard level, we can still perform the optimization + // when functionally match-all at segment level + if (!this.rangesBuiltAtShardLevel && !segmentMatchAll(this.context, ctx)) { + return false; + } + + Ranges ranges = this.ranges; + if (ranges == null) { + logger.debug( + "Shard {} segment {} functionally match all documents. Build the fast filter", + this.context.indexShard().shardId(), + ctx.ord + ); + ranges = this.buildRanges(ctx); + if (ranges == null) { + return false; + } + } + + DebugInfo debugInfo = this.aggregationType.tryFastFilterAggregation(values, ranges, incrementDocCount, bucketOrd); + this.consumeDebugInfo(debugInfo); + + this.optimizedSegments++; + logger.debug("Fast filter optimization applied to shard {} segment {}", this.context.indexShard().shardId(), ctx.ord); + logger.debug("crossed leaf nodes: {}, inner nodes: {}", this.leaf, this.inner); + return true; + } + private void consumeDebugInfo(DebugInfo debug) { leaf += debug.leaf; inner += debug.inner; @@ -250,9 +318,16 @@ private void consumeDebugInfo(DebugInfo debug) { interface AggregationType { boolean isRewriteable(Object parent, int subAggLength); - long[][] buildRanges(SearchContext ctx) throws IOException; + Ranges buildRanges(SearchContext ctx, MappedFieldType fieldType) throws IOException; - long[][] buildRanges(LeafReaderContext leaf, SearchContext ctx) throws IOException; + Ranges buildRanges(LeafReaderContext leaf, SearchContext ctx, MappedFieldType fieldType) throws IOException; + + DebugInfo tryFastFilterAggregation( + PointValues values, + Ranges ranges, + BiConsumer incrementDocCount, + Function bucketOrd + ) throws IOException; } /** @@ -286,13 +361,20 @@ public boolean isRewriteable(Object parent, int subAggLength) { } @Override - public long[][] buildRanges(SearchContext context) throws IOException { + public Ranges buildRanges(SearchContext context, MappedFieldType fieldType) throws IOException { long[] bounds = getDateHistoAggBounds(context, fieldType.name()); logger.debug("Bounds are {} for shard {}", bounds, context.indexShard().shardId()); return buildRanges(context, bounds); } - private long[][] buildRanges(SearchContext context, long[] bounds) throws IOException { + @Override + public Ranges buildRanges(LeafReaderContext leaf, SearchContext context, MappedFieldType fieldType) throws IOException { + long[] bounds = getSegmentBounds(leaf, fieldType.name()); + logger.debug("Bounds are {} for shard {} segment {}", bounds, context.indexShard().shardId(), leaf.ord); + return buildRanges(context, bounds); + } + + private Ranges buildRanges(SearchContext context, long[] bounds) throws IOException { bounds = processHardBounds(bounds); if (bounds == null) { return null; @@ -319,13 +401,6 @@ private long[][] buildRanges(SearchContext context, long[] bounds) throws IOExce ); } - @Override - public long[][] buildRanges(LeafReaderContext leaf, SearchContext context) throws IOException { - long[] bounds = getSegmentBounds(leaf, fieldType.name()); - logger.debug("Bounds are {} for shard {} segment {}", bounds, context.indexShard().shardId(), leaf.ord); - return buildRanges(context, bounds); - } - protected abstract Rounding getRounding(final long low, final long high); protected abstract Rounding.Prepared getRoundingPrepared(); @@ -354,86 +429,118 @@ public DateFieldMapper.DateFieldType getFieldType() { assert fieldType instanceof DateFieldMapper.DateFieldType; return (DateFieldMapper.DateFieldType) fieldType; } - } - public static boolean isCompositeAggRewriteable(CompositeValuesSourceConfig[] sourceConfigs) { - return sourceConfigs.length == 1 && sourceConfigs[0].valuesSource() instanceof RoundingValuesSource; - } + @Override + public DebugInfo tryFastFilterAggregation( + PointValues values, + Ranges ranges, + BiConsumer incrementDocCount, + Function bucketOrd + ) throws IOException { + int size = Integer.MAX_VALUE; + if (this instanceof CompositeAggregator.CompositeAggregationType) { + size = ((CompositeAggregator.CompositeAggregationType) this).getSize(); + } + + DateFieldMapper.DateFieldType fieldType = getFieldType(); + BiConsumer incrementFunc = (activeIndex, docCount) -> { + long rangeStart = LongPoint.decodeDimension(ranges.lowers[activeIndex], 0); + rangeStart = fieldType.convertNanosToMillis(rangeStart); + long ord = getBucketOrd(bucketOrd.apply(rangeStart)); + incrementDocCount.accept(ord, (long) docCount); + }; - public static long getBucketOrd(long bucketOrd) { - if (bucketOrd < 0) { // already seen - bucketOrd = -1 - bucketOrd; + return multiRangesTraverse(values.getPointTree(), ranges, incrementFunc, size); } - return bucketOrd; + private static long getBucketOrd(long bucketOrd) { + if (bucketOrd < 0) { // already seen + bucketOrd = -1 - bucketOrd; + } + + return bucketOrd; + } } /** - * Try to get the bucket doc counts for the date histogram aggregation - *

- * Usage: invoked at segment level — in getLeafCollector of aggregator - * - * @param incrementDocCount takes in the bucket key value and the bucket count + * For range aggregation */ - public static boolean tryFastFilterAggregation( - final LeafReaderContext ctx, - FastFilterContext fastFilterContext, - final BiConsumer incrementDocCount - ) throws IOException { - fastFilterContext.segments++; - if (!fastFilterContext.rewriteable) { - return false; - } + public static class RangeAggregationType implements AggregationType { - if (ctx.reader().hasDeletions()) return false; + private final ValuesSourceConfig config; + private final Range[] ranges; - PointValues values = ctx.reader().getPointValues(fastFilterContext.fieldName); - if (values == null) return false; - // only proceed if every document corresponds to exactly one point - if (values.getDocCount() != values.size()) return false; - - NumericDocValues docCountValues = DocValues.getNumeric(ctx.reader(), DocCountFieldMapper.NAME); - if (docCountValues.nextDoc() != NO_MORE_DOCS) { - logger.debug( - "Shard {} segment {} has at least one document with _doc_count field, skip fast filter optimization", - fastFilterContext.context.indexShard().shardId(), - ctx.ord - ); - return false; + public RangeAggregationType(ValuesSourceConfig config, Range[] ranges) { + this.config = config; + this.ranges = ranges; } - // even if no ranges built at shard level, we can still perform the optimization - // when functionally match-all at segment level - if (!fastFilterContext.rangesBuiltAtShardLevel && !segmentMatchAll(fastFilterContext.context, ctx)) { + @Override + public boolean isRewriteable(Object parent, int subAggLength) { + if (config.fieldType() == null) return false; + MappedFieldType fieldType = config.fieldType(); + if (fieldType.isSearchable() == false || !(fieldType instanceof NumericPointEncoder)) return false; + + if (parent == null && subAggLength == 0 && config.script() == null && config.missing() == null) { + if (config.getValuesSource() instanceof ValuesSource.Numeric.FieldData) { + // ranges are already sorted by from and then to + // we want ranges not overlapping with each other + double prevTo = ranges[0].getTo(); + for (int i = 1; i < ranges.length; i++) { + if (prevTo > ranges[i].getFrom()) { + return false; + } + prevTo = ranges[i].getTo(); + } + return true; + } + } return false; } - long[][] ranges = fastFilterContext.ranges; - if (ranges == null) { - logger.debug( - "Shard {} segment {} functionally match all documents. Build the fast filter", - fastFilterContext.context.indexShard().shardId(), - ctx.ord - ); - ranges = fastFilterContext.buildRanges(ctx); - if (ranges == null) { - return false; + + @Override + public Ranges buildRanges(SearchContext context, MappedFieldType fieldType) { + assert fieldType instanceof NumericPointEncoder; + NumericPointEncoder numericPointEncoder = (NumericPointEncoder) fieldType; + byte[][] lowers = new byte[ranges.length][]; + byte[][] uppers = new byte[ranges.length][]; + for (int i = 0; i < ranges.length; i++) { + double rangeMin = ranges[i].getFrom(); + double rangeMax = ranges[i].getTo(); + byte[] lower = numericPointEncoder.encodePoint(rangeMin); + byte[] upper = numericPointEncoder.encodePoint(rangeMax); + lowers[i] = lower; + uppers[i] = upper; } + + return new Ranges(lowers, uppers); } - final AggregationType aggregationType = fastFilterContext.aggregationType; - assert aggregationType instanceof AbstractDateHistogramAggregationType; - final DateFieldMapper.DateFieldType fieldType = ((AbstractDateHistogramAggregationType) aggregationType).getFieldType(); - int size = Integer.MAX_VALUE; - if (aggregationType instanceof CompositeAggregator.CompositeAggregationType) { - size = ((CompositeAggregator.CompositeAggregationType) aggregationType).getSize(); + @Override + public Ranges buildRanges(LeafReaderContext leaf, SearchContext ctx, MappedFieldType fieldType) { + throw new UnsupportedOperationException("Range aggregation should not build ranges at segment level"); + } + + @Override + public DebugInfo tryFastFilterAggregation( + PointValues values, + Ranges ranges, + BiConsumer incrementDocCount, + Function bucketOrd + ) throws IOException { + int size = Integer.MAX_VALUE; + + BiConsumer incrementFunc = (activeIndex, docCount) -> { + long ord = bucketOrd.apply(activeIndex); + incrementDocCount.accept(ord, (long) docCount); + }; + + return multiRangesTraverse(values.getPointTree(), ranges, incrementFunc, size); } - DebugInfo debugInfo = multiRangesTraverse(values.getPointTree(), ranges, incrementDocCount, fieldType, size); - fastFilterContext.consumeDebugInfo(debugInfo); + } - fastFilterContext.optimizedSegments++; - logger.debug("Fast filter optimization applied to shard {} segment {}", fastFilterContext.context.indexShard().shardId(), ctx.ord); - logger.debug("crossed leaf nodes: {}, inner nodes: {}", fastFilterContext.leaf, fastFilterContext.inner); - return true; + public static boolean isCompositeAggRewriteable(CompositeValuesSourceConfig[] sourceConfigs) { + return sourceConfigs.length == 1 && sourceConfigs[0].valuesSource() instanceof RoundingValuesSource; } private static boolean segmentMatchAll(SearchContext ctx, LeafReaderContext leafCtx) throws IOException { @@ -445,7 +552,7 @@ private static boolean segmentMatchAll(SearchContext ctx, LeafReaderContext leaf * Creates the date ranges from date histo aggregations using its interval, * and min/max boundaries */ - private static long[][] createRangesFromAgg( + private static Ranges createRangesFromAgg( final SearchContext context, final DateFieldMapper.DateFieldType fieldType, final long interval, @@ -481,9 +588,8 @@ private static long[][] createRangesFromAgg( long lower = i == 0 ? low : fieldType.convertRoundedMillisToNanos(roundedLow); roundedLow = preparedRounding.round(roundedLow + interval); - // Subtract -1 if the minimum is roundedLow as roundedLow itself - // is included in the next bucket - long upper = i + 1 == bucketCount ? high : fieldType.convertRoundedMillisToNanos(roundedLow) - 1; + // plus one on high value because upper bound is exclusive, but high value exists + long upper = i + 1 == bucketCount ? high + 1 : fieldType.convertRoundedMillisToNanos(roundedLow); ranges[i][0] = lower; ranges[i][1] = upper; @@ -491,7 +597,16 @@ private static long[][] createRangesFromAgg( } } - return ranges; + byte[][] lowers = new byte[ranges.length][]; + byte[][] uppers = new byte[ranges.length][]; + for (int i = 0; i < ranges.length; i++) { + byte[] lower = LONG.encodePoint(ranges[i][0]); + byte[] max = LONG.encodePoint(ranges[i][1]); + lowers[i] = lower; + uppers[i] = max; + } + + return new Ranges(lowers, uppers); } /** @@ -499,39 +614,18 @@ private static long[][] createRangesFromAgg( */ private static DebugInfo multiRangesTraverse( final PointValues.PointTree tree, - final long[][] ranges, - final BiConsumer incrementDocCount, - final DateFieldMapper.DateFieldType fieldType, + final Ranges ranges, + final BiConsumer incrementDocCount, final int maxNumNonZeroRanges ) throws IOException { - // ranges are connected and in ascending order - Iterator rangeIter = Arrays.stream(ranges).iterator(); - long[] activeRange = rangeIter.next(); - - // make sure the first range at least crosses the min value of the tree DebugInfo debugInfo = new DebugInfo(); - if (activeRange[0] > NumericUtils.sortableBytesToLong(tree.getMaxPackedValue(), 0)) { + int activeIndex = ranges.firstRangeIndex(tree.getMinPackedValue(), tree.getMaxPackedValue()); + if (activeIndex < 0) { logger.debug("No ranges match the query, skip the fast filter optimization"); return debugInfo; } - while (activeRange[1] < NumericUtils.sortableBytesToLong(tree.getMinPackedValue(), 0)) { - if (!rangeIter.hasNext()) { - logger.debug("No ranges match the query, skip the fast filter optimization"); - return debugInfo; - } - activeRange = rangeIter.next(); - } - - RangeCollectorForPointTree collector = new RangeCollectorForPointTree( - incrementDocCount, - fieldType, - rangeIter, - maxNumNonZeroRanges, - activeRange - ); - - final ArrayUtil.ByteArrayComparator comparator = ArrayUtil.getUnsignedComparator(8); - PointValues.IntersectVisitor visitor = getIntersectVisitor(collector, comparator); + RangeCollectorForPointTree collector = new RangeCollectorForPointTree(incrementDocCount, maxNumNonZeroRanges, ranges, activeIndex); + PointValues.IntersectVisitor visitor = getIntersectVisitor(collector); try { intersectWithRanges(visitor, tree, collector, debugInfo); } catch (CollectionTerminatedException e) { @@ -542,6 +636,49 @@ private static DebugInfo multiRangesTraverse( return debugInfo; } + private static class Ranges { + byte[][] lowers; // inclusive + byte[][] uppers; // exclusive + int size; + int byteLen; + static ArrayUtil.ByteArrayComparator comparator; + + Ranges(byte[][] lowers, byte[][] uppers) { + this.lowers = lowers; + this.uppers = uppers; + assert lowers.length == uppers.length; + this.size = lowers.length; + this.byteLen = lowers[0].length; + comparator = ArrayUtil.getUnsignedComparator(byteLen); + } + + public int firstRangeIndex(byte[] globalMin, byte[] globalMax) { + if (compareByteValue(lowers[0], globalMax) > 0) { + return -1; + } + int i = 0; + while (compareByteValue(uppers[i], globalMin) <= 0) { + i++; + if (i >= size) { + return -1; + } + } + return i; + } + + public static int compareByteValue(byte[] value1, byte[] value2) { + return comparator.compare(value1, 0, value2, 0); + } + + public static boolean withinLowerBound(byte[] value, byte[] lowerBound) { + return compareByteValue(value, lowerBound) >= 0; + } + + public static boolean withinUpperBound(byte[] value, byte[] upperBound) { + return compareByteValue(value, upperBound) < 0; + } + } + private static void intersectWithRanges( PointValues.IntersectVisitor visitor, PointValues.PointTree pointTree, @@ -570,10 +707,7 @@ private static void intersectWithRanges( } } - private static PointValues.IntersectVisitor getIntersectVisitor( - RangeCollectorForPointTree collector, - ArrayUtil.ByteArrayComparator comparator - ) { + private static PointValues.IntersectVisitor getIntersectVisitor(RangeCollectorForPointTree collector) { return new PointValues.IntersectVisitor() { @Override public void visit(int docID) throws IOException { @@ -591,86 +725,67 @@ public void visit(int docID, byte[] packedValue) throws IOException { @Override public void visit(DocIdSetIterator iterator, byte[] packedValue) throws IOException { visitPoints(packedValue, () -> { - for (int doc = iterator.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = iterator.nextDoc()) { + for (int doc = iterator.nextDoc(); doc != NO_MORE_DOCS; doc = iterator.nextDoc()) { collector.count(); } }); } private void visitPoints(byte[] packedValue, CheckedRunnable collect) throws IOException { - if (comparator.compare(packedValue, 0, collector.activeRangeAsByteArray[1], 0) > 0) { - // need to move to next range + if (!collector.withinUpperBound(packedValue)) { collector.finalizePreviousRange(); - if (collector.iterateRangeEnd(packedValue, this::compareByteValue)) { + if (collector.iterateRangeEnd(packedValue)) { throw new CollectionTerminatedException(); } } - if (pointCompare(collector.activeRangeAsByteArray[0], collector.activeRangeAsByteArray[1], packedValue)) { + if (collector.withinRange(packedValue)) { collect.run(); } } - private boolean pointCompare(byte[] lower, byte[] upper, byte[] packedValue) { - if (compareByteValue(packedValue, lower) < 0) { - return false; - } - return compareByteValue(packedValue, upper) <= 0; - } - - private int compareByteValue(byte[] value1, byte[] value2) { - return comparator.compare(value1, 0, value2, 0); - } - @Override public PointValues.Relation compare(byte[] minPackedValue, byte[] maxPackedValue) { - byte[] rangeMin = collector.activeRangeAsByteArray[0]; - byte[] rangeMax = collector.activeRangeAsByteArray[1]; - - if (compareByteValue(rangeMax, minPackedValue) < 0) { + // try to find the first range that may collect values from this cell + if (!collector.withinUpperBound(minPackedValue)) { collector.finalizePreviousRange(); - if (collector.iterateRangeEnd(minPackedValue, this::compareByteValue)) { + if (collector.iterateRangeEnd(minPackedValue)) { throw new CollectionTerminatedException(); } - // compare the next range with this node's min max again - // new rangeMin = previous rangeMax + 1 <= min - rangeMax = collector.activeRangeAsByteArray[1]; } - - if (compareByteValue(rangeMin, minPackedValue) > 0 || compareByteValue(rangeMax, maxPackedValue) < 0) { - return PointValues.Relation.CELL_CROSSES_QUERY; - } else { + // after the loop, min < upper + // cell could be outside [min max] lower + if (!collector.withinLowerBound(maxPackedValue)) { + return PointValues.Relation.CELL_OUTSIDE_QUERY; + } + if (collector.withinRange(minPackedValue) && collector.withinRange(maxPackedValue)) { return PointValues.Relation.CELL_INSIDE_QUERY; } + return PointValues.Relation.CELL_CROSSES_QUERY; } }; } private static class RangeCollectorForPointTree { - private final BiConsumer incrementDocCount; - private final DateFieldMapper.DateFieldType fieldType; + private final BiConsumer incrementRangeDocCount; private int counter = 0; - private long[] activeRange; - private byte[][] activeRangeAsByteArray; - private final Iterator rangeIter; + private final Ranges ranges; + private int activeIndex; private int visitedRange = 0; private final int maxNumNonZeroRange; public RangeCollectorForPointTree( - BiConsumer incrementDocCount, - DateFieldMapper.DateFieldType fieldType, - Iterator rangeIter, + BiConsumer incrementRangeDocCount, int maxNumNonZeroRange, - long[] activeRange + Ranges ranges, + int activeIndex ) { - this.incrementDocCount = incrementDocCount; - this.fieldType = fieldType; - this.rangeIter = rangeIter; + this.incrementRangeDocCount = incrementRangeDocCount; this.maxNumNonZeroRange = maxNumNonZeroRange; - this.activeRange = activeRange; - this.activeRangeAsByteArray = activeRangeAsByteArray(); + this.ranges = ranges; + this.activeIndex = activeIndex; } private void count() { @@ -683,9 +798,7 @@ private void countNode(int count) { private void finalizePreviousRange() { if (counter > 0) { - logger.debug("finalize previous range: {}", activeRange[0]); - logger.debug("counter: {}", counter); - incrementDocCount.accept(fieldType.convertNanosToMillis(activeRange[0]), counter); + incrementRangeDocCount.accept(activeIndex, counter); counter = 0; } } @@ -693,29 +806,34 @@ private void finalizePreviousRange() { /** * @return true when iterator exhausted or collect enough non-zero ranges */ - private boolean iterateRangeEnd(byte[] value, BiFunction comparator) { + private boolean iterateRangeEnd(byte[] value) { // the new value may not be contiguous to the previous one // so try to find the first next range that cross the new value - while (comparator.apply(activeRangeAsByteArray[1], value) < 0) { - if (!rangeIter.hasNext()) { + while (!withinUpperBound(value)) { + if (++activeIndex >= ranges.size) { return true; } - activeRange = rangeIter.next(); - activeRangeAsByteArray = activeRangeAsByteArray(); } visitedRange++; return visitedRange > maxNumNonZeroRange; } - private byte[][] activeRangeAsByteArray() { - byte[] lower = new byte[8]; - byte[] upper = new byte[8]; - NumericUtils.longToSortableBytes(activeRange[0], lower, 0); - NumericUtils.longToSortableBytes(activeRange[1], upper, 0); - return new byte[][] { lower, upper }; + private boolean withinLowerBound(byte[] value) { + return Ranges.withinLowerBound(value, ranges.lowers[activeIndex]); + } + + private boolean withinUpperBound(byte[] value) { + return Ranges.withinUpperBound(value, ranges.uppers[activeIndex]); + } + + private boolean withinRange(byte[] value) { + return withinLowerBound(value) && withinUpperBound(value); } } + /** + * Contains debug info of BKD traversal to show in profile + */ private static class DebugInfo { private int leaf = 0; // leaf node visited private int inner = 0; // inner node visited diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/CompositeAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/CompositeAggregator.java index 3713d8f83990d..bfb484dcf478d 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/CompositeAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/CompositeAggregator.java @@ -74,6 +74,7 @@ import org.opensearch.search.aggregations.MultiBucketConsumerService; import org.opensearch.search.aggregations.bucket.BucketsAggregator; import org.opensearch.search.aggregations.bucket.FastFilterRewriteHelper; +import org.opensearch.search.aggregations.bucket.FastFilterRewriteHelper.AbstractDateHistogramAggregationType; import org.opensearch.search.aggregations.bucket.missing.MissingOrder; import org.opensearch.search.aggregations.bucket.terms.LongKeyedBucketOrds; import org.opensearch.search.internal.SearchContext; @@ -166,21 +167,22 @@ public final class CompositeAggregator extends BucketsAggregator { this.rawAfterKey = rawAfterKey; fastFilterContext = new FastFilterRewriteHelper.FastFilterContext(context); - if (!FastFilterRewriteHelper.isCompositeAggRewriteable(sourceConfigs)) return; + if (!FastFilterRewriteHelper.isCompositeAggRewriteable(sourceConfigs)) { + return; + } fastFilterContext.setAggregationType(new CompositeAggregationType()); if (fastFilterContext.isRewriteable(parent, subAggregators.length)) { // bucketOrds is used for saving date histogram results bucketOrds = LongKeyedBucketOrds.build(context.bigArrays(), CardinalityUpperBound.ONE); preparedRounding = ((CompositeAggregationType) fastFilterContext.getAggregationType()).getRoundingPrepared(); - fastFilterContext.setFieldName(sourceConfigs[0].fieldType().name()); - fastFilterContext.buildRanges(); + fastFilterContext.buildRanges(sourceConfigs[0].fieldType()); } } /** * Currently the filter rewrite is only supported for date histograms */ - public class CompositeAggregationType extends FastFilterRewriteHelper.AbstractDateHistogramAggregationType { + public class CompositeAggregationType extends AbstractDateHistogramAggregationType { private final RoundingValuesSource valuesSource; private long afterKey = -1L; @@ -549,13 +551,10 @@ private void processLeafFromQuery(LeafReaderContext ctx, Sort indexSortPrefix) t @Override protected LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCollector sub) throws IOException { - boolean optimized = FastFilterRewriteHelper.tryFastFilterAggregation( + boolean optimized = fastFilterContext.tryFastFilterAggregation( ctx, - fastFilterContext, - (key, count) -> incrementBucketDocCount( - FastFilterRewriteHelper.getBucketOrd(bucketOrds.add(0, preparedRounding.round(key))), - count - ) + this::incrementBucketDocCount, + (key) -> bucketOrds.add(0, preparedRounding.round((long) key)) ); if (optimized) throw new CollectionTerminatedException(); diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java index f326426800909..d13d575a9d696 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java @@ -64,6 +64,7 @@ import java.io.IOException; import java.util.Collections; import java.util.Map; +import java.util.Objects; import java.util.function.BiConsumer; import java.util.function.Function; import java.util.function.LongToIntFunction; @@ -157,8 +158,8 @@ private AutoDateHistogramAggregator( this.roundingPreparer = roundingPreparer; this.preparedRounding = prepareRounding(0); - fastFilterContext = new FastFilterRewriteHelper.FastFilterContext(context); - fastFilterContext.setAggregationType( + fastFilterContext = new FastFilterRewriteHelper.FastFilterContext( + context, new AutoHistogramAggregationType( valuesSourceConfig.fieldType(), valuesSourceConfig.missing() != null, @@ -166,8 +167,7 @@ private AutoDateHistogramAggregator( ) ); if (fastFilterContext.isRewriteable(parent, subAggregators.length)) { - fastFilterContext.setFieldName(valuesSourceConfig.fieldType().name()); - fastFilterContext.buildRanges(); + fastFilterContext.buildRanges(Objects.requireNonNull(valuesSourceConfig.fieldType())); } } @@ -236,13 +236,10 @@ public final LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBuc return LeafBucketCollector.NO_OP_COLLECTOR; } - boolean optimized = FastFilterRewriteHelper.tryFastFilterAggregation( + boolean optimized = fastFilterContext.tryFastFilterAggregation( ctx, - fastFilterContext, - (key, count) -> incrementBucketDocCount( - FastFilterRewriteHelper.getBucketOrd(getBucketOrds().add(0, preparedRounding.round(key))), - count - ) + this::incrementBucketDocCount, + (key) -> getBucketOrds().add(0, preparedRounding.round((long) key)) ); if (optimized) throw new CollectionTerminatedException(); diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java index dd4ee9196fd62..4b84797c18922 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java @@ -58,6 +58,7 @@ import java.io.IOException; import java.util.Collections; import java.util.Map; +import java.util.Objects; import java.util.function.BiConsumer; /** @@ -116,8 +117,8 @@ class DateHistogramAggregator extends BucketsAggregator implements SizedBucketAg bucketOrds = LongKeyedBucketOrds.build(context.bigArrays(), cardinality); - fastFilterContext = new FastFilterRewriteHelper.FastFilterContext(context); - fastFilterContext.setAggregationType( + fastFilterContext = new FastFilterRewriteHelper.FastFilterContext( + context, new DateHistogramAggregationType( valuesSourceConfig.fieldType(), valuesSourceConfig.missing() != null, @@ -126,8 +127,7 @@ class DateHistogramAggregator extends BucketsAggregator implements SizedBucketAg ) ); if (fastFilterContext.isRewriteable(parent, subAggregators.length)) { - fastFilterContext.setFieldName(valuesSourceConfig.fieldType().name()); - fastFilterContext.buildRanges(); + fastFilterContext.buildRanges(Objects.requireNonNull(valuesSourceConfig.fieldType())); } } @@ -162,13 +162,10 @@ public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, LeafBucketCol return LeafBucketCollector.NO_OP_COLLECTOR; } - boolean optimized = FastFilterRewriteHelper.tryFastFilterAggregation( + boolean optimized = fastFilterContext.tryFastFilterAggregation( ctx, - fastFilterContext, - (key, count) -> incrementBucketDocCount( - FastFilterRewriteHelper.getBucketOrd(bucketOrds.add(0, preparedRounding.round(key))), - count - ) + this::incrementBucketDocCount, + (key) -> bucketOrds.add(0, preparedRounding.round((long) key)) ); if (optimized) throw new CollectionTerminatedException(); diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/range/AbstractRangeAggregatorFactory.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/range/AbstractRangeAggregatorFactory.java index 41f2768eb7544..fd334638a0c1f 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/range/AbstractRangeAggregatorFactory.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/range/AbstractRangeAggregatorFactory.java @@ -119,7 +119,8 @@ protected Aggregator doCreateInternal( searchContext, parent, cardinality, - metadata + metadata, + config ); } diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/range/GeoDistanceAggregatorSupplier.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/range/GeoDistanceAggregatorSupplier.java index c4a9efda18bda..d72c817c4515b 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/range/GeoDistanceAggregatorSupplier.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/range/GeoDistanceAggregatorSupplier.java @@ -39,6 +39,7 @@ import org.opensearch.search.aggregations.AggregatorFactories; import org.opensearch.search.aggregations.CardinalityUpperBound; import org.opensearch.search.aggregations.support.ValuesSource; +import org.opensearch.search.aggregations.support.ValuesSourceConfig; import org.opensearch.search.internal.SearchContext; import java.io.IOException; @@ -64,6 +65,7 @@ Aggregator build( SearchContext context, Aggregator parent, CardinalityUpperBound cardinality, - Map metadata + Map metadata, + ValuesSourceConfig valuesSourceConfig ) throws IOException; } diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/range/GeoDistanceRangeAggregatorFactory.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/range/GeoDistanceRangeAggregatorFactory.java index 728f43094cf7e..f9e966deb3cc9 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/range/GeoDistanceRangeAggregatorFactory.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/range/GeoDistanceRangeAggregatorFactory.java @@ -82,7 +82,8 @@ public static void registerAggregators(ValuesSourceRegistry.Builder builder) { context, parent, cardinality, - metadata) -> { + metadata, + config) -> { DistanceSource distanceSource = new DistanceSource((ValuesSource.GeoPoint) valuesSource, distanceType, origin, units); return new RangeAggregator( name, @@ -95,7 +96,8 @@ public static void registerAggregators(ValuesSourceRegistry.Builder builder) { context, parent, cardinality, - metadata + metadata, + config ); }, true @@ -168,7 +170,8 @@ protected Aggregator doCreateInternal( searchContext, parent, cardinality, - metadata + metadata, + config ); } diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/range/RangeAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/range/RangeAggregator.java index b56b817b8177b..2ba2b06514de1 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/range/RangeAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/range/RangeAggregator.java @@ -32,6 +32,7 @@ package org.opensearch.search.aggregations.bucket.range; import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.CollectionTerminatedException; import org.apache.lucene.search.ScoreMode; import org.opensearch.core.ParseField; import org.opensearch.core.common.io.stream.StreamInput; @@ -54,7 +55,9 @@ import org.opensearch.search.aggregations.LeafBucketCollectorBase; import org.opensearch.search.aggregations.NonCollectingAggregator; import org.opensearch.search.aggregations.bucket.BucketsAggregator; +import org.opensearch.search.aggregations.bucket.FastFilterRewriteHelper; import org.opensearch.search.aggregations.support.ValuesSource; +import org.opensearch.search.aggregations.support.ValuesSourceConfig; import org.opensearch.search.internal.SearchContext; import java.io.IOException; @@ -62,6 +65,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.function.BiConsumer; import static org.opensearch.core.xcontent.ConstructingObjectParser.optionalConstructorArg; @@ -245,6 +249,8 @@ public boolean equals(Object obj) { final double[] maxTo; + private final FastFilterRewriteHelper.FastFilterContext fastFilterContext; + public RangeAggregator( String name, AggregatorFactories factories, @@ -256,17 +262,16 @@ public RangeAggregator( SearchContext context, Aggregator parent, CardinalityUpperBound cardinality, - Map metadata + Map metadata, + ValuesSourceConfig config ) throws IOException { - super(name, factories, context, parent, cardinality.multiply(ranges.length), metadata); assert valuesSource != null; this.valuesSource = valuesSource; this.format = format; this.keyed = keyed; this.rangeFactory = rangeFactory; - - this.ranges = ranges; + this.ranges = ranges; // already sorted by the range.from and range.to maxTo = new double[this.ranges.length]; maxTo[0] = this.ranges[0].to; @@ -274,6 +279,13 @@ public RangeAggregator( maxTo[i] = Math.max(this.ranges[i].to, maxTo[i - 1]); } + fastFilterContext = new FastFilterRewriteHelper.FastFilterContext( + context, + new FastFilterRewriteHelper.RangeAggregationType(config, ranges) + ); + if (fastFilterContext.isRewriteable(parent, subAggregators.length)) { + fastFilterContext.buildRanges(Objects.requireNonNull(config.fieldType())); + } } @Override @@ -286,6 +298,13 @@ public ScoreMode scoreMode() { @Override public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, final LeafBucketCollector sub) throws IOException { + boolean optimized = fastFilterContext.tryFastFilterAggregation( + ctx, + this::incrementBucketDocCount, + (activeIndex) -> subBucketOrdinal(0, (int) activeIndex) + ); + if (optimized) throw new CollectionTerminatedException(); + final SortedNumericDoubleValues values = valuesSource.doubleValues(ctx); return new LeafBucketCollectorBase(sub, values) { @Override @@ -430,4 +449,14 @@ public InternalAggregation buildEmptyAggregation() { } } + @Override + public void collectDebugInfo(BiConsumer add) { + super.collectDebugInfo(add); + if (fastFilterContext.optimizedSegments > 0) { + add.accept("optimized_segments", fastFilterContext.optimizedSegments); + add.accept("unoptimized_segments", fastFilterContext.segments - fastFilterContext.optimizedSegments); + add.accept("leaf_visited", fastFilterContext.leaf); + add.accept("inner_visited", fastFilterContext.inner); + } + } } diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/range/RangeAggregatorSupplier.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/range/RangeAggregatorSupplier.java index de9b979a56107..02b0c2e612d57 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/range/RangeAggregatorSupplier.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/range/RangeAggregatorSupplier.java @@ -36,6 +36,7 @@ import org.opensearch.search.aggregations.AggregatorFactories; import org.opensearch.search.aggregations.CardinalityUpperBound; import org.opensearch.search.aggregations.support.ValuesSource; +import org.opensearch.search.aggregations.support.ValuesSourceConfig; import org.opensearch.search.internal.SearchContext; import java.io.IOException; @@ -58,6 +59,7 @@ Aggregator build( SearchContext context, Aggregator parent, CardinalityUpperBound cardinality, - Map metadata + Map metadata, + ValuesSourceConfig config ) throws IOException; } diff --git a/server/src/test/java/org/opensearch/search/aggregations/bucket/histogram/DateHistogramAggregatorTests.java b/server/src/test/java/org/opensearch/search/aggregations/bucket/histogram/DateHistogramAggregatorTests.java index cf95999ec5086..f6e06cce6e233 100644 --- a/server/src/test/java/org/opensearch/search/aggregations/bucket/histogram/DateHistogramAggregatorTests.java +++ b/server/src/test/java/org/opensearch/search/aggregations/bucket/histogram/DateHistogramAggregatorTests.java @@ -1614,7 +1614,7 @@ public void testMultiRangeTraversalNotApplicable() throws IOException { }, true, collectCount -> assertTrue(collectCount > 0), - true + false ); } diff --git a/server/src/test/java/org/opensearch/search/aggregations/bucket/range/RangeAggregatorTests.java b/server/src/test/java/org/opensearch/search/aggregations/bucket/range/RangeAggregatorTests.java index dd7ae915c3b45..7e796b684e869 100644 --- a/server/src/test/java/org/opensearch/search/aggregations/bucket/range/RangeAggregatorTests.java +++ b/server/src/test/java/org/opensearch/search/aggregations/bucket/range/RangeAggregatorTests.java @@ -37,29 +37,44 @@ import org.apache.lucene.document.SortedSetDocValuesField; import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.store.Directory; import org.apache.lucene.tests.index.RandomIndexWriter; +import org.apache.lucene.tests.util.TestUtil; import org.apache.lucene.util.BytesRef; import org.opensearch.common.CheckedConsumer; +import org.opensearch.core.common.breaker.CircuitBreaker; +import org.opensearch.core.indices.breaker.NoneCircuitBreakerService; import org.opensearch.index.mapper.DateFieldMapper; import org.opensearch.index.mapper.KeywordFieldMapper; import org.opensearch.index.mapper.MappedFieldType; -import org.opensearch.index.mapper.NumberFieldMapper; +import org.opensearch.index.mapper.NumberFieldMapper.NumberFieldType; +import org.opensearch.index.mapper.NumberFieldMapper.NumberType; +import org.opensearch.search.aggregations.AggregationBuilder; import org.opensearch.search.aggregations.AggregatorTestCase; import org.opensearch.search.aggregations.CardinalityUpperBound; +import org.opensearch.search.aggregations.InternalAggregation; +import org.opensearch.search.aggregations.MultiBucketConsumerService; +import org.opensearch.search.aggregations.pipeline.PipelineAggregator; import org.opensearch.search.aggregations.support.AggregationInspectionHelper; import java.io.IOException; import java.time.ZoneOffset; import java.time.ZonedDateTime; import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.LinkedList; import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; import static java.util.Collections.singleton; +import static org.opensearch.test.InternalAggregationTestCase.DEFAULT_MAX_BUCKETS; import static org.hamcrest.Matchers.equalTo; public class RangeAggregatorTests extends AggregatorTestCase { @@ -199,7 +214,7 @@ public void testMissingDateWithNumberField() throws IOException { .addRange(-2d, 5d) .missing("1979-01-01T00:00:00"); - MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(NUMBER_FIELD_NAME, NumberFieldMapper.NumberType.INTEGER); + MappedFieldType fieldType = new NumberFieldType(NUMBER_FIELD_NAME, NumberType.INTEGER); expectThrows(NumberFormatException.class, () -> testCase(aggregationBuilder, new MatchAllDocsQuery(), iw -> { iw.addDocument(singleton(new NumericDocValuesField(NUMBER_FIELD_NAME, 7))); @@ -212,7 +227,7 @@ public void testUnmappedWithMissingNumber() throws IOException { .addRange(-2d, 5d) .missing(0L); - MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(NUMBER_FIELD_NAME, NumberFieldMapper.NumberType.INTEGER); + MappedFieldType fieldType = new NumberFieldType(NUMBER_FIELD_NAME, NumberType.INTEGER); testCase(aggregationBuilder, new MatchAllDocsQuery(), iw -> { iw.addDocument(singleton(new NumericDocValuesField(NUMBER_FIELD_NAME, 7))); @@ -230,7 +245,7 @@ public void testUnmappedWithMissingDate() throws IOException { .addRange(-2d, 5d) .missing("2020-02-13T10:11:12"); - MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(NUMBER_FIELD_NAME, NumberFieldMapper.NumberType.INTEGER); + MappedFieldType fieldType = new NumberFieldType(NUMBER_FIELD_NAME, NumberType.INTEGER); expectThrows(NumberFormatException.class, () -> testCase(aggregationBuilder, new MatchAllDocsQuery(), iw -> { iw.addDocument(singleton(new NumericDocValuesField(NUMBER_FIELD_NAME, 7))); @@ -257,7 +272,7 @@ public void testBadMissingField() { .addRange(-2d, 5d) .missing("bogus"); - MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(NUMBER_FIELD_NAME, NumberFieldMapper.NumberType.INTEGER); + MappedFieldType fieldType = new NumberFieldType(NUMBER_FIELD_NAME, NumberType.INTEGER); expectThrows(NumberFormatException.class, () -> testCase(aggregationBuilder, new MatchAllDocsQuery(), iw -> { iw.addDocument(singleton(new NumericDocValuesField(NUMBER_FIELD_NAME, 7))); @@ -270,7 +285,7 @@ public void testUnmappedWithBadMissingField() { .addRange(-2d, 5d) .missing("bogus"); - MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(NUMBER_FIELD_NAME, NumberFieldMapper.NumberType.INTEGER); + MappedFieldType fieldType = new NumberFieldType(NUMBER_FIELD_NAME, NumberType.INTEGER); expectThrows(NumberFormatException.class, () -> testCase(aggregationBuilder, new MatchAllDocsQuery(), iw -> { iw.addDocument(singleton(new NumericDocValuesField(NUMBER_FIELD_NAME, 7))); @@ -305,12 +320,185 @@ public void testSubAggCollectsFromManyBucketsIfManyRanges() throws IOException { }); } + public void testOverlappingRanges() throws IOException { + testRewriteOptimizationCase( + new NumberFieldType(NumberType.DOUBLE.typeName(), NumberType.DOUBLE), + new double[][] { { 1, 2 }, { 1, 1.5 }, { 0, 0.5 } }, + new MatchAllDocsQuery(), + new Number[] { 0.1, 1.1, 2.1 }, + range -> { + List ranges = range.getBuckets(); + assertEquals(3, ranges.size()); + assertEquals("0.0-0.5", ranges.get(0).getKeyAsString()); + assertEquals(1, ranges.get(0).getDocCount()); + assertEquals("1.0-1.5", ranges.get(1).getKeyAsString()); + assertEquals(1, ranges.get(1).getDocCount()); + assertEquals("1.0-2.0", ranges.get(2).getKeyAsString()); + assertEquals(1, ranges.get(2).getDocCount()); + assertTrue(AggregationInspectionHelper.hasValue(range)); + }, + false + ); + } + + /** + * @return Map [lower, upper) TO data points + */ + private Map buildRandomRanges(double[][] possibleRanges) { + Map dataSet = new LinkedHashMap<>(); + for (double[] range : possibleRanges) { + double lower = randomDoubleBetween(range[0], range[1], true); + double upper = randomDoubleBetween(range[0], range[1], true); + if (lower > upper) { + double d = lower; + lower = upper; + upper = d; + } + + int dataNumber = randomInt(200); + double[] data = new double[dataNumber]; + for (int i = 0; i < dataNumber; i++) { + data[i] = randomDoubleBetween(lower, upper, true); + } + dataSet.put(new double[] { lower, upper }, data); + } + + return dataSet; + } + + public void testRandomRanges() throws IOException { + Map dataSet = buildRandomRanges(new double[][] { { 0, 100 }, { 200, 1000 }, { 1000, 3000 } }); + + int size = dataSet.size(); + double[][] ranges = new double[size][]; + int[] expected = new int[size]; + List dataPoints = new LinkedList<>(); + + int i = 0; + for (Map.Entry entry : dataSet.entrySet()) { + ranges[i] = entry.getKey(); + expected[i] = entry.getValue().length; + for (double dataPoint : entry.getValue()) { + dataPoints.add(dataPoint); + } + i++; + } + + testRewriteOptimizationCase( + new NumberFieldType(NumberType.DOUBLE.typeName(), NumberType.DOUBLE), + ranges, + new MatchAllDocsQuery(), + dataPoints.toArray(new Number[0]), + range -> { + List rangeBuckets = range.getBuckets(); + assertEquals(size, rangeBuckets.size()); + for (int j = 0; j < rangeBuckets.size(); j++) { + assertEquals(expected[j], rangeBuckets.get(j).getDocCount()); + } + }, + true + ); + } + + public void testDoubleType() throws IOException { + testRewriteOptimizationCase( + new NumberFieldType(NumberType.DOUBLE.typeName(), NumberType.DOUBLE), + new double[][] { { 1, 2 }, { 2, 3 } }, + new MatchAllDocsQuery(), + new Number[] { 0.1, 1.1, 2.1 }, + range -> { + List ranges = range.getBuckets(); + assertEquals(2, ranges.size()); + assertEquals("1.0-2.0", ranges.get(0).getKeyAsString()); + assertEquals(1, ranges.get(0).getDocCount()); + assertEquals("2.0-3.0", ranges.get(1).getKeyAsString()); + assertEquals(1, ranges.get(1).getDocCount()); + assertTrue(AggregationInspectionHelper.hasValue(range)); + }, + true + ); + } + + public void testHalfFloatType() throws IOException { + testRewriteOptimizationCase( + new NumberFieldType(NumberType.HALF_FLOAT.typeName(), NumberType.HALF_FLOAT), + new double[][] { { 1, 2 }, { 2, 3 } }, + new MatchAllDocsQuery(), + new Number[] { 0.1, 1.1, 2.1 }, + range -> { + List ranges = range.getBuckets(); + assertEquals(2, ranges.size()); + assertEquals("1.0-2.0", ranges.get(0).getKeyAsString()); + assertEquals(1, ranges.get(0).getDocCount()); + assertEquals("2.0-3.0", ranges.get(1).getKeyAsString()); + assertEquals(1, ranges.get(1).getDocCount()); + assertTrue(AggregationInspectionHelper.hasValue(range)); + }, + true + ); + } + + public void testFloatType() throws IOException { + testRewriteOptimizationCase( + new NumberFieldType(NumberType.FLOAT.typeName(), NumberType.FLOAT), + new double[][] { { 1, 2 }, { 2, 3 } }, + new MatchAllDocsQuery(), + new Number[] { 0.1, 1.1, 2.1 }, + range -> { + List ranges = range.getBuckets(); + assertEquals(2, ranges.size()); + assertEquals("1.0-2.0", ranges.get(0).getKeyAsString()); + assertEquals(1, ranges.get(0).getDocCount()); + assertEquals("2.0-3.0", ranges.get(1).getKeyAsString()); + assertEquals(1, ranges.get(1).getDocCount()); + assertTrue(AggregationInspectionHelper.hasValue(range)); + }, + true + ); + } + + public void testUnsignedLongType() throws IOException { + testRewriteOptimizationCase( + new NumberFieldType(NumberType.UNSIGNED_LONG.typeName(), NumberType.UNSIGNED_LONG), + new double[][] { { 1, 2 }, { 2, 3 } }, + new MatchAllDocsQuery(), + new Number[] { 0, 1, 2 }, + range -> { + List ranges = range.getBuckets(); + assertEquals(2, ranges.size()); + assertEquals("1.0-2.0", ranges.get(0).getKeyAsString()); + assertEquals(1, ranges.get(0).getDocCount()); + assertEquals("2.0-3.0", ranges.get(1).getKeyAsString()); + assertEquals(1, ranges.get(1).getDocCount()); + assertTrue(AggregationInspectionHelper.hasValue(range)); + }, + true + ); + + testRewriteOptimizationCase( + new NumberFieldType(NumberType.UNSIGNED_LONG.typeName(), NumberType.UNSIGNED_LONG), + new double[][] { { Double.NEGATIVE_INFINITY, 1 }, { 2, Double.POSITIVE_INFINITY } }, + new MatchAllDocsQuery(), + new Number[] { 0, 1, 2 }, + range -> { + List ranges = range.getBuckets(); + assertEquals(2, ranges.size()); + assertEquals("*-1.0", ranges.get(0).getKeyAsString()); + assertEquals(1, ranges.get(0).getDocCount()); + assertEquals("2.0-*", ranges.get(1).getKeyAsString()); + assertEquals(1, ranges.get(1).getDocCount()); + assertTrue(AggregationInspectionHelper.hasValue(range)); + }, + true + ); + } + private void testCase( Query query, CheckedConsumer buildIndex, Consumer> verify ) throws IOException { - MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(NUMBER_FIELD_NAME, NumberFieldMapper.NumberType.INTEGER); + MappedFieldType fieldType = new NumberFieldType(NUMBER_FIELD_NAME, NumberType.INTEGER); RangeAggregationBuilder aggregationBuilder = new RangeAggregationBuilder("test_range_agg"); aggregationBuilder.field(NUMBER_FIELD_NAME); aggregationBuilder.addRange(0d, 5d); @@ -323,9 +511,9 @@ private void simpleTestCase( Query query, Consumer> verify ) throws IOException { - MappedFieldType fieldType = new NumberFieldMapper.NumberFieldType(NUMBER_FIELD_NAME, NumberFieldMapper.NumberType.INTEGER); + MappedFieldType fieldType = new NumberFieldType(NUMBER_FIELD_NAME, NumberType.INTEGER); - testCase(aggregationBuilder, new MatchAllDocsQuery(), iw -> { + testCase(aggregationBuilder, query, iw -> { iw.addDocument(singleton(new SortedNumericDocValuesField(NUMBER_FIELD_NAME, 7))); iw.addDocument(singleton(new SortedNumericDocValuesField(NUMBER_FIELD_NAME, 2))); iw.addDocument(singleton(new SortedNumericDocValuesField(NUMBER_FIELD_NAME, 3))); @@ -354,8 +542,84 @@ private void testCase( fieldType ); verify.accept(agg); + } + } + } + private void testRewriteOptimizationCase( + NumberFieldType fieldType, + double[][] ranges, + Query query, + Number[] dataPoints, + Consumer> verify, + boolean optimized + ) throws IOException { + NumberType numberType = fieldType.numberType(); + String fieldName = numberType.typeName(); + + try (Directory directory = newDirectory()) { + try (IndexWriter indexWriter = new IndexWriter(directory, new IndexWriterConfig().setCodec(TestUtil.getDefaultCodec()))) { + for (Number dataPoint : dataPoints) { + indexWriter.addDocument(numberType.createFields(fieldName, dataPoint, true, true, false)); + } + } + + try (IndexReader indexReader = DirectoryReader.open(directory)) { + IndexSearcher indexSearcher = newSearcher(indexReader, true, true); + + RangeAggregationBuilder aggregationBuilder = new RangeAggregationBuilder("range").field(fieldName); + for (double[] range : ranges) { + aggregationBuilder.addRange(range[0], range[1]); + } + + CountingAggregator aggregator = createCountingAggregator(query, aggregationBuilder, indexSearcher, fieldType); + aggregator.preCollection(); + indexSearcher.search(query, aggregator); + aggregator.postCollection(); + + MultiBucketConsumerService.MultiBucketConsumer reduceBucketConsumer = new MultiBucketConsumerService.MultiBucketConsumer( + Integer.MAX_VALUE, + new NoneCircuitBreakerService().getBreaker(CircuitBreaker.REQUEST) + ); + InternalAggregation.ReduceContext context = InternalAggregation.ReduceContext.forFinalReduction( + aggregator.context().bigArrays(), + getMockScriptService(), + reduceBucketConsumer, + PipelineAggregator.PipelineTree.EMPTY + ); + InternalRange topLevel = (InternalRange) aggregator.buildTopLevel(); + InternalRange agg = (InternalRange) topLevel.reduce(Collections.singletonList(topLevel), context); + doAssertReducedMultiBucketConsumer(agg, reduceBucketConsumer); + + verify.accept(agg); + + if (optimized) { + assertEquals(0, aggregator.getCollectCount().get()); + } else { + assertTrue(aggregator.getCollectCount().get() > 0); + } } } } + + protected CountingAggregator createCountingAggregator( + Query query, + AggregationBuilder builder, + IndexSearcher searcher, + MappedFieldType... fieldTypes + ) throws IOException { + return new CountingAggregator( + new AtomicInteger(), + createAggregator( + query, + builder, + searcher, + new MultiBucketConsumerService.MultiBucketConsumer( + DEFAULT_MAX_BUCKETS, + new NoneCircuitBreakerService().getBreaker(CircuitBreaker.REQUEST) + ), + fieldTypes + ) + ); + } } From f80f3cdc18b7dbdcbcc372527a6bcbc71e405739 Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Wed, 19 Jun 2024 18:38:03 -0500 Subject: [PATCH 42/84] Update @dreamer-89 maintainership status (#14461) Signed-off-by: Suraj Singh --- .github/CODEOWNERS | 4 ++-- MAINTAINERS.md | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index b5b0a815b02b2..7b685fcc932ea 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -11,7 +11,7 @@ # 3. Use the command palette to run the CODEOWNERS: Show owners of current file command, which will display all code owners for the current file. # Default ownership for all repo files -* @anasalkouz @andrross @Bukhtawar @CEHENKLE @dblock @dbwiddis @dreamer-89 @gbbafna @kotwanikunal @mch2 @msfroh @nknize @owaiskazi19 @reta @Rishikesh1159 @sachinpkale @saratvemulapalli @shwetathareja @sohami @tlfeng @VachaShah +* @anasalkouz @andrross @Bukhtawar @CEHENKLE @dblock @dbwiddis @gbbafna @kotwanikunal @mch2 @msfroh @nknize @owaiskazi19 @reta @Rishikesh1159 @sachinpkale @saratvemulapalli @shwetathareja @sohami @tlfeng @VachaShah /modules/transport-netty4/ @peternied @@ -24,4 +24,4 @@ /.github/ @peternied -/MAINTAINERS.md @anasalkouz @andrross @Bukhtawar @CEHENKLE @dblock @dbwiddis @dreamer-89 @gbbafna @jed326 @kotwanikunal @mch2 @msfroh @nknize @owaiskazi19 @peternied @reta @Rishikesh1159 @sachinpkale @saratvemulapalli @shwetathareja @sohami @tlfeng @VachaShah +/MAINTAINERS.md @anasalkouz @andrross @Bukhtawar @CEHENKLE @dblock @dbwiddis @gbbafna @jed326 @kotwanikunal @mch2 @msfroh @nknize @owaiskazi19 @peternied @reta @Rishikesh1159 @sachinpkale @saratvemulapalli @shwetathareja @sohami @tlfeng @VachaShah diff --git a/MAINTAINERS.md b/MAINTAINERS.md index 6855281a488ca..b014da1157678 100644 --- a/MAINTAINERS.md +++ b/MAINTAINERS.md @@ -26,7 +26,6 @@ This document contains a list of maintainers in this repo. See [opensearch-proje | Sarat Vemulapalli | [saratvemulapalli](https://github.com/saratvemulapalli) | Amazon | | Shweta Thareja | [shwetathareja](https://github.com/shwetathareja) | Amazon | | Sorabh Hamirwasia | [sohami](https://github.com/sohami) | Amazon | -| Suraj Singh | [dreamer-89](https://github.com/dreamer-89) | Amazon | | Tianli Feng | [tlfeng](https://github.com/tlfeng) | Amazon | | Vacha Shah | [VachaShah](https://github.com/VachaShah) | Amazon | @@ -41,3 +40,4 @@ This document contains a list of maintainers in this repo. See [opensearch-proje | Himanshu Setia | [setiah](https://github.com/setiah) | Amazon | | Ryan Bogan | [ryanbogan](https://github.com/ryanbogan) | Amazon | | Rabi Panda | [adnapibar](https://github.com/adnapibar) | Independent | +| Suraj Singh | [dreamer-89](https://github.com/dreamer-89) | Amazon | From 9729a92c9b60ba95165f12baf3735813b4ea855e Mon Sep 17 00:00:00 2001 From: Andrew Ross Date: Wed, 19 Jun 2024 18:37:46 -0700 Subject: [PATCH 43/84] Move maintainer tlfeng to emeritus (#14466) --- .github/CODEOWNERS | 4 ++-- MAINTAINERS.md | 22 +++++++++++----------- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index 7b685fcc932ea..5a2d08756c49f 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -11,7 +11,7 @@ # 3. Use the command palette to run the CODEOWNERS: Show owners of current file command, which will display all code owners for the current file. # Default ownership for all repo files -* @anasalkouz @andrross @Bukhtawar @CEHENKLE @dblock @dbwiddis @gbbafna @kotwanikunal @mch2 @msfroh @nknize @owaiskazi19 @reta @Rishikesh1159 @sachinpkale @saratvemulapalli @shwetathareja @sohami @tlfeng @VachaShah +* @anasalkouz @andrross @Bukhtawar @CEHENKLE @dblock @dbwiddis @gbbafna @kotwanikunal @mch2 @msfroh @nknize @owaiskazi19 @reta @Rishikesh1159 @sachinpkale @saratvemulapalli @shwetathareja @sohami @VachaShah /modules/transport-netty4/ @peternied @@ -24,4 +24,4 @@ /.github/ @peternied -/MAINTAINERS.md @anasalkouz @andrross @Bukhtawar @CEHENKLE @dblock @dbwiddis @gbbafna @jed326 @kotwanikunal @mch2 @msfroh @nknize @owaiskazi19 @peternied @reta @Rishikesh1159 @sachinpkale @saratvemulapalli @shwetathareja @sohami @tlfeng @VachaShah +/MAINTAINERS.md @anasalkouz @andrross @Bukhtawar @CEHENKLE @dblock @dbwiddis @gbbafna @jed326 @kotwanikunal @mch2 @msfroh @nknize @owaiskazi19 @peternied @reta @Rishikesh1159 @sachinpkale @saratvemulapalli @shwetathareja @sohami @VachaShah diff --git a/MAINTAINERS.md b/MAINTAINERS.md index b014da1157678..91b57a4cbc74e 100644 --- a/MAINTAINERS.md +++ b/MAINTAINERS.md @@ -26,18 +26,18 @@ This document contains a list of maintainers in this repo. See [opensearch-proje | Sarat Vemulapalli | [saratvemulapalli](https://github.com/saratvemulapalli) | Amazon | | Shweta Thareja | [shwetathareja](https://github.com/shwetathareja) | Amazon | | Sorabh Hamirwasia | [sohami](https://github.com/sohami) | Amazon | -| Tianli Feng | [tlfeng](https://github.com/tlfeng) | Amazon | | Vacha Shah | [VachaShah](https://github.com/VachaShah) | Amazon | ## Emeritus -| Maintainer | GitHub ID | Affiliation | -| --------------------- | ----------------------------------------- | ----------- | -| Megha Sai Kavikondala | [meghasaik](https://github.com/meghasaik) | Amazon | -| Xue Zhou | [xuezhou25](https://github.com/xuezhou25) | Amazon | -| Kartik Ganesh | [kartg](https://github.com/kartg) | Amazon | -| Abbas Hussain | [abbashus](https://github.com/abbashus) | Meta | -| Himanshu Setia | [setiah](https://github.com/setiah) | Amazon | -| Ryan Bogan | [ryanbogan](https://github.com/ryanbogan) | Amazon | -| Rabi Panda | [adnapibar](https://github.com/adnapibar) | Independent | -| Suraj Singh | [dreamer-89](https://github.com/dreamer-89) | Amazon | +| Maintainer | GitHub ID | Affiliation | +| ---------------------- |-------------------------------------------- | ----------- | +| Megha Sai Kavikondala | [meghasaik](https://github.com/meghasaik) | Amazon | +| Xue Zhou | [xuezhou25](https://github.com/xuezhou25) | Amazon | +| Kartik Ganesh | [kartg](https://github.com/kartg) | Amazon | +| Abbas Hussain | [abbashus](https://github.com/abbashus) | Meta | +| Himanshu Setia | [setiah](https://github.com/setiah) | Amazon | +| Ryan Bogan | [ryanbogan](https://github.com/ryanbogan) | Amazon | +| Rabi Panda | [adnapibar](https://github.com/adnapibar) | Independent | +| Tianli Feng | [tlfeng](https://github.com/tlfeng) | Amazon | +| Suraj Singh | [dreamer-89](https://github.com/dreamer-89) | Amazon | From e59d77a8a3334981f9da1bdd14aed70b3a936e9f Mon Sep 17 00:00:00 2001 From: Andrew Ross Date: Thu, 20 Jun 2024 05:56:22 -0700 Subject: [PATCH 44/84] Add more details to testing bad practices (#14455) These are a few cases I have seen that have resulted in flaky tests. I would love to see more details added here so that this can be used as a sort of checklist when writing, reviewing, or trying to fix tests. Signed-off-by: Andrew Ross --- TESTING.md | 26 +++++++++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/TESTING.md b/TESTING.md index 80fc2412d736b..3928b2fc93afa 100644 --- a/TESTING.md +++ b/TESTING.md @@ -33,6 +33,9 @@ OpenSearch uses [jUnit](https://junit.org/junit5/) for testing, it also uses ran - [Bad practices](#bad-practices) - [Use randomized-testing for coverage](#use-randomized-testing-for-coverage) - [Abuse randomization in multi-threaded tests](#abuse-randomization-in-multi-threaded-tests) + - [Use `Thread.sleep`](#use-threadsleep) + - [Expect a specific segment topology](#expect-a-specific-segment-topology) + - [Leave environment in an unstable state after test](#leave-environment-in-an-unstable-state-after-test) - [Test coverage analysis](#test-coverage-analysis) - [Building with extra plugins](#building-with-extra-plugins) - [Environment misc](#environment-misc) @@ -455,7 +458,7 @@ Unit tests are the preferred way to test some functionality: most of the time th The reason why `OpenSearchSingleNodeTestCase` exists is that all our components used to be very hard to set up in isolation, which had led us to having a number of integration tests but close to no unit tests. `OpenSearchSingleNodeTestCase` is a workaround for this issue which provides an easy way to spin up a node and get access to components that are hard to instantiate like `IndicesService`. Whenever practical, you should prefer unit tests. -Finally, if the the functionality under test needs to be run in a cluster, there are two test classes to consider: +Finally, if the functionality under test needs to be run in a cluster, there are two test classes to consider: * `OpenSearchRestTestCase` will connect to an external cluster. This is a good option if the tests cases don't rely on a specific configuration of the test cluster. A test cluster is set up as part of the Gradle task running integration tests, and test cases using this class can connect to it. The configuration of the cluster is provided in the Gradle files. * `OpenSearchIntegTestCase` will create a local cluster as part of each test case. The configuration of the cluster is controlled by the test class. This is a good option if different tests cases depend on different cluster configurations, as it would be impractical (and limit parallelization) to keep re-configuring (and re-starting) the external cluster for each test case. A good example of when this class might come in handy is for testing security features, where different cluster configurations are needed to fully test each one. @@ -477,6 +480,27 @@ However, it should not be used for coverage. For instance if you are testing a p Multi-threaded tests are often not reproducible due to the fact that there is no guarantee on the order in which operations occur across threads. Adding randomization to the mix usually makes things worse and should be done with care. +### Use `Thread.sleep` + +`Thread.sleep()` is almost always a bad idea because it is very difficult to know that you've waited long enough. Using primitives like `waitUntil` or `assertBusy`, which use Thread.sleep internally, is okay to wait for a specific condition. However, it is almost always better to instrument your code with concurrency primitives like a `CountDownLatch` that will allow you to deterministically wait for a specific condition, without waiting longer than necessary that will happen with a polling approach used by `assertBusy`. + +Example: +- [PrimaryShardAllocatorIT](https://github.com/opensearch-project/OpenSearch/blob/7ffcd6500e0bd5956cef5c289ee66d9f99d533fc/server/src/internalClusterTest/java/org/opensearch/gateway/ReplicaShardAllocatorIT.java#L208-L235): This test is using two latches: one to wait for a recovery to start and one to block that recovery so that it can deterministically test things that happen during a recovery. + +### Expect a specific segment topology + +By design, OpenSearch integration tests will vary how the merge policy works because in almost all scenarios you should not depend on a specific segment topology (in the real world your code will see a huge diversity of indexing workloads with OpenSearch merging things in the background all the time!). If you do in fact need to care about the segment topology (e.g. for testing statistics that might vary slightly depending on number of segments), then you must take care to ensure that segment topology is deterministic by doing things like disabling background refreshes, force merging after indexing data, etc. + +Example: +- [SegmentReplicationResizeRequestIT](https://github.com/opensearch-project/OpenSearch/blob/f715ee1a485e550802accc1c2e3d8101208d4f0b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationResizeRequestIT.java#L102-L109): This test disables refreshes to prevent interfering with the segment replication behavior under test. + +### Leave environment in an unstable state after test + +The default test case will ensure that no open file handles or running threads are left after tear down. You must ensure that all resources are cleaned up at the end of each test case, or else the cleanup may end up racing with the tear down logic in the base test class in a way that is very difficult to reproduce. + +Example: +- [AwarenessAttributeDecommissionIT](https://github.com/opensearch-project/OpenSearch/blob/main/server/src/internalClusterTest/java/org/opensearch/cluster/coordination/AwarenessAttributeDecommissionIT.java#L951): Recommissions any decommissioned nodes at the end of the test to ensure the after-test checks succeed. + # Test coverage analysis The code coverage report can be generated through Gradle with [JaCoCo plugin](https://docs.gradle.org/current/userguide/jacoco_plugin.html). From 761e207f9f50a482461a049e8e9da287eeddf83f Mon Sep 17 00:00:00 2001 From: kkewwei Date: Thu, 20 Jun 2024 22:13:23 +0800 Subject: [PATCH 45/84] Fix the computed max shards of cluster to avoid int overflow (#14155) Signed-off-by: kkewwei --- CHANGELOG.md | 1 + .../indices/ShardLimitValidator.java | 7 ++++--- .../indices/ShardLimitValidatorTests.java | 19 +++++++++++++++++++ 3 files changed, 24 insertions(+), 3 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 8c3e63e36bc82..cfeb02d5a7b06 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -33,6 +33,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ### Fixed - Fix handling of Short and Byte data types in ScriptProcessor ingest pipeline ([#14379](https://github.com/opensearch-project/OpenSearch/issues/14379)) - Switch to iterative version of WKT format parser ([#14086](https://github.com/opensearch-project/OpenSearch/pull/14086)) +- Fix the computed max shards of cluster to avoid int overflow ([#14155](https://github.com/opensearch-project/OpenSearch/pull/14155)) ### Security diff --git a/server/src/main/java/org/opensearch/indices/ShardLimitValidator.java b/server/src/main/java/org/opensearch/indices/ShardLimitValidator.java index e345b613eebbd..94e91e2d4c3ac 100644 --- a/server/src/main/java/org/opensearch/indices/ShardLimitValidator.java +++ b/server/src/main/java/org/opensearch/indices/ShardLimitValidator.java @@ -261,14 +261,15 @@ static Optional checkShardLimit( return Optional.empty(); } + int computedMaxShards = (int) Math.min(Integer.MAX_VALUE, (long) maxShardsPerNodeSetting * nodeCount); int maxShardsInCluster = maxShardsPerClusterSetting; if (maxShardsInCluster == -1) { - maxShardsInCluster = maxShardsPerNodeSetting * nodeCount; + maxShardsInCluster = computedMaxShards; } else { - maxShardsInCluster = Math.min(maxShardsInCluster, maxShardsPerNodeSetting * nodeCount); + maxShardsInCluster = Math.min(maxShardsInCluster, computedMaxShards); } - int currentOpenShards = state.getMetadata().getTotalOpenIndexShards(); + long currentOpenShards = state.getMetadata().getTotalOpenIndexShards(); if ((currentOpenShards + newShards) > maxShardsInCluster) { String errorMessage = "this action would add [" + newShards diff --git a/server/src/test/java/org/opensearch/indices/ShardLimitValidatorTests.java b/server/src/test/java/org/opensearch/indices/ShardLimitValidatorTests.java index 040632ea3ed8d..0b1ec8fd85ae5 100644 --- a/server/src/test/java/org/opensearch/indices/ShardLimitValidatorTests.java +++ b/server/src/test/java/org/opensearch/indices/ShardLimitValidatorTests.java @@ -214,6 +214,25 @@ public void testNonSystemIndexCreationFailsWithMaxShardLimitOnCluster() { ); } + public void testComputedMaxShardsOfClusterIntOverFlow() { + final int maxShardLimitPerNode = 500_000_000; + ClusterState state = createClusterForShardLimitTest(15, 1, 1); + Optional errorMessage = ShardLimitValidator.checkShardLimit(2, state, maxShardLimitPerNode, -1); + assertFalse(errorMessage.isPresent()); + + errorMessage = ShardLimitValidator.checkShardLimit(Integer.MAX_VALUE - 1, state, maxShardLimitPerNode, -1); + assertEquals( + "this action would add [" + + (Integer.MAX_VALUE - 1) + + "] total shards, but this cluster currently has [" + + 2 + + "]/[" + + Integer.MAX_VALUE + + "] maximum shards open", + errorMessage.get() + ); + } + public void testNonSystemIndexCreationPassesWithMaxShardLimitOnCluster() { final int maxShardLimitOnCluster = 5; Settings limitOnlySettings = Settings.builder() From d2c08b320b49920e142ec70fa66621305fd4ccf8 Mon Sep 17 00:00:00 2001 From: Skyring100 <106502383+Skyring100@users.noreply.github.com> Date: Thu, 20 Jun 2024 07:29:16 -0700 Subject: [PATCH 46/84] Optimize UnsignedLong range queries to convert to MatchNoDocsQuery when lower > upper bounds (#14416) * Added check for lower > upper at end of function Signed-off-by: Skyring100 <106502383+Skyring100@users.noreply.github.com> * Fixed mistake of using < operator on BigInteger, now using compareTo Signed-off-by: Skyring100 <106502383+Skyring100@users.noreply.github.com> * Fixed simple mistake of flipping > operator Signed-off-by: Skyring100 <106502383+Skyring100@users.noreply.github.com> * Fixed space formatting Signed-off-by: Skyring100 <106502383+Skyring100@users.noreply.github.com> * Updated CHANGELOG.md Signed-off-by: Skyring100 <106502383+Skyring100@users.noreply.github.com> * Issue number linked in CHANGELOG.md Signed-off-by: Skyring100 <106502383+Skyring100@users.noreply.github.com> * doTestDocValueRangeQueries now accepts MatchNoDocsQuery alongside IndexOrDocValuesQuery Signed-off-by: Skyring100 <106502383+Skyring100@users.noreply.github.com> * dotestdoTestDocValueRangeQueries only checks indexQuery and randomAccessQuery only when query is type IndexIndexOrDocValuesQuery Signed-off-by: Skyring100 <106502383+Skyring100@users.noreply.github.com> * Ran gradlew spotlessApply to fix import formatting issues Signed-off-by: Skyring100 <106502383+Skyring100@users.noreply.github.com> * Imported Matchers.either method instead of entire Matchers class Signed-off-by: Skyring100 <106502383+Skyring100@users.noreply.github.com> --------- Signed-off-by: Skyring100 <106502383+Skyring100@users.noreply.github.com> Signed-off-by: Andriy Redko Co-authored-by: Andriy Redko --- CHANGELOG.md | 1 + .../org/opensearch/index/mapper/NumberFieldMapper.java | 3 +++ .../opensearch/index/mapper/NumberFieldTypeTests.java | 9 ++++++--- 3 files changed, 10 insertions(+), 3 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index cfeb02d5a7b06..6d14f55d56209 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -23,6 +23,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `opentelemetry` from 1.36.0 to 1.39.0 ([#14457](https://github.com/opensearch-project/OpenSearch/pull/14457)) ### Changed +- unsignedLongRangeQuery now returns MatchNoDocsQuery if the lower bounds are greater than the upper bounds ([#14416](https://github.com/opensearch-project/OpenSearch/pull/14416)) - Updated the `indices.query.bool.max_clause_count` setting from being static to dynamically updateable ([#13568](https://github.com/opensearch-project/OpenSearch/pull/13568)) - Make the class CommunityIdProcessor final ([#14448](https://github.com/opensearch-project/OpenSearch/pull/14448)) diff --git a/server/src/main/java/org/opensearch/index/mapper/NumberFieldMapper.java b/server/src/main/java/org/opensearch/index/mapper/NumberFieldMapper.java index 25e5f9970795f..27e62c3746a8e 100644 --- a/server/src/main/java/org/opensearch/index/mapper/NumberFieldMapper.java +++ b/server/src/main/java/org/opensearch/index/mapper/NumberFieldMapper.java @@ -1410,6 +1410,9 @@ public static Query unsignedLongRangeQuery( u = u.subtract(BigInteger.ONE); } } + if (l.compareTo(u) > 0) { + return new MatchNoDocsQuery(); + } return builder.apply(l, u); } } diff --git a/server/src/test/java/org/opensearch/index/mapper/NumberFieldTypeTests.java b/server/src/test/java/org/opensearch/index/mapper/NumberFieldTypeTests.java index af852b12e7a30..96487db6dd512 100644 --- a/server/src/test/java/org/opensearch/index/mapper/NumberFieldTypeTests.java +++ b/server/src/test/java/org/opensearch/index/mapper/NumberFieldTypeTests.java @@ -86,6 +86,7 @@ import java.util.function.Supplier; import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.either; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; @@ -673,9 +674,11 @@ public void doTestDocValueRangeQueries(NumberType type, Supplier valueSu true, MOCK_QSC ); - assertThat(query, instanceOf(IndexOrDocValuesQuery.class)); - IndexOrDocValuesQuery indexOrDvQuery = (IndexOrDocValuesQuery) query; - assertEquals(searcher.count(indexOrDvQuery.getIndexQuery()), searcher.count(indexOrDvQuery.getRandomAccessQuery())); + assertThat(query, either(instanceOf(IndexOrDocValuesQuery.class)).or(instanceOf(MatchNoDocsQuery.class))); + if (query instanceof IndexOrDocValuesQuery) { + IndexOrDocValuesQuery indexOrDvQuery = (IndexOrDocValuesQuery) query; + assertEquals(searcher.count(indexOrDvQuery.getIndexQuery()), searcher.count(indexOrDvQuery.getRandomAccessQuery())); + } } reader.close(); dir.close(); From b8c78196438897132f6819460ebb7d4222b39297 Mon Sep 17 00:00:00 2001 From: rayshrey <121871912+rayshrey@users.noreply.github.com> Date: Thu, 20 Jun 2024 22:53:18 +0530 Subject: [PATCH 47/84] [Writable Warm] Composite Directory implementation and integrating it with FileCache (#12782) * Composite Directory POC Signed-off-by: Shreyansh Ray * Refactor TransferManager interface to RemoteStoreFileTrackerAdapter Signed-off-by: Shreyansh Ray * Implement block level fetch for Composite Directory Signed-off-by: Shreyansh Ray * Removed CACHE state from FileTracker Signed-off-by: Shreyansh Ray * Fixes after latest pull Signed-off-by: Shreyansh Ray * Add new setting for warm, remove store type setting, FileTracker and RemoteStoreFileTrackerAdapter, CompositeDirectoryFactory and update Composite Directory implementation Signed-off-by: Shreyansh Ray * Modify TransferManager - replace BlobContainer with Functional Interface to fetch an InputStream instead Signed-off-by: Shreyansh Ray * Reuse OnDemandBlockSnapshotIndexInput instead of OnDemandBlockCompositeIndexInput Signed-off-by: Shreyansh Ray * Modify constructors to avoid breaking public api contract and code review fixes Signed-off-by: Shreyansh Ray * Add experimental annotations for newly created classes and review comment fixes Signed-off-by: Shreyansh Ray * Use ref count as a temporary measure to prevent file from eviction until uploaded to Remote Signed-off-by: Shreyansh Ray * Remove method level locks Signed-off-by: Shreyansh Ray * Handle tmp file deletion Signed-off-by: Shreyansh Ray * Nit fixes Signed-off-by: Shreyansh Ray * Handle delete and close in Composite Directory, log current state of FileCache and correct it's clear method and modify unit and integration tests as per review comments Signed-off-by: Shreyansh Ray * Refactor usages of WRITEABLE_REMOTE_INDEX_SETTING to TIERED_REMOTE_INDEX_SETTING Signed-off-by: Shreyansh Ray * Add tests for FileCachedIndexInput and review comment fixes Signed-off-by: Shreyansh Ray * Add additional IT for feature flag disabled Signed-off-by: Shreyansh Ray * Move setting for Partial Locality type behind Feature Flag, fix bug for ref count via cloneMap in FullFileCachedIndexInput and other review fixes Signed-off-by: Shreyansh Ray * Minor test and nit fixes Signed-off-by: Shreyansh Ray * Add javadocs for FullFileCachedIndexInput Signed-off-by: Shreyansh Ray * Minor precommit fixes Signed-off-by: Shreyansh Ray --------- Signed-off-by: Shreyansh Ray --- CHANGELOG.md | 1 + .../remotestore/WritableWarmIT.java | 160 ++++++++ .../common/settings/IndexScopedSettings.java | 6 +- .../org/opensearch/index/IndexModule.java | 78 +++- .../org/opensearch/index/IndexService.java | 96 ++++- .../org/opensearch/index/IndexSettings.java | 12 + .../shard/RemoteStoreRefreshListener.java | 5 + .../store/CloseableFilterIndexOutput.java | 53 +++ .../index/store/CompositeDirectory.java | 344 ++++++++++++++++++ .../index/store/RemoteDirectory.java | 26 +- .../RemoteSnapshotDirectoryFactory.java | 2 +- .../file/OnDemandBlockSnapshotIndexInput.java | 14 +- .../filecache/CachedFullFileIndexInput.java | 74 ++++ .../store/remote/filecache/FileCache.java | 11 + .../filecache/FileCachedIndexInput.java | 4 +- .../filecache/FullFileCachedIndexInput.java | 108 ++++++ .../store/remote/utils/BlockIOContext.java | 53 +++ .../store/remote/utils/FileTypeUtils.java | 40 ++ .../store/remote/utils/TransferManager.java | 33 +- .../store/remote/utils/cache/LRUCache.java | 26 +- .../remote/utils/cache/SegmentedCache.java | 13 + .../opensearch/indices/IndicesService.java | 73 +++- .../main/java/org/opensearch/node/Node.java | 11 +- .../BaseRemoteSegmentStoreDirectoryTests.java | 178 +++++++++ .../index/store/CompositeDirectoryTests.java | 202 ++++++++++ .../index/store/RemoteDirectoryTests.java | 13 + .../RemoteSegmentStoreDirectoryTests.java | 153 +------- .../OnDemandBlockSnapshotIndexInputTests.java | 4 +- .../filecache/FileCachedIndexInputTests.java | 79 ++++ .../FullFileCachedIndexInputTests.java | 79 ++++ ...ansferManagerBlobContainerReaderTests.java | 43 +++ ...sferManagerRemoteDirectoryReaderTests.java | 51 +++ ...ests.java => TransferManagerTestCase.java} | 37 +- .../utils/cache/RefCountedCacheTestCase.java | 9 + .../test/OpenSearchIntegTestCase.java | 6 +- 35 files changed, 1889 insertions(+), 208 deletions(-) create mode 100644 server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java create mode 100644 server/src/main/java/org/opensearch/index/store/CloseableFilterIndexOutput.java create mode 100644 server/src/main/java/org/opensearch/index/store/CompositeDirectory.java create mode 100644 server/src/main/java/org/opensearch/index/store/remote/filecache/CachedFullFileIndexInput.java create mode 100644 server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInput.java create mode 100644 server/src/main/java/org/opensearch/index/store/remote/utils/BlockIOContext.java create mode 100644 server/src/main/java/org/opensearch/index/store/remote/utils/FileTypeUtils.java create mode 100644 server/src/test/java/org/opensearch/index/store/BaseRemoteSegmentStoreDirectoryTests.java create mode 100644 server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java create mode 100644 server/src/test/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInputTests.java create mode 100644 server/src/test/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInputTests.java create mode 100644 server/src/test/java/org/opensearch/index/store/remote/utils/TransferManagerBlobContainerReaderTests.java create mode 100644 server/src/test/java/org/opensearch/index/store/remote/utils/TransferManagerRemoteDirectoryReaderTests.java rename server/src/test/java/org/opensearch/index/store/remote/utils/{TransferManagerTests.java => TransferManagerTestCase.java} (87%) diff --git a/CHANGELOG.md b/CHANGELOG.md index 6d14f55d56209..47244ba9afbb1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -8,6 +8,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add fingerprint ingest processor ([#13724](https://github.com/opensearch-project/OpenSearch/pull/13724)) - [Remote Store] Rate limiter for remote store low priority uploads ([#14374](https://github.com/opensearch-project/OpenSearch/pull/14374/)) - Apply the date histogram rewrite optimization to range aggregation ([#13865](https://github.com/opensearch-project/OpenSearch/pull/13865)) +- [Writable Warm] Add composite directory implementation and integrate it with FileCache ([12782](https://github.com/opensearch-project/OpenSearch/pull/12782)) ### Dependencies - Bump `org.gradle.test-retry` from 1.5.8 to 1.5.9 ([#13442](https://github.com/opensearch-project/OpenSearch/pull/13442)) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java new file mode 100644 index 0000000000000..a51bd6b20fff0 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java @@ -0,0 +1,160 @@ +/* + * 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.remotestore; + +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; + +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.FilterDirectory; +import org.opensearch.action.admin.indices.delete.DeleteIndexRequest; +import org.opensearch.action.admin.indices.get.GetIndexRequest; +import org.opensearch.action.admin.indices.get.GetIndexResponse; +import org.opensearch.action.search.SearchResponse; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.settings.SettingsException; +import org.opensearch.common.util.FeatureFlags; +import org.opensearch.index.IndexModule; +import org.opensearch.index.query.QueryBuilders; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.store.CompositeDirectory; +import org.opensearch.index.store.remote.file.CleanerDaemonThreadLeakFilter; +import org.opensearch.index.store.remote.filecache.FileCache; +import org.opensearch.index.store.remote.utils.FileTypeUtils; +import org.opensearch.indices.IndicesService; +import org.opensearch.node.Node; +import org.opensearch.test.InternalTestCluster; +import org.opensearch.test.OpenSearchIntegTestCase; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount; + +@ThreadLeakFilters(filters = CleanerDaemonThreadLeakFilter.class) +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0, supportsDedicatedMasters = false) +// Uncomment the below line to enable trace level logs for this test for better debugging +// @TestLogging(reason = "Getting trace logs from composite directory package", value = "org.opensearch.index.store:TRACE") +public class WritableWarmIT extends RemoteStoreBaseIntegTestCase { + + protected static final String INDEX_NAME = "test-idx-1"; + protected static final int NUM_DOCS_IN_BULK = 1000; + + /* + Disabling MockFSIndexStore plugin as the MockFSDirectoryFactory wraps the FSDirectory over a OpenSearchMockDirectoryWrapper which extends FilterDirectory (whereas FSDirectory extends BaseDirectory) + As a result of this wrapping the local directory of Composite Directory does not satisfy the assertion that local directory must be of type FSDirectory + */ + @Override + protected boolean addMockIndexStorePlugin() { + return false; + } + + @Override + protected Settings featureFlagSettings() { + Settings.Builder featureSettings = Settings.builder(); + featureSettings.put(FeatureFlags.TIERED_REMOTE_INDEX, true); + return featureSettings.build(); + } + + public void testWritableWarmFeatureFlagDisabled() { + Settings clusterSettings = Settings.builder().put(super.nodeSettings(0)).put(FeatureFlags.TIERED_REMOTE_INDEX, false).build(); + InternalTestCluster internalTestCluster = internalCluster(); + internalTestCluster.startClusterManagerOnlyNode(clusterSettings); + internalTestCluster.startDataOnlyNode(clusterSettings); + + Settings indexSettings = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexModule.INDEX_STORE_LOCALITY_SETTING.getKey(), IndexModule.DataLocalityType.PARTIAL.name()) + .build(); + + try { + prepareCreate(INDEX_NAME).setSettings(indexSettings).get(); + fail("Should have thrown Exception as setting should not be registered if Feature Flag is Disabled"); + } catch (SettingsException ex) { + assertEquals( + "unknown setting [" + + IndexModule.INDEX_STORE_LOCALITY_SETTING.getKey() + + "] please check that any required plugins are installed, or check the " + + "breaking changes documentation for removed settings", + ex.getMessage() + ); + } + } + + public void testWritableWarmBasic() throws Exception { + InternalTestCluster internalTestCluster = internalCluster(); + internalTestCluster.startClusterManagerOnlyNode(); + internalTestCluster.startDataOnlyNode(); + Settings settings = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexModule.INDEX_STORE_LOCALITY_SETTING.getKey(), IndexModule.DataLocalityType.PARTIAL.name()) + .build(); + assertAcked(client().admin().indices().prepareCreate(INDEX_NAME).setSettings(settings).get()); + + // Verify from the cluster settings if the data locality is partial + GetIndexResponse getIndexResponse = client().admin() + .indices() + .getIndex(new GetIndexRequest().indices(INDEX_NAME).includeDefaults(true)) + .get(); + Settings indexSettings = getIndexResponse.settings().get(INDEX_NAME); + assertEquals(IndexModule.DataLocalityType.PARTIAL.name(), indexSettings.get(IndexModule.INDEX_STORE_LOCALITY_SETTING.getKey())); + + // Ingesting some docs + indexBulk(INDEX_NAME, NUM_DOCS_IN_BULK); + flushAndRefresh(INDEX_NAME); + + // ensuring cluster is green after performing force-merge + ensureGreen(); + + SearchResponse searchResponse = client().prepareSearch(INDEX_NAME).setQuery(QueryBuilders.matchAllQuery()).get(); + // Asserting that search returns same number of docs as ingested + assertHitCount(searchResponse, NUM_DOCS_IN_BULK); + + // Ingesting docs again before force merge + indexBulk(INDEX_NAME, NUM_DOCS_IN_BULK); + flushAndRefresh(INDEX_NAME); + + FileCache fileCache = internalTestCluster.getDataNodeInstance(Node.class).fileCache(); + IndexShard shard = internalTestCluster.getDataNodeInstance(IndicesService.class) + .indexService(resolveIndex(INDEX_NAME)) + .getShardOrNull(0); + Directory directory = (((FilterDirectory) (((FilterDirectory) (shard.store().directory())).getDelegate())).getDelegate()); + + // Force merging the index + Set filesBeforeMerge = new HashSet<>(Arrays.asList(directory.listAll())); + client().admin().indices().prepareForceMerge(INDEX_NAME).setMaxNumSegments(1).get(); + flushAndRefresh(INDEX_NAME); + Set filesAfterMerge = new HashSet<>(Arrays.asList(directory.listAll())); + + Set filesFromPreviousGenStillPresent = filesBeforeMerge.stream() + .filter(filesAfterMerge::contains) + .filter(file -> !FileTypeUtils.isLockFile(file)) + .filter(file -> !FileTypeUtils.isSegmentsFile(file)) + .collect(Collectors.toUnmodifiableSet()); + + // Asserting that after merge all the files from previous gen are no more part of the directory + assertTrue(filesFromPreviousGenStillPresent.isEmpty()); + + // Asserting that files from previous gen are not present in File Cache as well + filesBeforeMerge.stream() + .filter(file -> !FileTypeUtils.isLockFile(file)) + .filter(file -> !FileTypeUtils.isSegmentsFile(file)) + .forEach(file -> assertNull(fileCache.get(((CompositeDirectory) directory).getFilePath(file)))); + + // Deleting the index (so that ref count drops to zero for all the files) and then pruning the cache to clear it to avoid any file + // leaks + assertAcked(client().admin().indices().delete(new DeleteIndexRequest(INDEX_NAME)).get()); + fileCache.prune(); + } +} diff --git a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java index 6fe8dec9c21b1..1488f5d30b4ba 100644 --- a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java @@ -41,6 +41,7 @@ import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.logging.Loggers; import org.opensearch.common.settings.Setting.Property; +import org.opensearch.common.util.FeatureFlags; import org.opensearch.index.IndexModule; import org.opensearch.index.IndexSettings; import org.opensearch.index.IndexSortConfig; @@ -260,7 +261,10 @@ public final class IndexScopedSettings extends AbstractScopedSettings { * is ready for production release, the feature flag can be removed, and the * setting should be moved to {@link #BUILT_IN_INDEX_SETTINGS}. */ - public static final Map> FEATURE_FLAGGED_INDEX_SETTINGS = Map.of(); + public static final Map> FEATURE_FLAGGED_INDEX_SETTINGS = Map.of( + FeatureFlags.TIERED_REMOTE_INDEX, + List.of(IndexModule.INDEX_STORE_LOCALITY_SETTING) + ); public static final IndexScopedSettings DEFAULT_SCOPED_SETTINGS = new IndexScopedSettings(Settings.EMPTY, BUILT_IN_INDEX_SETTINGS); diff --git a/server/src/main/java/org/opensearch/index/IndexModule.java b/server/src/main/java/org/opensearch/index/IndexModule.java index 3c4cb4fd596c1..4c494a6b35153 100644 --- a/server/src/main/java/org/opensearch/index/IndexModule.java +++ b/server/src/main/java/org/opensearch/index/IndexModule.java @@ -107,6 +107,8 @@ import java.util.function.Function; import java.util.function.Supplier; +import static org.apache.logging.log4j.util.Strings.toRootUpperCase; + /** * IndexModule represents the central extension point for index level custom implementations like: *