Skip to content
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

[Backport 2.x] Flaky test : Don't use async repo for SplitIndex and wait for translo… #15200

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,7 @@
import org.opensearch.action.index.IndexRequest;
import org.opensearch.action.index.IndexRequestBuilder;
import org.opensearch.action.search.SearchResponse;
import org.opensearch.action.support.IndicesOptions;
import org.opensearch.client.Client;
import org.opensearch.cluster.ClusterState;
import org.opensearch.cluster.metadata.IndexMetadata;
Expand All @@ -69,12 +70,15 @@
import org.opensearch.remotestore.RemoteStoreBaseIntegTestCase;
import org.opensearch.test.OpenSearchIntegTestCase;
import org.opensearch.test.VersionUtils;
import org.junit.After;
import org.junit.Before;

import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.function.BiFunction;
import java.util.stream.IntStream;

Expand All @@ -89,12 +93,32 @@

@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0)
public class RemoteSplitIndexIT extends RemoteStoreBaseIntegTestCase {
@Before
public void setup() {
asyncUploadMockFsRepo = false;
}

@Override
protected boolean forbidPrivateIndexSettings() {
return false;
}

@After
public void cleanUp() throws Exception {
// Delete is async.
assertAcked(
client().admin().indices().prepareDelete("*").setIndicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN_CLOSED_HIDDEN).get()
);
assertBusy(() -> {
try {
assertEquals(0, getFileCount(translogRepoPath));
} catch (IOException e) {
fail();
}
}, 30, TimeUnit.SECONDS);
super.teardown();
}

public Settings indexSettings() {
return Settings.builder()
.put(super.indexSettings())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -313,7 +313,7 @@ public void assertRemoteStoreRepositoryOnAllNodes(String repositoryName) {
}
}

public static int getFileCount(Path path) throws Exception {
public static int getFileCount(Path path) throws IOException {
final AtomicInteger filesExisting = new AtomicInteger(0);
Files.walkFileTree(path, new SimpleFileVisitor<>() {
@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import org.opensearch.test.InternalTestCluster;
import org.opensearch.test.OpenSearchIntegTestCase;
import org.opensearch.test.disruption.NetworkDisruption;
import org.opensearch.test.junit.annotations.TestLogging;
import org.opensearch.test.transport.MockTransportService;

import java.io.IOException;
Expand Down Expand Up @@ -257,6 +258,7 @@ public void testStatsResponseFromLocalNode() {
}
}

@TestLogging(reason = "Getting trace logs from remote store package", value = "org.opensearch.index.shard:TRACE")
public void testDownloadStatsCorrectnessSinglePrimarySingleReplica() throws Exception {
setup();
// Scenario:
Expand Down Expand Up @@ -285,6 +287,15 @@ public void testDownloadStatsCorrectnessSinglePrimarySingleReplica() throws Exce
.collect(Collectors.toList())
.get(0)
.getSegmentStats();
logger.info(
"Zero state primary stats: {}ms refresh time lag, {}b bytes lag, {}b upload bytes started, {}b upload bytes failed , {} uploads succeeded, {} upload byes succeeded.",
zeroStatePrimaryStats.refreshTimeLagMs,
zeroStatePrimaryStats.bytesLag,
zeroStatePrimaryStats.uploadBytesStarted,
zeroStatePrimaryStats.uploadBytesFailed,
zeroStatePrimaryStats.totalUploadsSucceeded,
zeroStatePrimaryStats.uploadBytesSucceeded
);
assertTrue(
zeroStatePrimaryStats.totalUploadsStarted == zeroStatePrimaryStats.totalUploadsSucceeded
&& zeroStatePrimaryStats.totalUploadsSucceeded == 1
Expand Down Expand Up @@ -347,6 +358,7 @@ public void testDownloadStatsCorrectnessSinglePrimarySingleReplica() throws Exce
}
}

@TestLogging(reason = "Getting trace logs from remote store package", value = "org.opensearch.index.shard:TRACE")
public void testDownloadStatsCorrectnessSinglePrimaryMultipleReplicaShards() throws Exception {
setup();
// Scenario:
Expand Down Expand Up @@ -379,6 +391,15 @@ public void testDownloadStatsCorrectnessSinglePrimaryMultipleReplicaShards() thr
.collect(Collectors.toList())
.get(0)
.getSegmentStats();
logger.info(
"Zero state primary stats: {}ms refresh time lag, {}b bytes lag, {}b upload bytes started, {}b upload bytes failed , {} uploads succeeded, {} upload byes succeeded.",
zeroStatePrimaryStats.refreshTimeLagMs,
zeroStatePrimaryStats.bytesLag,
zeroStatePrimaryStats.uploadBytesStarted,
zeroStatePrimaryStats.uploadBytesFailed,
zeroStatePrimaryStats.totalUploadsSucceeded,
zeroStatePrimaryStats.uploadBytesSucceeded
);
assertTrue(
zeroStatePrimaryStats.totalUploadsStarted == zeroStatePrimaryStats.totalUploadsSucceeded
&& zeroStatePrimaryStats.totalUploadsSucceeded == 1
Expand Down Expand Up @@ -608,7 +629,7 @@ public void testNonZeroPrimaryStatsOnNewlyCreatedIndexWithZeroDocs() throws Exce
}
assertZeroTranslogDownloadStats(translogStats);
});
}, 5, TimeUnit.SECONDS);
}, 10, TimeUnit.SECONDS);
}

public void testStatsCorrectnessOnFailover() {
Expand Down
Loading