-
Notifications
You must be signed in to change notification settings - Fork 2.3k
[Segment Replication] Avoid data loss in vanilla segment replication #20150
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
Open
guojialiang92
wants to merge
13
commits into
opensearch-project:main
Choose a base branch
from
guojialiang92:dev/fix-data-loss-in-vanilla-segment-replication
base: main
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
+192
−1
Open
Changes from 5 commits
Commits
Show all changes
13 commits
Select commit
Hold shift + click to select a range
aae28f2
Avoid data loss in vanilla segment replication.
guojialiang92 88ba97c
update
guojialiang92 1aa8d32
update
guojialiang92 14ce5db
update
guojialiang92 810aa28
update
guojialiang92 5b603a8
update
guojialiang92 8afe73c
If local segment replication is enabled, a force flush needs to be pe…
guojialiang92 2262497
Merge remote-tracking branch 'origin/main' into dev/fix-data-loss-in-…
guojialiang92 8ba8c98
Add comments
guojialiang92 740deb3
put Math.min(maxSeqNo, lastRefreshedCheckpoint) in max_seq_no
guojialiang92 f3fec76
update
guojialiang92 9231a7f
Add comments
guojialiang92 4265d71
Add comments
guojialiang92 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
Some comments aren't visible on the classic Files Changed page.
There are no files selected for viewing
169 changes: 169 additions & 0 deletions
169
...terTest/java/org/opensearch/indices/replication/SegmentReplicationPrimaryPromotionIT.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,169 @@ | ||
| /* | ||
| * 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.search.SearchResponse; | ||
| import org.opensearch.action.support.replication.TransportReplicationAction; | ||
| import org.opensearch.cluster.routing.ShardRouting; | ||
| import org.opensearch.common.settings.Settings; | ||
| import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; | ||
| import org.opensearch.index.IndexSettings; | ||
| import org.opensearch.index.engine.Engine; | ||
| import org.opensearch.index.engine.EngineConfig; | ||
| import org.opensearch.index.engine.EngineException; | ||
| import org.opensearch.index.engine.EngineFactory; | ||
| import org.opensearch.index.engine.InternalEngine; | ||
| import org.opensearch.index.engine.NRTReplicationEngine; | ||
| import org.opensearch.indices.replication.checkpoint.PublishCheckpointAction; | ||
| import org.opensearch.plugins.EnginePlugin; | ||
| import org.opensearch.plugins.Plugin; | ||
| import org.opensearch.test.InternalTestCluster; | ||
| import org.opensearch.test.OpenSearchIntegTestCase; | ||
| import org.opensearch.test.transport.MockTransportService; | ||
| import org.opensearch.transport.RemoteTransportException; | ||
| import org.opensearch.transport.TransportService; | ||
| import org.junit.Before; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.Collection; | ||
| import java.util.List; | ||
| import java.util.Optional; | ||
| import java.util.concurrent.CountDownLatch; | ||
| import java.util.concurrent.TimeUnit; | ||
|
|
||
| @OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) | ||
| public class SegmentReplicationPrimaryPromotionIT extends SegmentReplicationBaseIT { | ||
| private static boolean lockEnable; | ||
| private static CountDownLatch indexLuceneLatch; | ||
| private static CountDownLatch flushLatch; | ||
| private static CountDownLatch refreshLatch; | ||
|
|
||
| @Before | ||
| public void setup() { | ||
| lockEnable = false; | ||
| indexLuceneLatch = new CountDownLatch(1); | ||
| flushLatch = new CountDownLatch(1); | ||
| refreshLatch = new CountDownLatch(1); | ||
| internalCluster().startClusterManagerOnlyNode(); | ||
| } | ||
|
|
||
| @Override | ||
| protected Collection<Class<? extends Plugin>> getMockPlugins() { | ||
| List<Class<? extends Plugin>> plugins = super.getMockPlugins().stream() | ||
| .filter(plugin -> !plugin.getName().contains("MockEngineFactoryPlugin")) | ||
| .collect(java.util.stream.Collectors.toList()); | ||
| plugins.add(MockEnginePlugin.class); | ||
| return plugins; | ||
| } | ||
|
|
||
| public static class MockEnginePlugin extends Plugin implements EnginePlugin { | ||
| @Override | ||
| public Optional<EngineFactory> getEngineFactory(final IndexSettings indexSettings) { | ||
| return Optional.of(new MockEngineFactory()); | ||
| } | ||
| } | ||
|
|
||
| public static class MockEngineFactory implements EngineFactory { | ||
| @Override | ||
| public Engine newReadWriteEngine(EngineConfig config) { | ||
| return config.isReadOnlyReplica() ? new MockNRTReplicationEngine(config) : new MockInternalEngine(config); | ||
| } | ||
| } | ||
|
|
||
| public static class MockInternalEngine extends InternalEngine { | ||
| MockInternalEngine(EngineConfig config) throws EngineException { | ||
| super(config); | ||
| } | ||
|
|
||
| @Override | ||
| protected long generateSeqNoForOperationOnPrimary(final Operation operation) { | ||
| long seqNo = super.generateSeqNoForOperationOnPrimary(operation); | ||
| try { | ||
| if (lockEnable) { | ||
| flushLatch.countDown(); | ||
| indexLuceneLatch.await(); | ||
| } | ||
| } catch (Exception e) { | ||
| throw new RuntimeException(e); | ||
| } | ||
| return seqNo; | ||
| } | ||
| } | ||
|
|
||
| public static class MockNRTReplicationEngine extends NRTReplicationEngine { | ||
| MockNRTReplicationEngine(EngineConfig config) throws EngineException { | ||
| super(config); | ||
| } | ||
|
|
||
| @Override | ||
| public IndexResult index(Index index) throws IOException { | ||
| IndexResult indexResult = super.index(index); | ||
| if (lockEnable) { | ||
| refreshLatch.countDown(); | ||
| } | ||
| return indexResult; | ||
| } | ||
| } | ||
|
|
||
| // Used to test that primary promotion does not result in data loss. | ||
| public void testPrimaryStopped_ReplicaPromoted_no_data_loss() throws Exception { | ||
| final String primary = internalCluster().startDataOnlyNode(); | ||
| createIndex(INDEX_NAME, Settings.builder().put(indexSettings()).put("index.refresh_interval", -1).build()); | ||
| ensureYellowAndNoInitializingShards(INDEX_NAME); | ||
| final String replica = internalCluster().startDataOnlyNode(); | ||
| ensureGreen(INDEX_NAME); | ||
|
|
||
| client().prepareIndex(INDEX_NAME).setId("1").setSource("foo", "bar").get(); | ||
| lockEnable = true; | ||
| Thread writeThread = new Thread(() -> { client().prepareIndex(INDEX_NAME).setId("2").setSource("foo2", "bar2").get(); }); | ||
| writeThread.start(); | ||
| assertTrue("flushLatch timed out", flushLatch.await(30, TimeUnit.SECONDS)); | ||
|
|
||
| flush(INDEX_NAME); | ||
|
|
||
| waitForSearchableDocs(1, replica); | ||
|
|
||
| // mock network exception | ||
| MockTransportService replicaTransportService = ((MockTransportService) internalCluster().getInstance( | ||
| TransportService.class, | ||
| replica | ||
| )); | ||
| replicaTransportService.addRequestHandlingBehavior( | ||
| PublishCheckpointAction.ACTION_NAME + TransportReplicationAction.REPLICA_ACTION_SUFFIX, | ||
| (handler, request, channel, task) -> { | ||
| throw new RemoteTransportException("mock remote transport exception", new OpenSearchRejectedExecutionException()); | ||
| } | ||
| ); | ||
|
|
||
| refresh(INDEX_NAME); | ||
| waitForSearchableDocs(1, primary); | ||
| indexLuceneLatch.countDown(); | ||
| assertTrue("refreshLatch timed out", refreshLatch.await(30, TimeUnit.SECONDS)); | ||
| writeThread.join(); | ||
|
|
||
| logger.info("refresh index"); | ||
| refresh(INDEX_NAME); | ||
| flush(INDEX_NAME); | ||
| waitForSearchableDocs(2, primary); | ||
| waitForSearchableDocs(1, replica); | ||
|
|
||
| // stop the primary node - we only have one shard on here. | ||
| internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primary)); | ||
| ensureYellowAndNoInitializingShards(INDEX_NAME); | ||
|
|
||
| final ShardRouting replicaShardRouting = getShardRoutingForNodeName(replica); | ||
| assertNotNull(replicaShardRouting); | ||
| assertTrue(replicaShardRouting + " should be promoted as a primary", replicaShardRouting.primary()); | ||
|
|
||
| refresh(INDEX_NAME); | ||
| SearchResponse response = client(replica).prepareSearch(INDEX_NAME).setSize(0).setPreference("_only_local").get(); | ||
| assertEquals(2L, response.getHits().getTotalHits().value()); | ||
| replicaTransportService.clearAllRules(); | ||
| } | ||
| } | ||
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
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.