diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/AbstractRemoteStoreMockRepositoryIntegTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/AbstractRemoteStoreMockRepositoryIntegTestCase.java index bb8839a47eb69..6df559157d847 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/AbstractRemoteStoreMockRepositoryIntegTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/AbstractRemoteStoreMockRepositoryIntegTestCase.java @@ -34,6 +34,7 @@ public abstract class AbstractRemoteStoreMockRepositoryIntegTestCase extends AbstractSnapshotIntegTestCase { protected static final String REPOSITORY_NAME = "my-segment-repo-1"; + protected static final String TRANSLOG_REPOSITORY_NAME = "my-translog-repo-1"; protected static final String INDEX_NAME = "remote-store-test-idx-1"; @Override @@ -63,12 +64,15 @@ protected Settings remoteStoreIndexSettings(int numberOfReplicas) { .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) .put(IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY, REPOSITORY_NAME) + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, TRANSLOG_REPOSITORY_NAME) .build(); } protected void deleteRepo() { logger.info("--> Deleting the repository={}", REPOSITORY_NAME); assertAcked(clusterAdmin().prepareDeleteRepository(REPOSITORY_NAME)); + logger.info("--> Deleting the repository={}", TRANSLOG_REPOSITORY_NAME); + assertAcked(clusterAdmin().prepareDeleteRepository(TRANSLOG_REPOSITORY_NAME)); } protected String setup(Path repoLocation, double ioFailureRate, String skipExceptionBlobList, long maxFailure) { @@ -88,6 +92,8 @@ protected String setup(Path repoLocation, double ioFailureRate, String skipExcep .put("skip_exception_on_blobs", skipExceptionBlobList) .put("max_failure_number", maxFailure) ); + logger.info("--> Creating repository={} at the path={}", TRANSLOG_REPOSITORY_NAME, repoLocation); + createRepository(TRANSLOG_REPOSITORY_NAME, "mock", Settings.builder().put("location", repoLocation)); String dataNodeName = internalCluster().startDataOnlyNodes(1).get(0); createIndex(INDEX_NAME); diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java index 9f4106881fabb..b5d7451d102a1 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java @@ -68,6 +68,7 @@ private Settings defaultIndexSettings() { .put(IndexModule.INDEX_QUERY_CACHE_ENABLED_SETTING.getKey(), false) .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) .put(IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY, REPOSITORY_NAME) + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, REPOSITORY_NAME) .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, SHARD_COUNT) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, REPLICA_COUNT) .put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), "300s") @@ -95,10 +96,7 @@ protected Settings remoteStoreIndexSettings(int numberOfReplicas, long totalFiel } protected Settings remoteTranslogIndexSettings(int numberOfReplicas, int numberOfShards) { - return Settings.builder() - .put(remoteStoreIndexSettings(numberOfReplicas, numberOfShards)) - .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, REPOSITORY_NAME) - .build(); + return Settings.builder().put(remoteStoreIndexSettings(numberOfReplicas, numberOfShards)).build(); } protected Settings remoteTranslogIndexSettings(int numberOfReplicas) { diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java index f01e4969b1fe7..94498e4bb4f9d 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java @@ -112,13 +112,9 @@ private void verifyRestoredData(Map indexStats, boolean checkTotal assertHitCount(client().prepareSearch(INDEX_NAME).setSize(0).get(), indexStats.get(statsGranularity) + 1); } - private void testRestoreFlow(boolean remoteTranslog, int numberOfIterations, boolean invokeFlush) throws IOException { + private void testRestoreFlow(int numberOfIterations, boolean invokeFlush) throws IOException { internalCluster().startDataOnlyNodes(3); - if (remoteTranslog) { - createIndex(INDEX_NAME, remoteTranslogIndexSettings(0)); - } else { - createIndex(INDEX_NAME, remoteStoreIndexSettings(0)); - } + createIndex(INDEX_NAME, remoteStoreIndexSettings(0)); ensureYellowAndNoInitializingShards(INDEX_NAME); ensureGreen(INDEX_NAME); @@ -130,44 +126,40 @@ private void testRestoreFlow(boolean remoteTranslog, int numberOfIterations, boo client().admin().cluster().restoreRemoteStore(new RestoreRemoteStoreRequest().indices(INDEX_NAME), PlainActionFuture.newFuture()); ensureGreen(INDEX_NAME); - if (remoteTranslog) { - verifyRestoredData(indexStats, true); - } else { - verifyRestoredData(indexStats, false); - } - } - - public void testRemoteSegmentStoreRestoreWithNoDataPostCommit() throws IOException { - testRestoreFlow(false, 1, true); - } - - public void testRemoteSegmentStoreRestoreWithNoDataPostRefresh() throws IOException { - testRestoreFlow(false, 1, false); - } - - public void testRemoteSegmentStoreRestoreWithRefreshedData() throws IOException { - testRestoreFlow(false, randomIntBetween(2, 5), false); + verifyRestoredData(indexStats, true); } - public void testRemoteSegmentStoreRestoreWithCommittedData() throws IOException { - testRestoreFlow(false, randomIntBetween(2, 5), true); - } + // public void testRemoteSegmentStoreRestoreWithNoDataPostCommit() throws IOException { + // testRestoreFlow(false, 1, true); + // } + // + // public void testRemoteSegmentStoreRestoreWithNoDataPostRefresh() throws IOException { + // testRestoreFlow(false, 1, false); + // } + // + // public void testRemoteSegmentStoreRestoreWithRefreshedData() throws IOException { + // testRestoreFlow(false, randomIntBetween(2, 5), false); + // } + // + // public void testRemoteSegmentStoreRestoreWithCommittedData() throws IOException { + // testRestoreFlow(false, randomIntBetween(2, 5), true); + // } @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/6188") public void testRemoteTranslogRestoreWithNoDataPostCommit() throws IOException { - testRestoreFlow(true, 1, true); + testRestoreFlow(1, true); } public void testRemoteTranslogRestoreWithNoDataPostRefresh() throws IOException { - testRestoreFlow(true, 1, false); + testRestoreFlow(1, false); } public void testRemoteTranslogRestoreWithRefreshedData() throws IOException { - testRestoreFlow(true, randomIntBetween(2, 5), false); + testRestoreFlow(randomIntBetween(2, 5), false); } public void testRemoteTranslogRestoreWithCommittedData() throws IOException { - testRestoreFlow(true, randomIntBetween(2, 5), true); + testRestoreFlow(randomIntBetween(2, 5), true); } private void testPeerRecovery(boolean remoteTranslog, int numberOfIterations, boolean invokeFlush) throws Exception { @@ -223,22 +215,6 @@ private void testPeerRecovery(boolean remoteTranslog, int numberOfIterations, bo ); } - public void testPeerRecoveryWithRemoteStoreNoRemoteTranslogNoDataFlush() throws Exception { - testPeerRecovery(false, 1, true); - } - - public void testPeerRecoveryWithRemoteStoreNoRemoteTranslogFlush() throws Exception { - testPeerRecovery(false, randomIntBetween(2, 5), true); - } - - public void testPeerRecoveryWithRemoteStoreNoRemoteTranslogNoDataRefresh() throws Exception { - testPeerRecovery(false, 1, false); - } - - public void testPeerRecoveryWithRemoteStoreNoRemoteTranslogRefresh() throws Exception { - testPeerRecovery(false, randomIntBetween(2, 5), false); - } - public void testPeerRecoveryWithRemoteStoreAndRemoteTranslogNoDataFlush() throws Exception { testPeerRecovery(true, 1, true); } @@ -255,13 +231,9 @@ public void testPeerRecoveryWithRemoteStoreAndRemoteTranslogRefresh() throws Exc testPeerRecovery(true, randomIntBetween(2, 5), false); } - private void verifyRemoteStoreCleanup(boolean remoteTranslog) throws Exception { + private void verifyRemoteStoreCleanup() throws Exception { internalCluster().startDataOnlyNodes(3); - if (remoteTranslog) { - createIndex(INDEX_NAME, remoteTranslogIndexSettings(1)); - } else { - createIndex(INDEX_NAME, remoteStoreIndexSettings(1)); - } + createIndex(INDEX_NAME, remoteStoreIndexSettings(1)); indexData(5, randomBoolean()); String indexUUID = client().admin() @@ -280,12 +252,8 @@ private void verifyRemoteStoreCleanup(boolean remoteTranslog) throws Exception { }, 30, TimeUnit.SECONDS); } - public void testRemoteSegmentCleanup() throws Exception { - verifyRemoteStoreCleanup(false); - } - public void testRemoteTranslogCleanup() throws Exception { - verifyRemoteStoreCleanup(true); + verifyRemoteStoreCleanup(); } public void testStaleCommitDeletionWithInvokeFlush() throws Exception { diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationUsingRemoteStoreIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationUsingRemoteStoreIT.java index ef63b36f837bb..c43bf2d33a1bc 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationUsingRemoteStoreIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationUsingRemoteStoreIT.java @@ -37,6 +37,7 @@ public Settings indexSettings() { .put(super.indexSettings()) .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) .put(IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY, REPOSITORY_NAME) + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, REPOSITORY_NAME) .build(); } diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationWithRemoteStorePressureIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationWithRemoteStorePressureIT.java index f4e49b0b36e05..a3a3601401a54 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationWithRemoteStorePressureIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationWithRemoteStorePressureIT.java @@ -37,6 +37,7 @@ public Settings indexSettings() { .put(super.indexSettings()) .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) .put(IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY, REPOSITORY_NAME) + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, REPOSITORY_NAME) .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) .build(); } diff --git a/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTests.java b/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTests.java index 822cae1b9392d..d56cbda2fe6e4 100644 --- a/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTests.java +++ b/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTests.java @@ -35,6 +35,7 @@ import org.apache.lucene.codecs.Codec; import org.opensearch.action.ActionListener; import org.opensearch.action.support.replication.ReplicationResponse; +import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.routing.AllocationId; import org.opensearch.cluster.routing.IndexShardRoutingTable; import org.opensearch.cluster.routing.ShardRouting; @@ -1292,7 +1293,7 @@ public void testGlobalCheckpointUpdateWithRemoteTranslogEnabled() { assertThat(allocations.size(), equalTo(active.size() + initializing.size())); final AllocationId primaryId = active.iterator().next(); - Settings settings = Settings.builder().put("index.remote_store.translog.enabled", "true").build(); + Settings settings = Settings.builder().put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "true").build(); final ReplicationTracker tracker = newTracker(primaryId, settings); assertThat(tracker.getGlobalCheckpoint(), equalTo(UNASSIGNED_SEQ_NO)); @@ -1367,7 +1368,7 @@ public void testUpdateFromClusterManagerWithRemoteTranslogEnabled() { assertThat(allocations.size(), equalTo(active.size() + initializing.size())); final AllocationId primaryId = active.iterator().next(); - Settings settings = Settings.builder().put("index.remote_store.translog.enabled", "true").build(); + Settings settings = Settings.builder().put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "true").build(); final ReplicationTracker tracker = newTracker(primaryId, settings); assertThat(tracker.getGlobalCheckpoint(), equalTo(UNASSIGNED_SEQ_NO)); @@ -1437,7 +1438,7 @@ public void testUpdateFromClusterManagerWithRemoteTranslogEnabled() { */ public void testUpdateGlobalCheckpointOnReplicaWithRemoteTranslogEnabled() { final AllocationId active = AllocationId.newInitializing(); - Settings settings = Settings.builder().put("index.remote_store.translog.enabled", "true").build(); + Settings settings = Settings.builder().put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "true").build(); final ReplicationTracker tracker = newTracker(active, settings); final long globalCheckpoint = randomLongBetween(NO_OPS_PERFORMED, Long.MAX_VALUE - 1); tracker.updateGlobalCheckpointOnReplica(globalCheckpoint, "test"); @@ -1459,7 +1460,7 @@ public void testMarkAllocationIdAsInSyncWithRemoteTranslogEnabled() throws Excep Set initializing = new HashSet<>(initializingWithCheckpoints.keySet()); final AllocationId primaryId = active.iterator().next(); final AllocationId replicaId = initializing.iterator().next(); - Settings settings = Settings.builder().put("index.remote_store.translog.enabled", "true").build(); + Settings settings = Settings.builder().put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "true").build(); final ReplicationTracker tracker = newTracker(primaryId, settings); tracker.updateFromClusterManager(initialClusterStateVersion, ids(active), routingTable(initializing, primaryId)); final long localCheckpoint = randomLongBetween(0, Long.MAX_VALUE - 1); @@ -1484,7 +1485,7 @@ public void testMissingActiveIdsDoesNotPreventAdvanceWithRemoteTranslogEnabled() assigned.putAll(active); assigned.putAll(initializing); AllocationId primaryId = active.keySet().iterator().next(); - Settings settings = Settings.builder().put("index.remote_store.translog.enabled", "true").build(); + Settings settings = Settings.builder().put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "true").build(); final ReplicationTracker tracker = newTracker(primaryId, settings); tracker.updateFromClusterManager(randomNonNegativeLong(), ids(active.keySet()), routingTable(initializing.keySet(), primaryId)); tracker.activatePrimaryMode(NO_OPS_PERFORMED); @@ -1514,7 +1515,7 @@ public void testMissingInSyncIdsDoesNotPreventAdvanceWithRemoteTranslogEnabled() logger.info("active: {}, initializing: {}", active, initializing); AllocationId primaryId = active.keySet().iterator().next(); - Settings settings = Settings.builder().put("index.remote_store.translog.enabled", "true").build(); + Settings settings = Settings.builder().put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "true").build(); final ReplicationTracker tracker = newTracker(primaryId, settings); tracker.updateFromClusterManager(randomNonNegativeLong(), ids(active.keySet()), routingTable(initializing.keySet(), primaryId)); tracker.activatePrimaryMode(NO_OPS_PERFORMED); @@ -1539,7 +1540,7 @@ public void testInSyncIdsAreIgnoredIfNotValidatedByClusterManagerWithRemoteTrans final Map initializing = randomAllocationsWithLocalCheckpoints(1, 5); final Map nonApproved = randomAllocationsWithLocalCheckpoints(1, 5); final AllocationId primaryId = active.keySet().iterator().next(); - Settings settings = Settings.builder().put("index.remote_store.translog.enabled", "true").build(); + Settings settings = Settings.builder().put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "true").build(); final ReplicationTracker tracker = newTracker(primaryId, settings); tracker.updateFromClusterManager(randomNonNegativeLong(), ids(active.keySet()), routingTable(initializing.keySet(), primaryId)); tracker.activatePrimaryMode(NO_OPS_PERFORMED); @@ -1577,7 +1578,7 @@ public void testInSyncIdsAreRemovedIfNotValidatedByClusterManagerWithRemoteTrans if (randomBoolean()) { allocations.putAll(initializingToBeRemoved); } - Settings settings = Settings.builder().put("index.remote_store.translog.enabled", "true").build(); + Settings settings = Settings.builder().put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "true").build(); final ReplicationTracker tracker = newTracker(primaryId, settings); tracker.updateFromClusterManager(initialClusterStateVersion, ids(active), routingTable(initializing, primaryId)); tracker.activatePrimaryMode(NO_OPS_PERFORMED); @@ -1623,7 +1624,7 @@ public void testUpdateAllocationIdsFromClusterManagerWithRemoteTranslogEnabled() final Set initializingIds = activeAndInitializingAllocationIds.v2(); AllocationId primaryId = activeAllocationIds.iterator().next(); IndexShardRoutingTable routingTable = routingTable(initializingIds, primaryId); - Settings settings = Settings.builder().put("index.remote_store.translog.enabled", "true").build(); + Settings settings = Settings.builder().put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "true").build(); final ReplicationTracker tracker = newTracker(primaryId, settings); tracker.updateFromClusterManager(initialClusterStateVersion, ids(activeAllocationIds), routingTable); tracker.activatePrimaryMode(NO_OPS_PERFORMED); @@ -1927,7 +1928,7 @@ public void testSegmentReplicationCheckpointTrackingInvalidAllocationIDs() { } public void testPrimaryContextHandoffWithRemoteTranslogEnabled() throws IOException { - Settings settings = Settings.builder().put("index.remote_store.translog.enabled", "true").build(); + Settings settings = Settings.builder().put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "true").build(); final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test", settings); final ShardId shardId = new ShardId("test", "_na_", 0); @@ -2106,7 +2107,7 @@ public void testPrimaryContextHandoffWithRemoteTranslogEnabled() throws IOExcept public void testIllegalStateExceptionIfUnknownAllocationIdWithRemoteTranslogEnabled() { final AllocationId active = AllocationId.newInitializing(); final AllocationId initializing = AllocationId.newInitializing(); - Settings settings = Settings.builder().put("index.remote_store.translog.enabled", "true").build(); + Settings settings = Settings.builder().put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "true").build(); final ReplicationTracker tracker = newTracker(active, settings); tracker.updateFromClusterManager( randomNonNegativeLong(), 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 2dcca96c2a46a..bc423c38f3e01 100644 --- a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java @@ -2809,6 +2809,7 @@ public void testSyncSegmentsFromGivenRemoteSegmentStore() throws IOException { .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) .put(IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY, remoteStorePath + "__test") + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, remoteStorePath + "__test") .build(), new InternalEngineFactory() ); @@ -2847,6 +2848,8 @@ public void testSyncSegmentsFromGivenRemoteSegmentStore() throws IOException { Settings.builder() .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) + .put(IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY, remoteStorePath + "__test1") + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, remoteStorePath + "__test1") .build(), new InternalEngineFactory() ); @@ -2887,6 +2890,8 @@ public void testRestoreShardFromRemoteStore(boolean performFlush) throws IOExcep Settings.builder() .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) + .put(IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY, "temp-fs") + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, "temp-fs") .build(), new InternalEngineFactory() ); diff --git a/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java b/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java index 6f38f080e5035..5045f95d7b6ad 100644 --- a/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java +++ b/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java @@ -58,6 +58,8 @@ public void setup(boolean primary, int numberOfDocs) throws IOException { primary, Settings.builder() .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) + .put(IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY, "temp-fs") + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, "temp-fs") .put(SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) .build(), new InternalEngineFactory() @@ -386,6 +388,8 @@ private Tuple m true, Settings.builder() .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) + .put(IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY, "temp-fs") + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, "temp-fs") .put(SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) .build(), new InternalEngineFactory() diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java index aa590358efa25..461ab4df37065 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java @@ -892,6 +892,8 @@ private void testNRTReplicaWithRemoteStorePromotedAsPrimary(boolean performFlush Settings settings = Settings.builder() .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) + .put(IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY, "temp-fs") + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, "temp-fs") .build(); try (ReplicationGroup shards = createGroup(1, settings, indexMapping, new NRTReplicationEngineFactory(), createTempDir())) { @@ -934,6 +936,8 @@ private void testNRTReplicaWithRemoteStorePromotedAsPrimary(boolean performFlush assertTrue(nextPrimary.translogStats().estimatedNumberOfOperations() >= additonalDocs); assertTrue(nextPrimary.translogStats().getUncommittedOperations() >= additonalDocs); + int prevOperationCount = nextPrimary.translogStats().estimatedNumberOfOperations(); + // promote the replica shards.promoteReplicaToPrimary(nextPrimary).get(); @@ -946,7 +950,7 @@ private void testNRTReplicaWithRemoteStorePromotedAsPrimary(boolean performFlush // As we are downloading segments from remote segment store on failover, there should not be // any operations replayed from translog - assertEquals(0, nextPrimary.translogStats().estimatedNumberOfOperations()); + assertEquals(prevOperationCount, nextPrimary.translogStats().estimatedNumberOfOperations()); // refresh and push segments to our other replica. nextPrimary.refresh("test"); diff --git a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryTests.java b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryTests.java index c750bf0f56e36..7a92d7a1332bd 100644 --- a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryTests.java +++ b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryTests.java @@ -217,6 +217,7 @@ private Settings getRemoteStoreBackedIndexSettings(String remoteStoreRepo) { .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) .put(IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY, remoteStoreRepo) + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, remoteStoreRepo) .build(); } @@ -365,6 +366,7 @@ public void testGetRemoteStoreShallowCopyShardMetadata() throws IOException { .put("location", OpenSearchIntegTestCase.randomRepoPath(node().settings())) .build(); createRepository(client, remoteStoreRepositoryName, remoteStoreRepoSettings); + // createRepository(client, remoteStoreRepositoryName + "translog", remoteStoreRepoSettings); logger.info("--> creating a remote store enabled index and indexing documents"); final String remoteStoreIndexName = "test-rs-idx"; @@ -438,6 +440,7 @@ public void testRetrieveShallowCopySnapshotCase2() throws IOException { logger.info("--> creating remote store repository"); createRepository(client, remoteStoreRepositoryName); + createRepository(client, remoteStoreRepositoryName + "translog"); logger.info("--> creating an index and indexing documents"); final String indexName = "test-idx"; diff --git a/test/framework/src/main/java/org/opensearch/snapshots/AbstractSnapshotIntegTestCase.java b/test/framework/src/main/java/org/opensearch/snapshots/AbstractSnapshotIntegTestCase.java index 21c3b30b24939..7fdc21762965d 100644 --- a/test/framework/src/main/java/org/opensearch/snapshots/AbstractSnapshotIntegTestCase.java +++ b/test/framework/src/main/java/org/opensearch/snapshots/AbstractSnapshotIntegTestCase.java @@ -533,6 +533,7 @@ protected Settings getRemoteStoreBackedIndexSettings(String remoteStoreRepo) { .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) .put(IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY, remoteStoreRepo) + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, remoteStoreRepo) .build(); }