From 435af893d14b23685aa450169626379c455d1548 Mon Sep 17 00:00:00 2001 From: Rishabh Maurya Date: Wed, 12 Jun 2024 18:19:34 -0700 Subject: [PATCH 01/30] 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/30] [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/30] [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/30] 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/30] 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/30] 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/30] 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/30] [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/30] 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/30] 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/30] [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/30] 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/30] 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/30] [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/30] 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/30] 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/30] 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/30] 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/30] 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/30] 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/30] 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/30] 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/30] 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/30] 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/30] 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/30] [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/30] 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/30] 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/30] 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/30] 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