-
Notifications
You must be signed in to change notification settings - Fork 2.3k
[merged segment warmer] support remote merged segment warmer #18683
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
ashking94
merged 28 commits into
opensearch-project:main
from
kh3ra:dev/remote-merged-segment-warmer
Aug 5, 2025
+1,757
−108
Merged
Changes from all commits
Commits
Show all changes
28 commits
Select commit
Hold shift + click to select a range
fb1dd6d
Changes to support upload and download of merge segments using the In…
kh3ra 6439bf5
Unit tests
kh3ra 02dbf7a
Fixing build issues
kh3ra c91301e
Fixing build issues - forbiddenAPIs/spotlessApply
kh3ra 2265b3e
Upload merge segments in low priority, minor fixes
kh3ra 000b8f0
Addressing review comments + rebase main
kh3ra cfc8c46
Test fixes + javadocs
kh3ra 61d4f82
1. Bug fix to RemoteDirectory.DownloadRateLimiterProvider
kh3ra 712b9fa
Bug fix to replica updates to ActiveMergesRegistry
kh3ra ea6d616
Addressing review comments - round 2
kh3ra 28b5574
new tests + test fixes + minor bug fixes
kh3ra 471c8e9
Bug fix
kh3ra 0e72408
Fixes to RemoteStorePublishMergedSegmentActionTests
kh3ra 1b273f3
Adding integration tests
kh3ra b5a362c
Tracking stats for merged segment warmer
kh3ra 8eb8340
Revert "Tracking stats for merged segment warmer"
kh3ra ed2b52a
Addressing review comments for tests
kh3ra df85a75
Addressing review comments
kh3ra 5f310db
Rebasing
kh3ra ad5406a
spotlessApply
kh3ra 9f483ca
test fix
kh3ra 6f792ff
Empty commit
kh3ra 84ac6e9
Adding tests, enhancing logs
kh3ra a68171b
Adding MergedSegmentWarmerFactory tests + enhancing existing tests
kh3ra 9e49316
Applying spotless
kh3ra 0b4b8a4
Added test for Timeout case for RemoteStoreReplicationSource
kh3ra 21d8c13
Restored RemoteSegmentStoreDirectory PublicAPI + added changelog
kh3ra 162b3eb
Merge branch 'main' into dev/remote-merged-segment-warmer
kh3ra File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
182 changes: 182 additions & 0 deletions
182
...ClusterTest/java/org/opensearch/indices/replication/RemoteStoreMergedSegmentWarmerIT.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,182 @@ | ||
| /* | ||
| * 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.indices.replication; | ||
|
|
||
| import org.opensearch.action.admin.indices.forcemerge.ForceMergeRequest; | ||
| import org.opensearch.action.admin.indices.segments.IndicesSegmentResponse; | ||
| import org.opensearch.action.support.WriteRequest; | ||
| import org.opensearch.action.support.replication.TransportReplicationAction; | ||
| import org.opensearch.common.settings.Settings; | ||
| import org.opensearch.common.util.FeatureFlags; | ||
| import org.opensearch.index.IndexSettings; | ||
| import org.opensearch.index.TieredMergePolicyProvider; | ||
| import org.opensearch.indices.replication.checkpoint.RemoteStorePublishMergedSegmentRequest; | ||
| import org.opensearch.test.OpenSearchIntegTestCase; | ||
| import org.opensearch.test.transport.MockTransportService; | ||
| import org.opensearch.test.transport.StubbableTransport; | ||
| import org.opensearch.transport.TransportService; | ||
| import org.junit.Before; | ||
|
|
||
| import java.nio.file.Path; | ||
| import java.util.Set; | ||
| import java.util.concurrent.ConcurrentHashMap; | ||
| import java.util.concurrent.CountDownLatch; | ||
| import java.util.concurrent.TimeUnit; | ||
| import java.util.concurrent.atomic.AtomicLong; | ||
|
|
||
| @OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) | ||
| public class RemoteStoreMergedSegmentWarmerIT extends SegmentReplicationBaseIT { | ||
| private Path absolutePath; | ||
|
|
||
| @Override | ||
| protected Settings nodeSettings(int nodeOrdinal) { | ||
| if (absolutePath == null) { | ||
| absolutePath = randomRepoPath().toAbsolutePath(); | ||
| } | ||
| return Settings.builder() | ||
| .put(super.nodeSettings(nodeOrdinal)) | ||
| .put(remoteStoreClusterSettings("test-remote-store-repo", absolutePath)) | ||
| .build(); | ||
| } | ||
|
|
||
| @Override | ||
| protected Settings featureFlagSettings() { | ||
| Settings.Builder featureSettings = Settings.builder(); | ||
| featureSettings.put(FeatureFlags.MERGED_SEGMENT_WARMER_EXPERIMENTAL_FLAG, true); | ||
| return featureSettings.build(); | ||
| } | ||
|
|
||
| @Before | ||
| public void setup() { | ||
| internalCluster().startClusterManagerOnlyNode(); | ||
| } | ||
|
|
||
| public void testMergeSegmentWarmerRemote() throws Exception { | ||
| final String node1 = internalCluster().startDataOnlyNode(); | ||
| final String node2 = internalCluster().startDataOnlyNode(); | ||
| createIndex(INDEX_NAME); | ||
| ensureGreen(INDEX_NAME); | ||
| MockTransportService mockTransportServiceNode1 = (MockTransportService) internalCluster().getInstance( | ||
| TransportService.class, | ||
| node1 | ||
| ); | ||
| MockTransportService mockTransportServiceNode2 = (MockTransportService) internalCluster().getInstance( | ||
| TransportService.class, | ||
| node2 | ||
| ); | ||
| final CountDownLatch latch = new CountDownLatch(1); | ||
| StubbableTransport.SendRequestBehavior behavior = (connection, requestId, action, request, options) -> { | ||
| if (action.equals("indices:admin/remote_publish_merged_segment[r]")) { | ||
| assertTrue( | ||
| ((TransportReplicationAction.ConcreteReplicaRequest) request) | ||
| .getRequest() instanceof RemoteStorePublishMergedSegmentRequest | ||
| ); | ||
| latch.countDown(); | ||
| } | ||
| connection.sendRequest(requestId, action, request, options); | ||
| }; | ||
|
|
||
| for (int i = 0; i < 30; i++) { | ||
| client().prepareIndex(INDEX_NAME) | ||
| .setId(String.valueOf(i)) | ||
| .setSource("foo" + i, "bar" + i) | ||
| .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) | ||
| .get(); | ||
| } | ||
|
|
||
| waitForSearchableDocs(30, node1, node2); | ||
|
|
||
| mockTransportServiceNode1.addSendBehavior(behavior); | ||
| mockTransportServiceNode2.addSendBehavior(behavior); | ||
|
|
||
| client().admin().indices().forceMerge(new ForceMergeRequest(INDEX_NAME).maxNumSegments(2)); | ||
| waitForSegmentCount(INDEX_NAME, 2, logger); | ||
| assertTrue(latch.await(10, TimeUnit.SECONDS)); | ||
| mockTransportServiceNode1.clearAllRules(); | ||
| mockTransportServiceNode2.clearAllRules(); | ||
| } | ||
|
|
||
| public void testConcurrentMergeSegmentWarmerRemote() throws Exception { | ||
kh3ra marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| String node1 = internalCluster().startDataOnlyNode(); | ||
| String node2 = internalCluster().startDataOnlyNode(); | ||
| createIndex( | ||
| INDEX_NAME, | ||
| Settings.builder() | ||
| .put(indexSettings()) | ||
| .put(TieredMergePolicyProvider.INDEX_MERGE_POLICY_SEGMENTS_PER_TIER_SETTING.getKey(), 5) | ||
| .put(TieredMergePolicyProvider.INDEX_MERGE_POLICY_MAX_MERGE_AT_ONCE_SETTING.getKey(), 5) | ||
| .put(IndexSettings.INDEX_MERGE_ON_FLUSH_ENABLED.getKey(), false) | ||
| .build() | ||
| ); | ||
| ensureGreen(INDEX_NAME); | ||
| MockTransportService mockTransportServiceNode1 = (MockTransportService) internalCluster().getInstance( | ||
| TransportService.class, | ||
| node1 | ||
| ); | ||
| MockTransportService mockTransportServiceNode2 = (MockTransportService) internalCluster().getInstance( | ||
| TransportService.class, | ||
| node2 | ||
| ); | ||
| CountDownLatch latch = new CountDownLatch(2); | ||
| AtomicLong numInvocations = new AtomicLong(0); | ||
| Set<String> executingThreads = ConcurrentHashMap.newKeySet(); | ||
| StubbableTransport.SendRequestBehavior behavior = (connection, requestId, action, request, options) -> { | ||
| if (action.equals("indices:admin/remote_publish_merged_segment[r]")) { | ||
| assertTrue( | ||
| ((TransportReplicationAction.ConcreteReplicaRequest) request) | ||
| .getRequest() instanceof RemoteStorePublishMergedSegmentRequest | ||
| ); | ||
| latch.countDown(); | ||
| numInvocations.incrementAndGet(); | ||
| executingThreads.add(Thread.currentThread().getName()); | ||
| } | ||
| connection.sendRequest(requestId, action, request, options); | ||
| }; | ||
|
|
||
| mockTransportServiceNode1.addSendBehavior(behavior); | ||
| mockTransportServiceNode2.addSendBehavior(behavior); | ||
|
|
||
| for (int i = 0; i < 30; i++) { | ||
| client().prepareIndex(INDEX_NAME) | ||
| .setId(String.valueOf(i)) | ||
| .setSource("foo" + i, "bar" + i) | ||
| .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) | ||
| .get(); | ||
| } | ||
|
|
||
| client().admin().indices().forceMerge(new ForceMergeRequest(INDEX_NAME).maxNumSegments(2)); | ||
|
|
||
| waitForSegmentCount(INDEX_NAME, 2, logger); | ||
| logger.info("Number of merge invocations: {}", numInvocations.get()); | ||
| assertTrue(latch.await(10, TimeUnit.SECONDS)); | ||
| assertTrue(executingThreads.size() > 1); | ||
| // Verify concurrent execution by checking that multiple unique threads handled merge operations | ||
| assertTrue(numInvocations.get() > 1); | ||
| mockTransportServiceNode1.clearAllRules(); | ||
| mockTransportServiceNode2.clearAllRules(); | ||
| } | ||
|
|
||
| public void testMergeSegmentWarmerWithInactiveReplicaRemote() throws Exception { | ||
| internalCluster().startDataOnlyNode(); | ||
| createIndex(INDEX_NAME); | ||
| ensureYellowAndNoInitializingShards(INDEX_NAME); | ||
|
|
||
| for (int i = 0; i < 30; i++) { | ||
| client().prepareIndex(INDEX_NAME) | ||
| .setId(String.valueOf(i)) | ||
| .setSource("foo" + i, "bar" + i) | ||
| .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) | ||
| .get(); | ||
| } | ||
|
|
||
| client().admin().indices().forceMerge(new ForceMergeRequest(INDEX_NAME).maxNumSegments(1)).get(); | ||
| final IndicesSegmentResponse response = client().admin().indices().prepareSegments(INDEX_NAME).get(); | ||
| assertEquals(1, response.getIndices().get(INDEX_NAME).getShards().values().size()); | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
43 changes: 0 additions & 43 deletions
43
server/src/main/java/org/opensearch/index/engine/RemoteStoreMergedSegmentWarmer.java
This file was deleted.
Oops, something went wrong.
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.