From 6130d6feda5075a695bc384b2702a76d4db383a6 Mon Sep 17 00:00:00 2001 From: Gaurav Bafna <85113518+gbbafna@users.noreply.github.com> Date: Mon, 4 Jul 2022 12:22:33 +0530 Subject: [PATCH 001/157] Correctly updating the followerCheckpoint in stats api (#438) Summary : We need to update followerCheckpoint after writing to the follower index. Currently, we are not waiting for the writes and updating it with soon-to-be stale values Signed-off-by: Gaurav Bafna --- .../task/shard/ShardReplicationTask.kt | 2 +- .../task/shard/TranslogSequencer.kt | 5 ++++ .../integ/rest/StartReplicationIT.kt | 1 + .../task/shard/TranslogSequencerTests.kt | 28 +++++++++++++------ 4 files changed, 26 insertions(+), 10 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationTask.kt b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationTask.kt index a7418917..44493bc7 100644 --- a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationTask.kt @@ -217,6 +217,7 @@ class ShardReplicationTask(id: Long, type: String, action: String, description: TaskId(clusterService.nodeName, id), client, indexShard.localCheckpoint, followerClusterStats) val changeTracker = ShardReplicationChangesTracker(indexShard, replicationSettings) + followerClusterStats.stats[followerShardId]!!.followerCheckpoint = indexShard.localCheckpoint coroutineScope { while (isActive) { rateLimiter.acquire() @@ -273,7 +274,6 @@ class ShardReplicationTask(id: Long, type: String, action: String, description: //hence renew retention lease with lastSyncedGlobalCheckpoint + 1 so that any shard that picks up shard replication task has data until then. try { retentionLeaseHelper.renewRetentionLease(leaderShardId, indexShard.lastSyncedGlobalCheckpoint + 1, followerShardId) - followerClusterStats.stats[followerShardId]!!.followerCheckpoint = indexShard.lastSyncedGlobalCheckpoint lastLeaseRenewalMillis = System.currentTimeMillis() } catch (ex: Exception) { when (ex) { diff --git a/src/main/kotlin/org/opensearch/replication/task/shard/TranslogSequencer.kt b/src/main/kotlin/org/opensearch/replication/task/shard/TranslogSequencer.kt index be5fe89c..38b625bf 100644 --- a/src/main/kotlin/org/opensearch/replication/task/shard/TranslogSequencer.kt +++ b/src/main/kotlin/org/opensearch/replication/task/shard/TranslogSequencer.kt @@ -28,6 +28,7 @@ import org.opensearch.client.Client import org.opensearch.common.logging.Loggers import org.opensearch.index.shard.ShardId import org.opensearch.index.translog.Translog +import org.opensearch.replication.util.indicesService import org.opensearch.tasks.TaskId import java.util.ArrayList import java.util.concurrent.ConcurrentHashMap @@ -55,6 +56,9 @@ class TranslogSequencer(scope: CoroutineScope, private val replicationMetadata: private val log = Loggers.getLogger(javaClass, followerShardId)!! private val completed = CompletableDeferred() + val followerIndexService = indicesService.indexServiceSafe(followerShardId.index) + val indexShard = followerIndexService.getShard(followerShardId.id) + private val sequencer = scope.actor(capacity = Channel.UNLIMITED) { // Exceptions thrown here will mark the channel as failed and the next attempt to send to the channel will // raise the same exception. See [SendChannel.close] method for details. @@ -88,6 +92,7 @@ class TranslogSequencer(scope: CoroutineScope, private val replicationMetadata: val tookInNanos = System.nanoTime() - relativeStartNanos followerClusterStats.stats[followerShardId]!!.totalWriteTime.addAndGet(TimeUnit.NANOSECONDS.toMillis(tookInNanos)) followerClusterStats.stats[followerShardId]!!.opsWritten.addAndGet(replayRequest.changes.size.toLong()) + followerClusterStats.stats[followerShardId]!!.followerCheckpoint = indexShard.localCheckpoint } highWatermark = next.changes.lastOrNull()?.seqNo() ?: highWatermark } diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt index e9bc717e..4c50bf3a 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt @@ -1091,6 +1091,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { assertThat(stats.getValue("operations_read").toString()).isEqualTo("50") assertThat(stats.getValue("failed_read_requests").toString()).isEqualTo("0") assertThat(stats.getValue("failed_write_requests").toString()).isEqualTo("0") + assertThat(stats.getValue("follower_checkpoint").toString()).isEqualTo((docCount-1).toString()) assertThat(stats.containsKey("index_stats")) assertThat(stats.size).isEqualTo(16) diff --git a/src/test/kotlin/org/opensearch/replication/task/shard/TranslogSequencerTests.kt b/src/test/kotlin/org/opensearch/replication/task/shard/TranslogSequencerTests.kt index ed5afb06..ac377687 100644 --- a/src/test/kotlin/org/opensearch/replication/task/shard/TranslogSequencerTests.kt +++ b/src/test/kotlin/org/opensearch/replication/task/shard/TranslogSequencerTests.kt @@ -11,32 +11,37 @@ package org.opensearch.replication.task.shard -import org.opensearch.replication.action.changes.GetChangesResponse -import org.opensearch.replication.action.replay.ReplayChangesAction -import org.opensearch.replication.action.replay.ReplayChangesRequest -import org.opensearch.replication.action.replay.ReplayChangesResponse -import org.opensearch.replication.metadata.ReplicationOverallState -import org.opensearch.replication.metadata.store.ReplicationContext -import org.opensearch.replication.metadata.store.ReplicationMetadata -import org.opensearch.replication.metadata.store.ReplicationStoreMetadataType import kotlinx.coroutines.ExperimentalCoroutinesApi import kotlinx.coroutines.ObsoleteCoroutinesApi import kotlinx.coroutines.test.runBlockingTest import org.assertj.core.api.Assertions.assertThat +import org.mockito.Mockito import org.opensearch.action.ActionListener import org.opensearch.action.ActionRequest import org.opensearch.action.ActionResponse import org.opensearch.action.ActionType import org.opensearch.action.support.replication.ReplicationResponse.ShardInfo import org.opensearch.common.settings.Settings +import org.opensearch.index.IndexService +import org.opensearch.index.shard.IndexShard import org.opensearch.index.shard.ShardId import org.opensearch.index.translog.Translog +import org.opensearch.indices.IndicesService +import org.opensearch.replication.action.changes.GetChangesResponse +import org.opensearch.replication.action.replay.ReplayChangesAction +import org.opensearch.replication.action.replay.ReplayChangesRequest +import org.opensearch.replication.action.replay.ReplayChangesResponse +import org.opensearch.replication.metadata.ReplicationOverallState +import org.opensearch.replication.metadata.store.ReplicationContext +import org.opensearch.replication.metadata.store.ReplicationMetadata +import org.opensearch.replication.metadata.store.ReplicationStoreMetadataType +import org.opensearch.replication.util.indicesService import org.opensearch.tasks.TaskId.EMPTY_TASK_ID import org.opensearch.test.OpenSearchTestCase -import org.opensearch.test.OpenSearchTestCase.randomList import org.opensearch.test.client.NoOpClient import java.util.Locale + @ObsoleteCoroutinesApi class TranslogSequencerTests : OpenSearchTestCase() { @@ -83,6 +88,11 @@ class TranslogSequencerTests : OpenSearchTestCase() { val stats = FollowerClusterStats() stats.stats[followerShardId] = FollowerShardMetric() val startSeqNo = randomNonNegativeLong() + indicesService = Mockito.mock(IndicesService::class.java) + val followerIndexService = Mockito.mock(IndexService::class.java) + val indexShard = Mockito.mock(IndexShard::class.java) + Mockito.`when`(indicesService.indexServiceSafe(followerShardId.index)).thenReturn(followerIndexService) + Mockito.`when`(followerIndexService.getShard(followerShardId.id)).thenReturn(indexShard) val sequencer = TranslogSequencer(this, replicationMetadata, followerShardId, leaderAlias, leaderIndex, EMPTY_TASK_ID, client, startSeqNo, stats) From 08ad60bf28a3b9fa226871e97048a9cf496138c8 Mon Sep 17 00:00:00 2001 From: Rishikesh Pasham <62345295+Rishikesh1159@users.noreply.github.com> Date: Mon, 18 Jul 2022 09:32:04 +0000 Subject: [PATCH 002/157] Changing leader_cluster to leader_alias in readme file (#453) Signed-off-by: Rishikesh1159 --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index ee66e368..a404690c 100644 --- a/README.md +++ b/README.md @@ -106,7 +106,7 @@ curl -XPOST "http://${LEADER}/leader-01/_doc/1" -H 'Content-Type: application/js ```bash curl -XPUT "http://${FOLLOWER}/_plugins/_replication/follower-01/_start?pretty" \ -H 'Content-type: application/json' \ --d'{"leader_cluster":"leader-cluster", "leader_index": "leader-01"}' +-d'{"leader_alias":"leader-cluster", "leader_index": "leader-01"}' ``` ### Step 5: Make changes to data on leader index From 27898e2f166416992b2cf852903822352762c54e Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Tue, 19 Jul 2022 09:59:05 +0530 Subject: [PATCH 003/157] Use the published zip for security plugin (#455) Signed-off-by: Ankit Kala --- .github/workflows/security-tests.yml | 12 ------------ build.gradle | 18 +++++++++++++++++- 2 files changed, 17 insertions(+), 13 deletions(-) diff --git a/.github/workflows/security-tests.yml b/.github/workflows/security-tests.yml index ee435d15..9b7bcb3f 100644 --- a/.github/workflows/security-tests.yml +++ b/.github/workflows/security-tests.yml @@ -22,18 +22,6 @@ jobs: # This step uses the checkout Github action: https://github.com/actions/checkout - name: Checkout Branch uses: actions/checkout@v2 - # Security plugin dependency - - name: Checkout security - uses: actions/checkout@v2 - with: - repository: 'opensearch-project/security' - path: security - ref: 'main' - - name: Build security - working-directory: ./security - run: | - ./gradlew clean build -Dbuild.snapshot=false -x test - cp build/distributions/opensearch-security-*.zip ../src/test/resources/security/plugin/opensearch-security.zip - name: Build and run Replication tests run: | ls -al src/test/resources/security/plugin diff --git a/build.gradle b/build.gradle index 4ff5e3fe..fbd17a12 100644 --- a/build.gradle +++ b/build.gradle @@ -56,6 +56,14 @@ buildscript { common_utils_version = System.getProperty("common_utils.version", opensearch_build) kotlin_version = System.getProperty("kotlin.version", "1.6.0") + // For fetching security zip from Maven. + // https://ci.opensearch.org/ci/dbc/distribution-build-opensearch/2.1.0/latest/linux/x64/tar/builds/opensearch/plugins/opensearch-security-2.1.0.0.zip + opensearch_no_snapshot = opensearch_version.replace("-SNAPSHOT","") + security_no_snapshot = opensearch_build.replace("-SNAPSHOT","") + security_plugin_path = "build/dependencies/security" + security_plugin_download_url = 'https://ci.opensearch.org/ci/dbc/distribution-build-opensearch/' + opensearch_no_snapshot + + '/latest/linux/x64/tar/builds/opensearch/plugins/opensearch-security-' + security_no_snapshot + '.zip' + } repositories { @@ -63,6 +71,7 @@ buildscript { mavenCentral() maven { url "https://aws.oss.sonatype.org/content/repositories/snapshots" } maven { url "https://plugins.gradle.org/m2/" } + maven { url "https://d1nvenhzbhpy0q.cloudfront.net/snapshots/lucene/" } } dependencies { @@ -209,7 +218,14 @@ def securityPluginFile = new Callable() { return new RegularFile() { @Override File getAsFile() { - return fileTree("$projectDir/src/test/resources/security/plugin/opensearch-security.zip").getSingleFile() + if (new File("$project.rootDir/$security_plugin_path").exists()) { + project.delete(files("$project.rootDir/$security_plugin_path")) + } + project.mkdir security_plugin_path + ant.get(src: security_plugin_download_url, + dest: security_plugin_path, + httpusecaches: false) + return fileTree(security_plugin_path).getSingleFile() } } } From db5029a1ce517d62141efe44b1a6ef624f3f143d Mon Sep 17 00:00:00 2001 From: Prudhvi Godithi Date: Wed, 27 Jul 2022 03:28:09 -0400 Subject: [PATCH 004/157] Staging for version increment automation (#449) * Version increment automation Signed-off-by: pgodithi * Version increment automation: task rename updateVersion Signed-off-by: pgodithi --- build.gradle | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index fbd17a12..ea688661 100644 --- a/build.gradle +++ b/build.gradle @@ -910,4 +910,15 @@ publishing { } } } -} \ No newline at end of file +} + +// updateVersion: Task to auto increment to the next development iteration +task updateVersion { + onlyIf { System.getProperty('newVersion') } + doLast { + ext.newVersion = System.getProperty('newVersion') + println "Setting version to ${newVersion}." + // String tokenization to support -SNAPSHOT + ant.replaceregexp(file:'build.gradle', match: '"opensearch.version", "\\d.*"', replace: '"opensearch.version", "' + newVersion.tokenize('-')[0] + '-SNAPSHOT"', flags:'g', byline:true) + } +} From 93b43f4504b3be58a4083b4da29c79729940a1d4 Mon Sep 17 00:00:00 2001 From: Gaurav Bafna <85113518+gbbafna@users.noreply.github.com> Date: Wed, 3 Aug 2022 17:00:25 +0530 Subject: [PATCH 005/157] Adding Index Settings validation before starting replication (#461) * Adding Index Settings validation before starting replication Signed-off-by: Gaurav Bafna * Retrieving default index settings before starting replication Signed-off-by: Gaurav Bafna --- .../index/TransportReplicateIndexAction.kt | 15 +++++++++---- .../replication/util/ValidationUtil.kt | 21 +++++++++++++++++++ .../integ/rest/StartReplicationIT.kt | 20 ++++++++++++++++++ 3 files changed, 52 insertions(+), 4 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/action/index/TransportReplicateIndexAction.kt b/src/main/kotlin/org/opensearch/replication/action/index/TransportReplicateIndexAction.kt index 4557e44d..563a9996 100644 --- a/src/main/kotlin/org/opensearch/replication/action/index/TransportReplicateIndexAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/index/TransportReplicateIndexAction.kt @@ -33,7 +33,7 @@ import org.opensearch.action.support.HandledTransportAction import org.opensearch.action.support.IndicesOptions import org.opensearch.client.Client import org.opensearch.cluster.ClusterState -import org.opensearch.cluster.metadata.IndexMetadata +import org.opensearch.cluster.metadata.MetadataCreateIndexService import org.opensearch.common.inject.Inject import org.opensearch.common.settings.Settings import org.opensearch.env.Environment @@ -48,7 +48,8 @@ class TransportReplicateIndexAction @Inject constructor(transportService: Transp val threadPool: ThreadPool, actionFilters: ActionFilters, private val client : Client, - private val environment: Environment) : + private val environment: Environment, + private val metadataCreateIndexService: MetadataCreateIndexService) : HandledTransportAction(ReplicateIndexAction.NAME, transportService, actionFilters, ::ReplicateIndexRequest), CoroutineScope by GlobalScope { @@ -102,7 +103,13 @@ class TransportReplicateIndexAction @Inject constructor(transportService: Transp throw IllegalArgumentException("Cannot replicate k-NN index - ${request.leaderIndex}") } - ValidationUtil.validateAnalyzerSettings(environment, leaderSettings, request.settings) + ValidationUtil.validateIndexSettings( + environment, + request.followerIndex, + leaderSettings, + request.settings, + metadataCreateIndexService + ) // Setup checks are successful and trigger replication for the index // permissions evaluation to trigger replication is based on the current security context set @@ -128,7 +135,7 @@ class TransportReplicateIndexAction @Inject constructor(transportService: Transp private suspend fun getLeaderIndexSettings(leaderAlias: String, leaderIndex: String): Settings { val remoteClient = client.getRemoteClusterClient(leaderAlias) - val getSettingsRequest = GetSettingsRequest().includeDefaults(false).indices(leaderIndex) + val getSettingsRequest = GetSettingsRequest().includeDefaults(true).indices(leaderIndex) val settingsResponse = remoteClient.suspending(remoteClient.admin().indices()::getSettings, injectSecurityContext = true)(getSettingsRequest) return settingsResponse.indexToSettings.get(leaderIndex) ?: throw IndexNotFoundException("${leaderAlias}:${leaderIndex}") diff --git a/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt b/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt index ad4c20d4..515c96ec 100644 --- a/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt +++ b/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt @@ -31,6 +31,27 @@ object ValidationUtil { private val log = LogManager.getLogger(ValidationUtil::class.java) + fun validateIndexSettings( + environment: Environment, + followerIndex: String, + leaderSettings: Settings, + overriddenSettings: Settings, + metadataCreateIndexService: MetadataCreateIndexService + ) { + val settingsList = arrayOf(leaderSettings, overriddenSettings) + val desiredSettingsBuilder = Settings.builder() + // Desired settings are taking leader Settings and then overriding them with desired settings + for (settings in settingsList) { + for (key in settings.keySet()) { + desiredSettingsBuilder.copy(key, settings); + } + } + val desiredSettings = desiredSettingsBuilder.build() + + metadataCreateIndexService.validateIndexSettings(followerIndex,desiredSettings, false) + validateAnalyzerSettings(environment, leaderSettings, overriddenSettings) + } + fun validateAnalyzerSettings(environment: Environment, leaderSettings: Settings, overriddenSettings: Settings) { val analyserSettings = leaderSettings.filter { k: String? -> k!!.matches(Regex("index.analysis.*path")) } for (analyserSetting in analyserSettings.keySet()) { diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt index 4c50bf3a..246641eb 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt @@ -1169,6 +1169,26 @@ class StartReplicationIT: MultiClusterRestTestCase() { } } + fun `test start replication invalid settings`() { + val followerClient = getClientForCluster(FOLLOWER) + val leaderClient = getClientForCluster(LEADER) + + createConnectionBetweenClusters(FOLLOWER, LEADER) + + val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) + assertThat(createIndexResponse.isAcknowledged).isTrue() + val settings = Settings.builder() + .put("index.data_path", "/random-path/invalid-setting") + .build() + + try { + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName, settings = settings)) + } catch (e: ResponseException) { + Assert.assertEquals(400, e.response.statusLine.statusCode) + Assert.assertTrue(e.message!!.contains("Validation Failed: 1: custom path [/random-path/invalid-setting] is not a sub-path of path.shared_data")) + } + } + fun `test that replication is not started when all primary shards are not in active state`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) From db1b226f7b4bc7c940d44e9b0f65f6e6ed4931c6 Mon Sep 17 00:00:00 2001 From: Sai Kumar Date: Thu, 4 Aug 2022 11:48:33 +0530 Subject: [PATCH 006/157] Changes to support replication plugin on OpenSearch v2.2.0 (#469) Signed-off-by: Sai Kumar --- .github/workflows/security-tests.yml | 4 ++-- build.gradle | 6 +++--- .../replication/metadata/TransportUpdateMetadataAction.kt | 8 ++++---- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/.github/workflows/security-tests.yml b/.github/workflows/security-tests.yml index 9b7bcb3f..a1415749 100644 --- a/.github/workflows/security-tests.yml +++ b/.github/workflows/security-tests.yml @@ -15,10 +15,10 @@ jobs: runs-on: ubuntu-latest steps: # This step uses the setup-java Github action: https://github.com/actions/setup-java - - name: Set Up JDK 11 + - name: Set Up JDK 17 uses: actions/setup-java@v1 with: - java-version: 11 + java-version: 17 # This step uses the checkout Github action: https://github.com/actions/checkout - name: Checkout Branch uses: actions/checkout@v2 diff --git a/build.gradle b/build.gradle index ea688661..ed8b3088 100644 --- a/build.gradle +++ b/build.gradle @@ -36,7 +36,7 @@ import org.opensearch.gradle.test.RestIntegTestTask buildscript { ext { isSnapshot = "true" == System.getProperty("build.snapshot", "true") - opensearch_version = System.getProperty("opensearch.version", "2.1.0-SNAPSHOT") + opensearch_version = System.getProperty("opensearch.version", "2.2.0-SNAPSHOT") buildVersionQualifier = System.getProperty("build.version_qualifier", "") // e.g. 2.0.0-rc1-SNAPSHOT -> 2.0.0.0-rc1-SNAPSHOT version_tokens = opensearch_version.tokenize('-') @@ -50,7 +50,7 @@ buildscript { // for bwc tests - opensearch_previous_version = System.getProperty("bwc_older_version", "1.3.1") + opensearch_previous_version = System.getProperty("bwc_older_version", "2.1.0") plugin_previous_version = opensearch_previous_version.replaceAll(/(\.\d)([^\d]*)$/, '$1.0$2') common_utils_version = System.getProperty("common_utils.version", opensearch_build) @@ -62,7 +62,7 @@ buildscript { security_no_snapshot = opensearch_build.replace("-SNAPSHOT","") security_plugin_path = "build/dependencies/security" security_plugin_download_url = 'https://ci.opensearch.org/ci/dbc/distribution-build-opensearch/' + opensearch_no_snapshot + - '/latest/linux/x64/tar/builds/opensearch/plugins/opensearch-security-' + security_no_snapshot + '.zip' + '/5858/linux/x64/tar/builds/opensearch/plugins/opensearch-security-' + security_no_snapshot + '.zip' } diff --git a/src/main/kotlin/org/opensearch/replication/metadata/TransportUpdateMetadataAction.kt b/src/main/kotlin/org/opensearch/replication/metadata/TransportUpdateMetadataAction.kt index 7a7fb09d..2fab74ab 100644 --- a/src/main/kotlin/org/opensearch/replication/metadata/TransportUpdateMetadataAction.kt +++ b/src/main/kotlin/org/opensearch/replication/metadata/TransportUpdateMetadataAction.kt @@ -29,8 +29,8 @@ import org.opensearch.action.admin.indices.settings.put.UpdateSettingsClusterSta import org.opensearch.action.admin.indices.settings.put.UpdateSettingsRequest import org.opensearch.action.support.ActionFilters import org.opensearch.action.support.IndicesOptions +import org.opensearch.action.support.clustermanager.TransportClusterManagerNodeAction import org.opensearch.action.support.master.AcknowledgedResponse -import org.opensearch.action.support.master.TransportMasterNodeAction import org.opensearch.cluster.ClusterState import org.opensearch.cluster.ack.ClusterStateUpdateResponse import org.opensearch.cluster.ack.OpenIndexClusterStateUpdateResponse @@ -61,7 +61,7 @@ class TransportUpdateMetadataAction @Inject constructor( val updateSettingsService: MetadataUpdateSettingsService, val indexAliasService: MetadataIndexAliasesService, val indexStateService: MetadataIndexStateService -) : TransportMasterNodeAction(UpdateMetadataAction.NAME, +) : TransportClusterManagerNodeAction(UpdateMetadataAction.NAME, transportService, clusterService, threadPool, actionFilters, ::UpdateMetadataRequest, indexNameExpressionResolver) { companion object { @@ -75,7 +75,7 @@ class TransportUpdateMetadataAction @Inject constructor( return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE) } - override fun masterOperation( + override fun clusterManagerOperation( task: Task, request: UpdateMetadataRequest, state: ClusterState, @@ -286,7 +286,7 @@ class TransportUpdateMetadataAction @Inject constructor( } } - override fun masterOperation(request: UpdateMetadataRequest?, state: ClusterState?, listener: ActionListener?) { + override fun clusterManagerOperation(request: UpdateMetadataRequest?, state: ClusterState?, listener: ActionListener?) { throw UnsupportedOperationException("The task parameter is required") } From a1d17d3415fcd44844da53a649d9d3dea53aa2f6 Mon Sep 17 00:00:00 2001 From: Sai Kumar Date: Fri, 5 Aug 2022 10:37:36 +0530 Subject: [PATCH 007/157] Modified security artifacts to fetch from latest build version (#474) Signed-off-by: Sai Kumar --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index ed8b3088..c5e8e0e3 100644 --- a/build.gradle +++ b/build.gradle @@ -62,7 +62,7 @@ buildscript { security_no_snapshot = opensearch_build.replace("-SNAPSHOT","") security_plugin_path = "build/dependencies/security" security_plugin_download_url = 'https://ci.opensearch.org/ci/dbc/distribution-build-opensearch/' + opensearch_no_snapshot + - '/5858/linux/x64/tar/builds/opensearch/plugins/opensearch-security-' + security_no_snapshot + '.zip' + '/latest/linux/x64/tar/builds/opensearch/plugins/opensearch-security-' + security_no_snapshot + '.zip' } From a8595962e6278760931fe7b6f20795ab57c2cbea Mon Sep 17 00:00:00 2001 From: Sai Kumar Date: Tue, 9 Aug 2022 10:06:27 +0530 Subject: [PATCH 008/157] Added release notes for v2.2.0 release (#477) Signed-off-by: Sai Kumar --- ...ch-cross-cluster-replication.release-notes-2.2.0.0.md | 9 +++++++++ 1 file changed, 9 insertions(+) create mode 100644 release-notes/opensearch-cross-cluster-replication.release-notes-2.2.0.0.md diff --git a/release-notes/opensearch-cross-cluster-replication.release-notes-2.2.0.0.md b/release-notes/opensearch-cross-cluster-replication.release-notes-2.2.0.0.md new file mode 100644 index 00000000..d3530e21 --- /dev/null +++ b/release-notes/opensearch-cross-cluster-replication.release-notes-2.2.0.0.md @@ -0,0 +1,9 @@ +## Version 2.2.0.0 Release Notes + +Compatible with OpenSearch 2.2.0 + +### Bug Fixes +* Adding Index Settings validation before starting replication ([#461](https://github.com/opensearch-project/cross-cluster-replication/pull/461)) + +### Infrastructure +* Use the published zip for security plugin ([#455](https://github.com/opensearch-project/cross-cluster-replication/pull/455)) From 4470a774e4a49d72423c07af87a7754eeeb5116a Mon Sep 17 00:00:00 2001 From: Gaurav Bafna <85113518+gbbafna@users.noreply.github.com> Date: Wed, 24 Aug 2022 11:43:25 +0530 Subject: [PATCH 009/157] Updating filters as well during Alias update (#491) Testing : Integ Test, Local Signed-off-by: Gaurav Bafna --- .../task/index/IndexReplicationTask.kt | 19 ++++++++++++------- .../integ/rest/StartReplicationIT.kt | 10 +++++++--- 2 files changed, 19 insertions(+), 10 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt index 33a311c0..156c2732 100644 --- a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt @@ -518,13 +518,18 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript for (alias in toAdd) { log.info("Adding alias ${alias.alias} from $followerIndexName") // Copying writeIndex from leader doesn't cause any issue as writes will be blocked anyways - request.addAliasAction(AliasActions.add().index(followerIndexName) - .alias(alias.alias) - .indexRouting(alias.indexRouting) - .searchRouting(alias.searchRouting) - .writeIndex(alias.writeIndex()) - .isHidden(alias.isHidden) - ) + var aliasAction = AliasActions.add().index(followerIndexName) + .alias(alias.alias) + .indexRouting(alias.indexRouting) + .searchRouting(alias.searchRouting) + .writeIndex(alias.writeIndex()) + .isHidden(alias.isHidden) + + if (alias.filteringRequired()) { + aliasAction = aliasAction.filter(alias.filter.string()) + } + + request.addAliasAction(aliasAction) } var toRemove = followerAliases - leaderAliases diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt index 246641eb..ee82c3c4 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt @@ -65,6 +65,7 @@ import org.opensearch.index.mapper.MapperService import org.opensearch.repositories.fs.FsRepository import org.opensearch.test.OpenSearchTestCase.assertBusy import org.junit.Assert +import org.opensearch.cluster.metadata.AliasMetadata import org.opensearch.common.xcontent.DeprecationHandler import org.opensearch.common.xcontent.NamedXContentRegistry import org.opensearch.replication.ReplicationPlugin.Companion.REPLICATION_INDEX_TRANSLOG_PRUNING_ENABLED_SETTING @@ -344,8 +345,10 @@ class StartReplicationIT: MultiClusterRestTestCase() { createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName).alias(Alias("leaderAlias")), RequestOptions.DEFAULT) - assertThat(createIndexResponse.isAcknowledged).isTrue() + val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName) + .alias(Alias("leaderAlias").filter("{\"term\":{\"year\":2016}}").routing("1")) + , RequestOptions.DEFAULT) + assertThat(createIndexResponse.isAcknowledged).isTrue try { followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) @@ -361,6 +364,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { followerClient.indices().getAlias(GetAliasesRequest().indices(followerIndexName), RequestOptions.DEFAULT).aliases[followerIndexName] ) + }, 30L, TimeUnit.SECONDS) } finally { followerClient.stopReplication(followerIndexName) @@ -541,7 +545,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { var indicesAliasesRequest = IndicesAliasesRequest() var aliasAction = IndicesAliasesRequest.AliasActions.add() .index(leaderIndexName) - .alias("alias1") + .alias("alias1").filter("{\"term\":{\"year\":2016}}").routing("1") indicesAliasesRequest.addAliasAction(aliasAction) leaderClient.indices().updateAliases(indicesAliasesRequest, RequestOptions.DEFAULT) From 0bad3074ea75527af227cd9228e088b1877dae95 Mon Sep 17 00:00:00 2001 From: Sai Kumar Date: Wed, 24 Aug 2022 12:21:13 +0530 Subject: [PATCH 010/157] Modified _stop replication API to remove any stale replication settings on existing index (#410) Signed-off-by: Sai Kumar --- .../TransportStopIndexReplicationAction.kt | 29 ++++--- .../replication/MultiClusterRestTestCase.kt | 31 ++++++- .../integ/rest/StopReplicationIT.kt | 82 ++++++++++++++++++- 3 files changed, 128 insertions(+), 14 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/action/stop/TransportStopIndexReplicationAction.kt b/src/main/kotlin/org/opensearch/replication/action/stop/TransportStopIndexReplicationAction.kt index dcc5e9e5..fd82bba4 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stop/TransportStopIndexReplicationAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stop/TransportStopIndexReplicationAction.kt @@ -22,10 +22,7 @@ import org.opensearch.replication.metadata.UpdateMetadataAction import org.opensearch.replication.metadata.UpdateMetadataRequest import org.opensearch.replication.metadata.state.REPLICATION_LAST_KNOWN_OVERALL_STATE import org.opensearch.replication.metadata.state.getReplicationStateParamsForIndex -import org.opensearch.replication.metadata.store.ReplicationMetadata import org.opensearch.replication.seqno.RemoteClusterRetentionLeaseHelper -import org.opensearch.replication.task.index.IndexReplicationParams -import org.opensearch.replication.util.completeWith import org.opensearch.replication.util.coroutineContext import org.opensearch.replication.util.suspendExecute import org.opensearch.replication.util.suspending @@ -39,7 +36,6 @@ import org.opensearch.OpenSearchException import org.opensearch.action.ActionListener import org.opensearch.action.admin.indices.open.OpenIndexRequest import org.opensearch.action.support.ActionFilters -import org.opensearch.action.support.IndicesOptions import org.opensearch.action.support.master.AcknowledgedResponse import org.opensearch.action.support.master.TransportMasterNodeAction import org.opensearch.client.Client @@ -57,8 +53,6 @@ import org.opensearch.cluster.service.ClusterService import org.opensearch.common.inject.Inject import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.settings.Settings -import org.opensearch.index.IndexNotFoundException -import org.opensearch.index.shard.ShardId import org.opensearch.replication.util.stackTraceToString import org.opensearch.threadpool.ThreadPool import org.opensearch.transport.TransportService @@ -99,7 +93,7 @@ class TransportStopIndexReplicationAction @Inject constructor(transportService: throw OpenSearchException("Failed to remove index block on ${request.indexName}") } - validateStopReplicationRequest(request) + validateReplicationStateOfIndex(request) // Index will be deleted if replication is stopped while it is restoring. So no need to close/reopen val restoring = clusterService.state().custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY).any { entry -> @@ -117,8 +111,9 @@ class TransportStopIndexReplicationAction @Inject constructor(transportService: throw OpenSearchException("Unable to close index: ${request.indexName}") } } - val replMetadata = replicationMetadataManager.getIndexReplicationMetadata(request.indexName) + try { + val replMetadata = replicationMetadataManager.getIndexReplicationMetadata(request.indexName) val remoteClient = client.getRemoteClusterClient(replMetadata.connectionName) val retentionLeaseHelper = RemoteClusterRetentionLeaseHelper(clusterService.clusterName.value(), remoteClient) retentionLeaseHelper.attemptRemoveRetentionLease(clusterService, replMetadata, request.indexName) @@ -127,12 +122,12 @@ class TransportStopIndexReplicationAction @Inject constructor(transportService: } val clusterStateUpdateResponse : AcknowledgedResponse = - clusterService.waitForClusterStateUpdate("stop_replication") { l -> StopReplicationTask(request, l)} + clusterService.waitForClusterStateUpdate("stop_replication") { l -> StopReplicationTask(request, l)} if (!clusterStateUpdateResponse.isAcknowledged) { throw OpenSearchException("Failed to update cluster state") } - // Index will be deleted if stop is called while it is restoring. So no need to reopen + // Index will be deleted if stop is called while it is restoring. So no need to reopen if (!restoring && state.routingTable.hasIndex(request.indexName)) { val reopenResponse = client.suspending(client.admin().indices()::open, injectSecurityContext = true)(OpenIndexRequest(request.indexName)) @@ -149,7 +144,15 @@ class TransportStopIndexReplicationAction @Inject constructor(transportService: } } - private fun validateStopReplicationRequest(request: StopIndexReplicationRequest) { + private fun validateReplicationStateOfIndex(request: StopIndexReplicationRequest) { + // If replication blocks/settings are present, Stop action should proceed with the clean-up + // This can happen during settings of follower index are carried over in the snapshot and the restore is + // performed using this snapshot. + if (clusterService.state().blocks.hasIndexBlock(request.indexName, INDEX_REPLICATION_BLOCK) + || clusterService.state().metadata.index(request.indexName)?.settings?.get(REPLICATED_INDEX_SETTING.key) != null) { + return + } + val replicationStateParams = getReplicationStateParamsForIndex(clusterService, request.indexName) ?: throw IllegalArgumentException("No replication in progress for index:${request.indexName}") @@ -187,13 +190,15 @@ class TransportStopIndexReplicationAction @Inject constructor(transportService: val mdBuilder = Metadata.builder(currentState.metadata) // remove replicated index setting val currentIndexMetadata = currentState.metadata.index(request.indexName) - if (currentIndexMetadata != null) { + if (currentIndexMetadata != null && + currentIndexMetadata.settings[REPLICATED_INDEX_SETTING.key] != null) { val newIndexMetadata = IndexMetadata.builder(currentIndexMetadata) .settings(Settings.builder().put(currentIndexMetadata.settings).putNull(REPLICATED_INDEX_SETTING.key)) .settingsVersion(1 + currentIndexMetadata.settingsVersion) mdBuilder.put(newIndexMetadata) } newState.metadata(mdBuilder) + return newState.build() } diff --git a/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt b/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt index 8f09b766..888d8af9 100644 --- a/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt +++ b/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt @@ -108,6 +108,9 @@ abstract class MultiClusterRestTestCase : OpenSearchTestCase() { val lowLevelClient = restClient.lowLevelClient!! var defaultSecuritySetupCompleted = false + companion object { + const val FS_SNAPSHOT_REPO = "repl_repo" + } } companion object { @@ -253,7 +256,33 @@ abstract class MultiClusterRestTestCase : OpenSearchTestCase() { */ @Before fun setup() { - testClusters.values.forEach { if(it.securityEnabled && !it.defaultSecuritySetupCompleted) setupDefaultSecurityRoles(it) } + testClusters.values.forEach { + registerSnapshotRepository(it) + if(it.securityEnabled && !it.defaultSecuritySetupCompleted) + setupDefaultSecurityRoles(it) + } + } + + /** + * Register snapshot repo - "fs" type on all the clusters + */ + private fun registerSnapshotRepository(testCluster: TestCluster) { + val getResponse: Map = OpenSearchRestTestCase.entityAsMap(testCluster.lowLevelClient.performRequest( + Request("GET", "/_cluster/settings?include_defaults=true&flat_settings=true"))) + val configuredRepositories = (getResponse["defaults"] as Map<*, *>)["path.repo"] as List<*> + if(configuredRepositories.isEmpty()) { + return + } + val repo = configuredRepositories[0] as String + val repoConfig = """ + { + "type": "fs", + "settings": { + "location": "$repo" + } + } + """.trimIndent() + triggerRequest(testCluster.lowLevelClient, "PUT", "_snapshot/${TestCluster.FS_SNAPSHOT_REPO}", repoConfig) } /** diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/StopReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/StopReplicationIT.kt index 970d6109..09b797ae 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/StopReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/StopReplicationIT.kt @@ -17,22 +17,30 @@ import org.opensearch.replication.MultiClusterRestTestCase import org.opensearch.replication.StartReplicationRequest import org.opensearch.replication.startReplication import org.opensearch.replication.stopReplication +import org.opensearch.replication.replicationStatus +import org.opensearch.replication.getShardReplicationTasks +import org.opensearch.replication.`validate status syncing response` import org.apache.http.util.EntityUtils import org.assertj.core.api.Assertions.assertThat import org.assertj.core.api.Assertions.assertThatThrownBy +import org.junit.Assert import org.opensearch.OpenSearchStatusException import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest +import org.opensearch.action.admin.cluster.snapshots.create.CreateSnapshotRequest +import org.opensearch.action.admin.cluster.snapshots.restore.RestoreSnapshotRequest +import org.opensearch.action.admin.cluster.snapshots.status.SnapshotsStatusRequest import org.opensearch.action.index.IndexRequest import org.opensearch.client.Request import org.opensearch.client.RequestOptions import org.opensearch.client.ResponseException import org.opensearch.client.indices.CreateIndexRequest import org.opensearch.client.indices.GetIndexRequest +import org.opensearch.cluster.SnapshotsInProgress import org.opensearch.cluster.metadata.IndexMetadata import org.opensearch.common.settings.Settings import org.opensearch.common.unit.TimeValue import org.opensearch.index.mapper.MapperService -import org.opensearch.test.OpenSearchTestCase.assertBusy +import java.util.Random import java.util.concurrent.TimeUnit @@ -234,4 +242,76 @@ class StopReplicationIT: MultiClusterRestTestCase() { val sourceMap = mapOf("name" to randomAlphaOfLength(5)) followerClient.index(IndexRequest(followerIndexName).id("2").source(sourceMap), RequestOptions.DEFAULT) } + + fun `test stop replication with stale replication settings at leader cluster`() { + val followerClient = getClientForCluster(FOLLOWER) + val leaderClient = getClientForCluster(LEADER) + createConnectionBetweenClusters(FOLLOWER, LEADER, "source") + + val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) + assertThat(createIndexResponse.isAcknowledged).isTrue() + val snapshotSuffix = Random().nextInt(1000).toString() + + try { + followerClient.startReplication( + StartReplicationRequest("source", leaderIndexName, followerIndexName), + TimeValue.timeValueSeconds(10), + true + ) + + assertBusy({ + var statusResp = followerClient.replicationStatus(followerIndexName) + `validate status syncing response`(statusResp) + assertThat(followerClient.getShardReplicationTasks(followerIndexName)).isNotEmpty() + }, 60, TimeUnit.SECONDS) + + // Trigger snapshot on the follower cluster + val createSnapshotRequest = CreateSnapshotRequest(TestCluster.FS_SNAPSHOT_REPO, "test-$snapshotSuffix") + createSnapshotRequest.waitForCompletion(true) + followerClient.snapshot().create(createSnapshotRequest, RequestOptions.DEFAULT) + + assertBusy { + var snapshotStatusResponse = followerClient.snapshot().status(SnapshotsStatusRequest(TestCluster.FS_SNAPSHOT_REPO, + arrayOf("test-$snapshotSuffix")), RequestOptions.DEFAULT) + for (snapshotStatus in snapshotStatusResponse.snapshots) { + Assert.assertEquals(SnapshotsInProgress.State.SUCCESS, snapshotStatus.state) + } + } + + // Restore follower index on leader cluster + val restoreSnapshotRequest = RestoreSnapshotRequest(TestCluster.FS_SNAPSHOT_REPO, "test-$snapshotSuffix") + restoreSnapshotRequest.indices(followerIndexName) + restoreSnapshotRequest.waitForCompletion(true) + restoreSnapshotRequest.renamePattern("(.+)") + restoreSnapshotRequest.renameReplacement("restored-\$1") + leaderClient.snapshot().restore(restoreSnapshotRequest, RequestOptions.DEFAULT) + + assertBusy { + assertThat(leaderClient.indices().exists(GetIndexRequest("restored-$followerIndexName"), RequestOptions.DEFAULT)).isEqualTo(true) + } + + // Invoke stop on the new leader cluster index + assertThatThrownBy { leaderClient.stopReplication("restored-$followerIndexName") } + .isInstanceOf(ResponseException::class.java) + .hasMessageContaining("Metadata for restored-$followerIndexName doesn't exist") + + // Start replication on the new leader index + followerClient.startReplication( + StartReplicationRequest("source", "restored-$followerIndexName", "restored-$followerIndexName"), + TimeValue.timeValueSeconds(10), + true, true + ) + + assertBusy({ + var statusResp = followerClient.replicationStatus("restored-$followerIndexName") + `validate status syncing response`(statusResp) + assertThat(followerClient.getShardReplicationTasks("restored-$followerIndexName")).isNotEmpty() + }, 60, TimeUnit.SECONDS) + + } finally { + followerClient.stopReplication("restored-$followerIndexName") + followerClient.stopReplication(followerIndexName) + } + + } } From 8f0a55c9a7e32f643345a972d28e4829d7e3c530 Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Fri, 26 Aug 2022 14:00:04 +0530 Subject: [PATCH 011/157] Add lastExecutionTime for autofollow coroutine (#508) Signed-off-by: Ankit Kala Signed-off-by: Ankit Kala --- .../task/autofollow/AutoFollowTask.kt | 6 +++++ .../integ/rest/UpdateAutoFollowPatternIT.kt | 23 +++++++++++++++++++ 2 files changed, 29 insertions(+) diff --git a/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowTask.kt b/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowTask.kt index 570e39b7..0685b79d 100644 --- a/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowTask.kt @@ -74,6 +74,7 @@ class AutoFollowTask(id: Long, type: String, action: String, description: String try { addRetryScheduler() pollForIndices() + stat.lastExecutionTime = System.currentTimeMillis() delay(replicationSettings.autofollowFetchPollDuration.millis) } catch(e: OpenSearchException) { @@ -252,6 +253,7 @@ class AutoFollowStat: Task.Status { var failCounterForRun :Long=0 var successCount: Long=0 var failedLeaderCall :Long=0 + var lastExecutionTime : Long=0 constructor(name: String, pattern: String) { @@ -266,6 +268,7 @@ class AutoFollowStat: Task.Status { failedIndices = inp.readSet(StreamInput::readString) successCount = inp.readLong() failedLeaderCall = inp.readLong() + lastExecutionTime = inp.readLong() } override fun writeTo(out: StreamOutput) { @@ -275,6 +278,7 @@ class AutoFollowStat: Task.Status { out.writeCollection(failedIndices, StreamOutput::writeString) out.writeLong(successCount) out.writeLong(failedLeaderCall) + out.writeLong(lastExecutionTime) } override fun getWriteableName(): String { @@ -289,6 +293,8 @@ class AutoFollowStat: Task.Status { builder.field("num_failed_start_replication", failCount) builder.field("num_failed_leader_calls", failedLeaderCall) builder.field("failed_indices", failedIndices) + builder.field("last_execution_time", lastExecutionTime) return builder.endObject() } + } diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt index d1a757bd..82db8fb5 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt @@ -91,6 +91,7 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { .isEqualTo(true) followerClient.waitForShardTaskStart(leaderIndexNameNew, waitForShardTask) followerClient.waitForShardTaskStart(leaderIndexName, waitForShardTask) + var stats = followerClient.AutoFollowStats() var af_stats = stats.get("autofollow_stats")!! as ArrayList> for (key in af_stats) { @@ -118,8 +119,13 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { TimeValue.timeValueSeconds(30)) val clusterUpdateSetttingsReq = ClusterUpdateSettingsRequest().persistentSettings(settings) val clusterUpdateResponse = followerClient.cluster().putSettings(clusterUpdateSetttingsReq, RequestOptions.DEFAULT) + + var lastExecutionTime = 0L + var stats = followerClient.AutoFollowStats() + Assert.assertTrue(clusterUpdateResponse.isAcknowledged) followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, indexPattern) + leaderIndexNameNew = createRandomIndex(leaderClient) // Verify that newly created index on leader which match the pattern are also replicated. assertBusy({ @@ -127,8 +133,25 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { .exists(GetIndexRequest(leaderIndexNameNew), RequestOptions.DEFAULT)) .isEqualTo(true) followerClient.waitForShardTaskStart(leaderIndexNameNew, waitForShardTask) + var af_stats = stats.get("autofollow_stats")!! as ArrayList> + for (key in af_stats) { + if(key["name"] == indexPatternName) { + Assertions.assertThat(key["last_execution_time"]!! as Long).isNotEqualTo(0L) + lastExecutionTime = key["last_execution_time"]!! as Long + } + } + }, 30, TimeUnit.SECONDS) + assertBusy({ + var af_stats = stats.get("autofollow_stats")!! as ArrayList> + for (key in af_stats) { + if(key["name"] == indexPatternName) { + Assertions.assertThat(key["last_execution_time"]!! as Long).isNotEqualTo(lastExecutionTime) + } + } + }, 40, TimeUnit.SECONDS) + } finally { followerClient.deleteAutoFollowPattern(connectionAlias, indexPatternName) From 805f686a34393cf2cf26cf7011b71396c26f4fe3 Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Fri, 26 Aug 2022 14:06:14 +0530 Subject: [PATCH 012/157] Fix for missing ShardReplicationTasks on new nodes (#497) Signed-off-by: Ankit Kala Signed-off-by: Ankit Kala --- .../task/index/IndexReplicationTask.kt | 53 ++++++++-------- .../task/index/IndexReplicationTaskTests.kt | 61 ++++++++++++++++++- 2 files changed, 85 insertions(+), 29 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt index 156c2732..906312ac 100644 --- a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt @@ -183,7 +183,8 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript ReplicationState.INIT_FOLLOW -> { log.info("Starting shard tasks") addIndexBlockForReplication() - startShardFollowTasks(emptyMap()) + FollowingState(startNewOrMissingShardTasks()) + } ReplicationState.FOLLOWING -> { if (currentTaskState is FollowingState) { @@ -206,8 +207,8 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript // Tasks need to be started state } else { - state = pollShardTaskStatus((followingTaskState as FollowingState).shardReplicationTasks) - followingTaskState = startMissingShardTasks((followingTaskState as FollowingState).shardReplicationTasks) + state = pollShardTaskStatus() + followingTaskState = FollowingState(startNewOrMissingShardTasks()) when (state) { is MonitoringState -> { updateMetadata() @@ -285,24 +286,7 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript clusterService.addListener(this) } - private suspend fun startMissingShardTasks(shardTasks: Map>): IndexReplicationState { - val persistentTasks = clusterService.state().metadata.custom(PersistentTasksCustomMetadata.TYPE) - - val runningShardTasks = persistentTasks.findTasks(ShardReplicationExecutor.TASK_NAME, Predicate { true }).stream() - .map { task -> task.params as ShardReplicationParams } - .collect(Collectors.toList()) - - val runningTasksForCurrentIndex = shardTasks.filter { entry -> runningShardTasks.find { task -> task.followerShardId == entry.key } != null} - - val numMissingTasks = shardTasks.size - runningTasksForCurrentIndex.size - if (numMissingTasks > 0) { - log.info("Starting $numMissingTasks missing shard task(s)") - return startShardFollowTasks(runningTasksForCurrentIndex) - } - return FollowingState(shardTasks) - } - - private suspend fun pollShardTaskStatus(shardTasks: Map>): IndexReplicationState { + private suspend fun pollShardTaskStatus(): IndexReplicationState { val failedShardTasks = findAllReplicationFailedShardTasks(followerIndexName, clusterService.state()) if (failedShardTasks.isNotEmpty()) { log.info("Failed shard tasks - ", failedShardTasks) @@ -343,11 +327,16 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript registerCloseListeners() val clusterState = clusterService.state() val persistentTasks = clusterState.metadata.custom(PersistentTasksCustomMetadata.TYPE) - val runningShardTasks = persistentTasks.findTasks(ShardReplicationExecutor.TASK_NAME, Predicate { true }).stream() + + val followerShardIds = clusterService.state().routingTable.indicesRouting().get(followerIndexName).shards() + .map { shard -> shard.value.shardId } + .stream().collect(Collectors.toSet()) + val runningShardTasksForIndex = persistentTasks.findTasks(ShardReplicationExecutor.TASK_NAME, Predicate { true }).stream() .map { task -> task.params as ShardReplicationParams } + .filter {taskParam -> followerShardIds.contains(taskParam.followerShardId) } .collect(Collectors.toList()) - if (runningShardTasks.size == 0) { + if (runningShardTasksForIndex.size != followerShardIds.size) { return InitFollowState } @@ -696,19 +685,27 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript } } - private suspend fun - startShardFollowTasks(tasks: Map>): FollowingState { + suspend fun startNewOrMissingShardTasks(): Map> { assert(clusterService.state().routingTable.hasIndex(followerIndexName)) { "Can't find index $followerIndexName" } val shards = clusterService.state().routingTable.indicesRouting().get(followerIndexName).shards() - val newTasks = shards.map { + val persistentTasks = clusterService.state().metadata.custom(PersistentTasksCustomMetadata.TYPE) + val runningShardTasks = persistentTasks.findTasks(ShardReplicationExecutor.TASK_NAME, Predicate { true }).stream() + .map { task -> task as PersistentTask } + .filter { task -> task.params!!.followerShardId.indexName == followerIndexName} + .collect(Collectors.toMap( + {t: PersistentTask -> t.params!!.followerShardId}, + {t: PersistentTask -> t})) + + val tasks = shards.map { it.value.shardId }.associate { shardId -> - val task = tasks.getOrElse(shardId) { + val task = runningShardTasks.getOrElse(shardId) { startReplicationTask(ShardReplicationParams(leaderAlias, ShardId(leaderIndex, shardId.id), shardId)) } return@associate shardId to task } - return FollowingState(newTasks) + + return tasks } private suspend fun cancelRestore() { diff --git a/src/test/kotlin/org/opensearch/replication/task/index/IndexReplicationTaskTests.kt b/src/test/kotlin/org/opensearch/replication/task/index/IndexReplicationTaskTests.kt index 95b3f6f2..569eadf8 100644 --- a/src/test/kotlin/org/opensearch/replication/task/index/IndexReplicationTaskTests.kt +++ b/src/test/kotlin/org/opensearch/replication/task/index/IndexReplicationTaskTests.kt @@ -55,7 +55,6 @@ import org.opensearch.tasks.TaskManager import org.opensearch.test.ClusterServiceUtils import org.opensearch.test.ClusterServiceUtils.setState import org.opensearch.test.OpenSearchTestCase -import org.opensearch.test.OpenSearchTestCase.assertBusy import org.opensearch.threadpool.TestThreadPool import java.util.* import java.util.concurrent.TimeUnit @@ -150,6 +149,66 @@ class IndexReplicationTaskTests : OpenSearchTestCase() { } + fun testStartNewShardTasks() = runBlocking { + val replicationTask: IndexReplicationTask = spy(createIndexReplicationTask()) + var taskManager = Mockito.mock(TaskManager::class.java) + replicationTask.setPersistent(taskManager) + var rc = ReplicationContext(followerIndex) + var rm = ReplicationMetadata(connectionName, ReplicationStoreMetadataType.INDEX.name, ReplicationOverallState.RUNNING.name, "reason", rc, rc, Settings.EMPTY) + replicationTask.setReplicationMetadata(rm) + + // Build cluster state + val indices: MutableList = ArrayList() + indices.add(followerIndex) + var metadata = Metadata.builder() + .put(IndexMetadata.builder(REPLICATION_CONFIG_SYSTEM_INDEX).settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(0)) + .put(IndexMetadata.builder(followerIndex).settings(settings(Version.CURRENT)).numberOfShards(2).numberOfReplicas(0)) + .build() + var routingTableBuilder = RoutingTable.builder() + .addAsNew(metadata.index(REPLICATION_CONFIG_SYSTEM_INDEX)) + .addAsNew(metadata.index(followerIndex)) + var newClusterState = ClusterState.builder(clusterService.state()).routingTable(routingTableBuilder.build()).build() + setState(clusterService, newClusterState) + + // Try starting shard tasks + val shardTasks = replicationTask.startNewOrMissingShardTasks() + assertThat(shardTasks.size == 2).isTrue + } + + + fun testStartMissingShardTasks() = runBlocking { + val replicationTask: IndexReplicationTask = spy(createIndexReplicationTask()) + var taskManager = Mockito.mock(TaskManager::class.java) + replicationTask.setPersistent(taskManager) + var rc = ReplicationContext(followerIndex) + var rm = ReplicationMetadata(connectionName, ReplicationStoreMetadataType.INDEX.name, ReplicationOverallState.RUNNING.name, "reason", rc, rc, Settings.EMPTY) + replicationTask.setReplicationMetadata(rm) + + // Build cluster state + val indices: MutableList = ArrayList() + indices.add(followerIndex) + + val tasks = PersistentTasksCustomMetadata.builder() + var sId = ShardId(Index(followerIndex, "_na_"), 0) + tasks.addTask( "replication:0", ShardReplicationExecutor.TASK_NAME, ShardReplicationParams("remoteCluster", sId, sId), + PersistentTasksCustomMetadata.Assignment("other_node_", "test assignment on other node")) + + var metadata = Metadata.builder() + .put(IndexMetadata.builder(REPLICATION_CONFIG_SYSTEM_INDEX).settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(0)) + .put(IndexMetadata.builder(followerIndex).settings(settings(Version.CURRENT)).numberOfShards(2).numberOfReplicas(0)) + .putCustom(PersistentTasksCustomMetadata.TYPE, tasks.build()) + .build() + var routingTableBuilder = RoutingTable.builder() + .addAsNew(metadata.index(REPLICATION_CONFIG_SYSTEM_INDEX)) + .addAsNew(metadata.index(followerIndex)) + var newClusterState = ClusterState.builder(clusterService.state()).routingTable(routingTableBuilder.build()).build() + setState(clusterService, newClusterState) + + // Try starting shard tasks + val shardTasks = replicationTask.startNewOrMissingShardTasks() + assertThat(shardTasks.size == 2).isTrue + } + private fun createIndexReplicationTask() : IndexReplicationTask { var threadPool = TestThreadPool("IndexReplicationTask") //Hack Alert : Though it is meant to force rejection , this is to make overallTaskScope not null From 81c2002c889094ae4071e403e76393c00c9b567c Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Mon, 29 Aug 2022 04:23:08 -0700 Subject: [PATCH 013/157] For segrep enabled indices, use NRTReplicationEngine for replica shards (#486) * Update engine factory to return NRTReplicationEngine for replica shards Signed-off-by: Suraj Singh * Address review comments Signed-off-by: Suraj Singh Signed-off-by: Suraj Singh --- .../org/opensearch/replication/ReplicationPlugin.kt | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt b/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt index 3fa602b0..53c77866 100644 --- a/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt +++ b/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt @@ -143,6 +143,9 @@ import org.opensearch.watcher.ResourceWatcherService import java.util.Optional import java.util.function.Supplier +import org.opensearch.index.engine.NRTReplicationEngine + + @OpenForTesting internal class ReplicationPlugin : Plugin(), ActionPlugin, PersistentTaskPlugin, RepositoryPlugin, EnginePlugin { @@ -359,7 +362,14 @@ internal class ReplicationPlugin : Plugin(), ActionPlugin, PersistentTaskPlugin, override fun getEngineFactory(indexSettings: IndexSettings): Optional { return if (indexSettings.settings.get(REPLICATED_INDEX_SETTING.key) != null) { - Optional.of(EngineFactory { config -> ReplicationEngine(config) }) + Optional.of(EngineFactory { config -> + // Use NRTSegmentReplicationEngine for SEGMENT replication type indices replica shards + if (config.isReadOnlyReplica) { + NRTReplicationEngine(config) + } else { + ReplicationEngine(config) + } + }) } else { Optional.empty() } From 5ac1023fbe04b3fa4ca90b781279ab9ddf157a3f Mon Sep 17 00:00:00 2001 From: Aman Khare <85096200+amkhar@users.noreply.github.com> Date: Fri, 2 Sep 2022 09:49:55 +0530 Subject: [PATCH 014/157] Bumped snakeyaml version to address CVE-2022-25857 (#537) Signed-off-by: Aman Khare Signed-off-by: Aman Khare Co-authored-by: Aman Khare --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index c5e8e0e3..e56ddd5c 100644 --- a/build.gradle +++ b/build.gradle @@ -121,7 +121,7 @@ configurations.all { force 'org.apache.httpcomponents.client5:httpclient5:5.0.3' force 'org.apache.httpcomponents.client5:httpclient5-osgi:5.0.3' force 'com.fasterxml.jackson.core:jackson-databind:2.12.6' - force 'org.yaml:snakeyaml:1.26' + force 'org.yaml:snakeyaml:1.31' force 'org.codehaus.plexus:plexus-utils:3.0.24' } } From 967f833584b70fc5249f799cd9771541bf810a87 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Mon, 5 Sep 2022 15:35:30 +0530 Subject: [PATCH 015/157] Added support for integTest remote on Remote clusters Signed-off-by: Monu Singh --- build.gradle | 26 ++++++-- scripts/integtest.sh | 59 +++++++++++++++++-- .../integ/rest/ResumeReplicationIT.kt | 26 +++++++- .../integ/rest/StartReplicationIT.kt | 33 +++++++++++ .../singleCluster/SingleClusterSanityIT.kt | 2 +- 5 files changed, 134 insertions(+), 12 deletions(-) diff --git a/build.gradle b/build.gradle index e56ddd5c..a68f931e 100644 --- a/build.gradle +++ b/build.gradle @@ -877,17 +877,31 @@ task "bwcTestSuite"(type: RestIntegTestTask) { dependsOn tasks.named("fullRestartClusterTask") } -task integTestRemote(type: RestIntegTestTask) { +task integTestRemote (type: RestIntegTestTask) { doFirst { systemProperty "tests.cluster.follower.http_hosts", System.getProperty("follower.http_host") systemProperty "tests.cluster.follower.transport_hosts", System.getProperty("follower.transport_host") systemProperty "tests.cluster.follower.security_enabled", System.getProperty("security_enabled") + + + systemProperty "tests.cluster.leaderCluster.http_hosts", System.getProperty("leader.http_host") + systemProperty "tests.cluster.leaderCluster.transport_hosts", System.getProperty("leader.transport_host") + systemProperty "tests.cluster.leaderCluster.security_enabled", System.getProperty("leader.security_enabled") + + systemProperty "tests.cluster.followCluster.http_hosts", System.getProperty("follower.http_host") + systemProperty "tests.cluster.followCluster.transport_hosts", System.getProperty("follower.transport_host") + systemProperty "tests.cluster.followCluster.security_enabled", System.getProperty("follower.security_enabled") + + nonInputProperties.systemProperty('tests.integTestRemote', "true") + + systemProperty "build.dir", "${buildDir}" + filter { + setIncludePatterns("org.opensearch.replication.singleCluster.SingleClusterSanityIT") + + setExcludePatterns("org.opensearch.replication.bwc.BackwardsCompatibilityIT") + } + nonInputProperties.systemProperty('tests.sanitySingleCluster', "integTestSingleCluster") } - filter { - setIncludePatterns("org.opensearch.replication.singleCluster.SingleClusterSanityIT") - } - nonInputProperties.systemProperty('tests.sanitySingleCluster', "integTestSingleCluster") -} publishing { publications { diff --git a/scripts/integtest.sh b/scripts/integtest.sh index 6c7fb46f..bd53109b 100755 --- a/scripts/integtest.sh +++ b/scripts/integtest.sh @@ -1,7 +1,5 @@ #!/bin/bash - set -e - function usage() { echo "" echo "This script is used to run integration tests for plugin installed on a remote OpenSearch/Dashboards cluster." @@ -15,6 +13,12 @@ function usage() { echo -e "-b BIND_ADDRESS\t, defaults to localhost | 127.0.0.1, can be changed to any IP or domain name for the cluster location." echo -e "-p BIND_PORT\t, defaults to 9200, can be changed to any port for the cluster location." echo -e "-t TRANSPORT_PORT\t, defaults to 9300, can be changed to any port for the cluster location." + echo -e "-a Leader BIND_ADDRESS\t, defaults to localhost | 127.0.0.1, can be changed to any IP or domain name for the cluster location." + echo -e "-b Leader BIND_PORT\t, defaults to 9200, can be changed to any port for the cluster location." + echo -e "-c Leader TRANSPORT_PORT\t, defaults to 9300, can be changed to any port for the cluster location." + echo -e "-x Follower BIND_ADDRESS\t, defaults to localhost | 127.0.0.1, can be changed to any IP or domain name for the cluster location." + echo -e "-y Follower BIND_PORT\t, defaults to 9200, can be changed to any port for the cluster location." + echo -e "-z Follower TRANSPORT_PORT\t, defaults to 9300, can be changed to any port for the cluster location." echo -e "-s SECURITY_ENABLED\t(true | false), defaults to true. Specify the OpenSearch/Dashboards have security enabled or not." echo -e "-c CREDENTIAL\t(usename:password), no defaults, effective when SECURITY_ENABLED=true." echo -e "-h\tPrint this message." @@ -24,19 +28,34 @@ function usage() { } while getopts ":h:b:p:s:c:v:n:t:" arg; do +while getopts ":h:a:b:c:x:y:z:s:c:" arg; do case $arg in h) usage exit 1 ;; + a) + LBIND_ADDRESS=$OPTARG + ;; b) BIND_ADDRESS=$OPTARG + LBIND_PORT=$OPTARG + ;; + c) + LTRANSPORT_PORT=$OPTARG + ;; + x) + FBIND_ADDRESS=$OPTARG ;; p) BIND_PORT=$OPTARG + y) + FBIND_PORT=$OPTARG ;; t) TRANSPORT_PORT=$OPTARG + z) + FTRANSPORT_PORT=$OPTARG ;; s) SECURITY_ENABLED=$OPTARG @@ -64,31 +83,63 @@ done if [ -z "$BIND_ADDRESS" ] +if [ -z "$LBIND_ADDRESS" ] then BIND_ADDRESS="localhost" + echo "requires an argument -lb " + usage + exit 1 fi if [ -z "$BIND_PORT" ] +if [ -z "$LBIND_PORT" ] then BIND_PORT="9200" + echo "requires an argument -lp " + usage + exit 1 fi if [ -z "$TRANSPORT_PORT" ] +if [ -z "$LTRANSPORT_PORT" ] then TRANSPORT_PORT="9300" + echo "requires an argument -lt " + usage + exit 1 +fi + +if [ -z "$FBIND_ADDRESS" ] +then + echo "requires an argument -fb " + usage + exit 1 +fi + +if [ -z "$FBIND_PORT" ] +then + echo "requires an argument -fp " + usage + exit 1 +fi + +if [ -z "$FTRANSPORT_PORT" ] +then + echo "requires an argument -ft " + usage + exit 1 fi if [ -z "$SECURITY_ENABLED" ] then SECURITY_ENABLED="true" fi - if [ -z "$CREDENTIAL" ] then CREDENTIAL="admin:admin" fi - USERNAME=`echo $CREDENTIAL | awk -F ':' '{print $1}'` PASSWORD=`echo $CREDENTIAL | awk -F ':' '{print $2}'` ./gradlew integTestRemote -Dfollower.http_host="$BIND_ADDRESS:$BIND_PORT" -Dfollower.transport_host="$BIND_ADDRESS:$TRANSPORT_PORT" -Dsecurity_enabled=$SECURITY_ENABLED -Duser=$USERNAME -Dpassword=$PASSWORD --console=plain +eval "./gradlew integTestRemote -Dleader.http_host=\"$LBIND_ADDRESS:$LBIND_PORT\" -Dleader.transport_host=\"$LBIND_ADDRESS:$LTRANSPORT_PORT\" -Dfollower.http_host=\"$FBIND_ADDRESS:$FBIND_PORT\" -Dfollower.transport_host=\"$FBIND_ADDRESS:$FTRANSPORT_PORT\" -Dsecurity_enabled=\"$SECURITY_ENABLED\" -Duser=\"$USERNAME\" -Dpassword=\"$PASSWORD\" --console=plain " diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/ResumeReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/ResumeReplicationIT.kt index a67b46f5..d4c95cd2 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/ResumeReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/ResumeReplicationIT.kt @@ -42,7 +42,7 @@ import org.opensearch.common.settings.Settings import org.junit.Assert import java.nio.file.Files import java.util.concurrent.TimeUnit - +import org.opensearch.bootstrap.BootstrapInfo @MultiClusterAnnotations.ClusterConfigurations( MultiClusterAnnotations.ClusterConfiguration(clusterName = LEADER), @@ -210,6 +210,14 @@ class ResumeReplicationIT: MultiClusterRestTestCase() { } fun `test that replication fails to resume when custom analyser is not present in follower`() { + + //check if integTest remote = true + val systemProperties = BootstrapInfo.getSystemProperties() + val integTestRemote = systemProperties.get("tests.integTestRemote") as String? + if(integTestRemote.equals("true")){ + return; + } + val synonyms = javaClass.getResourceAsStream("/analyzers/synonyms.txt") val config = PathUtils.get(buildDir, leaderClusterPath, "config") val synonymPath = config.resolve("synonyms.txt") @@ -251,6 +259,14 @@ class ResumeReplicationIT: MultiClusterRestTestCase() { } fun `test that replication resumes when custom analyser is present in follower`() { + + //check if integTest remote = true + val systemProperties = BootstrapInfo.getSystemProperties() + val integTestRemote = systemProperties.get("tests.integTestRemote") as String? + if(integTestRemote.equals("true")){ + return; + } + val synonyms = javaClass.getResourceAsStream("/analyzers/synonyms.txt") val config = PathUtils.get(buildDir, leaderClusterPath, "config") val synonymFilename = "synonyms.txt" @@ -300,6 +316,14 @@ class ResumeReplicationIT: MultiClusterRestTestCase() { } fun `test that replication resumes when custom analyser is overridden and present in follower`() { + + //check if integTest remote = true + val systemProperties = BootstrapInfo.getSystemProperties() + val integTestRemote = systemProperties.get("tests.integTestRemote") as String? + if(integTestRemote.equals("true")){ + return; + } + val synonyms = javaClass.getResourceAsStream("/analyzers/synonyms.txt") val config = PathUtils.get(buildDir, leaderClusterPath, "config") val synonymPath = config.resolve("synonyms.txt") diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt index ee82c3c4..49d2934f 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt @@ -74,6 +74,7 @@ import org.opensearch.replication.leaderStats import org.opensearch.replication.updateReplicationStartBlockSetting import java.nio.file.Files import java.util.concurrent.TimeUnit +import org.opensearch.bootstrap.BootstrapInfo @MultiClusterAnnotations.ClusterConfigurations( @@ -700,6 +701,14 @@ class StartReplicationIT: MultiClusterRestTestCase() { } fun `test that replication fails to start when custom analyser is not present in follower`() { + + //check if integTest remote = true + val systemProperties = BootstrapInfo.getSystemProperties() + val integTestRemote = systemProperties.get("tests.integTestRemote") as String? + if(integTestRemote.equals("true")){ + return; + } + val synonyms = javaClass.getResourceAsStream("/analyzers/synonyms.txt") val config = PathUtils.get(buildDir, leaderClusterPath, "config") val synonymPath = config.resolve("synonyms.txt") @@ -733,6 +742,14 @@ class StartReplicationIT: MultiClusterRestTestCase() { } fun `test that replication starts successfully when custom analyser is present in follower`() { + + //check if integTest remote = true + val systemProperties = BootstrapInfo.getSystemProperties() + val integTestRemote = systemProperties.get("tests.integTestRemote") as String? + if(integTestRemote.equals("true")){ + return; + } + val synonyms = javaClass.getResourceAsStream("/analyzers/synonyms.txt") val leaderConfig = PathUtils.get(buildDir, leaderClusterPath, "config") val leaderSynonymPath = leaderConfig.resolve("synonyms.txt") @@ -778,6 +795,14 @@ class StartReplicationIT: MultiClusterRestTestCase() { } fun `test that replication starts successfully when custom analyser is overridden and present in follower`() { + + //check if integTest remote = true + val systemProperties = BootstrapInfo.getSystemProperties() + val integTestRemote = systemProperties.get("tests.integTestRemote") as String? + if(integTestRemote.equals("true")){ + return; + } + val synonyms = javaClass.getResourceAsStream("/analyzers/synonyms.txt") val leaderConfig = PathUtils.get(buildDir, leaderClusterPath, "config") val leaderSynonymPath = leaderConfig.resolve("synonyms.txt") @@ -942,6 +967,14 @@ class StartReplicationIT: MultiClusterRestTestCase() { } fun `test that snapshot on leader does not affect replication during bootstrap`() { + + //check if integTest remote = true + val systemProperties = BootstrapInfo.getSystemProperties() + val integTestRemote = systemProperties.get("tests.integTestRemote") as String? + if(integTestRemote.equals("true")){ + return; + } + val settings = Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 20) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) diff --git a/src/test/kotlin/org/opensearch/replication/singleCluster/SingleClusterSanityIT.kt b/src/test/kotlin/org/opensearch/replication/singleCluster/SingleClusterSanityIT.kt index f7021ccb..9760e7c2 100644 --- a/src/test/kotlin/org/opensearch/replication/singleCluster/SingleClusterSanityIT.kt +++ b/src/test/kotlin/org/opensearch/replication/singleCluster/SingleClusterSanityIT.kt @@ -20,7 +20,7 @@ class SingleClusterSanityIT : MultiClusterRestTestCase() { companion object { private val log = LogManager.getLogger(SingleClusterSanityIT::class.java) - private const val followerClusterName = "follower" + private const val followerClusterName = "followCluster" private const val REPLICATION_PLUGIN_NAME = "opensearch-cross-cluster-replication" private const val SAMPLE_INDEX = "sample_test_index" From 9e646bdcc7371cffe3bea9f9e453f98579076e4f Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Mon, 5 Sep 2022 15:40:33 +0530 Subject: [PATCH 016/157] Fixed missing '}' Signed-off-by: Monu Singh --- build.gradle | 2 ++ 1 file changed, 2 insertions(+) diff --git a/build.gradle b/build.gradle index a68f931e..f2e01f20 100644 --- a/build.gradle +++ b/build.gradle @@ -895,6 +895,8 @@ task integTestRemote (type: RestIntegTestTask) { nonInputProperties.systemProperty('tests.integTestRemote', "true") systemProperty "build.dir", "${buildDir}" + + } filter { setIncludePatterns("org.opensearch.replication.singleCluster.SingleClusterSanityIT") From fea39587c9dc0718ca5753b2e92e8498cdf0ca47 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Tue, 6 Sep 2022 14:56:30 +0530 Subject: [PATCH 017/157] Re Signed-off-by: Monu Singh --- scripts/integtest.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/scripts/integtest.sh b/scripts/integtest.sh index bd53109b..509be9e6 100755 --- a/scripts/integtest.sh +++ b/scripts/integtest.sh @@ -141,5 +141,4 @@ fi USERNAME=`echo $CREDENTIAL | awk -F ':' '{print $1}'` PASSWORD=`echo $CREDENTIAL | awk -F ':' '{print $2}'` -./gradlew integTestRemote -Dfollower.http_host="$BIND_ADDRESS:$BIND_PORT" -Dfollower.transport_host="$BIND_ADDRESS:$TRANSPORT_PORT" -Dsecurity_enabled=$SECURITY_ENABLED -Duser=$USERNAME -Dpassword=$PASSWORD --console=plain eval "./gradlew integTestRemote -Dleader.http_host=\"$LBIND_ADDRESS:$LBIND_PORT\" -Dleader.transport_host=\"$LBIND_ADDRESS:$LTRANSPORT_PORT\" -Dfollower.http_host=\"$FBIND_ADDRESS:$FBIND_PORT\" -Dfollower.transport_host=\"$FBIND_ADDRESS:$FTRANSPORT_PORT\" -Dsecurity_enabled=\"$SECURITY_ENABLED\" -Duser=\"$USERNAME\" -Dpassword=\"$PASSWORD\" --console=plain " From 58dd3307894a3bef9961af9f8cea52e1d9f7e899 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Tue, 6 Sep 2022 15:17:02 +0530 Subject: [PATCH 018/157] Re1 Signed-off-by: Monu Singh --- scripts/integtest.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/scripts/integtest.sh b/scripts/integtest.sh index 509be9e6..3a1ce4c3 100755 --- a/scripts/integtest.sh +++ b/scripts/integtest.sh @@ -141,4 +141,5 @@ fi USERNAME=`echo $CREDENTIAL | awk -F ':' '{print $1}'` PASSWORD=`echo $CREDENTIAL | awk -F ':' '{print $2}'` + eval "./gradlew integTestRemote -Dleader.http_host=\"$LBIND_ADDRESS:$LBIND_PORT\" -Dleader.transport_host=\"$LBIND_ADDRESS:$LTRANSPORT_PORT\" -Dfollower.http_host=\"$FBIND_ADDRESS:$FBIND_PORT\" -Dfollower.transport_host=\"$FBIND_ADDRESS:$FTRANSPORT_PORT\" -Dsecurity_enabled=\"$SECURITY_ENABLED\" -Duser=\"$USERNAME\" -Dpassword=\"$PASSWORD\" --console=plain " From 22c87583dcaf823226344fbb1a4f0aa89d611291 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Tue, 6 Sep 2022 15:53:30 +0530 Subject: [PATCH 019/157] Re2 Signed-off-by: Monu Singh --- build.gradle | 1 - scripts/integtest.sh | 25 ------------------------- 2 files changed, 26 deletions(-) diff --git a/build.gradle b/build.gradle index f2e01f20..c9074687 100644 --- a/build.gradle +++ b/build.gradle @@ -898,7 +898,6 @@ task integTestRemote (type: RestIntegTestTask) { } filter { - setIncludePatterns("org.opensearch.replication.singleCluster.SingleClusterSanityIT") setExcludePatterns("org.opensearch.replication.bwc.BackwardsCompatibilityIT") } diff --git a/scripts/integtest.sh b/scripts/integtest.sh index 3a1ce4c3..e1aafda3 100755 --- a/scripts/integtest.sh +++ b/scripts/integtest.sh @@ -7,12 +7,6 @@ function usage() { echo "Usage: $0 [args]" echo "" echo "Required arguments:" - echo "None" - echo "" - echo "Optional arguments:" - echo -e "-b BIND_ADDRESS\t, defaults to localhost | 127.0.0.1, can be changed to any IP or domain name for the cluster location." - echo -e "-p BIND_PORT\t, defaults to 9200, can be changed to any port for the cluster location." - echo -e "-t TRANSPORT_PORT\t, defaults to 9300, can be changed to any port for the cluster location." echo -e "-a Leader BIND_ADDRESS\t, defaults to localhost | 127.0.0.1, can be changed to any IP or domain name for the cluster location." echo -e "-b Leader BIND_PORT\t, defaults to 9200, can be changed to any port for the cluster location." echo -e "-c Leader TRANSPORT_PORT\t, defaults to 9300, can be changed to any port for the cluster location." @@ -27,7 +21,6 @@ function usage() { echo "--------------------------------------------------------------------------" } -while getopts ":h:b:p:s:c:v:n:t:" arg; do while getopts ":h:a:b:c:x:y:z:s:c:" arg; do case $arg in h) @@ -38,7 +31,6 @@ while getopts ":h:a:b:c:x:y:z:s:c:" arg; do LBIND_ADDRESS=$OPTARG ;; b) - BIND_ADDRESS=$OPTARG LBIND_PORT=$OPTARG ;; c) @@ -47,13 +39,9 @@ while getopts ":h:a:b:c:x:y:z:s:c:" arg; do x) FBIND_ADDRESS=$OPTARG ;; - p) - BIND_PORT=$OPTARG y) FBIND_PORT=$OPTARG ;; - t) - TRANSPORT_PORT=$OPTARG z) FTRANSPORT_PORT=$OPTARG ;; @@ -63,12 +51,6 @@ while getopts ":h:a:b:c:x:y:z:s:c:" arg; do c) CREDENTIAL=$OPTARG ;; - v) - # Do nothing as we're not consuming this param. - ;; - n) - # Do nothing as we're not consuming this param. - ;; :) echo "-${OPTARG} requires an argument" usage @@ -81,29 +63,22 @@ while getopts ":h:a:b:c:x:y:z:s:c:" arg; do esac done - -if [ -z "$BIND_ADDRESS" ] if [ -z "$LBIND_ADDRESS" ] then - BIND_ADDRESS="localhost" echo "requires an argument -lb " usage exit 1 fi -if [ -z "$BIND_PORT" ] if [ -z "$LBIND_PORT" ] then - BIND_PORT="9200" echo "requires an argument -lp " usage exit 1 fi -if [ -z "$TRANSPORT_PORT" ] if [ -z "$LTRANSPORT_PORT" ] then - TRANSPORT_PORT="9300" echo "requires an argument -lt " usage exit 1 From 336337b117b8aa14e4fa803fb67c3c463970278e Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Fri, 9 Sep 2022 13:05:17 +0530 Subject: [PATCH 020/157] Made changes to allow running IntegTestRemote for both single or multiple clusters to ensure tests in opensearch-build pass in both cases. Signed-off-by: Monu Singh --- build.gradle | 28 ++++----- scripts/integtest.sh | 145 +++++++++++++++++++++++++++---------------- 2 files changed, 103 insertions(+), 70 deletions(-) diff --git a/build.gradle b/build.gradle index c9074687..026524a0 100644 --- a/build.gradle +++ b/build.gradle @@ -879,30 +879,28 @@ task "bwcTestSuite"(type: RestIntegTestTask) { task integTestRemote (type: RestIntegTestTask) { doFirst { - systemProperty "tests.cluster.follower.http_hosts", System.getProperty("follower.http_host") - systemProperty "tests.cluster.follower.transport_hosts", System.getProperty("follower.transport_host") - systemProperty "tests.cluster.follower.security_enabled", System.getProperty("security_enabled") - - - systemProperty "tests.cluster.leaderCluster.http_hosts", System.getProperty("leader.http_host") - systemProperty "tests.cluster.leaderCluster.transport_hosts", System.getProperty("leader.transport_host") - systemProperty "tests.cluster.leaderCluster.security_enabled", System.getProperty("leader.security_enabled") systemProperty "tests.cluster.followCluster.http_hosts", System.getProperty("follower.http_host") systemProperty "tests.cluster.followCluster.transport_hosts", System.getProperty("follower.transport_host") systemProperty "tests.cluster.followCluster.security_enabled", System.getProperty("follower.security_enabled") - nonInputProperties.systemProperty('tests.integTestRemote', "true") - - systemProperty "build.dir", "${buildDir}" - + if (System.getProperty("leader.transport_host") != null) { + systemProperty "tests.cluster.leaderCluster.http_hosts", System.getProperty("leader.http_host") + systemProperty "tests.cluster.leaderCluster.transport_hosts", System.getProperty("leader.transport_host") + systemProperty "tests.cluster.leaderCluster.security_enabled", System.getProperty("leader.security_enabled") + nonInputProperties.systemProperty('tests.integTestRemote', "true") + systemProperty "build.dir", "${buildDir}" + } } - filter { - + filter { + if (System.getProperty("leader.transport_host") != null) { setExcludePatterns("org.opensearch.replication.bwc.BackwardsCompatibilityIT") + }else{ + setIncludePatterns("org.opensearch.replication.singleCluster.SingleClusterSanityIT") } - nonInputProperties.systemProperty('tests.sanitySingleCluster', "integTestSingleCluster") } + nonInputProperties.systemProperty('tests.sanitySingleCluster', "integTestSingleCluster") +} publishing { publications { diff --git a/scripts/integtest.sh b/scripts/integtest.sh index e1aafda3..12b2bf5c 100755 --- a/scripts/integtest.sh +++ b/scripts/integtest.sh @@ -1,39 +1,57 @@ #!/bin/bash + set -e + function usage() { echo "" echo "This script is used to run integration tests for plugin installed on a remote OpenSearch/Dashboards cluster." echo "--------------------------------------------------------------------------" echo "Usage: $0 [args]" echo "" - echo "Required arguments:" - echo -e "-a Leader BIND_ADDRESS\t, defaults to localhost | 127.0.0.1, can be changed to any IP or domain name for the cluster location." - echo -e "-b Leader BIND_PORT\t, defaults to 9200, can be changed to any port for the cluster location." - echo -e "-c Leader TRANSPORT_PORT\t, defaults to 9300, can be changed to any port for the cluster location." - echo -e "-x Follower BIND_ADDRESS\t, defaults to localhost | 127.0.0.1, can be changed to any IP or domain name for the cluster location." - echo -e "-y Follower BIND_PORT\t, defaults to 9200, can be changed to any port for the cluster location." - echo -e "-z Follower TRANSPORT_PORT\t, defaults to 9300, can be changed to any port for the cluster location." + echo "Optional arguments:" echo -e "-s SECURITY_ENABLED\t(true | false), defaults to true. Specify the OpenSearch/Dashboards have security enabled or not." echo -e "-c CREDENTIAL\t(usename:password), no defaults, effective when SECURITY_ENABLED=true." - echo -e "-h\tPrint this message." + echo -e "-h Print this message." echo -e "-v OPENSEARCH_VERSION\t, no defaults" - echo -e "-n SNAPSHOT\t, defaults to false" + echo -e "-n SNAPSHOT\t\t, defaults to false" + echo "Required arguments:" + echo "Single cluster test:" + echo -e "-b BIND_ADDRESS\t\t, IP or domain name for the cluster location." + echo -e "-p BIND_PORT\t\t, port for the cluster location." + echo -e "-t TRANSPORT_PORT\t, defaults to 9300, can be changed to any port for the cluster location." + echo "--------------------------------------------------------------------------" + echo "Multi cluster test:" + echo -e "-m Leader BIND_ADDRESS\t\t, defaults to localhost | 127.0.0.1, can be changed to any IP or domain name for the cluster location." + echo -e "-n Leader BIND_PORT\t\t, defaults to 9200, can be changed to any port for the cluster location." + echo -e "-o Leader TRANSPORT_PORT\t, defaults to 9300, can be changed to any port for the cluster location." + echo -e "-x Follower BIND_ADDRESS\t, defaults to localhost | 127.0.0.1, can be changed to any IP or domain name for the cluster location." + echo -e "-y Follower BIND_PORT\t\t, defaults to 9200, can be changed to any port for the cluster location." + echo -e "-z Follower TRANSPORT_PORT\t, defaults to 9300, can be changed to any port for the cluster location." echo "--------------------------------------------------------------------------" } -while getopts ":h:a:b:c:x:y:z:s:c:" arg; do +while getopts ":h:b:p:t:m:n:o:x:y:z:s:c:" arg; do case $arg in h) usage exit 1 ;; - a) + b) + BIND_ADDRESS=$OPTARG + ;; + p) + BIND_PORT=$OPTARG + ;; + t) + TRANSPORT_PORT=$OPTARG + ;; + m) LBIND_ADDRESS=$OPTARG ;; - b) + n) LBIND_PORT=$OPTARG ;; - c) + o) LTRANSPORT_PORT=$OPTARG ;; x) @@ -63,58 +81,75 @@ while getopts ":h:a:b:c:x:y:z:s:c:" arg; do esac done -if [ -z "$LBIND_ADDRESS" ] +# Common starts +if [ -z "$SECURITY_ENABLED" ] then - echo "requires an argument -lb " - usage - exit 1 + SECURITY_ENABLED="true" fi -if [ -z "$LBIND_PORT" ] +if [ -z "$CREDENTIAL" ] then - echo "requires an argument -lp " - usage - exit 1 + CREDENTIAL="admin:admin" fi -if [ -z "$LTRANSPORT_PORT" ] -then - echo "requires an argument -lt " - usage - exit 1 -fi +USERNAME=`echo $CREDENTIAL | awk -F ':' '{print $1}'` +PASSWORD=`echo $CREDENTIAL | awk -F ':' '{print $2}'` +# Common ends -if [ -z "$FBIND_ADDRESS" ] -then - echo "requires an argument -fb " - usage - exit 1 -fi -if [ -z "$FBIND_PORT" ] -then - echo "requires an argument -fp " - usage - exit 1 -fi +# Check if test is run on multiple cluster -if [ -z "$FTRANSPORT_PORT" ] +if [ -z "$BIND_ADDRESS" ] || [ -z "$BIND_PORT" ] then - echo "requires an argument -ft " - usage - exit 1 -fi + #Proceeding with multi cluster test + if [ -z "$LBIND_ADDRESS" ] + then + echo "requires an argument -m " + usage + exit 1 + fi -if [ -z "$SECURITY_ENABLED" ] -then - SECURITY_ENABLED="true" -fi -if [ -z "$CREDENTIAL" ] -then - CREDENTIAL="admin:admin" -fi -USERNAME=`echo $CREDENTIAL | awk -F ':' '{print $1}'` -PASSWORD=`echo $CREDENTIAL | awk -F ':' '{print $2}'` + if [ -z "$LBIND_PORT" ] + then + echo "requires an argument -n " + usage + exit 1 + fi + + if [ -z "$LTRANSPORT_PORT" ] + then + echo "requires an argument -o " + usage + exit 1 + fi + + if [ -z "$FBIND_ADDRESS" ] + then + echo "requires an argument -fb " + usage + exit 1 + fi + + if [ -z "$FBIND_PORT" ] + then + echo "requires an argument -fp " + usage + exit 1 + fi + if [ -z "$FTRANSPORT_PORT" ] + then + echo "requires an argument -ft " + usage + exit 1 + fi + eval "./gradlew integTestRemote -Dleader.http_host=\"$LBIND_ADDRESS:$LBIND_PORT\" -Dleader.transport_host=\"$LBIND_ADDRESS:$LTRANSPORT_PORT\" -Dfollower.http_host=\"$FBIND_ADDRESS:$FBIND_PORT\" -Dfollower.transport_host=\"$FBIND_ADDRESS:$FTRANSPORT_PORT\" -Dsecurity_enabled=\"$SECURITY_ENABLED\" -Duser=\"$USERNAME\" -Dpassword=\"$PASSWORD\" --console=plain " -eval "./gradlew integTestRemote -Dleader.http_host=\"$LBIND_ADDRESS:$LBIND_PORT\" -Dleader.transport_host=\"$LBIND_ADDRESS:$LTRANSPORT_PORT\" -Dfollower.http_host=\"$FBIND_ADDRESS:$FBIND_PORT\" -Dfollower.transport_host=\"$FBIND_ADDRESS:$FTRANSPORT_PORT\" -Dsecurity_enabled=\"$SECURITY_ENABLED\" -Duser=\"$USERNAME\" -Dpassword=\"$PASSWORD\" --console=plain " +else + # Single cluster + if [ -z "$TRANSPORT_PORT" ] + then + TRANSPORT_PORT="9300" + fi + ./gradlew integTestRemote -Dfollower.http_host="$BIND_ADDRESS:$BIND_PORT" -Dfollower.transport_host="$BIND_ADDRESS:$TRANSPORT_PORT" -Dsecurity_enabled=$SECURITY_ENABLED -Duser=$USERNAME -Dpassword=$PASSWORD --console=plain +fi \ No newline at end of file From 659122ec23c050a4a85ba60b046f1997645d8e9a Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Fri, 9 Sep 2022 13:23:20 +0530 Subject: [PATCH 021/157] Created seperate gradle target for single cluster and multi cluster integtest Signed-off-by: Monu Singh --- build.gradle | 30 ++++++++++++++++++++---------- scripts/integtest.sh | 2 +- 2 files changed, 21 insertions(+), 11 deletions(-) diff --git a/build.gradle b/build.gradle index 026524a0..1be4e2e7 100644 --- a/build.gradle +++ b/build.gradle @@ -884,20 +884,30 @@ task integTestRemote (type: RestIntegTestTask) { systemProperty "tests.cluster.followCluster.transport_hosts", System.getProperty("follower.transport_host") systemProperty "tests.cluster.followCluster.security_enabled", System.getProperty("follower.security_enabled") - if (System.getProperty("leader.transport_host") != null) { - systemProperty "tests.cluster.leaderCluster.http_hosts", System.getProperty("leader.http_host") - systemProperty "tests.cluster.leaderCluster.transport_hosts", System.getProperty("leader.transport_host") - systemProperty "tests.cluster.leaderCluster.security_enabled", System.getProperty("leader.security_enabled") - nonInputProperties.systemProperty('tests.integTestRemote', "true") - systemProperty "build.dir", "${buildDir}" - } + systemProperty "tests.cluster.leaderCluster.http_hosts", System.getProperty("leader.http_host") + systemProperty "tests.cluster.leaderCluster.transport_hosts", System.getProperty("leader.transport_host") + systemProperty "tests.cluster.leaderCluster.security_enabled", System.getProperty("leader.security_enabled") + + nonInputProperties.systemProperty('tests.integTestRemote', "true") + systemProperty "build.dir", "${buildDir}" + } filter { - if (System.getProperty("leader.transport_host") != null) { setExcludePatterns("org.opensearch.replication.bwc.BackwardsCompatibilityIT") - }else{ + } + nonInputProperties.systemProperty('tests.sanitySingleCluster', "integTestSingleCluster") +} + + +task singleClusterSanity (type: RestIntegTestTask) { + doFirst { + + systemProperty "tests.cluster.followCluster.http_hosts", System.getProperty("follower.http_host") + systemProperty "tests.cluster.followCluster.transport_hosts", System.getProperty("follower.transport_host") + systemProperty "tests.cluster.followCluster.security_enabled", System.getProperty("follower.security_enabled") + } + filter { setIncludePatterns("org.opensearch.replication.singleCluster.SingleClusterSanityIT") - } } nonInputProperties.systemProperty('tests.sanitySingleCluster', "integTestSingleCluster") } diff --git a/scripts/integtest.sh b/scripts/integtest.sh index 12b2bf5c..dcd747b8 100755 --- a/scripts/integtest.sh +++ b/scripts/integtest.sh @@ -151,5 +151,5 @@ else then TRANSPORT_PORT="9300" fi - ./gradlew integTestRemote -Dfollower.http_host="$BIND_ADDRESS:$BIND_PORT" -Dfollower.transport_host="$BIND_ADDRESS:$TRANSPORT_PORT" -Dsecurity_enabled=$SECURITY_ENABLED -Duser=$USERNAME -Dpassword=$PASSWORD --console=plain + ./gradlew singleClusterSanity -Dfollower.http_host="$BIND_ADDRESS:$BIND_PORT" -Dfollower.transport_host="$BIND_ADDRESS:$TRANSPORT_PORT" -Dsecurity_enabled=$SECURITY_ENABLED -Duser=$USERNAME -Dpassword=$PASSWORD --console=plain fi \ No newline at end of file From 7e66919973fe231affb75344a766326898fdbe08 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Fri, 9 Sep 2022 14:52:03 +0530 Subject: [PATCH 022/157] Created seperate gradle target for single cluster and multi cluster integtest Signed-off-by: Monu Singh --- build.gradle | 3 --- 1 file changed, 3 deletions(-) diff --git a/build.gradle b/build.gradle index 1be4e2e7..7e9d88cf 100644 --- a/build.gradle +++ b/build.gradle @@ -879,7 +879,6 @@ task "bwcTestSuite"(type: RestIntegTestTask) { task integTestRemote (type: RestIntegTestTask) { doFirst { - systemProperty "tests.cluster.followCluster.http_hosts", System.getProperty("follower.http_host") systemProperty "tests.cluster.followCluster.transport_hosts", System.getProperty("follower.transport_host") systemProperty "tests.cluster.followCluster.security_enabled", System.getProperty("follower.security_enabled") @@ -898,10 +897,8 @@ task integTestRemote (type: RestIntegTestTask) { nonInputProperties.systemProperty('tests.sanitySingleCluster', "integTestSingleCluster") } - task singleClusterSanity (type: RestIntegTestTask) { doFirst { - systemProperty "tests.cluster.followCluster.http_hosts", System.getProperty("follower.http_host") systemProperty "tests.cluster.followCluster.transport_hosts", System.getProperty("follower.transport_host") systemProperty "tests.cluster.followCluster.security_enabled", System.getProperty("follower.security_enabled") From ab3f9d411f98c4beba6c6235e092b7864a448a1d Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Fri, 9 Sep 2022 15:24:01 +0530 Subject: [PATCH 023/157] Created seperate gradle target for single cluster and multi cluster integtest Signed-off-by: Monu Singh --- build.gradle | 2 +- scripts/integtest.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/build.gradle b/build.gradle index 7e9d88cf..280d85f9 100644 --- a/build.gradle +++ b/build.gradle @@ -897,7 +897,7 @@ task integTestRemote (type: RestIntegTestTask) { nonInputProperties.systemProperty('tests.sanitySingleCluster', "integTestSingleCluster") } -task singleClusterSanity (type: RestIntegTestTask) { +task singleClusterSanityTest (type: RestIntegTestTask) { doFirst { systemProperty "tests.cluster.followCluster.http_hosts", System.getProperty("follower.http_host") systemProperty "tests.cluster.followCluster.transport_hosts", System.getProperty("follower.transport_host") diff --git a/scripts/integtest.sh b/scripts/integtest.sh index dcd747b8..005781af 100755 --- a/scripts/integtest.sh +++ b/scripts/integtest.sh @@ -151,5 +151,5 @@ else then TRANSPORT_PORT="9300" fi - ./gradlew singleClusterSanity -Dfollower.http_host="$BIND_ADDRESS:$BIND_PORT" -Dfollower.transport_host="$BIND_ADDRESS:$TRANSPORT_PORT" -Dsecurity_enabled=$SECURITY_ENABLED -Duser=$USERNAME -Dpassword=$PASSWORD --console=plain + ./gradlew singleClusterSanityTest -Dfollower.http_host="$BIND_ADDRESS:$BIND_PORT" -Dfollower.transport_host="$BIND_ADDRESS:$TRANSPORT_PORT" -Dsecurity_enabled=$SECURITY_ENABLED -Duser=$USERNAME -Dpassword=$PASSWORD --console=plain fi \ No newline at end of file From c1b16f4b79dab4455a8ce3857c8a7121a49f7461 Mon Sep 17 00:00:00 2001 From: Chris Moore <107723039+cwillum@users.noreply.github.com> Date: Thu, 22 Sep 2022 00:28:23 -0700 Subject: [PATCH 024/157] fix#921-README-forum-link-CCRepli (#548) Signed-off-by: cwillum Signed-off-by: cwillum --- README.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/README.md b/README.md index a404690c..f98b0259 100644 --- a/README.md +++ b/README.md @@ -157,6 +157,12 @@ For more details on design and architecture, please refer to [RFC](docs/RFC.md) See [CONTRIBUTING](CONTRIBUTING.md) for more information. +## Getting Help + +If you find a bug, or have a feature request, please don't hesitate to open an issue in this repository. + +For more information, see the [project website](https://opensearch.org/) and [technical documentation](https://opensearch.org/docs/latest/replication-plugin/index/). If you need help and are unsure where to open an issue, try the OpenSearch [Forum](https://forum.opensearch.org/c/plugins/cross-cluster-replication/53). + ## License This project is licensed under the Apache-2.0 License. From e1dc5110891f6f6df64f9cfaad7387902a7d6f12 Mon Sep 17 00:00:00 2001 From: Ralph Ursprung <39383228+rursprung@users.noreply.github.com> Date: Mon, 26 Sep 2022 10:41:31 +0200 Subject: [PATCH 025/157] use proper gradle dep. to fetch security plugin (#566) there's no need to manually construct a URL and try to fetch the artefact from there. the plugin is published on maven and we can directly use the gradle/maven dependency to fetch it. in order to more easily identify it, a new configuration type has been added: `opensearchPlugin`. for reference see the discussions on the following issues/PRs: - opensearch-project/cross-cluster-replication#274 - opensearch-project/cross-cluster-replication#455 Signed-off-by: Ralph Ursprung Signed-off-by: Ralph Ursprung --- build.gradle | 26 ++++++++++---------------- 1 file changed, 10 insertions(+), 16 deletions(-) diff --git a/build.gradle b/build.gradle index e56ddd5c..c410d30a 100644 --- a/build.gradle +++ b/build.gradle @@ -56,14 +56,7 @@ buildscript { common_utils_version = System.getProperty("common_utils.version", opensearch_build) kotlin_version = System.getProperty("kotlin.version", "1.6.0") - // For fetching security zip from Maven. - // https://ci.opensearch.org/ci/dbc/distribution-build-opensearch/2.1.0/latest/linux/x64/tar/builds/opensearch/plugins/opensearch-security-2.1.0.0.zip - opensearch_no_snapshot = opensearch_version.replace("-SNAPSHOT","") - security_no_snapshot = opensearch_build.replace("-SNAPSHOT","") - security_plugin_path = "build/dependencies/security" - security_plugin_download_url = 'https://ci.opensearch.org/ci/dbc/distribution-build-opensearch/' + opensearch_no_snapshot + - '/latest/linux/x64/tar/builds/opensearch/plugins/opensearch-security-' + security_no_snapshot + '.zip' - + security_plugin_version = opensearch_build.replace("-SNAPSHOT","") } repositories { @@ -126,6 +119,10 @@ configurations.all { } } +configurations { + opensearchPlugin +} + dependencies { runtimeOnly "org.opensearch:opensearch:${opensearch_version}" implementation "org.jetbrains.kotlin:kotlin-stdlib-jdk8" @@ -143,6 +140,8 @@ dependencies { testImplementation "org.jetbrains.kotlinx:kotlinx-coroutines-test:${kotlin_version}" testImplementation "org.jetbrains.kotlin:kotlin-test:${kotlin_version}" testImplementation "com.nhaarman.mockitokotlin2:mockito-kotlin:2.2.0" + + opensearchPlugin "org.opensearch.plugin:opensearch-security:${security_plugin_version}@zip" } repositories { @@ -218,14 +217,9 @@ def securityPluginFile = new Callable() { return new RegularFile() { @Override File getAsFile() { - if (new File("$project.rootDir/$security_plugin_path").exists()) { - project.delete(files("$project.rootDir/$security_plugin_path")) - } - project.mkdir security_plugin_path - ant.get(src: security_plugin_download_url, - dest: security_plugin_path, - httpusecaches: false) - return fileTree(security_plugin_path).getSingleFile() + return configurations.opensearchPlugin.resolvedConfiguration.resolvedArtifacts + .find { ResolvedArtifact f -> f.name.contains('opensearch-security') } + .file } } } From 39376f28f696548eaf218252d61b9d8cf4189888 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Wed, 28 Sep 2022 10:33:45 +0530 Subject: [PATCH 026/157] Created helper function to check if tests are being run for remote clusters. Signed-off-by: Monu Singh --- build.gradle | 1 - .../replication/MultiClusterRestTestCase.kt | 7 +++++++ .../integ/rest/ResumeReplicationIT.kt | 15 +++----------- .../integ/rest/StartReplicationIT.kt | 20 ++++--------------- 4 files changed, 14 insertions(+), 29 deletions(-) diff --git a/build.gradle b/build.gradle index 280d85f9..23249144 100644 --- a/build.gradle +++ b/build.gradle @@ -894,7 +894,6 @@ task integTestRemote (type: RestIntegTestTask) { filter { setExcludePatterns("org.opensearch.replication.bwc.BackwardsCompatibilityIT") } - nonInputProperties.systemProperty('tests.sanitySingleCluster', "integTestSingleCluster") } task singleClusterSanityTest (type: RestIntegTestTask) { diff --git a/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt b/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt index 888d8af9..ea710e99 100644 --- a/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt +++ b/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt @@ -609,4 +609,11 @@ abstract class MultiClusterRestTestCase : OpenSearchTestCase() { updateSettingsRequest.transientSettings(Collections.singletonMap(ReplicationPlugin.REPLICATION_METADATA_SYNC_INTERVAL.key, "5s")) followerClient.cluster().putSettings(updateSettingsRequest, RequestOptions.DEFAULT) } + +// TODO Find a way to skip tests when tests are run for remote clusters + protected fun checkifIntegTestRemote(): Boolean { + val systemProperties = BootstrapInfo.getSystemProperties() + val integTestRemote = systemProperties.get("tests.integTestRemote") as String? + return integTestRemote.equals("true") + } } diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/ResumeReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/ResumeReplicationIT.kt index d4c95cd2..6b1aa5b8 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/ResumeReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/ResumeReplicationIT.kt @@ -211,10 +211,7 @@ class ResumeReplicationIT: MultiClusterRestTestCase() { fun `test that replication fails to resume when custom analyser is not present in follower`() { - //check if integTest remote = true - val systemProperties = BootstrapInfo.getSystemProperties() - val integTestRemote = systemProperties.get("tests.integTestRemote") as String? - if(integTestRemote.equals("true")){ + if(checkifIntegTestRemote()){ return; } @@ -260,10 +257,7 @@ class ResumeReplicationIT: MultiClusterRestTestCase() { fun `test that replication resumes when custom analyser is present in follower`() { - //check if integTest remote = true - val systemProperties = BootstrapInfo.getSystemProperties() - val integTestRemote = systemProperties.get("tests.integTestRemote") as String? - if(integTestRemote.equals("true")){ + if(checkifIntegTestRemote()){ return; } @@ -317,10 +311,7 @@ class ResumeReplicationIT: MultiClusterRestTestCase() { fun `test that replication resumes when custom analyser is overridden and present in follower`() { - //check if integTest remote = true - val systemProperties = BootstrapInfo.getSystemProperties() - val integTestRemote = systemProperties.get("tests.integTestRemote") as String? - if(integTestRemote.equals("true")){ + if(checkifIntegTestRemote()){ return; } diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt index 49d2934f..7c8e4b80 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt @@ -702,10 +702,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { fun `test that replication fails to start when custom analyser is not present in follower`() { - //check if integTest remote = true - val systemProperties = BootstrapInfo.getSystemProperties() - val integTestRemote = systemProperties.get("tests.integTestRemote") as String? - if(integTestRemote.equals("true")){ + if(checkifIntegTestRemote()){ return; } @@ -743,10 +740,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { fun `test that replication starts successfully when custom analyser is present in follower`() { - //check if integTest remote = true - val systemProperties = BootstrapInfo.getSystemProperties() - val integTestRemote = systemProperties.get("tests.integTestRemote") as String? - if(integTestRemote.equals("true")){ + if(checkifIntegTestRemote()){ return; } @@ -796,10 +790,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { fun `test that replication starts successfully when custom analyser is overridden and present in follower`() { - //check if integTest remote = true - val systemProperties = BootstrapInfo.getSystemProperties() - val integTestRemote = systemProperties.get("tests.integTestRemote") as String? - if(integTestRemote.equals("true")){ + if(checkifIntegTestRemote()){ return; } @@ -968,10 +959,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { fun `test that snapshot on leader does not affect replication during bootstrap`() { - //check if integTest remote = true - val systemProperties = BootstrapInfo.getSystemProperties() - val integTestRemote = systemProperties.get("tests.integTestRemote") as String? - if(integTestRemote.equals("true")){ + if(checkifIntegTestRemote()){ return; } From bf7ac767d24d89cc44b4d11236c210932eb850bd Mon Sep 17 00:00:00 2001 From: Sooraj Sinha Date: Tue, 4 Oct 2022 10:38:43 +0530 Subject: [PATCH 027/157] Upgrade Snakeyml and Jackson Signed-off-by: Sooraj Sinha --- build.gradle | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/build.gradle b/build.gradle index c410d30a..4c5c2e97 100644 --- a/build.gradle +++ b/build.gradle @@ -113,8 +113,8 @@ configurations.all { force 'org.apache.httpcomponents:httpclient-osgi:4.5.13' force 'org.apache.httpcomponents.client5:httpclient5:5.0.3' force 'org.apache.httpcomponents.client5:httpclient5-osgi:5.0.3' - force 'com.fasterxml.jackson.core:jackson-databind:2.12.6' - force 'org.yaml:snakeyaml:1.31' + force 'com.fasterxml.jackson.core:jackson-databind:2.13.4' + force 'org.yaml:snakeyaml:1.32' force 'org.codehaus.plexus:plexus-utils:3.0.24' } } From 8fa868b6616d9fed18b6a88be677d6602f4c27a1 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Tue, 4 Oct 2022 14:53:57 +0530 Subject: [PATCH 028/157] Support to send transport port as well Signed-off-by: Monu Singh --- build.gradle | 4 +-- scripts/integtest.sh | 70 ++++++++------------------------------------ 2 files changed, 14 insertions(+), 60 deletions(-) diff --git a/build.gradle b/build.gradle index 23249144..2520446c 100644 --- a/build.gradle +++ b/build.gradle @@ -881,11 +881,11 @@ task integTestRemote (type: RestIntegTestTask) { doFirst { systemProperty "tests.cluster.followCluster.http_hosts", System.getProperty("follower.http_host") systemProperty "tests.cluster.followCluster.transport_hosts", System.getProperty("follower.transport_host") - systemProperty "tests.cluster.followCluster.security_enabled", System.getProperty("follower.security_enabled") + systemProperty "tests.cluster.followCluster.security_enabled", System.getProperty("security_enabled") systemProperty "tests.cluster.leaderCluster.http_hosts", System.getProperty("leader.http_host") systemProperty "tests.cluster.leaderCluster.transport_hosts", System.getProperty("leader.transport_host") - systemProperty "tests.cluster.leaderCluster.security_enabled", System.getProperty("leader.security_enabled") + systemProperty "tests.cluster.leaderCluster.security_enabled", System.getProperty("security_enabled") nonInputProperties.systemProperty('tests.integTestRemote', "true") systemProperty "build.dir", "${buildDir}" diff --git a/scripts/integtest.sh b/scripts/integtest.sh index 005781af..edec958f 100755 --- a/scripts/integtest.sh +++ b/scripts/integtest.sh @@ -21,16 +21,11 @@ function usage() { echo -e "-t TRANSPORT_PORT\t, defaults to 9300, can be changed to any port for the cluster location." echo "--------------------------------------------------------------------------" echo "Multi cluster test:" - echo -e "-m Leader BIND_ADDRESS\t\t, defaults to localhost | 127.0.0.1, can be changed to any IP or domain name for the cluster location." - echo -e "-n Leader BIND_PORT\t\t, defaults to 9200, can be changed to any port for the cluster location." - echo -e "-o Leader TRANSPORT_PORT\t, defaults to 9300, can be changed to any port for the cluster location." - echo -e "-x Follower BIND_ADDRESS\t, defaults to localhost | 127.0.0.1, can be changed to any IP or domain name for the cluster location." - echo -e "-y Follower BIND_PORT\t\t, defaults to 9200, can be changed to any port for the cluster location." - echo -e "-z Follower TRANSPORT_PORT\t, defaults to 9300, can be changed to any port for the cluster location." + echo -e "-e Comma seperated endpoint:port, ex: localhost:9200,localhost:9201... ." echo "--------------------------------------------------------------------------" } -while getopts ":h:b:p:t:m:n:o:x:y:z:s:c:" arg; do +while getopts ":h:b:p:t:e:s:c:" arg; do case $arg in h) usage @@ -45,23 +40,8 @@ while getopts ":h:b:p:t:m:n:o:x:y:z:s:c:" arg; do t) TRANSPORT_PORT=$OPTARG ;; - m) - LBIND_ADDRESS=$OPTARG - ;; - n) - LBIND_PORT=$OPTARG - ;; - o) - LTRANSPORT_PORT=$OPTARG - ;; - x) - FBIND_ADDRESS=$OPTARG - ;; - y) - FBIND_PORT=$OPTARG - ;; - z) - FTRANSPORT_PORT=$OPTARG + e) + ENDPOINT_LIST=$OPTARG ;; s) SECURITY_ENABLED=$OPTARG @@ -102,48 +82,22 @@ PASSWORD=`echo $CREDENTIAL | awk -F ':' '{print $2}'` if [ -z "$BIND_ADDRESS" ] || [ -z "$BIND_PORT" ] then #Proceeding with multi cluster test - if [ -z "$LBIND_ADDRESS" ] + if [ -z "$ENDPOINT_LIST" ] then - echo "requires an argument -m " + echo "requires an argument -e " usage exit 1 fi - if [ -z "$LBIND_PORT" ] - then - echo "requires an argument -n " - usage - exit 1 - fi - - if [ -z "$LTRANSPORT_PORT" ] - then - echo "requires an argument -o " - usage - exit 1 - fi + leader=$(echo $ENDPOINT_LIST | cut -d ',' -f1 | cut -d ':' -f1,2 ) + follower=$(echo $ENDPOINT_LIST | cut -d ',' -f1 | cut -d ':' -f1,2 ) - if [ -z "$FBIND_ADDRESS" ] - then - echo "requires an argument -fb " - usage - exit 1 - fi + FTRANSPORT_PORT=$(echo $ENDPOINT_LIST | cut -d ',' -f1 | cut -d ':' -f3 ) + LTRANSPORT_PORT=$(echo $ENDPOINT_LIST | cut -d ',' -f2 | cut -d ':' -f3 ) + echo "./gradlew integTestRemote -Dleader.http_host=\"$leader\" -Dfollower.http_host=\"$follower\" -Dfollower.transport_host=\"$FTRANSPORT_PORT\" -Dleader.transport_host=\"$LTRANSPORT_PORT\" -Dsecurity_enabled=\"$SECURITY_ENABLED\" -Duser=\"$USERNAME\" -Dpassword=\"$PASSWORD\" --console=plain " + eval "./gradlew integTestRemote -Dleader.http_host=\"$leader\" -Dfollower.http_host=\"$follower\" -Dfollower.transport_host=\"$FTRANSPORT_PORT\" -Dleader.transport_host=\"$LTRANSPORT_PORT\" -Dsecurity_enabled=\"$SECURITY_ENABLED\" -Duser=\"$USERNAME\" -Dpassword=\"$PASSWORD\" --console=plain " - if [ -z "$FBIND_PORT" ] - then - echo "requires an argument -fp " - usage - exit 1 - fi - if [ -z "$FTRANSPORT_PORT" ] - then - echo "requires an argument -ft " - usage - exit 1 - fi - eval "./gradlew integTestRemote -Dleader.http_host=\"$LBIND_ADDRESS:$LBIND_PORT\" -Dleader.transport_host=\"$LBIND_ADDRESS:$LTRANSPORT_PORT\" -Dfollower.http_host=\"$FBIND_ADDRESS:$FBIND_PORT\" -Dfollower.transport_host=\"$FBIND_ADDRESS:$FTRANSPORT_PORT\" -Dsecurity_enabled=\"$SECURITY_ENABLED\" -Duser=\"$USERNAME\" -Dpassword=\"$PASSWORD\" --console=plain " else # Single cluster From 3bdfd1da079a57d745c46690699b2d00f9bc5d22 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Thu, 6 Oct 2022 08:01:37 +0530 Subject: [PATCH 029/157] Endpoint correction Endpoint correction Signed-off-by: Monu Singh --- scripts/integtest.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/scripts/integtest.sh b/scripts/integtest.sh index edec958f..5bbe0300 100755 --- a/scripts/integtest.sh +++ b/scripts/integtest.sh @@ -90,7 +90,7 @@ then fi leader=$(echo $ENDPOINT_LIST | cut -d ',' -f1 | cut -d ':' -f1,2 ) - follower=$(echo $ENDPOINT_LIST | cut -d ',' -f1 | cut -d ':' -f1,2 ) + follower=$(echo $ENDPOINT_LIST | cut -d ',' -f2 | cut -d ':' -f1,2 ) FTRANSPORT_PORT=$(echo $ENDPOINT_LIST | cut -d ',' -f1 | cut -d ':' -f3 ) LTRANSPORT_PORT=$(echo $ENDPOINT_LIST | cut -d ',' -f2 | cut -d ':' -f3 ) @@ -106,4 +106,4 @@ else TRANSPORT_PORT="9300" fi ./gradlew singleClusterSanityTest -Dfollower.http_host="$BIND_ADDRESS:$BIND_PORT" -Dfollower.transport_host="$BIND_ADDRESS:$TRANSPORT_PORT" -Dsecurity_enabled=$SECURITY_ENABLED -Duser=$USERNAME -Dpassword=$PASSWORD --console=plain -fi \ No newline at end of file +fi From f8b500c08c6c473e2f640f4cf139ed7c07f8cf53 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Fri, 7 Oct 2022 17:47:31 +0530 Subject: [PATCH 030/157] Added support to pass endpoints as dict Signed-off-by: Monu Singh --- .DS_Store | Bin 0 -> 8196 bytes scripts/.DS_Store | Bin 0 -> 6148 bytes scripts/integtest.sh | 21 ++++++++++++--------- 3 files changed, 12 insertions(+), 9 deletions(-) create mode 100644 .DS_Store create mode 100644 scripts/.DS_Store diff --git a/.DS_Store b/.DS_Store new file mode 100644 index 0000000000000000000000000000000000000000..96c2ac8285d47f36e1cbefd8a93881cb3c2028b0 GIT binary patch literal 8196 zcmeHMQBTuQ6h4HThR&v)*<-#Kk}3lWLhpu0k}Ktu)(#&QYOgu>VP z7%Nk%=LS%KPeI4_0vl)LFN*;EYuD~sI7-Er8$XejJnneXa&X<;Jtf}Y;wup@KXEt$GEcmM;1xD3abI^ zJnGUi?NXoGXnEvhct0|{YqU;VRHp{DfKO2K`6haH&}zch0d>N1 z%qDP?o=#+c$*}X7XFweaGo$>@*iE0ks;S+KZ6Rq(!?b5&?Y<|pE2XV5Ya8lC85S#- zW<7%^R;mWYw&?l9)v{Je);#S|gufmwVa*;Q=8%u`Wr>HtPZUCrOuPlFmh@Ag7tnYe zaoZ$|8tJ(T)-Gv1OD{23j!9Et1F^Cu%~)!g%fPL)kfLTSikeE?P1tc-{F4RJ$`3r5i>w#qk;j+Ry)tS*`>_tz@Ni{{b8yOl-r-s;-%anZPb`_7~7*1mV( z2VaptC_=r?X(m2Dt-qm_QQ^?)`(EJloye>bXC?A;M?Mpp^kz~_W&V6l!wo|;xbEAX!h zh-s>lW0;>?iWPj50)qt)cgQCQh;<)3Y7jTzAJwQ$%NMagG7l_PD>nO^)=4+n!!Z|VM%m Date: Wed, 12 Oct 2022 06:24:42 +0530 Subject: [PATCH 031/157] added groupId to pluginzip publication (#568) Signed-off-by: sricharanvuppu Signed-off-by: sricharanvuppu --- build.gradle | 1 + 1 file changed, 1 insertion(+) diff --git a/build.gradle b/build.gradle index 4c5c2e97..757e6a62 100644 --- a/build.gradle +++ b/build.gradle @@ -889,6 +889,7 @@ publishing { pom { name = opensearchplugin.name description = opensearchplugin.description + group = "org.opensearch.plugin" licenses { license { name = "The Apache License, Version 2.0" From 719cf938583e224a237f999106cf6026b736fbb1 Mon Sep 17 00:00:00 2001 From: prudhvigodithi Date: Wed, 12 Oct 2022 13:32:53 -0700 Subject: [PATCH 032/157] add group = org.opensearch.plugin Signed-off-by: prudhvigodithi --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 757e6a62..f18e115a 100644 --- a/build.gradle +++ b/build.gradle @@ -889,7 +889,7 @@ publishing { pom { name = opensearchplugin.name description = opensearchplugin.description - group = "org.opensearch.plugin" + groupId = "org.opensearch.plugin" licenses { license { name = "The Apache License, Version 2.0" From 0f844288fcf1ab4500287dc8ea79e5c86069e2b4 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Thu, 13 Oct 2022 15:17:12 +0530 Subject: [PATCH 033/157] Update integtest.sh Transport correction Signed-off-by: Monu Singh --- scripts/integtest.sh | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/scripts/integtest.sh b/scripts/integtest.sh index 51029022..8b02027d 100755 --- a/scripts/integtest.sh +++ b/scripts/integtest.sh @@ -97,9 +97,9 @@ then leader=$(echo $data | cut -d ',' -f1 | cut -d ':' -f1,2 ) follower=$(echo $data | cut -d ',' -f2 | cut -d ':' -f1,2 ) - - FTRANSPORT_PORT=$(echo $data | cut -d ',' -f1 | cut -d ':' -f3 ) - LTRANSPORT_PORT=$(echo $data | cut -d ',' -f2 | cut -d ':' -f3 ) + + FTRANSPORT_PORT=$(echo $data | cut -d ',' -f1 | cut -d ':' -f1,3 ) + LTRANSPORT_PORT=$(echo $data | cut -d ',' -f2 | cut -d ':' -f1,3 ) eval "./gradlew integTestRemote -Dleader.http_host=\"$leader\" -Dfollower.http_host=\"$follower\" -Dfollower.transport_host=\"$FTRANSPORT_PORT\" -Dleader.transport_host=\"$LTRANSPORT_PORT\" -Dsecurity_enabled=\"$SECURITY_ENABLED\" -Duser=\"$USERNAME\" -Dpassword=\"$PASSWORD\" --console=plain " else @@ -109,4 +109,4 @@ else TRANSPORT_PORT="9300" fi ./gradlew singleClusterSanityTest -Dfollower.http_host="$BIND_ADDRESS:$BIND_PORT" -Dfollower.transport_host="$BIND_ADDRESS:$TRANSPORT_PORT" -Dsecurity_enabled=$SECURITY_ENABLED -Duser=$USERNAME -Dpassword=$PASSWORD --console=plain -fi \ No newline at end of file +fi From cbf9689f4ec2d356184cfc700392589cd69afcd7 Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Tue, 13 Sep 2022 20:27:14 +0530 Subject: [PATCH 034/157] Bump the main to 3.0 Signed-off-by: Ankit Kala --- build.gradle | 2 +- .../replication/repository/RemoteClusterRepository.kt | 4 ---- .../kotlin/org/opensearch/replication/util/ValidationUtil.kt | 2 +- 3 files changed, 2 insertions(+), 6 deletions(-) diff --git a/build.gradle b/build.gradle index 757e6a62..d9e3e7c7 100644 --- a/build.gradle +++ b/build.gradle @@ -36,7 +36,7 @@ import org.opensearch.gradle.test.RestIntegTestTask buildscript { ext { isSnapshot = "true" == System.getProperty("build.snapshot", "true") - opensearch_version = System.getProperty("opensearch.version", "2.2.0-SNAPSHOT") + opensearch_version = System.getProperty("opensearch.version", "3.0.0-SNAPSHOT") buildVersionQualifier = System.getProperty("build.version_qualifier", "") // e.g. 2.0.0-rc1-SNAPSHOT -> 2.0.0.0-rc1-SNAPSHOT version_tokens = opensearch_version.tokenize('-') diff --git a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt index 6cfbff0d..4dc9d765 100644 --- a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt +++ b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt @@ -118,10 +118,6 @@ class RemoteClusterRepository(private val repositoryMetadata: RepositoryMetadata throw UnsupportedOperationException("Operation not permitted") } - override fun initializeSnapshot(snapshotId: SnapshotId, indices: MutableList, metadata: Metadata) { - throw UnsupportedOperationException("Operation not permitted") - } - override fun startVerification(): String { throw UnsupportedOperationException("Operation not permitted") } diff --git a/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt b/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt index 515c96ec..5f9a6aee 100644 --- a/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt +++ b/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt @@ -56,7 +56,7 @@ object ValidationUtil { val analyserSettings = leaderSettings.filter { k: String? -> k!!.matches(Regex("index.analysis.*path")) } for (analyserSetting in analyserSettings.keySet()) { val settingValue = if (overriddenSettings.hasValue(analyserSetting)) overriddenSettings.get(analyserSetting) else analyserSettings.get(analyserSetting) - val path: Path = environment.configFile().resolve(settingValue) + val path: Path = environment.configDir().resolve(settingValue) if (!Files.exists(path)) { val message = "IOException while reading ${analyserSetting}: ${path.toString()}" log.error(message) From b4279edc916360f9c75fd5435688a0d727239a42 Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Tue, 13 Sep 2022 20:30:05 +0530 Subject: [PATCH 035/157] Use the new method to fetch the changes from pluggable translog Signed-off-by: Ankit Kala --- .../replication/seqno/RemoteClusterTranslogService.kt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterTranslogService.kt b/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterTranslogService.kt index 21edfea3..f7514daf 100644 --- a/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterTranslogService.kt +++ b/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterTranslogService.kt @@ -38,7 +38,7 @@ class RemoteClusterTranslogService : AbstractLifecycleComponent(){ public fun getHistoryOfOperations(indexShard: IndexShard, startSeqNo: Long, toSeqNo: Long): List { log.trace("Fetching translog snapshot for $indexShard - from $startSeqNo to $toSeqNo") // Ref issue: https://github.com/opensearch-project/OpenSearch/issues/2482 - val snapshot = indexShard.getHistoryOperationsFromTranslogFile(SOURCE_NAME, startSeqNo, toSeqNo) + val snapshot = indexShard.getHistoryOperationsFromTranslog(startSeqNo, toSeqNo) // Total ops to be fetched (both toSeqNo and startSeqNo are inclusive) val opsSize = toSeqNo - startSeqNo + 1 From 4bdf13e08e9da54e13771d9fb13f9761534e9076 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Fri, 14 Oct 2022 14:40:09 +0530 Subject: [PATCH 036/157] update build.gradle Signed-off-by: Monu Singh --- build.gradle | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/build.gradle b/build.gradle index 351e419b..c5219d1b 100644 --- a/build.gradle +++ b/build.gradle @@ -886,7 +886,7 @@ task integTestRemote (type: RestIntegTestTask) { } filter { - setExcludePatterns("org.opensearch.replication.bwc.BackwardsCompatibilityIT") + setExcludePatterns("org.opensearch.replication.bwc.BackwardsCompatibilityIT","org.opensearch.replication.singleCluster.SingleClusterSanityIT") } } @@ -894,7 +894,7 @@ task singleClusterSanityTest (type: RestIntegTestTask) { doFirst { systemProperty "tests.cluster.followCluster.http_hosts", System.getProperty("follower.http_host") systemProperty "tests.cluster.followCluster.transport_hosts", System.getProperty("follower.transport_host") - systemProperty "tests.cluster.followCluster.security_enabled", System.getProperty("follower.security_enabled") + systemProperty "tests.cluster.followCluster.security_enabled", System.getProperty("security_enabled") } filter { setIncludePatterns("org.opensearch.replication.singleCluster.SingleClusterSanityIT") From 2624c42808d9b2983607802804ebbc90d474d130 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Mon, 17 Oct 2022 10:30:28 +0530 Subject: [PATCH 037/157] Update integtest.sh Signed-off-by: Monu Singh --- scripts/integtest.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/scripts/integtest.sh b/scripts/integtest.sh index 8b02027d..ece01bc0 100755 --- a/scripts/integtest.sh +++ b/scripts/integtest.sh @@ -98,8 +98,8 @@ then leader=$(echo $data | cut -d ',' -f1 | cut -d ':' -f1,2 ) follower=$(echo $data | cut -d ',' -f2 | cut -d ':' -f1,2 ) - FTRANSPORT_PORT=$(echo $data | cut -d ',' -f1 | cut -d ':' -f1,3 ) - LTRANSPORT_PORT=$(echo $data | cut -d ',' -f2 | cut -d ':' -f1,3 ) + LTRANSPORT_PORT=$(echo $data | cut -d ',' -f1 | cut -d ':' -f1,3 ) + FTRANSPORT_PORT=$(echo $data | cut -d ',' -f2 | cut -d ':' -f1,3 ) eval "./gradlew integTestRemote -Dleader.http_host=\"$leader\" -Dfollower.http_host=\"$follower\" -Dfollower.transport_host=\"$FTRANSPORT_PORT\" -Dleader.transport_host=\"$LTRANSPORT_PORT\" -Dsecurity_enabled=\"$SECURITY_ENABLED\" -Duser=\"$USERNAME\" -Dpassword=\"$PASSWORD\" --console=plain " else From d02a6a8c4288fb15c1f59d7d3d21c480e3714a71 Mon Sep 17 00:00:00 2001 From: Mohit Kumar <113413713+mohitamg@users.noreply.github.com> Date: Wed, 19 Oct 2022 15:59:39 +0530 Subject: [PATCH 038/157] =?UTF-8?q?Added=20configurable=20parameter=20to?= =?UTF-8?q?=20use=2050%=20of=20heap=20memory=20for=20CCR=20perfor=E2=80=A6?= =?UTF-8?q?=20(#594)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Added configurable parameter to use 50% of heap memory for CCR performance testing Signed-off-by: Mohit Kumar * Hardcoded the value of use_50_percent_heap and removed it from test_suite.yaml file Signed-off-by: Mohit Kumar Signed-off-by: Mohit Kumar --- perf_workflow/requirements.txt | 1 + perf_workflow/run_perf_suite/ccr_perf_test.py | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/perf_workflow/requirements.txt b/perf_workflow/requirements.txt index a0e09e61..616ed43e 100644 --- a/perf_workflow/requirements.txt +++ b/perf_workflow/requirements.txt @@ -6,6 +6,7 @@ pipenv requests retry ndg-httpsclient +psutil pyopenssl pyasn1 aws_cdk.core~=1.143.0 diff --git a/perf_workflow/run_perf_suite/ccr_perf_test.py b/perf_workflow/run_perf_suite/ccr_perf_test.py index 40185c47..2d8afb51 100644 --- a/perf_workflow/run_perf_suite/ccr_perf_test.py +++ b/perf_workflow/run_perf_suite/ccr_perf_test.py @@ -52,7 +52,7 @@ def __init__( self.execution_id = None self.test_succeeded = False self.failure_reason = None - + self.use_50_percent_heap = "enable" def get_infra_repo_url(self): if "GITHUB_TOKEN" in os.environ: return "https://${GITHUB_TOKEN}@github.com/opensearch-project/opensearch-infra.git" @@ -79,6 +79,7 @@ def run(self): self.security, self.test_config.get("DataNodes", 1), self.test_config.get("MasterNodes", 0), + use_50_percent_heap = self.use_50_percent_heap ) # TODO: Add support for configurable instance type. Default is m5.2xlarge with self.create_cluster( From 0ec98b11c18ae06ab4454cbeb5e9f270891ac297 Mon Sep 17 00:00:00 2001 From: Mohit Kumar <113413713+mohitamg@users.noreply.github.com> Date: Thu, 20 Oct 2022 17:01:30 +0530 Subject: [PATCH 039/157] Updated jackson databind version to 2.13.4.2 (#596) Signed-off-by: Mohit Kumar Signed-off-by: Mohit Kumar --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index f18e115a..084d6b51 100644 --- a/build.gradle +++ b/build.gradle @@ -113,7 +113,7 @@ configurations.all { force 'org.apache.httpcomponents:httpclient-osgi:4.5.13' force 'org.apache.httpcomponents.client5:httpclient5:5.0.3' force 'org.apache.httpcomponents.client5:httpclient5-osgi:5.0.3' - force 'com.fasterxml.jackson.core:jackson-databind:2.13.4' + force 'com.fasterxml.jackson.core:jackson-databind:2.13.4.2' force 'org.yaml:snakeyaml:1.32' force 'org.codehaus.plexus:plexus-utils:3.0.24' } From d12271a2909f32cf766d2405f7b1d3b8b1746c44 Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Wed, 26 Oct 2022 12:29:36 +0530 Subject: [PATCH 040/157] Add windows & mac build (#591) Signed-off-by: Ankit Kala Signed-off-by: Ankit Kala --- .github/workflows/build-and-test.yml | 51 ++++++++++++++++++++++++++++ .github/workflows/build.yml | 32 ++++++----------- 2 files changed, 61 insertions(+), 22 deletions(-) create mode 100644 .github/workflows/build-and-test.yml diff --git a/.github/workflows/build-and-test.yml b/.github/workflows/build-and-test.yml new file mode 100644 index 00000000..e8a6ace5 --- /dev/null +++ b/.github/workflows/build-and-test.yml @@ -0,0 +1,51 @@ +name: CCR Test Workflow +# This workflow is triggered on pull requests to main branch +on: + pull_request: + branches: + - '*' + push: + branches: + - '*' + +# We build for all combinations but run tests only on one combination (linux & latest java) +jobs: + build: + continue-on-error: true + strategy: + matrix: + java: + - 11 + - 17 + # Job name + name: Run integration tests on linux with Java ${{ matrix.java }} + runs-on: ubuntu-latest + steps: + # This step uses the setup-java Github action: https://github.com/actions/setup-java + - name: Set Up JDK ${{ matrix.java }} + uses: actions/setup-java@v1 + with: + java-version: ${{ matrix.java }} + # This step uses the checkout Github action: https://github.com/actions/checkout + - name: Checkout Branch + uses: actions/checkout@v2 + - name: Build and run Replication tests + run: | + ./gradlew clean release -D"build.snapshot=true" + - name: Upload failed logs + uses: actions/upload-artifact@v2 + if: failure() + with: + name: logs + path: | + build/testclusters/integTest-*/logs/* + build/testclusters/leaderCluster-*/logs/* + build/testclusters/followCluster-*/logs/* + - name: Create Artifact Path + run: | + mkdir -p cross-cluster-replication-artifacts + cp ./build/distributions/*.zip cross-cluster-replication-artifacts + - name: Uploads coverage + with: + fetch-depth: 2 + uses: codecov/codecov-action@v1.2.1 diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index c3f599a0..1faf6511 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -1,4 +1,4 @@ -name: Test and Build Workflow +name: Build Replication plugin # This workflow is triggered on pull requests to main branch on: pull_request: @@ -8,16 +8,21 @@ on: branches: - '*' +# We build for other platforms except linux which is already covered in build-and-test. +# Also, We're not running tests here as those are already covered with linux build. jobs: build: + continue-on-error: true strategy: matrix: java: - - 11 - 17 + os: + - windows-latest + - macos-latest # Job name - name: Build Replication plugin - runs-on: ubuntu-latest + name: Java ${{ matrix.java }} On ${{ matrix.os }} + runs-on: ${{ matrix.os }} steps: # This step uses the setup-java Github action: https://github.com/actions/setup-java - name: Set Up JDK ${{ matrix.java }} @@ -29,21 +34,4 @@ jobs: uses: actions/checkout@v2 - name: Build and run Replication tests run: | - ./gradlew clean release -Dbuild.snapshot=true - - name: Upload failed logs - uses: actions/upload-artifact@v2 - if: failure() - with: - name: logs - path: | - build/testclusters/integTest-*/logs/* - build/testclusters/leaderCluster-*/logs/* - build/testclusters/followCluster-*/logs/* - - name: Create Artifact Path - run: | - mkdir -p cross-cluster-replication-artifacts - cp ./build/distributions/*.zip cross-cluster-replication-artifacts - - name: Uploads coverage - with: - fetch-depth: 2 - uses: codecov/codecov-action@v1.2.1 + ./gradlew clean release -D"build.snapshot=true" -x test -x IntegTest \ No newline at end of file From 67a7073e77d62e1d4b5cb75fc44ff00d06b2a6cc Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Wed, 26 Oct 2022 17:09:17 +0530 Subject: [PATCH 041/157] Include default index settings during leader setting validation (#601) Signed-off-by: Ankit Kala Signed-off-by: Ankit Kala --- .../index/TransportReplicateIndexAction.kt | 21 +++++++++++++++---- 1 file changed, 17 insertions(+), 4 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/action/index/TransportReplicateIndexAction.kt b/src/main/kotlin/org/opensearch/replication/action/index/TransportReplicateIndexAction.kt index 563a9996..80b0e30b 100644 --- a/src/main/kotlin/org/opensearch/replication/action/index/TransportReplicateIndexAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/index/TransportReplicateIndexAction.kt @@ -93,7 +93,9 @@ class TransportReplicateIndexAction @Inject constructor(transportService: Transp !leaderSettings.get(ReplicationPlugin.REPLICATED_INDEX_SETTING.key).isNullOrBlank()) { throw IllegalArgumentException("Cannot Replicate a Replicated Index ${request.leaderIndex}") } - if (!leaderSettings.getAsBoolean(IndexSettings.INDEX_SOFT_DELETES_SETTING.key, true)) { + + // Soft deletes should be enabled for replication to work. + if (!leaderSettings.getAsBoolean(IndexSettings.INDEX_SOFT_DELETES_SETTING.key, false)) { throw IllegalArgumentException("Cannot Replicate an index where the setting ${IndexSettings.INDEX_SOFT_DELETES_SETTING.key} is disabled") } @@ -120,6 +122,7 @@ class TransportReplicateIndexAction @Inject constructor(transportService: Transp } } + private suspend fun getLeaderClusterState(leaderAlias: String, leaderIndex: String): ClusterState { val remoteClusterClient = client.getRemoteClusterClient(leaderAlias) val clusterStateRequest = remoteClusterClient.admin().cluster().prepareState() @@ -136,8 +139,18 @@ class TransportReplicateIndexAction @Inject constructor(transportService: Transp private suspend fun getLeaderIndexSettings(leaderAlias: String, leaderIndex: String): Settings { val remoteClient = client.getRemoteClusterClient(leaderAlias) val getSettingsRequest = GetSettingsRequest().includeDefaults(true).indices(leaderIndex) - val settingsResponse = remoteClient.suspending(remoteClient.admin().indices()::getSettings, - injectSecurityContext = true)(getSettingsRequest) - return settingsResponse.indexToSettings.get(leaderIndex) ?: throw IndexNotFoundException("${leaderAlias}:${leaderIndex}") + val settingsResponse = remoteClient.suspending( + remoteClient.admin().indices()::getSettings, + injectSecurityContext = true + )(getSettingsRequest) + + val leaderSettings = settingsResponse.indexToSettings.get(leaderIndex) + ?: throw IndexNotFoundException("${leaderAlias}:${leaderIndex}") + val leaderDefaultSettings = settingsResponse.indexToDefaultSettings.get(leaderIndex) + ?: throw IndexNotFoundException("${leaderAlias}:${leaderIndex}") + + // Since we want user configured as well as default settings, we combine both by putting default settings + // and then the explicitly set ones to override the default settings. + return Settings.builder().put(leaderDefaultSettings).put(leaderSettings).build() } } From e59cafcad938afe960f0c059d37eff77b1fca7ec Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Thu, 27 Oct 2022 17:09:00 +0530 Subject: [PATCH 042/157] Added check to skip a test when run by integTestRemote Signed-off-by: Monu Singh --- .../opensearch/replication/integ/rest/StopReplicationIT.kt | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/StopReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/StopReplicationIT.kt index 09b797ae..7592e93f 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/StopReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/StopReplicationIT.kt @@ -244,6 +244,11 @@ class StopReplicationIT: MultiClusterRestTestCase() { } fun `test stop replication with stale replication settings at leader cluster`() { + + if(checkifIntegTestRemote()){ + return; + } + val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) createConnectionBetweenClusters(FOLLOWER, LEADER, "source") From 16039fb6635fdec516286f521e61707f573932cc Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Thu, 27 Oct 2022 19:46:57 +0530 Subject: [PATCH 043/157] Remove the reference to the deprecated BaseNodeRequest class Signed-off-by: Ankit Kala --- build.gradle | 1 + .../opensearch/replication/action/stats/NodeStatsRequest.kt | 6 +++--- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/build.gradle b/build.gradle index d9e3e7c7..1014d31e 100644 --- a/build.gradle +++ b/build.gradle @@ -116,6 +116,7 @@ configurations.all { force 'com.fasterxml.jackson.core:jackson-databind:2.13.4' force 'org.yaml:snakeyaml:1.32' force 'org.codehaus.plexus:plexus-utils:3.0.24' + force 'net.java.dev.jna:jna:5.12.1' } } diff --git a/src/main/kotlin/org/opensearch/replication/action/stats/NodeStatsRequest.kt b/src/main/kotlin/org/opensearch/replication/action/stats/NodeStatsRequest.kt index 19e180da..89c03f12 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stats/NodeStatsRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stats/NodeStatsRequest.kt @@ -11,16 +11,16 @@ package org.opensearch.replication.action.stats -import org.opensearch.action.support.nodes.BaseNodeRequest import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.transport.TransportRequest import java.io.IOException -class NodeStatsRequest : BaseNodeRequest { +class NodeStatsRequest : TransportRequest { constructor(inp :StreamInput) : super(inp) - constructor() + constructor() : super() @Throws(IOException::class) override fun writeTo(out: StreamOutput) { From 603e7e4c3af0022674ace88a2b0ae4c02a5ac14a Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Fri, 28 Oct 2022 18:29:51 +0530 Subject: [PATCH 044/157] Upgrade to to Apache HttpClient / Core 5.x (#4459) Signed-off-by: Ankit Kala --- build.gradle | 9 ++-- .../replication/MultiClusterRestTestCase.kt | 54 ++++++++++--------- .../replication/MultiClusterSetupIT.kt | 2 +- .../replication/ReplicationHelpers.kt | 2 +- .../integ/rest/ClusterRerouteFollowerIT.kt | 5 -- .../integ/rest/ClusterRerouteLeaderIT.kt | 4 +- .../integ/rest/PauseReplicationIT.kt | 3 +- .../replication/integ/rest/SecurityBase.kt | 26 +++++---- .../integ/rest/SecurityCustomRolesIT.kt | 9 ++-- .../integ/rest/SecurityCustomRolesLeaderIT.kt | 10 ++-- .../integ/rest/StartReplicationIT.kt | 12 ++--- .../integ/rest/StopReplicationIT.kt | 2 +- .../integ/rest/UpdateAutoFollowPatternIT.kt | 9 ++-- 13 files changed, 70 insertions(+), 77 deletions(-) diff --git a/build.gradle b/build.gradle index 1014d31e..bef440d9 100644 --- a/build.gradle +++ b/build.gradle @@ -11,7 +11,6 @@ import javax.management.ObjectName -import javax.management.remote.JMXConnector import javax.management.remote.JMXConnectorFactory import javax.management.remote.JMXServiceURL import javax.management.MBeanServerInvocationHandler @@ -97,6 +96,7 @@ apply plugin: 'org.jetbrains.kotlin.jvm' apply plugin: 'org.jetbrains.kotlin.plugin.allopen' apply plugin: 'opensearch.pluginzip' + forbiddenApisTest.ignoreFailures = true configurations.all { @@ -109,10 +109,6 @@ configurations.all { force 'com.google.guava:guava:30.1.1-jre' force 'com.puppycrawl.tools:checkstyle:8.29' force 'commons-codec:commons-codec:1.13' - force 'org.apache.httpcomponents:httpclient:4.5.13' - force 'org.apache.httpcomponents:httpclient-osgi:4.5.13' - force 'org.apache.httpcomponents.client5:httpclient5:5.0.3' - force 'org.apache.httpcomponents.client5:httpclient5-osgi:5.0.3' force 'com.fasterxml.jackson.core:jackson-databind:2.13.4' force 'org.yaml:snakeyaml:1.32' force 'org.codehaus.plexus:plexus-utils:3.0.24' @@ -134,6 +130,9 @@ dependencies { implementation "com.github.seancfoley:ipaddress:5.3.3" implementation "org.jetbrains.kotlinx:kotlinx-coroutines-core:${kotlin_version}" implementation "org.opensearch:common-utils:${common_utils_version}" + implementation "org.apache.httpcomponents.client5:httpclient5:5.1.3" + implementation "org.apache.httpcomponents.core5:httpcore5:5.1.4" + implementation "org.apache.httpcomponents.core5:httpcore5-h2:5.1.4" testImplementation "org.opensearch.test:framework:${opensearch_version}" testImplementation "org.assertj:assertj-core:3.17.2" diff --git a/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt b/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt index 888d8af9..2c4aba1f 100644 --- a/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt +++ b/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt @@ -15,17 +15,19 @@ import org.opensearch.replication.MultiClusterAnnotations.ClusterConfiguration import org.opensearch.replication.MultiClusterAnnotations.ClusterConfigurations import org.opensearch.replication.MultiClusterAnnotations.getAnnotationsFromClass import org.opensearch.replication.integ.rest.FOLLOWER -import org.apache.http.Header -import org.apache.http.HttpHost -import org.apache.http.HttpStatus -import org.apache.http.client.config.RequestConfig -import org.apache.http.entity.ContentType -import org.apache.http.impl.nio.client.HttpAsyncClientBuilder -import org.apache.http.message.BasicHeader -import org.apache.http.nio.conn.ssl.SSLIOSessionStrategy -import org.apache.http.nio.entity.NStringEntity -import org.apache.http.ssl.SSLContexts -import org.apache.http.util.EntityUtils +import org.apache.hc.core5.http.Header +import org.apache.hc.core5.http.HttpHost +import org.apache.hc.core5.http.HttpStatus +import org.apache.hc.client5.http.config.RequestConfig +import org.apache.hc.core5.http.ContentType +import org.apache.hc.client5.http.impl.async.HttpAsyncClientBuilder +import org.apache.hc.client5.http.impl.nio.PoolingAsyncClientConnectionManagerBuilder +import org.apache.hc.client5.http.ssl.ClientTlsStrategyBuilder +import org.apache.hc.core5.http.message.BasicHeader +import org.apache.hc.core5.http.io.entity.StringEntity +import org.apache.hc.core5.ssl.SSLContexts +import org.apache.hc.core5.http.io.entity.EntityUtils +import org.apache.hc.core5.util.Timeout import org.apache.lucene.util.SetOnce import org.opensearch.action.admin.cluster.node.tasks.list.ListTasksRequest import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest @@ -49,7 +51,6 @@ import org.opensearch.common.xcontent.json.JsonXContent import org.opensearch.snapshots.SnapshotState import org.opensearch.tasks.TaskInfo import org.opensearch.test.OpenSearchTestCase -import org.opensearch.test.OpenSearchTestCase.assertBusy import org.opensearch.test.rest.OpenSearchRestTestCase import org.hamcrest.Matchers import org.junit.After @@ -68,7 +69,6 @@ import java.util.Collections import javax.net.ssl.SSLContext import javax.net.ssl.TrustManager import javax.net.ssl.X509TrustManager -import javax.security.cert.X509Certificate /** * This class provides basic support of managing life-cyle of @@ -96,10 +96,13 @@ abstract class MultiClusterRestTestCase : OpenSearchTestCase() { }) val sslContext = SSLContext.getInstance("SSL") sslContext.init(null, trustCerts, java.security.SecureRandom()) + val tlsStrategy = ClientTlsStrategyBuilder.create().setSslContext(sslContext) + .setHostnameVerifier { _, _ -> true } // Disable hostname verification for local cluster + .build() + val connManager = PoolingAsyncClientConnectionManagerBuilder.create().setTlsStrategy(tlsStrategy).build() val builder = RestClient.builder(*httpHosts.toTypedArray()).setHttpClientConfigCallback { httpAsyncClientBuilder -> - httpAsyncClientBuilder.setSSLHostnameVerifier { _, _ -> true } // Disable hostname verification for local cluster - httpAsyncClientBuilder.setSSLContext(sslContext) + httpAsyncClientBuilder.setConnectionManager(connManager) } configureClient(builder, getClusterSettings(clusterName), securityEnabled) builder.setStrictDeprecationMode(false) @@ -208,9 +211,11 @@ abstract class MultiClusterRestTestCase : OpenSearchTestCase() { val keyStore = KeyStore.getInstance(keyStoreType) Files.newInputStream(path).use { `is` -> keyStore.load(`is`, keystorePass.toCharArray()) } val sslcontext = SSLContexts.custom().loadTrustMaterial(keyStore, null).build() - val sessionStrategy = SSLIOSessionStrategy(sslcontext) - builder.setHttpClientConfigCallback { httpClientBuilder: HttpAsyncClientBuilder -> - httpClientBuilder.setSSLStrategy(sessionStrategy) + val tlsStrategy = ClientTlsStrategyBuilder.create().setSslContext(sslcontext).build() + val connManager = PoolingAsyncClientConnectionManagerBuilder.create().setTlsStrategy(tlsStrategy).build() + + builder.setHttpClientConfigCallback { httpAsyncClientBuilder: HttpAsyncClientBuilder -> + httpAsyncClientBuilder.setConnectionManager(connManager) } } catch (e: KeyStoreException) { throw RuntimeException("Error setting up ssl", e) @@ -242,8 +247,7 @@ abstract class MultiClusterRestTestCase : OpenSearchTestCase() { val socketTimeout = TimeValue.parseTimeValue(socketTimeoutString ?: "60s", OpenSearchRestTestCase.CLIENT_SOCKET_TIMEOUT) builder.setRequestConfigCallback { conf: RequestConfig.Builder -> - conf.setSocketTimeout( - Math.toIntExact(socketTimeout.millis)) + conf.setResponseTimeout(Timeout.ofMilliseconds(socketTimeout.millis)) } if (settings.hasValue(OpenSearchRestTestCase.CLIENT_PATH_PREFIX)) { builder.setPathPrefix(settings[OpenSearchRestTestCase.CLIENT_PATH_PREFIX]) @@ -356,7 +360,7 @@ abstract class MultiClusterRestTestCase : OpenSearchTestCase() { private fun triggerRequest(client: RestClient, method: String, endpoint: String, reqBody: String) { val req = Request(method, endpoint) - req.entity = NStringEntity(reqBody, ContentType.APPLICATION_JSON) + req.entity = StringEntity(reqBody, ContentType.APPLICATION_JSON) val res = client.performRequest(req) assertTrue(HttpStatus.SC_CREATED.toLong() == res.statusLine.statusCode.toLong() || @@ -505,7 +509,7 @@ abstract class MultiClusterRestTestCase : OpenSearchTestCase() { } }""".trimMargin() - persistentConnectionRequest.entity = NStringEntity(entityAsString, ContentType.APPLICATION_JSON) + persistentConnectionRequest.entity = StringEntity(entityAsString, ContentType.APPLICATION_JSON) val persistentConnectionResponse = fromCluster.lowLevelClient.performRequest(persistentConnectionRequest) assertEquals(HttpStatus.SC_OK.toLong(), persistentConnectionResponse.statusLine.statusCode.toLong()) } @@ -561,7 +565,7 @@ abstract class MultiClusterRestTestCase : OpenSearchTestCase() { }] }""".trimMargin() - persistentConnectionRequest.entity = NStringEntity(entityAsString, ContentType.APPLICATION_JSON) + persistentConnectionRequest.entity = StringEntity(entityAsString, ContentType.APPLICATION_JSON) val persistentConnectionResponse = cluster.lowLevelClient.performRequest(persistentConnectionRequest) assertEquals(HttpStatus.SC_OK.toLong(), persistentConnectionResponse.statusLine.statusCode.toLong()) } @@ -579,7 +583,7 @@ abstract class MultiClusterRestTestCase : OpenSearchTestCase() { val entityAsString = """ {"value" : "$docValue"}""".trimMargin() - persistentConnectionRequest.entity = NStringEntity(entityAsString, ContentType.APPLICATION_JSON) + persistentConnectionRequest.entity = StringEntity(entityAsString, ContentType.APPLICATION_JSON) val persistentConnectionResponse = cluster.lowLevelClient.performRequest(persistentConnectionRequest) assertEquals(HttpStatus.SC_CREATED.toLong(), persistentConnectionResponse.statusLine.statusCode.toLong()) } @@ -599,7 +603,7 @@ abstract class MultiClusterRestTestCase : OpenSearchTestCase() { val entityAsString = """ {"template": "*", "settings": {"number_of_shards": 1, "number_of_replicas": 0}}""".trimMargin() - persistentConnectionRequest.entity = NStringEntity(entityAsString, ContentType.APPLICATION_JSON) + persistentConnectionRequest.entity = StringEntity(entityAsString, ContentType.APPLICATION_JSON) cluster.lowLevelClient.performRequest(persistentConnectionRequest) } diff --git a/src/test/kotlin/org/opensearch/replication/MultiClusterSetupIT.kt b/src/test/kotlin/org/opensearch/replication/MultiClusterSetupIT.kt index 7b50a389..a4859878 100644 --- a/src/test/kotlin/org/opensearch/replication/MultiClusterSetupIT.kt +++ b/src/test/kotlin/org/opensearch/replication/MultiClusterSetupIT.kt @@ -12,7 +12,7 @@ package org.opensearch.replication import org.opensearch.replication.MultiClusterAnnotations.ClusterConfiguration -import org.apache.http.util.EntityUtils +import org.apache.hc.core5.http.io.entity.EntityUtils import org.opensearch.client.Request @MultiClusterAnnotations.ClusterConfigurations( diff --git a/src/test/kotlin/org/opensearch/replication/ReplicationHelpers.kt b/src/test/kotlin/org/opensearch/replication/ReplicationHelpers.kt index d5ebf0c9..11c9fbba 100644 --- a/src/test/kotlin/org/opensearch/replication/ReplicationHelpers.kt +++ b/src/test/kotlin/org/opensearch/replication/ReplicationHelpers.kt @@ -96,7 +96,7 @@ fun RestHighLevelClient.startReplication(request: StartReplicationRequest, waitForNoInitializingShards() } fun getAckResponse(lowLevelResponse: Response): AcknowledgedResponse { - val xContentType = XContentType.fromMediaType(lowLevelResponse.entity.contentType.value) + val xContentType = XContentType.fromMediaType(lowLevelResponse.entity.contentType) val xcp = xContentType.xContent().createParser(NamedXContentRegistry.EMPTY, DeprecationHandler.IGNORE_DEPRECATIONS, lowLevelResponse.entity.content) return AcknowledgedResponse.fromXContent(xcp) diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/ClusterRerouteFollowerIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/ClusterRerouteFollowerIT.kt index 87cb313b..d53fb5f3 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/ClusterRerouteFollowerIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/ClusterRerouteFollowerIT.kt @@ -5,14 +5,9 @@ import org.opensearch.replication.MultiClusterAnnotations import org.opensearch.replication.StartReplicationRequest import org.opensearch.replication.startReplication import org.opensearch.replication.stopReplication -import org.apache.http.entity.ContentType -import org.apache.http.nio.entity.NStringEntity import org.assertj.core.api.Assertions -import org.opensearch.client.Request import org.opensearch.client.RequestOptions import org.opensearch.client.indices.CreateIndexRequest -import org.opensearch.client.indices.GetIndexRequest -import org.opensearch.test.OpenSearchTestCase.assertBusy import org.junit.Assert import org.junit.Assume import org.junit.Before diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/ClusterRerouteLeaderIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/ClusterRerouteLeaderIT.kt index 3d40ca89..1da2b9d1 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/ClusterRerouteLeaderIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/ClusterRerouteLeaderIT.kt @@ -5,8 +5,8 @@ import org.opensearch.replication.MultiClusterAnnotations import org.opensearch.replication.StartReplicationRequest import org.opensearch.replication.startReplication import org.opensearch.replication.stopReplication -import org.apache.http.entity.ContentType -import org.apache.http.nio.entity.NStringEntity +import org.apache.hc.core5.http.ContentType +import org.apache.hc.core5.http.io.entity.StringEntity import org.assertj.core.api.Assertions import org.opensearch.client.Request import org.opensearch.client.RequestOptions diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/PauseReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/PauseReplicationIT.kt index 2f582605..d8c7d628 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/PauseReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/PauseReplicationIT.kt @@ -26,7 +26,7 @@ import org.opensearch.replication.updateReplication import org.opensearch.replication.getShardReplicationTasks import org.opensearch.replication.`validate paused status response due to leader index deleted` import org.opensearch.replication.`validate status syncing response` -import org.apache.http.util.EntityUtils +import org.apache.hc.core5.http.io.entity.EntityUtils import org.assertj.core.api.Assertions.assertThat import org.assertj.core.api.Assertions.assertThatThrownBy import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest @@ -40,7 +40,6 @@ import org.opensearch.cluster.metadata.IndexMetadata import org.opensearch.common.settings.Settings import org.opensearch.common.unit.TimeValue import org.opensearch.index.mapper.MapperService -import org.opensearch.test.OpenSearchTestCase.assertBusy import java.util.concurrent.TimeUnit diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityBase.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityBase.kt index df37573c..90e31fe6 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityBase.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityBase.kt @@ -12,13 +12,11 @@ package org.opensearch.replication.integ.rest import org.opensearch.replication.MultiClusterRestTestCase -import org.apache.http.HttpStatus -import org.apache.http.entity.ContentType -import org.apache.http.nio.entity.NStringEntity +import org.apache.hc.core5.http.HttpStatus +import org.apache.hc.core5.http.ContentType +import org.apache.hc.core5.http.io.entity.StringEntity import org.opensearch.client.Request -import org.junit.AfterClass import org.junit.BeforeClass -import javax.swing.text.StyledEditorKit abstract class SecurityBase : MultiClusterRestTestCase() { companion object { @@ -82,7 +80,7 @@ abstract class SecurityBase : MultiClusterRestTestCase() { ] } """.trimMargin() - persistentConnectionRequest.entity = NStringEntity(entityAsString, ContentType.APPLICATION_JSON) + persistentConnectionRequest.entity = StringEntity(entityAsString, ContentType.APPLICATION_JSON) val persistentConnectionResponse = leaderClient!!.lowLevelClient.performRequest(persistentConnectionRequest) assertTrue(HttpStatus.SC_CREATED.toLong() == persistentConnectionResponse.statusLine.statusCode.toLong() || HttpStatus.SC_OK.toLong() == persistentConnectionResponse.statusLine.statusCode.toLong()) @@ -127,7 +125,7 @@ abstract class SecurityBase : MultiClusterRestTestCase() { ] } """.trimMargin() - persistentConnectionRequest.entity = NStringEntity(entityAsString, ContentType.APPLICATION_JSON) + persistentConnectionRequest.entity = StringEntity(entityAsString, ContentType.APPLICATION_JSON) val persistentConnectionResponse = leaderClient!!.lowLevelClient.performRequest(persistentConnectionRequest) assertTrue(HttpStatus.SC_CREATED.toLong() == persistentConnectionResponse.statusLine.statusCode.toLong() || HttpStatus.SC_OK.toLong() == persistentConnectionResponse.statusLine.statusCode.toLong()) @@ -159,7 +157,7 @@ abstract class SecurityBase : MultiClusterRestTestCase() { ] } """.trimMargin() - persistentConnectionRequest.entity = NStringEntity(entityAsString, ContentType.APPLICATION_JSON) + persistentConnectionRequest.entity = StringEntity(entityAsString, ContentType.APPLICATION_JSON) val persistentConnectionResponse = leaderClient!!.lowLevelClient.performRequest(persistentConnectionRequest) assertTrue(HttpStatus.SC_CREATED.toLong() == persistentConnectionResponse.statusLine.statusCode.toLong() || HttpStatus.SC_OK.toLong() == persistentConnectionResponse.statusLine.statusCode.toLong()) @@ -191,7 +189,7 @@ abstract class SecurityBase : MultiClusterRestTestCase() { ] } """.trimMargin() - persistentConnectionRequest.entity = NStringEntity(entityAsString, ContentType.APPLICATION_JSON) + persistentConnectionRequest.entity = StringEntity(entityAsString, ContentType.APPLICATION_JSON) val persistentConnectionResponse = leaderClient!!.lowLevelClient.performRequest(persistentConnectionRequest) assertTrue(HttpStatus.SC_CREATED.toLong() == persistentConnectionResponse.statusLine.statusCode.toLong() || HttpStatus.SC_OK.toLong() == persistentConnectionResponse.statusLine.statusCode.toLong()) @@ -211,7 +209,7 @@ abstract class SecurityBase : MultiClusterRestTestCase() { ] } """.trimMargin() - persistentConnectionRequest.entity = NStringEntity(entityAsString, ContentType.APPLICATION_JSON) + persistentConnectionRequest.entity = StringEntity(entityAsString, ContentType.APPLICATION_JSON) val persistentConnectionResponse = leaderClient!!.lowLevelClient.performRequest(persistentConnectionRequest) assertTrue(HttpStatus.SC_CREATED.toLong() == persistentConnectionResponse.statusLine.statusCode.toLong() || HttpStatus.SC_OK.toLong() == persistentConnectionResponse.statusLine.statusCode.toLong()) @@ -234,7 +232,7 @@ abstract class SecurityBase : MultiClusterRestTestCase() { ] } """.trimMargin() - persistentConnectionRequest.entity = NStringEntity(entityAsString, ContentType.APPLICATION_JSON) + persistentConnectionRequest.entity = StringEntity(entityAsString, ContentType.APPLICATION_JSON) val persistentConnectionResponse = leaderClient!!.lowLevelClient.performRequest(persistentConnectionRequest) assertTrue(HttpStatus.SC_CREATED.toLong() == persistentConnectionResponse.statusLine.statusCode.toLong() || HttpStatus.SC_OK.toLong() == persistentConnectionResponse.statusLine.statusCode.toLong()) @@ -266,7 +264,7 @@ abstract class SecurityBase : MultiClusterRestTestCase() { ] } """.trimMargin() - persistentConnectionRequest.entity = NStringEntity(entityAsString, ContentType.APPLICATION_JSON) + persistentConnectionRequest.entity = StringEntity(entityAsString, ContentType.APPLICATION_JSON) val persistentConnectionResponse = followerClient!!.lowLevelClient.performRequest(persistentConnectionRequest) assertTrue(HttpStatus.SC_CREATED.toLong() == persistentConnectionResponse.statusLine.statusCode.toLong() || HttpStatus.SC_OK.toLong() == persistentConnectionResponse.statusLine.statusCode.toLong()) @@ -279,7 +277,7 @@ abstract class SecurityBase : MultiClusterRestTestCase() { {"users": ["$user"]} """.trimMargin() - persistentConnectionRequest.entity = NStringEntity(entityAsString, ContentType.APPLICATION_JSON) + persistentConnectionRequest.entity = StringEntity(entityAsString, ContentType.APPLICATION_JSON) val persistentConnectionResponse = followerClient!!.lowLevelClient.performRequest(persistentConnectionRequest) assertTrue(HttpStatus.SC_CREATED.toLong() == persistentConnectionResponse.statusLine.statusCode.toLong() || HttpStatus.SC_OK.toLong() == persistentConnectionResponse.statusLine.statusCode.toLong()) @@ -308,7 +306,7 @@ abstract class SecurityBase : MultiClusterRestTestCase() { } """.trimMargin() - persistentConnectionRequest.entity = NStringEntity(entityAsString, ContentType.APPLICATION_JSON) + persistentConnectionRequest.entity = StringEntity(entityAsString, ContentType.APPLICATION_JSON) val persistentConnectionResponse = followerClient!!.lowLevelClient.performRequest(persistentConnectionRequest) assertTrue(HttpStatus.SC_CREATED.toLong() == persistentConnectionResponse.statusLine.statusCode.toLong() || HttpStatus.SC_OK.toLong() == persistentConnectionResponse.statusLine.statusCode.toLong()) diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesIT.kt index 8d5be9cc..cbe46825 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesIT.kt @@ -13,9 +13,9 @@ package org.opensearch.replication.integ.rest import org.opensearch.replication.* import org.opensearch.replication.util.addBasicAuthHeader -import org.apache.http.HttpStatus -import org.apache.http.entity.ContentType -import org.apache.http.nio.entity.NStringEntity +import org.apache.hc.core5.http.HttpStatus +import org.apache.hc.core5.http.ContentType +import org.apache.hc.core5.http.io.entity.StringEntity import org.assertj.core.api.Assertions import org.opensearch.action.admin.indices.settings.get.GetSettingsRequest import org.opensearch.client.Request @@ -26,7 +26,6 @@ import org.opensearch.client.indices.CreateIndexRequest import org.opensearch.client.indices.GetIndexRequest import org.opensearch.cluster.metadata.IndexMetadata import org.opensearch.common.settings.Settings -import org.opensearch.test.OpenSearchTestCase import org.junit.Assert import org.junit.Assume import java.util.* @@ -462,7 +461,7 @@ class SecurityCustomRolesIT: SecurityBase() { ] } """.trimMargin() - persistentConnectionRequest.entity = NStringEntity(entityAsString, ContentType.APPLICATION_JSON) + persistentConnectionRequest.entity = StringEntity(entityAsString, ContentType.APPLICATION_JSON) val persistentConnectionResponse = followerClient!!.lowLevelClient.performRequest(persistentConnectionRequest) assertEquals(HttpStatus.SC_OK.toLong(), persistentConnectionResponse.statusLine.statusCode.toLong()) } diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesLeaderIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesLeaderIT.kt index af81a6b5..8955d22f 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesLeaderIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesLeaderIT.kt @@ -13,9 +13,9 @@ package org.opensearch.replication.integ.rest import org.opensearch.replication.* import org.opensearch.replication.util.addBasicAuthHeader -import org.apache.http.HttpStatus -import org.apache.http.entity.ContentType -import org.apache.http.nio.entity.NStringEntity +import org.apache.hc.core5.http.HttpStatus +import org.apache.hc.core5.http.ContentType +import org.apache.hc.core5.http.io.entity.StringEntity import org.assertj.core.api.Assertions import org.opensearch.client.Request import org.opensearch.client.RequestOptions @@ -149,7 +149,7 @@ class SecurityCustomRolesLeaderIT: SecurityBase() { ] } """.trimMargin() - persistentConnectionRequest.entity = NStringEntity(entityAsString, ContentType.APPLICATION_JSON) + persistentConnectionRequest.entity = StringEntity(entityAsString, ContentType.APPLICATION_JSON) val persistentConnectionResponse = followerClient!!.lowLevelClient.performRequest(persistentConnectionRequest) assertEquals(HttpStatus.SC_OK.toLong(), persistentConnectionResponse.statusLine.statusCode.toLong()) } @@ -174,7 +174,7 @@ class SecurityCustomRolesLeaderIT: SecurityBase() { ] } """.trimMargin() - persistentConnectionRequest.entity = NStringEntity(entityAsString, ContentType.APPLICATION_JSON) + persistentConnectionRequest.entity = StringEntity(entityAsString, ContentType.APPLICATION_JSON) val persistentConnectionResponse = followerClient!!.lowLevelClient.performRequest(persistentConnectionRequest) assertEquals(HttpStatus.SC_OK.toLong(), persistentConnectionResponse.statusLine.statusCode.toLong()) } diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt index ee82c3c4..881ec922 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt @@ -26,10 +26,10 @@ import org.opensearch.replication.`validate status syncing response` import org.opensearch.replication.startReplication import org.opensearch.replication.stopReplication import org.opensearch.replication.updateReplication -import org.apache.http.HttpStatus -import org.apache.http.entity.ContentType -import org.apache.http.nio.entity.NStringEntity -import org.apache.http.util.EntityUtils +import org.apache.hc.core5.http.HttpStatus +import org.apache.hc.core5.http.ContentType +import org.apache.hc.core5.http.io.entity.StringEntity +import org.apache.hc.core5.http.io.entity.EntityUtils import org.assertj.core.api.Assertions.assertThat import org.assertj.core.api.Assertions.assertThatThrownBy import org.opensearch.OpenSearchStatusException @@ -492,7 +492,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { } }""".trimMargin() - persistentConnectionRequest.entity = NStringEntity(entityAsString, ContentType.APPLICATION_JSON) + persistentConnectionRequest.entity = StringEntity(entityAsString, ContentType.APPLICATION_JSON) val persistentConnectionResponse = cluster.lowLevelClient.performRequest(persistentConnectionRequest) assertEquals(HttpStatus.SC_OK.toLong(), persistentConnectionResponse.statusLine.statusCode.toLong()) } @@ -1231,7 +1231,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { "cluster.routing.allocation.exclude._ip": "${excludeIps.joinToString()}" } }""".trimMargin() - transientSettingsRequest.entity = NStringEntity(entityAsString, ContentType.APPLICATION_JSON) + transientSettingsRequest.entity = StringEntity(entityAsString, ContentType.APPLICATION_JSON) val transientSettingsResponse = getNamedCluster(clusterName).lowLevelClient.performRequest(transientSettingsRequest) assertEquals(HttpStatus.SC_OK.toLong(), transientSettingsResponse.statusLine.statusCode.toLong()) } diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/StopReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/StopReplicationIT.kt index 09b797ae..8f0c31bc 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/StopReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/StopReplicationIT.kt @@ -20,7 +20,7 @@ import org.opensearch.replication.stopReplication import org.opensearch.replication.replicationStatus import org.opensearch.replication.getShardReplicationTasks import org.opensearch.replication.`validate status syncing response` -import org.apache.http.util.EntityUtils +import org.apache.hc.core5.http.io.entity.EntityUtils import org.assertj.core.api.Assertions.assertThat import org.assertj.core.api.Assertions.assertThatThrownBy import org.junit.Assert diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt index 82db8fb5..042cd9fe 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt @@ -20,9 +20,9 @@ import org.opensearch.replication.stopReplication import org.opensearch.replication.updateAutoFollowPattern import org.opensearch.replication.task.autofollow.AutoFollowExecutor import org.opensearch.replication.task.index.IndexReplicationExecutor -import org.apache.http.HttpStatus -import org.apache.http.entity.ContentType -import org.apache.http.nio.entity.NStringEntity +import org.apache.hc.core5.http.HttpStatus +import org.apache.hc.core5.http.ContentType +import org.apache.hc.core5.http.io.entity.StringEntity import org.assertj.core.api.Assertions import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest import org.opensearch.action.admin.indices.settings.get.GetSettingsRequest @@ -44,7 +44,6 @@ import org.opensearch.replication.ReplicationPlugin import org.opensearch.replication.updateReplicationStartBlockSetting import org.opensearch.replication.updateAutoFollowConcurrentStartReplicationJobSetting import org.opensearch.replication.waitForShardTaskStart -import org.opensearch.test.OpenSearchTestCase.assertBusy import java.lang.Thread.sleep import java.util.HashMap import java.util.concurrent.TimeUnit @@ -506,7 +505,7 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { } }""".trimMargin() - persistentConnectionRequest.entity = NStringEntity(entityAsString, ContentType.APPLICATION_JSON) + persistentConnectionRequest.entity = StringEntity(entityAsString, ContentType.APPLICATION_JSON) val persistentConnectionResponse = fromCluster.lowLevelClient.performRequest(persistentConnectionRequest) assertEquals(HttpStatus.SC_OK.toLong(), persistentConnectionResponse.statusLine.statusCode.toLong()) } From 77074c45e5350568b83b97b2f865d876095ea34c Mon Sep 17 00:00:00 2001 From: Priyanka Sharma Date: Wed, 2 Nov 2022 05:36:40 -0700 Subject: [PATCH 045/157] Github-Issue-544:Replication auto pauses on follower cluster having wait_for_active_shards true:Fixed Signed-off-by: Priyanka Sharma --- .../opensearch/replication/task/index/IndexReplicationTask.kt | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt index 906312ac..395119c3 100644 --- a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt @@ -143,7 +143,8 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript IndexMetadata.INDEX_BLOCKS_READ_ONLY_ALLOW_DELETE_SETTING, EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE_SETTING, EnableAllocationDecider.INDEX_ROUTING_ALLOCATION_ENABLE_SETTING, - IndexSettings.INDEX_SOFT_DELETES_RETENTION_LEASE_PERIOD_SETTING + IndexSettings.INDEX_SOFT_DELETES_RETENTION_LEASE_PERIOD_SETTING, + IndexMetadata.SETTING_WAIT_FOR_ACTIVE_SHARDS ) val blockListedSettings :Set = blSettings.stream().map { k -> k.key }.collect(Collectors.toSet()) From 47c46498d95448a8e325c1b3b66d5a19fa737a19 Mon Sep 17 00:00:00 2001 From: Priyanka Sharma Date: Tue, 8 Nov 2022 21:56:42 -0800 Subject: [PATCH 046/157] Github-issue-544: Added integ tests to verify the wait_for_active_shards index setting behaviour. Signed-off-by: Priyanka Sharma --- .../integ/rest/StartReplicationIT.kt | 113 ++++++++++++++++++ 1 file changed, 113 insertions(+) diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt index 881ec922..9d4307e7 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt @@ -1221,6 +1221,119 @@ class StartReplicationIT: MultiClusterRestTestCase() { } + fun `test that wait_for_active_shards setting is set on leader and not on follower`() { + val followerClient = getClientForCluster(FOLLOWER) + val leaderClient = getClientForCluster(LEADER) + + createConnectionBetweenClusters(FOLLOWER, LEADER) + + val settings = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetadata.SETTING_WAIT_FOR_ACTIVE_SHARDS.getKey(), Integer.toString(2)) + .build() + + val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName).settings(settings), RequestOptions.DEFAULT) + assertThat(createIndexResponse.isAcknowledged).isTrue() + try { + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName)) + assertBusy { + assertThat(followerClient.indices() + .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) + .isEqualTo(true) + } + TimeUnit.SECONDS.sleep(SLEEP_TIME_BETWEEN_SYNC) + + val getSettingsRequest = GetSettingsRequest() + getSettingsRequest.indices(followerIndexName) + getSettingsRequest.includeDefaults(true) + + Assert.assertNull(followerClient.indices() + .getSettings(getSettingsRequest, RequestOptions.DEFAULT) + .indexToSettings[followerIndexName].get("index.write.wait_for_active_shards") + ) + } finally { + followerClient.stopReplication(followerIndexName) + } + } + + fun `test that wait_for_active_shards setting is updated on leader and not on follower`() { + val followerClient = getClientForCluster(FOLLOWER) + val leaderClient = getClientForCluster(LEADER) + + createConnectionBetweenClusters(FOLLOWER, LEADER) + + val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) + assertThat(createIndexResponse.isAcknowledged).isTrue() + try { + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName)) + assertBusy { + assertThat(followerClient.indices() + .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) + .isEqualTo(true) + } + TimeUnit.SECONDS.sleep(SLEEP_TIME_BETWEEN_SYNC) + + //Use Update API + val settings = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetadata.SETTING_WAIT_FOR_ACTIVE_SHARDS.getKey(), Integer.toString(2)) + .build() + + followerClient.updateReplication( followerIndexName, settings) + TimeUnit.SECONDS.sleep(SLEEP_TIME_BETWEEN_SYNC) + + val getSettingsRequest = GetSettingsRequest() + getSettingsRequest.indices(followerIndexName) + getSettingsRequest.includeDefaults(true) + + Assert.assertNull(followerClient.indices() + .getSettings(getSettingsRequest, RequestOptions.DEFAULT) + .indexToSettings[followerIndexName].get("index.write.wait_for_active_shards") + ) + } finally { + followerClient.stopReplication(followerIndexName) + } + } + + fun `test that wait_for_active_shards setting is updated on follower through start replication api`() { + val followerClient = getClientForCluster(FOLLOWER) + val leaderClient = getClientForCluster(LEADER) + + createConnectionBetweenClusters(FOLLOWER, LEADER) + + val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) + assertThat(createIndexResponse.isAcknowledged).isTrue() + + val settings = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetadata.SETTING_WAIT_FOR_ACTIVE_SHARDS.getKey(), Integer.toString(2)) + .build() + try { + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName, settings = settings)) + assertBusy { + assertThat(followerClient.indices() + .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) + .isEqualTo(true) + } + TimeUnit.SECONDS.sleep(SLEEP_TIME_BETWEEN_SYNC) + + val getSettingsRequest = GetSettingsRequest() + getSettingsRequest.indices(followerIndexName) + getSettingsRequest.includeDefaults(true) + assertBusy ({ + Assert.assertEquals( + "2", + followerClient.indices() + .getSettings(getSettingsRequest, RequestOptions.DEFAULT) + .indexToSettings[followerIndexName].get("index.write.wait_for_active_shards") + ) + }, 15, TimeUnit.SECONDS) + } finally { + followerClient.stopReplication(followerIndexName) + } + } + + private fun excludeAllClusterNodes(clusterName: String) { val transientSettingsRequest = Request("PUT", "_cluster/settings") // Get IPs directly from the cluster to handle all cases - single node cluster, multi node cluster and remote test cluster. From f3fee90980592df193fd14d795c842fce797e2cd Mon Sep 17 00:00:00 2001 From: Sai Kumar Date: Thu, 10 Nov 2022 10:45:28 +0530 Subject: [PATCH 047/157] Modified minsupportedversion to OS-2.0 for metadata based on OS-3.0 changes (#626) Signed-off-by: Sai Kumar --- .../replication/metadata/state/ReplicationStateMetadata.kt | 2 +- .../replication/task/autofollow/AutoFollowParams.kt | 2 +- .../replication/task/index/IndexReplicationParams.kt | 4 +--- .../replication/task/shard/ShardReplicationParams.kt | 4 +--- 4 files changed, 4 insertions(+), 8 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/metadata/state/ReplicationStateMetadata.kt b/src/main/kotlin/org/opensearch/replication/metadata/state/ReplicationStateMetadata.kt index edfec793..b6407be7 100644 --- a/src/main/kotlin/org/opensearch/replication/metadata/state/ReplicationStateMetadata.kt +++ b/src/main/kotlin/org/opensearch/replication/metadata/state/ReplicationStateMetadata.kt @@ -108,7 +108,7 @@ data class ReplicationStateMetadata(val replicationDetails: Map Date: Fri, 11 Nov 2022 04:12:46 -0800 Subject: [PATCH 048/157] Corrected integ tests to check for default value of the wait_for_active_shards setting on follower index Signed-off-by: Priyanka Sharma --- .../integ/rest/StartReplicationIT.kt | 65 +++++++++++++++---- 1 file changed, 52 insertions(+), 13 deletions(-) diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt index 9d4307e7..f5aa7310 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt @@ -1243,14 +1243,33 @@ class StartReplicationIT: MultiClusterRestTestCase() { } TimeUnit.SECONDS.sleep(SLEEP_TIME_BETWEEN_SYNC) + // Verify the setting on leader + val getLeaderSettingsRequest = GetSettingsRequest() + getLeaderSettingsRequest.indices(leaderIndexName) + getLeaderSettingsRequest.includeDefaults(true) + + assertBusy ({ + Assert.assertEquals( + "2", + leaderClient.indices() + .getSettings(getLeaderSettingsRequest, RequestOptions.DEFAULT) + .indexToSettings[leaderIndexName][IndexMetadata.SETTING_WAIT_FOR_ACTIVE_SHARDS.getKey()] + ) + }, 15, TimeUnit.SECONDS) + + // Verify that the setting is not updated on follower and follower has default value of the setting val getSettingsRequest = GetSettingsRequest() getSettingsRequest.indices(followerIndexName) getSettingsRequest.includeDefaults(true) - Assert.assertNull(followerClient.indices() - .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName].get("index.write.wait_for_active_shards") - ) + assertBusy ({ + Assert.assertEquals( + "1", + followerClient.indices() + .getSettings(getSettingsRequest, RequestOptions.DEFAULT) + .getSetting(followerIndexName, IndexMetadata.SETTING_WAIT_FOR_ACTIVE_SHARDS.key) + ) + }, 15, TimeUnit.SECONDS) } finally { followerClient.stopReplication(followerIndexName) } @@ -1274,22 +1293,43 @@ class StartReplicationIT: MultiClusterRestTestCase() { TimeUnit.SECONDS.sleep(SLEEP_TIME_BETWEEN_SYNC) //Use Update API - val settings = Settings.builder() + val settingsBuilder = Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) .put(IndexMetadata.SETTING_WAIT_FOR_ACTIVE_SHARDS.getKey(), Integer.toString(2)) - .build() - followerClient.updateReplication( followerIndexName, settings) + val settingsUpdateResponse = leaderClient.indices().putSettings(UpdateSettingsRequest(leaderIndexName) + .settings(settingsBuilder.build()), RequestOptions.DEFAULT) + Assert.assertEquals(settingsUpdateResponse.isAcknowledged, true) + TimeUnit.SECONDS.sleep(SLEEP_TIME_BETWEEN_SYNC) + // Verify the setting on leader + val getLeaderSettingsRequest = GetSettingsRequest() + getLeaderSettingsRequest.indices(leaderIndexName) + getLeaderSettingsRequest.includeDefaults(true) + + assertBusy ({ + Assert.assertEquals( + "2", + leaderClient.indices() + .getSettings(getLeaderSettingsRequest, RequestOptions.DEFAULT) + .indexToSettings[leaderIndexName][IndexMetadata.SETTING_WAIT_FOR_ACTIVE_SHARDS.getKey()] + ) + }, 15, TimeUnit.SECONDS) + + val getSettingsRequest = GetSettingsRequest() getSettingsRequest.indices(followerIndexName) getSettingsRequest.includeDefaults(true) - Assert.assertNull(followerClient.indices() - .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName].get("index.write.wait_for_active_shards") - ) + assertBusy ({ + Assert.assertEquals( + "1", + followerClient.indices() + .getSettings(getSettingsRequest, RequestOptions.DEFAULT) + .getSetting(followerIndexName, IndexMetadata.SETTING_WAIT_FOR_ACTIVE_SHARDS.key) + ) + }, 15, TimeUnit.SECONDS) } finally { followerClient.stopReplication(followerIndexName) } @@ -1325,7 +1365,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { "2", followerClient.indices() .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName].get("index.write.wait_for_active_shards") + .indexToSettings[followerIndexName][IndexMetadata.SETTING_WAIT_FOR_ACTIVE_SHARDS.getKey()] ) }, 15, TimeUnit.SECONDS) } finally { @@ -1333,7 +1373,6 @@ class StartReplicationIT: MultiClusterRestTestCase() { } } - private fun excludeAllClusterNodes(clusterName: String) { val transientSettingsRequest = Request("PUT", "_cluster/settings") // Get IPs directly from the cluster to handle all cases - single node cluster, multi node cluster and remote test cluster. From 2d83e429bc3be28f4a74d4a149b030494d13e93e Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Tue, 22 Nov 2022 11:41:44 +0530 Subject: [PATCH 049/157] Updated follower port in Readme (#629) Signed-off-by: Monu Singh Signed-off-by: Monu Singh --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index f98b0259..b2c537ef 100644 --- a/README.md +++ b/README.md @@ -70,7 +70,7 @@ Following steps will help you install the replication plugin on a test cluster. # Set variables for readability (in different terminal window/tab where you will run rest of the steps) export LEADER=localhost:9200 -export FOLLOWER=localhost:9201 +export FOLLOWER=localhost:9203 export LEADER_TRANSPORT=localhost:9300 ``` From ad8e8ded88c2548a6811855929d5c71f5cea4e3e Mon Sep 17 00:00:00 2001 From: sricharanvuppu <113983630+sricharanvuppu@users.noreply.github.com> Date: Tue, 29 Nov 2022 10:35:25 +0530 Subject: [PATCH 050/157] stopReplication API removed from integ test cases and stopAllReplicationJobs API added in MultiClusterRestTestCase (#619) Signed-off-by: sricharanvuppu Signed-off-by: sricharanvuppu --- .../replication/BasicReplicationIT.kt | 136 +-- .../replication/MultiClusterRestTestCase.kt | 28 +- .../integ/rest/ClusterRerouteFollowerIT.kt | 69 +- .../integ/rest/ClusterRerouteLeaderIT.kt | 67 +- .../integ/rest/PauseReplicationIT.kt | 201 ++-- .../integ/rest/ReplicationStatusIT.kt | 14 +- .../integ/rest/ResumeReplicationIT.kt | 207 ++-- .../replication/integ/rest/SecurityBase.kt | 3 - .../integ/rest/SecurityCustomRolesIT.kt | 242 ++-- .../integ/rest/SecurityCustomRolesLeaderIT.kt | 15 - .../integ/rest/SecurityDlsFlsIT.kt | 170 +-- .../integ/rest/StartReplicationIT.kt | 1056 +++++++---------- .../integ/rest/StopReplicationIT.kt | 107 +- .../integ/rest/UpdateAutoFollowPatternIT.kt | 104 +- .../replication/task/TaskCancellationIT.kt | 48 +- .../shard/TransportReplayChangesActionIT.kt | 96 +- 16 files changed, 999 insertions(+), 1564 deletions(-) diff --git a/src/test/kotlin/org/opensearch/replication/BasicReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/BasicReplicationIT.kt index 7d426ace..ff3bfa83 100644 --- a/src/test/kotlin/org/opensearch/replication/BasicReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/BasicReplicationIT.kt @@ -44,120 +44,96 @@ class BasicReplicationIT : MultiClusterRestTestCase() { val follower = getClientForCluster(FOLL) val leader = getClientForCluster(LEADER) createConnectionBetweenClusters(FOLL, LEADER) - val leaderIndex = randomAlphaOfLength(10).toLowerCase(Locale.ROOT) val followerIndex = randomAlphaOfLength(10).toLowerCase(Locale.ROOT) // Create an empty index on the leader and trigger replication on it val createIndexResponse = leader.indices().create(CreateIndexRequest(leaderIndex), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - follower.startReplication(StartReplicationRequest("source", leaderIndex, followerIndex), waitForRestore=true) - - val source = mapOf("name" to randomAlphaOfLength(20), "age" to randomInt().toString()) - var response = leader.index(IndexRequest(leaderIndex).id("1").source(source), RequestOptions.DEFAULT) + follower.startReplication(StartReplicationRequest("source", leaderIndex, followerIndex), waitForRestore=true) + val source = mapOf("name" to randomAlphaOfLength(20), "age" to randomInt().toString()) + var response = leader.index(IndexRequest(leaderIndex).id("1").source(source), RequestOptions.DEFAULT) + assertThat(response.result).isEqualTo(Result.CREATED) + assertBusy({ + val getResponse = follower.get(GetRequest(followerIndex, "1"), RequestOptions.DEFAULT) + assertThat(getResponse.isExists).isTrue() + assertThat(getResponse.sourceAsMap).isEqualTo(source) + }, 60L, TimeUnit.SECONDS) + // Ensure force merge on leader doesn't impact replication + for (i in 2..5) { + response = leader.index(IndexRequest(leaderIndex).id("$i").source(source), RequestOptions.DEFAULT) assertThat(response.result).isEqualTo(Result.CREATED) - - assertBusy({ - val getResponse = follower.get(GetRequest(followerIndex, "1"), RequestOptions.DEFAULT) + } + leader.indices().forcemerge(ForceMergeRequest(leaderIndex), RequestOptions.DEFAULT) + for (i in 6..10) { + response = leader.index(IndexRequest(leaderIndex).id("$i").source(source), RequestOptions.DEFAULT) + assertThat(response.result).isEqualTo(Result.CREATED) + } + assertBusy({ + for (i in 2..10) { + val getResponse = follower.get(GetRequest(followerIndex, "$i"), RequestOptions.DEFAULT) assertThat(getResponse.isExists).isTrue() assertThat(getResponse.sourceAsMap).isEqualTo(source) - }, 60L, TimeUnit.SECONDS) - - // Ensure force merge on leader doesn't impact replication - for (i in 2..5) { - response = leader.index(IndexRequest(leaderIndex).id("$i").source(source), RequestOptions.DEFAULT) - assertThat(response.result).isEqualTo(Result.CREATED) - } - leader.indices().forcemerge(ForceMergeRequest(leaderIndex), RequestOptions.DEFAULT) - for (i in 6..10) { - response = leader.index(IndexRequest(leaderIndex).id("$i").source(source), RequestOptions.DEFAULT) - assertThat(response.result).isEqualTo(Result.CREATED) } - assertBusy({ - for (i in 2..10) { - val getResponse = follower.get(GetRequest(followerIndex, "$i"), RequestOptions.DEFAULT) - assertThat(getResponse.isExists).isTrue() - assertThat(getResponse.sourceAsMap).isEqualTo(source) - } - }, 60L, TimeUnit.SECONDS) - - // Force merge on follower however isn't allowed due to WRITE block - Assertions.assertThatThrownBy { - follower.indices().forcemerge(ForceMergeRequest(followerIndex), RequestOptions.DEFAULT) - }.isInstanceOf(OpenSearchStatusException::class.java) - .hasMessage("OpenSearch exception [type=cluster_block_exception, reason=index [$followerIndex] " + - "blocked by: [FORBIDDEN/1000/index read-only(cross-cluster-replication)];]") - - } finally { - follower.stopReplication(followerIndex) - } + }, 60L, TimeUnit.SECONDS) + // Force merge on follower however isn't allowed due to WRITE block + Assertions.assertThatThrownBy { + follower.indices().forcemerge(ForceMergeRequest(followerIndex), RequestOptions.DEFAULT) + }.isInstanceOf(OpenSearchStatusException::class.java) + .hasMessage("OpenSearch exception [type=cluster_block_exception, reason=index [$followerIndex] " + + "blocked by: [FORBIDDEN/1000/index read-only(cross-cluster-replication)];]") } fun `test existing index replication`() { val follower = getClientForCluster(FOLL) val leader = getClientForCluster(LEADER) createConnectionBetweenClusters(FOLL, LEADER) - // Create an index with data before commencing replication val leaderIndex = randomAlphaOfLength(10).toLowerCase(Locale.ROOT) val followerIndex = randomAlphaOfLength(10).toLowerCase(Locale.ROOT) val source = mapOf("name" to randomAlphaOfLength(20), "age" to randomInt().toString()) val response = leader.index(IndexRequest(leaderIndex).id("1").source(source), RequestOptions.DEFAULT) assertThat(response.result).withFailMessage("Failed to create leader data").isEqualTo(Result.CREATED) - follower.startReplication(StartReplicationRequest("source", leaderIndex, followerIndex), waitForRestore=true) - assertBusy { val getResponse = follower.get(GetRequest(followerIndex, "1"), RequestOptions.DEFAULT) assertThat(getResponse.isExists).isTrue() assertThat(getResponse.sourceAsMap).isEqualTo(source) } - follower.stopReplication(followerIndex) } fun `test that index operations are replayed to follower during replication`() { val followerClient = getClientForCluster(FOLL) val leaderClient = getClientForCluster(LEADER) createConnectionBetweenClusters(FOLL, LEADER) - + val leaderIndexName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT) + val followerIndexName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT) val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore=true) - - // Create document - var source = mapOf("name" to randomAlphaOfLength(20), "age" to randomInt().toString()) - var response = leaderClient.index(IndexRequest(leaderIndexName).id("1").source(source), RequestOptions.DEFAULT) - assertThat(response.result).withFailMessage("Failed to create leader data").isEqualTo(Result.CREATED) - - assertBusy({ - val getResponse = followerClient.get(GetRequest(followerIndexName, "1"), RequestOptions.DEFAULT) - assertThat(getResponse.isExists).isTrue() - assertThat(getResponse.sourceAsMap).isEqualTo(source) - }, 60L, TimeUnit.SECONDS) - - // Update document - source = mapOf("name" to randomAlphaOfLength(20), "age" to randomInt().toString()) - response = leaderClient.index(IndexRequest(leaderIndexName).id("1").source(source), RequestOptions.DEFAULT) - assertThat(response.result).withFailMessage("Failed to update leader data").isEqualTo(Result.UPDATED) - - assertBusy({ - val getResponse = followerClient.get(GetRequest(followerIndexName, "1"), RequestOptions.DEFAULT) - assertThat(getResponse.isExists).isTrue() - assertThat(getResponse.sourceAsMap).isEqualTo(source) - },60L, TimeUnit.SECONDS) - - // Delete document - val deleteResponse = leaderClient.delete(DeleteRequest(leaderIndexName).id("1"), RequestOptions.DEFAULT) - assertThat(deleteResponse.result).withFailMessage("Failed to delete leader data").isEqualTo(Result.DELETED) - - assertBusy({ - val getResponse = followerClient.get(GetRequest(followerIndexName, "1"), RequestOptions.DEFAULT) - assertThat(getResponse.isExists).isFalse() - }, 60L, TimeUnit.SECONDS) - } finally { - followerClient.stopReplication(followerIndexName) - } + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore=true) + // Create document + var source = mapOf("name" to randomAlphaOfLength(20), "age" to randomInt().toString()) + var response = leaderClient.index(IndexRequest(leaderIndexName).id("1").source(source), RequestOptions.DEFAULT) + assertThat(response.result).withFailMessage("Failed to create leader data").isEqualTo(Result.CREATED) + assertBusy({ + val getResponse = followerClient.get(GetRequest(followerIndexName, "1"), RequestOptions.DEFAULT) + assertThat(getResponse.isExists).isTrue() + assertThat(getResponse.sourceAsMap).isEqualTo(source) + }, 60L, TimeUnit.SECONDS) + // Update document + source = mapOf("name" to randomAlphaOfLength(20), "age" to randomInt().toString()) + response = leaderClient.index(IndexRequest(leaderIndexName).id("1").source(source), RequestOptions.DEFAULT) + assertThat(response.result).withFailMessage("Failed to update leader data").isEqualTo(Result.UPDATED) + assertBusy({ + val getResponse = followerClient.get(GetRequest(followerIndexName, "1"), RequestOptions.DEFAULT) + assertThat(getResponse.isExists).isTrue() + assertThat(getResponse.sourceAsMap).isEqualTo(source) + },60L, TimeUnit.SECONDS) + // Delete document + val deleteResponse = leaderClient.delete(DeleteRequest(leaderIndexName).id("1"), RequestOptions.DEFAULT) + assertThat(deleteResponse.result).withFailMessage("Failed to delete leader data").isEqualTo(Result.DELETED) + assertBusy({ + val getResponse = followerClient.get(GetRequest(followerIndexName, "1"), RequestOptions.DEFAULT) + assertThat(getResponse.isExists).isFalse() + }, 60L, TimeUnit.SECONDS) } } diff --git a/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt b/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt index b1a52536..66d9e53d 100644 --- a/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt +++ b/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt @@ -11,6 +11,7 @@ package org.opensearch.replication +import com.nhaarman.mockitokotlin2.stub import org.opensearch.replication.MultiClusterAnnotations.ClusterConfiguration import org.opensearch.replication.MultiClusterAnnotations.ClusterConfigurations import org.opensearch.replication.MultiClusterAnnotations.getAnnotationsFromClass @@ -57,6 +58,7 @@ import org.junit.After import org.junit.AfterClass import org.junit.Before import org.junit.BeforeClass +import org.opensearch.index.mapper.ObjectMapper import java.nio.file.Files import java.security.KeyManagementException import java.security.KeyStore @@ -423,8 +425,32 @@ abstract class MultiClusterRestTestCase : OpenSearchTestCase() { testCluster.lowLevelClient.performRequest(request) } } - + private fun stopAllReplicationJobs(testCluster: TestCluster) { + val indicesResponse = testCluster.lowLevelClient.performRequest((Request("GET","/_cat/indices/*,-.*?format=json&pretty"))) + val indicesResponseEntity = EntityUtils.toString(indicesResponse.entity) + var parser = XContentType.JSON.xContent().createParser(NamedXContentRegistry.EMPTY, DeprecationHandler.THROW_UNSUPPORTED_OPERATION, indicesResponseEntity) + parser.list().forEach{ item-> + val str = item.toString() + val map = str.subSequence(1,str.length-1).split(",").associate { + val (key, value) = it.trim().split("=") + key to value + } + val ind = map.get("index") + try { + val stopRequest = Request("POST","/_plugins/_replication/" + ind.toString() + "/_stop") + stopRequest.setJsonEntity("{}") + stopRequest.setOptions(RequestOptions.DEFAULT) + val response=testCluster.lowLevelClient.performRequest(stopRequest) + } + catch (e:ResponseException){ + if(e.response.statusLine.statusCode!=400) { + throw e + } + } + } + } protected fun wipeIndicesFromCluster(testCluster: TestCluster) { + stopAllReplicationJobs(testCluster) try { val deleteRequest = Request("DELETE", "*,-.*") // All except system indices val response = testCluster.lowLevelClient.performRequest(deleteRequest) diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/ClusterRerouteFollowerIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/ClusterRerouteFollowerIT.kt index d53fb5f3..77cb164e 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/ClusterRerouteFollowerIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/ClusterRerouteFollowerIT.kt @@ -31,44 +31,35 @@ class ClusterRerouteFollowerIT : MultiClusterRestTestCase() { fun `test replication works after rerouting a shard from one node to another in follower cluster`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - try { - changeTemplate(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) - Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName)) - insertDocToIndex(LEADER, "1", "dummy data 1",leaderIndexName) - - //Querying ES cluster throws random exceptions like ClusterManagerNotDiscovered or ShardsFailed etc, so catching them and retrying - assertBusy ({ - try { - Assertions.assertThat(docs(FOLLOWER, followerIndexName)).contains("dummy data 1") - } catch (ex: Exception) { - Assert.fail("Exception while querying follower cluster. Failing to retry again") - } - }, 1, TimeUnit.MINUTES) - - val nodes = getNodesInCluster(FOLLOWER) - - val primaryNode = getPrimaryNodeForShard(FOLLOWER,followerIndexName, "0") - val unassignedNode = nodes.filter{!it.equals(primaryNode)}.stream().findFirst().get() - rerouteShard(FOLLOWER, "0", followerIndexName, primaryNode, unassignedNode) - - assertBusy ({ - Assertions.assertThat(getPrimaryNodeForShard(FOLLOWER,followerIndexName, "0")).isEqualTo(unassignedNode) - }, 1, TimeUnit.MINUTES) - logger.info("rereouted shard is " + getPrimaryNodeForShard(FOLLOWER,followerIndexName, "0")) - insertDocToIndex(LEADER, "2", "dummy data 2",leaderIndexName) - - assertBusy ({ - try { - Assertions.assertThat(docs(FOLLOWER, followerIndexName)).contains("dummy data 2") - } catch (ex: Exception) { - Assert.fail("Exception while querying follower cluster. Failing to retry again") - } - }, 1, TimeUnit.MINUTES) - } finally { - followerClient.stopReplication(followerIndexName) - } + changeTemplate(LEADER) + createConnectionBetweenClusters(FOLLOWER, LEADER) + val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) + Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName)) + insertDocToIndex(LEADER, "1", "dummy data 1",leaderIndexName) + //Querying ES cluster throws random exceptions like ClusterManagerNotDiscovered or ShardsFailed etc, so catching them and retrying + assertBusy ({ + try { + Assertions.assertThat(docs(FOLLOWER, followerIndexName)).contains("dummy data 1") + } catch (ex: Exception) { + Assert.fail("Exception while querying follower cluster. Failing to retry again") + } + }, 1, TimeUnit.MINUTES) + val nodes = getNodesInCluster(FOLLOWER) + val primaryNode = getPrimaryNodeForShard(FOLLOWER,followerIndexName, "0") + val unassignedNode = nodes.filter{!it.equals(primaryNode)}.stream().findFirst().get() + rerouteShard(FOLLOWER, "0", followerIndexName, primaryNode, unassignedNode) + assertBusy ({ + Assertions.assertThat(getPrimaryNodeForShard(FOLLOWER,followerIndexName, "0")).isEqualTo(unassignedNode) + }, 1, TimeUnit.MINUTES) + logger.info("rereouted shard is " + getPrimaryNodeForShard(FOLLOWER,followerIndexName, "0")) + insertDocToIndex(LEADER, "2", "dummy data 2",leaderIndexName) + assertBusy ({ + try { + Assertions.assertThat(docs(FOLLOWER, followerIndexName)).contains("dummy data 2") + } catch (ex: Exception) { + Assert.fail("Exception while querying follower cluster. Failing to retry again") + } + }, 1, TimeUnit.MINUTES) } } diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/ClusterRerouteLeaderIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/ClusterRerouteLeaderIT.kt index 1da2b9d1..36fce5a5 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/ClusterRerouteLeaderIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/ClusterRerouteLeaderIT.kt @@ -36,43 +36,34 @@ class ClusterRerouteLeaderIT : MultiClusterRestTestCase() { fun `test replication works after rerouting a shard from one node to another in leader cluster`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - try { - changeTemplate(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) - Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName)) - insertDocToIndex(LEADER, "1", "dummy data 1",leaderIndexName) - - //Querying ES cluster throws random exceptions like ClusterManagerNotDiscovered or ShardsFailed etc, so catching them and retrying - assertBusy ({ - try { - Assertions.assertThat(docs(FOLLOWER, followerIndexName)).contains("dummy data 1") - } catch (ex: Exception) { - Assert.fail("Exception while querying follower cluster. Failing to retry again") - } - }, 1, TimeUnit.MINUTES) - - val nodes = getNodesInCluster(LEADER) - val primaryNode = getPrimaryNodeForShard(LEADER,leaderIndexName, "0") - val unassignedNode = nodes.filter{!it.equals(primaryNode)}.stream().findFirst().get() - rerouteShard(LEADER, "0", leaderIndexName, primaryNode, unassignedNode) - - assertBusy ({ - Assertions.assertThat(getPrimaryNodeForShard(LEADER,leaderIndexName, "0")).isEqualTo(unassignedNode) - }, 1, TimeUnit.MINUTES) - - insertDocToIndex(LEADER, "2", "dummy data 2",leaderIndexName) - - assertBusy ({ - try { - Assertions.assertThat(docs(FOLLOWER, followerIndexName)).contains("dummy data 2") - } catch (ex: Exception) { - Assert.fail("Exception while querying follower cluster. Failing to retry again") - } - }, 1, TimeUnit.MINUTES) - } finally { - followerClient.stopReplication(followerIndexName) - } + changeTemplate(LEADER) + createConnectionBetweenClusters(FOLLOWER, LEADER) + val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) + Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName)) + insertDocToIndex(LEADER, "1", "dummy data 1",leaderIndexName) + //Querying ES cluster throws random exceptions like ClusterManagerNotDiscovered or ShardsFailed etc, so catching them and retrying + assertBusy ({ + try { + Assertions.assertThat(docs(FOLLOWER, followerIndexName)).contains("dummy data 1") + } catch (ex: Exception) { + Assert.fail("Exception while querying follower cluster. Failing to retry again") + } + }, 1, TimeUnit.MINUTES) + val nodes = getNodesInCluster(LEADER) + val primaryNode = getPrimaryNodeForShard(LEADER,leaderIndexName, "0") + val unassignedNode = nodes.filter{!it.equals(primaryNode)}.stream().findFirst().get() + rerouteShard(LEADER, "0", leaderIndexName, primaryNode, unassignedNode) + assertBusy ({ + Assertions.assertThat(getPrimaryNodeForShard(LEADER,leaderIndexName, "0")).isEqualTo(unassignedNode) + }, 1, TimeUnit.MINUTES) + insertDocToIndex(LEADER, "2", "dummy data 2",leaderIndexName) + assertBusy ({ + try { + Assertions.assertThat(docs(FOLLOWER, followerIndexName)).contains("dummy data 2") + } catch (ex: Exception) { + Assert.fail("Exception while querying follower cluster. Failing to retry again") + } + }, 1, TimeUnit.MINUTES) } } diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/PauseReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/PauseReplicationIT.kt index d8c7d628..68f9cb65 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/PauseReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/PauseReplicationIT.kt @@ -55,38 +55,28 @@ class PauseReplicationIT: MultiClusterRestTestCase() { val leaderClient = getClientForCluster(LEADER) val followerIndexName = "pause_index_follow_state" createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) - - val myReason = "I want to pause!" - - /* At this point, the follower cluster should be in FOLLOWING state. Next, we pause replication - and verify the same - */ - followerClient.pauseReplication(followerIndexName, myReason) - // Since, we were still in FOLLOWING phase when pause was called, the index - // in follower index should not have been deleted in follower cluster - assertBusy { - assertThat(followerClient.indices() - .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) - .isEqualTo(true) - } - - val statusResp = followerClient.replicationStatus(followerIndexName) - `validate paused status response`(statusResp, myReason) - - var settings = Settings.builder() - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) - .build() - - followerClient.updateReplication( followerIndexName, settings) - followerClient.resumeReplication(followerIndexName) - } finally { - followerClient.stopReplication(followerIndexName) + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) + val myReason = "I want to pause!" + /* At this point, the follower cluster should be in FOLLOWING state. Next, we pause replication + and verify the same + */ + followerClient.pauseReplication(followerIndexName, myReason) + // Since, we were still in FOLLOWING phase when pause was called, the index + // in follower index should not have been deleted in follower cluster + assertBusy { + assertThat(followerClient.indices() + .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) + .isEqualTo(true) } + val statusResp = followerClient.replicationStatus(followerIndexName) + `validate paused status response`(statusResp, myReason) + var settings = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .build() + followerClient.updateReplication( followerIndexName, settings) + followerClient.resumeReplication(followerIndexName) } fun `test pause replication in restoring state with multiple shards`() { @@ -112,7 +102,6 @@ class PauseReplicationIT: MultiClusterRestTestCase() { val leaderClient = getClientForCluster(LEADER) val followerIndexName = "pause_index_restore_state" createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName).settings(settings), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() @@ -122,24 +111,20 @@ class PauseReplicationIT: MultiClusterRestTestCase() { assertThat(leaderClient.indices() .exists(GetIndexRequest(leaderIndexName), RequestOptions.DEFAULT)) } - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), - TimeValue.timeValueSeconds(10), - false) - //Given the size of index, the replication should be in RESTORING phase at this point - assertThatThrownBy { - followerClient.pauseReplication(followerIndexName) - }.isInstanceOf(ResponseException::class.java) - .hasMessageContaining("Index is in restore phase currently for index: ${followerIndexName}") - // wait for the shard tasks to be up as the replication block is added before adding shard replication tasks - // During intermittent test failures, stop replication under finally block executes before this without removing - // replication block (even though next call to _stop replication API can succeed in removing this block). - assertBusy({ - assertTrue(followerClient.getShardReplicationTasks(followerIndexName).isNotEmpty()) - }, 30L, TimeUnit.SECONDS) - } finally { - followerClient.stopReplication(followerIndexName) - } + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), + TimeValue.timeValueSeconds(10), + false) + //Given the size of index, the replication should be in RESTORING phase at this point + assertThatThrownBy { + followerClient.pauseReplication(followerIndexName) + }.isInstanceOf(ResponseException::class.java) + .hasMessageContaining("Index is in restore phase currently for index: ${followerIndexName}") + // wait for the shard tasks to be up as the replication block is added before adding shard replication tasks + // During intermittent test failures, stop replication under finally block executes before this without removing + // replication block (even though next call to _stop replication API can succeed in removing this block). + assertBusy({ + assertTrue(followerClient.getShardReplicationTasks(followerIndexName).isNotEmpty()) + }, 30L, TimeUnit.SECONDS) } fun `test pause without replication in progress`() { @@ -165,12 +150,10 @@ class PauseReplicationIT: MultiClusterRestTestCase() { val leaderClient = getClientForCluster(LEADER) val followerIndexName = "pause_index_with_stop" createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() try { followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) - /* At this point, the follower cluster should be in FOLLOWING state. Next, we pause replication and verify the same */ @@ -194,38 +177,31 @@ class PauseReplicationIT: MultiClusterRestTestCase() { fun `test pause replication when leader cluster is unavailable`() { val followerClient = getClientForCluster(FOLLOWER) val followerIndexName = "pause_index_leader_down" - try { - val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) - assertThat(createIndexResponse.isAcknowledged).isTrue() - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), - waitForRestore = true) - // Need to wait till index blocks appear into state - assertBusy({ - val clusterBlocksResponse = followerClient.lowLevelClient.performRequest(Request("GET", "/_cluster/state/blocks")) - val clusterResponseString = EntityUtils.toString(clusterBlocksResponse.entity) - assertThat(clusterResponseString.contains("cross-cluster-replication")) - .withFailMessage("Cant find replication block after starting replication") - .isTrue() - }, 10, TimeUnit.SECONDS) - - // setting an invalid seed so that leader cluster is unavailable - val settings: Settings = Settings.builder() - .putList("cluster.remote.source.seeds", "127.0.0.1:9305") - .build() - val updateSettingsRequest = ClusterUpdateSettingsRequest() - updateSettingsRequest.persistentSettings(settings) - followerClient.cluster().putSettings(updateSettingsRequest, RequestOptions.DEFAULT) - - followerClient.pauseReplication(followerIndexName) - - val statusResp = followerClient.replicationStatus(followerIndexName) - `validate paused status response`(statusResp) - - } finally { - followerClient.stopReplication(followerIndexName) - } + val leaderClient = getClientForCluster(LEADER) + createConnectionBetweenClusters(FOLLOWER, LEADER) + val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) + assertThat(createIndexResponse.isAcknowledged).isTrue() + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), + waitForRestore = true) + // Need to wait till index blocks appear into state + assertBusy({ + val clusterBlocksResponse = followerClient.lowLevelClient.performRequest(Request("GET", "/_cluster/state/blocks")) + val clusterResponseString = EntityUtils.toString(clusterBlocksResponse.entity) + assertThat(clusterResponseString.contains("cross-cluster-replication")) + .withFailMessage("Cant find replication block after starting replication") + .isTrue() + }, 10, TimeUnit.SECONDS) + + // setting an invalid seed so that leader cluster is unavailable + val settings: Settings = Settings.builder() + .putList("cluster.remote.source.seeds", "127.0.0.1:9305") + .build() + val updateSettingsRequest = ClusterUpdateSettingsRequest() + updateSettingsRequest.persistentSettings(settings) + followerClient.cluster().putSettings(updateSettingsRequest, RequestOptions.DEFAULT) + followerClient.pauseReplication(followerIndexName) + val statusResp = followerClient.replicationStatus(followerIndexName) + `validate paused status response`(statusResp) } fun `test auto pause of index replication when leader index is unavailable`() { @@ -235,41 +211,30 @@ class PauseReplicationIT: MultiClusterRestTestCase() { val leaderIndexName2 = "leader2" val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - try { - createConnectionBetweenClusters(FOLLOWER, LEADER) - var createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName1), RequestOptions.DEFAULT) - assertThat(createIndexResponse.isAcknowledged).isTrue() - createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName2), RequestOptions.DEFAULT) - assertThat(createIndexResponse.isAcknowledged).isTrue() - - // For followerIndexName1 - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName1, - followerIndexName1), waitForRestore = true) - - // For followerIndexName2 - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName2, - followerIndexName2), waitForRestore = true) - - val deleteResponse = leaderClient.indices().delete(DeleteIndexRequest(leaderIndexName1), RequestOptions.DEFAULT) - assertThat(deleteResponse.isAcknowledged) - - // followerIndexName1 -> autopause - assertBusy({ - var statusResp = followerClient.replicationStatus(followerIndexName1) - assertThat(statusResp.containsKey("status")) - assertThat(statusResp.containsKey("reason")) - `validate paused status response due to leader index deleted`(statusResp) - }, 30, TimeUnit.SECONDS) - - // followerIndexName2 -> Syncing state - assertBusy({ - var statusResp = followerClient.replicationStatus(followerIndexName2) - `validate status syncing response`(statusResp) - }, 30, TimeUnit.SECONDS) - - } finally { - followerClient.stopReplication(followerIndexName2) - followerClient.stopReplication(followerIndexName1) - } + createConnectionBetweenClusters(FOLLOWER, LEADER) + var createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName1), RequestOptions.DEFAULT) + assertThat(createIndexResponse.isAcknowledged).isTrue() + createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName2), RequestOptions.DEFAULT) + assertThat(createIndexResponse.isAcknowledged).isTrue() + // For followerIndexName1 + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName1, + followerIndexName1), waitForRestore = true) + // For followerIndexName2 + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName2, + followerIndexName2), waitForRestore = true) + val deleteResponse = leaderClient.indices().delete(DeleteIndexRequest(leaderIndexName1), RequestOptions.DEFAULT) + assertThat(deleteResponse.isAcknowledged) + // followerIndexName1 -> autopause + assertBusy({ + var statusResp = followerClient.replicationStatus(followerIndexName1) + assertThat(statusResp.containsKey("status")) + assertThat(statusResp.containsKey("reason")) + `validate paused status response due to leader index deleted`(statusResp) + }, 30, TimeUnit.SECONDS) + // followerIndexName2 -> Syncing state + assertBusy({ + var statusResp = followerClient.replicationStatus(followerIndexName2) + `validate status syncing response`(statusResp) + }, 30, TimeUnit.SECONDS) } } diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/ReplicationStatusIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/ReplicationStatusIT.kt index 4b4e3762..46a4c2f7 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/ReplicationStatusIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/ReplicationStatusIT.kt @@ -38,15 +38,11 @@ class ReplicationStatusIT: MultiClusterRestTestCase() { createConnectionBetweenClusters(FOLLOWER, LEADER) val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(indexName), RequestOptions.DEFAULT) Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - followerClient.startReplication(StartReplicationRequest("source", indexName, indexName), waitForRestore = true) - assertBusy({ - var statusResp = followerClient.replicationStatus(indexName) - `validate status syncing response`(statusResp) - }, 30, TimeUnit.SECONDS) - } finally { - followerClient.stopReplication(indexName) - } + followerClient.startReplication(StartReplicationRequest("source", indexName, indexName), waitForRestore = true) + assertBusy({ + var statusResp = followerClient.replicationStatus(indexName) + `validate status syncing response`(statusResp) + }, 30, TimeUnit.SECONDS) } fun `test replication status without valid params`() { diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/ResumeReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/ResumeReplicationIT.kt index 6b1aa5b8..2a3b3bae 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/ResumeReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/ResumeReplicationIT.kt @@ -59,26 +59,19 @@ class ResumeReplicationIT: MultiClusterRestTestCase() { fun `test pause and resume replication in following state and empty index`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) - - /* At this point, the follower cluster should be in FOLLOWING state. Next, we pause replication - and verify the same - */ - followerClient.pauseReplication(followerIndexName) - var statusResp = followerClient.replicationStatus(followerIndexName) - `validate paused status response`(statusResp) - statusResp = followerClient.replicationStatus(followerIndexName,false) - `validate aggregated paused status response`(statusResp) - followerClient.resumeReplication(followerIndexName) - } finally { - followerClient.stopReplication(followerIndexName) - } + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) + /* At this point, the follower cluster should be in FOLLOWING state. Next, we pause replication + and verify the same + */ + followerClient.pauseReplication(followerIndexName) + var statusResp = followerClient.replicationStatus(followerIndexName) + `validate paused status response`(statusResp) + statusResp = followerClient.replicationStatus(followerIndexName,false) + `validate aggregated paused status response`(statusResp) + followerClient.resumeReplication(followerIndexName) } @@ -86,127 +79,88 @@ class ResumeReplicationIT: MultiClusterRestTestCase() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) - - assertThatThrownBy { - var statusResp = followerClient.replicationStatus(followerIndexName) - `validate status syncing response`(statusResp) - statusResp = followerClient.replicationStatus(followerIndexName,false) - `validate status syncing aggregated response`(statusResp) - followerClient.resumeReplication(followerIndexName) - statusResp = followerClient.replicationStatus(followerIndexName) - `validate not paused status response`(statusResp) - statusResp = followerClient.replicationStatus(followerIndexName,false) - `validate not paused status aggregated response`(statusResp) - }.isInstanceOf(ResponseException::class.java) - .hasMessageContaining("Replication on Index ${followerIndexName} is already running") - } finally { - followerClient.stopReplication(followerIndexName) - } + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) + assertThatThrownBy { + var statusResp = followerClient.replicationStatus(followerIndexName) + `validate status syncing response`(statusResp) + statusResp = followerClient.replicationStatus(followerIndexName,false) + `validate status syncing aggregated response`(statusResp) + followerClient.resumeReplication(followerIndexName) + statusResp = followerClient.replicationStatus(followerIndexName) + `validate not paused status response`(statusResp) + statusResp = followerClient.replicationStatus(followerIndexName,false) + `validate not paused status aggregated response`(statusResp) + }.isInstanceOf(ResponseException::class.java) + .hasMessageContaining("Replication on Index ${followerIndexName} is already running") } fun `test resume without retention lease`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - var createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) - - - followerClient.pauseReplication(followerIndexName) - - // If we delete the existing index and recreate the index with same name, retention leases should be lost - val deleteIndexResponse = leaderClient.indices().delete(DeleteIndexRequest(leaderIndexName), RequestOptions.DEFAULT) - assertThat(deleteIndexResponse.isAcknowledged).isTrue() - createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) - assertThat(createIndexResponse.isAcknowledged).isTrue() - - assertThatThrownBy { - followerClient.resumeReplication(followerIndexName) - }.isInstanceOf(ResponseException::class.java) - .hasMessageContaining("Retention lease doesn't exist. Replication can't be resumed for $followerIndexName") - } finally { - followerClient.stopReplication(followerIndexName) - } + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) + followerClient.pauseReplication(followerIndexName) + // If we delete the existing index and recreate the index with same name, retention leases should be lost + val deleteIndexResponse = leaderClient.indices().delete(DeleteIndexRequest(leaderIndexName), RequestOptions.DEFAULT) + assertThat(deleteIndexResponse.isAcknowledged).isTrue() + createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) + assertThat(createIndexResponse.isAcknowledged).isTrue() + assertThatThrownBy { + followerClient.resumeReplication(followerIndexName) + }.isInstanceOf(ResponseException::class.java) + .hasMessageContaining("Retention lease doesn't exist. Replication can't be resumed for $followerIndexName") } fun `test pause and resume replication amid leader index close and open`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) - - /* At this point, the follower cluster should be in FOLLOWING state. Next, we pause replication - and verify the same - */ - followerClient.pauseReplication(followerIndexName) - - leaderClient.indices().close(CloseIndexRequest(leaderIndexName), RequestOptions.DEFAULT); - leaderClient.indices().open(OpenIndexRequest(leaderIndexName), RequestOptions.DEFAULT); - - followerClient.resumeReplication(followerIndexName) - - //Update mapping post resume assert - val sourceMap : MutableMap = HashMap() - sourceMap["x"] = "y" - val indexResponse = leaderClient.index(IndexRequest(leaderIndexName).id("2").source(sourceMap), RequestOptions.DEFAULT) - assertThat(indexResponse.result).isIn(DocWriteResponse.Result.CREATED, DocWriteResponse.Result.UPDATED) - assertBusy ({ - Assert.assertEquals( - leaderClient.indices().getMapping(GetMappingsRequest().indices(leaderIndexName), RequestOptions.DEFAULT) - .mappings()[leaderIndexName], - followerClient.indices().getMapping(GetMappingsRequest().indices(followerIndexName), RequestOptions.DEFAULT) - .mappings()[followerIndexName] - ) - }, 60, TimeUnit.SECONDS) - - } finally { - followerClient.stopReplication(followerIndexName) - } + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) + + /* At this point, the follower cluster should be in FOLLOWING state. Next, we pause replication + and verify the same + */ + followerClient.pauseReplication(followerIndexName) + leaderClient.indices().close(CloseIndexRequest(leaderIndexName), RequestOptions.DEFAULT); + leaderClient.indices().open(OpenIndexRequest(leaderIndexName), RequestOptions.DEFAULT); + followerClient.resumeReplication(followerIndexName) + //Update mapping post resume assert + val sourceMap : MutableMap = HashMap() + sourceMap["x"] = "y" + val indexResponse = leaderClient.index(IndexRequest(leaderIndexName).id("2").source(sourceMap), RequestOptions.DEFAULT) + assertThat(indexResponse.result).isIn(DocWriteResponse.Result.CREATED, DocWriteResponse.Result.UPDATED) + assertBusy ({ + Assert.assertEquals( + leaderClient.indices().getMapping(GetMappingsRequest().indices(leaderIndexName), RequestOptions.DEFAULT) + .mappings()[leaderIndexName], + followerClient.indices().getMapping(GetMappingsRequest().indices(followerIndexName), RequestOptions.DEFAULT) + .mappings()[followerIndexName] + ) + }, 60, TimeUnit.SECONDS) } fun `test pause and resume replication amid index close`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) - - /* At this point, the follower cluster should be in FOLLOWING state. Next, we pause replication - and verify the same - */ - followerClient.pauseReplication(followerIndexName) - - leaderClient.indices().close(CloseIndexRequest(leaderIndexName), RequestOptions.DEFAULT); - - assertThatThrownBy { - followerClient.resumeReplication(followerIndexName) - }.isInstanceOf(ResponseException::class.java) - .hasMessageContaining("closed") - } finally { - try { - followerClient.stopReplication(followerIndexName) - } catch (e: Exception) { - // DO nothing - } - } + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) + /* At this point, the follower cluster should be in FOLLOWING state. Next, we pause replication + and verify the same + */ + followerClient.pauseReplication(followerIndexName) + leaderClient.indices().close(CloseIndexRequest(leaderIndexName), RequestOptions.DEFAULT); + assertThatThrownBy { + followerClient.resumeReplication(followerIndexName) + }.isInstanceOf(ResponseException::class.java) + .hasMessageContaining("closed") } fun `test that replication fails to resume when custom analyser is not present in follower`() { @@ -222,24 +176,20 @@ class ResumeReplicationIT: MultiClusterRestTestCase() { val followerClient = getClientForCluster(FOLLOWER) try { Files.copy(synonyms, synonymPath) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - createConnectionBetweenClusters(FOLLOWER, LEADER) followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) followerClient.pauseReplication(followerIndexName) leaderClient.indices().close(CloseIndexRequest(leaderIndexName), RequestOptions.DEFAULT); val settings: Settings = Settings.builder().loadFromStream(synonymsJson, javaClass.getResourceAsStream(synonymsJson), false) .build() - try { leaderClient.indices().putSettings(UpdateSettingsRequest(leaderIndexName).settings(settings), RequestOptions.DEFAULT) } catch (e: Exception) { assumeNoException("Ignored test as analyzer setting could not be added", e) } leaderClient.indices().open(OpenIndexRequest(leaderIndexName), RequestOptions.DEFAULT); - assertThatThrownBy { followerClient.resumeReplication(followerIndexName) }.isInstanceOf(ResponseException::class.java).hasMessageContaining("resource_not_found_exception") @@ -247,11 +197,6 @@ class ResumeReplicationIT: MultiClusterRestTestCase() { if (Files.exists(synonymPath)) { Files.delete(synonymPath) } - try { - followerClient.stopReplication(followerIndexName) - } catch (e: Exception) { - // DO nothing - } } } @@ -271,26 +216,21 @@ class ResumeReplicationIT: MultiClusterRestTestCase() { val followerClient = getClientForCluster(FOLLOWER) try { Files.copy(synonyms, synonymPath) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - createConnectionBetweenClusters(FOLLOWER, LEADER) - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) followerClient.pauseReplication(followerIndexName) leaderClient.indices().close(CloseIndexRequest(leaderIndexName), RequestOptions.DEFAULT); Files.copy(synonyms, followerSynonymPath) val settings: Settings = Settings.builder().loadFromStream(synonymsJson, javaClass.getResourceAsStream(synonymsJson), false) .build() - try { leaderClient.indices().putSettings(UpdateSettingsRequest(leaderIndexName).settings(settings), RequestOptions.DEFAULT) } catch (e: Exception) { assumeNoException("Ignored test as analyzer setting could not be added", e) } leaderClient.indices().open(OpenIndexRequest(leaderIndexName), RequestOptions.DEFAULT); - followerClient.resumeReplication(followerIndexName) var statusResp = followerClient.replicationStatus(followerIndexName) `validate status syncing response`(statusResp) @@ -301,11 +241,6 @@ class ResumeReplicationIT: MultiClusterRestTestCase() { if (Files.exists(followerSynonymPath)) { Files.delete(followerSynonymPath) } - try { - followerClient.stopReplication(followerIndexName) - } catch (e: Exception) { - // DO nothing - } } } @@ -327,7 +262,6 @@ class ResumeReplicationIT: MultiClusterRestTestCase() { try { Files.copy(synonyms, synonymPath) Files.copy(synonyms, followerSynonymPath) - var settings: Settings = Settings.builder().loadFromStream(synonymsJson, javaClass.getResourceAsStream(synonymsJson), false) .build() try { @@ -336,7 +270,6 @@ class ResumeReplicationIT: MultiClusterRestTestCase() { } catch (e: Exception) { assumeNoException("Ignored test as analyzer setting could not be added", e) } - createConnectionBetweenClusters(FOLLOWER, LEADER) val overriddenSettings: Settings = Settings.builder() .put("index.analysis.filter.my_filter.synonyms_path", followerSynonymFilename) @@ -344,7 +277,6 @@ class ResumeReplicationIT: MultiClusterRestTestCase() { followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName, overriddenSettings), waitForRestore = true) followerClient.pauseReplication(followerIndexName) leaderClient.indices().close(CloseIndexRequest(leaderIndexName), RequestOptions.DEFAULT); - Files.copy(synonyms, newSynonymPath) settings = Settings.builder() .put("index.analysis.filter.my_filter.synonyms_path", "synonyms_new.txt") @@ -355,7 +287,6 @@ class ResumeReplicationIT: MultiClusterRestTestCase() { assumeNoException("Ignored test as analyzer setting could not be added", e) } leaderClient.indices().open(OpenIndexRequest(leaderIndexName), RequestOptions.DEFAULT); - followerClient.resumeReplication(followerIndexName) var statusResp = followerClient.replicationStatus(followerIndexName) `validate status syncing response`(statusResp) @@ -369,12 +300,6 @@ class ResumeReplicationIT: MultiClusterRestTestCase() { if (Files.exists(newSynonymPath)) { Files.delete(newSynonymPath) } - try { - followerClient.stopReplication(followerIndexName) - } catch (e: Exception) { - // DO nothing - } } } - } diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityBase.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityBase.kt index 90e31fe6..7f12751b 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityBase.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityBase.kt @@ -241,7 +241,6 @@ abstract class SecurityBase : MultiClusterRestTestCase() { private fun createRoleWithPermissions(indexPattern: String, role: String) { val followerClient = testClusters.get(FOLLOWER) val persistentConnectionRequest = Request("PUT", "_plugins/_security/api/roles/"+role) - val entityAsString = """ { "cluster_permissions": [ @@ -276,7 +275,6 @@ abstract class SecurityBase : MultiClusterRestTestCase() { val entityAsString = """ {"users": ["$user"]} """.trimMargin() - persistentConnectionRequest.entity = StringEntity(entityAsString, ContentType.APPLICATION_JSON) val persistentConnectionResponse = followerClient!!.lowLevelClient.performRequest(persistentConnectionRequest) assertTrue(HttpStatus.SC_CREATED.toLong() == persistentConnectionResponse.statusLine.statusCode.toLong() || @@ -305,7 +303,6 @@ abstract class SecurityBase : MultiClusterRestTestCase() { "password":"$password" } """.trimMargin() - persistentConnectionRequest.entity = StringEntity(entityAsString, ContentType.APPLICATION_JSON) val persistentConnectionResponse = followerClient!!.lowLevelClient.performRequest(persistentConnectionRequest) assertTrue(HttpStatus.SC_CREATED.toLong() == persistentConnectionResponse.statusLine.statusCode.toLong() || diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesIT.kt index cbe46825..97038f3e 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesIT.kt @@ -55,17 +55,14 @@ class SecurityCustomRolesIT: SecurityBase() { val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, - useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) - followerClient.startReplication(startReplicationRequest, - requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password"), waitForRestore = true) - assertBusy { - Assertions.assertThat(followerClient.indices().exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)).isEqualTo(true) - } - } finally { - followerClient.stopReplication(followerIndexName) + var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, + useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) + + followerClient.startReplication(startReplicationRequest, + requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password"), waitForRestore = true) + assertBusy { + Assertions.assertThat(followerClient.indices().exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)).isEqualTo(true) } } @@ -115,26 +112,23 @@ class SecurityCustomRolesIT: SecurityBase() { val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, - useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) - var requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password") - followerClient.startReplication(startReplicationRequest, waitForRestore = true, - requestOptions = requestOptions) - - /* At this point, the follower cluster should be in FOLLOWING state. Next, we pause replication - and verify the same - */ - followerClient.pauseReplication(followerIndexName, - requestOptions = requestOptions) - // Validate paused replication using Status Api - assertBusy { - `validate aggregated paused status response`(followerClient.replicationStatus(followerIndexName, - requestOptions = requestOptions)) - } - } finally { - followerClient.stopReplication(followerIndexName) + var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, + useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) + var requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password") + followerClient.startReplication(startReplicationRequest, waitForRestore = true, + requestOptions = requestOptions) + + /* At this point, the follower cluster should be in FOLLOWING state. Next, we pause replication + and verify the same + */ + followerClient.pauseReplication(followerIndexName, + requestOptions = requestOptions) + + // Validate paused replication using Status Api + assertBusy { + `validate aggregated paused status response`(followerClient.replicationStatus(followerIndexName, + requestOptions = requestOptions)) } } @@ -146,22 +140,18 @@ class SecurityCustomRolesIT: SecurityBase() { val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, - useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) - followerClient.startReplication(startReplicationRequest, waitForRestore = true, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) + var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, + useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) - Assertions.assertThatThrownBy { - followerClient.pauseReplication(followerIndexName, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser2","password")) - }.isInstanceOf(ResponseException::class.java) - .hasMessageContaining("403 Forbidden") + followerClient.startReplication(startReplicationRequest, waitForRestore = true, + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) - } finally { - followerClient.stopReplication(followerIndexName) - } + Assertions.assertThatThrownBy { + followerClient.pauseReplication(followerIndexName, + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser2","password")) + }.isInstanceOf(ResponseException::class.java) + .hasMessageContaining("403 Forbidden") } fun `test for FOLLOWER that STATUS Api works for user with valid permissions`() { @@ -172,19 +162,16 @@ class SecurityCustomRolesIT: SecurityBase() { val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, - useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) - followerClient.startReplication(startReplicationRequest, waitForRestore = true, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) + var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, + useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) - assertBusy { - `validate status syncing response`(followerClient.replicationStatus(followerIndexName, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password"))) - } - } finally { - followerClient.stopReplication(followerIndexName) + followerClient.startReplication(startReplicationRequest, waitForRestore = true, + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) + + assertBusy { + `validate status syncing response`(followerClient.replicationStatus(followerIndexName, + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password"))) } } @@ -196,21 +183,18 @@ class SecurityCustomRolesIT: SecurityBase() { val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, - useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) - followerClient.startReplication(startReplicationRequest, waitForRestore = true, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) + var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, + useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) - Assertions.assertThatThrownBy { - followerClient.replicationStatus(followerIndexName, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser2","password")) - }.isInstanceOf(ResponseException::class.java) - .hasMessageContaining("403 Forbidden") - } finally { - followerClient.stopReplication(followerIndexName) - } + followerClient.startReplication(startReplicationRequest, waitForRestore = true, + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) + + Assertions.assertThatThrownBy { + followerClient.replicationStatus(followerIndexName, + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser2","password")) + }.isInstanceOf(ResponseException::class.java) + .hasMessageContaining("403 Forbidden") } fun `test for FOLLOWER that UPDATE settings works for user with valid permissions`() { @@ -228,89 +212,76 @@ class SecurityCustomRolesIT: SecurityBase() { val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName).settings(settings), RequestOptions.DEFAULT) Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName, - useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")), + + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName, + useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")), + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) + assertBusy { + Assertions.assertThat(followerClient.indices() + .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) + .isEqualTo(true) + } + val getSettingsRequest = GetSettingsRequest() + getSettingsRequest.indices(followerIndexName) + Assert.assertEquals( + "1", + followerClient.indices() + .getSettings(getSettingsRequest, RequestOptions.DEFAULT) + .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] + ) + + settings = Settings.builder() + .put("index.shard.check_on_startup", "checksum") + .build() + followerClient.updateReplication(followerIndexName, settings, requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) - assertBusy { - Assertions.assertThat(followerClient.indices() - .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) - .isEqualTo(true) - } - val getSettingsRequest = GetSettingsRequest() - getSettingsRequest.indices(followerIndexName) + + // Wait for the settings to get updated at follower cluster. + assertBusy ({ Assert.assertEquals( - "1", + "checksum", followerClient.indices() .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] + .indexToSettings[followerIndexName]["index.shard.check_on_startup"] ) - - settings = Settings.builder() - .put("index.shard.check_on_startup", "checksum") - .build() - followerClient.updateReplication(followerIndexName, settings, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) - - // Wait for the settings to get updated at follower cluster. - assertBusy ({ - Assert.assertEquals( - "checksum", - followerClient.indices() - .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName]["index.shard.check_on_startup"] - ) - }, 30L, TimeUnit.SECONDS) - } finally { - followerClient.stopReplication(followerIndexName) - } + }, 30L, TimeUnit.SECONDS) } fun `test for FOLLOWER that UPDATE settings is forbidden for user with invalid permissions`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) val followerIndexName = "follower-index1-settings-invalid-perm" - setMetadataSyncDelay() - createConnectionBetweenClusters(FOLLOWER, LEADER) - var settings = Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) .build() - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName).settings(settings), RequestOptions.DEFAULT) Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName, - useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")), - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password"), waitForRestore = true) - assertBusy { - Assertions.assertThat(followerClient.indices() - .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) - .isEqualTo(true) - } - val getSettingsRequest = GetSettingsRequest() - getSettingsRequest.indices(followerIndexName) - Assert.assertEquals( - "1", - followerClient.indices() - .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] - ) - - settings = Settings.builder() - .put("index.shard.check_on_startup", "checksum") - .build() - - Assertions.assertThatThrownBy { - followerClient.updateReplication(followerIndexName, settings, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser2","password")) - }.isInstanceOf(ResponseException::class.java) - .hasMessageContaining("403 Forbidden") - } finally { - followerClient.stopReplication(followerIndexName) + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName, + useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")), + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password"), waitForRestore = true) + assertBusy { + Assertions.assertThat(followerClient.indices() + .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) + .isEqualTo(true) } + val getSettingsRequest = GetSettingsRequest() + getSettingsRequest.indices(followerIndexName) + Assert.assertEquals( + "1", + followerClient.indices() + .getSettings(getSettingsRequest, RequestOptions.DEFAULT) + .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] + ) + settings = Settings.builder() + .put("index.shard.check_on_startup", "checksum") + .build() + Assertions.assertThatThrownBy { + followerClient.updateReplication(followerIndexName, settings, + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser2","password")) + }.isInstanceOf(ResponseException::class.java) + .hasMessageContaining("403 Forbidden") } fun `test for FOLLOWER that AutoFollow works for user with valid permissions`() { @@ -323,12 +294,10 @@ class SecurityCustomRolesIT: SecurityBase() { val leaderIndexName = createRandomIndex(indexPrefix, leaderClient) var leaderIndexNameNew = "" createConnectionBetweenClusters(FOLLOWER, LEADER, connectionAlias) - try { followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, indexPattern, useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms"), requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) - // Verify that existing index matching the pattern are replicated. assertBusy ({ Assertions.assertThat(followerClient.indices() @@ -336,7 +305,6 @@ class SecurityCustomRolesIT: SecurityBase() { .isEqualTo(true) }, 30, TimeUnit.SECONDS) Assertions.assertThat(getAutoFollowTasks(FOLLOWER).size).isEqualTo(1) - leaderIndexNameNew = createRandomIndex(indexPrefix, leaderClient) // Verify that newly created index on leader which match the pattern are also replicated. assertBusy ({ @@ -346,8 +314,6 @@ class SecurityCustomRolesIT: SecurityBase() { }, 60, TimeUnit.SECONDS) } finally { followerClient.deleteAutoFollowPattern(connectionAlias, indexPatternName) - followerClient.stopReplication(leaderIndexName, false) - followerClient.stopReplication(leaderIndexNameNew) } } @@ -357,7 +323,6 @@ class SecurityCustomRolesIT: SecurityBase() { val indexPattern = "follower-index1*" val indexPatternName = "test_pattern" createConnectionBetweenClusters(FOLLOWER, LEADER, connectionAlias) - Assertions.assertThatThrownBy { followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, indexPattern, useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleNoPerms"), @@ -387,16 +352,13 @@ class SecurityCustomRolesIT: SecurityBase() { val leaderClient = getClientForCluster(LEADER) val followerIndexName = "follower-index1" createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() try { var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) - followerClient.startReplication(startReplicationRequest, waitForRestore = true, requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) - insertDocToIndex(LEADER, "1", "dummy data 1",leaderIndexName) //Querying ES cluster throws random exceptions like ClusterManagerNotDiscovered or ShardsFailed etc, so catching them and retrying assertBusy ({ @@ -406,7 +368,6 @@ class SecurityCustomRolesIT: SecurityBase() { Assert.fail("Exception while querying follower cluster. Failing to retry again") } }, 1, TimeUnit.MINUTES) - assertBusy { `validate status syncing response`(followerClient.replicationStatus(followerIndexName, requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password"))) @@ -421,7 +382,6 @@ class SecurityCustomRolesIT: SecurityBase() { }, 100, TimeUnit.SECONDS) } finally { updateRole(followerIndexName,"followerRoleValidPerms", true) - followerClient.stopReplication(followerIndexName) } } diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesLeaderIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesLeaderIT.kt index 8955d22f..9332166b 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesLeaderIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesLeaderIT.kt @@ -42,13 +42,10 @@ class SecurityCustomRolesLeaderIT: SecurityBase() { val leaderClient = getClientForCluster(LEADER) val followerIndexName = "follower-index1" createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() - var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, useRoles = UseRoles(leaderClusterRole = "leaderRoleNoPerms",followerClusterRole = "followerRoleValidPerms")) - Assertions.assertThatThrownBy { followerClient.startReplication(startReplicationRequest, requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser6","password")) } .isInstanceOf(ResponseException::class.java) @@ -61,16 +58,13 @@ class SecurityCustomRolesLeaderIT: SecurityBase() { val leaderClient = getClientForCluster(LEADER) val followerIndexName = "follower-index1" createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() try { var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) - followerClient.startReplication(startReplicationRequest, waitForRestore = true, requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) - insertDocToIndex(LEADER, "1", "dummy data 1",leaderIndexName) //Querying ES cluster throws random exceptions like ClusterManagerNotDiscovered or ShardsFailed etc, so catching them and retrying assertBusy ({ @@ -80,23 +74,18 @@ class SecurityCustomRolesLeaderIT: SecurityBase() { Assert.fail("Exception while querying follower cluster. Failing to retry again") } }, 1, TimeUnit.MINUTES) - assertBusy { `validate status syncing response`(followerClient.replicationStatus(followerIndexName, requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password"))) } - updateRole(followerIndexName,"leaderRoleValidPerms", false) insertDocToIndex(LEADER, "2", "dummy data 2",leaderIndexName) - assertBusy ({ validatePausedState(followerClient.replicationStatus(followerIndexName, requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password"))) }, 100, TimeUnit.SECONDS) - } finally { updateRole(followerIndexName,"leaderRoleValidPerms", true) - followerClient.stopReplication(followerIndexName) } } @@ -105,17 +94,14 @@ class SecurityCustomRolesLeaderIT: SecurityBase() { val leaderClient = getClientForCluster(LEADER) val followerIndexName = "follower-index1" createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() try { var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) - updateFileChunkPermissions("","leaderRoleValidPerms", false) followerClient.startReplication(startReplicationRequest, requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) - assertBusy ({ validateFailedState(followerClient.replicationStatus(followerIndexName, requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password"))) @@ -125,7 +111,6 @@ class SecurityCustomRolesLeaderIT: SecurityBase() { Assert.assertNull(ex) } finally { updateFileChunkPermissions("","leaderRoleValidPerms", true) - followerClient.stopReplication(followerIndexName) } } diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityDlsFlsIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityDlsFlsIT.kt index 191471ae..82e7465d 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityDlsFlsIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityDlsFlsIT.kt @@ -42,16 +42,12 @@ class SecurityDlsFlsIT: SecurityBase() { fun `test for FOLLOWER that START replication is forbidden for user with DLS or FLS enabled`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - val followerIndexName = "follower-index1-dlsfls-enabled" createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() - var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerDlsRole")) - Assertions.assertThatThrownBy { followerClient.startReplication(startReplicationRequest, requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser3","password")) } .isInstanceOf(ResponseException::class.java) @@ -61,7 +57,6 @@ class SecurityDlsFlsIT: SecurityBase() { fun `test for FOLLOWER that STOP replication is forbidden for user with DLS or FLS enabled`() { val followerClient = getClientForCluster(FOLLOWER) - Assertions.assertThatThrownBy { followerClient.stopReplication("follower-index1-stop-forbidden", requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser3","password")) @@ -75,25 +70,18 @@ class SecurityDlsFlsIT: SecurityBase() { val leaderClient = getClientForCluster(LEADER) val followerIndexName = "follower-index1-pause-forbidden" createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, - useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) - - followerClient.startReplication(startReplicationRequest, waitForRestore = true, - requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) - - Assertions.assertThatThrownBy { - followerClient.pauseReplication(followerIndexName, - requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser3","password")) - }.isInstanceOf(ResponseException::class.java) - .hasMessageContaining(DLS_FLS_EXCEPTION_MESSAGE) - .hasMessageContaining("403 Forbidden") - } finally { - followerClient.stopReplication(followerIndexName) - } + var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, + useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) + followerClient.startReplication(startReplicationRequest, waitForRestore = true, + requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) + Assertions.assertThatThrownBy { + followerClient.pauseReplication(followerIndexName, + requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser3","password")) + }.isInstanceOf(ResponseException::class.java) + .hasMessageContaining(DLS_FLS_EXCEPTION_MESSAGE) + .hasMessageContaining("403 Forbidden") } fun `test for FOLLOWER that STATUS Api is forbidden for user with DLS or FLS enabled`() { @@ -101,88 +89,67 @@ class SecurityDlsFlsIT: SecurityBase() { val leaderClient = getClientForCluster(LEADER) val followerIndexName = "follower-index1-status-forbidden" createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, - useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) - - followerClient.startReplication(startReplicationRequest, waitForRestore = true, - requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) - - Assertions.assertThatThrownBy { - followerClient.replicationStatus(followerIndexName, - requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser3","password")) - }.isInstanceOf(ResponseException::class.java) - .hasMessageContaining(DLS_FLS_EXCEPTION_MESSAGE) - .hasMessageContaining("403 Forbidden") - } finally { - followerClient.stopReplication(followerIndexName) - } + var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, + useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) + followerClient.startReplication(startReplicationRequest, waitForRestore = true, + requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) + Assertions.assertThatThrownBy { + followerClient.replicationStatus(followerIndexName, + requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser3","password")) + }.isInstanceOf(ResponseException::class.java) + .hasMessageContaining(DLS_FLS_EXCEPTION_MESSAGE) + .hasMessageContaining("403 Forbidden") } fun `test for FOLLOWER that UPDATE settings is forbidden for user with DLS or FLS enabled`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) val followerIndexName = "follower-index1-update-forbidden" - setMetadataSyncDelay() - createConnectionBetweenClusters(FOLLOWER, LEADER) - var settings = Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) .build() - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName).settings(settings), RequestOptions.DEFAULT) Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName, - useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")), - requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password"), waitForRestore = true) - assertBusy { - Assertions.assertThat(followerClient.indices() - .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) - .isEqualTo(true) - } - val getSettingsRequest = GetSettingsRequest() - getSettingsRequest.indices(followerIndexName) - Assert.assertEquals( - "1", - followerClient.indices() - .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] - ) - - settings = Settings.builder() - .put("index.shard.check_on_startup", "checksum") - .build() - - Assertions.assertThatThrownBy { - followerClient.updateReplication(followerIndexName, settings, - requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser3","password")) - }.isInstanceOf(ResponseException::class.java) - .hasMessageContaining(DLS_FLS_EXCEPTION_MESSAGE) - .hasMessageContaining("403 Forbidden") - } finally { - followerClient.stopReplication(followerIndexName) + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName, + useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")), + requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password"), waitForRestore = true) + assertBusy { + Assertions.assertThat(followerClient.indices() + .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) + .isEqualTo(true) } + val getSettingsRequest = GetSettingsRequest() + getSettingsRequest.indices(followerIndexName) + Assert.assertEquals( + "1", + followerClient.indices() + .getSettings(getSettingsRequest, RequestOptions.DEFAULT) + .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] + ) + settings = Settings.builder() + .put("index.shard.check_on_startup", "checksum") + .build() + Assertions.assertThatThrownBy { + followerClient.updateReplication(followerIndexName, settings, + requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser3","password")) + }.isInstanceOf(ResponseException::class.java) + .hasMessageContaining(DLS_FLS_EXCEPTION_MESSAGE) + .hasMessageContaining("403 Forbidden") } fun `test for FOLLOWER that START replication is forbidden for user with FLS enabled`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - val followerIndexName = "follower-index1-start-forbidden" createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() - var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerFlsRole")) - Assertions.assertThatThrownBy { followerClient.startReplication(startReplicationRequest, requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser4","password")) } .isInstanceOf(ResponseException::class.java) @@ -193,16 +160,12 @@ class SecurityDlsFlsIT: SecurityBase() { fun `test for FOLLOWER that START replication is forbidden for user with Field Masking enabled`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - val followerIndexName = "follower-index1-start-only-fls" createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() - var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerFieldMaskRole")) - Assertions.assertThatThrownBy { followerClient.startReplication(startReplicationRequest, requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser5","password")) } .isInstanceOf(ResponseException::class.java) @@ -213,36 +176,27 @@ class SecurityDlsFlsIT: SecurityBase() { fun `test for FOLLOWER that START replication works for user with Field Masking enabled on a different index pattern`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - val followerIndexName = "follower-index1-allow-start" createConnectionBetweenClusters(FOLLOWER, LEADER) - - try { - val createIndexResponse = - leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) - Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() - - var startReplicationRequest = StartReplicationRequest( - "source", leaderIndexName, followerIndexName, - useRoles = UseRoles( - leaderClusterRole = "leaderRoleValidPerms", - followerClusterRole = "followerFieldMaskRole2" - ) - ) - followerClient.startReplication( - startReplicationRequest, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser7", "password"), - waitForRestore = true + val createIndexResponse = + leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) + Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() + var startReplicationRequest = StartReplicationRequest( + "source", leaderIndexName, followerIndexName, + useRoles = UseRoles( + leaderClusterRole = "leaderRoleValidPerms", + followerClusterRole = "followerFieldMaskRole2" ) - - OpenSearchTestCase.assertBusy { - Assertions.assertThat( - followerClient.indices().exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT) - ).isEqualTo(true) - } - } - finally { - followerClient.stopReplication(followerIndexName) + ) + followerClient.startReplication( + startReplicationRequest, + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser7", "password"), + waitForRestore = true + ) + OpenSearchTestCase.assertBusy { + Assertions.assertThat( + followerClient.indices().exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT) + ).isEqualTo(true) } } } \ No newline at end of file diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt index 73b28a15..599a2f19 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt @@ -73,6 +73,7 @@ import org.opensearch.replication.followerStats import org.opensearch.replication.leaderStats import org.opensearch.replication.updateReplicationStartBlockSetting import java.nio.file.Files +import java.util.* import java.util.concurrent.TimeUnit import org.opensearch.bootstrap.BootstrapInfo @@ -97,52 +98,39 @@ class StartReplicationIT: MultiClusterRestTestCase() { fun `test start replication in following state and empty index`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) - assertBusy { - assertThat(followerClient.indices().exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)).isEqualTo(true) - } - } finally { - followerClient.stopReplication(followerIndexName) + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) + assertBusy { + assertThat(followerClient.indices().exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)).isEqualTo(true) } } fun `test start replication with settings`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() val settings = Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 3) .build() - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName, settings = settings), waitForRestore = true) - assertBusy { - assertThat(followerClient.indices().exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)).isEqualTo(true) - } - - val getSettingsRequest = GetSettingsRequest() - getSettingsRequest.indices(followerIndexName) - getSettingsRequest.includeDefaults(true) - assertBusy ({ - Assert.assertEquals( - "3", - followerClient.indices() - .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] - ) - }, 15, TimeUnit.SECONDS) - } finally { - followerClient.stopReplication(followerIndexName) + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName, settings = settings), waitForRestore = true) + assertBusy { + assertThat(followerClient.indices().exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)).isEqualTo(true) } + val getSettingsRequest = GetSettingsRequest() + getSettingsRequest.indices(followerIndexName) + getSettingsRequest.includeDefaults(true) + assertBusy ({ + Assert.assertEquals( + "3", + followerClient.indices() + .getSettings(getSettingsRequest, RequestOptions.DEFAULT) + .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] + ) + }, 15, TimeUnit.SECONDS) } @@ -152,22 +140,18 @@ class StartReplicationIT: MultiClusterRestTestCase() { createConnectionBetweenClusters(FOLLOWER, LEADER) val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) - assertThat(followerClient.indices().exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)).isEqualTo(true) - leaderClient.lowLevelClient.performRequest(Request("POST", "/" + leaderIndexName + "/_close")) - assertBusy ({ - try { - assertThat(followerClient.replicationStatus(followerIndexName)).containsKey("status") - var statusResp = followerClient.replicationStatus(followerIndexName) - `validate paused status on closed index`(statusResp) - } catch (e : Exception) { - Assert.fail() - } - },30, TimeUnit.SECONDS) - } finally { - followerClient.stopReplication(followerIndexName) - } + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) + assertThat(followerClient.indices().exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)).isEqualTo(true) + leaderClient.lowLevelClient.performRequest(Request("POST", "/" + leaderIndexName + "/_close")) + assertBusy ({ + try { + assertThat(followerClient.replicationStatus(followerIndexName)).containsKey("status") + var statusResp = followerClient.replicationStatus(followerIndexName) + `validate paused status on closed index`(statusResp) + } catch (e : Exception) { + Assert.fail() + } + },30, TimeUnit.SECONDS) } @@ -177,52 +161,38 @@ class StartReplicationIT: MultiClusterRestTestCase() { createConnectionBetweenClusters(FOLLOWER, LEADER) val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) - followerClient.pauseReplication(followerIndexName) - leaderClient.lowLevelClient.performRequest(Request("POST", "/" + leaderIndexName + "/_close")) - leaderClient.lowLevelClient.performRequest(Request("POST", "/" + leaderIndexName + "/_open")) - followerClient.resumeReplication(followerIndexName) - var statusResp = followerClient.replicationStatus(followerIndexName) - `validate not paused status response`(statusResp) - - } finally { - followerClient.stopReplication(followerIndexName) - } + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) + followerClient.pauseReplication(followerIndexName) + leaderClient.lowLevelClient.performRequest(Request("POST", "/" + leaderIndexName + "/_close")) + leaderClient.lowLevelClient.performRequest(Request("POST", "/" + leaderIndexName + "/_open")) + followerClient.resumeReplication(followerIndexName) + var statusResp = followerClient.replicationStatus(followerIndexName) + `validate not paused status response`(statusResp) } fun `test start replication fails when replication has already been started for the same index`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - - try { - val createIndexResponse = - leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) - assertThat(createIndexResponse.isAcknowledged).isTrue() - followerClient.startReplication( - StartReplicationRequest("source", leaderIndexName, followerIndexName), - waitForRestore = true - ) - assertThatThrownBy { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName)) - }.isInstanceOf(ResponseException::class.java).hasMessageContaining( - "Cant use same index again for replication." + - " Delete the index:$followerIndexName" - ) - } - finally { - followerClient.stopReplication(followerIndexName) - } + val createIndexResponse = + leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) + assertThat(createIndexResponse.isAcknowledged).isTrue() + followerClient.startReplication( + StartReplicationRequest("source", leaderIndexName, followerIndexName), + waitForRestore = true + ) + assertThatThrownBy { + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName)) + }.isInstanceOf(ResponseException::class.java).hasMessageContaining( + "Cant use same index again for replication." + + " Delete the index:$followerIndexName" + ) } fun `test start replication fails when remote cluster alias does not exist`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() assertThatThrownBy { @@ -234,9 +204,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { fun `test start replication fails when index does not exist`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() assertThatThrownBy { @@ -248,9 +216,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { fun `test start replication fails when the follower cluster is write blocked or metadata blocked`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() addClusterMetadataBlock(FOLLOWER, "true") @@ -265,122 +231,99 @@ class StartReplicationIT: MultiClusterRestTestCase() { fun `test that follower index has same mapping as leader index`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) - assertBusy { - assertThat(followerClient.indices() - .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) - .isEqualTo(true) - } + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) + assertBusy { + assertThat(followerClient.indices() + .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) + .isEqualTo(true) + } + Assert.assertEquals( + leaderClient.indices().getMapping(GetMappingsRequest().indices(leaderIndexName), RequestOptions.DEFAULT) + .mappings()[leaderIndexName], + followerClient.indices().getMapping(GetMappingsRequest().indices(followerIndexName), RequestOptions.DEFAULT) + .mappings()[followerIndexName] + ) + // test that new mapping created on leader is also propagated to follower + val putMappingRequest = PutMappingRequest(leaderIndexName) + putMappingRequest.source("{\"properties\":{\"name\":{\"type\":\"keyword\"}}}", XContentType.JSON) + leaderClient.indices().putMapping(putMappingRequest, RequestOptions.DEFAULT) + val sourceMap = mapOf("name" to randomAlphaOfLength(5)) + leaderClient.index(IndexRequest(leaderIndexName).id("1").source(sourceMap), RequestOptions.DEFAULT) + val leaderMappings = leaderClient.indices().getMapping(GetMappingsRequest().indices(leaderIndexName), RequestOptions.DEFAULT) + .mappings()[leaderIndexName] + assertBusy({ Assert.assertEquals( - leaderClient.indices().getMapping(GetMappingsRequest().indices(leaderIndexName), RequestOptions.DEFAULT) - .mappings()[leaderIndexName], - followerClient.indices().getMapping(GetMappingsRequest().indices(followerIndexName), RequestOptions.DEFAULT) - .mappings()[followerIndexName] + leaderMappings, + followerClient.indices().getMapping(GetMappingsRequest().indices(followerIndexName), RequestOptions.DEFAULT) + .mappings()[followerIndexName] ) - // test that new mapping created on leader is also propagated to follower - val putMappingRequest = PutMappingRequest(leaderIndexName) - putMappingRequest.source("{\"properties\":{\"name\":{\"type\":\"keyword\"}}}", XContentType.JSON) - leaderClient.indices().putMapping(putMappingRequest, RequestOptions.DEFAULT) - val sourceMap = mapOf("name" to randomAlphaOfLength(5)) - leaderClient.index(IndexRequest(leaderIndexName).id("1").source(sourceMap), RequestOptions.DEFAULT) - val leaderMappings = leaderClient.indices().getMapping(GetMappingsRequest().indices(leaderIndexName), RequestOptions.DEFAULT) - .mappings()[leaderIndexName] - assertBusy({ - Assert.assertEquals( - leaderMappings, - followerClient.indices().getMapping(GetMappingsRequest().indices(followerIndexName), RequestOptions.DEFAULT) - .mappings()[followerIndexName] - ) - }, 30L, TimeUnit.SECONDS) - - } finally { - followerClient.stopReplication(followerIndexName) - } + }, 30L, TimeUnit.SECONDS) } fun `test that index settings are getting replicated`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - val settings = Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) .build() - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName).settings(settings), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), - waitForRestore = true) - assertBusy { - assertThat(followerClient.indices() - .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) - .isEqualTo(true) - } - val getSettingsRequest = GetSettingsRequest() - getSettingsRequest.indices(followerIndexName) - getSettingsRequest.names(IndexMetadata.SETTING_NUMBER_OF_REPLICAS) - assertBusy({ - Assert.assertEquals( - "0", - followerClient.indices() - .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] - ) - }, 30L, TimeUnit.SECONDS) - } finally { - followerClient.stopReplication(followerIndexName) + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), + waitForRestore = true) + assertBusy { + assertThat(followerClient.indices() + .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) + .isEqualTo(true) } + val getSettingsRequest = GetSettingsRequest() + getSettingsRequest.indices(followerIndexName) + getSettingsRequest.names(IndexMetadata.SETTING_NUMBER_OF_REPLICAS) + assertBusy({ + Assert.assertEquals( + "0", + followerClient.indices() + .getSettings(getSettingsRequest, RequestOptions.DEFAULT) + .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] + ) + }, 30L, TimeUnit.SECONDS) } fun `test that aliases settings are getting replicated`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName) .alias(Alias("leaderAlias").filter("{\"term\":{\"year\":2016}}").routing("1")) , RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), - waitForRestore = true) - assertBusy { - assertThat(followerClient.indices() - .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) - .isEqualTo(true) - } - assertBusy({ - Assert.assertEquals( - leaderClient.indices().getAlias(GetAliasesRequest().indices(leaderIndexName), - RequestOptions.DEFAULT).aliases[leaderIndexName], - followerClient.indices().getAlias(GetAliasesRequest().indices(followerIndexName), - RequestOptions.DEFAULT).aliases[followerIndexName] - ) - - }, 30L, TimeUnit.SECONDS) - } finally { - followerClient.stopReplication(followerIndexName) + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), + waitForRestore = true) + assertBusy { + assertThat(followerClient.indices() + .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) + .isEqualTo(true) } + assertBusy({ + Assert.assertEquals( + leaderClient.indices().getAlias(GetAliasesRequest().indices(leaderIndexName), + RequestOptions.DEFAULT).aliases[leaderIndexName], + followerClient.indices().getAlias(GetAliasesRequest().indices(followerIndexName), + RequestOptions.DEFAULT).aliases[followerIndexName] + ) + + }, 30L, TimeUnit.SECONDS) } fun `test that replication cannot be started on leader alias directly`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER, "source") - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName).alias(Alias("leader_alias")), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - try { followerClient.startReplication(StartReplicationRequest("source", "leader_alias", followerIndexName)) fail("Expected startReplication to fail") @@ -394,89 +337,67 @@ class StartReplicationIT: MultiClusterRestTestCase() { fun `test that translog settings are set on leader and not on follower`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), - waitForRestore = true) - assertBusy { - assertThat(followerClient.indices() - .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) - .isEqualTo(true) - assertThat(followerClient.indices() - .getSettings(GetSettingsRequest().indices(followerIndexName), RequestOptions.DEFAULT) - .getSetting(followerIndexName, - REPLICATION_INDEX_TRANSLOG_PRUNING_ENABLED_SETTING.key) - .isNullOrEmpty()) - } - - assertThat(leaderClient.indices() - .getSettings(GetSettingsRequest().indices(leaderIndexName), RequestOptions.DEFAULT) - .getSetting(leaderIndexName, - REPLICATION_INDEX_TRANSLOG_PRUNING_ENABLED_SETTING.key) == "true") - - } finally { - followerClient.stopReplication(followerIndexName) + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), + waitForRestore = true) + assertBusy { + assertThat(followerClient.indices() + .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) + .isEqualTo(true) + assertThat(followerClient.indices() + .getSettings(GetSettingsRequest().indices(followerIndexName), RequestOptions.DEFAULT) + .getSetting(followerIndexName, + REPLICATION_INDEX_TRANSLOG_PRUNING_ENABLED_SETTING.key) + .isNullOrEmpty()) } + assertThat(leaderClient.indices() + .getSettings(GetSettingsRequest().indices(leaderIndexName), RequestOptions.DEFAULT) + .getSetting(leaderIndexName, + REPLICATION_INDEX_TRANSLOG_PRUNING_ENABLED_SETTING.key) == "true") } fun `test that translog settings are set on leader`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), - waitForRestore = true) - - val leaderSettings = leaderClient.indices() - .getSettings(GetSettingsRequest().indices(leaderIndexName), RequestOptions.DEFAULT) - assertThat(leaderSettings.getSetting(leaderIndexName, - REPLICATION_INDEX_TRANSLOG_PRUNING_ENABLED_SETTING.key) == "true") - assertThat(leaderSettings.getSetting(leaderIndexName, - IndexSettings.INDEX_TRANSLOG_GENERATION_THRESHOLD_SIZE_SETTING.key) == "32mb") - - } finally { - followerClient.stopReplication(followerIndexName) - } + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), + waitForRestore = true) + val leaderSettings = leaderClient.indices() + .getSettings(GetSettingsRequest().indices(leaderIndexName), RequestOptions.DEFAULT) + assertThat(leaderSettings.getSetting(leaderIndexName, + REPLICATION_INDEX_TRANSLOG_PRUNING_ENABLED_SETTING.key) == "true") + assertThat(leaderSettings.getSetting(leaderIndexName, + IndexSettings.INDEX_TRANSLOG_GENERATION_THRESHOLD_SIZE_SETTING.key) == "32mb") } fun `test that replication continues after removing translog settings based on retention lease`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), - waitForRestore = true) - assertBusy { - assertThat(followerClient.indices() - .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) - .isEqualTo(true) - } - // Turn-off the settings and index doc - val settingsBuilder = Settings.builder() - .put(REPLICATION_INDEX_TRANSLOG_PRUNING_ENABLED_SETTING.key, false) - val settingsUpdateResponse = leaderClient.indices().putSettings(UpdateSettingsRequest(leaderIndexName) - .settings(settingsBuilder.build()), RequestOptions.DEFAULT) - Assert.assertEquals(settingsUpdateResponse.isAcknowledged, true) - val sourceMap = mapOf("name" to randomAlphaOfLength(5)) - leaderClient.index(IndexRequest(leaderIndexName).id("2").source(sourceMap), RequestOptions.DEFAULT) - assertBusy({ - followerClient.get(GetRequest(followerIndexName).id("2"), RequestOptions.DEFAULT).isExists - }, 30L, TimeUnit.SECONDS) - } finally { - followerClient.stopReplication(followerIndexName) + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), + waitForRestore = true) + assertBusy { + assertThat(followerClient.indices() + .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) + .isEqualTo(true) } + // Turn-off the settings and index doc + val settingsBuilder = Settings.builder() + .put(REPLICATION_INDEX_TRANSLOG_PRUNING_ENABLED_SETTING.key, false) + val settingsUpdateResponse = leaderClient.indices().putSettings(UpdateSettingsRequest(leaderIndexName) + .settings(settingsBuilder.build()), RequestOptions.DEFAULT) + Assert.assertEquals(settingsUpdateResponse.isAcknowledged, true) + val sourceMap = mapOf("name" to randomAlphaOfLength(5)) + leaderClient.index(IndexRequest(leaderIndexName).id("2").source(sourceMap), RequestOptions.DEFAULT) + assertBusy({ + followerClient.get(GetRequest(followerIndexName).id("2"), RequestOptions.DEFAULT).isExists + }, 30L, TimeUnit.SECONDS) } private fun addClusterMetadataBlock(clusterName: String, blockValue: String) { @@ -501,203 +422,163 @@ class StartReplicationIT: MultiClusterRestTestCase() { fun `test that dynamic index settings and alias are getting replicated `() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - setMetadataSyncDelay() - createConnectionBetweenClusters(FOLLOWER, LEADER) - var settings = Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) .build() - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName).settings(settings), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), - waitForRestore = true) - assertBusy { - assertThat(followerClient.indices() - .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) - .isEqualTo(true) - } - - settings = Settings.builder() - .build() - - followerClient.updateReplication( followerIndexName, settings) - - val getSettingsRequest = GetSettingsRequest() - getSettingsRequest.indices(followerIndexName) - getSettingsRequest.includeDefaults(true) - Assert.assertEquals( - "0", - followerClient.indices() - .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] - ) - - settings = Settings.builder() - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 2) - .put("routing.allocation.enable", "none") - .build() - - leaderClient.indices().putSettings(UpdateSettingsRequest(leaderIndexName).settings(settings), RequestOptions.DEFAULT) - - var indicesAliasesRequest = IndicesAliasesRequest() - var aliasAction = IndicesAliasesRequest.AliasActions.add() - .index(leaderIndexName) - .alias("alias1").filter("{\"term\":{\"year\":2016}}").routing("1") - indicesAliasesRequest.addAliasAction(aliasAction) - leaderClient.indices().updateAliases(indicesAliasesRequest, RequestOptions.DEFAULT) - - TimeUnit.SECONDS.sleep(SLEEP_TIME_BETWEEN_SYNC) - getSettingsRequest.indices(followerIndexName) - // Leader setting is copied - assertBusy({ - Assert.assertEquals( - "2", - followerClient.indices() + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), + waitForRestore = true) + assertBusy { + assertThat(followerClient.indices() + .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) + .isEqualTo(true) + } + settings = Settings.builder() + .build() + followerClient.updateReplication( followerIndexName, settings) + val getSettingsRequest = GetSettingsRequest() + getSettingsRequest.indices(followerIndexName) + getSettingsRequest.includeDefaults(true) + Assert.assertEquals( + "0", + followerClient.indices() .getSettings(getSettingsRequest, RequestOptions.DEFAULT) .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] - ) - assertEqualAliases() - }, 30L, TimeUnit.SECONDS) - - - // Case 2 : Blocklisted setting are not copied - Assert.assertNull(followerClient.indices() + ) + settings = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 2) + .put("routing.allocation.enable", "none") + .build() + leaderClient.indices().putSettings(UpdateSettingsRequest(leaderIndexName).settings(settings), RequestOptions.DEFAULT) + var indicesAliasesRequest = IndicesAliasesRequest() + var aliasAction = IndicesAliasesRequest.AliasActions.add() + .index(leaderIndexName) + .alias("alias1").filter("{\"term\":{\"year\":2016}}").routing("1") + indicesAliasesRequest.addAliasAction(aliasAction) + leaderClient.indices().updateAliases(indicesAliasesRequest, RequestOptions.DEFAULT) + TimeUnit.SECONDS.sleep(SLEEP_TIME_BETWEEN_SYNC) + getSettingsRequest.indices(followerIndexName) + // Leader setting is copied + assertBusy({ + Assert.assertEquals( + "2", + followerClient.indices() .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName].get("index.routing.allocation.enable")) - - //Alias test case 2: Update existing alias - aliasAction = IndicesAliasesRequest.AliasActions.add() - .index(leaderIndexName) - .routing("2") - .alias("alias1") - .writeIndex(true) - .isHidden(false) - indicesAliasesRequest.addAliasAction(aliasAction) - leaderClient.indices().updateAliases(indicesAliasesRequest, RequestOptions.DEFAULT) - - //Use Update API - settings = Settings.builder() - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 3) - .put("index.routing.allocation.enable", "none") - .put("index.search.idle.after", "10s") - .build() - - followerClient.updateReplication( followerIndexName, settings) - TimeUnit.SECONDS.sleep(SLEEP_TIME_BETWEEN_SYNC) - - // Case 3 : Updated Settings take higher priority. Blocklisted settins shouldn't matter for that - assertBusy({ - Assert.assertEquals( - "3", - followerClient.indices() - .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] - ) - - Assert.assertEquals( - "10s", - followerClient.indices() - .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName]["index.search.idle.after"] - ) - - Assert.assertEquals( - "none", - followerClient.indices() - .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName]["index.routing.allocation.enable"] - ) - - assertEqualAliases() - }, 30L, TimeUnit.SECONDS) - - //Clear the settings - settings = Settings.builder() - .build() - followerClient.updateReplication( followerIndexName, settings) - - //Alias test case 3: Delete one alias and add another alias - aliasAction = IndicesAliasesRequest.AliasActions.remove() - .index(leaderIndexName) - .alias("alias1") - indicesAliasesRequest.addAliasAction(aliasAction + .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] ) - leaderClient.indices().updateAliases(indicesAliasesRequest, RequestOptions.DEFAULT) - var aliasAction2 = IndicesAliasesRequest.AliasActions.add() - .index(leaderIndexName) - .routing("12") - .alias("alias2") - .indexRouting("indexRouting") - indicesAliasesRequest.addAliasAction(aliasAction2) - - TimeUnit.SECONDS.sleep(SLEEP_TIME_BETWEEN_SYNC) - - assertBusy({ - Assert.assertEquals( - null, - followerClient.indices() - .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName]["index.search.idle.after"] - ) - assertEqualAliases() - }, 30L, TimeUnit.SECONDS) - - } finally { - followerClient.stopReplication(followerIndexName) - } - + assertEqualAliases() + }, 30L, TimeUnit.SECONDS) + // Case 2 : Blocklisted setting are not copied + Assert.assertNull(followerClient.indices() + .getSettings(getSettingsRequest, RequestOptions.DEFAULT) + .indexToSettings[followerIndexName].get("index.routing.allocation.enable")) + //Alias test case 2: Update existing alias + aliasAction = IndicesAliasesRequest.AliasActions.add() + .index(leaderIndexName) + .routing("2") + .alias("alias1") + .writeIndex(true) + .isHidden(false) + indicesAliasesRequest.addAliasAction(aliasAction) + leaderClient.indices().updateAliases(indicesAliasesRequest, RequestOptions.DEFAULT) + //Use Update API + settings = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 3) + .put("index.routing.allocation.enable", "none") + .put("index.search.idle.after", "10s") + .build() + followerClient.updateReplication( followerIndexName, settings) + TimeUnit.SECONDS.sleep(SLEEP_TIME_BETWEEN_SYNC) + // Case 3 : Updated Settings take higher priority. Blocklisted settins shouldn't matter for that + assertBusy({ + Assert.assertEquals( + "3", + followerClient.indices() + .getSettings(getSettingsRequest, RequestOptions.DEFAULT) + .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] + ) + Assert.assertEquals( + "10s", + followerClient.indices() + .getSettings(getSettingsRequest, RequestOptions.DEFAULT) + .indexToSettings[followerIndexName]["index.search.idle.after"] + ) + Assert.assertEquals( + "none", + followerClient.indices() + .getSettings(getSettingsRequest, RequestOptions.DEFAULT) + .indexToSettings[followerIndexName]["index.routing.allocation.enable"] + ) + assertEqualAliases() + }, 30L, TimeUnit.SECONDS) + //Clear the settings + settings = Settings.builder() + .build() + followerClient.updateReplication( followerIndexName, settings) + //Alias test case 3: Delete one alias and add another alias + aliasAction = IndicesAliasesRequest.AliasActions.remove() + .index(leaderIndexName) + .alias("alias1") + indicesAliasesRequest.addAliasAction(aliasAction + ) + leaderClient.indices().updateAliases(indicesAliasesRequest, RequestOptions.DEFAULT) + var aliasAction2 = IndicesAliasesRequest.AliasActions.add() + .index(leaderIndexName) + .routing("12") + .alias("alias2") + .indexRouting("indexRouting") + indicesAliasesRequest.addAliasAction(aliasAction2) + TimeUnit.SECONDS.sleep(SLEEP_TIME_BETWEEN_SYNC) + assertBusy({ + Assert.assertEquals( + null, + followerClient.indices() + .getSettings(getSettingsRequest, RequestOptions.DEFAULT) + .indexToSettings[followerIndexName]["index.search.idle.after"] + ) + assertEqualAliases() + }, 30L, TimeUnit.SECONDS) } fun `test that static index settings are getting replicated `() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - setMetadataSyncDelay() - createConnectionBetweenClusters(FOLLOWER, LEADER) - var settings = Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) .build() - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName).settings(settings), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), - waitForRestore = true) - assertBusy { - assertThat(followerClient.indices() - .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) - .isEqualTo(true) - } - val getSettingsRequest = GetSettingsRequest() - getSettingsRequest.indices(followerIndexName) - Assert.assertEquals( - "1", - followerClient.indices() - .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] - ) - - settings = Settings.builder() - .put("index.shard.check_on_startup", "checksum") - .build() - followerClient.updateReplication(followerIndexName, settings) - - TimeUnit.SECONDS.sleep(SLEEP_TIME_BETWEEN_SYNC) - Assert.assertEquals( - "checksum", - followerClient.indices() - .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName]["index.shard.check_on_startup"] - ) - } finally { - followerClient.stopReplication(followerIndexName) + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), + waitForRestore = true) + assertBusy { + assertThat(followerClient.indices() + .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) + .isEqualTo(true) } + val getSettingsRequest = GetSettingsRequest() + getSettingsRequest.indices(followerIndexName) + Assert.assertEquals( + "1", + followerClient.indices() + .getSettings(getSettingsRequest, RequestOptions.DEFAULT) + .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] + ) + settings = Settings.builder() + .put("index.shard.check_on_startup", "checksum") + .build() + followerClient.updateReplication(followerIndexName, settings) + TimeUnit.SECONDS.sleep(SLEEP_TIME_BETWEEN_SYNC) + Assert.assertEquals( + "checksum", + followerClient.indices() + .getSettings(getSettingsRequest, RequestOptions.DEFAULT) + .indexToSettings[followerIndexName]["index.shard.check_on_startup"] + ) } fun `test that replication fails to start when custom analyser is not present in follower`() { @@ -715,7 +596,6 @@ class StartReplicationIT: MultiClusterRestTestCase() { val settings: Settings = Settings.builder().loadFromStream(synonymsJson, javaClass.getResourceAsStream(synonymsJson), false) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) .build() - val leaderClient = getClientForCluster(LEADER) val followerClient = getClientForCluster(FOLLOWER) try { @@ -725,9 +605,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { } catch (e: Exception) { assumeNoException("Ignored test as analyzer setting could not be added", e) } - createConnectionBetweenClusters(FOLLOWER, LEADER) - assertThatThrownBy { followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName)) }.isInstanceOf(ResponseException::class.java).hasMessageContaining("resource_not_found_exception") @@ -752,11 +630,9 @@ class StartReplicationIT: MultiClusterRestTestCase() { try { Files.copy(synonyms, leaderSynonymPath) Files.copy(synonyms, followerSynonymPath) - val settings: Settings = Settings.builder().loadFromStream(synonymsJson, javaClass.getResourceAsStream(synonymsJson), false) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) .build() - val leaderClient = getClientForCluster(LEADER) val followerClient = getClientForCluster(FOLLOWER) try { @@ -766,17 +642,11 @@ class StartReplicationIT: MultiClusterRestTestCase() { } catch (e: Exception) { assumeNoException("Ignored test as analyzer setting could not be added", e) } - createConnectionBetweenClusters(FOLLOWER, LEADER) - - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), - waitForRestore = true) - assertBusy { - assertThat(followerClient.indices().exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)).isEqualTo(true) - } - } finally { - followerClient.stopReplication(followerIndexName) + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), + waitForRestore = true) + assertBusy { + assertThat(followerClient.indices().exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)).isEqualTo(true) } } finally { if (Files.exists(leaderSynonymPath)) { @@ -803,11 +673,9 @@ class StartReplicationIT: MultiClusterRestTestCase() { try { Files.copy(synonyms, leaderSynonymPath) Files.copy(synonyms, followerSynonymPath) - val settings: Settings = Settings.builder().loadFromStream(synonymsJson, javaClass.getResourceAsStream(synonymsJson), false) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) .build() - val leaderClient = getClientForCluster(LEADER) val followerClient = getClientForCluster(FOLLOWER) try { @@ -817,20 +685,14 @@ class StartReplicationIT: MultiClusterRestTestCase() { } catch (e: Exception) { assumeNoException("Ignored test as analyzer setting could not be added", e) } - createConnectionBetweenClusters(FOLLOWER, LEADER) - - try { - val overriddenSettings: Settings = Settings.builder() - .put("index.analysis.filter.my_filter.synonyms_path", synonymFollowerFilename) - .build() - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName, overriddenSettings), - waitForRestore = true) - assertBusy { - assertThat(followerClient.indices().exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)).isEqualTo(true) - } - } finally { - followerClient.stopReplication(followerIndexName) + val overriddenSettings: Settings = Settings.builder() + .put("index.analysis.filter.my_filter.synonyms_path", synonymFollowerFilename) + .build() + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName, overriddenSettings), + waitForRestore = true) + assertBusy { + assertThat(followerClient.indices().exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)).isEqualTo(true) } } finally { if (Files.exists(leaderSynonymPath)) { @@ -845,73 +707,58 @@ class StartReplicationIT: MultiClusterRestTestCase() { fun `test that follower index cannot be deleted after starting replication`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), - waitForRestore = true) - assertBusy { - assertThat(followerClient.indices().exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)).isEqualTo(true) - } - // Need to wait till index blocks appear into state - assertBusy { - val clusterBlocksResponse = followerClient.lowLevelClient.performRequest(Request("GET", "/_cluster/state/blocks")) - val clusterResponseString = EntityUtils.toString(clusterBlocksResponse.entity) - assertThat(clusterResponseString.contains("cross-cluster-replication")) - .withFailMessage("Cant find replication block afer starting replication") - .isTrue() - } - // Delete index - assertThatThrownBy { - followerClient.indices().delete(DeleteIndexRequest(followerIndexName), RequestOptions.DEFAULT) - }.isInstanceOf(OpenSearchStatusException::class.java).hasMessageContaining("cluster_block_exception") - // Close index - assertThatThrownBy { - followerClient.indices().close(CloseIndexRequest(followerIndexName), RequestOptions.DEFAULT) - }.isInstanceOf(OpenSearchStatusException::class.java).hasMessageContaining("cluster_block_exception") - // Index document - assertThatThrownBy { - val sourceMap = mapOf("name" to randomAlphaOfLength(5)) - followerClient.index(IndexRequest(followerIndexName).id("1").source(sourceMap), RequestOptions.DEFAULT) - }.isInstanceOf(OpenSearchStatusException::class.java).hasMessageContaining("cluster_block_exception") - } finally { - followerClient.stopReplication(followerIndexName) + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), + waitForRestore = true) + assertBusy { + assertThat(followerClient.indices().exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)).isEqualTo(true) + } + // Need to wait till index blocks appear into state + assertBusy { + val clusterBlocksResponse = followerClient.lowLevelClient.performRequest(Request("GET", "/_cluster/state/blocks")) + val clusterResponseString = EntityUtils.toString(clusterBlocksResponse.entity) + assertThat(clusterResponseString.contains("cross-cluster-replication")) + .withFailMessage("Cant find replication block afer starting replication") + .isTrue() } + // Delete index + assertThatThrownBy { + followerClient.indices().delete(DeleteIndexRequest(followerIndexName), RequestOptions.DEFAULT) + }.isInstanceOf(OpenSearchStatusException::class.java).hasMessageContaining("cluster_block_exception") + // Close index + assertThatThrownBy { + followerClient.indices().close(CloseIndexRequest(followerIndexName), RequestOptions.DEFAULT) + }.isInstanceOf(OpenSearchStatusException::class.java).hasMessageContaining("cluster_block_exception") + // Index document + assertThatThrownBy { + val sourceMap = mapOf("name" to randomAlphaOfLength(5)) + followerClient.index(IndexRequest(followerIndexName).id("1").source(sourceMap), RequestOptions.DEFAULT) + }.isInstanceOf(OpenSearchStatusException::class.java).hasMessageContaining("cluster_block_exception") } fun `test that replication gets paused if the leader index is deleted`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), - waitForRestore = true) - assertBusy { - assertThat(followerClient.indices().exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)).isEqualTo(true) - } - assertBusy { - var statusResp = followerClient.replicationStatus(followerIndexName) - `validate status syncing response`(statusResp) - } - val deleteIndexResponse = leaderClient.indices().delete(DeleteIndexRequest(leaderIndexName), RequestOptions.DEFAULT) - assertThat(deleteIndexResponse.isAcknowledged).isTrue() - - assertBusy({ - var statusResp = followerClient.replicationStatus(followerIndexName) - `validate paused status response due to leader index deleted`(statusResp) - }, 15, TimeUnit.SECONDS) - } finally { - followerClient.stopReplication(followerIndexName) + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), + waitForRestore = true) + assertBusy { + assertThat(followerClient.indices().exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)).isEqualTo(true) } + assertBusy { + var statusResp = followerClient.replicationStatus(followerIndexName) + `validate status syncing response`(statusResp) + } + val deleteIndexResponse = leaderClient.indices().delete(DeleteIndexRequest(leaderIndexName), RequestOptions.DEFAULT) + assertThat(deleteIndexResponse.isAcknowledged).isTrue() + assertBusy({ + var statusResp = followerClient.replicationStatus(followerIndexName) + `validate paused status response due to leader index deleted`(statusResp) + }, 15, TimeUnit.SECONDS) } fun `test forcemerge on leader during replication bootstrap`() { @@ -923,7 +770,6 @@ class StartReplicationIT: MultiClusterRestTestCase() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName).settings(settings), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() @@ -933,28 +779,22 @@ class StartReplicationIT: MultiClusterRestTestCase() { assertThat(leaderClient.indices() .exists(GetIndexRequest(leaderIndexName), RequestOptions.DEFAULT)) } - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), - TimeValue.timeValueSeconds(10), - false) - //Given the size of index, the replication should be in RESTORING phase at this point - leaderClient.indices().forcemerge(ForceMergeRequest(leaderIndexName), RequestOptions.DEFAULT) - - assertBusy { - var statusResp = followerClient.replicationStatus(followerIndexName) - `validate status syncing response`(statusResp) - } - TimeUnit.SECONDS.sleep(30) - - assertBusy ({ - Assert.assertEquals(leaderClient.count(CountRequest(leaderIndexName), RequestOptions.DEFAULT).toString(), - followerClient.count(CountRequest(followerIndexName), RequestOptions.DEFAULT).toString()) - }, - 30, TimeUnit.SECONDS - ) - } finally { - followerClient.stopReplication(followerIndexName) + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), + TimeValue.timeValueSeconds(10), + false) + //Given the size of index, the replication should be in RESTORING phase at this point + leaderClient.indices().forcemerge(ForceMergeRequest(leaderIndexName), RequestOptions.DEFAULT) + assertBusy { + var statusResp = followerClient.replicationStatus(followerIndexName) + `validate status syncing response`(statusResp) } + TimeUnit.SECONDS.sleep(30) + assertBusy ({ + Assert.assertEquals(leaderClient.count(CountRequest(leaderIndexName), RequestOptions.DEFAULT).toString(), + followerClient.count(CountRequest(followerIndexName), RequestOptions.DEFAULT).toString()) + }, + 30, TimeUnit.SECONDS + ) } fun `test that snapshot on leader does not affect replication during bootstrap`() { @@ -971,15 +811,11 @@ class StartReplicationIT: MultiClusterRestTestCase() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) createConnectionBetweenClusters(FOLLOWER, LEADER) - val repoPath = PathUtils.get(buildDir, repoPath) - val putRepositoryRequest = PutRepositoryRequest("my-repo") .type(FsRepository.TYPE) .settings("{\"location\": \"$repoPath\"}", XContentType.JSON) - leaderClient.snapshot().createRepository(putRepositoryRequest, RequestOptions.DEFAULT) - val createIndexResponse = leaderClient.indices().create( CreateIndexRequest(leaderIndexName).settings(settings), RequestOptions.DEFAULT @@ -993,153 +829,122 @@ class StartReplicationIT: MultiClusterRestTestCase() { .exists(GetIndexRequest(leaderIndexName), RequestOptions.DEFAULT) ) } - try { - followerClient.startReplication( - StartReplicationRequest("source", leaderIndexName, followerIndexName), - TimeValue.timeValueSeconds(10), - false - ) - //Given the size of index, the replication should be in RESTORING phase at this point - leaderClient.snapshot().create(CreateSnapshotRequest("my-repo", "snapshot_1").indices(leaderIndexName), RequestOptions.DEFAULT) - - assertBusy({ - var statusResp = followerClient.replicationStatus(followerIndexName) - `validate status syncing response`(statusResp) - }, 30, TimeUnit.SECONDS - ) - assertBusy({ - Assert.assertEquals( - leaderClient.count(CountRequest(leaderIndexName), RequestOptions.DEFAULT).toString(), - followerClient.count(CountRequest(followerIndexName), RequestOptions.DEFAULT).toString() - )}, - 30, TimeUnit.SECONDS - ) - } finally { - followerClient.stopReplication(followerIndexName) - } + followerClient.startReplication( + StartReplicationRequest("source", leaderIndexName, followerIndexName), + TimeValue.timeValueSeconds(10), + false + ) + //Given the size of index, the replication should be in RESTORING phase at this point + leaderClient.snapshot().create(CreateSnapshotRequest("my-repo", "snapshot_1").indices(leaderIndexName), RequestOptions.DEFAULT) + assertBusy({ + var statusResp = followerClient.replicationStatus(followerIndexName) + `validate status syncing response`(statusResp) + }, 30, TimeUnit.SECONDS + ) + assertBusy({ + Assert.assertEquals( + leaderClient.count(CountRequest(leaderIndexName), RequestOptions.DEFAULT).toString(), + followerClient.count(CountRequest(followerIndexName), RequestOptions.DEFAULT).toString() + )}, + 30, TimeUnit.SECONDS + ) } fun `test leader stats`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - val settings = Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 2) .build() - val createIndexResponse = leaderClient.indices().create( CreateIndexRequest(leaderIndexName).settings(settings), RequestOptions.DEFAULT ) assertThat(createIndexResponse.isAcknowledged).isTrue() - - try { - followerClient.startReplication( - StartReplicationRequest("source", leaderIndexName, followerIndexName), - TimeValue.timeValueSeconds(10), - true - ) - - val docCount = 50 - - for (i in 1..docCount) { - val sourceMap = mapOf("name" to randomAlphaOfLength(5)) - leaderClient.index(IndexRequest(leaderIndexName).id(i.toString()).source(sourceMap), RequestOptions.DEFAULT) - } - - // Have to wait until the new operations are available to read at the leader cluster - assertBusy({ - val stats = leaderClient.leaderStats() - assertThat(stats.size).isEqualTo(9) - assertThat(stats.getValue("num_replicated_indices").toString()).isEqualTo("1") - assertThat(stats.getValue("operations_read").toString()).isEqualTo(docCount.toString()) - assertThat(stats.getValue("operations_read_lucene").toString()).isEqualTo("0") - assertThat(stats.getValue("operations_read_translog").toString()).isEqualTo(docCount.toString()) - assertThat(stats.containsKey("index_stats")) - }, 60L, TimeUnit.SECONDS) - - } finally { - followerClient.stopReplication(followerIndexName) + followerClient.startReplication( + StartReplicationRequest("source", leaderIndexName, followerIndexName), + TimeValue.timeValueSeconds(10), + true + ) + val docCount = 50 + for (i in 1..docCount) { + val sourceMap = mapOf("name" to randomAlphaOfLength(5)) + leaderClient.index(IndexRequest(leaderIndexName).id(i.toString()).source(sourceMap), RequestOptions.DEFAULT) } + // Have to wait until the new operations are available to read at the leader cluster + assertBusy({ + val stats = leaderClient.leaderStats() + assertThat(stats.size).isEqualTo(9) + assertThat(stats.getValue("num_replicated_indices").toString()).isEqualTo("1") + assertThat(stats.getValue("operations_read").toString()).isEqualTo(docCount.toString()) + assertThat(stats.getValue("operations_read_lucene").toString()).isEqualTo("0") + assertThat(stats.getValue("operations_read_translog").toString()).isEqualTo(docCount.toString()) + assertThat(stats.containsKey("index_stats")) + }, 60L, TimeUnit.SECONDS) } @AwaitsFix(bugUrl = "https://github.com/opensearch-project/cross-cluster-replication/issues/176") fun `test follower stats`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - - val followerIndex2 = "follower_index_2" - val followerIndex3 = "follower_index_3" - + val leaderIndexName2 = randomAlphaOfLength(10).toLowerCase(Locale.ROOT)+"leader" + val followerIndexName2 = randomAlphaOfLength(10).toLowerCase(Locale.ROOT)+"follower" + val leaderIndexName3 = randomAlphaOfLength(10).toLowerCase(Locale.ROOT)+"leader" + val followerIndexName3 = randomAlphaOfLength(10).toLowerCase(Locale.ROOT)+"follower" +// val followerIndex2 = "follower_index_2" +// val followerIndex3 = "follower_index_3" createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create( CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT ) assertThat(createIndexResponse.isAcknowledged).isTrue() - - try { - followerClient.startReplication( - StartReplicationRequest("source", leaderIndexName, followerIndexName), - TimeValue.timeValueSeconds(10), - true - ) - followerClient.startReplication( - StartReplicationRequest("source", leaderIndexName, followerIndex2), - TimeValue.timeValueSeconds(10), - true - ) - followerClient.startReplication( - StartReplicationRequest("source", leaderIndexName, followerIndex3), - TimeValue.timeValueSeconds(10), - true - ) - val docCount = 50 - for (i in 1..docCount) { - val sourceMap = mapOf("name" to randomAlphaOfLength(5)) - leaderClient.index(IndexRequest(leaderIndexName).id(i.toString()).source(sourceMap), RequestOptions.DEFAULT) - } - - followerClient.pauseReplication(followerIndex2) - followerClient.stopReplication(followerIndex3) - - - val stats = followerClient.followerStats() - assertThat(stats.getValue("num_syncing_indices").toString()).isEqualTo("1") - assertThat(stats.getValue("num_paused_indices").toString()).isEqualTo("1") - assertThat(stats.getValue("num_failed_indices").toString()).isEqualTo("0") - assertThat(stats.getValue("num_shard_tasks").toString()).isEqualTo("1") - assertThat(stats.getValue("operations_written").toString()).isEqualTo("50") - assertThat(stats.getValue("operations_read").toString()).isEqualTo("50") - assertThat(stats.getValue("failed_read_requests").toString()).isEqualTo("0") - assertThat(stats.getValue("failed_write_requests").toString()).isEqualTo("0") - assertThat(stats.getValue("follower_checkpoint").toString()).isEqualTo((docCount-1).toString()) - assertThat(stats.containsKey("index_stats")) - assertThat(stats.size).isEqualTo(16) - - } finally { - followerClient.stopReplication(followerIndexName) - followerClient.stopReplication(followerIndex2) + followerClient.startReplication( + StartReplicationRequest("source", leaderIndexName, followerIndexName), + TimeValue.timeValueSeconds(10), + true + ) + followerClient.startReplication( + StartReplicationRequest("source", leaderIndexName2, followerIndexName2), + TimeValue.timeValueSeconds(10), + true + ) + followerClient.startReplication( + StartReplicationRequest("source", leaderIndexName3, followerIndexName3), + TimeValue.timeValueSeconds(10), + true + ) + val docCount = 50 + for (i in 1..docCount) { + val sourceMap = mapOf("name" to randomAlphaOfLength(5)) + leaderClient.index(IndexRequest(leaderIndexName).id(i.toString()).source(sourceMap), RequestOptions.DEFAULT) } + followerClient.pauseReplication(followerIndexName2) + val stats = followerClient.followerStats() + assertThat(stats.getValue("num_syncing_indices").toString()).isEqualTo("1") + assertThat(stats.getValue("num_paused_indices").toString()).isEqualTo("1") + assertThat(stats.getValue("num_failed_indices").toString()).isEqualTo("0") + assertThat(stats.getValue("num_shard_tasks").toString()).isEqualTo("1") + assertThat(stats.getValue("operations_written").toString()).isEqualTo("50") + assertThat(stats.getValue("operations_read").toString()).isEqualTo("50") + assertThat(stats.getValue("failed_read_requests").toString()).isEqualTo("0") + assertThat(stats.getValue("failed_write_requests").toString()).isEqualTo("0") + assertThat(stats.getValue("follower_checkpoint").toString()).isEqualTo((docCount-1).toString()) + assertThat(stats.containsKey("index_stats")) + assertThat(stats.size).isEqualTo(16) } fun `test that replication cannot be started on invalid indexName`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName).alias(Alias("leaderAlias")), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - assertValidationFailure(followerClient, "leaderIndex", followerIndexName, "Value leaderIndex must be lowercase") assertValidationFailure(followerClient, "leaderindex", "followerIndex", "Value followerIndex must be lowercase") - assertValidationFailure(followerClient, "test*", followerIndexName, "Value test* must not contain the following characters") assertValidationFailure(followerClient, "test#", followerIndexName, @@ -1150,10 +955,8 @@ class StartReplicationIT: MultiClusterRestTestCase() { "Value . must not be '.' or '..'") assertValidationFailure(followerClient, "..", followerIndexName, "Value .. must not be '.' or '..'") - assertValidationFailure(followerClient, "_leader", followerIndexName, "Value _leader must not start with '_' or '-' or '+'") - assertValidationFailure(followerClient, "-leader", followerIndexName, "Value -leader must not start with '_' or '-' or '+'") assertValidationFailure(followerClient, "+leader", followerIndexName, @@ -1167,45 +970,33 @@ class StartReplicationIT: MultiClusterRestTestCase() { fun `test that replication is not started when start block is set`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) val createIndexResponse = leaderClient.indices().create( CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT ) assertThat(createIndexResponse.isAcknowledged).isTrue() - // Setting to add replication start block followerClient.updateReplicationStartBlockSetting(true) - assertThatThrownBy { followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) } .isInstanceOf(ResponseException::class.java) .hasMessageContaining("[FORBIDDEN] Replication START block is set") - // Remove replication start block and start replication followerClient.updateReplicationStartBlockSetting(false) - - try { followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) - } finally { - followerClient.stopReplication(followerIndexName) - } } fun `test start replication invalid settings`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() val settings = Settings.builder() .put("index.data_path", "/random-path/invalid-setting") .build() - try { followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName, settings = settings)) } catch (e: ResponseException) { @@ -1217,7 +1008,6 @@ class StartReplicationIT: MultiClusterRestTestCase() { fun `test that replication is not started when all primary shards are not in active state`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) // Exclude leader cluster nodes to stop assignment for the new shards excludeAllClusterNodes(LEADER) @@ -1233,13 +1023,11 @@ class StartReplicationIT: MultiClusterRestTestCase() { assertBusy { assertThat(leaderClient.indices().exists(GetIndexRequest(leaderIndexName), RequestOptions.DEFAULT)).isEqualTo(true) } - // start repilcation should fail as the shards are not active on the leader cluster assertThatThrownBy { followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore = true) } .isInstanceOf(ResponseException::class.java) .hasMessageContaining("Primary shards in the Index[source:${leaderIndexName}] are not active") - } fun `test that wait_for_active_shards setting is set on leader and not on follower`() { @@ -1436,10 +1224,8 @@ class StartReplicationIT: MultiClusterRestTestCase() { var getAliasesRequest = GetAliasesRequest().indices(followerIndexName) var aliasRespone = followerClient.indices().getAlias(getAliasesRequest, RequestOptions.DEFAULT) var followerAliases = aliasRespone.aliases.get(followerIndexName) - aliasRespone = leaderClient.indices().getAlias(GetAliasesRequest().indices(leaderIndexName), RequestOptions.DEFAULT) var leaderAliases = aliasRespone.aliases.get(leaderIndexName) - Assert.assertEquals(followerAliases, leaderAliases) } } diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/StopReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/StopReplicationIT.kt index 214b9205..f2f0a09e 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/StopReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/StopReplicationIT.kt @@ -229,7 +229,6 @@ class StopReplicationIT: MultiClusterRestTestCase() { .withFailMessage("Cant find replication block after starting replication") .isTrue() }, 10, TimeUnit.SECONDS) - // Remove leader cluster from settings val settings: Settings = Settings.builder() .putNull("cluster.remote.source.seeds") @@ -237,7 +236,6 @@ class StopReplicationIT: MultiClusterRestTestCase() { val updateSettingsRequest = ClusterUpdateSettingsRequest() updateSettingsRequest.persistentSettings(settings) followerClient.cluster().putSettings(updateSettingsRequest, RequestOptions.DEFAULT) - followerClient.stopReplication(followerIndexName) val sourceMap = mapOf("name" to randomAlphaOfLength(5)) followerClient.index(IndexRequest(followerIndexName).id("2").source(sourceMap), RequestOptions.DEFAULT) @@ -252,71 +250,54 @@ class StopReplicationIT: MultiClusterRestTestCase() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) createConnectionBetweenClusters(FOLLOWER, LEADER, "source") - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() val snapshotSuffix = Random().nextInt(1000).toString() - - try { - followerClient.startReplication( - StartReplicationRequest("source", leaderIndexName, followerIndexName), - TimeValue.timeValueSeconds(10), - true - ) - - assertBusy({ - var statusResp = followerClient.replicationStatus(followerIndexName) - `validate status syncing response`(statusResp) - assertThat(followerClient.getShardReplicationTasks(followerIndexName)).isNotEmpty() - }, 60, TimeUnit.SECONDS) - - // Trigger snapshot on the follower cluster - val createSnapshotRequest = CreateSnapshotRequest(TestCluster.FS_SNAPSHOT_REPO, "test-$snapshotSuffix") - createSnapshotRequest.waitForCompletion(true) - followerClient.snapshot().create(createSnapshotRequest, RequestOptions.DEFAULT) - - assertBusy { - var snapshotStatusResponse = followerClient.snapshot().status(SnapshotsStatusRequest(TestCluster.FS_SNAPSHOT_REPO, - arrayOf("test-$snapshotSuffix")), RequestOptions.DEFAULT) - for (snapshotStatus in snapshotStatusResponse.snapshots) { - Assert.assertEquals(SnapshotsInProgress.State.SUCCESS, snapshotStatus.state) - } - } - - // Restore follower index on leader cluster - val restoreSnapshotRequest = RestoreSnapshotRequest(TestCluster.FS_SNAPSHOT_REPO, "test-$snapshotSuffix") - restoreSnapshotRequest.indices(followerIndexName) - restoreSnapshotRequest.waitForCompletion(true) - restoreSnapshotRequest.renamePattern("(.+)") - restoreSnapshotRequest.renameReplacement("restored-\$1") - leaderClient.snapshot().restore(restoreSnapshotRequest, RequestOptions.DEFAULT) - - assertBusy { - assertThat(leaderClient.indices().exists(GetIndexRequest("restored-$followerIndexName"), RequestOptions.DEFAULT)).isEqualTo(true) + followerClient.startReplication( + StartReplicationRequest("source", leaderIndexName, followerIndexName), + TimeValue.timeValueSeconds(10), + true + ) + assertBusy({ + var statusResp = followerClient.replicationStatus(followerIndexName) + `validate status syncing response`(statusResp) + assertThat(followerClient.getShardReplicationTasks(followerIndexName)).isNotEmpty() + }, 60, TimeUnit.SECONDS) + // Trigger snapshot on the follower cluster + val createSnapshotRequest = CreateSnapshotRequest(TestCluster.FS_SNAPSHOT_REPO, "test-$snapshotSuffix") + createSnapshotRequest.waitForCompletion(true) + followerClient.snapshot().create(createSnapshotRequest, RequestOptions.DEFAULT) + assertBusy { + var snapshotStatusResponse = followerClient.snapshot().status(SnapshotsStatusRequest(TestCluster.FS_SNAPSHOT_REPO, + arrayOf("test-$snapshotSuffix")), RequestOptions.DEFAULT) + for (snapshotStatus in snapshotStatusResponse.snapshots) { + Assert.assertEquals(SnapshotsInProgress.State.SUCCESS, snapshotStatus.state) } - - // Invoke stop on the new leader cluster index - assertThatThrownBy { leaderClient.stopReplication("restored-$followerIndexName") } - .isInstanceOf(ResponseException::class.java) - .hasMessageContaining("Metadata for restored-$followerIndexName doesn't exist") - - // Start replication on the new leader index - followerClient.startReplication( - StartReplicationRequest("source", "restored-$followerIndexName", "restored-$followerIndexName"), - TimeValue.timeValueSeconds(10), - true, true - ) - - assertBusy({ - var statusResp = followerClient.replicationStatus("restored-$followerIndexName") - `validate status syncing response`(statusResp) - assertThat(followerClient.getShardReplicationTasks("restored-$followerIndexName")).isNotEmpty() - }, 60, TimeUnit.SECONDS) - - } finally { - followerClient.stopReplication("restored-$followerIndexName") - followerClient.stopReplication(followerIndexName) } - + // Restore follower index on leader cluster + val restoreSnapshotRequest = RestoreSnapshotRequest(TestCluster.FS_SNAPSHOT_REPO, "test-$snapshotSuffix") + restoreSnapshotRequest.indices(followerIndexName) + restoreSnapshotRequest.waitForCompletion(true) + restoreSnapshotRequest.renamePattern("(.+)") + restoreSnapshotRequest.renameReplacement("restored-\$1") + leaderClient.snapshot().restore(restoreSnapshotRequest, RequestOptions.DEFAULT) + assertBusy { + assertThat(leaderClient.indices().exists(GetIndexRequest("restored-$followerIndexName"), RequestOptions.DEFAULT)).isEqualTo(true) + } + // Invoke stop on the new leader cluster index + assertThatThrownBy { leaderClient.stopReplication("restored-$followerIndexName") } + .isInstanceOf(ResponseException::class.java) + .hasMessageContaining("Metadata for restored-$followerIndexName doesn't exist") + // Start replication on the new leader index + followerClient.startReplication( + StartReplicationRequest("source", "restored-$followerIndexName", "restored-$followerIndexName"), + TimeValue.timeValueSeconds(10), + true, true + ) + assertBusy({ + var statusResp = followerClient.replicationStatus("restored-$followerIndexName") + `validate status syncing response`(statusResp) + assertThat(followerClient.getShardReplicationTasks("restored-$followerIndexName")).isNotEmpty() + }, 60, TimeUnit.SECONDS) } } diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt index 042cd9fe..844c4539 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt @@ -67,21 +67,17 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { val leaderIndexName = createRandomIndex(leaderClient) var leaderIndexNameNew = "" createConnectionBetweenClusters(FOLLOWER, LEADER, connectionAlias) - try { followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, indexPattern) - // Verify that existing index matching the pattern are replicated. assertBusy ({ Assertions.assertThat(followerClient.indices() .exists(GetIndexRequest(leaderIndexName), RequestOptions.DEFAULT)) .isEqualTo(true) }, 30, TimeUnit.SECONDS) - assertBusy ({ Assertions.assertThat(getAutoFollowTasks(FOLLOWER).size).isEqualTo(1) }, 10, TimeUnit.SECONDS) - leaderIndexNameNew = createRandomIndex(leaderClient) // Verify that newly created index on leader which match the pattern are also replicated. assertBusy ({ @@ -101,8 +97,6 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { }, 60, TimeUnit.SECONDS) } finally { followerClient.deleteAutoFollowPattern(connectionAlias, indexPatternName) - followerClient.stopReplication(leaderIndexName, false) - followerClient.stopReplication(leaderIndexNameNew) } } @@ -111,20 +105,16 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { val leaderClient = getClientForCluster(LEADER) var leaderIndexNameNew = "" createConnectionBetweenClusters(FOLLOWER, LEADER, connectionAlias) - try { // Set poll duration to 30sec from 60sec (default) val settings = Settings.builder().put(ReplicationPlugin.REPLICATION_AUTOFOLLOW_REMOTE_INDICES_POLL_INTERVAL.key, TimeValue.timeValueSeconds(30)) val clusterUpdateSetttingsReq = ClusterUpdateSettingsRequest().persistentSettings(settings) val clusterUpdateResponse = followerClient.cluster().putSettings(clusterUpdateSetttingsReq, RequestOptions.DEFAULT) - var lastExecutionTime = 0L var stats = followerClient.AutoFollowStats() - Assert.assertTrue(clusterUpdateResponse.isAcknowledged) followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, indexPattern) - leaderIndexNameNew = createRandomIndex(leaderClient) // Verify that newly created index on leader which match the pattern are also replicated. assertBusy({ @@ -139,9 +129,7 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { lastExecutionTime = key["last_execution_time"]!! as Long } } - }, 30, TimeUnit.SECONDS) - assertBusy({ var af_stats = stats.get("autofollow_stats")!! as ArrayList> for (key in af_stats) { @@ -150,11 +138,8 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { } } }, 40, TimeUnit.SECONDS) - - } finally { followerClient.deleteAutoFollowPattern(connectionAlias, indexPatternName) - followerClient.stopReplication(leaderIndexNameNew) } } @@ -164,14 +149,11 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { val leaderClient = getClientForCluster(LEADER) val leaderIndexName = createRandomIndex(leaderClient) createConnectionBetweenClusters(FOLLOWER, LEADER, connectionAlias) - try { val settings = Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 3) .build() - followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, indexPattern, settings) - // Verify that existing index matching the pattern are replicated. assertBusy ({ Assertions.assertThat(followerClient.indices() @@ -179,8 +161,6 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { .isEqualTo(true) }, 30, TimeUnit.SECONDS) Assertions.assertThat(getAutoFollowTasks(FOLLOWER).size).isEqualTo(1) - - val getSettingsRequest = GetSettingsRequest() getSettingsRequest.indices(leaderIndexName) getSettingsRequest.includeDefaults(true) @@ -195,7 +175,6 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { }, 15, TimeUnit.SECONDS) } finally { followerClient.deleteAutoFollowPattern(connectionAlias, indexPatternName) - followerClient.stopReplication(leaderIndexName) } } @@ -203,27 +182,22 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { val indexPatternName2 = "test_pattern2" val indexPattern2 = "lead_index*" val leaderIndexName2 = "lead_index1" - val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) val leaderIndexName = createRandomIndex(leaderClient) leaderClient.indices().create(CreateIndexRequest(leaderIndexName2), RequestOptions.DEFAULT) createConnectionBetweenClusters(FOLLOWER, LEADER, connectionAlias) - try { followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, indexPattern) followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName2, indexPattern2) - assertBusy ({ Assertions.assertThat(getAutoFollowTasks(FOLLOWER).size).isEqualTo(2) }, 30, TimeUnit.SECONDS) - // Verify that existing index matching the pattern are replicated. assertBusy ({ Assertions.assertThat(followerClient.indices() .exists(GetIndexRequest(leaderIndexName2), RequestOptions.DEFAULT)) .isEqualTo(true) - var stats = followerClient.AutoFollowStats() Assertions.assertThat(stats.size).isEqualTo(5) assert(stats["num_success_start_replication"]!! as Int == 2) @@ -237,9 +211,7 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { followerClient.deleteAutoFollowPattern(connectionAlias, indexPatternName) followerClient.deleteAutoFollowPattern(connectionAlias, indexPatternName2) followerClient.waitForShardTaskStart(leaderIndexName) - followerClient.stopReplication(leaderIndexName) followerClient.waitForShardTaskStart(leaderIndexName2) - followerClient.stopReplication(leaderIndexName2) } } @@ -248,35 +220,26 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { val leaderClient = getClientForCluster(LEADER) val leaderIndexName = createRandomIndex(leaderClient) createConnectionBetweenClusters(FOLLOWER, LEADER, connectionAlias) - + followerClient.startReplication(StartReplicationRequest(connectionAlias, leaderIndexName, leaderIndexName), + TimeValue.timeValueSeconds(10),true, waitForRestore = true) + assertBusy({ + Assertions.assertThat(followerClient.indices() + .exists(GetIndexRequest(leaderIndexName), RequestOptions.DEFAULT)) + .isEqualTo(true) + }, 30, TimeUnit.SECONDS) + // Assert that there is no auto follow task & one index replication task + Assertions.assertThat(getAutoFollowTasks(FOLLOWER).size).isEqualTo(0) + Assertions.assertThat(getIndexReplicationTasks(FOLLOWER).size).isEqualTo(1) try { - followerClient.startReplication(StartReplicationRequest(connectionAlias, leaderIndexName, leaderIndexName), - TimeValue.timeValueSeconds(10),true, waitForRestore = true) - + followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, indexPattern) assertBusy({ - Assertions.assertThat(followerClient.indices() - .exists(GetIndexRequest(leaderIndexName), RequestOptions.DEFAULT)) - .isEqualTo(true) - }, 30, TimeUnit.SECONDS) - - // Assert that there is no auto follow task & one index replication task - Assertions.assertThat(getAutoFollowTasks(FOLLOWER).size).isEqualTo(0) - Assertions.assertThat(getIndexReplicationTasks(FOLLOWER).size).isEqualTo(1) - - try { - followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, indexPattern) - - assertBusy({ - // Assert that there is still only one index replication task - Assertions.assertThat(getAutoFollowTasks(FOLLOWER).size).isEqualTo(1) - Assertions.assertThat(getIndexReplicationTasks(FOLLOWER).size).isEqualTo(1) - followerClient.waitForShardTaskStart(leaderIndexName, waitForShardTask) - },30, TimeUnit.SECONDS) - } finally { - followerClient.deleteAutoFollowPattern(connectionAlias, indexPatternName) - } + // Assert that there is still only one index replication task + Assertions.assertThat(getAutoFollowTasks(FOLLOWER).size).isEqualTo(1) + Assertions.assertThat(getIndexReplicationTasks(FOLLOWER).size).isEqualTo(1) + followerClient.waitForShardTaskStart(leaderIndexName, waitForShardTask) + },30, TimeUnit.SECONDS) } finally { - followerClient.stopReplication(leaderIndexName) + followerClient.deleteAutoFollowPattern(connectionAlias, indexPatternName) } } @@ -292,7 +255,6 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { fun `test auto follow should fail on pattern name validation failure`() { val followerClient = getClientForCluster(FOLLOWER) createConnectionBetweenClusters(FOLLOWER, LEADER, connectionAlias) - assertPatternNameValidation(followerClient, "testPattern", "Value testPattern must be lowercase") assertPatternNameValidation(followerClient, "testPattern*", @@ -305,10 +267,8 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { "Value . must not be '.' or '..'") assertPatternNameValidation(followerClient, "..", "Value .. must not be '.' or '..'") - assertPatternNameValidation(followerClient, "_leader", "Value _leader must not start with '_' or '-' or '+'") - assertPatternNameValidation(followerClient, "-leader", "Value -leader must not start with '_' or '-' or '+'") assertPatternNameValidation(followerClient, "+leader", @@ -331,9 +291,7 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) createConnectionBetweenClusters(FOLLOWER, LEADER, connectionAlias) - val leaderIndexName = createRandomIndex(leaderClient) - try { followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, indexPattern) @@ -343,52 +301,42 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { .exists(GetIndexRequest(leaderIndexName), RequestOptions.DEFAULT)) .isEqualTo(true) } - Assertions.assertThat(getAutoFollowTasks(FOLLOWER).size).isEqualTo(1) Assertions.assertThat(getIndexReplicationTasks(FOLLOWER).size).isEqualTo(1) followerClient.waitForShardTaskStart(leaderIndexName, waitForShardTask) } finally { followerClient.deleteAutoFollowPattern(connectionAlias, indexPatternName) } - // Verify that auto follow tasks is stopped but the shard replication task remains. assertBusy ({ Assertions.assertThat(getAutoFollowTasks(FOLLOWER).size).isEqualTo(0) }, 30, TimeUnit.SECONDS) - Assertions.assertThat(getIndexReplicationTasks(FOLLOWER).size).isEqualTo(1) - followerClient.stopReplication(leaderIndexName) } fun `test autofollow task with start replication block`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) createConnectionBetweenClusters(FOLLOWER, LEADER, connectionAlias) - val leaderIndexName = createRandomIndex(leaderClient) try { - // Add replication start block followerClient.updateReplicationStartBlockSetting(true) followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, indexPattern) sleep(30000) // Default poll for auto follow in worst case - // verify both index replication tasks and autofollow tasks // Replication shouldn't have been started - 0 tasks // Autofollow task should still be up - 1 task Assertions.assertThat(getIndexReplicationTasks(FOLLOWER).size).isEqualTo(0) Assertions.assertThat(getAutoFollowTasks(FOLLOWER).size).isEqualTo(1) - // Remove replication start block followerClient.updateReplicationStartBlockSetting(false) sleep(45000) // poll for auto follow in worst case - // Index should be replicated and autofollow task should be present Assertions.assertThat(getIndexReplicationTasks(FOLLOWER).size).isEqualTo(1) Assertions.assertThat(getAutoFollowTasks(FOLLOWER).size).isEqualTo(1) } finally { followerClient.deleteAutoFollowPattern(connectionAlias, indexPatternName) - followerClient.stopReplication(leaderIndexName) } } @@ -396,38 +344,29 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) createConnectionBetweenClusters(FOLLOWER, LEADER, connectionAlias) - // create two leader indices and test autofollow to trigger to trigger jobs based on setting val leaderIndexName1 = createRandomIndex(leaderClient) val leaderIndexName2 = createRandomIndex(leaderClient) - followerClient.updateAutoFollowConcurrentStartReplicationJobSetting(2) try { followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, indexPattern) - // Verify that existing index matching the pattern are replicated. assertBusy { Assertions.assertThat(followerClient.indices() .exists(GetIndexRequest(leaderIndexName1), RequestOptions.DEFAULT)) .isEqualTo(true) } - assertBusy { Assertions.assertThat(followerClient.indices() .exists(GetIndexRequest(leaderIndexName2), RequestOptions.DEFAULT)) .isEqualTo(true) } - sleep(30000) // Default poll for auto follow in worst case - Assertions.assertThat(getAutoFollowTasks(FOLLOWER).size).isEqualTo(1) - } finally { // Reset default autofollow setting followerClient.updateAutoFollowConcurrentStartReplicationJobSetting(null) followerClient.deleteAutoFollowPattern(connectionAlias, indexPatternName) - followerClient.stopReplication(leaderIndexName1) - followerClient.stopReplication(leaderIndexName2) } } @@ -435,37 +374,28 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) createConnectionBetweenClusters(FOLLOWER, LEADER, connectionAlias) - // create two leader indices and test autofollow to trigger to trigger jobs based on setting val leaderIndexName1 = createRandomIndex(leaderClient) val leaderIndexName2 = createRandomIndex(leaderClient) - followerClient.updateAutoFollowConcurrentStartReplicationJobSetting(1) try { followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, indexPattern) - // Verify that existing index matching the pattern are replicated. assertBusy { // check that the index replication task is created for only index Assertions.assertThat(getIndexReplicationTasks(FOLLOWER).size).isEqualTo(1) } - sleep(30000) // Default poll for auto follow in worst case - assertBusy { // check that the index replication task is created for only index Assertions.assertThat(getIndexReplicationTasks(FOLLOWER).size).isEqualTo(2) } - sleep(30000) // Default poll for auto follow in worst case Assertions.assertThat(getAutoFollowTasks(FOLLOWER).size).isEqualTo(1) - } finally { // Reset default autofollow setting followerClient.updateAutoFollowConcurrentStartReplicationJobSetting(null) followerClient.deleteAutoFollowPattern(connectionAlias, indexPatternName) - followerClient.stopReplication(leaderIndexName1) - followerClient.stopReplication(leaderIndexName2) } } diff --git a/src/test/kotlin/org/opensearch/replication/task/TaskCancellationIT.kt b/src/test/kotlin/org/opensearch/replication/task/TaskCancellationIT.kt index 586290dc..5bbfef7e 100644 --- a/src/test/kotlin/org/opensearch/replication/task/TaskCancellationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/task/TaskCancellationIT.kt @@ -45,68 +45,52 @@ class TaskCancellationIT : MultiClusterRestTestCase() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) val primaryShards = 3 - createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create( CreateIndexRequest(leaderIndexName).settings(Settings.builder().put("index.number_of_shards", primaryShards).build()), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - try { - followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName)) - // Wait for Shard tasks to come up. - var tasks = Collections.emptyList() - assertBusy { - tasks = followerClient.getShardReplicationTasks(followerIndexName) - Assert.assertEquals(tasks.size, primaryShards) - } - - // Cancel one shard task - val cancelTasksRequest = CancelTasksRequest.Builder().withTaskId(TaskId(tasks[0])). - withWaitForCompletion(true).build() - followerClient.tasks().cancel(cancelTasksRequest, RequestOptions.DEFAULT) - - // Verify that replication is continuing and the shards tasks are up and running - assertBusy { - Assert.assertEquals(followerClient.getShardReplicationTasks(followerIndexName).size, primaryShards) - assertThat(followerClient.getIndexReplicationTask(followerIndexName).isNotBlank()).isTrue() - `validate status due shard task cancellation`(followerClient.replicationStatus(followerIndexName)) - } - } finally { - followerClient.stopReplication(followerIndexName) + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName)) + // Wait for Shard tasks to come up. + var tasks = Collections.emptyList() + assertBusy { + tasks = followerClient.getShardReplicationTasks(followerIndexName) + Assert.assertEquals(tasks.size, primaryShards) + } + // Cancel one shard task + val cancelTasksRequest = CancelTasksRequest.Builder().withTaskId(TaskId(tasks[0])). + withWaitForCompletion(true).build() + followerClient.tasks().cancel(cancelTasksRequest, RequestOptions.DEFAULT) + // Verify that replication is continuing and the shards tasks are up and running + assertBusy { + Assert.assertEquals(followerClient.getShardReplicationTasks(followerIndexName).size, primaryShards) + assertThat(followerClient.getIndexReplicationTask(followerIndexName).isNotBlank()).isTrue() + `validate status due shard task cancellation`(followerClient.replicationStatus(followerIndexName)) } } fun `test user triggering cancel on an index task`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - createConnectionBetweenClusters(FOLLOWER, LEADER) - val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) assertThat(createIndexResponse.isAcknowledged).isTrue() - try { followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName)) // Wait for Shard tasks to come up. assertBusy { assertThat(followerClient.getShardReplicationTasks(followerIndexName).isEmpty()).isEqualTo(false) } - // Cancel the index replication task var task = followerClient.getIndexReplicationTask(followerIndexName) assertThat(task.isNullOrBlank()).isFalse() val cancelTasksRequest = CancelTasksRequest.Builder().withTaskId(TaskId(task)). withWaitForCompletion(true).build() followerClient.tasks().cancel(cancelTasksRequest, RequestOptions.DEFAULT) - // Verify that replication has paused. assertBusy { assertThat(followerClient.getShardReplicationTasks(followerIndexName).isEmpty()).isTrue() assertThat(followerClient.getIndexReplicationTask(followerIndexName).isNullOrBlank()).isTrue() `validate status due index task cancellation`(followerClient.replicationStatus(followerIndexName)) } - } finally { - followerClient.stopReplication(followerIndexName) - } } } diff --git a/src/test/kotlin/org/opensearch/replication/task/shard/TransportReplayChangesActionIT.kt b/src/test/kotlin/org/opensearch/replication/task/shard/TransportReplayChangesActionIT.kt index 8df58e23..58a56112 100644 --- a/src/test/kotlin/org/opensearch/replication/task/shard/TransportReplayChangesActionIT.kt +++ b/src/test/kotlin/org/opensearch/replication/task/shard/TransportReplayChangesActionIT.kt @@ -36,60 +36,48 @@ class TransportReplayChangesActionIT : MultiClusterRestTestCase() { // Create a leader/follower index val leaderIndex = randomAlphaOfLength(10).toLowerCase(Locale.ROOT) val followerIndex = randomAlphaOfLength(10).toLowerCase(Locale.ROOT) - - try { - val doc1 = mapOf("name" to randomAlphaOfLength(20)) - // Create Leader Index - val response = leader.index(IndexRequest(leaderIndex).id("1").source(doc1), RequestOptions.DEFAULT) - Assertions.assertThat(response.result) - .withFailMessage("Failed to create leader data").isEqualTo(DocWriteResponse.Result.CREATED) - - // Setup Mapping on leader - var putMappingRequest = PutMappingRequest(leaderIndex) - putMappingRequest.source( - "{\"dynamic\":\"strict\",\"properties\":{\"name\":{\"type\":\"text\"}}}", - XContentType.JSON - ) - leader.indices().putMapping(putMappingRequest, RequestOptions.DEFAULT) - - // Start replication - follower.startReplication( - StartReplicationRequest("source", leaderIndex, followerIndex), - waitForRestore = true - ) - assertBusy { - val getResponse = follower.get(GetRequest(followerIndex, "1"), RequestOptions.DEFAULT) - Assertions.assertThat(getResponse.isExists).isTrue() - Assertions.assertThat(getResponse.sourceAsMap).isEqualTo(doc1) - } - - // Add a new field in mapping. - putMappingRequest = PutMappingRequest(leaderIndex) - putMappingRequest.source( - "{\"dynamic\":\"strict\",\"properties\":{\"name\":{\"type\":\"text\"},\"place\":{\"type\":\"text\"}}}", - XContentType.JSON - ) - leader.indices().putMapping(putMappingRequest, RequestOptions.DEFAULT) - - // Ingest a doc on the leader - val doc2 = mapOf("name" to randomAlphaOfLength(5), "place" to randomAlphaOfLength(5)) - leader.index(IndexRequest(leaderIndex).id("2").source(doc2), RequestOptions.DEFAULT) - - // Verify that replication is working as expected. - assertBusy ({ - Assert.assertEquals(leader.count(CountRequest(leaderIndex), RequestOptions.DEFAULT).toString(), - follower.count(CountRequest(followerIndex), RequestOptions.DEFAULT).toString()) - `validate status syncing response`(follower.replicationStatus(followerIndex)) - val getResponse = follower.get(GetRequest(followerIndex, "2"), RequestOptions.DEFAULT) - Assertions.assertThat(getResponse.isExists).isTrue() - Assertions.assertThat(getResponse.sourceAsMap).isEqualTo(doc2) - }, - 30, TimeUnit.SECONDS - ) - - } finally { - follower.stopReplication(followerIndex) + val doc1 = mapOf("name" to randomAlphaOfLength(20)) + // Create Leader Index + val response = leader.index(IndexRequest(leaderIndex).id("1").source(doc1), RequestOptions.DEFAULT) + Assertions.assertThat(response.result) + .withFailMessage("Failed to create leader data").isEqualTo(DocWriteResponse.Result.CREATED) + // Setup Mapping on leader + var putMappingRequest = PutMappingRequest(leaderIndex) + putMappingRequest.source( + "{\"dynamic\":\"strict\",\"properties\":{\"name\":{\"type\":\"text\"}}}", + XContentType.JSON + ) + leader.indices().putMapping(putMappingRequest, RequestOptions.DEFAULT) + // Start replication + follower.startReplication( + StartReplicationRequest("source", leaderIndex, followerIndex), + waitForRestore = true + ) + assertBusy { + val getResponse = follower.get(GetRequest(followerIndex, "1"), RequestOptions.DEFAULT) + Assertions.assertThat(getResponse.isExists).isTrue() + Assertions.assertThat(getResponse.sourceAsMap).isEqualTo(doc1) } - + // Add a new field in mapping. + putMappingRequest = PutMappingRequest(leaderIndex) + putMappingRequest.source( + "{\"dynamic\":\"strict\",\"properties\":{\"name\":{\"type\":\"text\"},\"place\":{\"type\":\"text\"}}}", + XContentType.JSON + ) + leader.indices().putMapping(putMappingRequest, RequestOptions.DEFAULT) + // Ingest a doc on the leader + val doc2 = mapOf("name" to randomAlphaOfLength(5), "place" to randomAlphaOfLength(5)) + leader.index(IndexRequest(leaderIndex).id("2").source(doc2), RequestOptions.DEFAULT) + // Verify that replication is working as expected. + assertBusy ({ + Assert.assertEquals(leader.count(CountRequest(leaderIndex), RequestOptions.DEFAULT).toString(), + follower.count(CountRequest(followerIndex), RequestOptions.DEFAULT).toString()) + `validate status syncing response`(follower.replicationStatus(followerIndex)) + val getResponse = follower.get(GetRequest(followerIndex, "2"), RequestOptions.DEFAULT) + Assertions.assertThat(getResponse.isExists).isTrue() + Assertions.assertThat(getResponse.sourceAsMap).isEqualTo(doc2) + }, + 30, TimeUnit.SECONDS + ) } } \ No newline at end of file From 5f4b5bc8473045dd9392c95e9683b30bb54c0f26 Mon Sep 17 00:00:00 2001 From: Sai Kumar Date: Mon, 12 Dec 2022 10:50:25 +0530 Subject: [PATCH 051/157] Handeled clean-up of stale index task during cancellation (#645) Signed-off-by: Sai Kumar --- .../task/index/IndexReplicationTask.kt | 53 ++++++++++++----- .../replication/ReplicationHelpers.kt | 2 +- .../task/index/IndexReplicationTaskTests.kt | 57 +++++++++++++++++++ 3 files changed, 98 insertions(+), 14 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt index 395119c3..d21d9232 100644 --- a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt @@ -99,7 +99,6 @@ import java.util.stream.Collectors import kotlin.coroutines.resume import kotlin.coroutines.resumeWithException import kotlin.coroutines.suspendCoroutine -import kotlin.streams.toList open class IndexReplicationTask(id: Long, type: String, action: String, description: String, parentTask: TaskId, @@ -150,7 +149,8 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript val blockListedSettings :Set = blSettings.stream().map { k -> k.key }.collect(Collectors.toSet()) const val SLEEP_TIME_BETWEEN_POLL_MS = 5000L - const val TASK_CANCELLATION_REASON = "Index replication task was cancelled by user" + const val AUTOPAUSED_REASON_PREFIX = "AutoPaused: " + const val TASK_CANCELLATION_REASON = AUTOPAUSED_REASON_PREFIX + "Index replication task was cancelled by user" } @@ -257,13 +257,6 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript } } - override fun onCancelled() { - log.info("Cancelling the index replication task.") - client.execute(PauseIndexReplicationAction.INSTANCE, - PauseIndexReplicationRequest(followerIndexName, TASK_CANCELLATION_REASON)) - super.onCancelled() - } - private suspend fun failReplication(failedState: FailedState) { withContext(NonCancellable) { val reason = failedState.errorMsg @@ -307,6 +300,23 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript return MonitoringState } + fun isTrackingTaskForIndex(): Boolean { + val persistentTasks = clusterService.state().metadata.custom(PersistentTasksCustomMetadata.TYPE) + val runningTasksForIndex = persistentTasks.findTasks(IndexReplicationExecutor.TASK_NAME, Predicate { true }).stream() + .map { task -> task as PersistentTask } + .filter { task -> task.params!!.followerIndexName == followerIndexName} + .toArray() + assert(runningTasksForIndex.size <= 1) { "Found more than one running index task for index[$followerIndexName]" } + for (runningTask in runningTasksForIndex) { + val currentTask = runningTask as PersistentTask + log.info("Verifying task details - currentTask={isAssigned=${currentTask.isAssigned},executorNode=${currentTask.executorNode}}") + if(currentTask.isAssigned && currentTask.executorNode == clusterService.state().nodes.localNodeId) { + return true + } + } + return false + } + private fun isResumed(): Boolean { return clusterService.state().routingTable.hasIndex(followerIndexName) } @@ -619,7 +629,7 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript log.error("Going to initiate auto-pause of $followerIndexName due to shard failures - $state") val pauseReplicationResponse = client.suspendExecute( replicationMetadata, - PauseIndexReplicationAction.INSTANCE, PauseIndexReplicationRequest(followerIndexName, "AutoPaused: ${state.errorMsg}"), + PauseIndexReplicationAction.INSTANCE, PauseIndexReplicationRequest(followerIndexName, "$AUTOPAUSED_REASON_PREFIX + ${state.errorMsg}"), defaultContext = true ) if (!pauseReplicationResponse.isAcknowledged) { @@ -656,10 +666,27 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript } override suspend fun cleanup() { - if (currentTaskState.state == ReplicationState.RESTORING) { - log.info("Replication stopped before restore could finish, so removing partial restore..") - cancelRestore() + // If the task is already running on the other node, + // OpenSearch persistent task framework cancels any stale tasks on the old nodes. + // Currently, we don't have view on the cancellation reason. Before triggering + // any further actions on the index from this task, verify that, this is the actual task tracking the index. + // - stale task during cancellation shouldn't trigger further actions. + if(isTrackingTaskForIndex()) { + if (currentTaskState.state == ReplicationState.RESTORING) { + log.info("Replication stopped before restore could finish, so removing partial restore..") + cancelRestore() + } + + // if cancelled and not in paused state. + val replicationStateParams = getReplicationStateParamsForIndex(clusterService, followerIndexName) + if(isCancelled && replicationStateParams != null + && replicationStateParams[REPLICATION_LAST_KNOWN_OVERALL_STATE] == ReplicationOverallState.RUNNING.name) { + log.info("Task is cancelled. Moving the index to auto-pause state") + client.execute(PauseIndexReplicationAction.INSTANCE, + PauseIndexReplicationRequest(followerIndexName, TASK_CANCELLATION_REASON)) + } } + /* This is to minimise overhead of calling an additional listener as * it continues to be called even after the task is completed. */ diff --git a/src/test/kotlin/org/opensearch/replication/ReplicationHelpers.kt b/src/test/kotlin/org/opensearch/replication/ReplicationHelpers.kt index 11c9fbba..2fa7d1fc 100644 --- a/src/test/kotlin/org/opensearch/replication/ReplicationHelpers.kt +++ b/src/test/kotlin/org/opensearch/replication/ReplicationHelpers.kt @@ -51,7 +51,7 @@ const val REST_REPLICATION_TASKS = "_tasks?actions=*replication*&detailed&pretty const val REST_LEADER_STATS = "${REST_REPLICATION_PREFIX}leader_stats" const val REST_FOLLOWER_STATS = "${REST_REPLICATION_PREFIX}follower_stats" const val REST_AUTO_FOLLOW_STATS = "${REST_REPLICATION_PREFIX}autofollow_stats" -const val INDEX_TASK_CANCELLATION_REASON = "Index replication task was cancelled by user" +const val INDEX_TASK_CANCELLATION_REASON = "AutoPaused: Index replication task was cancelled by user" const val STATUS_REASON_USER_INITIATED = "User initiated" const val STATUS_REASON_SHARD_TASK_CANCELLED = "Shard task killed or cancelled." const val STATUS_REASON_INDEX_NOT_FOUND = "no such index" diff --git a/src/test/kotlin/org/opensearch/replication/task/index/IndexReplicationTaskTests.kt b/src/test/kotlin/org/opensearch/replication/task/index/IndexReplicationTaskTests.kt index 569eadf8..9ba4919c 100644 --- a/src/test/kotlin/org/opensearch/replication/task/index/IndexReplicationTaskTests.kt +++ b/src/test/kotlin/org/opensearch/replication/task/index/IndexReplicationTaskTests.kt @@ -26,6 +26,8 @@ import org.opensearch.cluster.ClusterStateObserver import org.opensearch.cluster.RestoreInProgress import org.opensearch.cluster.metadata.IndexMetadata import org.opensearch.cluster.metadata.Metadata +import org.opensearch.cluster.node.DiscoveryNode +import org.opensearch.cluster.node.DiscoveryNodes import org.opensearch.cluster.routing.RoutingTable import org.opensearch.common.settings.Settings import org.opensearch.common.settings.SettingsModule @@ -209,6 +211,61 @@ class IndexReplicationTaskTests : OpenSearchTestCase() { assertThat(shardTasks.size == 2).isTrue } + fun testIsTrackingTaskForIndex() = runBlocking { + val replicationTask: IndexReplicationTask = spy(createIndexReplicationTask()) + var taskManager = Mockito.mock(TaskManager::class.java) + replicationTask.setPersistent(taskManager) + var rc = ReplicationContext(followerIndex) + var rm = ReplicationMetadata(connectionName, ReplicationStoreMetadataType.INDEX.name, ReplicationOverallState.RUNNING.name, "reason", rc, rc, Settings.EMPTY) + replicationTask.setReplicationMetadata(rm) + + // when index replication task is valid + var tasks = PersistentTasksCustomMetadata.builder() + var leaderIndex = Index(followerIndex, "_na_") + tasks.addTask( "replication:0", IndexReplicationExecutor.TASK_NAME, IndexReplicationParams("remoteCluster", leaderIndex, followerIndex), + PersistentTasksCustomMetadata.Assignment("same_node", "test assignment on other node")) + + var metadata = Metadata.builder() + .put(IndexMetadata.builder(REPLICATION_CONFIG_SYSTEM_INDEX).settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(0)) + .put(IndexMetadata.builder(followerIndex).settings(settings(Version.CURRENT)).numberOfShards(2).numberOfReplicas(0)) + .putCustom(PersistentTasksCustomMetadata.TYPE, tasks.build()) + .build() + var routingTableBuilder = RoutingTable.builder() + .addAsNew(metadata.index(REPLICATION_CONFIG_SYSTEM_INDEX)) + .addAsNew(metadata.index(followerIndex)) + var discoveryNodesBuilder = DiscoveryNodes.Builder() + .localNodeId("same_node") + var newClusterState = ClusterState.builder(clusterService.state()) + .metadata(metadata) + .routingTable(routingTableBuilder.build()) + .nodes(discoveryNodesBuilder.build()).build() + setState(clusterService, newClusterState) + assertThat(replicationTask.isTrackingTaskForIndex()).isTrue + + // when index replication task is not valid + tasks = PersistentTasksCustomMetadata.builder() + leaderIndex = Index(followerIndex, "_na_") + tasks.addTask( "replication:0", IndexReplicationExecutor.TASK_NAME, IndexReplicationParams("remoteCluster", leaderIndex, followerIndex), + PersistentTasksCustomMetadata.Assignment("other_node", "test assignment on other node")) + + metadata = Metadata.builder() + .put(IndexMetadata.builder(REPLICATION_CONFIG_SYSTEM_INDEX).settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(0)) + .put(IndexMetadata.builder(followerIndex).settings(settings(Version.CURRENT)).numberOfShards(2).numberOfReplicas(0)) + .putCustom(PersistentTasksCustomMetadata.TYPE, tasks.build()) + .build() + routingTableBuilder = RoutingTable.builder() + .addAsNew(metadata.index(REPLICATION_CONFIG_SYSTEM_INDEX)) + .addAsNew(metadata.index(followerIndex)) + discoveryNodesBuilder = DiscoveryNodes.Builder() + .localNodeId("same_node") + newClusterState = ClusterState.builder(clusterService.state()) + .metadata(metadata) + .routingTable(routingTableBuilder.build()) + .nodes(discoveryNodesBuilder.build()).build() + setState(clusterService, newClusterState) + assertThat(replicationTask.isTrackingTaskForIndex()).isFalse + } + private fun createIndexReplicationTask() : IndexReplicationTask { var threadPool = TestThreadPool("IndexReplicationTask") //Hack Alert : Though it is meant to force rejection , this is to make overallTaskScope not null From 02b9f7b408d73aeb6da1fd64ffc2c26159fc54ef Mon Sep 17 00:00:00 2001 From: Priyanka Sharma Date: Mon, 26 Dec 2022 04:10:46 -0800 Subject: [PATCH 052/157] Fixed build failure due to dependency conflicts. Signed-off-by: Priyanka Sharma --- build.gradle | 2 ++ 1 file changed, 2 insertions(+) diff --git a/build.gradle b/build.gradle index f17e6da7..b965e24d 100644 --- a/build.gradle +++ b/build.gradle @@ -111,6 +111,8 @@ configurations.all { force 'commons-codec:commons-codec:1.13' force 'org.apache.httpcomponents:httpclient:4.5.13' force 'org.apache.httpcomponents:httpclient-osgi:4.5.13' + force 'org.apache.httpcomponents.core5:httpcore5:5.1.4' + force 'org.apache.httpcomponents.core5:httpcore5-h2:5.1.4' force 'org.apache.httpcomponents.client5:httpclient5:5.0.3' force 'org.apache.httpcomponents.client5:httpclient5-osgi:5.0.3' force 'com.fasterxml.jackson.core:jackson-databind:2.13.4.2' From e7a9be6313a19193cf6f4a9e9d31c30a008bcdb7 Mon Sep 17 00:00:00 2001 From: sricharanvuppu <113983630+sricharanvuppu@users.noreply.github.com> Date: Tue, 27 Dec 2022 14:13:48 +0530 Subject: [PATCH 053/157] release notes for 1.3.7.0 (#664) Signed-off-by: sricharanvuppu --- ...cluster-replication.release-notes-1.3.7.0.md | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) create mode 100644 release-notes/opensearch-cross-cluster-replication.release-notes-1.3.7.0.md diff --git a/release-notes/opensearch-cross-cluster-replication.release-notes-1.3.7.0.md b/release-notes/opensearch-cross-cluster-replication.release-notes-1.3.7.0.md new file mode 100644 index 00000000..fbb15a18 --- /dev/null +++ b/release-notes/opensearch-cross-cluster-replication.release-notes-1.3.7.0.md @@ -0,0 +1,17 @@ +## Version 1.3.7.0 Release Notes + +Compatible with OpenSearch 1.3.7 + +### Bug Fixes + +* Include default index settings during leader setting validation ([#609](https://github.com/opensearch-project/cross-cluster-replication/pull/609)) + +### Security Fixes + +* Changed version of jackson databind to 2.13.4.2 for OS 1.3([#608](https://github.com/opensearch-project/cross-cluster-replication/pull/608)) + +### Infrastructure + +* Update the CI workflow to run the integ tests on all platforms ([#649](https://github.com/opensearch-project/cross-cluster-replication/pull/649)) + + From c45fe267f67ae327a66e49165d04d768c36e0e55 Mon Sep 17 00:00:00 2001 From: sricharanvuppu <113983630+sricharanvuppu@users.noreply.github.com> Date: Tue, 27 Dec 2022 14:14:25 +0530 Subject: [PATCH 054/157] release notes for 2.4.1 (#663) Signed-off-by: sricharanvuppu Signed-off-by: sricharanvuppu --- ...ensearch-cross-cluster-replication.release-notes-2.4.1.0.md | 3 +++ 1 file changed, 3 insertions(+) create mode 100644 release-notes/opensearch-cross-cluster-replication.release-notes-2.4.1.0.md diff --git a/release-notes/opensearch-cross-cluster-replication.release-notes-2.4.1.0.md b/release-notes/opensearch-cross-cluster-replication.release-notes-2.4.1.0.md new file mode 100644 index 00000000..d4b23962 --- /dev/null +++ b/release-notes/opensearch-cross-cluster-replication.release-notes-2.4.1.0.md @@ -0,0 +1,3 @@ +## Version 2.4.1.0 Release Notes + +Compatible with OpenSearch 2.4.1 From f99f1be7a6de1796b987eb1fb69ebac35efd367b Mon Sep 17 00:00:00 2001 From: Sai Kumar Date: Tue, 27 Dec 2022 16:59:57 +0530 Subject: [PATCH 055/157] Added toList import statement for java 11 version (#670) Signed-off-by: Sai Kumar Signed-off-by: Sai Kumar --- .../opensearch/replication/task/index/IndexReplicationTask.kt | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt index d21d9232..8b63a7b4 100644 --- a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt @@ -99,6 +99,7 @@ import java.util.stream.Collectors import kotlin.coroutines.resume import kotlin.coroutines.resumeWithException import kotlin.coroutines.suspendCoroutine +import kotlin.streams.toList open class IndexReplicationTask(id: Long, type: String, action: String, description: String, parentTask: TaskId, From b08b58e67a9963c9f19ebf310a5b1776786b9401 Mon Sep 17 00:00:00 2001 From: Priyanka Sharma Date: Fri, 6 Jan 2023 01:43:48 -0800 Subject: [PATCH 056/157] Removed jackson-databind dependency from build-gradle that now comes from core Signed-off-by: Priyanka Sharma --- build.gradle | 1 - 1 file changed, 1 deletion(-) diff --git a/build.gradle b/build.gradle index b965e24d..71624134 100644 --- a/build.gradle +++ b/build.gradle @@ -115,7 +115,6 @@ configurations.all { force 'org.apache.httpcomponents.core5:httpcore5-h2:5.1.4' force 'org.apache.httpcomponents.client5:httpclient5:5.0.3' force 'org.apache.httpcomponents.client5:httpclient5-osgi:5.0.3' - force 'com.fasterxml.jackson.core:jackson-databind:2.13.4.2' force 'org.yaml:snakeyaml:1.32' force 'org.codehaus.plexus:plexus-utils:3.0.24' force 'net.java.dev.jna:jna:5.12.1' From 1af1a0a8da6fa91a5937e0307294dc32d2793e07 Mon Sep 17 00:00:00 2001 From: "Daniel (dB.) Doubrovkine" Date: Mon, 9 Jan 2023 00:47:34 -0500 Subject: [PATCH 057/157] Updated MAINTAINERS.md to match recommended opensearch-project format. (#673) Signed-off-by: dblock Signed-off-by: dblock --- MAINTAINERS.md | 27 +++++++++++++++------------ 1 file changed, 15 insertions(+), 12 deletions(-) diff --git a/MAINTAINERS.md b/MAINTAINERS.md index b7d0ac64..14f485be 100644 --- a/MAINTAINERS.md +++ b/MAINTAINERS.md @@ -1,13 +1,16 @@ -# OpenSearch Cross Cluster Replication Maintainers +## Overview -## Maintainers -| Maintainer | GitHub ID | Affiliation | -| --------------- | --------- | ----------- | -| Rushi Agrawal | [rushiagr](https://github.com/rushiagr) | Amazon | -| Gopala Krishna A | [krishna-ggk](https://github.com/krishna-ggk) | Amazon | -| Ankit | [ankitkala](https://github.com/ankitkala) | Amazon | -| Naveen | [naveenpajjuri](https://github.com/naveenpajjuri) | Amazon | -| Sai | [saikaranam-amazon](https://github.com/saikaranam-amazon) | Amazon | -| Bhanu Tadepalli | [tbhanu-amzn](https://github.com/tbhanu-amzn) | Amazon | -| Sooraj Sinha | [soosinha](https://github.com/soosinha) | Amazon | -| Gaurav Bafna | [gbbafna](https://github.com/gbbafna) | Amazon | +This document contains a list of maintainers in this repo. See [opensearch-project/.github/RESPONSIBILITIES.md](https://github.com/opensearch-project/.github/blob/main/RESPONSIBILITIES.md#maintainer-responsibilities) that explains what the role of maintainer means, what maintainers do in this and other repos, and how they should be doing it. If you're interested in contributing, and becoming a maintainer, see [CONTRIBUTING](CONTRIBUTING.md). + +## Current Maintainers + +| Maintainer | GitHub ID | Affiliation | +| ---------------- | --------------------------------------------------------- | ----------- | +| Rushi Agrawal | [rushiagr](https://github.com/rushiagr) | Amazon | +| Gopala Krishna A | [krishna-ggk](https://github.com/krishna-ggk) | Amazon | +| Ankit | [ankitkala](https://github.com/ankitkala) | Amazon | +| Naveen | [naveenpajjuri](https://github.com/naveenpajjuri) | Amazon | +| Sai | [saikaranam-amazon](https://github.com/saikaranam-amazon) | Amazon | +| Bhanu Tadepalli | [tbhanu-amzn](https://github.com/tbhanu-amzn) | Amazon | +| Sooraj Sinha | [soosinha](https://github.com/soosinha) | Amazon | +| Gaurav Bafna | [gbbafna](https://github.com/gbbafna) | Amazon | From b09c70656c94bbf9367af170262d0d8e97dac748 Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Tue, 10 Jan 2023 12:56:19 +0530 Subject: [PATCH 058/157] Updating the maintainer list for the repo (#678) Signed-off-by: Ankit Kala Signed-off-by: Ankit Kala --- MAINTAINERS.md | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) diff --git a/MAINTAINERS.md b/MAINTAINERS.md index 14f485be..ee47b7dd 100644 --- a/MAINTAINERS.md +++ b/MAINTAINERS.md @@ -6,11 +6,16 @@ This document contains a list of maintainers in this repo. See [opensearch-proje | Maintainer | GitHub ID | Affiliation | | ---------------- | --------------------------------------------------------- | ----------- | -| Rushi Agrawal | [rushiagr](https://github.com/rushiagr) | Amazon | | Gopala Krishna A | [krishna-ggk](https://github.com/krishna-ggk) | Amazon | -| Ankit | [ankitkala](https://github.com/ankitkala) | Amazon | -| Naveen | [naveenpajjuri](https://github.com/naveenpajjuri) | Amazon | -| Sai | [saikaranam-amazon](https://github.com/saikaranam-amazon) | Amazon | -| Bhanu Tadepalli | [tbhanu-amzn](https://github.com/tbhanu-amzn) | Amazon | +| Ankit Kala | [ankitkala](https://github.com/ankitkala) | Amazon | +| Saikumar Karanam | [saikaranam-amazon](https://github.com/saikaranam-amazon) | Amazon | | Sooraj Sinha | [soosinha](https://github.com/soosinha) | Amazon | +| Suraj Kumar | [suranjay](https://github.com/suranjay) | Amazon | | Gaurav Bafna | [gbbafna](https://github.com/gbbafna) | Amazon | +| Monu Singh | [monu-aws](https://github.com/monu-aws) | Amazon | +| Priyanka Sharma | [priyatsh](https://github.com/priyatsh) | Amazon | +| Mohit Kumar | [mohitamg](https://github.com/mohitamg) | Amazon | +| Nishant Goel | [nisgoel-amazon](https://github.com/nisgoel-amazon) | Amazon | +| Ishank Katiyar | [ishankka](https://github.com/ishankka) | Amazon | +| Shreyansh Ray | [rayshrey](https://github.com/rayshrey) | Amazon | +| Ranganathan S | [ranga011](https://github.com/ranga011) | Amazon | \ No newline at end of file From 4d12d6cfb17e2dbac02fdee815aff9bc95062b0a Mon Sep 17 00:00:00 2001 From: sricharanvuppu <113983630+sricharanvuppu@users.noreply.github.com> Date: Tue, 10 Jan 2023 13:20:57 +0530 Subject: [PATCH 059/157] Updating multi-field mapping at follower (#671) * Updating multi-field mapping at follower Signed-off-by: sricharanvuppu * Integ tests were written for multi-field mapping update at follower Signed-off-by: sricharanvuppu * Modified Integ test cases for Multi-field mapping at follower Signed-off-by: sricharanvuppu * UpdateFollowerMapping method added instead of SyncRemoteMapping Signed-off-by: sricharanvuppu Signed-off-by: sricharanvuppu --- .../task/index/IndexReplicationTask.kt | 34 +++++++++ .../integ/rest/StartReplicationIT.kt | 76 +++++++++++++++++++ 2 files changed, 110 insertions(+) diff --git a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt index 8b63a7b4..cbaa8ec5 100644 --- a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt @@ -55,6 +55,9 @@ import org.opensearch.action.admin.indices.alias.get.GetAliasesRequest import org.opensearch.action.admin.indices.delete.DeleteIndexRequest import org.opensearch.action.admin.indices.settings.get.GetSettingsRequest import org.opensearch.action.admin.indices.settings.put.UpdateSettingsRequest +import org.opensearch.action.admin.indices.mapping.get.GetMappingsRequest +import org.opensearch.action.admin.indices.mapping.put.PutMappingRequest +import org.opensearch.action.support.IndicesOptions import org.opensearch.client.Client import org.opensearch.client.Requests import org.opensearch.cluster.ClusterChangedEvent @@ -75,6 +78,7 @@ import org.opensearch.common.unit.ByteSizeValue import org.opensearch.common.xcontent.ToXContent import org.opensearch.common.xcontent.ToXContentObject import org.opensearch.common.xcontent.XContentBuilder +import org.opensearch.common.xcontent.XContentType import org.opensearch.index.Index import org.opensearch.index.IndexService import org.opensearch.index.IndexSettings @@ -88,6 +92,7 @@ import org.opensearch.persistent.PersistentTasksCustomMetadata.PersistentTask import org.opensearch.persistent.PersistentTasksNodeService import org.opensearch.persistent.PersistentTasksService import org.opensearch.replication.ReplicationException +import org.opensearch.replication.MappingNotAvailableException import org.opensearch.replication.ReplicationPlugin.Companion.REPLICATION_INDEX_TRANSLOG_PRUNING_ENABLED_SETTING import org.opensearch.rest.RestStatus import org.opensearch.tasks.TaskId @@ -100,6 +105,7 @@ import kotlin.coroutines.resume import kotlin.coroutines.resumeWithException import kotlin.coroutines.suspendCoroutine import kotlin.streams.toList +import org.opensearch.cluster.DiffableUtils open class IndexReplicationTask(id: Long, type: String, action: String, description: String, parentTask: TaskId, @@ -407,6 +413,19 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript } } + private suspend fun UpdateFollowereMapping(followerIndex: String,mappingSource: String) { + + val options = IndicesOptions.strictSingleIndexNoExpandForbidClosed() + if (null == mappingSource) { + throw MappingNotAvailableException("MappingSource is not available") + } + val putMappingRequest = PutMappingRequest().indices(followerIndex).indicesOptions(options) + .source(mappingSource, XContentType.JSON) + val updateMappingRequest = UpdateMetadataRequest(followerIndex, UpdateMetadataRequest.Type.MAPPING, putMappingRequest) + client.suspendExecute(UpdateMetadataAction.INSTANCE, updateMappingRequest, injectSecurityContext = true) + log.debug("Mappings synced for $followerIndex") + } + private suspend fun pollForMetadata(scope: CoroutineScope) { while (scope.isActive) { try { @@ -547,6 +566,21 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript } else { metadataUpdate = null } + val options = IndicesOptions.strictSingleIndexNoExpandForbidClosed() + var gmr = GetMappingsRequest().indices(this.leaderIndex.name).indicesOptions(options) + var mappingResponse = remoteClient.suspending(remoteClient.admin().indices()::getMappings, injectSecurityContext = true)(gmr) + var leaderMappingSource = mappingResponse.mappings.get(this.leaderIndex.name).source().toString() + val leaderProperties = mappingResponse.mappings().get(this.leaderIndex.name).sourceAsMap().toMap().get("properties") as Map + gmr = GetMappingsRequest().indices(this.followerIndexName).indicesOptions(options) + mappingResponse = client.suspending(client.admin().indices()::getMappings, injectSecurityContext = true)(gmr) + val followerProperties = mappingResponse.mappings().get(this.followerIndexName).sourceAsMap().toMap().get("properties") as Map + for(iter in followerProperties) { + if(leaderProperties.containsKey(iter.key) && leaderProperties.getValue(iter.key).toString()!=(iter.value).toString()){ + log.info("Updating Multi-field Mapping at Follower") + UpdateFollowereMapping(this.followerIndexName,leaderMappingSource) + break; + } + } } catch (e: Exception) { log.error("Error in getting the required metadata ${e.stackTraceToString()}") diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt index 599a2f19..1c483298 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt @@ -12,6 +12,7 @@ package org.opensearch.replication.integ.rest +import kotlinx.coroutines.delay import org.opensearch.replication.IndexUtil import org.opensearch.replication.MultiClusterAnnotations import org.opensearch.replication.MultiClusterRestTestCase @@ -76,6 +77,7 @@ import java.nio.file.Files import java.util.* import java.util.concurrent.TimeUnit import org.opensearch.bootstrap.BootstrapInfo +import org.opensearch.index.mapper.Mapping @MultiClusterAnnotations.ClusterConfigurations( @@ -1181,6 +1183,80 @@ class StartReplicationIT: MultiClusterRestTestCase() { followerClient.stopReplication(followerIndexName) } } + fun `test that follower index mapping updates when leader index gets multi-field mapping`() { + val followerClient = getClientForCluster(FOLLOWER) + val leaderClient = getClientForCluster(LEADER) + createConnectionBetweenClusters(FOLLOWER, LEADER) + val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) + assertThat(createIndexResponse.isAcknowledged).isTrue() + var putMappingRequest = PutMappingRequest(leaderIndexName) + putMappingRequest.source("{\"properties\":{\"field1\":{\"type\":\"text\"}}}", XContentType.JSON) + leaderClient.indices().putMapping(putMappingRequest, RequestOptions.DEFAULT) + val sourceMap = mapOf("field1" to randomAlphaOfLength(5)) + leaderClient.index(IndexRequest(leaderIndexName).id("1").source(sourceMap), RequestOptions.DEFAULT) + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), + waitForRestore = true) + assertBusy { + assertThat(followerClient.indices() + .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) + .isEqualTo(true) + } + Assert.assertEquals( + leaderClient.indices().getMapping(GetMappingsRequest().indices(leaderIndexName), RequestOptions.DEFAULT) + .mappings()[leaderIndexName], + followerClient.indices().getMapping(GetMappingsRequest().indices(followerIndexName), RequestOptions.DEFAULT) + .mappings()[followerIndexName] + ) + putMappingRequest = PutMappingRequest(leaderIndexName) + putMappingRequest.source("{\"properties\":{\"field1\":{\"type\":\"text\",\"fields\":{\"field2\":{\"type\":\"text\",\"analyzer\":\"standard\"},\"field3\":{\"type\":\"text\",\"analyzer\":\"standard\"}}}}}",XContentType.JSON) + leaderClient.indices().putMapping(putMappingRequest, RequestOptions.DEFAULT) + val leaderMappings = leaderClient.indices().getMapping(GetMappingsRequest().indices(leaderIndexName), RequestOptions.DEFAULT) + .mappings()[leaderIndexName] + TimeUnit.MINUTES.sleep(2) + Assert.assertEquals( + leaderMappings, + followerClient.indices().getMapping(GetMappingsRequest().indices(followerIndexName), RequestOptions.DEFAULT) + .mappings()[followerIndexName] + ) + } + + fun `test that follower index mapping does not update when only new fields are added but not respective docs in leader index`() { + val followerClient = getClientForCluster(FOLLOWER) + val leaderClient = getClientForCluster(LEADER) + createConnectionBetweenClusters(FOLLOWER, LEADER) + val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) + assertThat(createIndexResponse.isAcknowledged).isTrue() + var putMappingRequest = PutMappingRequest(leaderIndexName) + putMappingRequest.source("{\"properties\":{\"name\":{\"type\":\"text\"}}}", XContentType.JSON) + leaderClient.indices().putMapping(putMappingRequest, RequestOptions.DEFAULT) + val sourceMap = mapOf("name" to randomAlphaOfLength(5)) + leaderClient.index(IndexRequest(leaderIndexName).id("1").source(sourceMap), RequestOptions.DEFAULT) + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), + waitForRestore = true) + assertBusy { + assertThat(followerClient.indices() + .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) + .isEqualTo(true) + } + Assert.assertEquals( + leaderClient.indices().getMapping(GetMappingsRequest().indices(leaderIndexName), RequestOptions.DEFAULT) + .mappings()[leaderIndexName], + followerClient.indices().getMapping(GetMappingsRequest().indices(followerIndexName), RequestOptions.DEFAULT) + .mappings()[followerIndexName] + ) + putMappingRequest = PutMappingRequest(leaderIndexName) + putMappingRequest.source("{\"properties\":{\"name\":{\"type\":\"text\"},\"age\":{\"type\":\"integer\"}}}",XContentType.JSON) + leaderClient.indices().putMapping(putMappingRequest, RequestOptions.DEFAULT) + val leaderMappings = leaderClient.indices().getMapping(GetMappingsRequest().indices(leaderIndexName), RequestOptions.DEFAULT) + .mappings()[leaderIndexName] + TimeUnit.MINUTES.sleep(2) + Assert.assertNotEquals( + leaderMappings, + followerClient.indices().getMapping(GetMappingsRequest().indices(followerIndexName), RequestOptions.DEFAULT) + .mappings()[followerIndexName] + ) + } + private fun excludeAllClusterNodes(clusterName: String) { val transientSettingsRequest = Request("PUT", "_cluster/settings") From b17376db075e4ced92301b570e904fc6c63b2ab7 Mon Sep 17 00:00:00 2001 From: Ryan Bogan <10944539+ryanbogan@users.noreply.github.com> Date: Tue, 10 Jan 2023 20:16:45 -0800 Subject: [PATCH 060/157] Revert "Removed jackson-databind dependency from build-gradle that now comes from core" (#691) This reverts commit b08b58e67a9963c9f19ebf310a5b1776786b9401. Signed-off-by: Ryan Bogan Signed-off-by: Ryan Bogan --- build.gradle | 1 + 1 file changed, 1 insertion(+) diff --git a/build.gradle b/build.gradle index 71624134..b965e24d 100644 --- a/build.gradle +++ b/build.gradle @@ -115,6 +115,7 @@ configurations.all { force 'org.apache.httpcomponents.core5:httpcore5-h2:5.1.4' force 'org.apache.httpcomponents.client5:httpclient5:5.0.3' force 'org.apache.httpcomponents.client5:httpclient5-osgi:5.0.3' + force 'com.fasterxml.jackson.core:jackson-databind:2.13.4.2' force 'org.yaml:snakeyaml:1.32' force 'org.codehaus.plexus:plexus-utils:3.0.24' force 'net.java.dev.jna:jna:5.12.1' From cafc069dd6b83182cf306cfea19f72ea1966ccb5 Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Wed, 11 Jan 2023 10:26:52 +0530 Subject: [PATCH 061/157] Revert "Updating the maintainer list for the repo (#678)" (#681) This reverts commit b09c70656c94bbf9367af170262d0d8e97dac748. Signed-off-by: Ankit Kala Signed-off-by: Ankit Kala --- MAINTAINERS.md | 15 +++++---------- 1 file changed, 5 insertions(+), 10 deletions(-) diff --git a/MAINTAINERS.md b/MAINTAINERS.md index ee47b7dd..14f485be 100644 --- a/MAINTAINERS.md +++ b/MAINTAINERS.md @@ -6,16 +6,11 @@ This document contains a list of maintainers in this repo. See [opensearch-proje | Maintainer | GitHub ID | Affiliation | | ---------------- | --------------------------------------------------------- | ----------- | +| Rushi Agrawal | [rushiagr](https://github.com/rushiagr) | Amazon | | Gopala Krishna A | [krishna-ggk](https://github.com/krishna-ggk) | Amazon | -| Ankit Kala | [ankitkala](https://github.com/ankitkala) | Amazon | -| Saikumar Karanam | [saikaranam-amazon](https://github.com/saikaranam-amazon) | Amazon | +| Ankit | [ankitkala](https://github.com/ankitkala) | Amazon | +| Naveen | [naveenpajjuri](https://github.com/naveenpajjuri) | Amazon | +| Sai | [saikaranam-amazon](https://github.com/saikaranam-amazon) | Amazon | +| Bhanu Tadepalli | [tbhanu-amzn](https://github.com/tbhanu-amzn) | Amazon | | Sooraj Sinha | [soosinha](https://github.com/soosinha) | Amazon | -| Suraj Kumar | [suranjay](https://github.com/suranjay) | Amazon | | Gaurav Bafna | [gbbafna](https://github.com/gbbafna) | Amazon | -| Monu Singh | [monu-aws](https://github.com/monu-aws) | Amazon | -| Priyanka Sharma | [priyatsh](https://github.com/priyatsh) | Amazon | -| Mohit Kumar | [mohitamg](https://github.com/mohitamg) | Amazon | -| Nishant Goel | [nisgoel-amazon](https://github.com/nisgoel-amazon) | Amazon | -| Ishank Katiyar | [ishankka](https://github.com/ishankka) | Amazon | -| Shreyansh Ray | [rayshrey](https://github.com/rayshrey) | Amazon | -| Ranganathan S | [ranga011](https://github.com/ranga011) | Amazon | \ No newline at end of file From 5c48ca68694def14127f025a3a837fcc000e08c9 Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Fri, 3 Feb 2023 16:46:17 +0530 Subject: [PATCH 062/157] Build Fix: Update Strings.toString to pass the JSON XContentType (#699) Signed-off-by: Ankit Kala --- .../replication/task/index/IndexReplicationParams.kt | 3 ++- .../replication/task/shard/ShardReplicationParams.kt | 5 +++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationParams.kt b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationParams.kt index 07c7ee00..6bbb6646 100644 --- a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationParams.kt +++ b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationParams.kt @@ -20,6 +20,7 @@ import org.opensearch.common.xcontent.ObjectParser import org.opensearch.common.xcontent.ToXContent import org.opensearch.common.xcontent.XContentBuilder import org.opensearch.common.xcontent.XContentParser +import org.opensearch.common.xcontent.XContentType import org.opensearch.index.Index import org.opensearch.persistent.PersistentTaskParams import java.io.IOException @@ -78,6 +79,6 @@ class IndexReplicationParams : PersistentTaskParams { override fun getMinimalSupportedVersion(): Version = Version.V_2_0_0 override fun toString(): String { - return Strings.toString(this) + return Strings.toString(XContentType.JSON, this) } } diff --git a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationParams.kt b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationParams.kt index 9dfad901..9ae3cf58 100644 --- a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationParams.kt +++ b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationParams.kt @@ -20,6 +20,7 @@ import org.opensearch.common.xcontent.ObjectParser import org.opensearch.common.xcontent.ToXContent import org.opensearch.common.xcontent.XContentBuilder import org.opensearch.common.xcontent.XContentParser +import org.opensearch.common.xcontent.XContentType import org.opensearch.index.shard.ShardId import org.opensearch.persistent.PersistentTaskParams import java.io.IOException @@ -82,7 +83,7 @@ class ShardReplicationParams : PersistentTaskParams { override fun getMinimalSupportedVersion(): Version = Version.V_2_0_0 override fun toString(): String { - return Strings.toString(this) + return Strings.toString(XContentType.JSON, this) } class Builder { @@ -120,4 +121,4 @@ class ShardReplicationParams : PersistentTaskParams { followerShardIdObj.id)) } } -} \ No newline at end of file +} From 8f6141add2dda411b39a246683522500ade1d103 Mon Sep 17 00:00:00 2001 From: "Daniel (dB.) Doubrovkine" Date: Mon, 20 Feb 2023 23:28:19 -0500 Subject: [PATCH 063/157] Created untriaged issue workflow. (#705) Signed-off-by: dblock --- .github/workflows/add-untriaged.yml | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) create mode 100644 .github/workflows/add-untriaged.yml diff --git a/.github/workflows/add-untriaged.yml b/.github/workflows/add-untriaged.yml new file mode 100644 index 00000000..9dcc7020 --- /dev/null +++ b/.github/workflows/add-untriaged.yml @@ -0,0 +1,19 @@ +name: Apply 'untriaged' label during issue lifecycle + +on: + issues: + types: [opened, reopened, transferred] + +jobs: + apply-label: + runs-on: ubuntu-latest + steps: + - uses: actions/github-script@v6 + with: + script: | + github.rest.issues.addLabels({ + issue_number: context.issue.number, + owner: context.repo.owner, + repo: context.repo.repo, + labels: ['untriaged'] + }) From 1a470d0b2d021ead0bcf6b6493780d80f4c234d7 Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Tue, 21 Feb 2023 12:38:56 +0530 Subject: [PATCH 064/157] Add github workflow for auto release (#712) Signed-off-by: Ankit Kala --- .github/workflows/auto-release.yml | 29 +++++++++++++++++++++++++++++ 1 file changed, 29 insertions(+) create mode 100644 .github/workflows/auto-release.yml diff --git a/.github/workflows/auto-release.yml b/.github/workflows/auto-release.yml new file mode 100644 index 00000000..24eeb273 --- /dev/null +++ b/.github/workflows/auto-release.yml @@ -0,0 +1,29 @@ +name: Releases + +on: + push: + tags: + - '*' + +jobs: + + build: + runs-on: ubuntu-latest + permissions: + contents: write + steps: + - name: GitHub App token + id: github_app_token + uses: tibdex/github-app-token@v1.5.0 + with: + app_id: ${{ secrets.APP_ID }} + private_key: ${{ secrets.APP_PRIVATE_KEY }} + installation_id: 22958780 + - name: Get tag + id: tag + uses: dawidd6/action-get-tag@v1 + - uses: actions/checkout@v2 + - uses: ncipollo/release-action@v1 + with: + github_token: ${{ steps.github_app_token.outputs.token }} + bodyFile: release-notes/opensearch.release-notes-${{steps.tag.outputs.tag}}.md \ No newline at end of file From c3fddb6f2b42ae5659e28de5a69789edd089635c Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Tue, 28 Feb 2023 17:35:51 +0530 Subject: [PATCH 065/157] Update release notes file path in auto release workflow (#726) Signed-off-by: Ankit Kala --- .github/workflows/auto-release.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/auto-release.yml b/.github/workflows/auto-release.yml index 24eeb273..5e6ac8c3 100644 --- a/.github/workflows/auto-release.yml +++ b/.github/workflows/auto-release.yml @@ -26,4 +26,4 @@ jobs: - uses: ncipollo/release-action@v1 with: github_token: ${{ steps.github_app_token.outputs.token }} - bodyFile: release-notes/opensearch.release-notes-${{steps.tag.outputs.tag}}.md \ No newline at end of file + bodyFile: release-notes/opensearch-cross-cluster-replication.release-notes-${{steps.tag.outputs.tag}}.md \ No newline at end of file From d29f88607269eb528588228acdf0857f0c49d397 Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Wed, 1 Mar 2023 13:49:40 +0530 Subject: [PATCH 066/157] Update imports from org.opensearch.common.xcontent to org.opensearch.core.xcontent (#718) Signed-off-by: Ankit Kala --- .../opensearch/replication/ReplicationPlugin.kt | 6 +++--- .../AutoFollowClusterManagerNodeRequest.kt | 6 +++--- .../autofollow/UpdateAutoFollowPatternRequest.kt | 12 ++++++------ .../ReplicateIndexClusterManagerNodeRequest.kt | 6 +++--- .../action/index/ReplicateIndexRequest.kt | 14 +++++++------- .../action/index/block/UpdateIndexBlockRequest.kt | 10 +++++----- .../action/pause/PauseIndexReplicationRequest.kt | 12 ++++++------ .../action/resume/ResumeIndexReplicationRequest.kt | 2 +- .../replication/action/setup/SetupChecksRequest.kt | 6 +++--- .../action/setup/ValidatePermissionsRequest.kt | 6 +++--- .../action/stats/AutoFollowStatsAction.kt | 8 ++++---- .../action/stats/FollowerStatsResponse.kt | 8 ++++---- .../action/stats/LeaderStatsResponse.kt | 8 ++++---- .../action/status/ReplicationStatusResponse.kt | 6 +++--- .../replication/action/status/ShardInfoRequest.kt | 6 +++--- .../replication/action/status/ShardInfoResponse.kt | 8 ++++---- .../action/stop/StopIndexReplicationRequest.kt | 4 ++-- .../action/update/UpdateIndexReplicationRequest.kt | 4 ++-- .../metadata/state/ReplicationStateMetadata.kt | 6 +++--- .../metadata/store/ReplicationMetadata.kt | 10 +++++----- .../metadata/store/ReplicationMetadataStore.kt | 8 +++++++- .../replication/rest/AutoFollowStatsHandler.kt | 4 ++-- .../replication/rest/FollowerStatsHandler.kt | 4 ++-- .../replication/rest/LeaderStatsHandler.kt | 4 ++-- .../replication/seqno/RemoteClusterStats.kt | 8 ++++---- .../task/CrossClusterReplicationTask.kt | 6 +++--- .../replication/task/ReplicationState.kt | 6 +++--- .../task/autofollow/AutoFollowParams.kt | 10 +++++----- .../replication/task/autofollow/AutoFollowTask.kt | 4 ++-- .../task/index/IndexReplicationParams.kt | 10 +++++----- .../task/index/IndexReplicationState.kt | 10 +++++----- .../replication/task/index/IndexReplicationTask.kt | 6 +++--- .../replication/task/shard/FollowerClusterStats.kt | 6 +++--- .../task/shard/ShardReplicationParams.kt | 10 +++++----- .../task/shard/ShardReplicationState.kt | 10 +++++----- .../replication/MultiClusterRestTestCase.kt | 5 ++--- .../opensearch/replication/ReplicationHelpers.kt | 4 ++-- .../replication/integ/rest/StartReplicationIT.kt | 4 ++-- .../task/index/IndexReplicationTaskTests.kt | 2 +- .../replication/task/index/NoOpClient.kt | 2 +- 40 files changed, 138 insertions(+), 133 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt b/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt index 53c77866..5a0dddfb 100644 --- a/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt +++ b/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt @@ -82,7 +82,7 @@ import org.opensearch.cluster.metadata.RepositoryMetadata import org.opensearch.cluster.node.DiscoveryNodes import org.opensearch.cluster.service.ClusterService import org.opensearch.common.CheckedFunction -import org.opensearch.common.ParseField +import org.opensearch.core.ParseField import org.opensearch.common.component.LifecycleComponent import org.opensearch.common.io.stream.NamedWriteableRegistry import org.opensearch.common.io.stream.Writeable @@ -96,8 +96,8 @@ import org.opensearch.common.unit.ByteSizeUnit import org.opensearch.common.unit.ByteSizeValue import org.opensearch.common.unit.TimeValue import org.opensearch.common.util.concurrent.OpenSearchExecutors -import org.opensearch.common.xcontent.NamedXContentRegistry -import org.opensearch.common.xcontent.XContentParser +import org.opensearch.core.xcontent.NamedXContentRegistry +import org.opensearch.core.xcontent.XContentParser import org.opensearch.commons.utils.OpenForTesting import org.opensearch.env.Environment import org.opensearch.env.NodeEnvironment diff --git a/src/main/kotlin/org/opensearch/replication/action/autofollow/AutoFollowClusterManagerNodeRequest.kt b/src/main/kotlin/org/opensearch/replication/action/autofollow/AutoFollowClusterManagerNodeRequest.kt index 80c05995..f9199ea0 100644 --- a/src/main/kotlin/org/opensearch/replication/action/autofollow/AutoFollowClusterManagerNodeRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/autofollow/AutoFollowClusterManagerNodeRequest.kt @@ -16,9 +16,9 @@ import org.opensearch.action.ActionRequestValidationException import org.opensearch.action.support.master.MasterNodeRequest import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.ToXContentObject -import org.opensearch.common.xcontent.XContentBuilder +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.ToXContentObject +import org.opensearch.core.xcontent.XContentBuilder class AutoFollowClusterManagerNodeRequest: MasterNodeRequest, ToXContentObject { var user: User? = null diff --git a/src/main/kotlin/org/opensearch/replication/action/autofollow/UpdateAutoFollowPatternRequest.kt b/src/main/kotlin/org/opensearch/replication/action/autofollow/UpdateAutoFollowPatternRequest.kt index 30d16b29..a7077793 100644 --- a/src/main/kotlin/org/opensearch/replication/action/autofollow/UpdateAutoFollowPatternRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/autofollow/UpdateAutoFollowPatternRequest.kt @@ -16,15 +16,15 @@ import org.opensearch.replication.metadata.store.KEY_SETTINGS import org.opensearch.replication.util.ValidationUtil.validateName import org.opensearch.action.ActionRequestValidationException import org.opensearch.action.support.master.AcknowledgedRequest -import org.opensearch.common.ParseField +import org.opensearch.core.ParseField import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput import org.opensearch.common.settings.Settings -import org.opensearch.common.xcontent.ObjectParser -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.ToXContentObject -import org.opensearch.common.xcontent.XContentBuilder -import org.opensearch.common.xcontent.XContentParser +import org.opensearch.core.xcontent.ObjectParser +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.ToXContentObject +import org.opensearch.core.xcontent.XContentBuilder +import org.opensearch.core.xcontent.XContentParser import java.util.Collections import java.util.function.BiConsumer import java.util.function.BiFunction diff --git a/src/main/kotlin/org/opensearch/replication/action/index/ReplicateIndexClusterManagerNodeRequest.kt b/src/main/kotlin/org/opensearch/replication/action/index/ReplicateIndexClusterManagerNodeRequest.kt index 2c06b6ca..63f77023 100644 --- a/src/main/kotlin/org/opensearch/replication/action/index/ReplicateIndexClusterManagerNodeRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/index/ReplicateIndexClusterManagerNodeRequest.kt @@ -16,9 +16,9 @@ import org.opensearch.action.ActionRequestValidationException import org.opensearch.action.support.master.MasterNodeRequest import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.ToXContentObject -import org.opensearch.common.xcontent.XContentBuilder +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.ToXContentObject +import org.opensearch.core.xcontent.XContentBuilder class ReplicateIndexClusterManagerNodeRequest: MasterNodeRequest, ToXContentObject { diff --git a/src/main/kotlin/org/opensearch/replication/action/index/ReplicateIndexRequest.kt b/src/main/kotlin/org/opensearch/replication/action/index/ReplicateIndexRequest.kt index 894996db..bb0f9aaf 100644 --- a/src/main/kotlin/org/opensearch/replication/action/index/ReplicateIndexRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/index/ReplicateIndexRequest.kt @@ -17,16 +17,16 @@ import org.opensearch.action.ActionRequestValidationException import org.opensearch.action.IndicesRequest import org.opensearch.action.support.IndicesOptions import org.opensearch.action.support.master.AcknowledgedRequest -import org.opensearch.common.ParseField +import org.opensearch.core.ParseField import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput import org.opensearch.common.settings.Settings -import org.opensearch.common.xcontent.ObjectParser -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.ToXContent.Params -import org.opensearch.common.xcontent.ToXContentObject -import org.opensearch.common.xcontent.XContentBuilder -import org.opensearch.common.xcontent.XContentParser +import org.opensearch.core.xcontent.ObjectParser +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.ToXContent.Params +import org.opensearch.core.xcontent.ToXContentObject +import org.opensearch.core.xcontent.XContentBuilder +import org.opensearch.core.xcontent.XContentParser import java.io.IOException import java.util.Collections import java.util.function.BiConsumer diff --git a/src/main/kotlin/org/opensearch/replication/action/index/block/UpdateIndexBlockRequest.kt b/src/main/kotlin/org/opensearch/replication/action/index/block/UpdateIndexBlockRequest.kt index 12eade15..d9b51933 100644 --- a/src/main/kotlin/org/opensearch/replication/action/index/block/UpdateIndexBlockRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/index/block/UpdateIndexBlockRequest.kt @@ -15,13 +15,13 @@ import org.opensearch.action.ActionRequestValidationException import org.opensearch.action.IndicesRequest import org.opensearch.action.support.IndicesOptions import org.opensearch.action.support.master.AcknowledgedRequest -import org.opensearch.common.ParseField +import org.opensearch.core.ParseField import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.common.xcontent.ObjectParser -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.ToXContentObject -import org.opensearch.common.xcontent.XContentBuilder +import org.opensearch.core.xcontent.ObjectParser +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.ToXContentObject +import org.opensearch.core.xcontent.XContentBuilder import java.util.function.Supplier enum class IndexBlockUpdateType { diff --git a/src/main/kotlin/org/opensearch/replication/action/pause/PauseIndexReplicationRequest.kt b/src/main/kotlin/org/opensearch/replication/action/pause/PauseIndexReplicationRequest.kt index a1e6bf8f..e69dcb44 100644 --- a/src/main/kotlin/org/opensearch/replication/action/pause/PauseIndexReplicationRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/pause/PauseIndexReplicationRequest.kt @@ -16,14 +16,14 @@ import org.opensearch.action.ActionRequestValidationException import org.opensearch.action.IndicesRequest import org.opensearch.action.support.IndicesOptions import org.opensearch.action.support.master.AcknowledgedRequest -import org.opensearch.common.ParseField +import org.opensearch.core.ParseField import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.common.xcontent.ObjectParser -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.ToXContentObject -import org.opensearch.common.xcontent.XContentBuilder -import org.opensearch.common.xcontent.XContentParser +import org.opensearch.core.xcontent.ObjectParser +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.ToXContentObject +import org.opensearch.core.xcontent.XContentBuilder +import org.opensearch.core.xcontent.XContentParser class PauseIndexReplicationRequest : AcknowledgedRequest, IndicesRequest.Replaceable, ToXContentObject { diff --git a/src/main/kotlin/org/opensearch/replication/action/resume/ResumeIndexReplicationRequest.kt b/src/main/kotlin/org/opensearch/replication/action/resume/ResumeIndexReplicationRequest.kt index 9ebef602..dd219272 100644 --- a/src/main/kotlin/org/opensearch/replication/action/resume/ResumeIndexReplicationRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/resume/ResumeIndexReplicationRequest.kt @@ -17,7 +17,7 @@ import org.opensearch.action.support.IndicesOptions import org.opensearch.action.support.master.AcknowledgedRequest import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.common.xcontent.* +import org.opensearch.core.xcontent.* class ResumeIndexReplicationRequest : AcknowledgedRequest, IndicesRequest.Replaceable, ToXContentObject { diff --git a/src/main/kotlin/org/opensearch/replication/action/setup/SetupChecksRequest.kt b/src/main/kotlin/org/opensearch/replication/action/setup/SetupChecksRequest.kt index 4d14d10f..57b9fffb 100644 --- a/src/main/kotlin/org/opensearch/replication/action/setup/SetupChecksRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/setup/SetupChecksRequest.kt @@ -16,9 +16,9 @@ import org.opensearch.action.ActionRequestValidationException import org.opensearch.action.support.master.AcknowledgedRequest import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.ToXContentObject -import org.opensearch.common.xcontent.XContentBuilder +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.ToXContentObject +import org.opensearch.core.xcontent.XContentBuilder class SetupChecksRequest: AcknowledgedRequest, ToXContentObject { val followerContext: ReplicationContext diff --git a/src/main/kotlin/org/opensearch/replication/action/setup/ValidatePermissionsRequest.kt b/src/main/kotlin/org/opensearch/replication/action/setup/ValidatePermissionsRequest.kt index 1b51d95d..430fe9b8 100644 --- a/src/main/kotlin/org/opensearch/replication/action/setup/ValidatePermissionsRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/setup/ValidatePermissionsRequest.kt @@ -17,9 +17,9 @@ import org.opensearch.action.support.IndicesOptions import org.opensearch.action.support.master.AcknowledgedRequest import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.ToXContentObject -import org.opensearch.common.xcontent.XContentBuilder +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.ToXContentObject +import org.opensearch.core.xcontent.XContentBuilder class ValidatePermissionsRequest: AcknowledgedRequest, IndicesRequest.Replaceable, ToXContentObject { val cluster: String diff --git a/src/main/kotlin/org/opensearch/replication/action/stats/AutoFollowStatsAction.kt b/src/main/kotlin/org/opensearch/replication/action/stats/AutoFollowStatsAction.kt index 678b6449..0878d377 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stats/AutoFollowStatsAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stats/AutoFollowStatsAction.kt @@ -18,10 +18,10 @@ import org.opensearch.action.support.tasks.BaseTasksResponse import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput import org.opensearch.common.io.stream.Writeable -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.ToXContent.EMPTY_PARAMS -import org.opensearch.common.xcontent.ToXContentObject -import org.opensearch.common.xcontent.XContentBuilder +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.ToXContent.EMPTY_PARAMS +import org.opensearch.core.xcontent.ToXContentObject +import org.opensearch.core.xcontent.XContentBuilder import org.opensearch.replication.task.autofollow.AutoFollowStat import java.io.IOException diff --git a/src/main/kotlin/org/opensearch/replication/action/stats/FollowerStatsResponse.kt b/src/main/kotlin/org/opensearch/replication/action/stats/FollowerStatsResponse.kt index 3b405bde..53271fe4 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stats/FollowerStatsResponse.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stats/FollowerStatsResponse.kt @@ -19,10 +19,10 @@ import org.opensearch.cluster.ClusterName import org.opensearch.common.Strings import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.common.xcontent.ToXContent.EMPTY_PARAMS -import org.opensearch.common.xcontent.ToXContent.Params -import org.opensearch.common.xcontent.ToXContentObject -import org.opensearch.common.xcontent.XContentBuilder +import org.opensearch.core.xcontent.ToXContent.EMPTY_PARAMS +import org.opensearch.core.xcontent.ToXContent.Params +import org.opensearch.core.xcontent.ToXContentObject +import org.opensearch.core.xcontent.XContentBuilder import org.opensearch.common.xcontent.XContentFactory import org.opensearch.index.shard.ShardId import org.opensearch.replication.metadata.ReplicationOverallState diff --git a/src/main/kotlin/org/opensearch/replication/action/stats/LeaderStatsResponse.kt b/src/main/kotlin/org/opensearch/replication/action/stats/LeaderStatsResponse.kt index 235331d5..47333152 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stats/LeaderStatsResponse.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stats/LeaderStatsResponse.kt @@ -19,10 +19,10 @@ import org.opensearch.cluster.ClusterName import org.opensearch.common.Strings import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.common.xcontent.ToXContent.EMPTY_PARAMS -import org.opensearch.common.xcontent.ToXContent.Params -import org.opensearch.common.xcontent.ToXContentObject -import org.opensearch.common.xcontent.XContentBuilder +import org.opensearch.core.xcontent.ToXContent.EMPTY_PARAMS +import org.opensearch.core.xcontent.ToXContent.Params +import org.opensearch.core.xcontent.ToXContentObject +import org.opensearch.core.xcontent.XContentBuilder import org.opensearch.common.xcontent.XContentFactory import org.opensearch.replication.seqno.RemoteShardMetric import org.opensearch.replication.seqno.RemoteShardMetric.RemoteStats diff --git a/src/main/kotlin/org/opensearch/replication/action/status/ReplicationStatusResponse.kt b/src/main/kotlin/org/opensearch/replication/action/status/ReplicationStatusResponse.kt index b7b6fa73..a4832381 100644 --- a/src/main/kotlin/org/opensearch/replication/action/status/ReplicationStatusResponse.kt +++ b/src/main/kotlin/org/opensearch/replication/action/status/ReplicationStatusResponse.kt @@ -16,9 +16,9 @@ import org.opensearch.action.support.DefaultShardOperationFailedException import org.opensearch.action.support.broadcast.BroadcastResponse import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.common.xcontent.ToXContent.Params -import org.opensearch.common.xcontent.ToXContentObject -import org.opensearch.common.xcontent.XContentBuilder +import org.opensearch.core.xcontent.ToXContent.Params +import org.opensearch.core.xcontent.ToXContentObject +import org.opensearch.core.xcontent.XContentBuilder import java.io.IOException class ReplicationStatusResponse : BroadcastResponse, ToXContentObject { diff --git a/src/main/kotlin/org/opensearch/replication/action/status/ShardInfoRequest.kt b/src/main/kotlin/org/opensearch/replication/action/status/ShardInfoRequest.kt index 23549470..024dd976 100644 --- a/src/main/kotlin/org/opensearch/replication/action/status/ShardInfoRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/status/ShardInfoRequest.kt @@ -16,9 +16,9 @@ import org.opensearch.action.support.IndicesOptions import org.opensearch.action.support.broadcast.BroadcastRequest import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.ToXContentObject -import org.opensearch.common.xcontent.XContentBuilder +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.ToXContentObject +import org.opensearch.core.xcontent.XContentBuilder class ShardInfoRequest : BroadcastRequest , ToXContentObject { diff --git a/src/main/kotlin/org/opensearch/replication/action/status/ShardInfoResponse.kt b/src/main/kotlin/org/opensearch/replication/action/status/ShardInfoResponse.kt index 4cadea82..af111889 100644 --- a/src/main/kotlin/org/opensearch/replication/action/status/ShardInfoResponse.kt +++ b/src/main/kotlin/org/opensearch/replication/action/status/ShardInfoResponse.kt @@ -13,12 +13,12 @@ package org.opensearch.replication.action.status import org.opensearch.action.support.broadcast.BroadcastResponse import org.opensearch.action.support.broadcast.BroadcastShardResponse -import org.opensearch.common.ParseField +import org.opensearch.core.ParseField import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.ToXContentObject -import org.opensearch.common.xcontent.XContentBuilder +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.ToXContentObject +import org.opensearch.core.xcontent.XContentBuilder import org.opensearch.index.shard.ShardId import java.io.IOException diff --git a/src/main/kotlin/org/opensearch/replication/action/stop/StopIndexReplicationRequest.kt b/src/main/kotlin/org/opensearch/replication/action/stop/StopIndexReplicationRequest.kt index 32d44b7a..2f447eb8 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stop/StopIndexReplicationRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stop/StopIndexReplicationRequest.kt @@ -15,10 +15,10 @@ import org.opensearch.action.ActionRequestValidationException import org.opensearch.action.IndicesRequest import org.opensearch.action.support.IndicesOptions import org.opensearch.action.support.master.AcknowledgedRequest -import org.opensearch.common.ParseField +import org.opensearch.core.ParseField import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.common.xcontent.* +import org.opensearch.core.xcontent.* class StopIndexReplicationRequest : AcknowledgedRequest, IndicesRequest.Replaceable, ToXContentObject { diff --git a/src/main/kotlin/org/opensearch/replication/action/update/UpdateIndexReplicationRequest.kt b/src/main/kotlin/org/opensearch/replication/action/update/UpdateIndexReplicationRequest.kt index 1dde6ddc..753e2f62 100644 --- a/src/main/kotlin/org/opensearch/replication/action/update/UpdateIndexReplicationRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/update/UpdateIndexReplicationRequest.kt @@ -15,12 +15,12 @@ import org.opensearch.action.ActionRequestValidationException import org.opensearch.action.IndicesRequest import org.opensearch.action.support.IndicesOptions import org.opensearch.action.support.master.AcknowledgedRequest -import org.opensearch.common.ParseField +import org.opensearch.core.ParseField import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput import org.opensearch.common.settings.Settings import org.opensearch.common.settings.Settings.readSettingsFromStream -import org.opensearch.common.xcontent.* +import org.opensearch.core.xcontent.* import java.io.IOException import java.util.* diff --git a/src/main/kotlin/org/opensearch/replication/metadata/state/ReplicationStateMetadata.kt b/src/main/kotlin/org/opensearch/replication/metadata/state/ReplicationStateMetadata.kt index b6407be7..86711ac0 100644 --- a/src/main/kotlin/org/opensearch/replication/metadata/state/ReplicationStateMetadata.kt +++ b/src/main/kotlin/org/opensearch/replication/metadata/state/ReplicationStateMetadata.kt @@ -20,9 +20,9 @@ import org.opensearch.cluster.metadata.Metadata import org.opensearch.cluster.service.ClusterService import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.XContentBuilder -import org.opensearch.common.xcontent.XContentParser +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.XContentBuilder +import org.opensearch.core.xcontent.XContentParser import java.io.IOException import java.util.EnumSet import kotlin.collections.HashMap diff --git a/src/main/kotlin/org/opensearch/replication/metadata/store/ReplicationMetadata.kt b/src/main/kotlin/org/opensearch/replication/metadata/store/ReplicationMetadata.kt index 892a86a9..86cd80c2 100644 --- a/src/main/kotlin/org/opensearch/replication/metadata/store/ReplicationMetadata.kt +++ b/src/main/kotlin/org/opensearch/replication/metadata/store/ReplicationMetadata.kt @@ -12,15 +12,15 @@ package org.opensearch.replication.metadata.store import org.opensearch.commons.authuser.User -import org.opensearch.common.ParseField +import org.opensearch.core.ParseField import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput import org.opensearch.common.io.stream.Writeable import org.opensearch.common.settings.Settings -import org.opensearch.common.xcontent.ObjectParser -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.XContentBuilder -import org.opensearch.common.xcontent.XContentParser +import org.opensearch.core.xcontent.ObjectParser +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.XContentBuilder +import org.opensearch.core.xcontent.XContentParser import java.io.IOException import java.util.* import java.util.function.BiConsumer diff --git a/src/main/kotlin/org/opensearch/replication/metadata/store/ReplicationMetadataStore.kt b/src/main/kotlin/org/opensearch/replication/metadata/store/ReplicationMetadataStore.kt index 0ce5683f..ba0122ee 100644 --- a/src/main/kotlin/org/opensearch/replication/metadata/store/ReplicationMetadataStore.kt +++ b/src/main/kotlin/org/opensearch/replication/metadata/store/ReplicationMetadataStore.kt @@ -33,7 +33,13 @@ import org.opensearch.cluster.service.ClusterService import org.opensearch.common.component.AbstractLifecycleComponent import org.opensearch.common.settings.Settings import org.opensearch.common.util.concurrent.ThreadContext -import org.opensearch.common.xcontent.* +import org.opensearch.common.xcontent.XContentType +import org.opensearch.common.xcontent.XContentFactory +import org.opensearch.common.xcontent.XContentHelper +import org.opensearch.common.xcontent.LoggingDeprecationHandler +import org.opensearch.core.xcontent.NamedXContentRegistry +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.XContentParser import org.opensearch.replication.util.suspendExecuteWithRetries class ReplicationMetadataStore constructor(val client: Client, val clusterService: ClusterService, diff --git a/src/main/kotlin/org/opensearch/replication/rest/AutoFollowStatsHandler.kt b/src/main/kotlin/org/opensearch/replication/rest/AutoFollowStatsHandler.kt index 2f09c4b1..42421eb4 100644 --- a/src/main/kotlin/org/opensearch/replication/rest/AutoFollowStatsHandler.kt +++ b/src/main/kotlin/org/opensearch/replication/rest/AutoFollowStatsHandler.kt @@ -2,8 +2,8 @@ package org.opensearch.replication.rest import org.apache.logging.log4j.LogManager import org.opensearch.client.node.NodeClient -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.XContentBuilder +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.XContentBuilder import org.opensearch.common.xcontent.XContentFactory import org.opensearch.replication.action.stats.AutoFollowStatsAction import org.opensearch.replication.action.stats.AutoFollowStatsRequest diff --git a/src/main/kotlin/org/opensearch/replication/rest/FollowerStatsHandler.kt b/src/main/kotlin/org/opensearch/replication/rest/FollowerStatsHandler.kt index 12026466..ce5013f6 100644 --- a/src/main/kotlin/org/opensearch/replication/rest/FollowerStatsHandler.kt +++ b/src/main/kotlin/org/opensearch/replication/rest/FollowerStatsHandler.kt @@ -2,8 +2,8 @@ package org.opensearch.replication.rest import org.apache.logging.log4j.LogManager import org.opensearch.client.node.NodeClient -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.XContentBuilder +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.XContentBuilder import org.opensearch.common.xcontent.XContentFactory import org.opensearch.replication.action.stats.FollowerStatsAction import org.opensearch.replication.action.stats.FollowerStatsRequest diff --git a/src/main/kotlin/org/opensearch/replication/rest/LeaderStatsHandler.kt b/src/main/kotlin/org/opensearch/replication/rest/LeaderStatsHandler.kt index 98a68c27..d71379bf 100644 --- a/src/main/kotlin/org/opensearch/replication/rest/LeaderStatsHandler.kt +++ b/src/main/kotlin/org/opensearch/replication/rest/LeaderStatsHandler.kt @@ -2,8 +2,8 @@ package org.opensearch.replication.rest import org.apache.logging.log4j.LogManager import org.opensearch.client.node.NodeClient -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.XContentBuilder +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.XContentBuilder import org.opensearch.common.xcontent.XContentFactory import org.opensearch.replication.action.stats.LeaderStatsAction import org.opensearch.replication.action.stats.LeaderStatsRequest diff --git a/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterStats.kt b/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterStats.kt index d6635288..cc565d03 100644 --- a/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterStats.kt +++ b/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterStats.kt @@ -15,10 +15,10 @@ import org.opensearch.common.component.AbstractLifecycleComponent import org.opensearch.common.inject.Singleton import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.ToXContentFragment -import org.opensearch.common.xcontent.ToXContentObject -import org.opensearch.common.xcontent.XContentBuilder +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.ToXContentFragment +import org.opensearch.core.xcontent.ToXContentObject +import org.opensearch.core.xcontent.XContentBuilder import org.opensearch.index.shard.ShardId import java.util.concurrent.atomic.AtomicLong diff --git a/src/main/kotlin/org/opensearch/replication/task/CrossClusterReplicationTask.kt b/src/main/kotlin/org/opensearch/replication/task/CrossClusterReplicationTask.kt index 933be175..75477625 100644 --- a/src/main/kotlin/org/opensearch/replication/task/CrossClusterReplicationTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/CrossClusterReplicationTask.kt @@ -36,9 +36,9 @@ import org.opensearch.client.Client import org.opensearch.cluster.service.ClusterService import org.opensearch.common.io.stream.StreamOutput import org.opensearch.common.settings.Settings -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.ToXContentObject -import org.opensearch.common.xcontent.XContentBuilder +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.ToXContentObject +import org.opensearch.core.xcontent.XContentBuilder import org.opensearch.index.IndexService import org.opensearch.index.shard.IndexShard import org.opensearch.index.shard.ShardId diff --git a/src/main/kotlin/org/opensearch/replication/task/ReplicationState.kt b/src/main/kotlin/org/opensearch/replication/task/ReplicationState.kt index 57c41f64..3a81f74e 100644 --- a/src/main/kotlin/org/opensearch/replication/task/ReplicationState.kt +++ b/src/main/kotlin/org/opensearch/replication/task/ReplicationState.kt @@ -14,9 +14,9 @@ package org.opensearch.replication.task import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput import org.opensearch.common.io.stream.Writeable -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.ToXContentFragment -import org.opensearch.common.xcontent.XContentBuilder +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.ToXContentFragment +import org.opensearch.core.xcontent.XContentBuilder /** * Enum that represents the state of replication of either shards or indices. diff --git a/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowParams.kt b/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowParams.kt index bc936410..c5b2b8bc 100644 --- a/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowParams.kt +++ b/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowParams.kt @@ -12,13 +12,13 @@ package org.opensearch.replication.task.autofollow import org.opensearch.Version -import org.opensearch.common.ParseField +import org.opensearch.core.ParseField import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.common.xcontent.ObjectParser -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.XContentBuilder -import org.opensearch.common.xcontent.XContentParser +import org.opensearch.core.xcontent.ObjectParser +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.XContentBuilder +import org.opensearch.core.xcontent.XContentParser import org.opensearch.persistent.PersistentTaskParams import java.io.IOException diff --git a/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowTask.kt b/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowTask.kt index 0685b79d..6b1db867 100644 --- a/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowTask.kt @@ -32,8 +32,8 @@ import org.opensearch.cluster.service.ClusterService import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput import org.opensearch.common.logging.Loggers -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.XContentBuilder +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.XContentBuilder import org.opensearch.persistent.PersistentTaskState import org.opensearch.replication.ReplicationException import org.opensearch.replication.action.status.ReplicationStatusAction diff --git a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationParams.kt b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationParams.kt index 6bbb6646..6541e041 100644 --- a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationParams.kt +++ b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationParams.kt @@ -12,14 +12,14 @@ package org.opensearch.replication.task.index import org.opensearch.Version -import org.opensearch.common.ParseField +import org.opensearch.core.ParseField import org.opensearch.common.Strings import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.common.xcontent.ObjectParser -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.XContentBuilder -import org.opensearch.common.xcontent.XContentParser +import org.opensearch.core.xcontent.ObjectParser +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.XContentBuilder +import org.opensearch.core.xcontent.XContentParser import org.opensearch.common.xcontent.XContentType import org.opensearch.index.Index import org.opensearch.persistent.PersistentTaskParams diff --git a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationState.kt b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationState.kt index 1ea7afaf..010d1447 100644 --- a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationState.kt +++ b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationState.kt @@ -13,13 +13,13 @@ package org.opensearch.replication.task.index import org.opensearch.replication.task.ReplicationState import org.opensearch.replication.task.shard.ShardReplicationParams -import org.opensearch.common.ParseField +import org.opensearch.core.ParseField import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.common.xcontent.ObjectParser -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.XContentBuilder -import org.opensearch.common.xcontent.XContentParser +import org.opensearch.core.xcontent.ObjectParser +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.XContentBuilder +import org.opensearch.core.xcontent.XContentParser import org.opensearch.index.shard.ShardId import org.opensearch.persistent.PersistentTaskState import org.opensearch.persistent.PersistentTasksCustomMetadata.PersistentTask diff --git a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt index cbaa8ec5..e7b668ed 100644 --- a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt @@ -75,9 +75,9 @@ import org.opensearch.common.settings.Settings import org.opensearch.common.settings.SettingsModule import org.opensearch.common.unit.ByteSizeUnit import org.opensearch.common.unit.ByteSizeValue -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.ToXContentObject -import org.opensearch.common.xcontent.XContentBuilder +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.ToXContentObject +import org.opensearch.core.xcontent.XContentBuilder import org.opensearch.common.xcontent.XContentType import org.opensearch.index.Index import org.opensearch.index.IndexService diff --git a/src/main/kotlin/org/opensearch/replication/task/shard/FollowerClusterStats.kt b/src/main/kotlin/org/opensearch/replication/task/shard/FollowerClusterStats.kt index 09a0bc60..db112a1f 100644 --- a/src/main/kotlin/org/opensearch/replication/task/shard/FollowerClusterStats.kt +++ b/src/main/kotlin/org/opensearch/replication/task/shard/FollowerClusterStats.kt @@ -15,9 +15,9 @@ import org.apache.logging.log4j.LogManager import org.opensearch.common.inject.Singleton import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.ToXContentFragment -import org.opensearch.common.xcontent.XContentBuilder +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.ToXContentFragment +import org.opensearch.core.xcontent.XContentBuilder import org.opensearch.common.xcontent.XContentType import org.opensearch.index.shard.ShardId import java.util.concurrent.atomic.AtomicLong diff --git a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationParams.kt b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationParams.kt index 9ae3cf58..357eb5b6 100644 --- a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationParams.kt +++ b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationParams.kt @@ -12,14 +12,14 @@ package org.opensearch.replication.task.shard import org.opensearch.Version -import org.opensearch.common.ParseField +import org.opensearch.core.ParseField import org.opensearch.common.Strings import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.common.xcontent.ObjectParser -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.XContentBuilder -import org.opensearch.common.xcontent.XContentParser +import org.opensearch.core.xcontent.ObjectParser +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.XContentBuilder +import org.opensearch.core.xcontent.XContentParser import org.opensearch.common.xcontent.XContentType import org.opensearch.index.shard.ShardId import org.opensearch.persistent.PersistentTaskParams diff --git a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationState.kt b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationState.kt index 52849d92..f0c3fc88 100644 --- a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationState.kt +++ b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationState.kt @@ -13,13 +13,13 @@ package org.opensearch.replication.task.shard import org.opensearch.replication.task.ReplicationState import org.opensearch.OpenSearchException -import org.opensearch.common.ParseField +import org.opensearch.core.ParseField import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.common.xcontent.ObjectParser -import org.opensearch.common.xcontent.ToXContent -import org.opensearch.common.xcontent.XContentBuilder -import org.opensearch.common.xcontent.XContentParser +import org.opensearch.core.xcontent.ObjectParser +import org.opensearch.core.xcontent.ToXContent +import org.opensearch.core.xcontent.XContentBuilder +import org.opensearch.core.xcontent.XContentParser import org.opensearch.persistent.PersistentTaskState import java.io.IOException import java.lang.IllegalArgumentException diff --git a/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt b/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt index 66d9e53d..0ac689a7 100644 --- a/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt +++ b/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt @@ -44,8 +44,8 @@ import org.opensearch.common.io.PathUtils import org.opensearch.common.settings.Settings import org.opensearch.common.unit.TimeValue import org.opensearch.common.util.concurrent.ThreadContext -import org.opensearch.common.xcontent.DeprecationHandler -import org.opensearch.common.xcontent.NamedXContentRegistry +import org.opensearch.core.xcontent.DeprecationHandler +import org.opensearch.core.xcontent.NamedXContentRegistry import org.opensearch.common.xcontent.XContentHelper import org.opensearch.common.xcontent.XContentType import org.opensearch.common.xcontent.json.JsonXContent @@ -58,7 +58,6 @@ import org.junit.After import org.junit.AfterClass import org.junit.Before import org.junit.BeforeClass -import org.opensearch.index.mapper.ObjectMapper import java.nio.file.Files import java.security.KeyManagementException import java.security.KeyStore diff --git a/src/test/kotlin/org/opensearch/replication/ReplicationHelpers.kt b/src/test/kotlin/org/opensearch/replication/ReplicationHelpers.kt index 2fa7d1fc..d6250f32 100644 --- a/src/test/kotlin/org/opensearch/replication/ReplicationHelpers.kt +++ b/src/test/kotlin/org/opensearch/replication/ReplicationHelpers.kt @@ -23,8 +23,8 @@ import org.opensearch.client.Response import org.opensearch.client.RestHighLevelClient import org.opensearch.common.settings.Settings import org.opensearch.common.unit.TimeValue -import org.opensearch.common.xcontent.DeprecationHandler -import org.opensearch.common.xcontent.NamedXContentRegistry +import org.opensearch.core.xcontent.DeprecationHandler +import org.opensearch.core.xcontent.NamedXContentRegistry import org.opensearch.common.xcontent.XContentType import org.opensearch.test.OpenSearchTestCase.assertBusy import org.opensearch.test.rest.OpenSearchRestTestCase diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt index 1c483298..93997599 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt @@ -67,8 +67,8 @@ import org.opensearch.repositories.fs.FsRepository import org.opensearch.test.OpenSearchTestCase.assertBusy import org.junit.Assert import org.opensearch.cluster.metadata.AliasMetadata -import org.opensearch.common.xcontent.DeprecationHandler -import org.opensearch.common.xcontent.NamedXContentRegistry +import org.opensearch.core.xcontent.DeprecationHandler +import org.opensearch.core.xcontent.NamedXContentRegistry import org.opensearch.replication.ReplicationPlugin.Companion.REPLICATION_INDEX_TRANSLOG_PRUNING_ENABLED_SETTING import org.opensearch.replication.followerStats import org.opensearch.replication.leaderStats diff --git a/src/test/kotlin/org/opensearch/replication/task/index/IndexReplicationTaskTests.kt b/src/test/kotlin/org/opensearch/replication/task/index/IndexReplicationTaskTests.kt index 9ba4919c..2032cc26 100644 --- a/src/test/kotlin/org/opensearch/replication/task/index/IndexReplicationTaskTests.kt +++ b/src/test/kotlin/org/opensearch/replication/task/index/IndexReplicationTaskTests.kt @@ -32,7 +32,7 @@ import org.opensearch.cluster.routing.RoutingTable import org.opensearch.common.settings.Settings import org.opensearch.common.settings.SettingsModule import org.opensearch.common.unit.TimeValue -import org.opensearch.common.xcontent.NamedXContentRegistry +import org.opensearch.core.xcontent.NamedXContentRegistry import org.opensearch.index.Index import org.opensearch.index.shard.ShardId import org.opensearch.persistent.PersistentTaskParams diff --git a/src/test/kotlin/org/opensearch/replication/task/index/NoOpClient.kt b/src/test/kotlin/org/opensearch/replication/task/index/NoOpClient.kt index 1c92c0bb..35af7cb4 100644 --- a/src/test/kotlin/org/opensearch/replication/task/index/NoOpClient.kt +++ b/src/test/kotlin/org/opensearch/replication/task/index/NoOpClient.kt @@ -32,7 +32,7 @@ import org.opensearch.common.UUIDs import org.opensearch.common.bytes.BytesReference import org.opensearch.common.collect.ImmutableOpenMap import org.opensearch.common.settings.Settings -import org.opensearch.common.xcontent.ToXContent +import org.opensearch.core.xcontent.ToXContent import org.opensearch.common.xcontent.XContentFactory import org.opensearch.index.Index import org.opensearch.index.get.GetResult From 22dfffaf2ea7ab1258aac256e8ee20ae010d21ee Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Wed, 1 Mar 2023 17:55:06 +0530 Subject: [PATCH 067/157] Baselining the maintainer list (#730) Signed-off-by: Ankit Kala --- .github/CODEOWNERS | 8 ++++++-- MAINTAINERS.md | 3 --- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index 76af11cd..af340718 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -1,2 +1,6 @@ -# This should match the owning team set up in https://github.com/orgs/opensearch-project/teams -* @opensearch-project/replication \ No newline at end of file +# This list should match with the maintainters list. +* @krishna-ggk +* @ankitkala +* @saikaranam-amazon +* @soosinha +* @gbbafna \ No newline at end of file diff --git a/MAINTAINERS.md b/MAINTAINERS.md index 14f485be..0e708f28 100644 --- a/MAINTAINERS.md +++ b/MAINTAINERS.md @@ -6,11 +6,8 @@ This document contains a list of maintainers in this repo. See [opensearch-proje | Maintainer | GitHub ID | Affiliation | | ---------------- | --------------------------------------------------------- | ----------- | -| Rushi Agrawal | [rushiagr](https://github.com/rushiagr) | Amazon | | Gopala Krishna A | [krishna-ggk](https://github.com/krishna-ggk) | Amazon | | Ankit | [ankitkala](https://github.com/ankitkala) | Amazon | -| Naveen | [naveenpajjuri](https://github.com/naveenpajjuri) | Amazon | | Sai | [saikaranam-amazon](https://github.com/saikaranam-amazon) | Amazon | -| Bhanu Tadepalli | [tbhanu-amzn](https://github.com/tbhanu-amzn) | Amazon | | Sooraj Sinha | [soosinha](https://github.com/soosinha) | Amazon | | Gaurav Bafna | [gbbafna](https://github.com/gbbafna) | Amazon | From be023e490f5cd647f652bc5bd5b5953ac7b48657 Mon Sep 17 00:00:00 2001 From: Sai Kumar Date: Mon, 6 Mar 2023 22:28:05 +0530 Subject: [PATCH 068/157] Modified autofollow stats to rely on single source for failed indices (#708) Modified autofollow stats to rely on single source for failed indices and further improved logging for the initial failures during leader calls. Signed-off-by: Sai Kumar --- .../replication/ReplicationPlugin.kt | 2 +- .../task/autofollow/AutoFollowTask.kt | 27 +++++++++++-------- .../replication/ReplicationHelpers.kt | 10 +++++++ .../integ/rest/UpdateAutoFollowPatternIT.kt | 12 ++++++++- 4 files changed, 38 insertions(+), 13 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt b/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt index 5a0dddfb..9b50bb2b 100644 --- a/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt +++ b/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt @@ -178,7 +178,7 @@ internal class ReplicationPlugin : Plugin(), ActionPlugin, PersistentTaskPlugin, TimeValue.timeValueSeconds(1), Setting.Property.Dynamic, Setting.Property.NodeScope) val REPLICATION_AUTOFOLLOW_REMOTE_INDICES_POLL_INTERVAL = Setting.timeSetting ("plugins.replication.autofollow.fetch_poll_interval", TimeValue.timeValueSeconds(30), TimeValue.timeValueSeconds(30), TimeValue.timeValueHours(1), Setting.Property.Dynamic, Setting.Property.NodeScope) - val REPLICATION_AUTOFOLLOW_REMOTE_INDICES_RETRY_POLL_INTERVAL = Setting.timeSetting ("plugins.replication.autofollow.retry_poll_interval", TimeValue.timeValueHours(1), TimeValue.timeValueMinutes(30), + val REPLICATION_AUTOFOLLOW_REMOTE_INDICES_RETRY_POLL_INTERVAL = Setting.timeSetting ("plugins.replication.autofollow.retry_poll_interval", TimeValue.timeValueHours(1), TimeValue.timeValueMinutes(1), TimeValue.timeValueHours(4), Setting.Property.Dynamic, Setting.Property.NodeScope) val REPLICATION_METADATA_SYNC_INTERVAL = Setting.timeSetting("plugins.replication.follower.metadata_sync_interval", TimeValue.timeValueSeconds(60), TimeValue.timeValueSeconds(5), diff --git a/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowTask.kt b/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowTask.kt index 6b1db867..da89580e 100644 --- a/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowTask.kt @@ -63,7 +63,6 @@ class AutoFollowTask(id: Long, type: String, action: String, description: String override val followerIndexName: String = params.patternName //Special case for auto follow override val log = Loggers.getLogger(javaClass, leaderAlias) private var trackingIndicesOnTheCluster = setOf() - private var failedIndices = ConcurrentSkipListSet() // Failed indices for replication from this autofollow task private var replicationJobsQueue = ConcurrentSkipListSet() // To keep track of outstanding replication jobs for this autofollow task private var retryScheduler: Scheduler.ScheduledCancellable? = null lateinit var stat: AutoFollowStat @@ -91,14 +90,21 @@ class AutoFollowTask(id: Long, type: String, action: String, description: String } private fun addRetryScheduler() { + log.debug("Adding retry scheduler") if(retryScheduler != null && !retryScheduler!!.isCancelled) { return } - retryScheduler = try { - threadPool.schedule({ failedIndices.clear() }, replicationSettings.autofollowRetryPollDuration, ThreadPool.Names.GENERIC) + try { + retryScheduler = threadPool.schedule( + { + log.debug("Clearing failed indices to schedule for the next retry") + stat.failedIndices.clear() + }, + replicationSettings.autofollowRetryPollDuration, + ThreadPool.Names.SAME) } catch (e: Exception) { log.error("Error scheduling retry on failed autofollow indices ${e.stackTraceToString()}") - null + retryScheduler = null } } @@ -123,10 +129,10 @@ class AutoFollowTask(id: Long, type: String, action: String, description: String } catch (e: Exception) { // Ideally, Calls to the remote cluster shouldn't fail and autofollow task should be able to pick-up the newly created indices // matching the pattern. Should be safe to retry after configured delay. - stat.failedLeaderCall++ - if(stat.failedLeaderCall > 0 && stat.failedLeaderCall.rem(10) == 0L) { + if(stat.failedLeaderCall >= 0 && stat.failedLeaderCall.rem(10) == 0L) { log.error("Fetching remote indices failed with error - ${e.stackTraceToString()}") } + stat.failedLeaderCall++ } var currentIndices = clusterService.state().metadata().concreteAllIndices.asIterable() // All indices - open and closed on the cluster @@ -138,7 +144,7 @@ class AutoFollowTask(id: Long, type: String, action: String, description: String trackingIndicesOnTheCluster = currentIndices.toSet() } } - remoteIndices = remoteIndices.minus(currentIndices).minus(failedIndices).minus(replicationJobsQueue) + remoteIndices = remoteIndices.minus(currentIndices).minus(stat.failedIndices).minus(replicationJobsQueue) stat.failCounterForRun = 0 startReplicationJobs(remoteIndices) @@ -207,8 +213,6 @@ class AutoFollowTask(id: Long, type: String, action: String, description: String } catch (e: OpenSearchSecurityException) { // For permission related failures, Adding as part of failed indices as autofollow role doesn't have required permissions. log.trace("Cannot start replication on $leaderIndex due to missing permissions $e") - failedIndices.add(leaderIndex) - } catch (e: Exception) { // Any failure other than security exception can be safely retried and not adding to the failed indices log.warn("Failed to start replication for $leaderAlias:$leaderIndex -> $leaderIndex.", e) @@ -249,7 +253,7 @@ class AutoFollowStat: Task.Status { val name :String val pattern :String var failCount: Long=0 - var failedIndices :MutableSet = mutableSetOf() + var failedIndices = ConcurrentSkipListSet() // Failed indices for replication from this autofollow task var failCounterForRun :Long=0 var successCount: Long=0 var failedLeaderCall :Long=0 @@ -265,7 +269,8 @@ class AutoFollowStat: Task.Status { name = inp.readString() pattern = inp.readString() failCount = inp.readLong() - failedIndices = inp.readSet(StreamInput::readString) + val inpFailedIndices = inp.readList(StreamInput::readString) + failedIndices = ConcurrentSkipListSet(inpFailedIndices) successCount = inp.readLong() failedLeaderCall = inp.readLong() lastExecutionTime = inp.readLong() diff --git a/src/test/kotlin/org/opensearch/replication/ReplicationHelpers.kt b/src/test/kotlin/org/opensearch/replication/ReplicationHelpers.kt index d6250f32..eb5f6404 100644 --- a/src/test/kotlin/org/opensearch/replication/ReplicationHelpers.kt +++ b/src/test/kotlin/org/opensearch/replication/ReplicationHelpers.kt @@ -381,6 +381,16 @@ fun RestHighLevelClient.updateReplicationStartBlockSetting(enabled: Boolean) { assertThat(response.isAcknowledged).isTrue() } +fun RestHighLevelClient.updateAutofollowRetrySetting(duration: String) { + var settings: Settings = Settings.builder() + .put("plugins.replication.autofollow.retry_poll_interval", duration) + .build() + var updateSettingsRequest = ClusterUpdateSettingsRequest() + updateSettingsRequest.persistentSettings(settings) + val response = this.cluster().putSettings(updateSettingsRequest, RequestOptions.DEFAULT) + assertThat(response.isAcknowledged).isTrue() +} + fun RestHighLevelClient.updateAutoFollowConcurrentStartReplicationJobSetting(concurrentJobs: Int?) { val settings = if(concurrentJobs != null) { Settings.builder() diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt index 844c4539..1d366005 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt @@ -42,6 +42,7 @@ import org.opensearch.cluster.metadata.MetadataCreateIndexService import org.opensearch.replication.AutoFollowStats import org.opensearch.replication.ReplicationPlugin import org.opensearch.replication.updateReplicationStartBlockSetting +import org.opensearch.replication.updateAutofollowRetrySetting import org.opensearch.replication.updateAutoFollowConcurrentStartReplicationJobSetting import org.opensearch.replication.waitForShardTaskStart import java.lang.Thread.sleep @@ -320,6 +321,8 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { createConnectionBetweenClusters(FOLLOWER, LEADER, connectionAlias) val leaderIndexName = createRandomIndex(leaderClient) try { + //modify retry duration to account for autofollow trigger in next retry + followerClient.updateAutofollowRetrySetting("1m") // Add replication start block followerClient.updateReplicationStartBlockSetting(true) followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, indexPattern) @@ -329,12 +332,19 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { // Autofollow task should still be up - 1 task Assertions.assertThat(getIndexReplicationTasks(FOLLOWER).size).isEqualTo(0) Assertions.assertThat(getAutoFollowTasks(FOLLOWER).size).isEqualTo(1) + + var stats = followerClient.AutoFollowStats() + var failedIndices = stats["failed_indices"] as List<*> + assert(failedIndices.size == 1) // Remove replication start block followerClient.updateReplicationStartBlockSetting(false) - sleep(45000) // poll for auto follow in worst case + sleep(60000) // wait for auto follow trigger in the worst case // Index should be replicated and autofollow task should be present Assertions.assertThat(getIndexReplicationTasks(FOLLOWER).size).isEqualTo(1) Assertions.assertThat(getAutoFollowTasks(FOLLOWER).size).isEqualTo(1) + stats = followerClient.AutoFollowStats() + failedIndices = stats["failed_indices"] as List<*> + assert(failedIndices.isEmpty()) } finally { followerClient.deleteAutoFollowPattern(connectionAlias, indexPatternName) } From 7ce444f717629ce62c5b12ba52c3f96a9bfa2c20 Mon Sep 17 00:00:00 2001 From: Sai Kumar Date: Tue, 21 Mar 2023 10:29:42 +0530 Subject: [PATCH 069/157] Modified security github workflow to run based on plugin availability in central repo (#744) Signed-off-by: Sai Kumar --- .github/workflows/security-tests.yml | 28 ++++++++++++++++++++++++++++ build.gradle | 5 ++++- 2 files changed, 32 insertions(+), 1 deletion(-) diff --git a/.github/workflows/security-tests.yml b/.github/workflows/security-tests.yml index a1415749..ddf6046e 100644 --- a/.github/workflows/security-tests.yml +++ b/.github/workflows/security-tests.yml @@ -9,7 +9,35 @@ on: - '*' jobs: + req: + # Job name + name: Security plugin check + runs-on: ubuntu-latest + outputs: + isSecurityPluginAvailable: ${{ steps.security-plugin-availability-check.outputs.isSecurityPluginAvailable }} + steps: + # This step uses the checkout Github action: https://github.com/actions/checkout + - name: Checkout Branch + uses: actions/checkout@v2 + - id: security-plugin-availability-check + name: "Security plugin check" + run: | + opensearch_version=$(grep "System.getProperty(\"opensearch.version\", \"" build.gradle | grep '\([0-9]\|[.]\)\{5\}' -o) + opensearch_version=$opensearch_version".0-SNAPSHOT" + # we publish build artifacts to the below url + plugin_url="https://aws.oss.sonatype.org/content/repositories/snapshots/org/opensearch/plugin/opensearch-security/"$opensearch_version"/" + st=$(curl -s -o /dev/null -w "%{http_code}" $plugin_url) + if [ "$st" = "200" ]; then + echo "isSecurityPluginAvailable=True" >> $GITHUB_OUTPUT + cat $GITHUB_OUTPUT + else + echo "isSecurityPluginAvailable=False" >> $GITHUB_OUTPUT + cat $GITHUB_OUTPUT + fi + build: + needs: req + if: ${{ 'True' == needs.req.outputs.isSecurityPluginAvailable }} # Job name name: Build and Run Security tests runs-on: ubuntu-latest diff --git a/build.gradle b/build.gradle index b965e24d..e916f99d 100644 --- a/build.gradle +++ b/build.gradle @@ -55,7 +55,10 @@ buildscript { common_utils_version = System.getProperty("common_utils.version", opensearch_build) kotlin_version = System.getProperty("kotlin.version", "1.6.0") - security_plugin_version = opensearch_build.replace("-SNAPSHOT","") + security_plugin_version = opensearch_build + if (!isSnapshot) { + security_plugin_version = opensearch_build.replace("-SNAPSHOT","") + } } repositories { From db289245a68830055638acb12ef547edf5393faa Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Wed, 22 Mar 2023 18:47:52 +0530 Subject: [PATCH 070/157] Update test to wait for 60 seconds UpdateAutoFollowPatternIT test "test auto follow stats" was waiting for 30 seconds to poll for shard replication task to start. This was causing intermittent failure as shard replication task may take more time to start. Signed-off-by: Monu Singh --- .../replication/integ/rest/UpdateAutoFollowPatternIT.kt | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt index 1d366005..aad0f993 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt @@ -196,6 +196,8 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { }, 30, TimeUnit.SECONDS) // Verify that existing index matching the pattern are replicated. assertBusy ({ + followerClient.waitForShardTaskStart(leaderIndexName) + followerClient.waitForShardTaskStart(leaderIndexName2) Assertions.assertThat(followerClient.indices() .exists(GetIndexRequest(leaderIndexName2), RequestOptions.DEFAULT)) .isEqualTo(true) @@ -207,12 +209,10 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { assert(key["num_success_start_replication"]!! as Int == 1) } assertTrue(af_stats.size == 2) - }, 30, TimeUnit.SECONDS) + }, 60, TimeUnit.SECONDS) } finally { followerClient.deleteAutoFollowPattern(connectionAlias, indexPatternName) followerClient.deleteAutoFollowPattern(connectionAlias, indexPatternName2) - followerClient.waitForShardTaskStart(leaderIndexName) - followerClient.waitForShardTaskStart(leaderIndexName2) } } @@ -450,4 +450,4 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { assertEquals(HttpStatus.SC_OK.toLong(), persistentConnectionResponse.statusLine.statusCode.toLong()) } -} \ No newline at end of file +} From c3fcd00f25c24f2df4759727ffff285e05a982bd Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Fri, 24 Mar 2023 12:25:51 +0530 Subject: [PATCH 071/157] Add autofollow_stats to HANDBOOK.md Currently HANDBOOK.md does not have autofollow_stats example. Signed-off-by: Monu Singh --- HANDBOOK.md | 28 ++++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/HANDBOOK.md b/HANDBOOK.md index 543b041e..ebb63090 100644 --- a/HANDBOOK.md +++ b/HANDBOOK.md @@ -271,6 +271,34 @@ curl -k -u testuser:testuser -XDELETE \ -d'{"leader_alias":"leader-cluster", "name":"my-replication"}' ``` +## Check AutoFollow stats +Stats for all AutoFollow tasks running on the follower cluster can be checked using `autofollow_stats`. + +```bash +curl -k -u admin:admin -XGET "https://${FOLLOWER}/_plugins/_replication/autofollow_stats" +{ +"num_success_start_replication" : 16, + "num_failed_start_replication" : 0, + "num_failed_leader_calls" : 0, + "failed_indices" : [ + "leader-08" + ], + "autofollow_stats" : [ + { + "name" : "first_rule", + "pattern" : "leader-*", + "num_success_start_replication" : 16, + "num_failed_start_replication" : 0, + "num_failed_leader_calls" : 0, + "failed_indices" : [ + "leader-08" + ], + "last_execution_time" : 1679640596573 + } + ] +} +``` + ## Check ongoing replication tasks Until a status API is added, you can check ongoing replication via the tasks API. From 5ccb7b74431d8803e20c01358e376712eaaae42e Mon Sep 17 00:00:00 2001 From: Sai Kumar Date: Tue, 4 Apr 2023 16:43:57 +0530 Subject: [PATCH 072/157] Updated plugin code based on OpenSearch-3.0 upstream changes (#766) Signed-off-by: Sai Kumar --- .../replication/repository/RemoteClusterRestoreLeaderService.kt | 2 +- .../index/translog/ReplicationTranslogDeletionPolicyTests.kt | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRestoreLeaderService.kt b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRestoreLeaderService.kt index 7adfc8aa..5eea937b 100644 --- a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRestoreLeaderService.kt +++ b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRestoreLeaderService.kt @@ -21,7 +21,7 @@ import org.opensearch.common.component.AbstractLifecycleComponent import org.opensearch.common.inject.Inject import org.opensearch.common.inject.Singleton import org.opensearch.common.lucene.store.InputStreamIndexInput -import org.opensearch.core.internal.io.IOUtils +import org.opensearch.common.util.io.IOUtils import org.opensearch.index.seqno.RetentionLeaseActions import org.opensearch.index.store.Store import org.opensearch.indices.IndicesService diff --git a/src/test/kotlin/org/opensearch/index/translog/ReplicationTranslogDeletionPolicyTests.kt b/src/test/kotlin/org/opensearch/index/translog/ReplicationTranslogDeletionPolicyTests.kt index cebf7890..38387621 100644 --- a/src/test/kotlin/org/opensearch/index/translog/ReplicationTranslogDeletionPolicyTests.kt +++ b/src/test/kotlin/org/opensearch/index/translog/ReplicationTranslogDeletionPolicyTests.kt @@ -18,7 +18,7 @@ import org.opensearch.common.bytes.BytesArray import org.opensearch.common.bytes.ReleasableBytesReference import org.opensearch.common.collect.Tuple import org.opensearch.common.util.BigArrays -import org.opensearch.core.internal.io.IOUtils +import org.opensearch.common.util.io.IOUtils import org.opensearch.index.seqno.RetentionLease import org.opensearch.index.seqno.RetentionLeases import org.opensearch.index.shard.ShardId From c5d4cdc5332c0a0839f29ffdea4f2d52f2307f0d Mon Sep 17 00:00:00 2001 From: Sai Kumar Date: Thu, 6 Apr 2023 12:33:27 +0530 Subject: [PATCH 073/157] Handled batch requests for replication metadata update under cluster state (#772) Signed-off-by: Sai Kumar --- .../metadata/UpdateReplicationMetadata.kt | 23 +++--- .../state/UpdateReplicationMetadataTests.kt | 75 +++++++++++++++++++ 2 files changed, 89 insertions(+), 9 deletions(-) create mode 100644 src/test/kotlin/org/opensearch/replication/metadata/state/UpdateReplicationMetadataTests.kt diff --git a/src/main/kotlin/org/opensearch/replication/metadata/UpdateReplicationMetadata.kt b/src/main/kotlin/org/opensearch/replication/metadata/UpdateReplicationMetadata.kt index 856adbb3..54d4663e 100644 --- a/src/main/kotlin/org/opensearch/replication/metadata/UpdateReplicationMetadata.kt +++ b/src/main/kotlin/org/opensearch/replication/metadata/UpdateReplicationMetadata.kt @@ -49,29 +49,34 @@ class UpdateReplicationStateDetailsTaskExecutor private constructor() override fun execute(currentState: ClusterState, tasks: List) : ClusterStateTaskExecutor.ClusterTasksResult { - return getClusterStateUpdateTaskResult(tasks[0], currentState) + log.debug("Executing replication state update for $tasks") + return getClusterStateUpdateTaskResult(tasks, currentState) } - private fun getClusterStateUpdateTaskResult(request: UpdateReplicationStateDetailsRequest, + private fun getClusterStateUpdateTaskResult(requests: List, currentState: ClusterState) : ClusterStateTaskExecutor.ClusterTasksResult { val currentMetadata = currentState.metadata().custom(ReplicationStateMetadata.NAME) ?: ReplicationStateMetadata.EMPTY - val newMetadata = getUpdatedReplicationMetadata(request, currentMetadata) - if (currentMetadata == newMetadata) { - return getStateUpdateTaskResultForClusterState(request, currentState) // no change + var updatedMetadata = currentMetadata + // compute metadata update for the batched requests + for(request in requests) { + updatedMetadata = getUpdatedReplicationMetadata(request, updatedMetadata) + } + if (currentMetadata == updatedMetadata) { + return getStateUpdateTaskResultForClusterState(requests, currentState) // no change } else { val mdBuilder = Metadata.builder(currentState.metadata) - .putCustom(ReplicationStateMetadata.NAME, newMetadata) + .putCustom(ReplicationStateMetadata.NAME, updatedMetadata) val newClusterState = ClusterState.Builder(currentState).metadata(mdBuilder).build() - return getStateUpdateTaskResultForClusterState(request, newClusterState) + return getStateUpdateTaskResultForClusterState(requests, newClusterState) } } - private fun getStateUpdateTaskResultForClusterState(request: UpdateReplicationStateDetailsRequest, + private fun getStateUpdateTaskResultForClusterState(requests: List, clusterState: ClusterState) : ClusterStateTaskExecutor.ClusterTasksResult { return ClusterStateTaskExecutor.ClusterTasksResult.builder() - .success(request).build(clusterState) + .successes(requests).build(clusterState) } private fun getUpdatedReplicationMetadata(request: UpdateReplicationStateDetailsRequest, diff --git a/src/test/kotlin/org/opensearch/replication/metadata/state/UpdateReplicationMetadataTests.kt b/src/test/kotlin/org/opensearch/replication/metadata/state/UpdateReplicationMetadataTests.kt new file mode 100644 index 00000000..a3a68430 --- /dev/null +++ b/src/test/kotlin/org/opensearch/replication/metadata/state/UpdateReplicationMetadataTests.kt @@ -0,0 +1,75 @@ +package org.opensearch.replication.metadata.state + +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope +import org.junit.Assert +import org.opensearch.cluster.ClusterState +import org.opensearch.replication.action.replicationstatedetails.UpdateReplicationStateDetailsRequest +import org.opensearch.replication.metadata.ReplicationOverallState +import org.opensearch.replication.metadata.UpdateReplicationStateDetailsTaskExecutor +import org.opensearch.test.ClusterServiceUtils +import org.opensearch.test.OpenSearchTestCase +import org.opensearch.threadpool.TestThreadPool + +@ThreadLeakScope(ThreadLeakScope.Scope.NONE) +class UpdateReplicationMetadataTests : OpenSearchTestCase() { + + var threadPool = TestThreadPool("ReplicationPluginTest") + var clusterService = ClusterServiceUtils.createClusterService(threadPool) + + fun `test single task update`() { + val currentState: ClusterState = clusterService.state() + // single task + val tasks = arrayListOf(UpdateReplicationStateDetailsRequest("test-index", + hashMapOf("REPLICATION_LAST_KNOWN_OVERALL_STATE" to "RUNNING"), UpdateReplicationStateDetailsRequest.UpdateType.ADD)) + val tasksResult = UpdateReplicationStateDetailsTaskExecutor.INSTANCE.execute(currentState, tasks) + + val updatedReplicationDetails = tasksResult.resultingState?.metadata + ?.custom(ReplicationStateMetadata.NAME)?.replicationDetails + + Assert.assertNotNull(updatedReplicationDetails) + Assert.assertNotNull(updatedReplicationDetails?.get("test-index")) + val replicationStateParams = updatedReplicationDetails?.get("test-index") + + Assert.assertEquals(ReplicationOverallState.RUNNING.name, replicationStateParams?.get(REPLICATION_LAST_KNOWN_OVERALL_STATE)) + } + + fun `test multiple tasks to add replication metadata`() { + val currentState: ClusterState = clusterService.state() + // multiple tasks + val tasks = arrayListOf(UpdateReplicationStateDetailsRequest("test-index-1", + hashMapOf("REPLICATION_LAST_KNOWN_OVERALL_STATE" to "RUNNING"), UpdateReplicationStateDetailsRequest.UpdateType.ADD), + UpdateReplicationStateDetailsRequest("test-index-2", + hashMapOf("REPLICATION_LAST_KNOWN_OVERALL_STATE" to "RUNNING"), UpdateReplicationStateDetailsRequest.UpdateType.ADD)) + val tasksResult = UpdateReplicationStateDetailsTaskExecutor.INSTANCE.execute(currentState, tasks) + + val updatedReplicationDetails = tasksResult.resultingState?.metadata + ?.custom(ReplicationStateMetadata.NAME)?.replicationDetails + + Assert.assertNotNull(updatedReplicationDetails) + Assert.assertNotNull(updatedReplicationDetails?.get("test-index-1")) + var replicationStateParams = updatedReplicationDetails?.get("test-index-1") + Assert.assertEquals(ReplicationOverallState.RUNNING.name, replicationStateParams?.get(REPLICATION_LAST_KNOWN_OVERALL_STATE)) + Assert.assertNotNull(updatedReplicationDetails?.get("test-index-2")) + replicationStateParams = updatedReplicationDetails?.get("test-index-2") + Assert.assertEquals(ReplicationOverallState.RUNNING.name, replicationStateParams?.get(REPLICATION_LAST_KNOWN_OVERALL_STATE)) + } + + fun `test multiple tasks to add and delete replication metadata`() { + val currentState: ClusterState = clusterService.state() + // multiple tasks + val tasks = arrayListOf(UpdateReplicationStateDetailsRequest("test-index-1", + hashMapOf("REPLICATION_LAST_KNOWN_OVERALL_STATE" to "RUNNING"), UpdateReplicationStateDetailsRequest.UpdateType.ADD), + UpdateReplicationStateDetailsRequest("test-index-2", + hashMapOf("REPLICATION_LAST_KNOWN_OVERALL_STATE" to "RUNNING"), UpdateReplicationStateDetailsRequest.UpdateType.REMOVE)) + val tasksResult = UpdateReplicationStateDetailsTaskExecutor.INSTANCE.execute(currentState, tasks) + + val updatedReplicationDetails = tasksResult.resultingState?.metadata + ?.custom(ReplicationStateMetadata.NAME)?.replicationDetails + + Assert.assertNotNull(updatedReplicationDetails) + Assert.assertNotNull(updatedReplicationDetails?.get("test-index-1")) + var replicationStateParams = updatedReplicationDetails?.get("test-index-1") + Assert.assertEquals(ReplicationOverallState.RUNNING.name, replicationStateParams?.get(REPLICATION_LAST_KNOWN_OVERALL_STATE)) + Assert.assertNull(updatedReplicationDetails?.get("test-index-2")) + } +} \ No newline at end of file From 501bd283505f61ebecf617849da19d28e2724da9 Mon Sep 17 00:00:00 2001 From: sricharanvuppu <113983630+sricharanvuppu@users.noreply.github.com> Date: Fri, 14 Apr 2023 11:20:54 +0530 Subject: [PATCH 074/157] handling null pointer exception in multi-filed mapping (#757) * handling null pointer exception in multi-filed mapping Signed-off-by: sricharanvuppu --- .../task/index/IndexReplicationTask.kt | 24 +++++++++---------- 1 file changed, 11 insertions(+), 13 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt index e7b668ed..4bf8b3d5 100644 --- a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt @@ -413,12 +413,9 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript } } - private suspend fun UpdateFollowereMapping(followerIndex: String,mappingSource: String) { + private suspend fun updateFollowerMapping(followerIndex: String,mappingSource: String?) { val options = IndicesOptions.strictSingleIndexNoExpandForbidClosed() - if (null == mappingSource) { - throw MappingNotAvailableException("MappingSource is not available") - } val putMappingRequest = PutMappingRequest().indices(followerIndex).indicesOptions(options) .source(mappingSource, XContentType.JSON) val updateMappingRequest = UpdateMetadataRequest(followerIndex, UpdateMetadataRequest.Type.MAPPING, putMappingRequest) @@ -569,19 +566,20 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript val options = IndicesOptions.strictSingleIndexNoExpandForbidClosed() var gmr = GetMappingsRequest().indices(this.leaderIndex.name).indicesOptions(options) var mappingResponse = remoteClient.suspending(remoteClient.admin().indices()::getMappings, injectSecurityContext = true)(gmr) - var leaderMappingSource = mappingResponse.mappings.get(this.leaderIndex.name).source().toString() - val leaderProperties = mappingResponse.mappings().get(this.leaderIndex.name).sourceAsMap().toMap().get("properties") as Map + var leaderMappingSource = mappingResponse?.mappings?.get(this.leaderIndex.name)?.source()?.toString() + @Suppress("UNCHECKED_CAST") + val leaderProperties = mappingResponse?.mappings()?.get(this.leaderIndex.name)?.sourceAsMap()?.toMap()?.get("properties") as? Map? gmr = GetMappingsRequest().indices(this.followerIndexName).indicesOptions(options) mappingResponse = client.suspending(client.admin().indices()::getMappings, injectSecurityContext = true)(gmr) - val followerProperties = mappingResponse.mappings().get(this.followerIndexName).sourceAsMap().toMap().get("properties") as Map - for(iter in followerProperties) { - if(leaderProperties.containsKey(iter.key) && leaderProperties.getValue(iter.key).toString()!=(iter.value).toString()){ - log.info("Updating Multi-field Mapping at Follower") - UpdateFollowereMapping(this.followerIndexName,leaderMappingSource) - break; + @Suppress("UNCHECKED_CAST") + val followerProperties = mappingResponse?.mappings()?.get(this.followerIndexName)?.sourceAsMap()?.toMap()?.get("properties") as? Map? + for((key,value) in followerProperties?: emptyMap()) { + if (leaderProperties?.getValue(key).toString() != (value).toString()) { + log.debug("Updating Multi-field Mapping at Follower") + updateFollowerMapping(this.followerIndexName, leaderMappingSource) + break } } - } catch (e: Exception) { log.error("Error in getting the required metadata ${e.stackTraceToString()}") } finally { From fcbbfb74d288c2eb5cd2f933214b2d205d825832 Mon Sep 17 00:00:00 2001 From: ishankka <111563763+ishankka@users.noreply.github.com> Date: Fri, 14 Apr 2023 12:31:53 +0530 Subject: [PATCH 075/157] Adding a proxy mode connection setup for CCR (#786) Adding a proxy mode feature for CCR setup Signed-off-by: Ishank katiyar --- .../RemoteClusterRepositoriesService.kt | 16 +++- .../RemoteClusterRepositoriesServiceTests.kt | 90 +++++++++++++++++++ 2 files changed, 102 insertions(+), 4 deletions(-) create mode 100644 src/test/kotlin/org/opensearch/replication/repository/RemoteClusterRepositoriesServiceTests.kt diff --git a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepositoriesService.kt b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepositoriesService.kt index 46823482..c3995e47 100644 --- a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepositoriesService.kt +++ b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepositoriesService.kt @@ -13,6 +13,7 @@ package org.opensearch.replication.repository import org.opensearch.cluster.service.ClusterService import org.opensearch.common.settings.ClusterSettings import org.opensearch.repositories.RepositoriesService +import org.opensearch.transport.ProxyConnectionStrategy.PROXY_ADDRESS import org.opensearch.transport.SniffConnectionStrategy.REMOTE_CLUSTER_SEEDS import java.util.function.Supplier @@ -24,12 +25,12 @@ class RemoteClusterRepositoriesService(private val repositoriesService: Supplier } private fun listenForUpdates(clusterSettings: ClusterSettings) { - // TODO: Proxy support from ES 7.7. Needs additional handling based on those settings - clusterSettings.addAffixUpdateConsumer(REMOTE_CLUSTER_SEEDS, this::updateRepositoryDetails) { _, _ -> Unit } + clusterSettings.addAffixUpdateConsumer(REMOTE_CLUSTER_SEEDS, this::updateRepositoryDetailsForSeeds) { _, _ -> Unit } + clusterSettings.addAffixUpdateConsumer(PROXY_ADDRESS, this::updateRepositoryDetailsForProxy) { _, _ -> Unit } } - private fun updateRepositoryDetails(alias: String, seeds: List?) { - if(seeds == null || seeds.isEmpty()) { + private fun updateRepositoryDetailsForSeeds(alias: String, seeds: List?) { + if(seeds.isNullOrEmpty()) { repositoriesService.get().unregisterInternalRepository(REMOTE_REPOSITORY_PREFIX + alias) return } @@ -37,4 +38,11 @@ class RemoteClusterRepositoriesService(private val repositoriesService: Supplier repositoriesService.get().registerInternalRepository(REMOTE_REPOSITORY_PREFIX + alias, REMOTE_REPOSITORY_TYPE) } + private fun updateRepositoryDetailsForProxy(alias: String, proxyIp: String?) { + if(proxyIp.isNullOrEmpty()) { + repositoriesService.get().unregisterInternalRepository(REMOTE_REPOSITORY_PREFIX + alias) + return + } + repositoriesService.get().registerInternalRepository(REMOTE_REPOSITORY_PREFIX + alias, REMOTE_REPOSITORY_TYPE) + } } \ No newline at end of file diff --git a/src/test/kotlin/org/opensearch/replication/repository/RemoteClusterRepositoriesServiceTests.kt b/src/test/kotlin/org/opensearch/replication/repository/RemoteClusterRepositoriesServiceTests.kt new file mode 100644 index 00000000..6ec80b9b --- /dev/null +++ b/src/test/kotlin/org/opensearch/replication/repository/RemoteClusterRepositoriesServiceTests.kt @@ -0,0 +1,90 @@ +package org.opensearch.replication.repository + +import com.nhaarman.mockitokotlin2.times +import org.mockito.Mockito +import org.opensearch.Version +import org.opensearch.cluster.node.DiscoveryNode +import org.opensearch.cluster.node.DiscoveryNodeRole +import org.opensearch.common.settings.ClusterSettings +import org.opensearch.common.settings.Settings +import org.opensearch.repositories.RepositoriesService +import org.opensearch.test.ClusterServiceUtils +import org.opensearch.test.OpenSearchTestCase +import org.opensearch.threadpool.TestThreadPool +import java.util.function.Supplier + +class RemoteClusterRepositoriesServiceTests : OpenSearchTestCase() { + + fun `test changes in seed_nodes`() { + var clusterSetting = ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) + var threadPool = TestThreadPool("ReplicationPluginTest") + val discoveryNode = DiscoveryNode( + "node", + buildNewFakeTransportAddress(), emptyMap(), + DiscoveryNodeRole.BUILT_IN_ROLES, + Version.CURRENT + ) + var clusterService = ClusterServiceUtils.createClusterService(threadPool, discoveryNode, clusterSetting) + val repositoriesService = Mockito.mock(RepositoriesService::class.java) + RemoteClusterRepositoriesService(Supplier { repositoriesService }, clusterService) + clusterSetting.applySettings(Settings.builder().putList("cluster.remote.con-alias.seeds", "127.0.0.1:9300", "127.0.0.2:9300").build()) + Mockito.verify(repositoriesService, times(1)).registerInternalRepository(REMOTE_REPOSITORY_PREFIX + "con-alias", REMOTE_REPOSITORY_TYPE) + threadPool.shutdown() + } + + fun `test removal of seed_nodes`() { + var clusterSetting = ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) + var threadPool = TestThreadPool("ReplicationPluginTest") + val discoveryNode = DiscoveryNode( + "node", + buildNewFakeTransportAddress(), emptyMap(), + DiscoveryNodeRole.BUILT_IN_ROLES, + Version.CURRENT + ) + var clusterService = ClusterServiceUtils.createClusterService(threadPool, discoveryNode, clusterSetting) + val repositoriesService = Mockito.mock(RepositoriesService::class.java) + RemoteClusterRepositoriesService(Supplier { repositoriesService }, clusterService) + clusterSetting.applySettings(Settings.builder().putList("cluster.remote.con-alias.seeds", "127.0.0.1:9300", "127.0.0.2:9300").build()) + Mockito.verify(repositoriesService, times(1)).registerInternalRepository(REMOTE_REPOSITORY_PREFIX + "con-alias", REMOTE_REPOSITORY_TYPE) + clusterSetting.applySettings(Settings.builder().putNull("cluster.remote.con-alias.seeds").build()) + Mockito.verify(repositoriesService, times(1)).unregisterInternalRepository(REMOTE_REPOSITORY_PREFIX + "con-alias") + threadPool.shutdown() + } + + fun `test changes in proxy_id for proxy-setup`() { + var clusterSetting = ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) + var threadPool = TestThreadPool("ReplicationPluginTest") + val discoveryNode = DiscoveryNode( + "node", + buildNewFakeTransportAddress(), emptyMap(), + DiscoveryNodeRole.BUILT_IN_ROLES, + Version.CURRENT + ) + var clusterService = ClusterServiceUtils.createClusterService(threadPool, discoveryNode, clusterSetting) + val repositoriesService = Mockito.mock(RepositoriesService::class.java) + RemoteClusterRepositoriesService(Supplier { repositoriesService }, clusterService) + clusterSetting.applySettings(Settings.builder().put("cluster.remote.con-alias.mode", "proxy").put("cluster.remote.con-alias.proxy_address", "127.0.0.1:100").build()) + Mockito.verify(repositoriesService, times(1)).registerInternalRepository(REMOTE_REPOSITORY_PREFIX + "con-alias", REMOTE_REPOSITORY_TYPE) + threadPool.shutdown() + } + + fun `test removal of proxy_id for proxy-setup`() { + var clusterSetting = ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) + var threadPool = TestThreadPool("ReplicationPluginTest") + val discoveryNode = DiscoveryNode( + "node", + buildNewFakeTransportAddress(), emptyMap(), + DiscoveryNodeRole.BUILT_IN_ROLES, + Version.CURRENT + ) + var clusterService = ClusterServiceUtils.createClusterService(threadPool, discoveryNode, clusterSetting) + val repositoriesService = Mockito.mock(RepositoriesService::class.java) + RemoteClusterRepositoriesService(Supplier { repositoriesService }, clusterService) + clusterSetting.applySettings(Settings.builder().put("cluster.remote.con-alias.mode", "proxy").put("cluster.remote.con-alias.proxy_address", "127.0.0.1:100").build()) + Mockito.verify(repositoriesService, times(1)).registerInternalRepository(REMOTE_REPOSITORY_PREFIX + "con-alias", REMOTE_REPOSITORY_TYPE) + clusterSetting.applySettings(Settings.builder().putNull("cluster.remote.con-alias.mode").build()) + clusterSetting.applySettings(Settings.builder().putNull("cluster.remote.con-alias.proxy_address").build()) + Mockito.verify(repositoriesService, times(1)).unregisterInternalRepository(REMOTE_REPOSITORY_PREFIX + "con-alias") + threadPool.shutdown() + } +} From 7380597c0f66edb05e7f9f9538948d47c67d34b7 Mon Sep 17 00:00:00 2001 From: Sai Kumar Date: Mon, 17 Apr 2023 19:30:11 +0530 Subject: [PATCH 076/157] Add release notes for v2.7.0 Signed-off-by: Sai Kumar --- ...ss-cluster-replication.release-notes-2.7.0.0.md | 14 ++++++++++++++ 1 file changed, 14 insertions(+) create mode 100644 release-notes/opensearch-cross-cluster-replication.release-notes-2.7.0.0.md diff --git a/release-notes/opensearch-cross-cluster-replication.release-notes-2.7.0.0.md b/release-notes/opensearch-cross-cluster-replication.release-notes-2.7.0.0.md new file mode 100644 index 00000000..e45008d2 --- /dev/null +++ b/release-notes/opensearch-cross-cluster-replication.release-notes-2.7.0.0.md @@ -0,0 +1,14 @@ +## Version 2.7.0.0 Release Notes + +Compatible with OpenSearch 2.7.0 + +### Bug Fixes +* Modified autofollow stats to rely on single source for failed indices ([#736](https://github.com/opensearch-project/cross-cluster-replication/pull/736)) +* Update UpdateAutoFollowPatternIT "test auto follow stats" to wait for 60 seconds ([#745](https://github.com/opensearch-project/cross-cluster-replication/pull/745)) +* Update imports from common to core package ([#761](https://github.com/opensearch-project/cross-cluster-replication/pull/761)) +* Adding a proxy mode connection setup for CCR ([#795](https://github.com/opensearch-project/cross-cluster-replication/pull/795)) +* Handled exception under multi-field mapping update ([#789](https://github.com/opensearch-project/cross-cluster-replication/pull/789)) +* Handled batch requests for replication metadata update under cluster state ([#778](https://github.com/opensearch-project/cross-cluster-replication/pull/778)) + +### Infrastructure +* Added support for running Integtest on Remote clusters ([#733](https://github.com/opensearch-project/cross-cluster-replication/pull/733)) From 55b6968af90d739d8448c5d88ed5204a240d0f86 Mon Sep 17 00:00:00 2001 From: Rishikesh Pasham <62345295+Rishikesh1159@users.noreply.github.com> Date: Mon, 17 Apr 2023 21:21:42 -0700 Subject: [PATCH 077/157] Add setting to use document replication for system indices. (#802) Signed-off-by: Rishikesh1159 --- .../replication/metadata/store/ReplicationMetadataStore.kt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/main/kotlin/org/opensearch/replication/metadata/store/ReplicationMetadataStore.kt b/src/main/kotlin/org/opensearch/replication/metadata/store/ReplicationMetadataStore.kt index ba0122ee..18a0cd6c 100644 --- a/src/main/kotlin/org/opensearch/replication/metadata/store/ReplicationMetadataStore.kt +++ b/src/main/kotlin/org/opensearch/replication/metadata/store/ReplicationMetadataStore.kt @@ -40,6 +40,7 @@ import org.opensearch.common.xcontent.LoggingDeprecationHandler import org.opensearch.core.xcontent.NamedXContentRegistry import org.opensearch.core.xcontent.ToXContent import org.opensearch.core.xcontent.XContentParser +import org.opensearch.indices.replication.common.ReplicationType import org.opensearch.replication.util.suspendExecuteWithRetries class ReplicationMetadataStore constructor(val client: Client, val clusterService: ClusterService, @@ -265,6 +266,7 @@ class ReplicationMetadataStore constructor(val client: Client, val clusterServic .put(IndexMetadata.INDEX_AUTO_EXPAND_REPLICAS_SETTING.key, "0-1") .put(IndexMetadata.INDEX_PRIORITY_SETTING.key, Int.MAX_VALUE) .put(IndexMetadata.INDEX_HIDDEN_SETTING.key, true) + .put(IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.key, ReplicationType.DOCUMENT) // System Indices should use Document Replication strategy .build() } From 641c5f751027cc0b5c415963079b812e2abef455 Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Tue, 18 Apr 2023 10:21:01 +0530 Subject: [PATCH 078/157] BWC fix: Changed version to 2.6 and disabled rolling upgrade tests (#801) Signed-off-by: Ankit Kala --- .github/workflows/bwc.yml | 1 - build.gradle | 12 +++++++----- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/.github/workflows/bwc.yml b/.github/workflows/bwc.yml index 547edea0..432c7dce 100644 --- a/.github/workflows/bwc.yml +++ b/.github/workflows/bwc.yml @@ -26,7 +26,6 @@ jobs: run: | echo "Running backwards compatibility tests ..." ./gradlew clean release -Dbuild.snapshot=true -x test -x IntegTest - ./gradlew mixedClusterTask --stacktrace ./gradlew fullRestartClusterTask --stacktrace - name: Upload failed logs uses: actions/upload-artifact@v2 diff --git a/build.gradle b/build.gradle index e916f99d..c8fc2dc7 100644 --- a/build.gradle +++ b/build.gradle @@ -49,7 +49,7 @@ buildscript { // for bwc tests - opensearch_previous_version = System.getProperty("bwc_older_version", "2.1.0") + opensearch_previous_version = System.getProperty("bwc_older_version", "2.6.0") plugin_previous_version = opensearch_previous_version.replaceAll(/(\.\d)([^\d]*)$/, '$1.0$2') common_utils_version = System.getProperty("common_utils.version", opensearch_build) @@ -851,8 +851,9 @@ task "fullRestartClusterTask"(type: RestIntegTestTask) { - full cluster restart: oldVersionClusterTask1 --> fullRestartClusterTask */ task "bwcTestSuite"(type: RestIntegTestTask) { - useCluster testClusters.bwcLeader0 - useCluster testClusters.bwcFollower0 + // Disabling the bwcLeader0 & bwcFollower0 as rolling upgrade has been disabled. + //useCluster testClusters.bwcLeader0 + //useCluster testClusters.bwcFollower0 useCluster testClusters.bwcLeader1 useCluster testClusters.bwcFollower1 doFirst { @@ -875,8 +876,9 @@ task "bwcTestSuite"(type: RestIntegTestTask) { } nonInputProperties.systemProperty('tests.cluster_suffix', "1") nonInputProperties.systemProperty('tests.bwcTask', "bwcTestSuite") - dependsOn tasks.named("mixedClusterTask") - dependsOn tasks.named("rollingUpgradeClusterTask") + // Disabling the rolling upgrade bwc tests as 2.x -> 3.0 is not supported yet. + //dependsOn tasks.named("mixedClusterTask") + //dependsOn tasks.named("rollingUpgradeClusterTask") dependsOn tasks.named("fullRestartClusterTask") } From 58501b0eb32fff8f9bf74e456f462ffc93e38e09 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Mon, 24 Apr 2023 10:24:17 +0530 Subject: [PATCH 079/157] FIx to use List instead of ImmutableOpenMap (#811) Fix build failures due to upstream changes. Signed-off-by: Monu Singh --- .../repository/RemoteClusterRepository.kt | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt index 4dc9d765..bf0392e9 100644 --- a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt +++ b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt @@ -188,15 +188,14 @@ class RemoteClusterRepository(private val repositoryMetadata: RepositoryMetadata override fun getRepositoryData(listener: ActionListener) { val clusterState = getLeaderClusterState(false, false) val shardGenerations = ShardGenerations.builder() - clusterState.metadata.indices.values() - .map { it.value } - .forEach { indexMetadata -> - val indexId = IndexId(indexMetadata.index.name, indexMetadata.indexUUID) - for (i in 0 until indexMetadata.numberOfShards) { - // Generations only make sense for eventually consistent BlobStores so just use a dummy value here. - shardGenerations.put(indexId, i, "dummy") - } + clusterState.metadata.indices.values + .forEach { indexMetadata -> + val indexId = IndexId(indexMetadata.index.name, indexMetadata.indexUUID) + for (i in 0 until indexMetadata.numberOfShards) { + // Generations only make sense for eventually consistent BlobStores so just use a dummy value here. + shardGenerations.put(indexId, i, "dummy") } + } val snapshotId = SnapshotId(REMOTE_SNAPSHOT_NAME, REMOTE_SNAPSHOT_NAME.asUUID()) val repositoryData = RepositoryData.EMPTY .addSnapshot(snapshotId, SnapshotState.SUCCESS, Version.CURRENT, shardGenerations.build(), null, null) @@ -210,7 +209,7 @@ class RemoteClusterRepository(private val repositoryMetadata: RepositoryMetadata override fun getSnapshotInfo(snapshotId: SnapshotId): SnapshotInfo { val leaderClusterState = getLeaderClusterState(false, false) assert(REMOTE_SNAPSHOT_NAME.equals(snapshotId.name), { "SnapshotName differs" }) - val indices = leaderClusterState.metadata().indices().keys().map { x -> x.value } + val indices = leaderClusterState.metadata().indices().keys.toList() return SnapshotInfo(snapshotId, indices, emptyList(), SnapshotState.SUCCESS, Version.CURRENT) } From 3b43e55ce05011eca30432bcbdd20a1c6b405d5a Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Mon, 24 Apr 2023 15:50:29 +0530 Subject: [PATCH 080/157] Allow knn index to be used with ccr (#760) * Allow knn index to be replicated Signed-off-by: Monu Singh --- .github/workflows/security-knn-tests.yml | 119 ++++++++++++++++++ .github/workflows/security-tests.yml | 73 ----------- build.gradle | 40 +++++- .../replication/ReplicationPlugin.kt | 3 +- .../index/TransportReplicateIndexAction.kt | 11 +- .../TransportResumeIndexReplicationAction.kt | 10 +- .../replication/util/ValidationUtil.kt | 16 +++ .../replication/BasicReplicationIT.kt | 65 ++++++++++ .../replication/MultiClusterRestTestCase.kt | 2 + 9 files changed, 251 insertions(+), 88 deletions(-) create mode 100644 .github/workflows/security-knn-tests.yml delete mode 100644 .github/workflows/security-tests.yml diff --git a/.github/workflows/security-knn-tests.yml b/.github/workflows/security-knn-tests.yml new file mode 100644 index 00000000..de049f0c --- /dev/null +++ b/.github/workflows/security-knn-tests.yml @@ -0,0 +1,119 @@ +name: Security and knn tests +# This workflow is triggered on pull requests to main branch +on: + pull_request: + branches: + - '*' + push: + branches: + - '*' + +jobs: + req: + # Job name + name: plugin check + runs-on: ubuntu-latest + outputs: + isSecurityPluginAvailable: ${{ steps.plugin-availability-check.outputs.isSecurityPluginAvailable }} + isKnnPluginAvailable: ${{ steps.plugin-availability-check.outputs.isKnnPluginAvailable }} + steps: + # This step uses the checkout Github action: https://github.com/actions/checkout + - name: Checkout Branch + uses: actions/checkout@v2 + - id: plugin-availability-check + name: "plugin check" + run: | + opensearch_version=$(grep "System.getProperty(\"opensearch.version\", \"" build.gradle | grep '\([0-9]\|[.]\)\{5\}' -o) + opensearch_version=$opensearch_version".0-SNAPSHOT" + # we publish build artifacts to the below url + sec_plugin_url="https://aws.oss.sonatype.org/content/repositories/snapshots/org/opensearch/plugin/opensearch-security/"$opensearch_version"/" + sec_st=$(curl -s -o /dev/null -w "%{http_code}" $sec_plugin_url) + if [ "$sec_st" = "200" ]; then + echo "isSecurityPluginAvailable=True" >> $GITHUB_OUTPUT + cat $GITHUB_OUTPUT + else + echo "isSecurityPluginAvailable=False" >> $GITHUB_OUTPUT + cat $GITHUB_OUTPUT + fi + knn_plugin_url="https://aws.oss.sonatype.org/content/repositories/snapshots/org/opensearch/plugin/opensearch-knn/"$opensearch_version"/" + knn_st=$(curl -s -o /dev/null -w "%{http_code}" $knn_plugin_url) + if [ "$knn_st" = "200" ]; then + echo "isKnnPluginAvailable=True" >> $GITHUB_OUTPUT + cat $GITHUB_OUTPUT + else + echo "isKnnPluginAvailable=False" >> $GITHUB_OUTPUT + cat $GITHUB_OUTPUT + fi + + build: + needs: req + if: ${{ 'True' == needs.req.outputs.isSecurityPluginAvailable }} + # Job name + name: Build and Run Security tests + runs-on: ubuntu-latest + steps: + # This step uses the setup-java Github action: https://github.com/actions/setup-java + - name: Set Up JDK 17 + uses: actions/setup-java@v1 + with: + java-version: 17 + # This step uses the checkout Github action: https://github.com/actions/checkout + - name: Checkout Branch + uses: actions/checkout@v2 + - name: Build and run Replication tests + run: | + ls -al src/test/resources/security/plugin + ./gradlew clean release -Dbuild.snapshot=true -PnumNodes=1 -Psecurity=true + - name: Upload failed logs + uses: actions/upload-artifact@v2 + if: failure() + with: + name: logs + path: | + build/testclusters/integTest-*/logs/* + build/testclusters/leaderCluster-*/logs/* + build/testclusters/followCluster-*/logs/* + - name: Create Artifact Path + run: | + mkdir -p cross-cluster-replication-artifacts + cp ./build/distributions/*.zip cross-cluster-replication-artifacts + - name: Uploads coverage + with: + fetch-depth: 2 + uses: codecov/codecov-action@v1.2.1 + + knn-build: + needs: req + if: ${{ 'True' == needs.req.outputs.isKnnPluginAvailable }} + # Job name + name: Build and Run Knn tests + runs-on: ubuntu-latest + steps: + # This step uses the setup-java Github action: https://github.com/actions/setup-java + - name: Set Up JDK 17 + uses: actions/setup-java@v1 + with: + java-version: 17 + # This step uses the checkout Github action: https://github.com/actions/checkout + - name: Checkout Branch + uses: actions/checkout@v2 + - name: Build and run Replication tests + run: | + ./gradlew clean release -Dbuild.snapshot=true -PnumNodes=1 -Dtests.class=org.opensearch.replication.BasicReplicationIT -Dtests.method="test knn index replication" -Pknn=true + - name: Upload failed logs + uses: actions/upload-artifact@v2 + if: failure() + with: + name: logs + path: | + build/testclusters/integTest-*/logs/* + build/testclusters/leaderCluster-*/logs/* + build/testclusters/followCluster-*/logs/* + - name: Create Artifact Path + run: | + mkdir -p cross-cluster-replication-artifacts + cp ./build/distributions/*.zip cross-cluster-replication-artifacts + - name: Uploads coverage + with: + fetch-depth: 2 + uses: codecov/codecov-action@v1.2.1 \ No newline at end of file diff --git a/.github/workflows/security-tests.yml b/.github/workflows/security-tests.yml deleted file mode 100644 index ddf6046e..00000000 --- a/.github/workflows/security-tests.yml +++ /dev/null @@ -1,73 +0,0 @@ -name: Security tests -# This workflow is triggered on pull requests to main branch -on: - pull_request: - branches: - - '*' - push: - branches: - - '*' - -jobs: - req: - # Job name - name: Security plugin check - runs-on: ubuntu-latest - outputs: - isSecurityPluginAvailable: ${{ steps.security-plugin-availability-check.outputs.isSecurityPluginAvailable }} - steps: - # This step uses the checkout Github action: https://github.com/actions/checkout - - name: Checkout Branch - uses: actions/checkout@v2 - - id: security-plugin-availability-check - name: "Security plugin check" - run: | - opensearch_version=$(grep "System.getProperty(\"opensearch.version\", \"" build.gradle | grep '\([0-9]\|[.]\)\{5\}' -o) - opensearch_version=$opensearch_version".0-SNAPSHOT" - # we publish build artifacts to the below url - plugin_url="https://aws.oss.sonatype.org/content/repositories/snapshots/org/opensearch/plugin/opensearch-security/"$opensearch_version"/" - st=$(curl -s -o /dev/null -w "%{http_code}" $plugin_url) - if [ "$st" = "200" ]; then - echo "isSecurityPluginAvailable=True" >> $GITHUB_OUTPUT - cat $GITHUB_OUTPUT - else - echo "isSecurityPluginAvailable=False" >> $GITHUB_OUTPUT - cat $GITHUB_OUTPUT - fi - - build: - needs: req - if: ${{ 'True' == needs.req.outputs.isSecurityPluginAvailable }} - # Job name - name: Build and Run Security tests - runs-on: ubuntu-latest - steps: - # This step uses the setup-java Github action: https://github.com/actions/setup-java - - name: Set Up JDK 17 - uses: actions/setup-java@v1 - with: - java-version: 17 - # This step uses the checkout Github action: https://github.com/actions/checkout - - name: Checkout Branch - uses: actions/checkout@v2 - - name: Build and run Replication tests - run: | - ls -al src/test/resources/security/plugin - ./gradlew clean release -Dbuild.snapshot=true -PnumNodes=1 -Psecurity=true - - name: Upload failed logs - uses: actions/upload-artifact@v2 - if: failure() - with: - name: logs - path: | - build/testclusters/integTest-*/logs/* - build/testclusters/leaderCluster-*/logs/* - build/testclusters/followCluster-*/logs/* - - name: Create Artifact Path - run: | - mkdir -p cross-cluster-replication-artifacts - cp ./build/distributions/*.zip cross-cluster-replication-artifacts - - name: Uploads coverage - with: - fetch-depth: 2 - uses: codecov/codecov-action@v1.2.1 diff --git a/build.gradle b/build.gradle index c8fc2dc7..bce10727 100644 --- a/build.gradle +++ b/build.gradle @@ -59,6 +59,8 @@ buildscript { if (!isSnapshot) { security_plugin_version = opensearch_build.replace("-SNAPSHOT","") } + knn_plugin_version = security_plugin_version + } repositories { @@ -234,6 +236,29 @@ def securityPluginFile = new Callable() { } } +def knnEnabled = findProperty("knn") == "true" + +if(knnEnabled == true ){ + dependencies{ + opensearchPlugin "org.opensearch.plugin:opensearch-knn:${knn_plugin_version}@zip" + } +} + +def knnPluginFile = new Callable() { + @Override + RegularFile call() throws Exception { + return new RegularFile() { + @Override + File getAsFile() { + return configurations.opensearchPlugin.resolvedConfiguration.resolvedArtifacts + .find { ResolvedArtifact f -> f.name.contains('opensearch-knn') } + .file + } + } + } +} + + // Clone of WaitForHttpResource with updated code to support Cross cluster usecase class CrossClusterWaitForHttpResource { @@ -350,8 +375,13 @@ testClusters { if(securityEnabled) { plugin(provider(securityPluginFile)) } - int debugPort = 5005 testDistribution = "INTEG_TEST" + if(knnEnabled) { + plugin(provider(knnPluginFile)) + testDistribution = "ARCHIVE" + } + int debugPort = 5005 + if (_numNodes > 1) numberOfNodes = _numNodes //numberOfNodes = 3 setting 'path.repo', repo.absolutePath @@ -363,6 +393,10 @@ testClusters { if(securityEnabled) { plugin(provider(securityPluginFile)) } + if(knnEnabled) { + plugin(provider(knnPluginFile)) + testDistribution = "ARCHIVE" + } int debugPort = 5010 if (_numNodes > 1) numberOfNodes = _numNodes //numberOfNodes = 3 @@ -418,6 +452,10 @@ integTest { useCluster testClusters.leaderCluster useCluster testClusters.followCluster + if(knnEnabled){ + nonInputProperties.systemProperty('tests.knn_plugin_enabled', "true") + } + // We skip BWC test here as those get run as part of separate target `bwcTestSuite`. filter { excludeTestsMatching "org.opensearch.replication.bwc.*IT" diff --git a/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt b/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt index 9b50bb2b..3ea64fe0 100644 --- a/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt +++ b/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt @@ -156,9 +156,10 @@ internal class ReplicationPlugin : Plugin(), ActionPlugin, PersistentTaskPlugin, private var followerClusterStats = FollowerClusterStats() companion object { + const val KNN_INDEX_SETTING = "index.knn" + const val KNN_PLUGIN_PRESENT_SETTING = "knn.plugin.enabled" const val REPLICATION_EXECUTOR_NAME_LEADER = "replication_leader" const val REPLICATION_EXECUTOR_NAME_FOLLOWER = "replication_follower" - const val KNN_INDEX_SETTING = "index.knn" val REPLICATED_INDEX_SETTING: Setting = Setting.simpleString("index.plugins.replication.follower.leader_index", Setting.Property.InternalIndex, Setting.Property.IndexScope) val REPLICATION_FOLLOWER_OPS_BATCH_SIZE: Setting = Setting.intSetting("plugins.replication.follower.index.ops_batch_size", 50000, 16, diff --git a/src/main/kotlin/org/opensearch/replication/action/index/TransportReplicateIndexAction.kt b/src/main/kotlin/org/opensearch/replication/action/index/TransportReplicateIndexAction.kt index 80b0e30b..becb1360 100644 --- a/src/main/kotlin/org/opensearch/replication/action/index/TransportReplicateIndexAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/index/TransportReplicateIndexAction.kt @@ -36,15 +36,16 @@ import org.opensearch.cluster.ClusterState import org.opensearch.cluster.metadata.MetadataCreateIndexService import org.opensearch.common.inject.Inject import org.opensearch.common.settings.Settings +import org.opensearch.cluster.service.ClusterService import org.opensearch.env.Environment import org.opensearch.index.IndexNotFoundException import org.opensearch.index.IndexSettings -import org.opensearch.replication.ReplicationPlugin.Companion.KNN_INDEX_SETTING import org.opensearch.tasks.Task import org.opensearch.threadpool.ThreadPool import org.opensearch.transport.TransportService class TransportReplicateIndexAction @Inject constructor(transportService: TransportService, + private val clusterService: ClusterService, val threadPool: ThreadPool, actionFilters: ActionFilters, private val client : Client, @@ -98,12 +99,8 @@ class TransportReplicateIndexAction @Inject constructor(transportService: Transp if (!leaderSettings.getAsBoolean(IndexSettings.INDEX_SOFT_DELETES_SETTING.key, false)) { throw IllegalArgumentException("Cannot Replicate an index where the setting ${IndexSettings.INDEX_SOFT_DELETES_SETTING.key} is disabled") } - - // For k-NN indices, k-NN loads its own engine and this conflicts with the replication follower engine - // Blocking k-NN indices for replication - if(leaderSettings.getAsBoolean(KNN_INDEX_SETTING, false)) { - throw IllegalArgumentException("Cannot replicate k-NN index - ${request.leaderIndex}") - } + //Not starting replication if leader index is knn as knn plugin is not installed on follower. + ValidationUtil.checkKNNEligibility(leaderSettings, clusterService, request.leaderIndex) ValidationUtil.validateIndexSettings( environment, diff --git a/src/main/kotlin/org/opensearch/replication/action/resume/TransportResumeIndexReplicationAction.kt b/src/main/kotlin/org/opensearch/replication/action/resume/TransportResumeIndexReplicationAction.kt index 7be95e67..55d87cf0 100644 --- a/src/main/kotlin/org/opensearch/replication/action/resume/TransportResumeIndexReplicationAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/resume/TransportResumeIndexReplicationAction.kt @@ -49,11 +49,12 @@ import org.opensearch.cluster.metadata.IndexMetadata import org.opensearch.cluster.metadata.IndexNameExpressionResolver import org.opensearch.cluster.service.ClusterService import org.opensearch.common.inject.Inject +import org.opensearch.replication.ReplicationPlugin.Companion.KNN_INDEX_SETTING +import org.opensearch.replication.ReplicationPlugin.Companion.KNN_PLUGIN_PRESENT_SETTING import org.opensearch.common.io.stream.StreamInput import org.opensearch.env.Environment import org.opensearch.index.IndexNotFoundException import org.opensearch.index.shard.ShardId -import org.opensearch.replication.ReplicationPlugin.Companion.KNN_INDEX_SETTING import org.opensearch.threadpool.ThreadPool import org.opensearch.transport.TransportService import java.io.IOException @@ -102,11 +103,8 @@ class TransportResumeIndexReplicationAction @Inject constructor(transportService val leaderSettings = settingsResponse.indexToSettings.get(params.leaderIndex.name) ?: throw IndexNotFoundException(params.leaderIndex.name) - // k-NN Setting is a static setting. In case the setting is changed at the leader index before resume, - // block the resume. - if(leaderSettings.getAsBoolean(KNN_INDEX_SETTING, false)) { - throw IllegalStateException("Cannot resume replication for k-NN enabled index ${params.leaderIndex.name}.") - } + /// Not starting replication if leader index is knn as knn plugin is not installed on follower. + ValidationUtil.checkKNNEligibility(leaderSettings, clusterService, params.leaderIndex.name) ValidationUtil.validateAnalyzerSettings(environment, leaderSettings, replMetdata.settings) diff --git a/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt b/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt index 5f9a6aee..9f0bb4fc 100644 --- a/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt +++ b/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt @@ -23,6 +23,9 @@ import org.opensearch.common.settings.Settings import org.opensearch.env.Environment import org.opensearch.index.IndexNotFoundException import java.io.UnsupportedEncodingException +import org.opensearch.cluster.service.ClusterService +import org.opensearch.replication.ReplicationPlugin.Companion.KNN_INDEX_SETTING +import org.opensearch.replication.ReplicationPlugin.Companion.KNN_PLUGIN_PRESENT_SETTING import java.nio.file.Files import java.nio.file.Path import java.util.Locale @@ -138,4 +141,17 @@ object ValidationUtil { throw validationException } } + + /** + * Throw exception if leader index is knn a knn is not installed + */ + fun checkKNNEligibility(leaderSettings: Settings, clusterService: ClusterService, leaderIndex: String) { + if(leaderSettings.getAsBoolean(KNN_INDEX_SETTING, false)) { + if(clusterService.clusterSettings.get(KNN_PLUGIN_PRESENT_SETTING) == null){ + throw IllegalStateException("Cannot proceed with replication for k-NN enabled index ${leaderIndex} as knn plugin is not installed.") + } + } + + } + } diff --git a/src/test/kotlin/org/opensearch/replication/BasicReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/BasicReplicationIT.kt index ff3bfa83..c87fe4ec 100644 --- a/src/test/kotlin/org/opensearch/replication/BasicReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/BasicReplicationIT.kt @@ -23,8 +23,10 @@ import org.opensearch.action.get.GetRequest import org.opensearch.action.index.IndexRequest import org.opensearch.client.RequestOptions import org.opensearch.client.indices.CreateIndexRequest +import org.opensearch.common.xcontent.XContentType import org.opensearch.common.CheckedRunnable import org.opensearch.test.OpenSearchTestCase.assertBusy +import org.opensearch.client.indices.PutMappingRequest import org.junit.Assert import java.util.Locale import java.util.concurrent.TimeUnit @@ -83,6 +85,69 @@ class BasicReplicationIT : MultiClusterRestTestCase() { "blocked by: [FORBIDDEN/1000/index read-only(cross-cluster-replication)];]") } + fun `test knn index replication`() { + + + val followerClient = getClientForCluster(FOLL) + val leaderClient = getClientForCluster(LEADER) + createConnectionBetweenClusters(FOLL, LEADER) + val leaderIndexName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT) + val followerIndexNameInitial = randomAlphaOfLength(10).toLowerCase(Locale.ROOT) + val followerIndexName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT) + val KNN_INDEX_MAPPING = "{\"properties\":{\"my_vector1\":{\"type\":\"knn_vector\",\"dimension\":2},\"my_vector2\":{\"type\":\"knn_vector\",\"dimension\":4}}}" + // create knn-index on leader cluster + try { + val createIndexResponse = leaderClient.indices().create( + CreateIndexRequest(leaderIndexName) + .mapping(KNN_INDEX_MAPPING, XContentType.JSON), RequestOptions.DEFAULT + ) + assertThat(createIndexResponse.isAcknowledged).isTrue() + } catch (e: Exception){ + //index creation will fail if Knn plugin is not installed + assumeNoException("Could not create Knn index on leader cluster", e) + } + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName), waitForRestore=true) + // Create document + var source = mapOf("my_vector1" to listOf(2.5,3.5) , "price" to 7.1) + var response = leaderClient.index(IndexRequest(leaderIndexName).id("1").source(source), RequestOptions.DEFAULT) + assertThat(response.result).withFailMessage("Failed to create leader data").isEqualTo(Result.CREATED) + assertBusy({ + val getResponse = followerClient.get(GetRequest(followerIndexName, "1"), RequestOptions.DEFAULT) + assertThat(getResponse.isExists).isTrue() + assertThat(getResponse.sourceAsMap).isEqualTo(source) + }, 60L, TimeUnit.SECONDS) + + // Update document + source = mapOf("my_vector1" to listOf(3.5,4.5) , "price" to 12.9) + response = leaderClient.index(IndexRequest(leaderIndexName).id("1").source(source), RequestOptions.DEFAULT) + assertThat(response.result).withFailMessage("Failed to update leader data").isEqualTo(Result.UPDATED) + assertBusy({ + val getResponse = followerClient.get(GetRequest(followerIndexName, "1"), RequestOptions.DEFAULT) + assertThat(getResponse.isExists).isTrue() + assertThat(getResponse.sourceAsMap).isEqualTo(source) + },60L, TimeUnit.SECONDS) + val KNN_INDEX_MAPPING1 = "{\"properties\":{\"my_vector1\":{\"type\":\"knn_vector\",\"dimension\":2},\"my_vector2\":{\"type\":\"knn_vector\",\"dimension\":4},\"my_vector3\":{\"type\":\"knn_vector\",\"dimension\":4}}}" + val updateIndexResponse = leaderClient.indices().putMapping( + PutMappingRequest(leaderIndexName).source(KNN_INDEX_MAPPING1, XContentType.JSON) , RequestOptions.DEFAULT + ) + source = mapOf("my_vector3" to listOf(3.1,4.5,5.7,8.9) , "price" to 17.9) + response = leaderClient.index(IndexRequest(leaderIndexName).id("2").source(source), RequestOptions.DEFAULT) + assertThat(response.result).withFailMessage("Failed to update leader data").isEqualTo(Result.CREATED) + assertBusy({ + val getResponse = followerClient.get(GetRequest(followerIndexName, "2"), RequestOptions.DEFAULT) + assertThat(getResponse.isExists).isTrue() + assertThat(getResponse.sourceAsMap).isEqualTo(source) + },60L, TimeUnit.SECONDS) + assertThat(updateIndexResponse.isAcknowledged).isTrue() + // Delete document + val deleteResponse = leaderClient.delete(DeleteRequest(leaderIndexName).id("1"), RequestOptions.DEFAULT) + assertThat(deleteResponse.result).withFailMessage("Failed to delete leader data").isEqualTo(Result.DELETED) + assertBusy({ + val getResponse = followerClient.get(GetRequest(followerIndexName, "1"), RequestOptions.DEFAULT) + assertThat(getResponse.isExists).isFalse() + }, 60L, TimeUnit.SECONDS) + } + fun `test existing index replication`() { val follower = getClientForCluster(FOLL) val leader = getClientForCluster(LEADER) diff --git a/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt b/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt index 0ac689a7..5969156c 100644 --- a/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt +++ b/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt @@ -645,4 +645,6 @@ abstract class MultiClusterRestTestCase : OpenSearchTestCase() { val integTestRemote = systemProperties.get("tests.integTestRemote") as String? return integTestRemote.equals("true") } + + } From 49e8781df5bf9d48525302bfc5e523c8c28c55ef Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Wed, 10 May 2023 20:07:47 +0530 Subject: [PATCH 081/157] Open Upgrade gradle version and remove remaining use of ImmutableOpenMap (#814) Signed-off-by: Monu Singh --- gradle/wrapper/gradle-wrapper.properties | 2 +- .../TransportResumeIndexReplicationAction.kt | 6 ++-- .../metadata/TransportUpdateMetadataAction.kt | 4 +-- .../metadata/UpdateIndexBlockTask.kt | 8 ++--- .../repository/RemoteClusterRepository.kt | 2 +- .../RemoteClusterRetentionLeaseHelper.kt | 4 +-- .../task/index/IndexReplicationTask.kt | 36 +++++++++---------- .../integ/rest/SecurityCustomRolesIT.kt | 6 ++-- .../integ/rest/SecurityDlsFlsIT.kt | 2 +- .../integ/rest/StartReplicationIT.kt | 28 +++++++-------- .../integ/rest/UpdateAutoFollowPatternIT.kt | 2 +- .../replication/task/index/NoOpClient.kt | 8 ++--- 12 files changed, 52 insertions(+), 56 deletions(-) diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index f51fff5e..68efe1de 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -11,6 +11,6 @@ distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists -distributionUrl=https\://services.gradle.org/distributions/gradle-7.4.2-all.zip +distributionUrl=https\://services.gradle.org/distributions/gradle-7.6.1-all.zip zipStoreBase=GRADLE_USER_HOME zipStorePath=wrapper/dists diff --git a/src/main/kotlin/org/opensearch/replication/action/resume/TransportResumeIndexReplicationAction.kt b/src/main/kotlin/org/opensearch/replication/action/resume/TransportResumeIndexReplicationAction.kt index 55d87cf0..39472736 100644 --- a/src/main/kotlin/org/opensearch/replication/action/resume/TransportResumeIndexReplicationAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/resume/TransportResumeIndexReplicationAction.kt @@ -131,9 +131,9 @@ class TransportResumeIndexReplicationAction @Inject constructor(transportService private suspend fun isResumable(params :IndexReplicationParams): Boolean { var isResumable = true val remoteClient = client.getRemoteClusterClient(params.leaderAlias) - val shards = clusterService.state().routingTable.indicesRouting().get(params.followerIndexName).shards() + val shards = clusterService.state().routingTable.indicesRouting().get(params.followerIndexName)?.shards() val retentionLeaseHelper = RemoteClusterRetentionLeaseHelper(clusterService.clusterName.value(), remoteClient) - shards.forEach { + shards?.forEach { val followerShardId = it.value.shardId if (!retentionLeaseHelper.verifyRetentionLeaseExist(ShardId(params.leaderIndex, followerShardId.id), followerShardId)) { isResumable = false @@ -146,7 +146,7 @@ class TransportResumeIndexReplicationAction @Inject constructor(transportService // clean up all retention leases we may have accidentally took while doing verifyRetentionLeaseExist . // Idempotent Op which does no harm - shards.forEach { + shards?.forEach { val followerShardId = it.value.shardId log.debug("Removing lease for $followerShardId.id ") retentionLeaseHelper.attemptRetentionLeaseRemoval(ShardId(params.leaderIndex, followerShardId.id), followerShardId) diff --git a/src/main/kotlin/org/opensearch/replication/metadata/TransportUpdateMetadataAction.kt b/src/main/kotlin/org/opensearch/replication/metadata/TransportUpdateMetadataAction.kt index 2fab74ab..eb3b1750 100644 --- a/src/main/kotlin/org/opensearch/replication/metadata/TransportUpdateMetadataAction.kt +++ b/src/main/kotlin/org/opensearch/replication/metadata/TransportUpdateMetadataAction.kt @@ -274,8 +274,8 @@ class TransportUpdateMetadataAction @Inject constructor( val indexAsArray = arrayOf(concreteIndex) val aliasMetadata = metadata.findAliases(action, indexAsArray) val finalAliases: MutableList = ArrayList() - for (curAliases in aliasMetadata.values()) { - for (aliasMeta in curAliases.value) { + for (curAliases in aliasMetadata.values) { + for (aliasMeta in curAliases) { finalAliases.add(aliasMeta.alias()) } } diff --git a/src/main/kotlin/org/opensearch/replication/metadata/UpdateIndexBlockTask.kt b/src/main/kotlin/org/opensearch/replication/metadata/UpdateIndexBlockTask.kt index 8d9d385f..3f8e9a90 100644 --- a/src/main/kotlin/org/opensearch/replication/metadata/UpdateIndexBlockTask.kt +++ b/src/main/kotlin/org/opensearch/replication/metadata/UpdateIndexBlockTask.kt @@ -22,10 +22,10 @@ import org.opensearch.cluster.block.ClusterBlockException import org.opensearch.cluster.block.ClusterBlockLevel import org.opensearch.cluster.block.ClusterBlocks import org.opensearch.cluster.service.ClusterService -import org.opensearch.common.collect.ImmutableOpenMap import org.opensearch.index.IndexNotFoundException import org.opensearch.rest.RestStatus -import java.util.* +import java.util.Collections +import java.util.EnumSet /* This is our custom index block to prevent changes to follower @@ -49,11 +49,11 @@ fun checkIfIndexBlockedWithLevel(clusterService: ClusterService, clusterBlockLevel: ClusterBlockLevel) { clusterService.state().routingTable.index(indexName) ?: throw IndexNotFoundException("Index with name:$indexName doesn't exist") - val writeIndexBlockMap : ImmutableOpenMap> = clusterService.state().blocks() + val writeIndexBlockMap : Map> = clusterService.state().blocks() .indices(clusterBlockLevel) if (!writeIndexBlockMap.containsKey(indexName)) return - val clusterBlocksSet : Set = writeIndexBlockMap.get(indexName) + val clusterBlocksSet : Set = writeIndexBlockMap.getOrDefault(indexName, Collections.emptySet()) if (clusterBlocksSet.contains(INDEX_REPLICATION_BLOCK) && clusterBlocksSet.size > 1) throw ClusterBlockException(clusterBlocksSet) diff --git a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt index bf0392e9..1036317c 100644 --- a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt +++ b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt @@ -238,7 +238,7 @@ class RemoteClusterRepository(private val repositoryMetadata: RepositoryMetadata builder.remove(REPLICATION_INDEX_TRANSLOG_PRUNING_ENABLED_SETTING.key) val indexMdBuilder = IndexMetadata.builder(indexMetadata).settings(builder) - indexMetadata.aliases.valuesIt().forEach { + indexMetadata.aliases.values.forEach { indexMdBuilder.putAlias(it) } return indexMdBuilder.build() diff --git a/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterRetentionLeaseHelper.kt b/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterRetentionLeaseHelper.kt index 8771e25d..d8daf425 100644 --- a/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterRetentionLeaseHelper.kt +++ b/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterRetentionLeaseHelper.kt @@ -77,9 +77,9 @@ class RemoteClusterRetentionLeaseHelper constructor(val followerClusterName: Str val remoteMetadata = getLeaderIndexMetadata(replMetadata.connectionName, replMetadata.leaderContext.resource) val params = IndexReplicationParams(replMetadata.connectionName, remoteMetadata.index, followerIndexName) val remoteClient = client.getRemoteClusterClient(params.leaderAlias) - val shards = clusterService.state().routingTable.indicesRouting().get(params.followerIndexName).shards() + val shards = clusterService.state().routingTable.indicesRouting().get(params.followerIndexName)?.shards() val retentionLeaseHelper = RemoteClusterRetentionLeaseHelper(clusterService.clusterName.value(), remoteClient) - shards.forEach { + shards?.forEach { val followerShardId = it.value.shardId log.debug("Removing lease for $followerShardId.id ") retentionLeaseHelper.attemptRetentionLeaseRemoval(ShardId(params.leaderIndex, followerShardId.id), followerShardId) diff --git a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt index 4bf8b3d5..29876e85 100644 --- a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt @@ -346,9 +346,9 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript val clusterState = clusterService.state() val persistentTasks = clusterState.metadata.custom(PersistentTasksCustomMetadata.TYPE) - val followerShardIds = clusterService.state().routingTable.indicesRouting().get(followerIndexName).shards() - .map { shard -> shard.value.shardId } - .stream().collect(Collectors.toSet()) + val followerShardIds = clusterService.state().routingTable.indicesRouting().get(followerIndexName)?.shards() + ?.map { shard -> shard.value.shardId } + ?.stream()?.collect(Collectors.toSet()).orEmpty() val runningShardTasksForIndex = persistentTasks.findTasks(ShardReplicationExecutor.TASK_NAME, Predicate { true }).stream() .map { task -> task.params as ShardReplicationParams } .filter {taskParam -> followerShardIds.contains(taskParam.followerShardId) } @@ -434,16 +434,16 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript // If we we want to retrieve just the version of settings and alias versions, there are two options // 1. Include this in GetChanges and communicate it to IndexTask via Metadata // 2. Add another API to retrieve version of settings & aliases. Persist current version in Metadata - var leaderSettings = settingsResponse.indexToSettings.get(this.leaderIndex.name) - leaderSettings = leaderSettings.filter { k: String? -> + var leaderSettings = settingsResponse.indexToSettings.getOrDefault(this.leaderIndex.name, Settings.EMPTY) + leaderSettings = leaderSettings.filter { k: String -> !blockListedSettings.contains(k) } gsr = GetSettingsRequest().includeDefaults(false).indices(this.followerIndexName) settingsResponse = client.suspending(client.admin().indices()::getSettings, injectSecurityContext = true)(gsr) - var followerSettings = settingsResponse.indexToSettings.get(this.followerIndexName) + var followerSettings = settingsResponse.indexToSettings.getOrDefault(this.followerIndexName, Settings.EMPTY) - followerSettings = followerSettings.filter { k: String? -> + followerSettings = followerSettings.filter { k: String -> k != REPLICATED_INDEX_SETTING.key } @@ -516,11 +516,11 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript //Alias var getAliasesRequest = GetAliasesRequest().indices(this.leaderIndex.name) var getAliasesRes = remoteClient.suspending(remoteClient.admin().indices()::getAliases, injectSecurityContext = true)(getAliasesRequest) - var leaderAliases = getAliasesRes.aliases.get(this.leaderIndex.name) + var leaderAliases = getAliasesRes.aliases.getOrDefault(this.leaderIndex.name, Collections.emptyList()) getAliasesRequest = GetAliasesRequest().indices(followerIndexName) getAliasesRes = client.suspending(client.admin().indices()::getAliases, injectSecurityContext = true)(getAliasesRequest) - var followerAliases = getAliasesRes.aliases.get(followerIndexName) + var followerAliases = getAliasesRes.aliases.getOrDefault(followerIndexName, Collections.emptyList()) var request :IndicesAliasesRequest? @@ -606,8 +606,8 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript try { //Step 1 : Remove the tasks - val shards = clusterService.state().routingTable.indicesRouting().get(followerIndexName).shards() - shards.forEach { + val shards = clusterService.state().routingTable.indicesRouting().get(followerIndexName)?.shards() + shards?.forEach { persistentTasksService.removeTask(ShardReplicationTask.taskIdForShard(it.value.shardId)) } @@ -748,7 +748,7 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript suspend fun startNewOrMissingShardTasks(): Map> { assert(clusterService.state().routingTable.hasIndex(followerIndexName)) { "Can't find index $followerIndexName" } - val shards = clusterService.state().routingTable.indicesRouting().get(followerIndexName).shards() + val shards = clusterService.state().routingTable.indicesRouting().get(followerIndexName)?.shards() val persistentTasks = clusterService.state().metadata.custom(PersistentTasksCustomMetadata.TYPE) val runningShardTasks = persistentTasks.findTasks(ShardReplicationExecutor.TASK_NAME, Predicate { true }).stream() .map { task -> task as PersistentTask } @@ -757,14 +757,14 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript {t: PersistentTask -> t.params!!.followerShardId}, {t: PersistentTask -> t})) - val tasks = shards.map { + val tasks = shards?.map { it.value.shardId - }.associate { shardId -> + }?.associate { shardId -> val task = runningShardTasks.getOrElse(shardId) { startReplicationTask(ShardReplicationParams(leaderAlias, ShardId(leaderIndex, shardId.id), shardId)) } return@associate shardId to task - } + }.orEmpty() return tasks } @@ -865,9 +865,9 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript This can happen if there was a badly timed cluster manager node failure.""".trimIndent()) } } else if (restore.state() == RestoreInProgress.State.FAILURE) { - val failureReason = restore.shards().values().find { - it.value.state() == RestoreInProgress.State.FAILURE - }!!.value.reason() + val failureReason = restore.shards().values.find { + it.state() == RestoreInProgress.State.FAILURE + }!!.reason() return FailedState(Collections.emptyMap(), failureReason) } else { return InitFollowState diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesIT.kt index 97038f3e..04f0378b 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesIT.kt @@ -227,7 +227,7 @@ class SecurityCustomRolesIT: SecurityBase() { "1", followerClient.indices() .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] + .indexToSettings.getOrDefault(followerIndexName, Settings.EMPTY)[IndexMetadata.SETTING_NUMBER_OF_REPLICAS] ) settings = Settings.builder() @@ -242,7 +242,7 @@ class SecurityCustomRolesIT: SecurityBase() { "checksum", followerClient.indices() .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName]["index.shard.check_on_startup"] + .indexToSettings.getOrDefault(followerIndexName, Settings.EMPTY)["index.shard.check_on_startup"] ) }, 30L, TimeUnit.SECONDS) } @@ -272,7 +272,7 @@ class SecurityCustomRolesIT: SecurityBase() { "1", followerClient.indices() .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] + .indexToSettings.getOrDefault(followerIndexName, Settings.EMPTY)[IndexMetadata.SETTING_NUMBER_OF_REPLICAS] ) settings = Settings.builder() .put("index.shard.check_on_startup", "checksum") diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityDlsFlsIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityDlsFlsIT.kt index 82e7465d..89d8269a 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityDlsFlsIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityDlsFlsIT.kt @@ -128,7 +128,7 @@ class SecurityDlsFlsIT: SecurityBase() { "1", followerClient.indices() .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] + .indexToSettings.getOrDefault(followerIndexName, Settings.EMPTY)[IndexMetadata.SETTING_NUMBER_OF_REPLICAS] ) settings = Settings.builder() .put("index.shard.check_on_startup", "checksum") diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt index 93997599..53493457 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt @@ -130,7 +130,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { "3", followerClient.indices() .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] + .indexToSettings.getOrDefault(followerIndexName, Settings.EMPTY)[IndexMetadata.SETTING_NUMBER_OF_REPLICAS] ) }, 15, TimeUnit.SECONDS) } @@ -289,7 +289,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { "0", followerClient.indices() .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] + .indexToSettings.getOrDefault(followerIndexName, Settings.EMPTY)[IndexMetadata.SETTING_NUMBER_OF_REPLICAS] ) }, 30L, TimeUnit.SECONDS) } @@ -448,7 +448,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { "0", followerClient.indices() .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] + .indexToSettings.getOrDefault(followerIndexName, Settings.EMPTY)[IndexMetadata.SETTING_NUMBER_OF_REPLICAS] ) settings = Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 2) @@ -469,14 +469,14 @@ class StartReplicationIT: MultiClusterRestTestCase() { "2", followerClient.indices() .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] + .indexToSettings.getOrDefault(followerIndexName, Settings.EMPTY)[IndexMetadata.SETTING_NUMBER_OF_REPLICAS] ) assertEqualAliases() }, 30L, TimeUnit.SECONDS) // Case 2 : Blocklisted setting are not copied Assert.assertNull(followerClient.indices() .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName].get("index.routing.allocation.enable")) + .indexToSettings.getOrDefault(followerIndexName, Settings.EMPTY).get("index.routing.allocation.enable")) //Alias test case 2: Update existing alias aliasAction = IndicesAliasesRequest.AliasActions.add() .index(leaderIndexName) @@ -500,19 +500,19 @@ class StartReplicationIT: MultiClusterRestTestCase() { "3", followerClient.indices() .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] + .indexToSettings.getOrDefault(followerIndexName, Settings.EMPTY)[IndexMetadata.SETTING_NUMBER_OF_REPLICAS] ) Assert.assertEquals( "10s", followerClient.indices() .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName]["index.search.idle.after"] + .indexToSettings.getOrDefault(followerIndexName, Settings.EMPTY)["index.search.idle.after"] ) Assert.assertEquals( "none", followerClient.indices() .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName]["index.routing.allocation.enable"] + .indexToSettings.getOrDefault(followerIndexName, Settings.EMPTY)["index.routing.allocation.enable"] ) assertEqualAliases() }, 30L, TimeUnit.SECONDS) @@ -539,7 +539,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { null, followerClient.indices() .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName]["index.search.idle.after"] + .indexToSettings.getOrDefault(followerIndexName, Settings.EMPTY)["index.search.idle.after"] ) assertEqualAliases() }, 30L, TimeUnit.SECONDS) @@ -568,7 +568,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { "1", followerClient.indices() .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] + .indexToSettings.getOrDefault(followerIndexName, Settings.EMPTY)[IndexMetadata.SETTING_NUMBER_OF_REPLICAS] ) settings = Settings.builder() .put("index.shard.check_on_startup", "checksum") @@ -579,7 +579,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { "checksum", followerClient.indices() .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName]["index.shard.check_on_startup"] + .indexToSettings.getOrDefault(followerIndexName, Settings.EMPTY)["index.shard.check_on_startup"] ) } @@ -1064,7 +1064,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { "2", leaderClient.indices() .getSettings(getLeaderSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[leaderIndexName][IndexMetadata.SETTING_WAIT_FOR_ACTIVE_SHARDS.getKey()] + .indexToSettings.getOrDefault(leaderIndexName, Settings.EMPTY)[IndexMetadata.SETTING_WAIT_FOR_ACTIVE_SHARDS.getKey()] ) }, 15, TimeUnit.SECONDS) @@ -1124,7 +1124,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { "2", leaderClient.indices() .getSettings(getLeaderSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[leaderIndexName][IndexMetadata.SETTING_WAIT_FOR_ACTIVE_SHARDS.getKey()] + .indexToSettings.getOrDefault(leaderIndexName, Settings.EMPTY)[IndexMetadata.SETTING_WAIT_FOR_ACTIVE_SHARDS.getKey()] ) }, 15, TimeUnit.SECONDS) @@ -1176,7 +1176,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { "2", followerClient.indices() .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[followerIndexName][IndexMetadata.SETTING_WAIT_FOR_ACTIVE_SHARDS.getKey()] + .indexToSettings.getOrDefault(followerIndexName, Settings.EMPTY)[IndexMetadata.SETTING_WAIT_FOR_ACTIVE_SHARDS.getKey()] ) }, 15, TimeUnit.SECONDS) } finally { diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt index aad0f993..c8d3620e 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt @@ -170,7 +170,7 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { "3", followerClient.indices() .getSettings(getSettingsRequest, RequestOptions.DEFAULT) - .indexToSettings[leaderIndexName][IndexMetadata.SETTING_NUMBER_OF_REPLICAS] + .indexToSettings.getOrDefault(leaderIndexName, Settings.EMPTY)[IndexMetadata.SETTING_NUMBER_OF_REPLICAS] ) followerClient.waitForShardTaskStart(leaderIndexName, waitForShardTask) }, 15, TimeUnit.SECONDS) diff --git a/src/test/kotlin/org/opensearch/replication/task/index/NoOpClient.kt b/src/test/kotlin/org/opensearch/replication/task/index/NoOpClient.kt index 35af7cb4..34e711fd 100644 --- a/src/test/kotlin/org/opensearch/replication/task/index/NoOpClient.kt +++ b/src/test/kotlin/org/opensearch/replication/task/index/NoOpClient.kt @@ -30,7 +30,6 @@ import org.opensearch.action.get.GetResponse import org.opensearch.action.support.master.AcknowledgedResponse import org.opensearch.common.UUIDs import org.opensearch.common.bytes.BytesReference -import org.opensearch.common.collect.ImmutableOpenMap import org.opensearch.common.settings.Settings import org.opensearch.core.xcontent.ToXContent import org.opensearch.common.xcontent.XContentFactory @@ -55,8 +54,7 @@ import org.opensearch.snapshots.RestoreInfo import org.opensearch.test.OpenSearchTestCase import org.opensearch.test.client.NoOpNodeClient import java.lang.reflect.Field -import java.util.ArrayList -import java.util.HashMap +import java.util.* open class NoOpClient(testName :String) : NoOpNodeClient(testName) { @Override @@ -109,9 +107,7 @@ open class NoOpClient(testName :String) : NoOpNodeClient(testName) { val indexToSettings = HashMap() indexToSettings[IndexReplicationTaskTests.followerIndex] = desiredSettingsBuilder.build() - - val settingsMap = ImmutableOpenMap.builder().putAll(indexToSettings).build() - var settingResponse = GetSettingsResponse(settingsMap, settingsMap) + var settingResponse = GetSettingsResponse(indexToSettings, indexToSettings) listener.onResponse(settingResponse as Response) } else if (action == RecoveryAction.INSTANCE) { val shardRecoveryStates: MutableMap> = HashMap() From 9408b3d2b21dc786af593e7e93ca43181ed5bba9 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Tue, 16 May 2023 11:25:04 +0530 Subject: [PATCH 082/157] Extract number of nodes (#826) * Extract number of nodes Extract number of nodes from the input to pass as PnumNodes Signed-off-by: Monu Singh Signed-off-by: Monu Singh * Update integtest.sh Added further logging Signed-off-by: Monu Singh Signed-off-by: Monu Singh --------- Signed-off-by: Monu Singh --- scripts/integtest.sh | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/scripts/integtest.sh b/scripts/integtest.sh index ece01bc0..1a0498ae 100755 --- a/scripts/integtest.sh +++ b/scripts/integtest.sh @@ -97,10 +97,19 @@ then leader=$(echo $data | cut -d ',' -f1 | cut -d ':' -f1,2 ) follower=$(echo $data | cut -d ',' -f2 | cut -d ':' -f1,2 ) + echo "leader: $leader" + echo "follower: $follower" + + # Get number of nodes, assuming both leader and follower have same number of nodes + numNodes=$((${follower##*:} - ${leader##*:})) + echo "numNodes: $numNodes" LTRANSPORT_PORT=$(echo $data | cut -d ',' -f1 | cut -d ':' -f1,3 ) FTRANSPORT_PORT=$(echo $data | cut -d ',' -f2 | cut -d ':' -f1,3 ) - eval "./gradlew integTestRemote -Dleader.http_host=\"$leader\" -Dfollower.http_host=\"$follower\" -Dfollower.transport_host=\"$FTRANSPORT_PORT\" -Dleader.transport_host=\"$LTRANSPORT_PORT\" -Dsecurity_enabled=\"$SECURITY_ENABLED\" -Duser=\"$USERNAME\" -Dpassword=\"$PASSWORD\" --console=plain " + echo "LTRANSPORT_PORT: $LTRANSPORT_PORT" + echo "FTRANSPORT_PORT: $FTRANSPORT_PORT" + + eval "./gradlew integTestRemote -Dleader.http_host=\"$leader\" -Dfollower.http_host=\"$follower\" -Dfollower.transport_host=\"$FTRANSPORT_PORT\" -Dleader.transport_host=\"$LTRANSPORT_PORT\" -Dsecurity_enabled=\"$SECURITY_ENABLED\" -Duser=\"$USERNAME\" -Dpassword=\"$PASSWORD\" -PnumNodes=$numNodes --console=plain " else # Single cluster From dacfd78638b9682b80784c6c50d323379d202903 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Fri, 19 May 2023 09:19:00 +0530 Subject: [PATCH 083/157] Update to set remoteTranslogEnabled=false (#834) Upstream change https://github.com/opensearch-project/OpenSearch/commit/613f4aa046912b583925a9a03cb2294efd2a002c#diff-73db07f833f37213626303b1b984703bdf4cfb539529aa72e2ad8f55cea0a5e7R168 started failing builds due to mandatory param remoteTranslogEnabled Signed-off-by: Monu Singh --- .../index/translog/ReplicationTranslogDeletionPolicyTests.kt | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/test/kotlin/org/opensearch/index/translog/ReplicationTranslogDeletionPolicyTests.kt b/src/test/kotlin/org/opensearch/index/translog/ReplicationTranslogDeletionPolicyTests.kt index 38387621..8378bbbb 100644 --- a/src/test/kotlin/org/opensearch/index/translog/ReplicationTranslogDeletionPolicyTests.kt +++ b/src/test/kotlin/org/opensearch/index/translog/ReplicationTranslogDeletionPolicyTests.kt @@ -177,7 +177,8 @@ class ReplicationTranslogDeletionPolicyTests : OpenSearchTestCase() { randomNonNegativeLong(), TragicExceptionHolder(), { }, - BigArrays.NON_RECYCLING_INSTANCE + BigArrays.NON_RECYCLING_INSTANCE, + false ) writer = Mockito.spy(writer) Mockito.doReturn(now - (numberOfReaders - gen + 1) * 1000).`when`(writer).lastModifiedTime @@ -193,4 +194,4 @@ class ReplicationTranslogDeletionPolicyTests : OpenSearchTestCase() { } return Tuple(readers, writer) } -} \ No newline at end of file +} From cbc2c96425344fc3c7fc02c5944895ebda64ca11 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Fri, 19 May 2023 11:18:53 +0530 Subject: [PATCH 084/157] Update UpdateIndexHandler.kt to use org.opensearch.core.common.Strings; Update UpdateIndexHandler.kt to use org.opensearch.core.common.Strings; Signed-off-by: Monu Singh Signed-off-by: Monu Singh --- .../org/opensearch/replication/rest/UpdateIndexHandler.kt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/kotlin/org/opensearch/replication/rest/UpdateIndexHandler.kt b/src/main/kotlin/org/opensearch/replication/rest/UpdateIndexHandler.kt index 22e4a6e1..9a5eb086 100644 --- a/src/main/kotlin/org/opensearch/replication/rest/UpdateIndexHandler.kt +++ b/src/main/kotlin/org/opensearch/replication/rest/UpdateIndexHandler.kt @@ -17,7 +17,7 @@ import org.opensearch.replication.task.index.IndexReplicationExecutor.Companion. import org.opensearch.action.support.IndicesOptions import org.opensearch.client.Requests import org.opensearch.client.node.NodeClient -import org.opensearch.common.Strings +import org.opensearch.core.common.Strings import org.opensearch.rest.BaseRestHandler import org.opensearch.rest.BaseRestHandler.RestChannelConsumer import org.opensearch.rest.RestChannel From f738ecc33bee7946cb1a4983561deb4fe17d1584 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Mon, 22 May 2023 16:09:23 +0530 Subject: [PATCH 085/157] Add retention lease with followerClusterUUID (#833) * New retention lease Signed-off-by: Monu Singh * Add bwc test Signed-off-by: Monu Singh Signed-off-by: Monu Singh * Address comments Signed-off-by: Monu Singh * Refactor Signed-off-by: Monu Singh * fix bwc Signed-off-by: Monu Singh Signed-off-by: Monu Singh * Run bwc test when full cluster restart is done Signed-off-by: Monu Singh * Address comments Signed-off-by: Monu Singh * Address comments Signed-off-by: Monu Singh * Address comments Signed-off-by: Monu Singh * check retention lease with REATIL_ALL Signed-off-by: Monu Singh * Fix bwc to not check for specific shard Signed-off-by: Monu Singh --------- Signed-off-by: Monu Singh --- .../TransportResumeIndexReplicationAction.kt | 14 ++-- .../TransportStopIndexReplicationAction.kt | 2 +- .../repository/RemoteClusterRepository.kt | 4 +- .../RemoteClusterRetentionLeaseHelper.kt | 82 ++++++++++++++++--- .../task/index/IndexReplicationTask.kt | 2 +- .../task/shard/ShardReplicationTask.kt | 2 +- .../bwc/BackwardsCompatibilityIT.kt | 32 ++++++++ 7 files changed, 117 insertions(+), 21 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/action/resume/TransportResumeIndexReplicationAction.kt b/src/main/kotlin/org/opensearch/replication/action/resume/TransportResumeIndexReplicationAction.kt index 39472736..c3c36419 100644 --- a/src/main/kotlin/org/opensearch/replication/action/resume/TransportResumeIndexReplicationAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/resume/TransportResumeIndexReplicationAction.kt @@ -49,16 +49,16 @@ import org.opensearch.cluster.metadata.IndexMetadata import org.opensearch.cluster.metadata.IndexNameExpressionResolver import org.opensearch.cluster.service.ClusterService import org.opensearch.common.inject.Inject -import org.opensearch.replication.ReplicationPlugin.Companion.KNN_INDEX_SETTING -import org.opensearch.replication.ReplicationPlugin.Companion.KNN_PLUGIN_PRESENT_SETTING + import org.opensearch.common.io.stream.StreamInput import org.opensearch.env.Environment import org.opensearch.index.IndexNotFoundException import org.opensearch.index.shard.ShardId +import org.opensearch.replication.util.indicesService import org.opensearch.threadpool.ThreadPool import org.opensearch.transport.TransportService import java.io.IOException -import java.lang.IllegalStateException + class TransportResumeIndexReplicationAction @Inject constructor(transportService: TransportService, clusterService: ClusterService, @@ -132,10 +132,14 @@ class TransportResumeIndexReplicationAction @Inject constructor(transportService var isResumable = true val remoteClient = client.getRemoteClusterClient(params.leaderAlias) val shards = clusterService.state().routingTable.indicesRouting().get(params.followerIndexName)?.shards() - val retentionLeaseHelper = RemoteClusterRetentionLeaseHelper(clusterService.clusterName.value(), remoteClient) + val retentionLeaseHelper = RemoteClusterRetentionLeaseHelper(clusterService.clusterName.value(), clusterService.state().metadata.clusterUUID(), remoteClient) shards?.forEach { val followerShardId = it.value.shardId - if (!retentionLeaseHelper.verifyRetentionLeaseExist(ShardId(params.leaderIndex, followerShardId.id), followerShardId)) { + + val followerIndexService = indicesService.indexServiceSafe(followerShardId.index) + val indexShard = followerIndexService.getShard(followerShardId.id) + + if (!retentionLeaseHelper.verifyRetentionLeaseExist(ShardId(params.leaderIndex, followerShardId.id), followerShardId, indexShard.lastSyncedGlobalCheckpoint+1)) { isResumable = false } } diff --git a/src/main/kotlin/org/opensearch/replication/action/stop/TransportStopIndexReplicationAction.kt b/src/main/kotlin/org/opensearch/replication/action/stop/TransportStopIndexReplicationAction.kt index fd82bba4..8f7ff425 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stop/TransportStopIndexReplicationAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stop/TransportStopIndexReplicationAction.kt @@ -115,7 +115,7 @@ class TransportStopIndexReplicationAction @Inject constructor(transportService: try { val replMetadata = replicationMetadataManager.getIndexReplicationMetadata(request.indexName) val remoteClient = client.getRemoteClusterClient(replMetadata.connectionName) - val retentionLeaseHelper = RemoteClusterRetentionLeaseHelper(clusterService.clusterName.value(), remoteClient) + val retentionLeaseHelper = RemoteClusterRetentionLeaseHelper(clusterService.clusterName.value(), clusterService.state().metadata.clusterUUID(), remoteClient) retentionLeaseHelper.attemptRemoveRetentionLease(clusterService, replMetadata, request.indexName) } catch(e: Exception) { log.error("Failed to remove retention lease from the leader cluster", e) diff --git a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt index 1036317c..4146caad 100644 --- a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt +++ b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt @@ -57,6 +57,7 @@ import org.opensearch.index.store.Store import org.opensearch.indices.recovery.RecoverySettings import org.opensearch.indices.recovery.RecoveryState import org.opensearch.replication.ReplicationPlugin.Companion.REPLICATION_INDEX_TRANSLOG_PRUNING_ENABLED_SETTING +import org.opensearch.replication.seqno.RemoteClusterRetentionLeaseHelper import org.opensearch.replication.util.stackTraceToString import org.opensearch.repositories.IndexId import org.opensearch.repositories.Repository @@ -280,7 +281,8 @@ class RemoteClusterRepository(private val repositoryMetadata: RepositoryMetadata snapshotShardId.id) restoreUUID = UUIDs.randomBase64UUID() val getStoreMetadataRequest = GetStoreMetadataRequest(restoreUUID, leaderShardNode, leaderShardId, - clusterService.clusterName.value(), followerShardId) + RemoteClusterRetentionLeaseHelper.getFollowerClusterNameWithUUID(clusterService.clusterName.value(), clusterService.state().metadata.clusterUUID()), + followerShardId) // Gets the remote store metadata val metadataResponse = executeActionOnRemote(GetStoreMetadataAction.INSTANCE, getStoreMetadataRequest, followerIndexName) diff --git a/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterRetentionLeaseHelper.kt b/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterRetentionLeaseHelper.kt index d8daf425..638d9359 100644 --- a/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterRetentionLeaseHelper.kt +++ b/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterRetentionLeaseHelper.kt @@ -23,29 +23,42 @@ import org.opensearch.index.seqno.RetentionLeaseAlreadyExistsException import org.opensearch.index.seqno.RetentionLeaseInvalidRetainingSeqNoException import org.opensearch.index.seqno.RetentionLeaseNotFoundException import org.opensearch.index.shard.ShardId -import org.opensearch.replication.action.stop.TransportStopIndexReplicationAction import org.opensearch.replication.metadata.store.ReplicationMetadata +import org.opensearch.replication.repository.RemoteClusterRepository import org.opensearch.replication.task.index.IndexReplicationParams import org.opensearch.replication.util.stackTraceToString import org.opensearch.replication.util.suspending -class RemoteClusterRetentionLeaseHelper constructor(val followerClusterName: String, val client: Client) { +class RemoteClusterRetentionLeaseHelper constructor(var followerClusterNameWithUUID: String, val client: Client) { - private val retentionLeaseSource = retentionLeaseSource(followerClusterName) + private val retentionLeaseSource = retentionLeaseSource(followerClusterNameWithUUID) + private var followerClusterUUID : String = "" + private var followerClusterName : String = "" + + constructor(followerClusterName: String, followerClusterUUID: String, client: Client) :this(followerClusterName, client){ + this.followerClusterUUID = followerClusterUUID + this.followerClusterName = followerClusterName + this.followerClusterNameWithUUID = getFollowerClusterNameWithUUID(followerClusterName, followerClusterUUID) + } companion object { private val log = LogManager.getLogger(RemoteClusterRetentionLeaseHelper::class.java) const val RETENTION_LEASE_PREFIX = "replication:" - fun retentionLeaseSource(followerClusterName: String): String = "${RETENTION_LEASE_PREFIX}${followerClusterName}" + fun retentionLeaseSource(followerClusterName: String): String + = "${RETENTION_LEASE_PREFIX}${followerClusterName}" fun retentionLeaseIdForShard(followerClusterName: String, followerShardId: ShardId): String { val retentionLeaseSource = retentionLeaseSource(followerClusterName) return "$retentionLeaseSource:${followerShardId}" } + + fun getFollowerClusterNameWithUUID(followerClusterName: String, followerClusterUUID: String): String{ + return "$followerClusterName:$followerClusterUUID" + } } - public suspend fun verifyRetentionLeaseExist(leaderShardId: ShardId, followerShardId: ShardId): Boolean { - val retentionLeaseId = retentionLeaseIdForShard(followerClusterName, followerShardId) + public suspend fun verifyRetentionLeaseExist(leaderShardId: ShardId, followerShardId: ShardId, seqNo: Long): Boolean { + val retentionLeaseId = retentionLeaseIdForShard(followerClusterNameWithUUID, followerShardId) // Currently there is no API to describe/list the retention leases . // So we are verifying the existence of lease by trying to renew a lease by same name . // If retention lease doesn't exist, this will throw an RetentionLeaseNotFoundException exception @@ -60,15 +73,60 @@ class RemoteClusterRetentionLeaseHelper constructor(val followerClusterName: Str return true } catch (e: RetentionLeaseNotFoundException) { + return addNewRetentionLeaseIfOldExists(leaderShardId, followerShardId, seqNo) + }catch (e : Exception) { return false } return true } + private suspend fun addNewRetentionLeaseIfOldExists(leaderShardId: ShardId, followerShardId: ShardId, seqNo: Long): Boolean { + //Check for old retention lease id + val oldRetentionLeaseId = retentionLeaseIdForShard(followerClusterName, followerShardId) + val requestForOldId = RetentionLeaseActions.RenewRequest(leaderShardId, oldRetentionLeaseId, RetentionLeaseActions.RETAIN_ALL, retentionLeaseSource) + try { + client.suspendExecute(RetentionLeaseActions.Renew.INSTANCE, requestForOldId) + } catch (ex: RetentionLeaseInvalidRetainingSeqNoException) { + //old retention lease id present, will add new retention lease + log.info("Old retention lease Id ${oldRetentionLeaseId} present with invalid seq number, adding new retention lease with ID:" + + "${retentionLeaseIdForShard(followerClusterNameWithUUID, followerShardId)} ") + return addNewRetentionLease(leaderShardId, seqNo, followerShardId, RemoteClusterRepository.REMOTE_CLUSTER_REPO_REQ_TIMEOUT_IN_MILLI_SEC ) + }catch (ex: Exception){ + log.info("Encountered Exception while checking for old retention lease: ${ex.stackTraceToString()}") + return false + } + log.info("Old retention lease Id ${oldRetentionLeaseId}, adding new retention lease with ID:" + + "${retentionLeaseIdForShard(followerClusterNameWithUUID, followerShardId)} ") + return addNewRetentionLease(leaderShardId,seqNo, followerShardId, RemoteClusterRepository.REMOTE_CLUSTER_REPO_REQ_TIMEOUT_IN_MILLI_SEC ) + } + + + private suspend fun addNewRetentionLease(leaderShardId: ShardId, seqNo: Long, followerShardId: ShardId, timeout: Long): Boolean { + val retentionLeaseId = retentionLeaseIdForShard(followerClusterNameWithUUID, followerShardId) + val request = RetentionLeaseActions.AddRequest(leaderShardId, retentionLeaseId, seqNo, retentionLeaseSource) + try { + client.suspendExecute(RetentionLeaseActions.Add.INSTANCE, request) + return true + } catch (e: Exception) { + log.info("Exception while adding new retention lease with i: $retentionLeaseId") + return false + } + } + public suspend fun renewRetentionLease(leaderShardId: ShardId, seqNo: Long, followerShardId: ShardId) { - val retentionLeaseId = retentionLeaseIdForShard(followerClusterName, followerShardId) + val retentionLeaseId = retentionLeaseIdForShard(followerClusterNameWithUUID, followerShardId) val request = RetentionLeaseActions.RenewRequest(leaderShardId, retentionLeaseId, seqNo, retentionLeaseSource) - client.suspendExecute(RetentionLeaseActions.Renew.INSTANCE, request) + try { + client.suspendExecute(RetentionLeaseActions.Renew.INSTANCE, request) + }catch (e: RetentionLeaseNotFoundException){ + //New retention lease not found, checking presense of old retention lease + log.info("Retention lease with ID: ${retentionLeaseId} not found," + + " checking for old retention lease with ID: ${retentionLeaseIdForShard(followerClusterName, followerShardId)}") + if(!addNewRetentionLeaseIfOldExists(leaderShardId, followerShardId, seqNo)){ + log.info("Both new $retentionLeaseId and old ${retentionLeaseIdForShard(followerClusterNameWithUUID, followerShardId)} retention lease not found.") + throw e + } + } } public suspend fun attemptRemoveRetentionLease(clusterService: ClusterService, replMetadata: ReplicationMetadata, @@ -78,7 +136,7 @@ class RemoteClusterRetentionLeaseHelper constructor(val followerClusterName: Str val params = IndexReplicationParams(replMetadata.connectionName, remoteMetadata.index, followerIndexName) val remoteClient = client.getRemoteClusterClient(params.leaderAlias) val shards = clusterService.state().routingTable.indicesRouting().get(params.followerIndexName)?.shards() - val retentionLeaseHelper = RemoteClusterRetentionLeaseHelper(clusterService.clusterName.value(), remoteClient) + val retentionLeaseHelper = RemoteClusterRetentionLeaseHelper( clusterService.clusterName.value(), followerClusterUUID, remoteClient) shards?.forEach { val followerShardId = it.value.shardId log.debug("Removing lease for $followerShardId.id ") @@ -102,7 +160,7 @@ class RemoteClusterRetentionLeaseHelper constructor(val followerClusterName: Str public suspend fun attemptRetentionLeaseRemoval(leaderShardId: ShardId, followerShardId: ShardId) { - val retentionLeaseId = retentionLeaseIdForShard(followerClusterName, followerShardId) + val retentionLeaseId = retentionLeaseIdForShard(followerClusterNameWithUUID, followerShardId) val request = RetentionLeaseActions.RemoveRequest(leaderShardId, retentionLeaseId) try { client.suspendExecute(RetentionLeaseActions.Remove.INSTANCE, request) @@ -123,7 +181,7 @@ class RemoteClusterRetentionLeaseHelper constructor(val followerClusterName: Str */ public fun addRetentionLease(leaderShardId: ShardId, seqNo: Long, followerShardId: ShardId, timeout: Long) { - val retentionLeaseId = retentionLeaseIdForShard(followerClusterName, followerShardId) + val retentionLeaseId = retentionLeaseIdForShard(followerClusterNameWithUUID, followerShardId) val request = RetentionLeaseActions.AddRequest(leaderShardId, retentionLeaseId, seqNo, retentionLeaseSource) try { client.execute(RetentionLeaseActions.Add.INSTANCE, request).actionGet(timeout) @@ -138,7 +196,7 @@ class RemoteClusterRetentionLeaseHelper constructor(val followerClusterName: Str public fun renewRetentionLease(leaderShardId: ShardId, seqNo: Long, followerShardId: ShardId, timeout: Long) { - val retentionLeaseId = retentionLeaseIdForShard(followerClusterName, followerShardId) + val retentionLeaseId = retentionLeaseIdForShard(followerClusterNameWithUUID, followerShardId) val request = RetentionLeaseActions.RenewRequest(leaderShardId, retentionLeaseId, seqNo, retentionLeaseSource) client.execute(RetentionLeaseActions.Renew.INSTANCE, request).actionGet(timeout) } diff --git a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt index 29876e85..5c9ac2d9 100644 --- a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt @@ -132,7 +132,7 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript override val followerIndexName = params.followerIndexName override val log = Loggers.getLogger(javaClass, Index(params.followerIndexName, ClusterState.UNKNOWN_UUID)) - private val retentionLeaseHelper = RemoteClusterRetentionLeaseHelper(clusterService.clusterName.value(), remoteClient) + private val retentionLeaseHelper = RemoteClusterRetentionLeaseHelper(clusterService.clusterName.value(), clusterService.state().metadata.clusterUUID(), remoteClient) private var shouldCallEvalMonitoring = true private var updateSettingsContinuousFailCount = 0 private var updateAliasContinousFailCount = 0 diff --git a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationTask.kt b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationTask.kt index 44493bc7..7b58d419 100644 --- a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationTask.kt @@ -69,7 +69,7 @@ class ShardReplicationTask(id: Long, type: String, action: String, description: private val leaderShardId = params.leaderShardId private val followerShardId = params.followerShardId private val remoteClient = client.getRemoteClusterClient(leaderAlias) - private val retentionLeaseHelper = RemoteClusterRetentionLeaseHelper(clusterService.clusterName.value(), remoteClient) + private val retentionLeaseHelper = RemoteClusterRetentionLeaseHelper(clusterService.clusterName.value(), clusterService.state().metadata.clusterUUID(), remoteClient) private var lastLeaseRenewalMillis = System.currentTimeMillis() //Start backOff for exceptions with a second diff --git a/src/test/kotlin/org/opensearch/replication/bwc/BackwardsCompatibilityIT.kt b/src/test/kotlin/org/opensearch/replication/bwc/BackwardsCompatibilityIT.kt index 41b8d960..e856eedd 100644 --- a/src/test/kotlin/org/opensearch/replication/bwc/BackwardsCompatibilityIT.kt +++ b/src/test/kotlin/org/opensearch/replication/bwc/BackwardsCompatibilityIT.kt @@ -1,5 +1,6 @@ package org.opensearch.replication.bwc; +import org.apache.hc.core5.http.io.entity.EntityUtils import org.assertj.core.api.Assertions import org.junit.Assert import org.junit.BeforeClass @@ -8,13 +9,16 @@ import org.opensearch.action.admin.cluster.health.ClusterHealthRequest import org.opensearch.action.delete.DeleteRequest import org.opensearch.action.get.GetRequest import org.opensearch.action.index.IndexRequest +import org.opensearch.client.Request import org.opensearch.client.RequestOptions +import org.opensearch.client.RestHighLevelClient import org.opensearch.client.indices.CreateIndexRequest import org.opensearch.replication.MultiClusterAnnotations import org.opensearch.replication.MultiClusterRestTestCase import org.opensearch.replication.StartReplicationRequest import org.opensearch.replication.startReplication import org.opensearch.test.OpenSearchTestCase.assertBusy +import org.opensearch.test.rest.OpenSearchRestTestCase import java.util.Collections import java.util.concurrent.TimeUnit import java.util.stream.Collectors @@ -139,12 +143,40 @@ class BackwardsCompatibilityIT : MultiClusterRestTestCase() { Assertions.assertThat(getResponse.isExists).isTrue() Assertions.assertThat(getResponse.sourceAsMap).isEqualTo(source) }, 60, TimeUnit.SECONDS) + + //Check for latest retention lease when full cluster restart is done + if (ClusterStatus.from(System.getProperty("tests.bwcTask")) == ClusterStatus.FULL_CLUSTER_RESTART || ClusterStatus.from( + System.getProperty("tests.bwcTask")) == ClusterStatus.ROLLING_UPGRADED) { + validateNewRetentionLeaseId(follower, leader) + } + } catch (e: Exception) { logger.info("Exception while verifying the replication ${e.printStackTrace()}") throw e } } + private fun validateNewRetentionLeaseId( + follower: RestHighLevelClient, + leader: RestHighLevelClient + ) { + assertBusy({ + val followerClusterInfo: Map = + OpenSearchRestTestCase.entityAsMap(follower.lowLevelClient.performRequest(Request("GET", "/"))) + val clusterUUID = (followerClusterInfo["cluster_uuid"] as String) + val clusterName = (followerClusterInfo["cluster_name"] as String) + assert(clusterUUID.isNotEmpty()) + assert(clusterName.isNotEmpty()) + val expectedRetentionLeaseId = + "replication" + ":" + clusterName + ":" + clusterUUID + ":[" + LEADER_INDEX + "]" + + val retentionLeaseinfo = + leader.lowLevelClient.performRequest(Request("GET", "/$LEADER_INDEX/_stats/docs?level=shards")) + val retentionLeaseInfoString = EntityUtils.toString(retentionLeaseinfo.entity) + assertTrue(retentionLeaseInfoString.contains(expectedRetentionLeaseId)) + }, 60, TimeUnit.SECONDS) + } + // Verifies that replication plugin is installed on all the nodes og the cluster. @Throws(java.lang.Exception::class) private fun verifyReplicationPluginInstalled(clusterName: String) { From e3256ebf6d556c8f6c7612a30d8737cb443b564c Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Tue, 23 May 2023 11:17:43 +0530 Subject: [PATCH 086/157] Update snakeyaml to version 2.0 Signed-off-by: Monu Singh --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index bce10727..cf3c6bbf 100644 --- a/build.gradle +++ b/build.gradle @@ -121,7 +121,7 @@ configurations.all { force 'org.apache.httpcomponents.client5:httpclient5:5.0.3' force 'org.apache.httpcomponents.client5:httpclient5-osgi:5.0.3' force 'com.fasterxml.jackson.core:jackson-databind:2.13.4.2' - force 'org.yaml:snakeyaml:1.32' + force 'org.yaml:snakeyaml:2.0' force 'org.codehaus.plexus:plexus-utils:3.0.24' force 'net.java.dev.jna:jna:5.12.1' } From 3e3787dddaa99a4085153024f064566730881fb9 Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Tue, 23 May 2023 16:02:34 +0530 Subject: [PATCH 087/157] Handle serialization issues with UpdateReplicationStateDetailsRequest (#866) Signed-off-by: Ankit Kala --- .../UpdateReplicationStateDetailsRequest.kt | 3 +- ...dateReplicationStateDetailsRequestTests.kt | 42 +++++++++++++++++++ 2 files changed, 44 insertions(+), 1 deletion(-) create mode 100644 src/test/kotlin/org/opensearch/replication/action/replicationstatedetails/UpdateReplicationStateDetailsRequestTests.kt diff --git a/src/main/kotlin/org/opensearch/replication/action/replicationstatedetails/UpdateReplicationStateDetailsRequest.kt b/src/main/kotlin/org/opensearch/replication/action/replicationstatedetails/UpdateReplicationStateDetailsRequest.kt index bddcd528..04fedd0a 100644 --- a/src/main/kotlin/org/opensearch/replication/action/replicationstatedetails/UpdateReplicationStateDetailsRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/replicationstatedetails/UpdateReplicationStateDetailsRequest.kt @@ -48,6 +48,7 @@ class UpdateReplicationStateDetailsRequest: AcknowledgedRequest Date: Fri, 26 May 2023 15:26:48 +0530 Subject: [PATCH 088/157] Modify autofollow retry scheduler logic check to account for completed runs (#839) Signed-off-by: Sai Kumar --- .../task/autofollow/AutoFollowTask.kt | 3 +- .../integ/rest/UpdateAutoFollowPatternIT.kt | 59 +++++++++++-------- 2 files changed, 38 insertions(+), 24 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowTask.kt b/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowTask.kt index da89580e..15c22922 100644 --- a/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowTask.kt @@ -45,6 +45,7 @@ import org.opensearch.tasks.TaskId import org.opensearch.threadpool.Scheduler import org.opensearch.threadpool.ThreadPool import java.util.concurrent.ConcurrentSkipListSet +import java.util.concurrent.TimeUnit class AutoFollowTask(id: Long, type: String, action: String, description: String, parentTask: TaskId, headers: Map, @@ -91,7 +92,7 @@ class AutoFollowTask(id: Long, type: String, action: String, description: String private fun addRetryScheduler() { log.debug("Adding retry scheduler") - if(retryScheduler != null && !retryScheduler!!.isCancelled) { + if(retryScheduler != null && retryScheduler!!.getDelay(TimeUnit.NANOSECONDS) > 0L) { return } try { diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt index c8d3620e..57593241 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt @@ -23,6 +23,7 @@ import org.opensearch.replication.task.index.IndexReplicationExecutor import org.apache.hc.core5.http.HttpStatus import org.apache.hc.core5.http.ContentType import org.apache.hc.core5.http.io.entity.StringEntity +import org.apache.logging.log4j.LogManager import org.assertj.core.api.Assertions import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest import org.opensearch.action.admin.indices.settings.get.GetSettingsRequest @@ -41,6 +42,7 @@ import org.opensearch.cluster.metadata.IndexMetadata import org.opensearch.cluster.metadata.MetadataCreateIndexService import org.opensearch.replication.AutoFollowStats import org.opensearch.replication.ReplicationPlugin +import org.opensearch.replication.action.changes.TransportGetChangesAction import org.opensearch.replication.updateReplicationStartBlockSetting import org.opensearch.replication.updateAutofollowRetrySetting import org.opensearch.replication.updateAutoFollowConcurrentStartReplicationJobSetting @@ -62,6 +64,10 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { private val longIndexPatternName = "index_".repeat(43) private val waitForShardTask = TimeValue.timeValueSeconds(10) + companion object { + private val log = LogManager.getLogger(UpdateAutoFollowPatternIT::class.java) + } + fun `test auto follow pattern`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) @@ -315,36 +321,43 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { Assertions.assertThat(getIndexReplicationTasks(FOLLOWER).size).isEqualTo(1) } - fun `test autofollow task with start replication block`() { + fun `test autofollow task with start replication block and retries`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) createConnectionBetweenClusters(FOLLOWER, LEADER, connectionAlias) - val leaderIndexName = createRandomIndex(leaderClient) try { //modify retry duration to account for autofollow trigger in next retry followerClient.updateAutofollowRetrySetting("1m") - // Add replication start block - followerClient.updateReplicationStartBlockSetting(true) - followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, indexPattern) - sleep(30000) // Default poll for auto follow in worst case - // verify both index replication tasks and autofollow tasks - // Replication shouldn't have been started - 0 tasks - // Autofollow task should still be up - 1 task - Assertions.assertThat(getIndexReplicationTasks(FOLLOWER).size).isEqualTo(0) - Assertions.assertThat(getAutoFollowTasks(FOLLOWER).size).isEqualTo(1) + for (repeat in 1..2) { + log.info("Current Iteration $repeat") + // Add replication start block + followerClient.updateReplicationStartBlockSetting(true) + createRandomIndex(leaderClient) + followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, indexPattern) + sleep(95000) // wait for auto follow trigger in the worst case + // verify both index replication tasks and autofollow tasks + // Replication shouldn't have been started - (repeat-1) tasks as for current loop index shouldn't be + // created yet. + // Autofollow task should still be up - 1 task + Assertions.assertThat(getIndexReplicationTasks(FOLLOWER).size).isEqualTo(repeat-1) + Assertions.assertThat(getAutoFollowTasks(FOLLOWER).size).isEqualTo(1) - var stats = followerClient.AutoFollowStats() - var failedIndices = stats["failed_indices"] as List<*> - assert(failedIndices.size == 1) - // Remove replication start block - followerClient.updateReplicationStartBlockSetting(false) - sleep(60000) // wait for auto follow trigger in the worst case - // Index should be replicated and autofollow task should be present - Assertions.assertThat(getIndexReplicationTasks(FOLLOWER).size).isEqualTo(1) - Assertions.assertThat(getAutoFollowTasks(FOLLOWER).size).isEqualTo(1) - stats = followerClient.AutoFollowStats() - failedIndices = stats["failed_indices"] as List<*> - assert(failedIndices.isEmpty()) + var stats = followerClient.AutoFollowStats() + var failedIndices = stats["failed_indices"] as List<*> + // Every time failed replication task will be 1 as + // there are already running jobs in the previous iteration + log.info("Current failed indices $failedIndices") + assert(failedIndices.size == 1) + // Remove replication start block + followerClient.updateReplicationStartBlockSetting(false) + sleep(95000) // wait for auto follow trigger in the worst case + // Index should be replicated and autofollow task should be present + Assertions.assertThat(getIndexReplicationTasks(FOLLOWER).size).isEqualTo(repeat) + Assertions.assertThat(getAutoFollowTasks(FOLLOWER).size).isEqualTo(1) + stats = followerClient.AutoFollowStats() + failedIndices = stats["failed_indices"] as List<*> + assert(failedIndices.isEmpty()) + } } finally { followerClient.deleteAutoFollowPattern(connectionAlias, indexPatternName) } From 3e55af48249a6570e3a71376b59f97f92cf4279e Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Mon, 29 May 2023 16:17:09 +0530 Subject: [PATCH 089/157] Add release notes for 2.8 release Signed-off-by: Monu Singh --- ...cross-cluster-replication.release-notes-2.8.0.0.md | 11 +++++++++++ 1 file changed, 11 insertions(+) create mode 100644 release-notes/opensearch-cross-cluster-replication.release-notes-2.8.0.0.md diff --git a/release-notes/opensearch-cross-cluster-replication.release-notes-2.8.0.0.md b/release-notes/opensearch-cross-cluster-replication.release-notes-2.8.0.0.md new file mode 100644 index 00000000..47719aef --- /dev/null +++ b/release-notes/opensearch-cross-cluster-replication.release-notes-2.8.0.0.md @@ -0,0 +1,11 @@ +## Version 2.8.0.0 Release Notes + +Compatible with OpenSearch 2.8.0 + + +### Enhancements +* Support CCR for k-NN enabled indices ([#760](https://github.com/opensearch-project/cross-cluster-replication/pull/760)) + +### Bug Fixes +* Handle serialization issues with UpdateReplicationStateDetailsRequest ([#866](https://github.com/opensearch-project/cross-cluster-replication/pull/866)) +* Two followers using same remote alias can result in replication being auto-paused ([#833](https://github.com/opensearch-project/cross-cluster-replication/pull/833)) From 14b9268249b53f3ec6f69cffa451dbe86c2c1a33 Mon Sep 17 00:00:00 2001 From: Nishant Goel <113011736+nisgoel-amazon@users.noreply.github.com> Date: Wed, 31 May 2023 14:37:38 +0530 Subject: [PATCH 090/157] Handling exception in getAssignment method (#881) * Handling exception in getAssignment method Handling exception in getAssignment method Signed-off-by: Nishant Goel Signed-off-by: Nishant Goel <113011736+nisgoel-amazon@users.noreply.github.com> * Adding UT for getAssignment Method Signed-off-by: Nishant Goel --------- Signed-off-by: Nishant Goel <113011736+nisgoel-amazon@users.noreply.github.com> Signed-off-by: Nishant Goel --- .../task/shard/ShardReplicationExecutor.kt | 11 +- .../shard/ShardReplicationExecutorTests.kt | 148 ++++++++++++++++++ 2 files changed, 156 insertions(+), 3 deletions(-) create mode 100644 src/test/kotlin/org/opensearch/replication/task/shard/ShardReplicationExecutorTests.kt diff --git a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationExecutor.kt b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationExecutor.kt index 9e509a79..11be6056 100644 --- a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationExecutor.kt +++ b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationExecutor.kt @@ -56,9 +56,14 @@ class ShardReplicationExecutor(executor: String, private val clusterService : Cl } override fun getAssignment(params: ShardReplicationParams, clusterState: ClusterState) : Assignment { - val primaryShard = clusterState.routingTable().shardRoutingTable(params.followerShardId).primaryShard() - if (!primaryShard.active()) return SHARD_NOT_ACTIVE - return Assignment(primaryShard.currentNodeId(), "node with primary shard") + try { + val primaryShard = clusterState.routingTable().shardRoutingTable(params.followerShardId).primaryShard() + if (!primaryShard.active()) return SHARD_NOT_ACTIVE + return Assignment(primaryShard.currentNodeId(), "node with primary shard") + } catch (e: Exception) { + log.error("Failed to assign shard replication task with id ${params.followerShardId}", e) + return SHARD_NOT_ACTIVE + } } override fun nodeOperation(task: AllocatedPersistentTask, params: ShardReplicationParams, state: PersistentTaskState?) { diff --git a/src/test/kotlin/org/opensearch/replication/task/shard/ShardReplicationExecutorTests.kt b/src/test/kotlin/org/opensearch/replication/task/shard/ShardReplicationExecutorTests.kt new file mode 100644 index 00000000..630234f8 --- /dev/null +++ b/src/test/kotlin/org/opensearch/replication/task/shard/ShardReplicationExecutorTests.kt @@ -0,0 +1,148 @@ +package org.opensearch.replication.task.shard + +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope +import org.junit.Assert +import org.junit.Before +import org.junit.Test +import org.mockito.Mockito +import org.opensearch.Version +import org.opensearch.cluster.ClusterState +import org.opensearch.cluster.metadata.IndexMetadata +import org.opensearch.cluster.metadata.Metadata +import org.opensearch.cluster.routing.* +import org.opensearch.common.unit.TimeValue +import org.opensearch.core.xcontent.NamedXContentRegistry +import org.opensearch.index.Index +import org.opensearch.index.shard.ShardId +import org.opensearch.replication.ReplicationSettings +import org.opensearch.replication.metadata.ReplicationMetadataManager +import org.opensearch.replication.metadata.store.ReplicationMetadataStore +import org.opensearch.replication.task.index.* +import org.opensearch.test.ClusterServiceUtils +import org.opensearch.test.OpenSearchTestCase +import org.opensearch.threadpool.TestThreadPool +import java.util.ArrayList +import java.util.concurrent.TimeUnit + +@ThreadLeakScope(ThreadLeakScope.Scope.NONE) +class ShardReplicationExecutorTests: OpenSearchTestCase() { + + companion object { + var followerIndex = "follower-index" + var remoteCluster = "remote-cluster" + } + + private lateinit var shardReplicationExecutor: ShardReplicationExecutor + + private var threadPool = TestThreadPool("ShardExecutorTest") + private var clusterService = ClusterServiceUtils.createClusterService(threadPool) + + @Before + fun setup() { + val spyClient = Mockito.spy(NoOpClient("testName")) + val replicationMetadataManager = ReplicationMetadataManager(clusterService, spyClient, + ReplicationMetadataStore(spyClient, clusterService, NamedXContentRegistry.EMPTY) + ) + val followerStats = FollowerClusterStats() + val followerShardId = ShardId("follower", "follower_uuid", 0) + followerStats.stats[followerShardId] = FollowerShardMetric() + + val replicationSettings = Mockito.mock(ReplicationSettings::class.java) + replicationSettings.metadataSyncInterval = TimeValue(100, TimeUnit.MILLISECONDS) + shardReplicationExecutor = ShardReplicationExecutor( + "test_executor", + clusterService, + threadPool, + spyClient, + replicationMetadataManager, + replicationSettings, + followerStats + ) + } + + @Test + fun `getAssignment should not throw exception when no shard is present` () { + val sId = ShardId(Index(followerIndex, "_na_"), 0) + val params = ShardReplicationParams(remoteCluster, sId, sId) + val clusterState = createClusterState(null, null) + + try { + val assignment = shardReplicationExecutor.getAssignment(params, clusterState) + Assert.assertEquals(null, assignment.executorNode) + } catch (e: Exception) { + // Validation should not throw an exception, so the test should fail if it reaches this line + Assert.fail("Expected Exception should not be thrown") + } + } + + @Test + fun `getAssignment should return null if shard is present but is not active` () { + val sId = ShardId(Index(followerIndex, "_na_"), 0) + val params = ShardReplicationParams(remoteCluster, sId, sId) + val unassignedShard = ShardRouting.newUnassigned( + sId, + true, + RecoverySource.EmptyStoreRecoverySource.INSTANCE, + UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, null) + ) + val clusterState = createClusterState(sId, unassignedShard) + + try { + val assignment = shardReplicationExecutor.getAssignment(params, clusterState) + Assert.assertEquals(null, assignment.executorNode) + } catch (e: Exception) { + // Validation should not throw an exception, so the test should fail if it reaches this line + Assert.fail("Expected Exception should not be thrown") + } + } + + @Test + fun `getAssignment should return node when shard is present` () { + val sId = ShardId(Index(followerIndex, "_na_"), 0) + val params = ShardReplicationParams(remoteCluster, sId, sId) + val initializingShard = TestShardRouting.newShardRouting( + followerIndex, + sId.id, + "1", + true, + ShardRoutingState.INITIALIZING + ) + val startedShard = initializingShard.moveToStarted() + val clusterState = createClusterState(sId, startedShard) + + try { + val assignment = shardReplicationExecutor.getAssignment(params, clusterState) + Assert.assertEquals(initializingShard.currentNodeId(), assignment.executorNode) + } catch (e: Exception) { + // Validation should not throw an exception, so the test should fail if it reaches this line + Assert.fail("Expected Exception should not be thrown") + } + } + + private fun createClusterState(shardId: ShardId?, shardRouting: ShardRouting?): ClusterState { + val indices: MutableList = ArrayList() + indices.add(followerIndex) + val metadata = Metadata.builder() + .put( + IndexMetadata.builder(ReplicationMetadataStore.REPLICATION_CONFIG_SYSTEM_INDEX).settings(settings( + Version.CURRENT)).numberOfShards(1).numberOfReplicas(0)) + .put( + IndexMetadata.builder(IndexReplicationTaskTests.followerIndex).settings(settings( + Version.CURRENT)).numberOfShards(2).numberOfReplicas(0)) + .build() + + val routingTableBuilder = RoutingTable.builder() + .addAsNew(metadata.index(ReplicationMetadataStore.REPLICATION_CONFIG_SYSTEM_INDEX)) + .addAsNew(metadata.index(followerIndex)) + + if (shardId != null) { + routingTableBuilder.add( + IndexRoutingTable.builder(shardId.index) + .addShard(shardRouting) + .build() + ) + } + + return ClusterState.builder(clusterService.state()).routingTable(routingTableBuilder.build()).build() + } +} \ No newline at end of file From 83478445e45774159364e215f8205255ae6f40ad Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Wed, 31 May 2023 15:23:25 +0530 Subject: [PATCH 091/157] Add monusingh-1 as a maintainer (#929) Signed-off-by: Ankit Kala --- CONTRIBUTING.md | 7 ++++++- MAINTAINERS.md | 13 +++++++------ 2 files changed, 13 insertions(+), 7 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 0e594dfd..e9826fd1 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -5,6 +5,7 @@ - [Feature Requests](#feature-requests) - [Documentation Changes](#documentation-changes) - [Contributing Code](#contributing-code) +- [Becoming a maintainer](#becoming-a-maintainer) - [Developer Certificate of Origin](#developer-certificate-of-origin) - [License Headers](#license-headers) - [Backports](#backports) @@ -40,12 +41,16 @@ If you've thought of a way that OpenSearch could be better, we want to hear abou ### Documentation Changes -TODO +If you would like to contribute to the documentation, please do so in the [documentation-website](https://github.com/opensearch-project/documentation-website) repo. ### Contributing Code As with other types of contributions, the first step is to [open an issue on GitHub](issues/new/choose). Opening an issue before you make changes makes sure that someone else isn't already working on that particular problem. It also lets us all work together to find the right approach before you spend a bunch of time on a PR. So again, when in doubt, open an issue. +## Becoming a maintainer +You can become a maintainer by actively contributing to any project, and being nominated by an existing maintainer. +Refer to [this](https://github.com/opensearch-project/.github/blob/main/RESPONSIBILITIES.md#becoming-a-maintainer) for the nomination process. + ## Developer Certificate of Origin OpenSearch is an open source product released under the Apache 2.0 license (see either [the Apache site](https://www.apache.org/licenses/LICENSE-2.0) or the [LICENSE.txt file](LICENSE.txt)). The Apache 2.0 license allows you to freely use, modify, distribute, and sell your own products that include Apache 2.0 licensed software. diff --git a/MAINTAINERS.md b/MAINTAINERS.md index 0e708f28..0be97c82 100644 --- a/MAINTAINERS.md +++ b/MAINTAINERS.md @@ -4,10 +4,11 @@ This document contains a list of maintainers in this repo. See [opensearch-proje ## Current Maintainers -| Maintainer | GitHub ID | Affiliation | -| ---------------- | --------------------------------------------------------- | ----------- | -| Gopala Krishna A | [krishna-ggk](https://github.com/krishna-ggk) | Amazon | -| Ankit | [ankitkala](https://github.com/ankitkala) | Amazon | +| Maintainer | GitHub ID | Affiliation | +|------------------| -------------------------------------------------- | ----------- | +| Gopala Krishna A | [krishna-ggk](https://github.com/krishna-ggk) | Amazon | +| Ankit | [ankitkala](https://github.com/ankitkala) | Amazon | | Sai | [saikaranam-amazon](https://github.com/saikaranam-amazon) | Amazon | -| Sooraj Sinha | [soosinha](https://github.com/soosinha) | Amazon | -| Gaurav Bafna | [gbbafna](https://github.com/gbbafna) | Amazon | +| Sooraj Sinha | [soosinha](https://github.com/soosinha) | Amazon | +| Gaurav Bafna | [gbbafna](https://github.com/gbbafna) | Amazon | +| Monu Singh | [gbbafna](https://github.com/monusingh-1) | Amazon | From 59b55437d97242fc84ac189ef250320a3265e20c Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Wed, 31 May 2023 16:18:40 +0530 Subject: [PATCH 092/157] Update MAINTAINERS.md (#931) Correct username Signed-off-by: Monu Singh --- MAINTAINERS.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/MAINTAINERS.md b/MAINTAINERS.md index 0be97c82..5c1d73ef 100644 --- a/MAINTAINERS.md +++ b/MAINTAINERS.md @@ -11,4 +11,4 @@ This document contains a list of maintainers in this repo. See [opensearch-proje | Sai | [saikaranam-amazon](https://github.com/saikaranam-amazon) | Amazon | | Sooraj Sinha | [soosinha](https://github.com/soosinha) | Amazon | | Gaurav Bafna | [gbbafna](https://github.com/gbbafna) | Amazon | -| Monu Singh | [gbbafna](https://github.com/monusingh-1) | Amazon | +| Monu Singh | [monusingh-1](https://github.com/monusingh-1) | Amazon | From 596a07e75c8c636f642a142ad031f3a815d123f8 Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Thu, 1 Jun 2023 19:25:05 +0530 Subject: [PATCH 093/157] Update CODEOWNERS (#935) Signed-off-by: Ankit Kala --- .github/CODEOWNERS | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index af340718..3fcb6af0 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -3,4 +3,5 @@ * @ankitkala * @saikaranam-amazon * @soosinha -* @gbbafna \ No newline at end of file +* @gbbafna +* @monusingh-1 From ba7d1fa303299dfd874349ee9bc117fae5b2157c Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Fri, 2 Jun 2023 13:00:44 +0530 Subject: [PATCH 094/157] Initialize the leaderCheckpoint with follower shard's localCheckpoint (#904) Signed-off-by: Ankit Kala --- .../RemoteClusterRestoreLeaderService.kt | 4 +- .../RemoteClusterRetentionLeaseHelper.kt | 44 ++++++++++--- .../task/index/IndexReplicationTask.kt | 2 +- .../task/shard/ShardReplicationTask.kt | 4 ++ .../replication/MultiClusterRestTestCase.kt | 34 +++++++++- .../rest/ReplicationStopThenRestartIT.kt | 62 +++++++++++++++++++ 6 files changed, 137 insertions(+), 13 deletions(-) create mode 100644 src/test/kotlin/org/opensearch/replication/integ/rest/ReplicationStopThenRestartIT.kt diff --git a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRestoreLeaderService.kt b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRestoreLeaderService.kt index 5eea937b..5c06e4d4 100644 --- a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRestoreLeaderService.kt +++ b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRestoreLeaderService.kt @@ -114,8 +114,8 @@ class RemoteClusterRestoreLeaderService @Inject constructor(private val indicesS var fromSeqNo = RetentionLeaseActions.RETAIN_ALL // Adds the retention lease for fromSeqNo for the next stage of the replication. - retentionLeaseHelper.addRetentionLease(request.leaderShardId, fromSeqNo, - request.followerShardId, RemoteClusterRepository.REMOTE_CLUSTER_REPO_REQ_TIMEOUT_IN_MILLI_SEC) + retentionLeaseHelper.addRetentionLease(request.leaderShardId, fromSeqNo, request.followerShardId, + RemoteClusterRepository.REMOTE_CLUSTER_REPO_REQ_TIMEOUT_IN_MILLI_SEC) /** * At this point, it should be safe to release retention lock as the retention lease diff --git a/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterRetentionLeaseHelper.kt b/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterRetentionLeaseHelper.kt index 638d9359..80037455 100644 --- a/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterRetentionLeaseHelper.kt +++ b/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterRetentionLeaseHelper.kt @@ -22,6 +22,7 @@ import org.opensearch.index.seqno.RetentionLeaseActions import org.opensearch.index.seqno.RetentionLeaseAlreadyExistsException import org.opensearch.index.seqno.RetentionLeaseInvalidRetainingSeqNoException import org.opensearch.index.seqno.RetentionLeaseNotFoundException +import org.opensearch.index.shard.IndexShard import org.opensearch.index.shard.ShardId import org.opensearch.replication.metadata.store.ReplicationMetadata import org.opensearch.replication.repository.RemoteClusterRepository @@ -175,22 +176,47 @@ class RemoteClusterRetentionLeaseHelper constructor(var followerClusterNameWithU } } + public fun attemptRetentionLeaseRemoval(leaderShardId: ShardId, followerShardId: ShardId, timeout: Long) { + val retentionLeaseId = retentionLeaseIdForShard(followerClusterNameWithUUID, followerShardId) + val request = RetentionLeaseActions.RemoveRequest(leaderShardId, retentionLeaseId) + try { + client.execute(RetentionLeaseActions.Remove.INSTANCE, request).actionGet(timeout) + log.info("Removed retention lease with id - $retentionLeaseId") + } catch(e: RetentionLeaseNotFoundException) { + // log error and bail + log.error(e.stackTraceToString()) + } catch (e: Exception) { + // We are not bubbling up the exception as the stop action/ task cleanup should succeed + // even if we fail to remove the retention lease from leader cluster + log.error("Exception in removing retention lease", e) + } + } + /** * Remove these once the callers are moved to above APIs */ public fun addRetentionLease(leaderShardId: ShardId, seqNo: Long, - followerShardId: ShardId, timeout: Long) { + followerShardId: ShardId, timeout: Long) { val retentionLeaseId = retentionLeaseIdForShard(followerClusterNameWithUUID, followerShardId) val request = RetentionLeaseActions.AddRequest(leaderShardId, retentionLeaseId, seqNo, retentionLeaseSource) - try { - client.execute(RetentionLeaseActions.Add.INSTANCE, request).actionGet(timeout) - } catch (e: RetentionLeaseAlreadyExistsException) { - log.error(e.stackTraceToString()) - log.info("Renew retention lease as it already exists $retentionLeaseId with $seqNo") - // Only one retention lease should exists for the follower shard - // Ideally, this should have got cleaned-up - renewRetentionLease(leaderShardId, seqNo, followerShardId, timeout) + var canRetry = true + while (true) { + try { + log.info("Adding retention lease $retentionLeaseId") + client.execute(RetentionLeaseActions.Add.INSTANCE, request).actionGet(timeout) + break + } catch (e: RetentionLeaseAlreadyExistsException) { + log.info("Found a stale retention lease $retentionLeaseId on leader.") + if (canRetry) { + canRetry = false + attemptRetentionLeaseRemoval(leaderShardId, followerShardId, timeout) + log.info("Cleared stale retention lease $retentionLeaseId on leader. Retrying...") + } else { + log.error(e.stackTraceToString()) + throw e + } + } } } diff --git a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt index 5c9ac2d9..d51af7c0 100644 --- a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt @@ -290,7 +290,7 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript private suspend fun pollShardTaskStatus(): IndexReplicationState { val failedShardTasks = findAllReplicationFailedShardTasks(followerIndexName, clusterService.state()) if (failedShardTasks.isNotEmpty()) { - log.info("Failed shard tasks - ", failedShardTasks) + log.info("Failed shard tasks - $failedShardTasks") var msg = "" for ((shard, task) in failedShardTasks) { val taskState = task.state diff --git a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationTask.kt b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationTask.kt index 7b58d419..f08b2c6b 100644 --- a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationTask.kt @@ -218,6 +218,10 @@ class ShardReplicationTask(id: Long, type: String, action: String, description: val changeTracker = ShardReplicationChangesTracker(indexShard, replicationSettings) followerClusterStats.stats[followerShardId]!!.followerCheckpoint = indexShard.localCheckpoint + // In case the shard task starts on a new node and there are no active writes on the leader shard, leader checkpoint + // never gets initialized and defaults to 0. To get around this, we set the leaderCheckpoint to follower shard's + // localCheckpoint as the leader shard is guaranteed to equal or more. + followerClusterStats.stats[followerShardId]!!.leaderCheckpoint = indexShard.localCheckpoint coroutineScope { while (isActive) { rateLimiter.acquire() diff --git a/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt b/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt index 5969156c..d6981ad2 100644 --- a/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt +++ b/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt @@ -11,7 +11,6 @@ package org.opensearch.replication -import com.nhaarman.mockitokotlin2.stub import org.opensearch.replication.MultiClusterAnnotations.ClusterConfiguration import org.opensearch.replication.MultiClusterAnnotations.ClusterConfigurations import org.opensearch.replication.MultiClusterAnnotations.getAnnotationsFromClass @@ -516,6 +515,28 @@ abstract class MultiClusterRestTestCase : OpenSearchTestCase() { return OpenSearchRestTestCase.entityAsList(client.performRequest(Request("GET", endpoint))) } + protected fun deleteConnection(fromClusterName: String, connectionName: String="source") { + val fromCluster = getNamedCluster(fromClusterName) + val persistentConnectionRequest = Request("PUT", "_cluster/settings") + + val entityAsString = """ + { + "persistent": { + "cluster": { + "remote": { + "$connectionName": { + "seeds": null + } + } + } + } + }""".trimMargin() + + persistentConnectionRequest.entity = StringEntity(entityAsString, ContentType.APPLICATION_JSON) + val persistentConnectionResponse = fromCluster.lowLevelClient.performRequest(persistentConnectionRequest) + assertEquals(HttpStatus.SC_OK.toLong(), persistentConnectionResponse.statusLine.statusCode.toLong()) + } + protected fun createConnectionBetweenClusters(fromClusterName: String, toClusterName: String, connectionName: String="source") { val toCluster = getNamedCluster(toClusterName) val fromCluster = getNamedCluster(fromClusterName) @@ -646,5 +667,16 @@ abstract class MultiClusterRestTestCase : OpenSearchTestCase() { return integTestRemote.equals("true") } + protected fun docCount(cluster: RestHighLevelClient, indexName: String) : Int { + val persistentConnectionRequest = Request("GET", "/$indexName/_search?pretty&q=*") + + val persistentConnectionResponse = cluster.lowLevelClient.performRequest(persistentConnectionRequest) + val statusResponse: Map>> = OpenSearchRestTestCase.entityAsMap(persistentConnectionResponse) as Map>> + return statusResponse["hits"]?.get("total")?.get("value") as Int + } + + protected fun deleteIndex(testCluster: RestHighLevelClient, indexName: String) { + testCluster.lowLevelClient.performRequest(Request("DELETE", indexName)) + } } diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/ReplicationStopThenRestartIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/ReplicationStopThenRestartIT.kt new file mode 100644 index 00000000..33ebb790 --- /dev/null +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/ReplicationStopThenRestartIT.kt @@ -0,0 +1,62 @@ +package org.opensearch.replication.integ.rest + +import org.opensearch.replication.MultiClusterRestTestCase +import org.opensearch.replication.MultiClusterAnnotations +import org.opensearch.replication.StartReplicationRequest +import org.opensearch.replication.startReplication +import org.opensearch.replication.stopReplication +import org.assertj.core.api.Assertions +import org.opensearch.client.RequestOptions +import org.opensearch.client.indices.CreateIndexRequest +import org.junit.Assert +import java.util.concurrent.TimeUnit + + +@MultiClusterAnnotations.ClusterConfigurations( + MultiClusterAnnotations.ClusterConfiguration(clusterName = LEADER), + MultiClusterAnnotations.ClusterConfiguration(clusterName = FOLLOWER) +) + +class ReplicationStopThenRestartIT : MultiClusterRestTestCase() { + private val leaderIndexName = "leader_index" + private val followerIndexName = "follower_index" + + fun `test replication works after unclean stop and start`() { + val followerClient = getClientForCluster(FOLLOWER) + val leaderClient = getClientForCluster(LEADER) + changeTemplate(LEADER) + createConnectionBetweenClusters(FOLLOWER, LEADER) + val createIndexResponse = leaderClient.indices().create(CreateIndexRequest(leaderIndexName), RequestOptions.DEFAULT) + Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName)) + insertDocToIndex(LEADER, "1", "dummy data 1",leaderIndexName) + insertDocToIndex(LEADER, "2", "dummy data 1",leaderIndexName) + + assertBusy ({ + try { + Assert.assertEquals(2, docCount(followerClient, followerIndexName)) + } catch (ex: Exception) { + ex.printStackTrace(); + Assert.fail("Exception while querying follower cluster. Failing to retry again {}") + } + }, 1, TimeUnit.MINUTES) + + + deleteConnection(FOLLOWER) + followerClient.stopReplication(followerIndexName, shouldWait = true) + deleteIndex(followerClient, followerIndexName) + + createConnectionBetweenClusters(FOLLOWER, LEADER) + insertDocToIndex(LEADER, "3", "dummy data 1",leaderIndexName) + insertDocToIndex(LEADER, "4", "dummy data 1",leaderIndexName) + followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName)) + + assertBusy ({ + try { + Assert.assertEquals(4, docCount(followerClient, followerIndexName)) + } catch (ex: Exception) { + Assert.fail("Exception while querying follower cluster. Failing to retry again") + } + }, 1, TimeUnit.MINUTES) + } +} From bc9b61a4807e93d660d7b34cc12409fa6de778c7 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Fri, 2 Jun 2023 16:39:52 +0530 Subject: [PATCH 095/157] Remove any stale replication tasks from cluster state (#905) Clear stale replication tasks in STOP API Signed-off-by: monusingh-1 --- .../TransportStopIndexReplicationAction.kt | 38 +++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/src/main/kotlin/org/opensearch/replication/action/stop/TransportStopIndexReplicationAction.kt b/src/main/kotlin/org/opensearch/replication/action/stop/TransportStopIndexReplicationAction.kt index 8f7ff425..8a6fdf71 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stop/TransportStopIndexReplicationAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stop/TransportStopIndexReplicationAction.kt @@ -54,6 +54,8 @@ import org.opensearch.common.inject.Inject import org.opensearch.common.io.stream.StreamInput import org.opensearch.common.settings.Settings import org.opensearch.replication.util.stackTraceToString +import org.opensearch.persistent.PersistentTasksCustomMetadata +import org.opensearch.persistent.RemovePersistentTaskAction import org.opensearch.threadpool.ThreadPool import org.opensearch.transport.TransportService import java.io.IOException @@ -136,6 +138,7 @@ class TransportStopIndexReplicationAction @Inject constructor(transportService: } } replicationMetadataManager.deleteIndexReplicationMetadata(request.indexName) + removeStaleReplicationTasksFromClusterState(request) listener.onResponse(AcknowledgedResponse(true)) } catch (e: Exception) { log.error("Stop replication failed for index[${request.indexName}] with error ${e.stackTraceToString()}") @@ -144,6 +147,32 @@ class TransportStopIndexReplicationAction @Inject constructor(transportService: } } + private suspend fun removeStaleReplicationTasksFromClusterState(request: StopIndexReplicationRequest) { + try { + val allTasks: PersistentTasksCustomMetadata = + clusterService.state().metadata().custom(PersistentTasksCustomMetadata.TYPE) + for (singleTask in allTasks.tasks()) { + if (isReplicationTask(singleTask, request) && !singleTask.isAssigned){ + log.info("Removing task: ${singleTask.id} from cluster state") + val removeRequest: RemovePersistentTaskAction.Request = + RemovePersistentTaskAction.Request(singleTask.id) + client.suspendExecute(RemovePersistentTaskAction.INSTANCE, removeRequest) + } + } + } catch (e: Exception) { + log.info("Could not update cluster state") + } + } + + // Remove index replication task metadata, format replication:index:fruit-1 + // Remove shard replication task metadata, format replication:[fruit-1][0] + private fun isReplicationTask( + singleTask: PersistentTasksCustomMetadata.PersistentTask<*>, + request: StopIndexReplicationRequest + ) = singleTask.id.startsWith("replication:") && + (singleTask.id == "replication:index:${request.indexName}" || singleTask.id.split(":")[1].contains(request.indexName)) + + private fun validateReplicationStateOfIndex(request: StopIndexReplicationRequest) { // If replication blocks/settings are present, Stop action should proceed with the clean-up // This can happen during settings of follower index are carried over in the snapshot and the restore is @@ -153,6 +182,15 @@ class TransportStopIndexReplicationAction @Inject constructor(transportService: return } + //check for stale replication tasks + val allTasks: PersistentTasksCustomMetadata? = + clusterService.state()?.metadata()?.custom(PersistentTasksCustomMetadata.TYPE) + allTasks?.tasks()?.forEach{ + if (isReplicationTask(it, request) && !it.isAssigned){ + return + } + } + val replicationStateParams = getReplicationStateParamsForIndex(clusterService, request.indexName) ?: throw IllegalArgumentException("No replication in progress for index:${request.indexName}") From 93205a1caba599d526f71cb807f7cb66de256cf2 Mon Sep 17 00:00:00 2001 From: Sooraj Sinha <81695996+soosinha@users.noreply.github.com> Date: Wed, 7 Jun 2023 18:23:15 +0530 Subject: [PATCH 096/157] Use strong password in integ test (#987) Signed-off-by: Sooraj Sinha --- .../replication/integ/rest/SecurityBase.kt | 24 ++++++----- .../integ/rest/SecurityCustomRolesIT.kt | 40 +++++++++---------- .../integ/rest/SecurityCustomRolesLeaderIT.kt | 12 +++--- .../integ/rest/SecurityDlsFlsIT.kt | 22 +++++----- 4 files changed, 50 insertions(+), 48 deletions(-) diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityBase.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityBase.kt index 7f12751b..515893e2 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityBase.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityBase.kt @@ -18,6 +18,8 @@ import org.apache.hc.core5.http.io.entity.StringEntity import org.opensearch.client.Request import org.junit.BeforeClass +const val INTEG_TEST_PASSWORD = "ccr-integ-test@123" + abstract class SecurityBase : MultiClusterRestTestCase() { companion object { var initialized : Boolean = false @@ -282,17 +284,17 @@ abstract class SecurityBase : MultiClusterRestTestCase() { } private fun addUsers(){ - addUserToCluster("testUser1","password", FOLLOWER) - addUserToCluster("testUser1","password", LEADER) - addUserToCluster("testUser2","password", FOLLOWER) - addUserToCluster("testUser2","password", LEADER) - addUserToCluster("testUser3","password", FOLLOWER) - addUserToCluster("testUser4","password", FOLLOWER) - addUserToCluster("testUser5","password", FOLLOWER) - addUserToCluster("testUser6","password", LEADER) - addUserToCluster("testUser6","password", FOLLOWER) - addUserToCluster("testUser7","password", LEADER) - addUserToCluster("testUser7","password", FOLLOWER) + addUserToCluster("testUser1", INTEG_TEST_PASSWORD, FOLLOWER) + addUserToCluster("testUser1", INTEG_TEST_PASSWORD, LEADER) + addUserToCluster("testUser2", INTEG_TEST_PASSWORD, FOLLOWER) + addUserToCluster("testUser2", INTEG_TEST_PASSWORD, LEADER) + addUserToCluster("testUser3", INTEG_TEST_PASSWORD, FOLLOWER) + addUserToCluster("testUser4", INTEG_TEST_PASSWORD, FOLLOWER) + addUserToCluster("testUser5", INTEG_TEST_PASSWORD, FOLLOWER) + addUserToCluster("testUser6", INTEG_TEST_PASSWORD, LEADER) + addUserToCluster("testUser6", INTEG_TEST_PASSWORD, FOLLOWER) + addUserToCluster("testUser7", INTEG_TEST_PASSWORD, LEADER) + addUserToCluster("testUser7", INTEG_TEST_PASSWORD, FOLLOWER) } private fun addUserToCluster(userName: String, password: String, clusterName: String) { diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesIT.kt index 04f0378b..bdb7e44d 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesIT.kt @@ -60,7 +60,7 @@ class SecurityCustomRolesIT: SecurityBase() { useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) followerClient.startReplication(startReplicationRequest, - requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password"), waitForRestore = true) + requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser1",INTEG_TEST_PASSWORD), waitForRestore = true) assertBusy { Assertions.assertThat(followerClient.indices().exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)).isEqualTo(true) } @@ -79,7 +79,7 @@ class SecurityCustomRolesIT: SecurityBase() { useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleNoPerms")) Assertions.assertThatThrownBy { followerClient.startReplication(startReplicationRequest, - requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser2","password")) } + requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser2",INTEG_TEST_PASSWORD)) } .isInstanceOf(ResponseException::class.java) .hasMessageContaining("403 Forbidden") } @@ -89,7 +89,7 @@ class SecurityCustomRolesIT: SecurityBase() { Assertions.assertThatThrownBy { followerClient.stopReplication("follower-index1", - requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) + requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser1",INTEG_TEST_PASSWORD)) }.isInstanceOf(ResponseException::class.java) .hasMessageContaining("No replication in progress for index:follower-index1") } @@ -99,7 +99,7 @@ class SecurityCustomRolesIT: SecurityBase() { Assertions.assertThatThrownBy { followerClient.stopReplication("follower-index1", - requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser2","password")) + requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser2",INTEG_TEST_PASSWORD)) }.isInstanceOf(ResponseException::class.java) .hasMessageContaining("403 Forbidden") } @@ -115,7 +115,7 @@ class SecurityCustomRolesIT: SecurityBase() { var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) - var requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password") + var requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1",INTEG_TEST_PASSWORD) followerClient.startReplication(startReplicationRequest, waitForRestore = true, requestOptions = requestOptions) @@ -145,11 +145,11 @@ class SecurityCustomRolesIT: SecurityBase() { useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) followerClient.startReplication(startReplicationRequest, waitForRestore = true, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1",INTEG_TEST_PASSWORD)) Assertions.assertThatThrownBy { followerClient.pauseReplication(followerIndexName, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser2","password")) + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser2",INTEG_TEST_PASSWORD)) }.isInstanceOf(ResponseException::class.java) .hasMessageContaining("403 Forbidden") } @@ -167,11 +167,11 @@ class SecurityCustomRolesIT: SecurityBase() { useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) followerClient.startReplication(startReplicationRequest, waitForRestore = true, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1",INTEG_TEST_PASSWORD)) assertBusy { `validate status syncing response`(followerClient.replicationStatus(followerIndexName, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password"))) + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1",INTEG_TEST_PASSWORD))) } } @@ -188,11 +188,11 @@ class SecurityCustomRolesIT: SecurityBase() { useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) followerClient.startReplication(startReplicationRequest, waitForRestore = true, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1",INTEG_TEST_PASSWORD)) Assertions.assertThatThrownBy { followerClient.replicationStatus(followerIndexName, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser2","password")) + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser2",INTEG_TEST_PASSWORD)) }.isInstanceOf(ResponseException::class.java) .hasMessageContaining("403 Forbidden") } @@ -215,7 +215,7 @@ class SecurityCustomRolesIT: SecurityBase() { followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName, useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")), - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1",INTEG_TEST_PASSWORD)) assertBusy { Assertions.assertThat(followerClient.indices() .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) @@ -234,7 +234,7 @@ class SecurityCustomRolesIT: SecurityBase() { .put("index.shard.check_on_startup", "checksum") .build() followerClient.updateReplication(followerIndexName, settings, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1",INTEG_TEST_PASSWORD)) // Wait for the settings to get updated at follower cluster. assertBusy ({ @@ -260,7 +260,7 @@ class SecurityCustomRolesIT: SecurityBase() { Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName, useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")), - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password"), waitForRestore = true) + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1",INTEG_TEST_PASSWORD), waitForRestore = true) assertBusy { Assertions.assertThat(followerClient.indices() .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) @@ -279,7 +279,7 @@ class SecurityCustomRolesIT: SecurityBase() { .build() Assertions.assertThatThrownBy { followerClient.updateReplication(followerIndexName, settings, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser2","password")) + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser2",INTEG_TEST_PASSWORD)) }.isInstanceOf(ResponseException::class.java) .hasMessageContaining("403 Forbidden") } @@ -297,7 +297,7 @@ class SecurityCustomRolesIT: SecurityBase() { try { followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, indexPattern, useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms"), - requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) + requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser1",INTEG_TEST_PASSWORD)) // Verify that existing index matching the pattern are replicated. assertBusy ({ Assertions.assertThat(followerClient.indices() @@ -326,7 +326,7 @@ class SecurityCustomRolesIT: SecurityBase() { Assertions.assertThatThrownBy { followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, indexPattern, useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleNoPerms"), - requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser2","password")) + requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser2",INTEG_TEST_PASSWORD)) }.isInstanceOf(ResponseException::class.java) .hasMessageContaining("403 Forbidden") } @@ -358,7 +358,7 @@ class SecurityCustomRolesIT: SecurityBase() { var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) followerClient.startReplication(startReplicationRequest, waitForRestore = true, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1",INTEG_TEST_PASSWORD)) insertDocToIndex(LEADER, "1", "dummy data 1",leaderIndexName) //Querying ES cluster throws random exceptions like ClusterManagerNotDiscovered or ShardsFailed etc, so catching them and retrying assertBusy ({ @@ -370,7 +370,7 @@ class SecurityCustomRolesIT: SecurityBase() { }, 1, TimeUnit.MINUTES) assertBusy { `validate status syncing response`(followerClient.replicationStatus(followerIndexName, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password"))) + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1",INTEG_TEST_PASSWORD))) } updateRole(followerIndexName,"followerRoleValidPerms", false) @@ -378,7 +378,7 @@ class SecurityCustomRolesIT: SecurityBase() { assertBusy ({ validatePausedState(followerClient.replicationStatus(followerIndexName, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password"))) + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1",INTEG_TEST_PASSWORD))) }, 100, TimeUnit.SECONDS) } finally { updateRole(followerIndexName,"followerRoleValidPerms", true) diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesLeaderIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesLeaderIT.kt index 9332166b..cc4d773f 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesLeaderIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityCustomRolesLeaderIT.kt @@ -47,7 +47,7 @@ class SecurityCustomRolesLeaderIT: SecurityBase() { var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, useRoles = UseRoles(leaderClusterRole = "leaderRoleNoPerms",followerClusterRole = "followerRoleValidPerms")) Assertions.assertThatThrownBy { followerClient.startReplication(startReplicationRequest, - requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser6","password")) } + requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser6",INTEG_TEST_PASSWORD)) } .isInstanceOf(ResponseException::class.java) .hasMessageContaining("403 Forbidden") .hasMessageContaining("no permissions for [indices:admin/plugins/replication/index/setup/validate]") @@ -64,7 +64,7 @@ class SecurityCustomRolesLeaderIT: SecurityBase() { var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) followerClient.startReplication(startReplicationRequest, waitForRestore = true, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1",INTEG_TEST_PASSWORD)) insertDocToIndex(LEADER, "1", "dummy data 1",leaderIndexName) //Querying ES cluster throws random exceptions like ClusterManagerNotDiscovered or ShardsFailed etc, so catching them and retrying assertBusy ({ @@ -76,13 +76,13 @@ class SecurityCustomRolesLeaderIT: SecurityBase() { }, 1, TimeUnit.MINUTES) assertBusy { `validate status syncing response`(followerClient.replicationStatus(followerIndexName, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password"))) + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1",INTEG_TEST_PASSWORD))) } updateRole(followerIndexName,"leaderRoleValidPerms", false) insertDocToIndex(LEADER, "2", "dummy data 2",leaderIndexName) assertBusy ({ validatePausedState(followerClient.replicationStatus(followerIndexName, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password"))) + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1",INTEG_TEST_PASSWORD))) }, 100, TimeUnit.SECONDS) } finally { updateRole(followerIndexName,"leaderRoleValidPerms", true) @@ -101,10 +101,10 @@ class SecurityCustomRolesLeaderIT: SecurityBase() { useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) updateFileChunkPermissions("","leaderRoleValidPerms", false) followerClient.startReplication(startReplicationRequest, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1",INTEG_TEST_PASSWORD)) assertBusy ({ validateFailedState(followerClient.replicationStatus(followerIndexName, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password"))) + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser1",INTEG_TEST_PASSWORD))) }, 60, TimeUnit.SECONDS) } catch (ex : Exception) { logger.info("Exception is", ex) diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityDlsFlsIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityDlsFlsIT.kt index 89d8269a..b8ababe8 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityDlsFlsIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/SecurityDlsFlsIT.kt @@ -49,7 +49,7 @@ class SecurityDlsFlsIT: SecurityBase() { var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerDlsRole")) Assertions.assertThatThrownBy { followerClient.startReplication(startReplicationRequest, - requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser3","password")) } + requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser3",INTEG_TEST_PASSWORD)) } .isInstanceOf(ResponseException::class.java) .hasMessageContaining(DLS_FLS_EXCEPTION_MESSAGE) .hasMessageContaining("403 Forbidden") @@ -59,7 +59,7 @@ class SecurityDlsFlsIT: SecurityBase() { val followerClient = getClientForCluster(FOLLOWER) Assertions.assertThatThrownBy { followerClient.stopReplication("follower-index1-stop-forbidden", - requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser3","password")) + requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser3",INTEG_TEST_PASSWORD)) }.isInstanceOf(ResponseException::class.java) .hasMessageContaining(DLS_FLS_EXCEPTION_MESSAGE) .hasMessageContaining("403 Forbidden") @@ -75,10 +75,10 @@ class SecurityDlsFlsIT: SecurityBase() { var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) followerClient.startReplication(startReplicationRequest, waitForRestore = true, - requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) + requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser1",INTEG_TEST_PASSWORD)) Assertions.assertThatThrownBy { followerClient.pauseReplication(followerIndexName, - requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser3","password")) + requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser3",INTEG_TEST_PASSWORD)) }.isInstanceOf(ResponseException::class.java) .hasMessageContaining(DLS_FLS_EXCEPTION_MESSAGE) .hasMessageContaining("403 Forbidden") @@ -94,10 +94,10 @@ class SecurityDlsFlsIT: SecurityBase() { var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")) followerClient.startReplication(startReplicationRequest, waitForRestore = true, - requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password")) + requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser1",INTEG_TEST_PASSWORD)) Assertions.assertThatThrownBy { followerClient.replicationStatus(followerIndexName, - requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser3","password")) + requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser3",INTEG_TEST_PASSWORD)) }.isInstanceOf(ResponseException::class.java) .hasMessageContaining(DLS_FLS_EXCEPTION_MESSAGE) .hasMessageContaining("403 Forbidden") @@ -116,7 +116,7 @@ class SecurityDlsFlsIT: SecurityBase() { Assertions.assertThat(createIndexResponse.isAcknowledged).isTrue() followerClient.startReplication(StartReplicationRequest("source", leaderIndexName, followerIndexName, useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerRoleValidPerms")), - requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser1","password"), waitForRestore = true) + requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser1",INTEG_TEST_PASSWORD), waitForRestore = true) assertBusy { Assertions.assertThat(followerClient.indices() .exists(GetIndexRequest(followerIndexName), RequestOptions.DEFAULT)) @@ -135,7 +135,7 @@ class SecurityDlsFlsIT: SecurityBase() { .build() Assertions.assertThatThrownBy { followerClient.updateReplication(followerIndexName, settings, - requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser3","password")) + requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser3",INTEG_TEST_PASSWORD)) }.isInstanceOf(ResponseException::class.java) .hasMessageContaining(DLS_FLS_EXCEPTION_MESSAGE) .hasMessageContaining("403 Forbidden") @@ -151,7 +151,7 @@ class SecurityDlsFlsIT: SecurityBase() { var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerFlsRole")) Assertions.assertThatThrownBy { followerClient.startReplication(startReplicationRequest, - requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser4","password")) } + requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser4",INTEG_TEST_PASSWORD)) } .isInstanceOf(ResponseException::class.java) .hasMessageContaining(DLS_FLS_EXCEPTION_MESSAGE) .hasMessageContaining("403 Forbidden") @@ -167,7 +167,7 @@ class SecurityDlsFlsIT: SecurityBase() { var startReplicationRequest = StartReplicationRequest("source",leaderIndexName,followerIndexName, useRoles = UseRoles(leaderClusterRole = "leaderRoleValidPerms",followerClusterRole = "followerFieldMaskRole")) Assertions.assertThatThrownBy { followerClient.startReplication(startReplicationRequest, - requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser5","password")) } + requestOptions= RequestOptions.DEFAULT.addBasicAuthHeader("testUser5",INTEG_TEST_PASSWORD)) } .isInstanceOf(ResponseException::class.java) .hasMessageContaining(DLS_FLS_EXCEPTION_MESSAGE) .hasMessageContaining("403 Forbidden") @@ -190,7 +190,7 @@ class SecurityDlsFlsIT: SecurityBase() { ) followerClient.startReplication( startReplicationRequest, - requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser7", "password"), + requestOptions = RequestOptions.DEFAULT.addBasicAuthHeader("testUser7", INTEG_TEST_PASSWORD), waitForRestore = true ) OpenSearchTestCase.assertBusy { From 232818a198feaeef2a293ba6794074b22dc3481e Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Tue, 20 Jun 2023 12:09:23 +0530 Subject: [PATCH 097/157] Update CODEOWNERS (#947) Update CODEOWNERS.md to have unique rule Signed-off-by: Monu Singh --- .github/CODEOWNERS | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index 3fcb6af0..93ccb51c 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -1,7 +1,2 @@ # This list should match with the maintainters list. -* @krishna-ggk -* @ankitkala -* @saikaranam-amazon -* @soosinha -* @gbbafna -* @monusingh-1 +* @krishna-ggk @ankitkala @saikaranam-amazon @soosinha @gbbafna @monusingh-1 From d66a7210f89b752f88573960a3b82c36697d8e32 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Tue, 20 Jun 2023 12:10:58 +0530 Subject: [PATCH 098/157] Refresh dependencies (#899) Refresh dependencies before running any GitHub workflow Signed-off-by: Monu Singh --- .github/workflows/build-and-test.yml | 2 +- .github/workflows/build.yml | 2 +- .github/workflows/bwc.yml | 2 +- .github/workflows/security-knn-tests.yml | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/.github/workflows/build-and-test.yml b/.github/workflows/build-and-test.yml index e8a6ace5..4a257534 100644 --- a/.github/workflows/build-and-test.yml +++ b/.github/workflows/build-and-test.yml @@ -31,7 +31,7 @@ jobs: uses: actions/checkout@v2 - name: Build and run Replication tests run: | - ./gradlew clean release -D"build.snapshot=true" + ./gradlew --refresh-dependencies clean release -D"build.snapshot=true" - name: Upload failed logs uses: actions/upload-artifact@v2 if: failure() diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index 1faf6511..2e4d54e4 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -34,4 +34,4 @@ jobs: uses: actions/checkout@v2 - name: Build and run Replication tests run: | - ./gradlew clean release -D"build.snapshot=true" -x test -x IntegTest \ No newline at end of file + ./gradlew --refresh-dependencies clean release -D"build.snapshot=true" -x test -x IntegTest \ No newline at end of file diff --git a/.github/workflows/bwc.yml b/.github/workflows/bwc.yml index 432c7dce..59e082c7 100644 --- a/.github/workflows/bwc.yml +++ b/.github/workflows/bwc.yml @@ -25,7 +25,7 @@ jobs: - name: Build and run Replication tests run: | echo "Running backwards compatibility tests ..." - ./gradlew clean release -Dbuild.snapshot=true -x test -x IntegTest + ./gradlew --refresh-dependencies clean release -Dbuild.snapshot=true -x test -x IntegTest ./gradlew fullRestartClusterTask --stacktrace - name: Upload failed logs uses: actions/upload-artifact@v2 diff --git a/.github/workflows/security-knn-tests.yml b/.github/workflows/security-knn-tests.yml index de049f0c..3f98f3cb 100644 --- a/.github/workflows/security-knn-tests.yml +++ b/.github/workflows/security-knn-tests.yml @@ -63,7 +63,7 @@ jobs: - name: Build and run Replication tests run: | ls -al src/test/resources/security/plugin - ./gradlew clean release -Dbuild.snapshot=true -PnumNodes=1 -Psecurity=true + ./gradlew --refresh-dependencies clean release -Dbuild.snapshot=true -PnumNodes=1 -Psecurity=true - name: Upload failed logs uses: actions/upload-artifact@v2 if: failure() @@ -99,7 +99,7 @@ jobs: uses: actions/checkout@v2 - name: Build and run Replication tests run: | - ./gradlew clean release -Dbuild.snapshot=true -PnumNodes=1 -Dtests.class=org.opensearch.replication.BasicReplicationIT -Dtests.method="test knn index replication" -Pknn=true + ./gradlew --refresh-dependencies clean release -Dbuild.snapshot=true -PnumNodes=1 -Dtests.class=org.opensearch.replication.BasicReplicationIT -Dtests.method="test knn index replication" -Pknn=true - name: Upload failed logs uses: actions/upload-artifact@v2 if: failure() From 448e7a7501e5ff8740dc2d7635c08ae62d19147e Mon Sep 17 00:00:00 2001 From: sricharanvuppu <113983630+sricharanvuppu@users.noreply.github.com> Date: Wed, 28 Jun 2023 10:32:00 +0530 Subject: [PATCH 099/157] Handling OpenSearchRejectExecuteException Exception (#1004) * Handling OpenSearchRejectExecuteException Exception * introduced writersPerShard setting. Signed-off-by: sricharanvuppu --- .../replication/ReplicationException.kt | 14 +++- .../replication/ReplicationPlugin.kt | 16 ++-- .../replication/ReplicationSettings.kt | 2 + .../task/shard/ShardReplicationTask.kt | 3 +- .../task/shard/TranslogSequencer.kt | 78 ++++++++++++++----- .../opensearch/replication/util/Extensions.kt | 25 ++++-- .../task/shard/TranslogSequencerTests.kt | 2 +- 7 files changed, 100 insertions(+), 40 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/ReplicationException.kt b/src/main/kotlin/org/opensearch/replication/ReplicationException.kt index 89d2456c..891be0a3 100644 --- a/src/main/kotlin/org/opensearch/replication/ReplicationException.kt +++ b/src/main/kotlin/org/opensearch/replication/ReplicationException.kt @@ -12,22 +12,28 @@ package org.opensearch.replication import org.opensearch.OpenSearchException +import org.opensearch.OpenSearchStatusException import org.opensearch.action.ShardOperationFailedException import org.opensearch.cluster.metadata.IndexMetadata.INDEX_UUID_NA_VALUE import org.opensearch.index.shard.ShardId +import org.opensearch.rest.RestStatus /** * Base class replication exceptions. Note: Replication process may throw exceptions that do not derive from this such as * [org.opensearch.ResourceAlreadyExistsException], [org.opensearch.index.IndexNotFoundException] or * [org.opensearch.index.shard.ShardNotFoundException]. */ -class ReplicationException: OpenSearchException { +class ReplicationException: OpenSearchStatusException { - constructor(message: String, vararg args: Any) : super(message, *args) + constructor(message: String, status : RestStatus, cause: Throwable, vararg args: Any) : super(message, status, cause, *args) - constructor(message: String, cause: Throwable, vararg args: Any) : super(message, cause, *args) + constructor(message: String, vararg args: Any) : super(message, RestStatus.INTERNAL_SERVER_ERROR, *args) - constructor(message: String, shardFailures: Array) : super(message) { + constructor(message: String, status: RestStatus, vararg args: Any) : super(message, status, *args) + + constructor(cause: Throwable, status: RestStatus, vararg args: Any) : super(cause.message, status, *args) + + constructor(message: String, shardFailures: Array): super(message, shardFailures.firstOrNull()?.status()?:RestStatus.INTERNAL_SERVER_ERROR) { shardFailures.firstOrNull()?.let { setShard(ShardId(it.index(), INDEX_UUID_NA_VALUE, it.shardId())) // Add first failure as cause and rest as suppressed... diff --git a/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt b/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt index 3ea64fe0..4254412c 100644 --- a/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt +++ b/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt @@ -175,6 +175,8 @@ internal class ReplicationPlugin : Plugin(), ActionPlugin, PersistentTaskPlugin, Setting.Property.Dynamic, Setting.Property.NodeScope) val REPLICATION_FOLLOWER_CONCURRENT_READERS_PER_SHARD = Setting.intSetting("plugins.replication.follower.concurrent_readers_per_shard", 2, 1, Setting.Property.Dynamic, Setting.Property.NodeScope) + val REPLICATION_FOLLOWER_CONCURRENT_WRITERS_PER_SHARD = Setting.intSetting("plugins.replication.follower.concurrent_writers_per_shard", 2, 1, + Setting.Property.Dynamic, Setting.Property.NodeScope) val REPLICATION_PARALLEL_READ_POLL_INTERVAL = Setting.timeSetting ("plugins.replication.follower.poll_interval", TimeValue.timeValueMillis(50), TimeValue.timeValueMillis(1), TimeValue.timeValueSeconds(1), Setting.Property.Dynamic, Setting.Property.NodeScope) val REPLICATION_AUTOFOLLOW_REMOTE_INDICES_POLL_INTERVAL = Setting.timeSetting ("plugins.replication.autofollow.fetch_poll_interval", TimeValue.timeValueSeconds(30), TimeValue.timeValueSeconds(30), @@ -346,14 +348,14 @@ internal class ReplicationPlugin : Plugin(), ActionPlugin, PersistentTaskPlugin, override fun getSettings(): List> { return listOf(REPLICATED_INDEX_SETTING, REPLICATION_FOLLOWER_OPS_BATCH_SIZE, REPLICATION_LEADER_THREADPOOL_SIZE, - REPLICATION_LEADER_THREADPOOL_QUEUE_SIZE, REPLICATION_FOLLOWER_CONCURRENT_READERS_PER_SHARD, - REPLICATION_FOLLOWER_RECOVERY_CHUNK_SIZE, REPLICATION_FOLLOWER_RECOVERY_PARALLEL_CHUNKS, - REPLICATION_PARALLEL_READ_POLL_INTERVAL, REPLICATION_AUTOFOLLOW_REMOTE_INDICES_POLL_INTERVAL, - REPLICATION_AUTOFOLLOW_REMOTE_INDICES_RETRY_POLL_INTERVAL, REPLICATION_METADATA_SYNC_INTERVAL, - REPLICATION_RETENTION_LEASE_MAX_FAILURE_DURATION, REPLICATION_INDEX_TRANSLOG_PRUNING_ENABLED_SETTING, - REPLICATION_INDEX_TRANSLOG_RETENTION_SIZE, REPLICATION_FOLLOWER_BLOCK_START, REPLICATION_AUTOFOLLOW_CONCURRENT_REPLICATION_JOBS_TRIGGER_SIZE) + REPLICATION_LEADER_THREADPOOL_QUEUE_SIZE, REPLICATION_FOLLOWER_CONCURRENT_READERS_PER_SHARD, + REPLICATION_FOLLOWER_RECOVERY_CHUNK_SIZE, REPLICATION_FOLLOWER_RECOVERY_PARALLEL_CHUNKS, + REPLICATION_PARALLEL_READ_POLL_INTERVAL, REPLICATION_AUTOFOLLOW_REMOTE_INDICES_POLL_INTERVAL, + REPLICATION_AUTOFOLLOW_REMOTE_INDICES_RETRY_POLL_INTERVAL, REPLICATION_METADATA_SYNC_INTERVAL, + REPLICATION_RETENTION_LEASE_MAX_FAILURE_DURATION, REPLICATION_INDEX_TRANSLOG_PRUNING_ENABLED_SETTING, + REPLICATION_INDEX_TRANSLOG_RETENTION_SIZE, REPLICATION_FOLLOWER_BLOCK_START, REPLICATION_AUTOFOLLOW_CONCURRENT_REPLICATION_JOBS_TRIGGER_SIZE, + REPLICATION_FOLLOWER_CONCURRENT_WRITERS_PER_SHARD) } - override fun getInternalRepositories(env: Environment, namedXContentRegistry: NamedXContentRegistry, clusterService: ClusterService, recoverySettings: RecoverySettings): Map { val repoFactory = Repository.Factory { repoMetadata: RepositoryMetadata -> diff --git a/src/main/kotlin/org/opensearch/replication/ReplicationSettings.kt b/src/main/kotlin/org/opensearch/replication/ReplicationSettings.kt index a6d1bbd3..2b516f8e 100644 --- a/src/main/kotlin/org/opensearch/replication/ReplicationSettings.kt +++ b/src/main/kotlin/org/opensearch/replication/ReplicationSettings.kt @@ -24,6 +24,7 @@ open class ReplicationSettings(clusterService: ClusterService) { @Volatile var chunkSize = ReplicationPlugin.REPLICATION_FOLLOWER_RECOVERY_CHUNK_SIZE.get(clusterService.settings) @Volatile var concurrentFileChunks = ReplicationPlugin.REPLICATION_FOLLOWER_RECOVERY_PARALLEL_CHUNKS.get(clusterService.settings) @Volatile var readersPerShard = clusterService.clusterSettings.get(ReplicationPlugin.REPLICATION_FOLLOWER_CONCURRENT_READERS_PER_SHARD) + @Volatile var writersPerShard = clusterService.clusterSettings.get(ReplicationPlugin.REPLICATION_FOLLOWER_CONCURRENT_WRITERS_PER_SHARD) @Volatile var batchSize = clusterService.clusterSettings.get(ReplicationPlugin.REPLICATION_FOLLOWER_OPS_BATCH_SIZE) @Volatile var pollDuration: TimeValue = clusterService.clusterSettings.get(ReplicationPlugin.REPLICATION_PARALLEL_READ_POLL_INTERVAL) @Volatile var autofollowFetchPollDuration = clusterService.clusterSettings.get(ReplicationPlugin.REPLICATION_AUTOFOLLOW_REMOTE_INDICES_POLL_INTERVAL) @@ -41,6 +42,7 @@ open class ReplicationSettings(clusterService: ClusterService) { clusterSettings.addSettingsUpdateConsumer(ReplicationPlugin.REPLICATION_FOLLOWER_RECOVERY_CHUNK_SIZE) { value: ByteSizeValue -> this.chunkSize = value} clusterSettings.addSettingsUpdateConsumer(ReplicationPlugin.REPLICATION_FOLLOWER_RECOVERY_PARALLEL_CHUNKS) { value: Int -> this.concurrentFileChunks = value} clusterSettings.addSettingsUpdateConsumer(ReplicationPlugin.REPLICATION_FOLLOWER_CONCURRENT_READERS_PER_SHARD) { value: Int -> this.readersPerShard = value} + clusterSettings.addSettingsUpdateConsumer(ReplicationPlugin.REPLICATION_FOLLOWER_CONCURRENT_WRITERS_PER_SHARD) { value: Int -> this.writersPerShard = value} clusterSettings.addSettingsUpdateConsumer(ReplicationPlugin.REPLICATION_FOLLOWER_OPS_BATCH_SIZE) { batchSize = it } clusterSettings.addSettingsUpdateConsumer(ReplicationPlugin.REPLICATION_PARALLEL_READ_POLL_INTERVAL) { pollDuration = it } clusterSettings.addSettingsUpdateConsumer(ReplicationPlugin.REPLICATION_RETENTION_LEASE_MAX_FAILURE_DURATION) { leaseRenewalMaxFailureDuration = it } diff --git a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationTask.kt b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationTask.kt index f08b2c6b..e393805e 100644 --- a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationTask.kt @@ -214,7 +214,7 @@ class ShardReplicationTask(id: Long, type: String, action: String, description: // Since this setting is not dynamic, setting update would only reflect after pause-resume or on a new replication job. val rateLimiter = Semaphore(replicationSettings.readersPerShard) val sequencer = TranslogSequencer(scope, replicationMetadata, followerShardId, leaderAlias, leaderShardId.indexName, - TaskId(clusterService.nodeName, id), client, indexShard.localCheckpoint, followerClusterStats) + TaskId(clusterService.nodeName, id), client, indexShard.localCheckpoint, followerClusterStats, replicationSettings.writersPerShard) val changeTracker = ShardReplicationChangesTracker(indexShard, replicationSettings) followerClusterStats.stats[followerShardId]!!.followerCheckpoint = indexShard.localCheckpoint @@ -255,7 +255,6 @@ class ShardReplicationTask(id: Long, type: String, action: String, description: followerClusterStats.stats[followerShardId]!!.opsReadFailures.addAndGet(1) logInfo("Unable to get changes from seqNo: $fromSeqNo. ${e.stackTraceToString()}") changeTracker.updateBatchFetched(false, fromSeqNo, toSeqNo, fromSeqNo - 1,-1) - // Propagate 4xx exceptions up the chain and halt replication as they are irrecoverable val range4xx = 400.rangeTo(499) if (e is OpenSearchException && diff --git a/src/main/kotlin/org/opensearch/replication/task/shard/TranslogSequencer.kt b/src/main/kotlin/org/opensearch/replication/task/shard/TranslogSequencer.kt index 38b625bf..fc14d0b5 100644 --- a/src/main/kotlin/org/opensearch/replication/task/shard/TranslogSequencer.kt +++ b/src/main/kotlin/org/opensearch/replication/task/shard/TranslogSequencer.kt @@ -21,11 +21,14 @@ import org.opensearch.replication.util.suspendExecuteWithRetries import kotlinx.coroutines.CompletableDeferred import kotlinx.coroutines.CoroutineScope import kotlinx.coroutines.ObsoleteCoroutinesApi -import kotlinx.coroutines.channels.Channel import kotlinx.coroutines.channels.actor import kotlinx.coroutines.launch +import kotlinx.coroutines.sync.Semaphore import org.opensearch.client.Client +import org.opensearch.OpenSearchException +import org.opensearch.action.support.TransportActions import org.opensearch.common.logging.Loggers +import org.opensearch.index.IndexNotFoundException import org.opensearch.index.shard.ShardId import org.opensearch.index.translog.Translog import org.opensearch.replication.util.indicesService @@ -33,6 +36,8 @@ import org.opensearch.tasks.TaskId import java.util.ArrayList import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.TimeUnit +import org.opensearch.rest.RestStatus + /** * A TranslogSequencer allows multiple producers of [Translog.Operation]s to write them in sequence number order to an @@ -50,7 +55,7 @@ class TranslogSequencer(scope: CoroutineScope, private val replicationMetadata: private val followerShardId: ShardId, private val leaderAlias: String, private val leaderIndexName: String, private val parentTaskId: TaskId, private val client: Client, initialSeqNo: Long, - private val followerClusterStats: FollowerClusterStats) { + private val followerClusterStats: FollowerClusterStats, writersPerShard : Int) { private val unAppliedChanges = ConcurrentHashMap() private val log = Loggers.getLogger(javaClass, followerShardId)!! @@ -59,11 +64,14 @@ class TranslogSequencer(scope: CoroutineScope, private val replicationMetadata: val followerIndexService = indicesService.indexServiceSafe(followerShardId.index) val indexShard = followerIndexService.getShard(followerShardId.id) - private val sequencer = scope.actor(capacity = Channel.UNLIMITED) { + private val sequencer = scope.actor(capacity = 0) { + // Exceptions thrown here will mark the channel as failed and the next attempt to send to the channel will // raise the same exception. See [SendChannel.close] method for details. + val rateLimiter = Semaphore(writersPerShard) var highWatermark = initialSeqNo for (m in channel) { + rateLimiter.acquire() while (unAppliedChanges.containsKey(highWatermark + 1)) { val next = unAppliedChanges.remove(highWatermark + 1)!! val replayRequest = ReplayChangesRequest(followerShardId, next.changes, next.maxSeqNoOfUpdatesOrDeletes, @@ -73,26 +81,55 @@ class TranslogSequencer(scope: CoroutineScope, private val replicationMetadata: var relativeStartNanos = System.nanoTime() val retryOnExceptions = ArrayList>() retryOnExceptions.add(MappingNotAvailableException::class.java) + var tryReplay = true + try { + while (tryReplay) { + tryReplay = false + try { + val replayResponse = client.suspendExecuteWithRetries( + replicationMetadata, + ReplayChangesAction.INSTANCE, + replayRequest, + log = log, + retryOn = retryOnExceptions + ) + if (replayResponse.shardInfo.failed > 0) { + replayResponse.shardInfo.failures.forEachIndexed { i, failure -> + log.error("Failed replaying changes. Failure:$i:$failure}") + } + followerClusterStats.stats[followerShardId]!!.opsWriteFailures.addAndGet( + replayResponse.shardInfo.failed.toLong() + ) + throw ReplicationException( + "failed to replay changes", + replayResponse.shardInfo.failures + ) + } - val replayResponse = client.suspendExecuteWithRetries( - replicationMetadata, - ReplayChangesAction.INSTANCE, - replayRequest, - log = log, - retryOn = retryOnExceptions - ) - if (replayResponse.shardInfo.failed > 0) { - replayResponse.shardInfo.failures.forEachIndexed { i, failure -> - log.error("Failed replaying changes. Failure:$i:$failure}") + val tookInNanos = System.nanoTime() - relativeStartNanos + followerClusterStats.stats[followerShardId]!!.totalWriteTime.addAndGet( + TimeUnit.NANOSECONDS.toMillis(tookInNanos) + ) + followerClusterStats.stats[followerShardId]!!.opsWritten.addAndGet( + replayRequest.changes.size.toLong() + ) + } catch (e: OpenSearchException) { + if (e !is IndexNotFoundException && (retryOnExceptions.contains(e.javaClass) + || TransportActions.isShardNotAvailableException(e) + // This waits for the dependencies to load and retry. Helps during boot-up + || e.status().status >= 500 + || e.status() == RestStatus.TOO_MANY_REQUESTS)) { + tryReplay = true + } + else { + log.error("Got non-retriable Exception:${e.message} with status:${e.status()}") + throw e + } + } } - followerClusterStats.stats[followerShardId]!!.opsWriteFailures.addAndGet(replayResponse.shardInfo.failed.toLong()) - throw ReplicationException("failed to replay changes", replayResponse.shardInfo.failures) + } finally { + rateLimiter.release() } - - val tookInNanos = System.nanoTime() - relativeStartNanos - followerClusterStats.stats[followerShardId]!!.totalWriteTime.addAndGet(TimeUnit.NANOSECONDS.toMillis(tookInNanos)) - followerClusterStats.stats[followerShardId]!!.opsWritten.addAndGet(replayRequest.changes.size.toLong()) - followerClusterStats.stats[followerShardId]!!.followerCheckpoint = indexShard.localCheckpoint } highWatermark = next.changes.lastOrNull()?.seqNo() ?: highWatermark } @@ -105,6 +142,7 @@ class TranslogSequencer(scope: CoroutineScope, private val replicationMetadata: completed.await() } + suspend fun send(changes : GetChangesResponse) { unAppliedChanges[changes.fromSeqNo] = changes sequencer.send(Unit) diff --git a/src/main/kotlin/org/opensearch/replication/util/Extensions.kt b/src/main/kotlin/org/opensearch/replication/util/Extensions.kt index 96749f7b..643cc010 100644 --- a/src/main/kotlin/org/opensearch/replication/util/Extensions.kt +++ b/src/main/kotlin/org/opensearch/replication/util/Extensions.kt @@ -29,6 +29,7 @@ import org.opensearch.action.index.IndexResponse import org.opensearch.action.support.TransportActions import org.opensearch.client.Client import org.opensearch.common.util.concurrent.ThreadContext +import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException import org.opensearch.index.IndexNotFoundException import org.opensearch.index.shard.ShardId import org.opensearch.index.store.Store @@ -43,6 +44,7 @@ import org.opensearch.transport.NodeDisconnectedException import org.opensearch.transport.NodeNotConnectedException import java.io.PrintWriter import java.io.StringWriter +import java.lang.Exception /* * Extension function to use the store object @@ -110,7 +112,8 @@ suspend fun Client.suspendExecuteWith defaultContext: Boolean = false): Resp { var currentBackoff = backoff retryOn.addAll(defaultRetryableExceptions()) - repeat(numberOfRetries - 1) { + var retryException: Exception + repeat(numberOfRetries - 1) { index -> try { return suspendExecute(replicationMetadata, action, req, injectSecurityContext = injectSecurityContext, defaultContext = defaultContext) @@ -122,19 +125,29 @@ suspend fun Client.suspendExecuteWith // This waits for the dependencies to load and retry. Helps during boot-up || e.status().status >= 500 || e.status() == RestStatus.TOO_MANY_REQUESTS)) { - log.warn("Encountered a failure while executing in $req. Retrying in ${currentBackoff/1000} seconds" + - ".", e) - delay(currentBackoff) - currentBackoff = (currentBackoff * factor).toLong().coerceAtMost(maxTimeOut) + retryException = e; } else { throw e } + } catch (e: OpenSearchRejectedExecutionException) { + if(index < numberOfRetries-2) { + retryException = e; + } + else { + throw ReplicationException(e, RestStatus.TOO_MANY_REQUESTS) + } } + log.warn( + "Encountered a failure while executing in $req. Retrying in ${currentBackoff / 1000} seconds" + + ".", retryException + ) + delay(currentBackoff) + currentBackoff = (currentBackoff * factor).toLong().coerceAtMost(maxTimeOut) + } return suspendExecute(replicationMetadata, action, req, injectSecurityContext = injectSecurityContext, defaultContext = defaultContext) // last attempt } - /** * Restore shard from leader cluster with retries. * Only specified error are retried diff --git a/src/test/kotlin/org/opensearch/replication/task/shard/TranslogSequencerTests.kt b/src/test/kotlin/org/opensearch/replication/task/shard/TranslogSequencerTests.kt index ac377687..fe6ad1c8 100644 --- a/src/test/kotlin/org/opensearch/replication/task/shard/TranslogSequencerTests.kt +++ b/src/test/kotlin/org/opensearch/replication/task/shard/TranslogSequencerTests.kt @@ -94,7 +94,7 @@ class TranslogSequencerTests : OpenSearchTestCase() { Mockito.`when`(indicesService.indexServiceSafe(followerShardId.index)).thenReturn(followerIndexService) Mockito.`when`(followerIndexService.getShard(followerShardId.id)).thenReturn(indexShard) val sequencer = TranslogSequencer(this, replicationMetadata, followerShardId, leaderAlias, leaderIndex, EMPTY_TASK_ID, - client, startSeqNo, stats) + client, startSeqNo, stats, 2) // Send requests out of order (shuffled seqNo) and await for them to be processed. var batchSeqNo = startSeqNo From 66d126e626b5ddbe84d1139655478e14fb9de0be Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Wed, 5 Jul 2023 15:29:03 +0530 Subject: [PATCH 100/157] Avoid use of indicesService in Resume replicaiton flow (#1030) Avoid use of indicesService in Resume replicaiton flow. Signed-off-by: monusingh-1 --- .../action/resume/TransportResumeIndexReplicationAction.kt | 6 +----- .../replication/seqno/RemoteClusterRetentionLeaseHelper.kt | 4 ++-- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/action/resume/TransportResumeIndexReplicationAction.kt b/src/main/kotlin/org/opensearch/replication/action/resume/TransportResumeIndexReplicationAction.kt index c3c36419..97fb017e 100644 --- a/src/main/kotlin/org/opensearch/replication/action/resume/TransportResumeIndexReplicationAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/resume/TransportResumeIndexReplicationAction.kt @@ -54,7 +54,6 @@ import org.opensearch.common.io.stream.StreamInput import org.opensearch.env.Environment import org.opensearch.index.IndexNotFoundException import org.opensearch.index.shard.ShardId -import org.opensearch.replication.util.indicesService import org.opensearch.threadpool.ThreadPool import org.opensearch.transport.TransportService import java.io.IOException @@ -136,10 +135,7 @@ class TransportResumeIndexReplicationAction @Inject constructor(transportService shards?.forEach { val followerShardId = it.value.shardId - val followerIndexService = indicesService.indexServiceSafe(followerShardId.index) - val indexShard = followerIndexService.getShard(followerShardId.id) - - if (!retentionLeaseHelper.verifyRetentionLeaseExist(ShardId(params.leaderIndex, followerShardId.id), followerShardId, indexShard.lastSyncedGlobalCheckpoint+1)) { + if (!retentionLeaseHelper.verifyRetentionLeaseExist(ShardId(params.leaderIndex, followerShardId.id), followerShardId)) { isResumable = false } } diff --git a/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterRetentionLeaseHelper.kt b/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterRetentionLeaseHelper.kt index 80037455..d50540a3 100644 --- a/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterRetentionLeaseHelper.kt +++ b/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterRetentionLeaseHelper.kt @@ -58,7 +58,7 @@ class RemoteClusterRetentionLeaseHelper constructor(var followerClusterNameWithU } } - public suspend fun verifyRetentionLeaseExist(leaderShardId: ShardId, followerShardId: ShardId, seqNo: Long): Boolean { + public suspend fun verifyRetentionLeaseExist(leaderShardId: ShardId, followerShardId: ShardId): Boolean { val retentionLeaseId = retentionLeaseIdForShard(followerClusterNameWithUUID, followerShardId) // Currently there is no API to describe/list the retention leases . // So we are verifying the existence of lease by trying to renew a lease by same name . @@ -74,7 +74,7 @@ class RemoteClusterRetentionLeaseHelper constructor(var followerClusterNameWithU return true } catch (e: RetentionLeaseNotFoundException) { - return addNewRetentionLeaseIfOldExists(leaderShardId, followerShardId, seqNo) + return addNewRetentionLeaseIfOldExists(leaderShardId, followerShardId, RetentionLeaseActions.RETAIN_ALL) }catch (e : Exception) { return false } From ea077a8598c945863875a4681750f3ee16235619 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Mon, 10 Jul 2023 11:08:35 +0530 Subject: [PATCH 101/157] Update TranslogSequencer.kt (#1057) Correct the flow for rateLimiter.acquire() The change makes sure that lock is aquired only when a change is going to be processed Signed-off-by: Monu Singh --- .../opensearch/replication/task/shard/TranslogSequencer.kt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/task/shard/TranslogSequencer.kt b/src/main/kotlin/org/opensearch/replication/task/shard/TranslogSequencer.kt index fc14d0b5..7ccf8c94 100644 --- a/src/main/kotlin/org/opensearch/replication/task/shard/TranslogSequencer.kt +++ b/src/main/kotlin/org/opensearch/replication/task/shard/TranslogSequencer.kt @@ -70,13 +70,13 @@ class TranslogSequencer(scope: CoroutineScope, private val replicationMetadata: // raise the same exception. See [SendChannel.close] method for details. val rateLimiter = Semaphore(writersPerShard) var highWatermark = initialSeqNo - for (m in channel) { - rateLimiter.acquire() + for (m in channel) { while (unAppliedChanges.containsKey(highWatermark + 1)) { val next = unAppliedChanges.remove(highWatermark + 1)!! val replayRequest = ReplayChangesRequest(followerShardId, next.changes, next.maxSeqNoOfUpdatesOrDeletes, leaderAlias, leaderIndexName) replayRequest.parentTask = parentTaskId + rateLimiter.acquire() launch { var relativeStartNanos = System.nanoTime() val retryOnExceptions = ArrayList>() From 96299efbe515571a454c260cb913e34917130d34 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Tue, 11 Jul 2023 17:08:17 +0530 Subject: [PATCH 102/157] Add release notes for 2.9.0 (#1063) Add release notes for 2.9 release Signed-off-by: monusingh-1 --- ...ross-cluster-replication.release-notes-2.9.0.0.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) create mode 100644 release-notes/opensearch-cross-cluster-replication.release-notes-2.9.0.0.md diff --git a/release-notes/opensearch-cross-cluster-replication.release-notes-2.9.0.0.md b/release-notes/opensearch-cross-cluster-replication.release-notes-2.9.0.0.md new file mode 100644 index 00000000..45b2379d --- /dev/null +++ b/release-notes/opensearch-cross-cluster-replication.release-notes-2.9.0.0.md @@ -0,0 +1,12 @@ +## Version 2.9.0.0 Release Notes + +Compatible with OpenSearch 2.9.0 + + +### Bug Fixes +* Handle bug in Shard replication task assignment ([#881](https://github.com/opensearch-project/cross-cluster-replication/pull/881)) +* Update Leader checkpoint when shard replication task is reinitialized ([#904](https://github.com/opensearch-project/cross-cluster-replication/pull/904)) +* Correctly handle retention lease renewal (if the lease already exists) during bootstrap ([#904](https://github.com/opensearch-project/cross-cluster-replication/pull/904)) +* Clear persistent tasks from cluster state after STOP API is triggered ([#905](https://github.com/opensearch-project/cross-cluster-replication/pull/905)) +* Handle OpenSearchRejectExecuteException Exception during replay ([#1004](https://github.com/opensearch-project/cross-cluster-replication/pull/1004)) +* Fix Resume replication flow in dedicated master node configuration ([#1030](https://github.com/opensearch-project/cross-cluster-replication/pull/1030)) \ No newline at end of file From 324b10337ca46c57579186b21f74c9f9d9fd2889 Mon Sep 17 00:00:00 2001 From: sricharanvuppu <113983630+sricharanvuppu@users.noreply.github.com> Date: Wed, 12 Jul 2023 20:29:16 +0530 Subject: [PATCH 103/157] adding missing checkpoint and correcting follower stats test case (#1064) Signed-off-by: sricharanvuppu --- .../task/shard/TranslogSequencer.kt | 1 + .../integ/rest/StartReplicationIT.kt | 18 +++++++++--------- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/task/shard/TranslogSequencer.kt b/src/main/kotlin/org/opensearch/replication/task/shard/TranslogSequencer.kt index 7ccf8c94..b274bdf0 100644 --- a/src/main/kotlin/org/opensearch/replication/task/shard/TranslogSequencer.kt +++ b/src/main/kotlin/org/opensearch/replication/task/shard/TranslogSequencer.kt @@ -113,6 +113,7 @@ class TranslogSequencer(scope: CoroutineScope, private val replicationMetadata: followerClusterStats.stats[followerShardId]!!.opsWritten.addAndGet( replayRequest.changes.size.toLong() ) + followerClusterStats.stats[followerShardId]!!.followerCheckpoint = indexShard.localCheckpoint } catch (e: OpenSearchException) { if (e !is IndexNotFoundException && (retryOnExceptions.contains(e.javaClass) || TransportActions.isShardNotAvailableException(e) diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt index 53493457..d4737c61 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt @@ -886,16 +886,12 @@ class StartReplicationIT: MultiClusterRestTestCase() { }, 60L, TimeUnit.SECONDS) } - @AwaitsFix(bugUrl = "https://github.com/opensearch-project/cross-cluster-replication/issues/176") + fun `test follower stats`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) - val leaderIndexName2 = randomAlphaOfLength(10).toLowerCase(Locale.ROOT)+"leader" val followerIndexName2 = randomAlphaOfLength(10).toLowerCase(Locale.ROOT)+"follower" - val leaderIndexName3 = randomAlphaOfLength(10).toLowerCase(Locale.ROOT)+"leader" val followerIndexName3 = randomAlphaOfLength(10).toLowerCase(Locale.ROOT)+"follower" -// val followerIndex2 = "follower_index_2" -// val followerIndex3 = "follower_index_3" createConnectionBetweenClusters(FOLLOWER, LEADER) val createIndexResponse = leaderClient.indices().create( CreateIndexRequest(leaderIndexName), @@ -908,12 +904,12 @@ class StartReplicationIT: MultiClusterRestTestCase() { true ) followerClient.startReplication( - StartReplicationRequest("source", leaderIndexName2, followerIndexName2), + StartReplicationRequest("source", leaderIndexName, followerIndexName2), TimeValue.timeValueSeconds(10), true ) followerClient.startReplication( - StartReplicationRequest("source", leaderIndexName3, followerIndexName3), + StartReplicationRequest("source", leaderIndexName, followerIndexName3), TimeValue.timeValueSeconds(10), true ) @@ -923,12 +919,16 @@ class StartReplicationIT: MultiClusterRestTestCase() { leaderClient.index(IndexRequest(leaderIndexName).id(i.toString()).source(sourceMap), RequestOptions.DEFAULT) } followerClient.pauseReplication(followerIndexName2) - val stats = followerClient.followerStats() + followerClient.stopReplication(followerIndexName3) + var stats = followerClient.followerStats() assertThat(stats.getValue("num_syncing_indices").toString()).isEqualTo("1") assertThat(stats.getValue("num_paused_indices").toString()).isEqualTo("1") assertThat(stats.getValue("num_failed_indices").toString()).isEqualTo("0") assertThat(stats.getValue("num_shard_tasks").toString()).isEqualTo("1") - assertThat(stats.getValue("operations_written").toString()).isEqualTo("50") + assertBusy({ + stats = followerClient.followerStats() + assertThat(stats.getValue("operations_written").toString()).isEqualTo("50") + }, 60, TimeUnit.SECONDS) assertThat(stats.getValue("operations_read").toString()).isEqualTo("50") assertThat(stats.getValue("failed_read_requests").toString()).isEqualTo("0") assertThat(stats.getValue("failed_write_requests").toString()).isEqualTo("0") From fa7b9c539807e2089f6b8eb23435cd5b9c54c4aa Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Fri, 21 Jul 2023 16:21:43 +0530 Subject: [PATCH 104/157] Move from common to core.common (#1087) Keep up with core changes opensearch-project/OpenSearch#8157 move from common to core.common Change kotlin.version "1.8.21" as Class 'org.opensearch.commons.utils.OpenForTesting' was compiled with "1.8.21" version of Kotlin. Ref Set following to use 1.6 as 1.8.21 is not available yet. Checked that other OpenSearch plugins are also doing the same. Set kotlinx-coroutines-core to 1.6.0 set kotlinx-coroutines-test to 1.6.0 Added else in when statements as from Kotlin 1.7 version onward exhaustive list is mandatory. 'when' expression must be exhaustive, add necessary 'else' branch --------- Signed-off-by: monusingh-1 --- build.gradle | 6 +++--- .../org/opensearch/replication/ReplicationException.kt | 6 +++--- .../org/opensearch/replication/ReplicationPlugin.kt | 4 ++-- .../autofollow/AutoFollowClusterManagerNodeRequest.kt | 4 ++-- .../TransportAutoFollowClusterManagerNodeAction.kt | 2 +- .../action/autofollow/UpdateAutoFollowPatternRequest.kt | 4 ++-- .../replication/action/changes/GetChangesRequest.kt | 6 +++--- .../replication/action/changes/GetChangesResponse.kt | 4 ++-- .../action/changes/TransportGetChangesAction.kt | 6 +++--- .../index/ReplicateIndexClusterManagerNodeRequest.kt | 4 ++-- .../replication/action/index/ReplicateIndexRequest.kt | 4 ++-- .../replication/action/index/ReplicateIndexResponse.kt | 4 ++-- .../TransportReplicateIndexClusterManagerNodeAction.kt | 4 ++-- .../index/block/TransportUpddateIndexBlockAction.kt | 2 +- .../action/index/block/UpdateIndexBlockRequest.kt | 4 ++-- .../action/pause/PauseIndexReplicationRequest.kt | 4 ++-- .../action/pause/TransportPauseIndexReplicationAction.kt | 2 +- .../replication/action/replay/ReplayChangesRequest.kt | 6 +++--- .../replication/action/replay/ReplayChangesResponse.kt | 2 +- .../action/replay/TransportReplayChangesAction.kt | 6 +++--- .../TransportUpdateReplicationStateDetails.kt | 2 +- .../UpdateReplicationStateDetailsRequest.kt | 4 ++-- .../replication/action/repository/GetFileChunkRequest.kt | 6 +++--- .../replication/action/repository/GetFileChunkResponse.kt | 6 +++--- .../action/repository/GetStoreMetadataRequest.kt | 4 ++-- .../action/repository/GetStoreMetadataResponse.kt | 4 ++-- .../action/repository/ReleaseLeaderResourcesRequest.kt | 4 ++-- .../action/repository/RemoteClusterRepositoryRequest.kt | 6 +++--- .../action/repository/TransportGetFileChunkAction.kt | 8 ++++---- .../action/repository/TransportGetStoreMetadataAction.kt | 6 +++--- .../repository/TransportReleaseLeaderResourcesAction.kt | 6 +++--- .../action/resume/ResumeIndexReplicationRequest.kt | 4 ++-- .../resume/TransportResumeIndexReplicationAction.kt | 4 ++-- .../replication/action/setup/SetupChecksRequest.kt | 4 ++-- .../action/setup/TransportSetupChecksAction.kt | 2 +- .../action/setup/ValidatePermissionsRequest.kt | 4 ++-- .../replication/action/stats/AutoFollowStatsAction.kt | 6 +++--- .../replication/action/stats/AutoFollowStatsRequest.kt | 4 ++-- .../replication/action/stats/FollowerNodeStatsResponse.kt | 6 +++--- .../replication/action/stats/FollowerStatsAction.kt | 2 +- .../replication/action/stats/FollowerStatsRequest.kt | 4 ++-- .../replication/action/stats/FollowerStatsResponse.kt | 6 +++--- .../replication/action/stats/LeaderNodeStatsResponse.kt | 6 +++--- .../replication/action/stats/LeaderStatsAction.kt | 2 +- .../replication/action/stats/LeaderStatsRequest.kt | 4 ++-- .../replication/action/stats/LeaderStatsResponse.kt | 4 ++-- .../replication/action/stats/NodeStatsRequest.kt | 4 ++-- .../action/stats/TransportFollowerStatsAction.kt | 2 +- .../action/stats/TransportLeaderStatsAction.kt | 4 ++-- .../replication/action/status/ReplicationStatusAction.kt | 2 +- .../action/status/ReplicationStatusResponse.kt | 6 +++--- .../replication/action/status/ShardInfoRequest.kt | 4 ++-- .../replication/action/status/ShardInfoResponse.kt | 6 +++--- .../replication/action/status/ShardsInfoAction.kt | 2 +- .../replication/action/status/TranportShardsInfoAction.kt | 6 +++--- .../action/stop/StopIndexReplicationRequest.kt | 4 ++-- .../action/stop/TransportStopIndexReplicationAction.kt | 2 +- .../update/TransportUpdateIndexReplicationAction.kt | 2 +- .../action/update/UpdateIndexReplicationRequest.kt | 4 ++-- .../replication/metadata/TransportUpdateMetadataAction.kt | 4 ++-- .../replication/metadata/UpdateIndexBlockTask.kt | 2 +- .../replication/metadata/UpdateMetadataRequest.kt | 4 ++-- .../metadata/state/ReplicationStateMetadata.kt | 4 ++-- .../replication/metadata/store/ReplicationMetadata.kt | 6 +++--- .../repository/RemoteClusterMultiChunkTransfer.kt | 2 +- .../replication/repository/RemoteClusterRepository.kt | 2 +- .../opensearch/replication/rest/AutoFollowStatsHandler.kt | 2 +- .../opensearch/replication/rest/FollowerStatsHandler.kt | 2 +- .../org/opensearch/replication/rest/LeaderStatsHandler.kt | 2 +- .../replication/rest/UpdateAutoFollowPatternsHandler.kt | 2 +- .../seqno/RemoteClusterRetentionLeaseHelper.kt | 2 +- .../opensearch/replication/seqno/RemoteClusterStats.kt | 6 +++--- .../replication/task/CrossClusterReplicationTask.kt | 6 +++--- .../org/opensearch/replication/task/IndexCloseListener.kt | 2 +- .../org/opensearch/replication/task/ReplicationState.kt | 6 +++--- .../replication/task/autofollow/AutoFollowParams.kt | 4 ++-- .../replication/task/autofollow/AutoFollowTask.kt | 6 +++--- .../replication/task/index/IndexReplicationParams.kt | 6 +++--- .../replication/task/index/IndexReplicationState.kt | 6 +++--- .../replication/task/index/IndexReplicationTask.kt | 8 ++++---- .../replication/task/shard/FollowerClusterStats.kt | 6 +++--- .../replication/task/shard/ShardReplicationParams.kt | 8 ++++---- .../replication/task/shard/ShardReplicationState.kt | 4 ++-- .../replication/task/shard/ShardReplicationTask.kt | 4 ++-- .../replication/task/shard/TranslogSequencer.kt | 4 ++-- .../kotlin/org/opensearch/replication/util/Extensions.kt | 4 ++-- .../translog/ReplicationTranslogDeletionPolicyTests.kt | 4 ++-- .../replication/bwc/BackwardsCompatibilityIT.kt | 1 + .../replication/singleCluster/SingleClusterSanityIT.kt | 1 + .../replication/task/index/IndexReplicationTaskTests.kt | 4 ++-- .../org/opensearch/replication/task/index/NoOpClient.kt | 6 +++--- .../task/shard/ShardReplicationExecutorTests.kt | 4 ++-- .../replication/task/shard/TranslogSequencerTests.kt | 2 +- 93 files changed, 194 insertions(+), 192 deletions(-) diff --git a/build.gradle b/build.gradle index cf3c6bbf..426993aa 100644 --- a/build.gradle +++ b/build.gradle @@ -53,7 +53,7 @@ buildscript { plugin_previous_version = opensearch_previous_version.replaceAll(/(\.\d)([^\d]*)$/, '$1.0$2') common_utils_version = System.getProperty("common_utils.version", opensearch_build) - kotlin_version = System.getProperty("kotlin.version", "1.6.0") + kotlin_version = System.getProperty("kotlin.version", "1.8.21") security_plugin_version = opensearch_build if (!isSnapshot) { @@ -139,7 +139,7 @@ dependencies { implementation "org.jetbrains.kotlin:kotlin-stdlib-common:${kotlin_version}" implementation "org.jetbrains:annotations:13.0" implementation "com.github.seancfoley:ipaddress:5.3.3" - implementation "org.jetbrains.kotlinx:kotlinx-coroutines-core:${kotlin_version}" + implementation "org.jetbrains.kotlinx:kotlinx-coroutines-core:1.6.0" // Moving away from kotlin_version implementation "org.opensearch:common-utils:${common_utils_version}" implementation "org.apache.httpcomponents.client5:httpclient5:5.1.3" implementation "org.apache.httpcomponents.core5:httpcore5:5.1.4" @@ -148,7 +148,7 @@ dependencies { testImplementation "org.opensearch.test:framework:${opensearch_version}" testImplementation "org.assertj:assertj-core:3.17.2" testImplementation "org.opensearch.client:opensearch-rest-high-level-client:${opensearch_version}" - testImplementation "org.jetbrains.kotlinx:kotlinx-coroutines-test:${kotlin_version}" + testImplementation "org.jetbrains.kotlinx:kotlinx-coroutines-test:1.6.0" // Moving away from kotlin_version testImplementation "org.jetbrains.kotlin:kotlin-test:${kotlin_version}" testImplementation "com.nhaarman.mockitokotlin2:mockito-kotlin:2.2.0" diff --git a/src/main/kotlin/org/opensearch/replication/ReplicationException.kt b/src/main/kotlin/org/opensearch/replication/ReplicationException.kt index 891be0a3..83e59220 100644 --- a/src/main/kotlin/org/opensearch/replication/ReplicationException.kt +++ b/src/main/kotlin/org/opensearch/replication/ReplicationException.kt @@ -13,10 +13,10 @@ package org.opensearch.replication import org.opensearch.OpenSearchException import org.opensearch.OpenSearchStatusException -import org.opensearch.action.ShardOperationFailedException +import org.opensearch.core.action.ShardOperationFailedException import org.opensearch.cluster.metadata.IndexMetadata.INDEX_UUID_NA_VALUE -import org.opensearch.index.shard.ShardId -import org.opensearch.rest.RestStatus +import org.opensearch.core.index.shard.ShardId +import org.opensearch.core.rest.RestStatus /** * Base class replication exceptions. Note: Replication process may throw exceptions that do not derive from this such as diff --git a/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt b/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt index 4254412c..7d66c976 100644 --- a/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt +++ b/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt @@ -84,8 +84,8 @@ import org.opensearch.cluster.service.ClusterService import org.opensearch.common.CheckedFunction import org.opensearch.core.ParseField import org.opensearch.common.component.LifecycleComponent -import org.opensearch.common.io.stream.NamedWriteableRegistry -import org.opensearch.common.io.stream.Writeable +import org.opensearch.core.common.io.stream.NamedWriteableRegistry +import org.opensearch.core.common.io.stream.Writeable import org.opensearch.common.settings.ClusterSettings import org.opensearch.common.settings.IndexScopedSettings import org.opensearch.common.settings.Setting diff --git a/src/main/kotlin/org/opensearch/replication/action/autofollow/AutoFollowClusterManagerNodeRequest.kt b/src/main/kotlin/org/opensearch/replication/action/autofollow/AutoFollowClusterManagerNodeRequest.kt index f9199ea0..216809d8 100644 --- a/src/main/kotlin/org/opensearch/replication/action/autofollow/AutoFollowClusterManagerNodeRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/autofollow/AutoFollowClusterManagerNodeRequest.kt @@ -14,8 +14,8 @@ package org.opensearch.replication.action.autofollow import org.opensearch.commons.authuser.User import org.opensearch.action.ActionRequestValidationException import org.opensearch.action.support.master.MasterNodeRequest -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.ToXContent import org.opensearch.core.xcontent.ToXContentObject import org.opensearch.core.xcontent.XContentBuilder diff --git a/src/main/kotlin/org/opensearch/replication/action/autofollow/TransportAutoFollowClusterManagerNodeAction.kt b/src/main/kotlin/org/opensearch/replication/action/autofollow/TransportAutoFollowClusterManagerNodeAction.kt index e3c59444..cc93d88a 100644 --- a/src/main/kotlin/org/opensearch/replication/action/autofollow/TransportAutoFollowClusterManagerNodeAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/autofollow/TransportAutoFollowClusterManagerNodeAction.kt @@ -39,7 +39,7 @@ import org.opensearch.cluster.block.ClusterBlockLevel import org.opensearch.cluster.metadata.IndexNameExpressionResolver import org.opensearch.cluster.service.ClusterService import org.opensearch.common.inject.Inject -import org.opensearch.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamInput import org.opensearch.common.settings.IndexScopedSettings import org.opensearch.replication.ReplicationException import org.opensearch.threadpool.ThreadPool diff --git a/src/main/kotlin/org/opensearch/replication/action/autofollow/UpdateAutoFollowPatternRequest.kt b/src/main/kotlin/org/opensearch/replication/action/autofollow/UpdateAutoFollowPatternRequest.kt index a7077793..165ede8b 100644 --- a/src/main/kotlin/org/opensearch/replication/action/autofollow/UpdateAutoFollowPatternRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/autofollow/UpdateAutoFollowPatternRequest.kt @@ -17,8 +17,8 @@ import org.opensearch.replication.util.ValidationUtil.validateName import org.opensearch.action.ActionRequestValidationException import org.opensearch.action.support.master.AcknowledgedRequest import org.opensearch.core.ParseField -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.common.settings.Settings import org.opensearch.core.xcontent.ObjectParser import org.opensearch.core.xcontent.ToXContent diff --git a/src/main/kotlin/org/opensearch/replication/action/changes/GetChangesRequest.kt b/src/main/kotlin/org/opensearch/replication/action/changes/GetChangesRequest.kt index d3aeb8e7..43e1fa33 100644 --- a/src/main/kotlin/org/opensearch/replication/action/changes/GetChangesRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/changes/GetChangesRequest.kt @@ -14,9 +14,9 @@ package org.opensearch.replication.action.changes import org.opensearch.action.ActionRequestValidationException import org.opensearch.action.support.single.shard.SingleShardRequest import org.opensearch.cluster.node.DiscoveryNode -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.index.shard.ShardId +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput +import org.opensearch.core.index.shard.ShardId import org.opensearch.transport.RemoteClusterAwareRequest class GetChangesRequest : SingleShardRequest { diff --git a/src/main/kotlin/org/opensearch/replication/action/changes/GetChangesResponse.kt b/src/main/kotlin/org/opensearch/replication/action/changes/GetChangesResponse.kt index f3adaf2e..c71b4795 100644 --- a/src/main/kotlin/org/opensearch/replication/action/changes/GetChangesResponse.kt +++ b/src/main/kotlin/org/opensearch/replication/action/changes/GetChangesResponse.kt @@ -12,8 +12,8 @@ package org.opensearch.replication.action.changes import org.opensearch.action.ActionResponse -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.index.translog.Translog class GetChangesResponse(val changes: List, diff --git a/src/main/kotlin/org/opensearch/replication/action/changes/TransportGetChangesAction.kt b/src/main/kotlin/org/opensearch/replication/action/changes/TransportGetChangesAction.kt index c483ebad..a3995999 100644 --- a/src/main/kotlin/org/opensearch/replication/action/changes/TransportGetChangesAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/changes/TransportGetChangesAction.kt @@ -23,10 +23,10 @@ import org.opensearch.cluster.metadata.IndexNameExpressionResolver import org.opensearch.cluster.routing.ShardsIterator import org.opensearch.cluster.service.ClusterService import org.opensearch.common.inject.Inject -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.Writeable +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.Writeable import org.opensearch.common.unit.TimeValue -import org.opensearch.index.shard.ShardId +import org.opensearch.core.index.shard.ShardId import org.opensearch.index.translog.Translog import org.opensearch.indices.IndicesService import org.opensearch.replication.ReplicationPlugin.Companion.REPLICATION_INDEX_TRANSLOG_PRUNING_ENABLED_SETTING diff --git a/src/main/kotlin/org/opensearch/replication/action/index/ReplicateIndexClusterManagerNodeRequest.kt b/src/main/kotlin/org/opensearch/replication/action/index/ReplicateIndexClusterManagerNodeRequest.kt index 63f77023..0b741d4e 100644 --- a/src/main/kotlin/org/opensearch/replication/action/index/ReplicateIndexClusterManagerNodeRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/index/ReplicateIndexClusterManagerNodeRequest.kt @@ -14,8 +14,8 @@ package org.opensearch.replication.action.index import org.opensearch.commons.authuser.User import org.opensearch.action.ActionRequestValidationException import org.opensearch.action.support.master.MasterNodeRequest -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.ToXContent import org.opensearch.core.xcontent.ToXContentObject import org.opensearch.core.xcontent.XContentBuilder diff --git a/src/main/kotlin/org/opensearch/replication/action/index/ReplicateIndexRequest.kt b/src/main/kotlin/org/opensearch/replication/action/index/ReplicateIndexRequest.kt index bb0f9aaf..6024798a 100644 --- a/src/main/kotlin/org/opensearch/replication/action/index/ReplicateIndexRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/index/ReplicateIndexRequest.kt @@ -18,8 +18,8 @@ import org.opensearch.action.IndicesRequest import org.opensearch.action.support.IndicesOptions import org.opensearch.action.support.master.AcknowledgedRequest import org.opensearch.core.ParseField -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.common.settings.Settings import org.opensearch.core.xcontent.ObjectParser import org.opensearch.core.xcontent.ToXContent diff --git a/src/main/kotlin/org/opensearch/replication/action/index/ReplicateIndexResponse.kt b/src/main/kotlin/org/opensearch/replication/action/index/ReplicateIndexResponse.kt index 6ab3cc75..31963036 100644 --- a/src/main/kotlin/org/opensearch/replication/action/index/ReplicateIndexResponse.kt +++ b/src/main/kotlin/org/opensearch/replication/action/index/ReplicateIndexResponse.kt @@ -12,8 +12,8 @@ package org.opensearch.replication.action.index import org.opensearch.action.support.master.AcknowledgedResponse -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput class ReplicateIndexResponse(val ack: Boolean) : AcknowledgedResponse(ack) { constructor(inp: StreamInput) : this(inp.readBoolean()) diff --git a/src/main/kotlin/org/opensearch/replication/action/index/TransportReplicateIndexClusterManagerNodeAction.kt b/src/main/kotlin/org/opensearch/replication/action/index/TransportReplicateIndexClusterManagerNodeAction.kt index 1a926798..36fe81f2 100644 --- a/src/main/kotlin/org/opensearch/replication/action/index/TransportReplicateIndexClusterManagerNodeAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/index/TransportReplicateIndexClusterManagerNodeAction.kt @@ -40,14 +40,14 @@ import org.opensearch.cluster.metadata.IndexMetadata import org.opensearch.cluster.metadata.IndexNameExpressionResolver import org.opensearch.cluster.service.ClusterService import org.opensearch.common.inject.Inject -import org.opensearch.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamInput import org.opensearch.common.settings.IndexScopedSettings import org.opensearch.index.IndexNotFoundException import org.opensearch.persistent.PersistentTasksService import org.opensearch.replication.ReplicationPlugin import org.opensearch.replication.util.stackTraceToString import org.opensearch.repositories.RepositoriesService -import org.opensearch.rest.RestStatus +import org.opensearch.core.rest.RestStatus import org.opensearch.threadpool.ThreadPool import org.opensearch.transport.TransportService import java.io.IOException diff --git a/src/main/kotlin/org/opensearch/replication/action/index/block/TransportUpddateIndexBlockAction.kt b/src/main/kotlin/org/opensearch/replication/action/index/block/TransportUpddateIndexBlockAction.kt index 189e7acc..55e569aa 100644 --- a/src/main/kotlin/org/opensearch/replication/action/index/block/TransportUpddateIndexBlockAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/index/block/TransportUpddateIndexBlockAction.kt @@ -31,7 +31,7 @@ import org.opensearch.cluster.block.ClusterBlockLevel import org.opensearch.cluster.metadata.IndexNameExpressionResolver import org.opensearch.cluster.service.ClusterService import org.opensearch.common.inject.Inject -import org.opensearch.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamInput import org.opensearch.threadpool.ThreadPool import org.opensearch.transport.TransportService import java.io.IOException diff --git a/src/main/kotlin/org/opensearch/replication/action/index/block/UpdateIndexBlockRequest.kt b/src/main/kotlin/org/opensearch/replication/action/index/block/UpdateIndexBlockRequest.kt index d9b51933..4c766468 100644 --- a/src/main/kotlin/org/opensearch/replication/action/index/block/UpdateIndexBlockRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/index/block/UpdateIndexBlockRequest.kt @@ -16,8 +16,8 @@ import org.opensearch.action.IndicesRequest import org.opensearch.action.support.IndicesOptions import org.opensearch.action.support.master.AcknowledgedRequest import org.opensearch.core.ParseField -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.ObjectParser import org.opensearch.core.xcontent.ToXContent import org.opensearch.core.xcontent.ToXContentObject diff --git a/src/main/kotlin/org/opensearch/replication/action/pause/PauseIndexReplicationRequest.kt b/src/main/kotlin/org/opensearch/replication/action/pause/PauseIndexReplicationRequest.kt index e69dcb44..fce15d2e 100644 --- a/src/main/kotlin/org/opensearch/replication/action/pause/PauseIndexReplicationRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/pause/PauseIndexReplicationRequest.kt @@ -17,8 +17,8 @@ import org.opensearch.action.IndicesRequest import org.opensearch.action.support.IndicesOptions import org.opensearch.action.support.master.AcknowledgedRequest import org.opensearch.core.ParseField -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.ObjectParser import org.opensearch.core.xcontent.ToXContent import org.opensearch.core.xcontent.ToXContentObject diff --git a/src/main/kotlin/org/opensearch/replication/action/pause/TransportPauseIndexReplicationAction.kt b/src/main/kotlin/org/opensearch/replication/action/pause/TransportPauseIndexReplicationAction.kt index abd54403..eaf828a1 100644 --- a/src/main/kotlin/org/opensearch/replication/action/pause/TransportPauseIndexReplicationAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/pause/TransportPauseIndexReplicationAction.kt @@ -37,7 +37,7 @@ import org.opensearch.cluster.block.ClusterBlockLevel import org.opensearch.cluster.metadata.IndexNameExpressionResolver import org.opensearch.cluster.service.ClusterService import org.opensearch.common.inject.Inject -import org.opensearch.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamInput import org.opensearch.threadpool.ThreadPool import org.opensearch.transport.TransportService import java.io.IOException diff --git a/src/main/kotlin/org/opensearch/replication/action/replay/ReplayChangesRequest.kt b/src/main/kotlin/org/opensearch/replication/action/replay/ReplayChangesRequest.kt index 7ac35526..a7366b1a 100644 --- a/src/main/kotlin/org/opensearch/replication/action/replay/ReplayChangesRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/replay/ReplayChangesRequest.kt @@ -12,9 +12,9 @@ package org.opensearch.replication.action.replay import org.opensearch.action.support.replication.ReplicatedWriteRequest -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.index.shard.ShardId +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput +import org.opensearch.core.index.shard.ShardId import org.opensearch.index.translog.Translog class ReplayChangesRequest : ReplicatedWriteRequest { diff --git a/src/main/kotlin/org/opensearch/replication/action/replay/ReplayChangesResponse.kt b/src/main/kotlin/org/opensearch/replication/action/replay/ReplayChangesResponse.kt index b0bab6fb..5fe155f4 100644 --- a/src/main/kotlin/org/opensearch/replication/action/replay/ReplayChangesResponse.kt +++ b/src/main/kotlin/org/opensearch/replication/action/replay/ReplayChangesResponse.kt @@ -13,7 +13,7 @@ package org.opensearch.replication.action.replay import org.opensearch.action.support.WriteResponse import org.opensearch.action.support.replication.ReplicationResponse -import org.opensearch.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamInput class ReplayChangesResponse : ReplicationResponse, WriteResponse { diff --git a/src/main/kotlin/org/opensearch/replication/action/replay/TransportReplayChangesAction.kt b/src/main/kotlin/org/opensearch/replication/action/replay/TransportReplayChangesAction.kt index 9b3a78b6..c874e92f 100644 --- a/src/main/kotlin/org/opensearch/replication/action/replay/TransportReplayChangesAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/replay/TransportReplayChangesAction.kt @@ -39,10 +39,10 @@ import org.opensearch.cluster.action.index.MappingUpdatedAction import org.opensearch.cluster.action.shard.ShardStateAction import org.opensearch.cluster.block.ClusterBlockLevel import org.opensearch.cluster.service.ClusterService -import org.opensearch.common.bytes.BytesReference +import org.opensearch.core.common.bytes.BytesReference import org.opensearch.common.inject.Inject -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.Writeable +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.Writeable import org.opensearch.common.settings.Settings import org.opensearch.common.xcontent.XContentType import org.opensearch.index.IndexingPressureService diff --git a/src/main/kotlin/org/opensearch/replication/action/replicationstatedetails/TransportUpdateReplicationStateDetails.kt b/src/main/kotlin/org/opensearch/replication/action/replicationstatedetails/TransportUpdateReplicationStateDetails.kt index ada0aefa..0b744482 100644 --- a/src/main/kotlin/org/opensearch/replication/action/replicationstatedetails/TransportUpdateReplicationStateDetails.kt +++ b/src/main/kotlin/org/opensearch/replication/action/replicationstatedetails/TransportUpdateReplicationStateDetails.kt @@ -30,7 +30,7 @@ import org.opensearch.cluster.block.ClusterBlockLevel import org.opensearch.cluster.metadata.IndexNameExpressionResolver import org.opensearch.cluster.service.ClusterService import org.opensearch.common.inject.Inject -import org.opensearch.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamInput import org.opensearch.threadpool.ThreadPool import org.opensearch.transport.TransportService diff --git a/src/main/kotlin/org/opensearch/replication/action/replicationstatedetails/UpdateReplicationStateDetailsRequest.kt b/src/main/kotlin/org/opensearch/replication/action/replicationstatedetails/UpdateReplicationStateDetailsRequest.kt index 04fedd0a..1b139495 100644 --- a/src/main/kotlin/org/opensearch/replication/action/replicationstatedetails/UpdateReplicationStateDetailsRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/replicationstatedetails/UpdateReplicationStateDetailsRequest.kt @@ -14,8 +14,8 @@ package org.opensearch.replication.action.replicationstatedetails import org.opensearch.replication.metadata.state.ReplicationStateParams import org.opensearch.action.ActionRequestValidationException import org.opensearch.action.support.master.AcknowledgedRequest -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput class UpdateReplicationStateDetailsRequest: AcknowledgedRequest { diff --git a/src/main/kotlin/org/opensearch/replication/action/repository/GetFileChunkRequest.kt b/src/main/kotlin/org/opensearch/replication/action/repository/GetFileChunkRequest.kt index 8f9213b1..d959866e 100644 --- a/src/main/kotlin/org/opensearch/replication/action/repository/GetFileChunkRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/repository/GetFileChunkRequest.kt @@ -13,9 +13,9 @@ package org.opensearch.replication.action.repository import org.opensearch.action.ActionRequestValidationException import org.opensearch.cluster.node.DiscoveryNode -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.index.shard.ShardId +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput +import org.opensearch.core.index.shard.ShardId import org.opensearch.index.store.StoreFileMetadata class GetFileChunkRequest : RemoteClusterRepositoryRequest { diff --git a/src/main/kotlin/org/opensearch/replication/action/repository/GetFileChunkResponse.kt b/src/main/kotlin/org/opensearch/replication/action/repository/GetFileChunkResponse.kt index 5e292138..325e20f0 100644 --- a/src/main/kotlin/org/opensearch/replication/action/repository/GetFileChunkResponse.kt +++ b/src/main/kotlin/org/opensearch/replication/action/repository/GetFileChunkResponse.kt @@ -12,9 +12,9 @@ package org.opensearch.replication.action.repository import org.opensearch.action.ActionResponse -import org.opensearch.common.bytes.BytesReference -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.bytes.BytesReference +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.index.store.StoreFileMetadata class GetFileChunkResponse : ActionResponse { diff --git a/src/main/kotlin/org/opensearch/replication/action/repository/GetStoreMetadataRequest.kt b/src/main/kotlin/org/opensearch/replication/action/repository/GetStoreMetadataRequest.kt index f8940730..d26e37e1 100644 --- a/src/main/kotlin/org/opensearch/replication/action/repository/GetStoreMetadataRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/repository/GetStoreMetadataRequest.kt @@ -13,8 +13,8 @@ package org.opensearch.replication.action.repository import org.opensearch.action.ActionRequestValidationException import org.opensearch.cluster.node.DiscoveryNode -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.index.shard.ShardId +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.index.shard.ShardId class GetStoreMetadataRequest : RemoteClusterRepositoryRequest { diff --git a/src/main/kotlin/org/opensearch/replication/action/repository/GetStoreMetadataResponse.kt b/src/main/kotlin/org/opensearch/replication/action/repository/GetStoreMetadataResponse.kt index baeb62b5..9ef97259 100644 --- a/src/main/kotlin/org/opensearch/replication/action/repository/GetStoreMetadataResponse.kt +++ b/src/main/kotlin/org/opensearch/replication/action/repository/GetStoreMetadataResponse.kt @@ -12,8 +12,8 @@ package org.opensearch.replication.action.repository import org.opensearch.action.ActionResponse -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.index.store.Store class GetStoreMetadataResponse : ActionResponse { diff --git a/src/main/kotlin/org/opensearch/replication/action/repository/ReleaseLeaderResourcesRequest.kt b/src/main/kotlin/org/opensearch/replication/action/repository/ReleaseLeaderResourcesRequest.kt index 2ea55a01..97908e9f 100644 --- a/src/main/kotlin/org/opensearch/replication/action/repository/ReleaseLeaderResourcesRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/repository/ReleaseLeaderResourcesRequest.kt @@ -13,8 +13,8 @@ package org.opensearch.replication.action.repository import org.opensearch.action.ActionRequestValidationException import org.opensearch.cluster.node.DiscoveryNode -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.index.shard.ShardId +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.index.shard.ShardId class ReleaseLeaderResourcesRequest: RemoteClusterRepositoryRequest { diff --git a/src/main/kotlin/org/opensearch/replication/action/repository/RemoteClusterRepositoryRequest.kt b/src/main/kotlin/org/opensearch/replication/action/repository/RemoteClusterRepositoryRequest.kt index 8a600a4e..8dd702d9 100644 --- a/src/main/kotlin/org/opensearch/replication/action/repository/RemoteClusterRepositoryRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/repository/RemoteClusterRepositoryRequest.kt @@ -13,9 +13,9 @@ package org.opensearch.replication.action.repository import org.opensearch.action.support.single.shard.SingleShardRequest import org.opensearch.cluster.node.DiscoveryNode -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.index.shard.ShardId +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput +import org.opensearch.core.index.shard.ShardId import org.opensearch.transport.RemoteClusterAwareRequest abstract class RemoteClusterRepositoryRequest?>: diff --git a/src/main/kotlin/org/opensearch/replication/action/repository/TransportGetFileChunkAction.kt b/src/main/kotlin/org/opensearch/replication/action/repository/TransportGetFileChunkAction.kt index 2564a0e3..9b8b13e1 100644 --- a/src/main/kotlin/org/opensearch/replication/action/repository/TransportGetFileChunkAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/repository/TransportGetFileChunkAction.kt @@ -20,11 +20,11 @@ import org.opensearch.cluster.ClusterState import org.opensearch.cluster.metadata.IndexNameExpressionResolver import org.opensearch.cluster.routing.ShardsIterator import org.opensearch.cluster.service.ClusterService -import org.opensearch.common.bytes.BytesArray +import org.opensearch.core.common.bytes.BytesArray import org.opensearch.common.inject.Inject -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.Writeable -import org.opensearch.index.shard.ShardId +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.Writeable +import org.opensearch.core.index.shard.ShardId import org.opensearch.indices.IndicesService import org.opensearch.threadpool.ThreadPool import org.opensearch.transport.TransportActionProxy diff --git a/src/main/kotlin/org/opensearch/replication/action/repository/TransportGetStoreMetadataAction.kt b/src/main/kotlin/org/opensearch/replication/action/repository/TransportGetStoreMetadataAction.kt index e3c677d8..44ce8570 100644 --- a/src/main/kotlin/org/opensearch/replication/action/repository/TransportGetStoreMetadataAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/repository/TransportGetStoreMetadataAction.kt @@ -20,9 +20,9 @@ import org.opensearch.cluster.metadata.IndexNameExpressionResolver import org.opensearch.cluster.routing.ShardsIterator import org.opensearch.cluster.service.ClusterService import org.opensearch.common.inject.Inject -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.Writeable -import org.opensearch.index.shard.ShardId +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.Writeable +import org.opensearch.core.index.shard.ShardId import org.opensearch.threadpool.ThreadPool import org.opensearch.transport.TransportActionProxy import org.opensearch.transport.TransportService diff --git a/src/main/kotlin/org/opensearch/replication/action/repository/TransportReleaseLeaderResourcesAction.kt b/src/main/kotlin/org/opensearch/replication/action/repository/TransportReleaseLeaderResourcesAction.kt index 9818ebcc..62a615b3 100644 --- a/src/main/kotlin/org/opensearch/replication/action/repository/TransportReleaseLeaderResourcesAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/repository/TransportReleaseLeaderResourcesAction.kt @@ -21,9 +21,9 @@ import org.opensearch.cluster.metadata.IndexNameExpressionResolver import org.opensearch.cluster.routing.ShardsIterator import org.opensearch.cluster.service.ClusterService import org.opensearch.common.inject.Inject -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.Writeable -import org.opensearch.index.shard.ShardId +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.Writeable +import org.opensearch.core.index.shard.ShardId import org.opensearch.threadpool.ThreadPool import org.opensearch.transport.TransportActionProxy import org.opensearch.transport.TransportService diff --git a/src/main/kotlin/org/opensearch/replication/action/resume/ResumeIndexReplicationRequest.kt b/src/main/kotlin/org/opensearch/replication/action/resume/ResumeIndexReplicationRequest.kt index dd219272..10220a7f 100644 --- a/src/main/kotlin/org/opensearch/replication/action/resume/ResumeIndexReplicationRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/resume/ResumeIndexReplicationRequest.kt @@ -15,8 +15,8 @@ import org.opensearch.action.ActionRequestValidationException import org.opensearch.action.IndicesRequest import org.opensearch.action.support.IndicesOptions import org.opensearch.action.support.master.AcknowledgedRequest -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.* class ResumeIndexReplicationRequest : AcknowledgedRequest, IndicesRequest.Replaceable, ToXContentObject { diff --git a/src/main/kotlin/org/opensearch/replication/action/resume/TransportResumeIndexReplicationAction.kt b/src/main/kotlin/org/opensearch/replication/action/resume/TransportResumeIndexReplicationAction.kt index 97fb017e..a1128913 100644 --- a/src/main/kotlin/org/opensearch/replication/action/resume/TransportResumeIndexReplicationAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/resume/TransportResumeIndexReplicationAction.kt @@ -50,10 +50,10 @@ import org.opensearch.cluster.metadata.IndexNameExpressionResolver import org.opensearch.cluster.service.ClusterService import org.opensearch.common.inject.Inject -import org.opensearch.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamInput import org.opensearch.env.Environment import org.opensearch.index.IndexNotFoundException -import org.opensearch.index.shard.ShardId +import org.opensearch.core.index.shard.ShardId import org.opensearch.threadpool.ThreadPool import org.opensearch.transport.TransportService import java.io.IOException diff --git a/src/main/kotlin/org/opensearch/replication/action/setup/SetupChecksRequest.kt b/src/main/kotlin/org/opensearch/replication/action/setup/SetupChecksRequest.kt index 57b9fffb..44e53460 100644 --- a/src/main/kotlin/org/opensearch/replication/action/setup/SetupChecksRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/setup/SetupChecksRequest.kt @@ -14,8 +14,8 @@ package org.opensearch.replication.action.setup import org.opensearch.replication.metadata.store.ReplicationContext import org.opensearch.action.ActionRequestValidationException import org.opensearch.action.support.master.AcknowledgedRequest -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.ToXContent import org.opensearch.core.xcontent.ToXContentObject import org.opensearch.core.xcontent.XContentBuilder diff --git a/src/main/kotlin/org/opensearch/replication/action/setup/TransportSetupChecksAction.kt b/src/main/kotlin/org/opensearch/replication/action/setup/TransportSetupChecksAction.kt index 82315e37..0b1169b6 100644 --- a/src/main/kotlin/org/opensearch/replication/action/setup/TransportSetupChecksAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/setup/TransportSetupChecksAction.kt @@ -26,7 +26,7 @@ import org.opensearch.cluster.service.ClusterService import org.opensearch.common.inject.Inject import org.opensearch.common.util.concurrent.ThreadContext import org.opensearch.replication.util.stackTraceToString -import org.opensearch.rest.RestStatus +import org.opensearch.core.rest.RestStatus import org.opensearch.tasks.Task import org.opensearch.threadpool.ThreadPool import org.opensearch.transport.ActionNotFoundTransportException diff --git a/src/main/kotlin/org/opensearch/replication/action/setup/ValidatePermissionsRequest.kt b/src/main/kotlin/org/opensearch/replication/action/setup/ValidatePermissionsRequest.kt index 430fe9b8..7808fd85 100644 --- a/src/main/kotlin/org/opensearch/replication/action/setup/ValidatePermissionsRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/setup/ValidatePermissionsRequest.kt @@ -15,8 +15,8 @@ import org.opensearch.action.ActionRequestValidationException import org.opensearch.action.IndicesRequest import org.opensearch.action.support.IndicesOptions import org.opensearch.action.support.master.AcknowledgedRequest -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.ToXContent import org.opensearch.core.xcontent.ToXContentObject import org.opensearch.core.xcontent.XContentBuilder diff --git a/src/main/kotlin/org/opensearch/replication/action/stats/AutoFollowStatsAction.kt b/src/main/kotlin/org/opensearch/replication/action/stats/AutoFollowStatsAction.kt index 0878d377..3c7b60fe 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stats/AutoFollowStatsAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stats/AutoFollowStatsAction.kt @@ -15,9 +15,9 @@ import org.opensearch.action.ActionType import org.opensearch.action.FailedNodeException import org.opensearch.action.TaskOperationFailure import org.opensearch.action.support.tasks.BaseTasksResponse -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.common.io.stream.Writeable +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.Writeable import org.opensearch.core.xcontent.ToXContent import org.opensearch.core.xcontent.ToXContent.EMPTY_PARAMS import org.opensearch.core.xcontent.ToXContentObject diff --git a/src/main/kotlin/org/opensearch/replication/action/stats/AutoFollowStatsRequest.kt b/src/main/kotlin/org/opensearch/replication/action/stats/AutoFollowStatsRequest.kt index 93102cf5..7bce9a0d 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stats/AutoFollowStatsRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stats/AutoFollowStatsRequest.kt @@ -13,8 +13,8 @@ package org.opensearch.replication.action.stats import org.opensearch.action.ActionRequestValidationException import org.opensearch.action.support.tasks.BaseTasksRequest -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.replication.task.autofollow.AutoFollowTask import org.opensearch.tasks.Task import java.io.IOException diff --git a/src/main/kotlin/org/opensearch/replication/action/stats/FollowerNodeStatsResponse.kt b/src/main/kotlin/org/opensearch/replication/action/stats/FollowerNodeStatsResponse.kt index cdff07ab..a20ae942 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stats/FollowerNodeStatsResponse.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stats/FollowerNodeStatsResponse.kt @@ -13,9 +13,9 @@ package org.opensearch.replication.action.stats import org.opensearch.action.support.nodes.BaseNodeResponse import org.opensearch.cluster.node.DiscoveryNode -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.index.shard.ShardId +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput +import org.opensearch.core.index.shard.ShardId import org.opensearch.replication.task.shard.FollowerShardMetric import org.opensearch.replication.task.shard.FollowerShardMetric.FollowerStats import java.io.IOException diff --git a/src/main/kotlin/org/opensearch/replication/action/stats/FollowerStatsAction.kt b/src/main/kotlin/org/opensearch/replication/action/stats/FollowerStatsAction.kt index 8ecafb98..9d07bea1 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stats/FollowerStatsAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stats/FollowerStatsAction.kt @@ -12,7 +12,7 @@ package org.opensearch.replication.action.stats import org.opensearch.action.ActionType -import org.opensearch.common.io.stream.Writeable +import org.opensearch.core.common.io.stream.Writeable class FollowerStatsAction : ActionType(NAME, reader) { companion object { diff --git a/src/main/kotlin/org/opensearch/replication/action/stats/FollowerStatsRequest.kt b/src/main/kotlin/org/opensearch/replication/action/stats/FollowerStatsRequest.kt index e79e4d2e..9dfdf8f5 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stats/FollowerStatsRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stats/FollowerStatsRequest.kt @@ -12,8 +12,8 @@ package org.opensearch.replication.action.stats import org.opensearch.action.support.nodes.BaseNodesRequest -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import java.io.IOException /** diff --git a/src/main/kotlin/org/opensearch/replication/action/stats/FollowerStatsResponse.kt b/src/main/kotlin/org/opensearch/replication/action/stats/FollowerStatsResponse.kt index 53271fe4..9d4ae15c 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stats/FollowerStatsResponse.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stats/FollowerStatsResponse.kt @@ -17,14 +17,14 @@ import org.opensearch.action.FailedNodeException import org.opensearch.action.support.nodes.BaseNodesResponse import org.opensearch.cluster.ClusterName import org.opensearch.common.Strings -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.ToXContent.EMPTY_PARAMS import org.opensearch.core.xcontent.ToXContent.Params import org.opensearch.core.xcontent.ToXContentObject import org.opensearch.core.xcontent.XContentBuilder import org.opensearch.common.xcontent.XContentFactory -import org.opensearch.index.shard.ShardId +import org.opensearch.core.index.shard.ShardId import org.opensearch.replication.metadata.ReplicationOverallState import org.opensearch.replication.metadata.state.REPLICATION_LAST_KNOWN_OVERALL_STATE import org.opensearch.replication.metadata.state.ReplicationStateMetadata diff --git a/src/main/kotlin/org/opensearch/replication/action/stats/LeaderNodeStatsResponse.kt b/src/main/kotlin/org/opensearch/replication/action/stats/LeaderNodeStatsResponse.kt index a326772b..cdc4b4d8 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stats/LeaderNodeStatsResponse.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stats/LeaderNodeStatsResponse.kt @@ -13,9 +13,9 @@ package org.opensearch.replication.action.stats import org.opensearch.action.support.nodes.BaseNodeResponse import org.opensearch.cluster.node.DiscoveryNode -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.index.shard.ShardId +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput +import org.opensearch.core.index.shard.ShardId import org.opensearch.replication.seqno.RemoteShardMetric import org.opensearch.replication.seqno.RemoteShardMetric.RemoteStats import java.io.IOException diff --git a/src/main/kotlin/org/opensearch/replication/action/stats/LeaderStatsAction.kt b/src/main/kotlin/org/opensearch/replication/action/stats/LeaderStatsAction.kt index be7b3cd4..83c100d7 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stats/LeaderStatsAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stats/LeaderStatsAction.kt @@ -12,7 +12,7 @@ package org.opensearch.replication.action.stats import org.opensearch.action.ActionType -import org.opensearch.common.io.stream.Writeable +import org.opensearch.core.common.io.stream.Writeable class LeaderStatsAction : ActionType(NAME, reader) { companion object { diff --git a/src/main/kotlin/org/opensearch/replication/action/stats/LeaderStatsRequest.kt b/src/main/kotlin/org/opensearch/replication/action/stats/LeaderStatsRequest.kt index 5523a859..53ed027b 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stats/LeaderStatsRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stats/LeaderStatsRequest.kt @@ -12,8 +12,8 @@ package org.opensearch.replication.action.stats import org.opensearch.action.support.nodes.BaseNodesRequest -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import java.io.IOException /** diff --git a/src/main/kotlin/org/opensearch/replication/action/stats/LeaderStatsResponse.kt b/src/main/kotlin/org/opensearch/replication/action/stats/LeaderStatsResponse.kt index 47333152..29abc78c 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stats/LeaderStatsResponse.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stats/LeaderStatsResponse.kt @@ -17,8 +17,8 @@ import org.opensearch.action.FailedNodeException import org.opensearch.action.support.nodes.BaseNodesResponse import org.opensearch.cluster.ClusterName import org.opensearch.common.Strings -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.ToXContent.EMPTY_PARAMS import org.opensearch.core.xcontent.ToXContent.Params import org.opensearch.core.xcontent.ToXContentObject diff --git a/src/main/kotlin/org/opensearch/replication/action/stats/NodeStatsRequest.kt b/src/main/kotlin/org/opensearch/replication/action/stats/NodeStatsRequest.kt index 89c03f12..7d6676de 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stats/NodeStatsRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stats/NodeStatsRequest.kt @@ -11,8 +11,8 @@ package org.opensearch.replication.action.stats -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.transport.TransportRequest import java.io.IOException diff --git a/src/main/kotlin/org/opensearch/replication/action/stats/TransportFollowerStatsAction.kt b/src/main/kotlin/org/opensearch/replication/action/stats/TransportFollowerStatsAction.kt index e33d9a0c..7a258ef1 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stats/TransportFollowerStatsAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stats/TransportFollowerStatsAction.kt @@ -20,7 +20,7 @@ import org.opensearch.action.support.nodes.TransportNodesAction import org.opensearch.client.node.NodeClient import org.opensearch.cluster.service.ClusterService import org.opensearch.common.inject.Inject -import org.opensearch.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamInput import org.opensearch.replication.metadata.state.ReplicationStateMetadata import org.opensearch.replication.seqno.RemoteClusterStats import org.opensearch.replication.task.shard.FollowerClusterStats diff --git a/src/main/kotlin/org/opensearch/replication/action/stats/TransportLeaderStatsAction.kt b/src/main/kotlin/org/opensearch/replication/action/stats/TransportLeaderStatsAction.kt index 7cc9ec11..f6a91f2c 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stats/TransportLeaderStatsAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stats/TransportLeaderStatsAction.kt @@ -20,8 +20,8 @@ import org.opensearch.action.support.nodes.TransportNodesAction import org.opensearch.client.node.NodeClient import org.opensearch.cluster.service.ClusterService import org.opensearch.common.inject.Inject -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.index.shard.ShardId +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.index.shard.ShardId import org.opensearch.indices.IndicesService import org.opensearch.replication.seqno.RemoteClusterRetentionLeaseHelper.Companion.RETENTION_LEASE_PREFIX import org.opensearch.replication.seqno.RemoteClusterStats diff --git a/src/main/kotlin/org/opensearch/replication/action/status/ReplicationStatusAction.kt b/src/main/kotlin/org/opensearch/replication/action/status/ReplicationStatusAction.kt index f2416354..eb4de39f 100644 --- a/src/main/kotlin/org/opensearch/replication/action/status/ReplicationStatusAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/status/ReplicationStatusAction.kt @@ -12,7 +12,7 @@ package org.opensearch.replication.action.status import org.opensearch.action.ActionType -import org.opensearch.common.io.stream.Writeable +import org.opensearch.core.common.io.stream.Writeable class ReplicationStatusAction : ActionType(NAME, reader) { companion object { diff --git a/src/main/kotlin/org/opensearch/replication/action/status/ReplicationStatusResponse.kt b/src/main/kotlin/org/opensearch/replication/action/status/ReplicationStatusResponse.kt index a4832381..dc982cee 100644 --- a/src/main/kotlin/org/opensearch/replication/action/status/ReplicationStatusResponse.kt +++ b/src/main/kotlin/org/opensearch/replication/action/status/ReplicationStatusResponse.kt @@ -12,10 +12,10 @@ package org.opensearch.replication.action.status -import org.opensearch.action.support.DefaultShardOperationFailedException +import org.opensearch.core.action.support.DefaultShardOperationFailedException import org.opensearch.action.support.broadcast.BroadcastResponse -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.ToXContent.Params import org.opensearch.core.xcontent.ToXContentObject import org.opensearch.core.xcontent.XContentBuilder diff --git a/src/main/kotlin/org/opensearch/replication/action/status/ShardInfoRequest.kt b/src/main/kotlin/org/opensearch/replication/action/status/ShardInfoRequest.kt index 024dd976..67e09d6c 100644 --- a/src/main/kotlin/org/opensearch/replication/action/status/ShardInfoRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/status/ShardInfoRequest.kt @@ -14,8 +14,8 @@ package org.opensearch.replication.action.status import org.opensearch.action.ActionRequestValidationException import org.opensearch.action.support.IndicesOptions import org.opensearch.action.support.broadcast.BroadcastRequest -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.ToXContent import org.opensearch.core.xcontent.ToXContentObject import org.opensearch.core.xcontent.XContentBuilder diff --git a/src/main/kotlin/org/opensearch/replication/action/status/ShardInfoResponse.kt b/src/main/kotlin/org/opensearch/replication/action/status/ShardInfoResponse.kt index af111889..528e86df 100644 --- a/src/main/kotlin/org/opensearch/replication/action/status/ShardInfoResponse.kt +++ b/src/main/kotlin/org/opensearch/replication/action/status/ShardInfoResponse.kt @@ -14,12 +14,12 @@ package org.opensearch.replication.action.status import org.opensearch.action.support.broadcast.BroadcastResponse import org.opensearch.action.support.broadcast.BroadcastShardResponse import org.opensearch.core.ParseField -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.ToXContent import org.opensearch.core.xcontent.ToXContentObject import org.opensearch.core.xcontent.XContentBuilder -import org.opensearch.index.shard.ShardId +import org.opensearch.core.index.shard.ShardId import java.io.IOException class ShardInfoResponse : BroadcastShardResponse, ToXContentObject { diff --git a/src/main/kotlin/org/opensearch/replication/action/status/ShardsInfoAction.kt b/src/main/kotlin/org/opensearch/replication/action/status/ShardsInfoAction.kt index 81f6ba4e..d77cd714 100644 --- a/src/main/kotlin/org/opensearch/replication/action/status/ShardsInfoAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/status/ShardsInfoAction.kt @@ -12,7 +12,7 @@ package org.opensearch.replication.action.status import org.opensearch.action.ActionType -import org.opensearch.common.io.stream.Writeable +import org.opensearch.core.common.io.stream.Writeable class ShardsInfoAction : ActionType(NAME, reader) { diff --git a/src/main/kotlin/org/opensearch/replication/action/status/TranportShardsInfoAction.kt b/src/main/kotlin/org/opensearch/replication/action/status/TranportShardsInfoAction.kt index d3f542d8..4fd73b2f 100644 --- a/src/main/kotlin/org/opensearch/replication/action/status/TranportShardsInfoAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/status/TranportShardsInfoAction.kt @@ -13,7 +13,7 @@ package org.opensearch.replication.action.status import org.apache.logging.log4j.LogManager import org.opensearch.action.support.ActionFilters -import org.opensearch.action.support.DefaultShardOperationFailedException +import org.opensearch.core.action.support.DefaultShardOperationFailedException import org.opensearch.action.support.broadcast.node.TransportBroadcastByNodeAction import org.opensearch.cluster.ClusterState import org.opensearch.cluster.block.ClusterBlockException @@ -21,8 +21,8 @@ import org.opensearch.cluster.metadata.IndexNameExpressionResolver import org.opensearch.cluster.routing.* import org.opensearch.cluster.service.ClusterService import org.opensearch.common.inject.Inject -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.Writeable +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.Writeable import org.opensearch.index.IndexService import org.opensearch.indices.IndicesService import org.opensearch.threadpool.ThreadPool diff --git a/src/main/kotlin/org/opensearch/replication/action/stop/StopIndexReplicationRequest.kt b/src/main/kotlin/org/opensearch/replication/action/stop/StopIndexReplicationRequest.kt index 2f447eb8..3ae5eff9 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stop/StopIndexReplicationRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stop/StopIndexReplicationRequest.kt @@ -16,8 +16,8 @@ import org.opensearch.action.IndicesRequest import org.opensearch.action.support.IndicesOptions import org.opensearch.action.support.master.AcknowledgedRequest import org.opensearch.core.ParseField -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.* class StopIndexReplicationRequest : AcknowledgedRequest, IndicesRequest.Replaceable, ToXContentObject { diff --git a/src/main/kotlin/org/opensearch/replication/action/stop/TransportStopIndexReplicationAction.kt b/src/main/kotlin/org/opensearch/replication/action/stop/TransportStopIndexReplicationAction.kt index 8a6fdf71..f18cbb30 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stop/TransportStopIndexReplicationAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stop/TransportStopIndexReplicationAction.kt @@ -51,7 +51,7 @@ import org.opensearch.cluster.metadata.IndexNameExpressionResolver import org.opensearch.cluster.metadata.Metadata import org.opensearch.cluster.service.ClusterService import org.opensearch.common.inject.Inject -import org.opensearch.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamInput import org.opensearch.common.settings.Settings import org.opensearch.replication.util.stackTraceToString import org.opensearch.persistent.PersistentTasksCustomMetadata diff --git a/src/main/kotlin/org/opensearch/replication/action/update/TransportUpdateIndexReplicationAction.kt b/src/main/kotlin/org/opensearch/replication/action/update/TransportUpdateIndexReplicationAction.kt index 7ca866d8..333ed9f7 100644 --- a/src/main/kotlin/org/opensearch/replication/action/update/TransportUpdateIndexReplicationAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/update/TransportUpdateIndexReplicationAction.kt @@ -33,7 +33,7 @@ import org.opensearch.cluster.block.ClusterBlockLevel import org.opensearch.cluster.metadata.IndexNameExpressionResolver import org.opensearch.cluster.service.ClusterService import org.opensearch.common.inject.Inject -import org.opensearch.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamInput import org.opensearch.common.settings.IndexScopedSettings import org.opensearch.threadpool.ThreadPool import org.opensearch.transport.TransportService diff --git a/src/main/kotlin/org/opensearch/replication/action/update/UpdateIndexReplicationRequest.kt b/src/main/kotlin/org/opensearch/replication/action/update/UpdateIndexReplicationRequest.kt index 753e2f62..b498ed92 100644 --- a/src/main/kotlin/org/opensearch/replication/action/update/UpdateIndexReplicationRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/update/UpdateIndexReplicationRequest.kt @@ -16,8 +16,8 @@ import org.opensearch.action.IndicesRequest import org.opensearch.action.support.IndicesOptions import org.opensearch.action.support.master.AcknowledgedRequest import org.opensearch.core.ParseField -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.common.settings.Settings import org.opensearch.common.settings.Settings.readSettingsFromStream import org.opensearch.core.xcontent.* diff --git a/src/main/kotlin/org/opensearch/replication/metadata/TransportUpdateMetadataAction.kt b/src/main/kotlin/org/opensearch/replication/metadata/TransportUpdateMetadataAction.kt index eb3b1750..21b1643f 100644 --- a/src/main/kotlin/org/opensearch/replication/metadata/TransportUpdateMetadataAction.kt +++ b/src/main/kotlin/org/opensearch/replication/metadata/TransportUpdateMetadataAction.kt @@ -40,8 +40,8 @@ import org.opensearch.cluster.metadata.* import org.opensearch.cluster.metadata.AliasAction.RemoveIndex import org.opensearch.cluster.service.ClusterService import org.opensearch.common.inject.Inject -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.index.Index +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.index.Index import org.opensearch.index.IndexNotFoundException import org.opensearch.replication.util.stackTraceToString import org.opensearch.rest.action.admin.indices.AliasesNotFoundException diff --git a/src/main/kotlin/org/opensearch/replication/metadata/UpdateIndexBlockTask.kt b/src/main/kotlin/org/opensearch/replication/metadata/UpdateIndexBlockTask.kt index 3f8e9a90..e67293b6 100644 --- a/src/main/kotlin/org/opensearch/replication/metadata/UpdateIndexBlockTask.kt +++ b/src/main/kotlin/org/opensearch/replication/metadata/UpdateIndexBlockTask.kt @@ -23,7 +23,7 @@ import org.opensearch.cluster.block.ClusterBlockLevel import org.opensearch.cluster.block.ClusterBlocks import org.opensearch.cluster.service.ClusterService import org.opensearch.index.IndexNotFoundException -import org.opensearch.rest.RestStatus +import org.opensearch.core.rest.RestStatus import java.util.Collections import java.util.EnumSet diff --git a/src/main/kotlin/org/opensearch/replication/metadata/UpdateMetadataRequest.kt b/src/main/kotlin/org/opensearch/replication/metadata/UpdateMetadataRequest.kt index 25abe070..5f2e4488 100644 --- a/src/main/kotlin/org/opensearch/replication/metadata/UpdateMetadataRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/metadata/UpdateMetadataRequest.kt @@ -18,8 +18,8 @@ import org.opensearch.action.admin.indices.mapping.put.PutMappingRequest import org.opensearch.action.admin.indices.open.OpenIndexRequest import org.opensearch.action.admin.indices.settings.put.UpdateSettingsRequest import org.opensearch.action.support.master.AcknowledgedRequest -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput class UpdateMetadataRequest : AcknowledgedRequest { var indexName: String diff --git a/src/main/kotlin/org/opensearch/replication/metadata/state/ReplicationStateMetadata.kt b/src/main/kotlin/org/opensearch/replication/metadata/state/ReplicationStateMetadata.kt index 86711ac0..96c5c2e2 100644 --- a/src/main/kotlin/org/opensearch/replication/metadata/state/ReplicationStateMetadata.kt +++ b/src/main/kotlin/org/opensearch/replication/metadata/state/ReplicationStateMetadata.kt @@ -18,8 +18,8 @@ import org.opensearch.cluster.DiffableUtils.getStringKeySerializer import org.opensearch.cluster.NamedDiff import org.opensearch.cluster.metadata.Metadata import org.opensearch.cluster.service.ClusterService -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.ToXContent import org.opensearch.core.xcontent.XContentBuilder import org.opensearch.core.xcontent.XContentParser diff --git a/src/main/kotlin/org/opensearch/replication/metadata/store/ReplicationMetadata.kt b/src/main/kotlin/org/opensearch/replication/metadata/store/ReplicationMetadata.kt index 86cd80c2..891d9970 100644 --- a/src/main/kotlin/org/opensearch/replication/metadata/store/ReplicationMetadata.kt +++ b/src/main/kotlin/org/opensearch/replication/metadata/store/ReplicationMetadata.kt @@ -13,9 +13,9 @@ package org.opensearch.replication.metadata.store import org.opensearch.commons.authuser.User import org.opensearch.core.ParseField -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.common.io.stream.Writeable +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.Writeable import org.opensearch.common.settings.Settings import org.opensearch.core.xcontent.ObjectParser import org.opensearch.core.xcontent.ToXContent diff --git a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterMultiChunkTransfer.kt b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterMultiChunkTransfer.kt index f0995346..b3097f13 100644 --- a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterMultiChunkTransfer.kt +++ b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterMultiChunkTransfer.kt @@ -28,7 +28,7 @@ import org.opensearch.client.Client import org.opensearch.cluster.node.DiscoveryNode import org.opensearch.common.unit.ByteSizeValue import org.opensearch.common.util.concurrent.ThreadContext -import org.opensearch.index.shard.ShardId +import org.opensearch.core.index.shard.ShardId import org.opensearch.index.store.Store import org.opensearch.index.store.StoreFileMetadata import org.opensearch.indices.recovery.MultiChunkTransfer diff --git a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt index 4146caad..aa26d905 100644 --- a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt +++ b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt @@ -51,7 +51,7 @@ import org.opensearch.common.component.AbstractLifecycleComponent import org.opensearch.common.metrics.CounterMetric import org.opensearch.common.settings.Settings import org.opensearch.index.mapper.MapperService -import org.opensearch.index.shard.ShardId +import org.opensearch.core.index.shard.ShardId import org.opensearch.index.snapshots.IndexShardSnapshotStatus import org.opensearch.index.store.Store import org.opensearch.indices.recovery.RecoverySettings diff --git a/src/main/kotlin/org/opensearch/replication/rest/AutoFollowStatsHandler.kt b/src/main/kotlin/org/opensearch/replication/rest/AutoFollowStatsHandler.kt index 42421eb4..f9055c70 100644 --- a/src/main/kotlin/org/opensearch/replication/rest/AutoFollowStatsHandler.kt +++ b/src/main/kotlin/org/opensearch/replication/rest/AutoFollowStatsHandler.kt @@ -15,7 +15,7 @@ import org.opensearch.rest.RestChannel import org.opensearch.rest.RestHandler import org.opensearch.rest.RestRequest import org.opensearch.rest.RestResponse -import org.opensearch.rest.RestStatus +import org.opensearch.core.rest.RestStatus import org.opensearch.rest.action.RestResponseListener import java.io.IOException diff --git a/src/main/kotlin/org/opensearch/replication/rest/FollowerStatsHandler.kt b/src/main/kotlin/org/opensearch/replication/rest/FollowerStatsHandler.kt index ce5013f6..3dc0e4dc 100644 --- a/src/main/kotlin/org/opensearch/replication/rest/FollowerStatsHandler.kt +++ b/src/main/kotlin/org/opensearch/replication/rest/FollowerStatsHandler.kt @@ -15,7 +15,7 @@ import org.opensearch.rest.RestChannel import org.opensearch.rest.RestHandler import org.opensearch.rest.RestRequest import org.opensearch.rest.RestResponse -import org.opensearch.rest.RestStatus +import org.opensearch.core.rest.RestStatus import org.opensearch.rest.action.RestResponseListener import java.io.IOException diff --git a/src/main/kotlin/org/opensearch/replication/rest/LeaderStatsHandler.kt b/src/main/kotlin/org/opensearch/replication/rest/LeaderStatsHandler.kt index d71379bf..17acc842 100644 --- a/src/main/kotlin/org/opensearch/replication/rest/LeaderStatsHandler.kt +++ b/src/main/kotlin/org/opensearch/replication/rest/LeaderStatsHandler.kt @@ -15,7 +15,7 @@ import org.opensearch.rest.RestChannel import org.opensearch.rest.RestHandler import org.opensearch.rest.RestRequest import org.opensearch.rest.RestResponse -import org.opensearch.rest.RestStatus +import org.opensearch.core.rest.RestStatus import org.opensearch.rest.action.RestResponseListener import java.io.IOException diff --git a/src/main/kotlin/org/opensearch/replication/rest/UpdateAutoFollowPatternsHandler.kt b/src/main/kotlin/org/opensearch/replication/rest/UpdateAutoFollowPatternsHandler.kt index ccb9463a..a8a738e2 100644 --- a/src/main/kotlin/org/opensearch/replication/rest/UpdateAutoFollowPatternsHandler.kt +++ b/src/main/kotlin/org/opensearch/replication/rest/UpdateAutoFollowPatternsHandler.kt @@ -19,7 +19,7 @@ import org.opensearch.rest.BaseRestHandler import org.opensearch.rest.BaseRestHandler.RestChannelConsumer import org.opensearch.rest.RestHandler import org.opensearch.rest.RestRequest -import org.opensearch.rest.RestStatus +import org.opensearch.core.rest.RestStatus import org.opensearch.rest.action.RestToXContentListener class UpdateAutoFollowPatternsHandler : BaseRestHandler() { diff --git a/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterRetentionLeaseHelper.kt b/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterRetentionLeaseHelper.kt index d50540a3..ddc0ee1b 100644 --- a/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterRetentionLeaseHelper.kt +++ b/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterRetentionLeaseHelper.kt @@ -23,7 +23,7 @@ import org.opensearch.index.seqno.RetentionLeaseAlreadyExistsException import org.opensearch.index.seqno.RetentionLeaseInvalidRetainingSeqNoException import org.opensearch.index.seqno.RetentionLeaseNotFoundException import org.opensearch.index.shard.IndexShard -import org.opensearch.index.shard.ShardId +import org.opensearch.core.index.shard.ShardId import org.opensearch.replication.metadata.store.ReplicationMetadata import org.opensearch.replication.repository.RemoteClusterRepository import org.opensearch.replication.task.index.IndexReplicationParams diff --git a/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterStats.kt b/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterStats.kt index cc565d03..80a9d6d2 100644 --- a/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterStats.kt +++ b/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterStats.kt @@ -13,13 +13,13 @@ package org.opensearch.replication.seqno import org.opensearch.common.component.AbstractLifecycleComponent import org.opensearch.common.inject.Singleton -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.ToXContent import org.opensearch.core.xcontent.ToXContentFragment import org.opensearch.core.xcontent.ToXContentObject import org.opensearch.core.xcontent.XContentBuilder -import org.opensearch.index.shard.ShardId +import org.opensearch.core.index.shard.ShardId import java.util.concurrent.atomic.AtomicLong class RemoteShardMetric { diff --git a/src/main/kotlin/org/opensearch/replication/task/CrossClusterReplicationTask.kt b/src/main/kotlin/org/opensearch/replication/task/CrossClusterReplicationTask.kt index 75477625..7f03dd79 100644 --- a/src/main/kotlin/org/opensearch/replication/task/CrossClusterReplicationTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/CrossClusterReplicationTask.kt @@ -34,20 +34,20 @@ import org.opensearch.action.ActionListener import org.opensearch.action.ActionResponse import org.opensearch.client.Client import org.opensearch.cluster.service.ClusterService -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.common.settings.Settings import org.opensearch.core.xcontent.ToXContent import org.opensearch.core.xcontent.ToXContentObject import org.opensearch.core.xcontent.XContentBuilder import org.opensearch.index.IndexService import org.opensearch.index.shard.IndexShard -import org.opensearch.index.shard.ShardId +import org.opensearch.core.index.shard.ShardId import org.opensearch.indices.cluster.IndicesClusterStateService import org.opensearch.persistent.AllocatedPersistentTask import org.opensearch.persistent.PersistentTaskState import org.opensearch.persistent.PersistentTasksService import org.opensearch.replication.util.stackTraceToString -import org.opensearch.rest.RestStatus +import org.opensearch.core.rest.RestStatus import org.opensearch.tasks.TaskId import org.opensearch.tasks.TaskManager import org.opensearch.threadpool.ThreadPool diff --git a/src/main/kotlin/org/opensearch/replication/task/IndexCloseListener.kt b/src/main/kotlin/org/opensearch/replication/task/IndexCloseListener.kt index ecd87698..cbc4f676 100644 --- a/src/main/kotlin/org/opensearch/replication/task/IndexCloseListener.kt +++ b/src/main/kotlin/org/opensearch/replication/task/IndexCloseListener.kt @@ -15,7 +15,7 @@ import org.opensearch.common.settings.Settings import org.opensearch.index.IndexService import org.opensearch.index.shard.IndexEventListener import org.opensearch.index.shard.IndexShard -import org.opensearch.index.shard.ShardId +import org.opensearch.core.index.shard.ShardId import org.opensearch.indices.cluster.IndicesClusterStateService import java.util.Collections import java.util.concurrent.ConcurrentHashMap diff --git a/src/main/kotlin/org/opensearch/replication/task/ReplicationState.kt b/src/main/kotlin/org/opensearch/replication/task/ReplicationState.kt index 3a81f74e..1f0763b8 100644 --- a/src/main/kotlin/org/opensearch/replication/task/ReplicationState.kt +++ b/src/main/kotlin/org/opensearch/replication/task/ReplicationState.kt @@ -11,9 +11,9 @@ package org.opensearch.replication.task -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput -import org.opensearch.common.io.stream.Writeable +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.Writeable import org.opensearch.core.xcontent.ToXContent import org.opensearch.core.xcontent.ToXContentFragment import org.opensearch.core.xcontent.XContentBuilder diff --git a/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowParams.kt b/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowParams.kt index c5b2b8bc..e58d6586 100644 --- a/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowParams.kt +++ b/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowParams.kt @@ -13,8 +13,8 @@ package org.opensearch.replication.task.autofollow import org.opensearch.Version import org.opensearch.core.ParseField -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.ObjectParser import org.opensearch.core.xcontent.ToXContent import org.opensearch.core.xcontent.XContentBuilder diff --git a/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowTask.kt b/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowTask.kt index 15c22922..d47843d6 100644 --- a/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowTask.kt @@ -29,8 +29,8 @@ import org.opensearch.action.admin.indices.get.GetIndexRequest import org.opensearch.action.support.IndicesOptions import org.opensearch.client.Client import org.opensearch.cluster.service.ClusterService -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.common.logging.Loggers import org.opensearch.core.xcontent.ToXContent import org.opensearch.core.xcontent.XContentBuilder @@ -39,7 +39,7 @@ import org.opensearch.replication.ReplicationException import org.opensearch.replication.action.status.ReplicationStatusAction import org.opensearch.replication.action.status.ShardInfoRequest import org.opensearch.replication.action.status.ShardInfoResponse -import org.opensearch.rest.RestStatus +import org.opensearch.core.rest.RestStatus import org.opensearch.tasks.Task import org.opensearch.tasks.TaskId import org.opensearch.threadpool.Scheduler diff --git a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationParams.kt b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationParams.kt index 6541e041..644b73e3 100644 --- a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationParams.kt +++ b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationParams.kt @@ -14,14 +14,14 @@ package org.opensearch.replication.task.index import org.opensearch.Version import org.opensearch.core.ParseField import org.opensearch.common.Strings -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.ObjectParser import org.opensearch.core.xcontent.ToXContent import org.opensearch.core.xcontent.XContentBuilder import org.opensearch.core.xcontent.XContentParser import org.opensearch.common.xcontent.XContentType -import org.opensearch.index.Index +import org.opensearch.core.index.Index import org.opensearch.persistent.PersistentTaskParams import java.io.IOException diff --git a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationState.kt b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationState.kt index 010d1447..d55accd7 100644 --- a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationState.kt +++ b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationState.kt @@ -14,13 +14,13 @@ package org.opensearch.replication.task.index import org.opensearch.replication.task.ReplicationState import org.opensearch.replication.task.shard.ShardReplicationParams import org.opensearch.core.ParseField -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.ObjectParser import org.opensearch.core.xcontent.ToXContent import org.opensearch.core.xcontent.XContentBuilder import org.opensearch.core.xcontent.XContentParser -import org.opensearch.index.shard.ShardId +import org.opensearch.core.index.shard.ShardId import org.opensearch.persistent.PersistentTaskState import org.opensearch.persistent.PersistentTasksCustomMetadata.PersistentTask import java.io.IOException diff --git a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt index d51af7c0..f520926b 100644 --- a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt @@ -68,7 +68,7 @@ import org.opensearch.cluster.RestoreInProgress import org.opensearch.cluster.metadata.IndexMetadata import org.opensearch.cluster.routing.allocation.decider.EnableAllocationDecider import org.opensearch.cluster.service.ClusterService -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.common.logging.Loggers import org.opensearch.common.settings.Setting import org.opensearch.common.settings.Settings @@ -79,11 +79,11 @@ import org.opensearch.core.xcontent.ToXContent import org.opensearch.core.xcontent.ToXContentObject import org.opensearch.core.xcontent.XContentBuilder import org.opensearch.common.xcontent.XContentType -import org.opensearch.index.Index +import org.opensearch.core.index.Index import org.opensearch.index.IndexService import org.opensearch.index.IndexSettings import org.opensearch.index.shard.IndexShard -import org.opensearch.index.shard.ShardId +import org.opensearch.core.index.shard.ShardId import org.opensearch.indices.cluster.IndicesClusterStateService import org.opensearch.indices.recovery.RecoveryState import org.opensearch.persistent.PersistentTaskState @@ -94,7 +94,7 @@ import org.opensearch.persistent.PersistentTasksService import org.opensearch.replication.ReplicationException import org.opensearch.replication.MappingNotAvailableException import org.opensearch.replication.ReplicationPlugin.Companion.REPLICATION_INDEX_TRANSLOG_PRUNING_ENABLED_SETTING -import org.opensearch.rest.RestStatus +import org.opensearch.core.rest.RestStatus import org.opensearch.tasks.TaskId import org.opensearch.tasks.TaskManager import org.opensearch.threadpool.ThreadPool diff --git a/src/main/kotlin/org/opensearch/replication/task/shard/FollowerClusterStats.kt b/src/main/kotlin/org/opensearch/replication/task/shard/FollowerClusterStats.kt index db112a1f..77cfb672 100644 --- a/src/main/kotlin/org/opensearch/replication/task/shard/FollowerClusterStats.kt +++ b/src/main/kotlin/org/opensearch/replication/task/shard/FollowerClusterStats.kt @@ -13,13 +13,13 @@ package org.opensearch.replication.task.shard import org.apache.logging.log4j.LogManager import org.opensearch.common.inject.Singleton -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.ToXContent import org.opensearch.core.xcontent.ToXContentFragment import org.opensearch.core.xcontent.XContentBuilder import org.opensearch.common.xcontent.XContentType -import org.opensearch.index.shard.ShardId +import org.opensearch.core.index.shard.ShardId import java.util.concurrent.atomic.AtomicLong class FollowerShardMetric { diff --git a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationParams.kt b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationParams.kt index 357eb5b6..95b22425 100644 --- a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationParams.kt +++ b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationParams.kt @@ -14,17 +14,17 @@ package org.opensearch.replication.task.shard import org.opensearch.Version import org.opensearch.core.ParseField import org.opensearch.common.Strings -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.ObjectParser import org.opensearch.core.xcontent.ToXContent import org.opensearch.core.xcontent.XContentBuilder import org.opensearch.core.xcontent.XContentParser import org.opensearch.common.xcontent.XContentType -import org.opensearch.index.shard.ShardId +import org.opensearch.core.index.shard.ShardId import org.opensearch.persistent.PersistentTaskParams import java.io.IOException -import org.opensearch.index.Index +import org.opensearch.core.index.Index class ShardReplicationParams : PersistentTaskParams { diff --git a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationState.kt b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationState.kt index f0c3fc88..33efd3ca 100644 --- a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationState.kt +++ b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationState.kt @@ -14,8 +14,8 @@ package org.opensearch.replication.task.shard import org.opensearch.replication.task.ReplicationState import org.opensearch.OpenSearchException import org.opensearch.core.ParseField -import org.opensearch.common.io.stream.StreamInput -import org.opensearch.common.io.stream.StreamOutput +import org.opensearch.core.common.io.stream.StreamInput +import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.ObjectParser import org.opensearch.core.xcontent.ToXContent import org.opensearch.core.xcontent.XContentBuilder diff --git a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationTask.kt b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationTask.kt index e393805e..e165c916 100644 --- a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationTask.kt @@ -47,10 +47,10 @@ import org.opensearch.common.logging.Loggers import org.opensearch.index.seqno.RetentionLeaseActions import org.opensearch.index.seqno.RetentionLeaseInvalidRetainingSeqNoException import org.opensearch.index.seqno.RetentionLeaseNotFoundException -import org.opensearch.index.shard.ShardId +import org.opensearch.core.index.shard.ShardId import org.opensearch.persistent.PersistentTaskState import org.opensearch.persistent.PersistentTasksNodeService -import org.opensearch.rest.RestStatus +import org.opensearch.core.rest.RestStatus import org.opensearch.tasks.TaskId import org.opensearch.threadpool.ThreadPool import org.opensearch.transport.NodeNotConnectedException diff --git a/src/main/kotlin/org/opensearch/replication/task/shard/TranslogSequencer.kt b/src/main/kotlin/org/opensearch/replication/task/shard/TranslogSequencer.kt index b274bdf0..ad4e7681 100644 --- a/src/main/kotlin/org/opensearch/replication/task/shard/TranslogSequencer.kt +++ b/src/main/kotlin/org/opensearch/replication/task/shard/TranslogSequencer.kt @@ -29,14 +29,14 @@ import org.opensearch.OpenSearchException import org.opensearch.action.support.TransportActions import org.opensearch.common.logging.Loggers import org.opensearch.index.IndexNotFoundException -import org.opensearch.index.shard.ShardId +import org.opensearch.core.index.shard.ShardId import org.opensearch.index.translog.Translog import org.opensearch.replication.util.indicesService import org.opensearch.tasks.TaskId import java.util.ArrayList import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.TimeUnit -import org.opensearch.rest.RestStatus +import org.opensearch.core.rest.RestStatus /** diff --git a/src/main/kotlin/org/opensearch/replication/util/Extensions.kt b/src/main/kotlin/org/opensearch/replication/util/Extensions.kt index 643cc010..1928ad6b 100644 --- a/src/main/kotlin/org/opensearch/replication/util/Extensions.kt +++ b/src/main/kotlin/org/opensearch/replication/util/Extensions.kt @@ -31,13 +31,13 @@ import org.opensearch.client.Client import org.opensearch.common.util.concurrent.ThreadContext import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException import org.opensearch.index.IndexNotFoundException -import org.opensearch.index.shard.ShardId +import org.opensearch.core.index.shard.ShardId import org.opensearch.index.store.Store import org.opensearch.indices.recovery.RecoveryState import org.opensearch.replication.ReplicationException import org.opensearch.replication.util.stackTraceToString import org.opensearch.repositories.IndexId -import org.opensearch.rest.RestStatus +import org.opensearch.core.rest.RestStatus import org.opensearch.snapshots.SnapshotId import org.opensearch.transport.ConnectTransportException import org.opensearch.transport.NodeDisconnectedException diff --git a/src/test/kotlin/org/opensearch/index/translog/ReplicationTranslogDeletionPolicyTests.kt b/src/test/kotlin/org/opensearch/index/translog/ReplicationTranslogDeletionPolicyTests.kt index 8378bbbb..a59e75fe 100644 --- a/src/test/kotlin/org/opensearch/index/translog/ReplicationTranslogDeletionPolicyTests.kt +++ b/src/test/kotlin/org/opensearch/index/translog/ReplicationTranslogDeletionPolicyTests.kt @@ -14,14 +14,14 @@ import org.apache.lucene.store.ByteArrayDataOutput import org.hamcrest.Matchers.equalTo import org.mockito.Mockito import org.opensearch.common.UUIDs -import org.opensearch.common.bytes.BytesArray +import org.opensearch.core.common.bytes.BytesArray import org.opensearch.common.bytes.ReleasableBytesReference import org.opensearch.common.collect.Tuple import org.opensearch.common.util.BigArrays import org.opensearch.common.util.io.IOUtils import org.opensearch.index.seqno.RetentionLease import org.opensearch.index.seqno.RetentionLeases -import org.opensearch.index.shard.ShardId +import org.opensearch.core.index.shard.ShardId import org.opensearch.test.OpenSearchTestCase import java.io.IOException import java.nio.channels.FileChannel diff --git a/src/test/kotlin/org/opensearch/replication/bwc/BackwardsCompatibilityIT.kt b/src/test/kotlin/org/opensearch/replication/bwc/BackwardsCompatibilityIT.kt index e856eedd..a53e2714 100644 --- a/src/test/kotlin/org/opensearch/replication/bwc/BackwardsCompatibilityIT.kt +++ b/src/test/kotlin/org/opensearch/replication/bwc/BackwardsCompatibilityIT.kt @@ -82,6 +82,7 @@ class BackwardsCompatibilityIT : MultiClusterRestTestCase() { ClusterStatus.ONE_THIRD_UPGRADED, ClusterStatus.TWO_THIRD_UPGRADED, ClusterStatus.ROLLING_UPGRADED, ClusterStatus.FULL_CLUSTER_RESTART -> verifyReplication() ClusterStatus.COMPLETE_SUITE -> {} // Do nothing as all tests have run already + else -> {throw AssertionError("${ClusterStatus.from(System.getProperty("tests.bwcTask"))} is not a valid option for ClusterStatus")} } } diff --git a/src/test/kotlin/org/opensearch/replication/singleCluster/SingleClusterSanityIT.kt b/src/test/kotlin/org/opensearch/replication/singleCluster/SingleClusterSanityIT.kt index 9760e7c2..24b9bb15 100644 --- a/src/test/kotlin/org/opensearch/replication/singleCluster/SingleClusterSanityIT.kt +++ b/src/test/kotlin/org/opensearch/replication/singleCluster/SingleClusterSanityIT.kt @@ -45,6 +45,7 @@ class SingleClusterSanityIT : MultiClusterRestTestCase() { fun testReplicationPluginWithSingleCluster() { when(ClusterState.from(System.getProperty("tests.sanitySingleCluster"))) { ClusterState.SINGLE_CLUSTER_SANITY_SUITE -> basicReplicationSanityWithSingleCluster() + else -> {throw AssertionError("${System.getProperty("tests.sanitySingleCluster")} is not a valid option for ClusterState")} } } diff --git a/src/test/kotlin/org/opensearch/replication/task/index/IndexReplicationTaskTests.kt b/src/test/kotlin/org/opensearch/replication/task/index/IndexReplicationTaskTests.kt index 2032cc26..00bde557 100644 --- a/src/test/kotlin/org/opensearch/replication/task/index/IndexReplicationTaskTests.kt +++ b/src/test/kotlin/org/opensearch/replication/task/index/IndexReplicationTaskTests.kt @@ -33,8 +33,8 @@ import org.opensearch.common.settings.Settings import org.opensearch.common.settings.SettingsModule import org.opensearch.common.unit.TimeValue import org.opensearch.core.xcontent.NamedXContentRegistry -import org.opensearch.index.Index -import org.opensearch.index.shard.ShardId +import org.opensearch.core.index.Index +import org.opensearch.core.index.shard.ShardId import org.opensearch.persistent.PersistentTaskParams import org.opensearch.persistent.PersistentTasksCustomMetadata import org.opensearch.persistent.PersistentTasksService diff --git a/src/test/kotlin/org/opensearch/replication/task/index/NoOpClient.kt b/src/test/kotlin/org/opensearch/replication/task/index/NoOpClient.kt index 34e711fd..606d7a18 100644 --- a/src/test/kotlin/org/opensearch/replication/task/index/NoOpClient.kt +++ b/src/test/kotlin/org/opensearch/replication/task/index/NoOpClient.kt @@ -29,13 +29,13 @@ import org.opensearch.action.get.GetAction import org.opensearch.action.get.GetResponse import org.opensearch.action.support.master.AcknowledgedResponse import org.opensearch.common.UUIDs -import org.opensearch.common.bytes.BytesReference +import org.opensearch.core.common.bytes.BytesReference import org.opensearch.common.settings.Settings import org.opensearch.core.xcontent.ToXContent import org.opensearch.common.xcontent.XContentFactory -import org.opensearch.index.Index +import org.opensearch.core.index.Index import org.opensearch.index.get.GetResult -import org.opensearch.index.shard.ShardId +import org.opensearch.core.index.shard.ShardId import org.opensearch.indices.recovery.RecoveryState import org.opensearch.persistent.PersistentTaskResponse import org.opensearch.persistent.PersistentTasksCustomMetadata diff --git a/src/test/kotlin/org/opensearch/replication/task/shard/ShardReplicationExecutorTests.kt b/src/test/kotlin/org/opensearch/replication/task/shard/ShardReplicationExecutorTests.kt index 630234f8..39275542 100644 --- a/src/test/kotlin/org/opensearch/replication/task/shard/ShardReplicationExecutorTests.kt +++ b/src/test/kotlin/org/opensearch/replication/task/shard/ShardReplicationExecutorTests.kt @@ -12,8 +12,8 @@ import org.opensearch.cluster.metadata.Metadata import org.opensearch.cluster.routing.* import org.opensearch.common.unit.TimeValue import org.opensearch.core.xcontent.NamedXContentRegistry -import org.opensearch.index.Index -import org.opensearch.index.shard.ShardId +import org.opensearch.core.index.Index +import org.opensearch.core.index.shard.ShardId import org.opensearch.replication.ReplicationSettings import org.opensearch.replication.metadata.ReplicationMetadataManager import org.opensearch.replication.metadata.store.ReplicationMetadataStore diff --git a/src/test/kotlin/org/opensearch/replication/task/shard/TranslogSequencerTests.kt b/src/test/kotlin/org/opensearch/replication/task/shard/TranslogSequencerTests.kt index fe6ad1c8..730289ed 100644 --- a/src/test/kotlin/org/opensearch/replication/task/shard/TranslogSequencerTests.kt +++ b/src/test/kotlin/org/opensearch/replication/task/shard/TranslogSequencerTests.kt @@ -24,7 +24,7 @@ import org.opensearch.action.support.replication.ReplicationResponse.ShardInfo import org.opensearch.common.settings.Settings import org.opensearch.index.IndexService import org.opensearch.index.shard.IndexShard -import org.opensearch.index.shard.ShardId +import org.opensearch.core.index.shard.ShardId import org.opensearch.index.translog.Translog import org.opensearch.indices.IndicesService import org.opensearch.replication.action.changes.GetChangesResponse From bcf1144a48671de372542b032de93f2ffbd46f19 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Tue, 25 Jul 2023 13:03:56 +0530 Subject: [PATCH 105/157] Change use of XContentType.fromMediaType to MediaType.fromMediaType (#1094) Signed-off-by: monusingh-1 --- .../kotlin/org/opensearch/replication/ReplicationHelpers.kt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/test/kotlin/org/opensearch/replication/ReplicationHelpers.kt b/src/test/kotlin/org/opensearch/replication/ReplicationHelpers.kt index eb5f6404..2d42a0d2 100644 --- a/src/test/kotlin/org/opensearch/replication/ReplicationHelpers.kt +++ b/src/test/kotlin/org/opensearch/replication/ReplicationHelpers.kt @@ -25,7 +25,7 @@ import org.opensearch.common.settings.Settings import org.opensearch.common.unit.TimeValue import org.opensearch.core.xcontent.DeprecationHandler import org.opensearch.core.xcontent.NamedXContentRegistry -import org.opensearch.common.xcontent.XContentType +import org.opensearch.core.xcontent.MediaType import org.opensearch.test.OpenSearchTestCase.assertBusy import org.opensearch.test.rest.OpenSearchRestTestCase import org.junit.Assert @@ -96,7 +96,7 @@ fun RestHighLevelClient.startReplication(request: StartReplicationRequest, waitForNoInitializingShards() } fun getAckResponse(lowLevelResponse: Response): AcknowledgedResponse { - val xContentType = XContentType.fromMediaType(lowLevelResponse.entity.contentType) + val xContentType = MediaType.fromMediaType(lowLevelResponse.entity.contentType) val xcp = xContentType.xContent().createParser(NamedXContentRegistry.EMPTY, DeprecationHandler.IGNORE_DEPRECATIONS, lowLevelResponse.entity.content) return AcknowledgedResponse.fromXContent(xcp) From 0e6ad7a9722a82ad4c6dba5cbfbcb8a96b914a3a Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Wed, 2 Aug 2023 12:10:51 +0530 Subject: [PATCH 106/157] Refactor for CircuitBreaker and Lifecycle (#1096) Refactor for CircuitBreaker and Lifecycle (#1096) Signed-off-by: monusingh-1 --- .../index/translog/ReplicationTranslogDeletionPolicy.kt | 2 +- .../kotlin/org/opensearch/replication/ReplicationPlugin.kt | 6 +++--- .../org/opensearch/replication/ReplicationSettings.kt | 2 +- .../replication/metadata/store/ReplicationMetadataStore.kt | 2 +- .../repository/RemoteClusterMultiChunkTransfer.kt | 2 +- .../replication/repository/RemoteClusterRepository.kt | 2 +- .../repository/RemoteClusterRestoreLeaderService.kt | 2 +- .../org/opensearch/replication/seqno/RemoteClusterStats.kt | 2 +- .../replication/seqno/RemoteClusterTranslogService.kt | 2 +- .../replication/task/index/IndexReplicationTask.kt | 4 ++-- .../kotlin/org/opensearch/replication/util/Injectables.kt | 2 +- 11 files changed, 14 insertions(+), 14 deletions(-) diff --git a/src/main/kotlin/org/opensearch/index/translog/ReplicationTranslogDeletionPolicy.kt b/src/main/kotlin/org/opensearch/index/translog/ReplicationTranslogDeletionPolicy.kt index ec7a36d0..963044dd 100644 --- a/src/main/kotlin/org/opensearch/index/translog/ReplicationTranslogDeletionPolicy.kt +++ b/src/main/kotlin/org/opensearch/index/translog/ReplicationTranslogDeletionPolicy.kt @@ -1,6 +1,6 @@ package org.opensearch.index.translog -import org.opensearch.common.unit.ByteSizeValue +import org.opensearch.core.common.unit.ByteSizeValue import org.opensearch.index.IndexSettings import org.opensearch.index.seqno.RetentionLease import org.opensearch.index.seqno.RetentionLeases diff --git a/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt b/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt index 7d66c976..8a17fc51 100644 --- a/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt +++ b/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt @@ -83,7 +83,7 @@ import org.opensearch.cluster.node.DiscoveryNodes import org.opensearch.cluster.service.ClusterService import org.opensearch.common.CheckedFunction import org.opensearch.core.ParseField -import org.opensearch.common.component.LifecycleComponent +import org.opensearch.common.lifecycle.LifecycleComponent import org.opensearch.core.common.io.stream.NamedWriteableRegistry import org.opensearch.core.common.io.stream.Writeable import org.opensearch.common.settings.ClusterSettings @@ -92,8 +92,8 @@ import org.opensearch.common.settings.Setting import org.opensearch.common.settings.Settings import org.opensearch.common.settings.SettingsFilter import org.opensearch.common.settings.SettingsModule -import org.opensearch.common.unit.ByteSizeUnit -import org.opensearch.common.unit.ByteSizeValue +import org.opensearch.core.common.unit.ByteSizeUnit +import org.opensearch.core.common.unit.ByteSizeValue import org.opensearch.common.unit.TimeValue import org.opensearch.common.util.concurrent.OpenSearchExecutors import org.opensearch.core.xcontent.NamedXContentRegistry diff --git a/src/main/kotlin/org/opensearch/replication/ReplicationSettings.kt b/src/main/kotlin/org/opensearch/replication/ReplicationSettings.kt index 2b516f8e..a5065c6d 100644 --- a/src/main/kotlin/org/opensearch/replication/ReplicationSettings.kt +++ b/src/main/kotlin/org/opensearch/replication/ReplicationSettings.kt @@ -13,7 +13,7 @@ package org.opensearch.replication import org.opensearch.cluster.service.ClusterService import org.opensearch.common.settings.ClusterSettings -import org.opensearch.common.unit.ByteSizeValue +import org.opensearch.core.common.unit.ByteSizeValue import org.opensearch.common.unit.TimeValue import org.opensearch.commons.utils.OpenForTesting diff --git a/src/main/kotlin/org/opensearch/replication/metadata/store/ReplicationMetadataStore.kt b/src/main/kotlin/org/opensearch/replication/metadata/store/ReplicationMetadataStore.kt index 18a0cd6c..d5cbe751 100644 --- a/src/main/kotlin/org/opensearch/replication/metadata/store/ReplicationMetadataStore.kt +++ b/src/main/kotlin/org/opensearch/replication/metadata/store/ReplicationMetadataStore.kt @@ -30,7 +30,7 @@ import org.opensearch.client.Client import org.opensearch.cluster.health.ClusterHealthStatus import org.opensearch.cluster.metadata.IndexMetadata import org.opensearch.cluster.service.ClusterService -import org.opensearch.common.component.AbstractLifecycleComponent +import org.opensearch.common.lifecycle.AbstractLifecycleComponent import org.opensearch.common.settings.Settings import org.opensearch.common.util.concurrent.ThreadContext import org.opensearch.common.xcontent.XContentType diff --git a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterMultiChunkTransfer.kt b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterMultiChunkTransfer.kt index b3097f13..344fbcbb 100644 --- a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterMultiChunkTransfer.kt +++ b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterMultiChunkTransfer.kt @@ -26,7 +26,7 @@ import org.apache.logging.log4j.Logger import org.opensearch.action.ActionListener import org.opensearch.client.Client import org.opensearch.cluster.node.DiscoveryNode -import org.opensearch.common.unit.ByteSizeValue +import org.opensearch.core.common.unit.ByteSizeValue import org.opensearch.common.util.concurrent.ThreadContext import org.opensearch.core.index.shard.ShardId import org.opensearch.index.store.Store diff --git a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt index aa26d905..366b9ee0 100644 --- a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt +++ b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt @@ -47,7 +47,7 @@ import org.opensearch.cluster.node.DiscoveryNode import org.opensearch.cluster.service.ClusterService import org.opensearch.common.Nullable import org.opensearch.common.UUIDs -import org.opensearch.common.component.AbstractLifecycleComponent +import org.opensearch.common.lifecycle.AbstractLifecycleComponent import org.opensearch.common.metrics.CounterMetric import org.opensearch.common.settings.Settings import org.opensearch.index.mapper.MapperService diff --git a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRestoreLeaderService.kt b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRestoreLeaderService.kt index 5c06e4d4..22c279a2 100644 --- a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRestoreLeaderService.kt +++ b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRestoreLeaderService.kt @@ -17,7 +17,7 @@ import org.opensearch.replication.util.performOp import org.opensearch.OpenSearchException import org.opensearch.action.support.single.shard.SingleShardRequest import org.opensearch.client.node.NodeClient -import org.opensearch.common.component.AbstractLifecycleComponent +import org.opensearch.common.lifecycle.AbstractLifecycleComponent import org.opensearch.common.inject.Inject import org.opensearch.common.inject.Singleton import org.opensearch.common.lucene.store.InputStreamIndexInput diff --git a/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterStats.kt b/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterStats.kt index 80a9d6d2..9481543a 100644 --- a/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterStats.kt +++ b/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterStats.kt @@ -11,7 +11,7 @@ package org.opensearch.replication.seqno -import org.opensearch.common.component.AbstractLifecycleComponent +import org.opensearch.common.lifecycle.AbstractLifecycleComponent import org.opensearch.common.inject.Singleton import org.opensearch.core.common.io.stream.StreamInput import org.opensearch.core.common.io.stream.StreamOutput diff --git a/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterTranslogService.kt b/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterTranslogService.kt index f7514daf..db0171ca 100644 --- a/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterTranslogService.kt +++ b/src/main/kotlin/org/opensearch/replication/seqno/RemoteClusterTranslogService.kt @@ -13,7 +13,7 @@ package org.opensearch.replication.seqno import org.apache.logging.log4j.LogManager import org.opensearch.ResourceNotFoundException -import org.opensearch.common.component.AbstractLifecycleComponent +import org.opensearch.common.lifecycle.AbstractLifecycleComponent import org.opensearch.common.inject.Singleton import org.opensearch.index.engine.Engine import org.opensearch.index.shard.IndexShard diff --git a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt index f520926b..ce3cb43a 100644 --- a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt @@ -73,8 +73,8 @@ import org.opensearch.common.logging.Loggers import org.opensearch.common.settings.Setting import org.opensearch.common.settings.Settings import org.opensearch.common.settings.SettingsModule -import org.opensearch.common.unit.ByteSizeUnit -import org.opensearch.common.unit.ByteSizeValue +import org.opensearch.core.common.unit.ByteSizeUnit +import org.opensearch.core.common.unit.ByteSizeValue import org.opensearch.core.xcontent.ToXContent import org.opensearch.core.xcontent.ToXContentObject import org.opensearch.core.xcontent.XContentBuilder diff --git a/src/main/kotlin/org/opensearch/replication/util/Injectables.kt b/src/main/kotlin/org/opensearch/replication/util/Injectables.kt index 55f45690..88e4cd48 100644 --- a/src/main/kotlin/org/opensearch/replication/util/Injectables.kt +++ b/src/main/kotlin/org/opensearch/replication/util/Injectables.kt @@ -11,7 +11,7 @@ package org.opensearch.replication.util -import org.opensearch.common.component.AbstractLifecycleComponent +import org.opensearch.common.lifecycle.AbstractLifecycleComponent import org.opensearch.common.inject.Inject import org.opensearch.indices.IndicesService import org.opensearch.persistent.PersistentTasksService From 7385adeeebb92fb74d046007ad948167090634e2 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Tue, 8 Aug 2023 11:11:38 +0530 Subject: [PATCH 107/157] Move to core.common.Strings (#1098) Signed-off-by: monusingh-1 --- .../replication/action/stats/FollowerStatsResponse.kt | 3 +-- .../opensearch/replication/action/stats/LeaderStatsResponse.kt | 3 +-- .../replication/task/index/IndexReplicationParams.kt | 2 +- .../replication/task/shard/ShardReplicationParams.kt | 2 +- .../kotlin/org/opensearch/replication/util/ValidationUtil.kt | 2 +- .../org/opensearch/replication/MultiClusterRestTestCase.kt | 3 +-- 6 files changed, 6 insertions(+), 9 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/action/stats/FollowerStatsResponse.kt b/src/main/kotlin/org/opensearch/replication/action/stats/FollowerStatsResponse.kt index 9d4ae15c..3b4c4503 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stats/FollowerStatsResponse.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stats/FollowerStatsResponse.kt @@ -16,7 +16,6 @@ import org.apache.logging.log4j.LogManager import org.opensearch.action.FailedNodeException import org.opensearch.action.support.nodes.BaseNodesResponse import org.opensearch.cluster.ClusterName -import org.opensearch.common.Strings import org.opensearch.core.common.io.stream.StreamInput import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.ToXContent.EMPTY_PARAMS @@ -117,7 +116,7 @@ class FollowerStatsResponse : BaseNodesResponse, ToX override fun toString(): String { val builder: XContentBuilder = XContentFactory.jsonBuilder().prettyPrint() toXContent(builder, EMPTY_PARAMS) - return Strings.toString(builder) + return builder.toString() } } diff --git a/src/main/kotlin/org/opensearch/replication/action/stats/LeaderStatsResponse.kt b/src/main/kotlin/org/opensearch/replication/action/stats/LeaderStatsResponse.kt index 29abc78c..d3a54454 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stats/LeaderStatsResponse.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stats/LeaderStatsResponse.kt @@ -16,7 +16,6 @@ import org.apache.logging.log4j.LogManager import org.opensearch.action.FailedNodeException import org.opensearch.action.support.nodes.BaseNodesResponse import org.opensearch.cluster.ClusterName -import org.opensearch.common.Strings import org.opensearch.core.common.io.stream.StreamInput import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.ToXContent.EMPTY_PARAMS @@ -76,7 +75,7 @@ class LeaderStatsResponse : BaseNodesResponse, ToXCont override fun toString(): String { val builder: XContentBuilder = XContentFactory.jsonBuilder().prettyPrint() toXContent(builder, EMPTY_PARAMS) - return Strings.toString(builder) + return builder.toString() } } diff --git a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationParams.kt b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationParams.kt index 644b73e3..fc842f34 100644 --- a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationParams.kt +++ b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationParams.kt @@ -13,7 +13,7 @@ package org.opensearch.replication.task.index import org.opensearch.Version import org.opensearch.core.ParseField -import org.opensearch.common.Strings +import org.opensearch.core.common.Strings import org.opensearch.core.common.io.stream.StreamInput import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.ObjectParser diff --git a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationParams.kt b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationParams.kt index 95b22425..0f7c798b 100644 --- a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationParams.kt +++ b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationParams.kt @@ -13,7 +13,7 @@ package org.opensearch.replication.task.shard import org.opensearch.Version import org.opensearch.core.ParseField -import org.opensearch.common.Strings +import org.opensearch.core.common.Strings import org.opensearch.core.common.io.stream.StreamInput import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.core.xcontent.ObjectParser diff --git a/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt b/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt index 9f0bb4fc..0e2b4e62 100644 --- a/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt +++ b/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt @@ -17,7 +17,7 @@ import org.opensearch.Version import org.opensearch.cluster.ClusterState import org.opensearch.cluster.metadata.IndexMetadata import org.opensearch.cluster.metadata.MetadataCreateIndexService -import org.opensearch.common.Strings +import org.opensearch.core.common.Strings import org.opensearch.common.ValidationException import org.opensearch.common.settings.Settings import org.opensearch.env.Environment diff --git a/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt b/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt index d6981ad2..76519067 100644 --- a/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt +++ b/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt @@ -38,7 +38,6 @@ import org.opensearch.client.ResponseException import org.opensearch.client.RestClient import org.opensearch.client.RestClientBuilder import org.opensearch.client.RestHighLevelClient -import org.opensearch.common.Strings import org.opensearch.common.io.PathUtils import org.opensearch.common.settings.Settings import org.opensearch.common.unit.TimeValue @@ -419,7 +418,7 @@ abstract class MultiClusterRestTestCase : OpenSearchTestCase() { clearCommand.endObject() if (mustClear) { val request = Request("PUT", "/_cluster/settings") - request.setJsonEntity(Strings.toString(clearCommand)) + request.setJsonEntity(clearCommand.toString()) testCluster.lowLevelClient.performRequest(request) } } From 505e242e59023ea84ae20723299530654c398c23 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Wed, 16 Aug 2023 13:38:57 +0530 Subject: [PATCH 108/157] Refactor for upstream changes from PR import 9082 (#1102) Signed-off-by: monusingh-1 --- .../kotlin/org/opensearch/replication/ReplicationPlugin.kt | 2 +- .../TransportAutoFollowClusterManagerNodeAction.kt | 2 +- .../autofollow/TransportUpdateAutoFollowPatternAction.kt | 2 +- .../replication/action/changes/GetChangesResponse.kt | 2 +- .../replication/action/changes/TransportGetChangesAction.kt | 2 +- .../action/index/TransportReplicateIndexAction.kt | 2 +- .../TransportReplicateIndexClusterManagerNodeAction.kt | 2 +- .../action/index/block/TransportUpddateIndexBlockAction.kt | 2 +- .../action/pause/TransportPauseIndexReplicationAction.kt | 2 +- .../action/replay/TransportReplayChangesAction.kt | 2 +- .../TransportUpdateReplicationStateDetails.kt | 2 +- .../replication/action/repository/GetFileChunkResponse.kt | 2 +- .../action/repository/GetStoreMetadataResponse.kt | 2 +- .../action/resume/TransportResumeIndexReplicationAction.kt | 2 +- .../replication/action/setup/TransportSetupChecksAction.kt | 2 +- .../action/setup/TransportValidatePermissionsAction.kt | 2 +- .../action/stats/TransportAutoFollowStatsAction.kt | 2 +- .../action/status/TransportReplicationStatusAction.kt | 2 +- .../action/stop/TransportStopIndexReplicationAction.kt | 2 +- .../action/update/TransportUpdateIndexReplicationAction.kt | 2 +- .../replication/metadata/TransportUpdateMetadataAction.kt | 2 +- .../opensearch/replication/metadata/UpdateIndexBlockTask.kt | 2 +- .../replication/metadata/UpdateReplicationMetadata.kt | 2 +- .../repository/RemoteClusterMultiChunkTransfer.kt | 2 +- .../replication/repository/RemoteClusterRepository.kt | 4 ++-- .../replication/task/CrossClusterReplicationTask.kt | 6 +++--- .../replication/task/autofollow/AutoFollowExecutor.kt | 2 +- .../replication/task/autofollow/AutoFollowTask.kt | 2 +- .../replication/task/index/IndexReplicationExecutor.kt | 2 +- .../replication/task/index/IndexReplicationTask.kt | 4 ++-- .../replication/task/shard/ShardReplicationExecutor.kt | 2 +- .../replication/task/shard/ShardReplicationTask.kt | 2 +- .../opensearch/replication/task/shard/TranslogSequencer.kt | 2 +- .../kotlin/org/opensearch/replication/util/Coroutines.kt | 4 ++-- .../kotlin/org/opensearch/replication/util/Extensions.kt | 4 ++-- .../org/opensearch/replication/util/SecurityContext.kt | 2 +- .../replication/task/index/IndexReplicationTaskTests.kt | 2 +- .../org/opensearch/replication/task/index/NoOpClient.kt | 4 ++-- .../replication/task/shard/TranslogSequencerTests.kt | 6 +++--- 39 files changed, 48 insertions(+), 48 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt b/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt index 8a17fc51..ecc1f888 100644 --- a/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt +++ b/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt @@ -73,7 +73,7 @@ import org.opensearch.replication.task.shard.ShardReplicationParams import org.opensearch.replication.task.shard.ShardReplicationState import org.opensearch.replication.util.Injectables import org.opensearch.action.ActionRequest -import org.opensearch.action.ActionResponse +import org.opensearch.core.action.ActionResponse import org.opensearch.client.Client import org.opensearch.cluster.NamedDiff import org.opensearch.cluster.metadata.IndexNameExpressionResolver diff --git a/src/main/kotlin/org/opensearch/replication/action/autofollow/TransportAutoFollowClusterManagerNodeAction.kt b/src/main/kotlin/org/opensearch/replication/action/autofollow/TransportAutoFollowClusterManagerNodeAction.kt index cc93d88a..140a708d 100644 --- a/src/main/kotlin/org/opensearch/replication/action/autofollow/TransportAutoFollowClusterManagerNodeAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/autofollow/TransportAutoFollowClusterManagerNodeAction.kt @@ -28,7 +28,7 @@ import org.apache.logging.log4j.LogManager import org.opensearch.OpenSearchException import org.opensearch.ResourceAlreadyExistsException import org.opensearch.ResourceNotFoundException -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.action.support.ActionFilters import org.opensearch.action.support.master.AcknowledgedResponse import org.opensearch.action.support.master.TransportMasterNodeAction diff --git a/src/main/kotlin/org/opensearch/replication/action/autofollow/TransportUpdateAutoFollowPatternAction.kt b/src/main/kotlin/org/opensearch/replication/action/autofollow/TransportUpdateAutoFollowPatternAction.kt index 4cb32b1a..7b0f8c67 100644 --- a/src/main/kotlin/org/opensearch/replication/action/autofollow/TransportUpdateAutoFollowPatternAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/autofollow/TransportUpdateAutoFollowPatternAction.kt @@ -24,7 +24,7 @@ import kotlinx.coroutines.CoroutineScope import kotlinx.coroutines.GlobalScope import kotlinx.coroutines.launch import org.apache.logging.log4j.LogManager -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.action.support.ActionFilters import org.opensearch.action.support.HandledTransportAction import org.opensearch.action.support.master.AcknowledgedResponse diff --git a/src/main/kotlin/org/opensearch/replication/action/changes/GetChangesResponse.kt b/src/main/kotlin/org/opensearch/replication/action/changes/GetChangesResponse.kt index c71b4795..8de61312 100644 --- a/src/main/kotlin/org/opensearch/replication/action/changes/GetChangesResponse.kt +++ b/src/main/kotlin/org/opensearch/replication/action/changes/GetChangesResponse.kt @@ -11,7 +11,7 @@ package org.opensearch.replication.action.changes -import org.opensearch.action.ActionResponse +import org.opensearch.core.action.ActionResponse import org.opensearch.core.common.io.stream.StreamInput import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.index.translog.Translog diff --git a/src/main/kotlin/org/opensearch/replication/action/changes/TransportGetChangesAction.kt b/src/main/kotlin/org/opensearch/replication/action/changes/TransportGetChangesAction.kt index a3995999..dbee183d 100644 --- a/src/main/kotlin/org/opensearch/replication/action/changes/TransportGetChangesAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/changes/TransportGetChangesAction.kt @@ -15,7 +15,7 @@ import kotlinx.coroutines.GlobalScope import kotlinx.coroutines.launch import org.apache.logging.log4j.LogManager import org.opensearch.OpenSearchTimeoutException -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.action.support.ActionFilters import org.opensearch.action.support.single.shard.TransportSingleShardAction import org.opensearch.cluster.ClusterState diff --git a/src/main/kotlin/org/opensearch/replication/action/index/TransportReplicateIndexAction.kt b/src/main/kotlin/org/opensearch/replication/action/index/TransportReplicateIndexAction.kt index becb1360..8b2de1ea 100644 --- a/src/main/kotlin/org/opensearch/replication/action/index/TransportReplicateIndexAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/index/TransportReplicateIndexAction.kt @@ -26,7 +26,7 @@ import kotlinx.coroutines.CoroutineScope import kotlinx.coroutines.GlobalScope import kotlinx.coroutines.launch import org.apache.logging.log4j.LogManager -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.action.admin.indices.settings.get.GetSettingsRequest import org.opensearch.action.support.ActionFilters import org.opensearch.action.support.HandledTransportAction diff --git a/src/main/kotlin/org/opensearch/replication/action/index/TransportReplicateIndexClusterManagerNodeAction.kt b/src/main/kotlin/org/opensearch/replication/action/index/TransportReplicateIndexClusterManagerNodeAction.kt index 36fe81f2..042509f2 100644 --- a/src/main/kotlin/org/opensearch/replication/action/index/TransportReplicateIndexClusterManagerNodeAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/index/TransportReplicateIndexClusterManagerNodeAction.kt @@ -27,7 +27,7 @@ import kotlinx.coroutines.GlobalScope import kotlinx.coroutines.launch import org.apache.logging.log4j.LogManager import org.opensearch.OpenSearchStatusException -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.action.support.ActionFilters import org.opensearch.action.support.IndicesOptions import org.opensearch.action.support.master.AcknowledgedResponse diff --git a/src/main/kotlin/org/opensearch/replication/action/index/block/TransportUpddateIndexBlockAction.kt b/src/main/kotlin/org/opensearch/replication/action/index/block/TransportUpddateIndexBlockAction.kt index 55e569aa..c2e58bdd 100644 --- a/src/main/kotlin/org/opensearch/replication/action/index/block/TransportUpddateIndexBlockAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/index/block/TransportUpddateIndexBlockAction.kt @@ -20,7 +20,7 @@ import kotlinx.coroutines.GlobalScope import kotlinx.coroutines.launch import org.apache.logging.log4j.LogManager import org.opensearch.OpenSearchException -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.action.support.ActionFilters import org.opensearch.action.support.master.AcknowledgedResponse import org.opensearch.action.support.master.TransportMasterNodeAction diff --git a/src/main/kotlin/org/opensearch/replication/action/pause/TransportPauseIndexReplicationAction.kt b/src/main/kotlin/org/opensearch/replication/action/pause/TransportPauseIndexReplicationAction.kt index eaf828a1..c4663685 100644 --- a/src/main/kotlin/org/opensearch/replication/action/pause/TransportPauseIndexReplicationAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/pause/TransportPauseIndexReplicationAction.kt @@ -22,7 +22,7 @@ import kotlinx.coroutines.launch import org.apache.logging.log4j.LogManager import org.opensearch.OpenSearchException import org.opensearch.ResourceAlreadyExistsException -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.action.support.ActionFilters import org.opensearch.action.support.master.AcknowledgedRequest import org.opensearch.action.support.master.AcknowledgedResponse diff --git a/src/main/kotlin/org/opensearch/replication/action/replay/TransportReplayChangesAction.kt b/src/main/kotlin/org/opensearch/replication/action/replay/TransportReplayChangesAction.kt index c874e92f..fbf9e181 100644 --- a/src/main/kotlin/org/opensearch/replication/action/replay/TransportReplayChangesAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/replay/TransportReplayChangesAction.kt @@ -25,7 +25,7 @@ import kotlinx.coroutines.SupervisorJob import kotlinx.coroutines.asCoroutineDispatcher import kotlinx.coroutines.launch import org.apache.logging.log4j.LogManager -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.action.admin.indices.mapping.get.GetMappingsRequest import org.opensearch.action.admin.indices.mapping.put.PutMappingRequest import org.opensearch.action.index.IndexRequest diff --git a/src/main/kotlin/org/opensearch/replication/action/replicationstatedetails/TransportUpdateReplicationStateDetails.kt b/src/main/kotlin/org/opensearch/replication/action/replicationstatedetails/TransportUpdateReplicationStateDetails.kt index 0b744482..bc629db7 100644 --- a/src/main/kotlin/org/opensearch/replication/action/replicationstatedetails/TransportUpdateReplicationStateDetails.kt +++ b/src/main/kotlin/org/opensearch/replication/action/replicationstatedetails/TransportUpdateReplicationStateDetails.kt @@ -18,7 +18,7 @@ import org.opensearch.replication.util.submitClusterStateUpdateTask import kotlinx.coroutines.CoroutineScope import kotlinx.coroutines.GlobalScope import kotlinx.coroutines.launch -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.action.support.ActionFilters import org.opensearch.action.support.master.AcknowledgedRequest import org.opensearch.action.support.master.AcknowledgedResponse diff --git a/src/main/kotlin/org/opensearch/replication/action/repository/GetFileChunkResponse.kt b/src/main/kotlin/org/opensearch/replication/action/repository/GetFileChunkResponse.kt index 325e20f0..6f432c4b 100644 --- a/src/main/kotlin/org/opensearch/replication/action/repository/GetFileChunkResponse.kt +++ b/src/main/kotlin/org/opensearch/replication/action/repository/GetFileChunkResponse.kt @@ -11,7 +11,7 @@ package org.opensearch.replication.action.repository -import org.opensearch.action.ActionResponse +import org.opensearch.core.action.ActionResponse import org.opensearch.core.common.bytes.BytesReference import org.opensearch.core.common.io.stream.StreamInput import org.opensearch.core.common.io.stream.StreamOutput diff --git a/src/main/kotlin/org/opensearch/replication/action/repository/GetStoreMetadataResponse.kt b/src/main/kotlin/org/opensearch/replication/action/repository/GetStoreMetadataResponse.kt index 9ef97259..ad3d66ab 100644 --- a/src/main/kotlin/org/opensearch/replication/action/repository/GetStoreMetadataResponse.kt +++ b/src/main/kotlin/org/opensearch/replication/action/repository/GetStoreMetadataResponse.kt @@ -11,7 +11,7 @@ package org.opensearch.replication.action.repository -import org.opensearch.action.ActionResponse +import org.opensearch.core.action.ActionResponse import org.opensearch.core.common.io.stream.StreamInput import org.opensearch.core.common.io.stream.StreamOutput import org.opensearch.index.store.Store diff --git a/src/main/kotlin/org/opensearch/replication/action/resume/TransportResumeIndexReplicationAction.kt b/src/main/kotlin/org/opensearch/replication/action/resume/TransportResumeIndexReplicationAction.kt index a1128913..d748d479 100644 --- a/src/main/kotlin/org/opensearch/replication/action/resume/TransportResumeIndexReplicationAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/resume/TransportResumeIndexReplicationAction.kt @@ -35,7 +35,7 @@ import kotlinx.coroutines.launch import org.apache.logging.log4j.LogManager import org.opensearch.ResourceAlreadyExistsException import org.opensearch.ResourceNotFoundException -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.action.admin.indices.settings.get.GetSettingsRequest import org.opensearch.action.support.ActionFilters import org.opensearch.action.support.IndicesOptions diff --git a/src/main/kotlin/org/opensearch/replication/action/setup/TransportSetupChecksAction.kt b/src/main/kotlin/org/opensearch/replication/action/setup/TransportSetupChecksAction.kt index 0b1169b6..ca495da5 100644 --- a/src/main/kotlin/org/opensearch/replication/action/setup/TransportSetupChecksAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/setup/TransportSetupChecksAction.kt @@ -16,7 +16,7 @@ import org.opensearch.replication.util.SecurityContext import org.apache.logging.log4j.LogManager import org.opensearch.OpenSearchSecurityException import org.opensearch.ExceptionsHelper -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.action.StepListener import org.opensearch.action.support.ActionFilters import org.opensearch.action.support.HandledTransportAction diff --git a/src/main/kotlin/org/opensearch/replication/action/setup/TransportValidatePermissionsAction.kt b/src/main/kotlin/org/opensearch/replication/action/setup/TransportValidatePermissionsAction.kt index 2b746c1a..a592bc55 100644 --- a/src/main/kotlin/org/opensearch/replication/action/setup/TransportValidatePermissionsAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/setup/TransportValidatePermissionsAction.kt @@ -13,7 +13,7 @@ package org.opensearch.replication.action.setup import org.opensearch.replication.util.completeWith import org.apache.logging.log4j.LogManager -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.action.support.ActionFilters import org.opensearch.action.support.HandledTransportAction import org.opensearch.action.support.master.AcknowledgedResponse diff --git a/src/main/kotlin/org/opensearch/replication/action/stats/TransportAutoFollowStatsAction.kt b/src/main/kotlin/org/opensearch/replication/action/stats/TransportAutoFollowStatsAction.kt index 8176a478..677039ca 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stats/TransportAutoFollowStatsAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stats/TransportAutoFollowStatsAction.kt @@ -14,7 +14,7 @@ package org.opensearch.replication.action.stats import kotlinx.coroutines.CoroutineScope import kotlinx.coroutines.GlobalScope import org.apache.logging.log4j.LogManager -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.action.FailedNodeException import org.opensearch.action.TaskOperationFailure import org.opensearch.action.support.ActionFilters diff --git a/src/main/kotlin/org/opensearch/replication/action/status/TransportReplicationStatusAction.kt b/src/main/kotlin/org/opensearch/replication/action/status/TransportReplicationStatusAction.kt index 317fb621..76cc1153 100644 --- a/src/main/kotlin/org/opensearch/replication/action/status/TransportReplicationStatusAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/status/TransportReplicationStatusAction.kt @@ -20,7 +20,7 @@ import kotlinx.coroutines.GlobalScope import kotlinx.coroutines.launch import org.apache.logging.log4j.LogManager import org.opensearch.ResourceNotFoundException -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.action.support.ActionFilters import org.opensearch.action.support.HandledTransportAction import org.opensearch.client.Client diff --git a/src/main/kotlin/org/opensearch/replication/action/stop/TransportStopIndexReplicationAction.kt b/src/main/kotlin/org/opensearch/replication/action/stop/TransportStopIndexReplicationAction.kt index f18cbb30..d9af050a 100644 --- a/src/main/kotlin/org/opensearch/replication/action/stop/TransportStopIndexReplicationAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/stop/TransportStopIndexReplicationAction.kt @@ -33,7 +33,7 @@ import kotlinx.coroutines.GlobalScope import kotlinx.coroutines.launch import org.apache.logging.log4j.LogManager import org.opensearch.OpenSearchException -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.action.admin.indices.open.OpenIndexRequest import org.opensearch.action.support.ActionFilters import org.opensearch.action.support.master.AcknowledgedResponse diff --git a/src/main/kotlin/org/opensearch/replication/action/update/TransportUpdateIndexReplicationAction.kt b/src/main/kotlin/org/opensearch/replication/action/update/TransportUpdateIndexReplicationAction.kt index 333ed9f7..4baf7a1d 100644 --- a/src/main/kotlin/org/opensearch/replication/action/update/TransportUpdateIndexReplicationAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/update/TransportUpdateIndexReplicationAction.kt @@ -22,7 +22,7 @@ import kotlinx.coroutines.Dispatchers import kotlinx.coroutines.GlobalScope import kotlinx.coroutines.launch import org.apache.logging.log4j.LogManager -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.action.support.ActionFilters import org.opensearch.action.support.master.AcknowledgedResponse import org.opensearch.action.support.master.TransportMasterNodeAction diff --git a/src/main/kotlin/org/opensearch/replication/metadata/TransportUpdateMetadataAction.kt b/src/main/kotlin/org/opensearch/replication/metadata/TransportUpdateMetadataAction.kt index 21b1643f..6f2ecfa7 100644 --- a/src/main/kotlin/org/opensearch/replication/metadata/TransportUpdateMetadataAction.kt +++ b/src/main/kotlin/org/opensearch/replication/metadata/TransportUpdateMetadataAction.kt @@ -13,7 +13,7 @@ package org.opensearch.replication.metadata import org.apache.logging.log4j.LogManager import org.apache.logging.log4j.message.ParameterizedMessage -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.action.IndicesRequest import org.opensearch.action.admin.indices.alias.IndicesAliasesClusterStateUpdateRequest import org.opensearch.action.admin.indices.alias.IndicesAliasesRequest diff --git a/src/main/kotlin/org/opensearch/replication/metadata/UpdateIndexBlockTask.kt b/src/main/kotlin/org/opensearch/replication/metadata/UpdateIndexBlockTask.kt index e67293b6..38f2d85a 100644 --- a/src/main/kotlin/org/opensearch/replication/metadata/UpdateIndexBlockTask.kt +++ b/src/main/kotlin/org/opensearch/replication/metadata/UpdateIndexBlockTask.kt @@ -13,7 +13,7 @@ package org.opensearch.replication.metadata import org.opensearch.replication.action.index.block.IndexBlockUpdateType import org.opensearch.replication.action.index.block.UpdateIndexBlockRequest -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.action.support.master.AcknowledgedResponse import org.opensearch.cluster.AckedClusterStateUpdateTask import org.opensearch.cluster.ClusterState diff --git a/src/main/kotlin/org/opensearch/replication/metadata/UpdateReplicationMetadata.kt b/src/main/kotlin/org/opensearch/replication/metadata/UpdateReplicationMetadata.kt index 54d4663e..5c69a10a 100644 --- a/src/main/kotlin/org/opensearch/replication/metadata/UpdateReplicationMetadata.kt +++ b/src/main/kotlin/org/opensearch/replication/metadata/UpdateReplicationMetadata.kt @@ -14,7 +14,7 @@ package org.opensearch.replication.metadata import org.opensearch.replication.action.replicationstatedetails.UpdateReplicationStateDetailsRequest import org.opensearch.replication.metadata.state.ReplicationStateMetadata import org.apache.logging.log4j.LogManager -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.cluster.AckedClusterStateUpdateTask import org.opensearch.cluster.ClusterState import org.opensearch.cluster.ClusterStateTaskExecutor diff --git a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterMultiChunkTransfer.kt b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterMultiChunkTransfer.kt index 344fbcbb..e1460e0b 100644 --- a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterMultiChunkTransfer.kt +++ b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterMultiChunkTransfer.kt @@ -23,7 +23,7 @@ import kotlinx.coroutines.launch import kotlinx.coroutines.sync.Mutex import kotlinx.coroutines.sync.withLock import org.apache.logging.log4j.Logger -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.client.Client import org.opensearch.cluster.node.DiscoveryNode import org.opensearch.core.common.unit.ByteSizeValue diff --git a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt index 366b9ee0..929d6585 100644 --- a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt +++ b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt @@ -30,9 +30,9 @@ import kotlinx.coroutines.Dispatchers import org.apache.logging.log4j.LogManager import org.apache.lucene.index.IndexCommit import org.opensearch.Version -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.action.ActionRequest -import org.opensearch.action.ActionResponse +import org.opensearch.core.action.ActionResponse import org.opensearch.action.ActionType import org.opensearch.action.admin.indices.stats.IndicesStatsAction import org.opensearch.action.admin.indices.stats.IndicesStatsRequest diff --git a/src/main/kotlin/org/opensearch/replication/task/CrossClusterReplicationTask.kt b/src/main/kotlin/org/opensearch/replication/task/CrossClusterReplicationTask.kt index 7f03dd79..9f39da74 100644 --- a/src/main/kotlin/org/opensearch/replication/task/CrossClusterReplicationTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/CrossClusterReplicationTask.kt @@ -30,8 +30,8 @@ import kotlinx.coroutines.withTimeoutOrNull import kotlinx.coroutines.ObsoleteCoroutinesApi import org.apache.logging.log4j.Logger import org.opensearch.OpenSearchException -import org.opensearch.action.ActionListener -import org.opensearch.action.ActionResponse +import org.opensearch.core.action.ActionListener +import org.opensearch.core.action.ActionResponse import org.opensearch.client.Client import org.opensearch.cluster.service.ClusterService import org.opensearch.core.common.io.stream.StreamOutput @@ -48,7 +48,7 @@ import org.opensearch.persistent.PersistentTaskState import org.opensearch.persistent.PersistentTasksService import org.opensearch.replication.util.stackTraceToString import org.opensearch.core.rest.RestStatus -import org.opensearch.tasks.TaskId +import org.opensearch.core.tasks.TaskId import org.opensearch.tasks.TaskManager import org.opensearch.threadpool.ThreadPool diff --git a/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowExecutor.kt b/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowExecutor.kt index afa8a30e..4faebf45 100644 --- a/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowExecutor.kt +++ b/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowExecutor.kt @@ -19,7 +19,7 @@ import org.opensearch.persistent.AllocatedPersistentTask import org.opensearch.persistent.PersistentTaskState import org.opensearch.persistent.PersistentTasksCustomMetadata.PersistentTask import org.opensearch.persistent.PersistentTasksExecutor -import org.opensearch.tasks.TaskId +import org.opensearch.core.tasks.TaskId import org.opensearch.threadpool.ThreadPool class AutoFollowExecutor(executor: String, private val clusterService: ClusterService, diff --git a/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowTask.kt b/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowTask.kt index d47843d6..ca376e59 100644 --- a/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/autofollow/AutoFollowTask.kt @@ -41,7 +41,7 @@ import org.opensearch.replication.action.status.ShardInfoRequest import org.opensearch.replication.action.status.ShardInfoResponse import org.opensearch.core.rest.RestStatus import org.opensearch.tasks.Task -import org.opensearch.tasks.TaskId +import org.opensearch.core.tasks.TaskId import org.opensearch.threadpool.Scheduler import org.opensearch.threadpool.ThreadPool import java.util.concurrent.ConcurrentSkipListSet diff --git a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationExecutor.kt b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationExecutor.kt index 8c2e88ca..72b1484f 100644 --- a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationExecutor.kt +++ b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationExecutor.kt @@ -27,7 +27,7 @@ import org.opensearch.persistent.AllocatedPersistentTask import org.opensearch.persistent.PersistentTaskState import org.opensearch.persistent.PersistentTasksCustomMetadata.PersistentTask import org.opensearch.persistent.PersistentTasksExecutor -import org.opensearch.tasks.TaskId +import org.opensearch.core.tasks.TaskId import org.opensearch.threadpool.ThreadPool class IndexReplicationExecutor(executor: String, private val clusterService: ClusterService, diff --git a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt index ce3cb43a..d0c812a7 100644 --- a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt @@ -48,7 +48,7 @@ import kotlinx.coroutines.launch import kotlinx.coroutines.withContext import org.opensearch.OpenSearchException import org.opensearch.OpenSearchTimeoutException -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.action.admin.indices.alias.IndicesAliasesRequest import org.opensearch.action.admin.indices.alias.IndicesAliasesRequest.AliasActions import org.opensearch.action.admin.indices.alias.get.GetAliasesRequest @@ -95,7 +95,7 @@ import org.opensearch.replication.ReplicationException import org.opensearch.replication.MappingNotAvailableException import org.opensearch.replication.ReplicationPlugin.Companion.REPLICATION_INDEX_TRANSLOG_PRUNING_ENABLED_SETTING import org.opensearch.core.rest.RestStatus -import org.opensearch.tasks.TaskId +import org.opensearch.core.tasks.TaskId import org.opensearch.tasks.TaskManager import org.opensearch.threadpool.ThreadPool import java.util.Collections diff --git a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationExecutor.kt b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationExecutor.kt index 11be6056..526935ef 100644 --- a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationExecutor.kt +++ b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationExecutor.kt @@ -26,7 +26,7 @@ import org.opensearch.persistent.PersistentTaskState import org.opensearch.persistent.PersistentTasksCustomMetadata.Assignment import org.opensearch.persistent.PersistentTasksCustomMetadata.PersistentTask import org.opensearch.persistent.PersistentTasksExecutor -import org.opensearch.tasks.TaskId +import org.opensearch.core.tasks.TaskId import org.opensearch.threadpool.ThreadPool class ShardReplicationExecutor(executor: String, private val clusterService : ClusterService, diff --git a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationTask.kt b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationTask.kt index e165c916..c41ee084 100644 --- a/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/shard/ShardReplicationTask.kt @@ -51,7 +51,7 @@ import org.opensearch.core.index.shard.ShardId import org.opensearch.persistent.PersistentTaskState import org.opensearch.persistent.PersistentTasksNodeService import org.opensearch.core.rest.RestStatus -import org.opensearch.tasks.TaskId +import org.opensearch.core.tasks.TaskId import org.opensearch.threadpool.ThreadPool import org.opensearch.transport.NodeNotConnectedException import java.time.Duration diff --git a/src/main/kotlin/org/opensearch/replication/task/shard/TranslogSequencer.kt b/src/main/kotlin/org/opensearch/replication/task/shard/TranslogSequencer.kt index ad4e7681..1c02f747 100644 --- a/src/main/kotlin/org/opensearch/replication/task/shard/TranslogSequencer.kt +++ b/src/main/kotlin/org/opensearch/replication/task/shard/TranslogSequencer.kt @@ -32,7 +32,7 @@ import org.opensearch.index.IndexNotFoundException import org.opensearch.core.index.shard.ShardId import org.opensearch.index.translog.Translog import org.opensearch.replication.util.indicesService -import org.opensearch.tasks.TaskId +import org.opensearch.core.tasks.TaskId import java.util.ArrayList import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.TimeUnit diff --git a/src/main/kotlin/org/opensearch/replication/util/Coroutines.kt b/src/main/kotlin/org/opensearch/replication/util/Coroutines.kt index 47e7b723..b561c795 100644 --- a/src/main/kotlin/org/opensearch/replication/util/Coroutines.kt +++ b/src/main/kotlin/org/opensearch/replication/util/Coroutines.kt @@ -15,9 +15,9 @@ import org.opensearch.replication.metadata.store.ReplicationMetadata import kotlinx.coroutines.* import org.opensearch.OpenSearchTimeoutException import org.opensearch.ExceptionsHelper -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.action.ActionRequest -import org.opensearch.action.ActionResponse +import org.opensearch.core.action.ActionResponse import org.opensearch.action.ActionType import org.opensearch.action.support.master.AcknowledgedRequest import org.opensearch.action.support.master.MasterNodeRequest diff --git a/src/main/kotlin/org/opensearch/replication/util/Extensions.kt b/src/main/kotlin/org/opensearch/replication/util/Extensions.kt index 1928ad6b..7a73fa5f 100644 --- a/src/main/kotlin/org/opensearch/replication/util/Extensions.kt +++ b/src/main/kotlin/org/opensearch/replication/util/Extensions.kt @@ -20,9 +20,9 @@ import org.apache.logging.log4j.Logger import org.opensearch.OpenSearchException import org.opensearch.OpenSearchSecurityException import org.opensearch.ResourceNotFoundException -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.action.ActionRequest -import org.opensearch.action.ActionResponse +import org.opensearch.core.action.ActionResponse import org.opensearch.action.ActionType import org.opensearch.action.index.IndexRequestBuilder import org.opensearch.action.index.IndexResponse diff --git a/src/main/kotlin/org/opensearch/replication/util/SecurityContext.kt b/src/main/kotlin/org/opensearch/replication/util/SecurityContext.kt index 2490dd0c..f811324a 100644 --- a/src/main/kotlin/org/opensearch/replication/util/SecurityContext.kt +++ b/src/main/kotlin/org/opensearch/replication/util/SecurityContext.kt @@ -29,7 +29,7 @@ import org.opensearch.commons.ConfigConstants import org.opensearch.commons.authuser.User import org.apache.logging.log4j.LogManager import org.opensearch.action.ActionRequest -import org.opensearch.action.ActionResponse +import org.opensearch.core.action.ActionResponse import org.opensearch.action.ActionType import org.opensearch.common.util.concurrent.ThreadContext import org.opensearch.transport.RemoteClusterAwareRequest diff --git a/src/test/kotlin/org/opensearch/replication/task/index/IndexReplicationTaskTests.kt b/src/test/kotlin/org/opensearch/replication/task/index/IndexReplicationTaskTests.kt index 00bde557..b1e44f39 100644 --- a/src/test/kotlin/org/opensearch/replication/task/index/IndexReplicationTaskTests.kt +++ b/src/test/kotlin/org/opensearch/replication/task/index/IndexReplicationTaskTests.kt @@ -52,7 +52,7 @@ import org.opensearch.replication.task.shard.ShardReplicationExecutor import org.opensearch.replication.task.shard.ShardReplicationParams import org.opensearch.snapshots.Snapshot import org.opensearch.snapshots.SnapshotId -import org.opensearch.tasks.TaskId.EMPTY_TASK_ID +import org.opensearch.core.tasks.TaskId.EMPTY_TASK_ID import org.opensearch.tasks.TaskManager import org.opensearch.test.ClusterServiceUtils import org.opensearch.test.ClusterServiceUtils.setState diff --git a/src/test/kotlin/org/opensearch/replication/task/index/NoOpClient.kt b/src/test/kotlin/org/opensearch/replication/task/index/NoOpClient.kt index 606d7a18..6f1904fd 100644 --- a/src/test/kotlin/org/opensearch/replication/task/index/NoOpClient.kt +++ b/src/test/kotlin/org/opensearch/replication/task/index/NoOpClient.kt @@ -12,9 +12,9 @@ package org.opensearch.replication.task.index import com.nhaarman.mockitokotlin2.doReturn import org.mockito.Mockito -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.action.ActionRequest -import org.opensearch.action.ActionResponse +import org.opensearch.core.action.ActionResponse import org.opensearch.action.ActionType import org.opensearch.action.admin.cluster.health.ClusterHealthAction import org.opensearch.action.admin.cluster.health.ClusterHealthResponse diff --git a/src/test/kotlin/org/opensearch/replication/task/shard/TranslogSequencerTests.kt b/src/test/kotlin/org/opensearch/replication/task/shard/TranslogSequencerTests.kt index 730289ed..e33e0f87 100644 --- a/src/test/kotlin/org/opensearch/replication/task/shard/TranslogSequencerTests.kt +++ b/src/test/kotlin/org/opensearch/replication/task/shard/TranslogSequencerTests.kt @@ -16,9 +16,9 @@ import kotlinx.coroutines.ObsoleteCoroutinesApi import kotlinx.coroutines.test.runBlockingTest import org.assertj.core.api.Assertions.assertThat import org.mockito.Mockito -import org.opensearch.action.ActionListener +import org.opensearch.core.action.ActionListener import org.opensearch.action.ActionRequest -import org.opensearch.action.ActionResponse +import org.opensearch.core.action.ActionResponse import org.opensearch.action.ActionType import org.opensearch.action.support.replication.ReplicationResponse.ShardInfo import org.opensearch.common.settings.Settings @@ -36,7 +36,7 @@ import org.opensearch.replication.metadata.store.ReplicationContext import org.opensearch.replication.metadata.store.ReplicationMetadata import org.opensearch.replication.metadata.store.ReplicationStoreMetadataType import org.opensearch.replication.util.indicesService -import org.opensearch.tasks.TaskId.EMPTY_TASK_ID +import org.opensearch.core.tasks.TaskId.EMPTY_TASK_ID import org.opensearch.test.OpenSearchTestCase import org.opensearch.test.client.NoOpClient import java.util.Locale From 81b37ba67efedbbebc804214e7384620006bc4da Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Thu, 17 Aug 2023 15:48:18 +0530 Subject: [PATCH 109/157] Fix test for single node and consume numNodes (#1091) Signed-off-by: monusingh-1 --- build.gradle | 3 +++ .../replication/MultiClusterRestTestCase.kt | 18 ++++++++++---- .../replication/ReplicationHelpers.kt | 3 +++ .../integ/rest/ClusterRerouteFollowerIT.kt | 4 ++-- .../integ/rest/ClusterRerouteLeaderIT.kt | 3 ++- .../integ/rest/ResumeReplicationIT.kt | 15 ++++-------- .../integ/rest/StartReplicationIT.kt | 24 +++++++++---------- .../integ/rest/StopReplicationIT.kt | 6 ++--- 8 files changed, 42 insertions(+), 34 deletions(-) diff --git a/build.gradle b/build.gradle index 426993aa..aa4422a7 100644 --- a/build.gradle +++ b/build.gradle @@ -931,6 +931,9 @@ task integTestRemote (type: RestIntegTestTask) { systemProperty "tests.cluster.leaderCluster.security_enabled", System.getProperty("security_enabled") nonInputProperties.systemProperty('tests.integTestRemote', "true") + var numberOfNodes = findProperty('numNodes') as Integer + systemProperty "tests.cluster.followCluster.total_nodes", "${-> numberOfNodes.toString()}" + systemProperty "tests.cluster.leaderCluster.total_nodes", "${-> numberOfNodes.toString()}" systemProperty "build.dir", "${buildDir}" } diff --git a/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt b/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt index 76519067..eca8750f 100644 --- a/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt +++ b/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt @@ -119,7 +119,6 @@ abstract class MultiClusterRestTestCase : OpenSearchTestCase() { lateinit var testClusters : Map var isSecurityPropertyEnabled = false var forceInitSecurityConfiguration = false - var isMultiNodeClusterConfiguration = true internal fun createTestCluster(configuration: ClusterConfiguration) : TestCluster { return createTestCluster(configuration.clusterName, configuration.preserveSnapshots, configuration.preserveIndices, @@ -132,7 +131,6 @@ abstract class MultiClusterRestTestCase : OpenSearchTestCase() { val httpHostsProp = systemProperties.get("tests.cluster.${cluster}.http_hosts") as String? val transportHostsProp = systemProperties.get("tests.cluster.${cluster}.transport_hosts") as String? val securityEnabled = systemProperties.get("tests.cluster.${cluster}.security_enabled") as String? - val totalNodes = systemProperties.get("tests.cluster.${cluster}.total_nodes") as String? requireNotNull(httpHostsProp) { "Missing http hosts property for cluster: $cluster."} requireNotNull(transportHostsProp) { "Missing transport hosts property for cluster: $cluster."} @@ -144,9 +142,6 @@ abstract class MultiClusterRestTestCase : OpenSearchTestCase() { isSecurityPropertyEnabled = true } - if(totalNodes != null && totalNodes < "2") { - isMultiNodeClusterConfiguration = false - } forceInitSecurityConfiguration = isSecurityPropertyEnabled && initSecurityConfiguration @@ -666,6 +661,19 @@ abstract class MultiClusterRestTestCase : OpenSearchTestCase() { return integTestRemote.equals("true") } + protected fun isMultiNodeClusterConfiguration(leaderCluster: String, followerCluster: String): Boolean{ + val systemProperties = BootstrapInfo.getSystemProperties() + val totalLeaderNodes = systemProperties.get("tests.cluster.${leaderCluster}.total_nodes") as String + val totalFollowerNodes = systemProperties.get("tests.cluster.${followerCluster}.total_nodes") as String + + assertNotNull(totalLeaderNodes) + assertNotNull(totalFollowerNodes) + if(totalLeaderNodes < "2" || totalFollowerNodes < "2" ) { + return false + } + return true + } + protected fun docCount(cluster: RestHighLevelClient, indexName: String) : Int { val persistentConnectionRequest = Request("GET", "/$indexName/_search?pretty&q=*") diff --git a/src/test/kotlin/org/opensearch/replication/ReplicationHelpers.kt b/src/test/kotlin/org/opensearch/replication/ReplicationHelpers.kt index 2d42a0d2..892470f5 100644 --- a/src/test/kotlin/org/opensearch/replication/ReplicationHelpers.kt +++ b/src/test/kotlin/org/opensearch/replication/ReplicationHelpers.kt @@ -55,6 +55,9 @@ const val INDEX_TASK_CANCELLATION_REASON = "AutoPaused: Index replication task w const val STATUS_REASON_USER_INITIATED = "User initiated" const val STATUS_REASON_SHARD_TASK_CANCELLED = "Shard task killed or cancelled." const val STATUS_REASON_INDEX_NOT_FOUND = "no such index" +const val ANALYZERS_NOT_ACCESSIBLE_FOR_REMOTE_CLUSTERS = "Analysers are not accessible when run on remote clusters." +const val SNAPSHOTS_NOT_ACCESSIBLE_FOR_REMOTE_CLUSTERS = "Snapshots are not accessible when run on remote clusters." +const val REROUTE_TESTS_NOT_ELIGIBLE_FOR_SINGLE_NODE_CLUSTER = "Reroute not eligible for single node clusters" fun RestHighLevelClient.startReplication(request: StartReplicationRequest, diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/ClusterRerouteFollowerIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/ClusterRerouteFollowerIT.kt index 77cb164e..a9bd76ac 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/ClusterRerouteFollowerIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/ClusterRerouteFollowerIT.kt @@ -4,7 +4,7 @@ import org.opensearch.replication.MultiClusterRestTestCase import org.opensearch.replication.MultiClusterAnnotations import org.opensearch.replication.StartReplicationRequest import org.opensearch.replication.startReplication -import org.opensearch.replication.stopReplication +import org.opensearch.replication.REROUTE_TESTS_NOT_ELIGIBLE_FOR_SINGLE_NODE_CLUSTER import org.assertj.core.api.Assertions import org.opensearch.client.RequestOptions import org.opensearch.client.indices.CreateIndexRequest @@ -25,7 +25,7 @@ class ClusterRerouteFollowerIT : MultiClusterRestTestCase() { @Before fun beforeTest() { - Assume.assumeTrue(isMultiNodeClusterConfiguration) + Assume.assumeTrue(REROUTE_TESTS_NOT_ELIGIBLE_FOR_SINGLE_NODE_CLUSTER, isMultiNodeClusterConfiguration(LEADER, FOLLOWER)) } fun `test replication works after rerouting a shard from one node to another in follower cluster`() { diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/ClusterRerouteLeaderIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/ClusterRerouteLeaderIT.kt index 36fce5a5..e343d7a8 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/ClusterRerouteLeaderIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/ClusterRerouteLeaderIT.kt @@ -17,6 +17,7 @@ import org.junit.Assert import org.junit.Assume import org.junit.Before import org.junit.Ignore +import org.opensearch.replication.REROUTE_TESTS_NOT_ELIGIBLE_FOR_SINGLE_NODE_CLUSTER import java.util.concurrent.TimeUnit @MultiClusterAnnotations.ClusterConfigurations( @@ -30,7 +31,7 @@ class ClusterRerouteLeaderIT : MultiClusterRestTestCase() { @Before fun beforeTest() { - Assume.assumeTrue(isMultiNodeClusterConfiguration) + Assume.assumeTrue(REROUTE_TESTS_NOT_ELIGIBLE_FOR_SINGLE_NODE_CLUSTER, isMultiNodeClusterConfiguration(LEADER, FOLLOWER),) } fun `test replication works after rerouting a shard from one node to another in leader cluster`() { diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/ResumeReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/ResumeReplicationIT.kt index 2a3b3bae..deca986c 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/ResumeReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/ResumeReplicationIT.kt @@ -40,9 +40,10 @@ import org.opensearch.client.indices.GetMappingsRequest import org.opensearch.common.io.PathUtils import org.opensearch.common.settings.Settings import org.junit.Assert +import org.junit.Assume import java.nio.file.Files import java.util.concurrent.TimeUnit -import org.opensearch.bootstrap.BootstrapInfo +import org.opensearch.replication.ANALYZERS_NOT_ACCESSIBLE_FOR_REMOTE_CLUSTERS @MultiClusterAnnotations.ClusterConfigurations( MultiClusterAnnotations.ClusterConfiguration(clusterName = LEADER), @@ -165,9 +166,7 @@ class ResumeReplicationIT: MultiClusterRestTestCase() { fun `test that replication fails to resume when custom analyser is not present in follower`() { - if(checkifIntegTestRemote()){ - return; - } + Assume.assumeFalse(ANALYZERS_NOT_ACCESSIBLE_FOR_REMOTE_CLUSTERS, checkifIntegTestRemote()) val synonyms = javaClass.getResourceAsStream("/analyzers/synonyms.txt") val config = PathUtils.get(buildDir, leaderClusterPath, "config") @@ -202,9 +201,7 @@ class ResumeReplicationIT: MultiClusterRestTestCase() { fun `test that replication resumes when custom analyser is present in follower`() { - if(checkifIntegTestRemote()){ - return; - } + Assume.assumeFalse(ANALYZERS_NOT_ACCESSIBLE_FOR_REMOTE_CLUSTERS, checkifIntegTestRemote()) val synonyms = javaClass.getResourceAsStream("/analyzers/synonyms.txt") val config = PathUtils.get(buildDir, leaderClusterPath, "config") @@ -246,9 +243,7 @@ class ResumeReplicationIT: MultiClusterRestTestCase() { fun `test that replication resumes when custom analyser is overridden and present in follower`() { - if(checkifIntegTestRemote()){ - return; - } + Assume.assumeFalse(ANALYZERS_NOT_ACCESSIBLE_FOR_REMOTE_CLUSTERS, checkifIntegTestRemote()) val synonyms = javaClass.getResourceAsStream("/analyzers/synonyms.txt") val config = PathUtils.get(buildDir, leaderClusterPath, "config") diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt index d4737c61..0fa3c5ab 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt @@ -25,6 +25,8 @@ import org.opensearch.replication.resumeReplication import org.opensearch.replication.`validate paused status response due to leader index deleted` import org.opensearch.replication.`validate status syncing response` import org.opensearch.replication.startReplication +import org.opensearch.replication.ANALYZERS_NOT_ACCESSIBLE_FOR_REMOTE_CLUSTERS +import org.opensearch.replication.SNAPSHOTS_NOT_ACCESSIBLE_FOR_REMOTE_CLUSTERS import org.opensearch.replication.stopReplication import org.opensearch.replication.updateReplication import org.apache.hc.core5.http.HttpStatus @@ -66,7 +68,7 @@ import org.opensearch.index.mapper.MapperService import org.opensearch.repositories.fs.FsRepository import org.opensearch.test.OpenSearchTestCase.assertBusy import org.junit.Assert -import org.opensearch.cluster.metadata.AliasMetadata +import org.junit.Assume import org.opensearch.core.xcontent.DeprecationHandler import org.opensearch.core.xcontent.NamedXContentRegistry import org.opensearch.replication.ReplicationPlugin.Companion.REPLICATION_INDEX_TRANSLOG_PRUNING_ENABLED_SETTING @@ -585,9 +587,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { fun `test that replication fails to start when custom analyser is not present in follower`() { - if(checkifIntegTestRemote()){ - return; - } + Assume.assumeFalse(ANALYZERS_NOT_ACCESSIBLE_FOR_REMOTE_CLUSTERS, checkifIntegTestRemote()) val synonyms = javaClass.getResourceAsStream("/analyzers/synonyms.txt") val config = PathUtils.get(buildDir, leaderClusterPath, "config") @@ -620,9 +620,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { fun `test that replication starts successfully when custom analyser is present in follower`() { - if(checkifIntegTestRemote()){ - return; - } + Assume.assumeFalse(ANALYZERS_NOT_ACCESSIBLE_FOR_REMOTE_CLUSTERS, checkifIntegTestRemote()) val synonyms = javaClass.getResourceAsStream("/analyzers/synonyms.txt") val leaderConfig = PathUtils.get(buildDir, leaderClusterPath, "config") @@ -662,9 +660,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { fun `test that replication starts successfully when custom analyser is overridden and present in follower`() { - if(checkifIntegTestRemote()){ - return; - } + Assume.assumeFalse(ANALYZERS_NOT_ACCESSIBLE_FOR_REMOTE_CLUSTERS, checkifIntegTestRemote()) val synonyms = javaClass.getResourceAsStream("/analyzers/synonyms.txt") val leaderConfig = PathUtils.get(buildDir, leaderClusterPath, "config") @@ -801,9 +797,7 @@ class StartReplicationIT: MultiClusterRestTestCase() { fun `test that snapshot on leader does not affect replication during bootstrap`() { - if(checkifIntegTestRemote()){ - return; - } + Assume.assumeFalse(SNAPSHOTS_NOT_ACCESSIBLE_FOR_REMOTE_CLUSTERS,checkifIntegTestRemote()) val settings = Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 20) @@ -1147,6 +1141,10 @@ class StartReplicationIT: MultiClusterRestTestCase() { } fun `test that wait_for_active_shards setting is updated on follower through start replication api`() { + + Assume.assumeTrue("Ignore this test if clusters dont have multiple nodes as this test reles on wait_for_active_shards", + isMultiNodeClusterConfiguration(LEADER, FOLLOWER)) + val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/StopReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/StopReplicationIT.kt index f2f0a09e..788fb6de 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/StopReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/StopReplicationIT.kt @@ -24,6 +24,7 @@ import org.apache.hc.core5.http.io.entity.EntityUtils import org.assertj.core.api.Assertions.assertThat import org.assertj.core.api.Assertions.assertThatThrownBy import org.junit.Assert +import org.junit.Assume import org.opensearch.OpenSearchStatusException import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest import org.opensearch.action.admin.cluster.snapshots.create.CreateSnapshotRequest @@ -40,6 +41,7 @@ import org.opensearch.cluster.metadata.IndexMetadata import org.opensearch.common.settings.Settings import org.opensearch.common.unit.TimeValue import org.opensearch.index.mapper.MapperService +import org.opensearch.replication.SNAPSHOTS_NOT_ACCESSIBLE_FOR_REMOTE_CLUSTERS import java.util.Random import java.util.concurrent.TimeUnit @@ -243,9 +245,7 @@ class StopReplicationIT: MultiClusterRestTestCase() { fun `test stop replication with stale replication settings at leader cluster`() { - if(checkifIntegTestRemote()){ - return; - } + Assume.assumeFalse(SNAPSHOTS_NOT_ACCESSIBLE_FOR_REMOTE_CLUSTERS, checkifIntegTestRemote()) val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) From 9750b7800a208b9f2bea2646813dc59857cfed73 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Tue, 22 Aug 2023 18:55:50 +0530 Subject: [PATCH 110/157] Revert "Add setting to use document replication for system indices. (#802)" (#1107) This reverts commit 55b6968af90d739d8448c5d88ed5204a240d0f86. Signed-off-by: monusingh-1 --- .../replication/metadata/store/ReplicationMetadataStore.kt | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/metadata/store/ReplicationMetadataStore.kt b/src/main/kotlin/org/opensearch/replication/metadata/store/ReplicationMetadataStore.kt index d5cbe751..d838a21c 100644 --- a/src/main/kotlin/org/opensearch/replication/metadata/store/ReplicationMetadataStore.kt +++ b/src/main/kotlin/org/opensearch/replication/metadata/store/ReplicationMetadataStore.kt @@ -40,7 +40,6 @@ import org.opensearch.common.xcontent.LoggingDeprecationHandler import org.opensearch.core.xcontent.NamedXContentRegistry import org.opensearch.core.xcontent.ToXContent import org.opensearch.core.xcontent.XContentParser -import org.opensearch.indices.replication.common.ReplicationType import org.opensearch.replication.util.suspendExecuteWithRetries class ReplicationMetadataStore constructor(val client: Client, val clusterService: ClusterService, @@ -266,7 +265,6 @@ class ReplicationMetadataStore constructor(val client: Client, val clusterServic .put(IndexMetadata.INDEX_AUTO_EXPAND_REPLICAS_SETTING.key, "0-1") .put(IndexMetadata.INDEX_PRIORITY_SETTING.key, Int.MAX_VALUE) .put(IndexMetadata.INDEX_HIDDEN_SETTING.key, true) - .put(IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.key, ReplicationType.DOCUMENT) // System Indices should use Document Replication strategy .build() } From f00ac43decea640e6a8a497f5c1d1b4be97aed5f Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Fri, 1 Sep 2023 16:08:45 +0530 Subject: [PATCH 111/157] update requirements to use latest version of request (#1106) Signed-off-by: monusingh-1 --- perf_workflow/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/perf_workflow/requirements.txt b/perf_workflow/requirements.txt index 616ed43e..8ee4edfa 100644 --- a/perf_workflow/requirements.txt +++ b/perf_workflow/requirements.txt @@ -3,7 +3,7 @@ validators yamlfix cerberus pipenv -requests +requests~=2.31.0 retry ndg-httpsclient psutil From b6d1b56654a83220a33db162b592a9835ec132ec Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Fri, 1 Sep 2023 16:09:40 +0530 Subject: [PATCH 112/157] Implement abstract methods of Repository (#1111) * Implement methods Signed-off-by: monusingh-1 * Implement methods Signed-off-by: monusingh-1 --------- Signed-off-by: monusingh-1 --- .../replication/repository/RemoteClusterRepository.kt | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt index 929d6585..77233828 100644 --- a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt +++ b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt @@ -107,6 +107,14 @@ class RemoteClusterRepository(private val repositoryMetadata: RepositoryMetadata return restoreRateLimitingTimeInNanos.count() } + override fun getRemoteUploadThrottleTimeInNanos(): Long { + throw UnsupportedOperationException("Operation not permitted") + } + + override fun getRemoteDownloadThrottleTimeInNanos(): Long { + throw UnsupportedOperationException("Operation not permitted") + } + override fun finalizeSnapshot(shardGenerations: ShardGenerations?, repositoryStateId: Long, clusterMetadata: Metadata?, snapshotInfo: SnapshotInfo?, repositoryMetaVersion: Version?, stateTransformer: Function?, From 426a2deb85bc3c58a3439e829e6902e68e6f658d Mon Sep 17 00:00:00 2001 From: Mohit Kumar <113413713+mohitamg@users.noreply.github.com> Date: Mon, 4 Sep 2023 15:38:16 +0530 Subject: [PATCH 113/157] Removed timeout excessive logging in case of index is idle in replication (#1114) * Removed timeout excessive logging in case of index is idle in replication Signed-off-by: Mohit Kumar * Removed timeout excessive logging in case of index is idle in replication Signed-off-by: Mohit Kumar * Removed timeout excessive stack trace logging in case of index is idle in replication Signed-off-by: Mohit Kumar * Removed timeout excessive stack trace logging in case of index is idle in replication Signed-off-by: Mohit Kumar * Changed the log statement Signed-off-by: Mohit Kumar --------- Signed-off-by: Mohit Kumar --- src/main/kotlin/org/opensearch/replication/util/Extensions.kt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/util/Extensions.kt b/src/main/kotlin/org/opensearch/replication/util/Extensions.kt index 7a73fa5f..81aaa446 100644 --- a/src/main/kotlin/org/opensearch/replication/util/Extensions.kt +++ b/src/main/kotlin/org/opensearch/replication/util/Extensions.kt @@ -138,8 +138,8 @@ suspend fun Client.suspendExecuteWith } } log.warn( - "Encountered a failure while executing in $req. Retrying in ${currentBackoff / 1000} seconds" + - ".", retryException + "Encountered a failure(can be ignored) while getting changes: OpenSearchTimeoutException. Retrying in ${currentBackoff / 1000} seconds" + + "." ) delay(currentBackoff) currentBackoff = (currentBackoff * factor).toLong().coerceAtMost(maxTimeOut) From 4273a1953714debfbfbe395007e3d7c4d3f00888 Mon Sep 17 00:00:00 2001 From: Mohit Kumar <113413713+mohitamg@users.noreply.github.com> Date: Wed, 6 Sep 2023 14:03:26 +0530 Subject: [PATCH 114/157] Create opensearch-cross-cluster-replication.release-notes-2.10.0.0.md (#1117) Signed-off-by: Mohit Kumar <113413713+mohitamg@users.noreply.github.com> --- ...cross-cluster-replication.release-notes-2.10.0.0.md | 10 ++++++++++ 1 file changed, 10 insertions(+) create mode 100644 release-notes/opensearch-cross-cluster-replication.release-notes-2.10.0.0.md diff --git a/release-notes/opensearch-cross-cluster-replication.release-notes-2.10.0.0.md b/release-notes/opensearch-cross-cluster-replication.release-notes-2.10.0.0.md new file mode 100644 index 00000000..be0d8015 --- /dev/null +++ b/release-notes/opensearch-cross-cluster-replication.release-notes-2.10.0.0.md @@ -0,0 +1,10 @@ +## Version 2.10.0.0 Release Notes + +Compatible with OpenSearch 2.10.0 + + +### Bug Fixes +* Settings are synced before syncing mapping ([#994](https://github.com/opensearch-project/cross-cluster-replication/pull/994)) +* Handled OpenSearchRejectExecuteException, introduced new setting ```plugins.replication.follower.concurrent_writers_per_shard```. ([#1004](https://github.com/opensearch-project/cross-cluster-replication/pull/1004)) +* Fixed tests relying on wait_for_active_shards, fixed test for single Node and consume numNodes ([#1091](https://github.com/opensearch-project/cross-cluster-replication/pull/1091)) +* Excessive logging avoided during certain exception types such as OpensearchTimeoutException ([#1114](https://github.com/opensearch-project/cross-cluster-replication/pull/1114)) From 0fbe5a7aeb1be4c3dccd657332c2f41c3513028d Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Thu, 7 Sep 2023 18:06:51 +0530 Subject: [PATCH 115/157] Implement abstract methods (#1121) Signed-off-by: monusingh-1 --- .../replication/repository/RemoteClusterRepository.kt | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt index 77233828..eb1d4e47 100644 --- a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt +++ b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt @@ -359,6 +359,10 @@ class RemoteClusterRepository(private val repositoryMetadata: RepositoryMetadata return true } + override fun isSystemRepository(): Boolean { + throw UnsupportedOperationException("Operation not permitted") + } + /* * This method makes a blocking call to the leader cluster From c19dcba791a8a60f660e1367925bd08d01a57b9e Mon Sep 17 00:00:00 2001 From: Sooraj Sinha <81695996+soosinha@users.noreply.github.com> Date: Mon, 11 Sep 2023 11:29:12 +0530 Subject: [PATCH 116/157] Remove numNodes parameter in security tests (#991) Some of the integ tests like those for wait_for_active_shards need atleast 2 nodes in the cluster to run successfully. Removing the numNodes=1 parameter so that by default 2 node cluster is created Signed-off-by: Sooraj Sinha --- .github/workflows/security-knn-tests.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/security-knn-tests.yml b/.github/workflows/security-knn-tests.yml index 3f98f3cb..45d4e898 100644 --- a/.github/workflows/security-knn-tests.yml +++ b/.github/workflows/security-knn-tests.yml @@ -63,7 +63,7 @@ jobs: - name: Build and run Replication tests run: | ls -al src/test/resources/security/plugin - ./gradlew --refresh-dependencies clean release -Dbuild.snapshot=true -PnumNodes=1 -Psecurity=true + ./gradlew --refresh-dependencies clean release -Dbuild.snapshot=true -Psecurity=true - name: Upload failed logs uses: actions/upload-artifact@v2 if: failure() From 633ef8b74f0c21370cf69ae8c00cd036a9a585cc Mon Sep 17 00:00:00 2001 From: Mohit Kumar <113413713+mohitamg@users.noreply.github.com> Date: Wed, 4 Oct 2023 18:38:13 +0530 Subject: [PATCH 117/157] Create opensearch-cross-cluster-replication.release-notes-2.11.0.md Release notes for v2.11.0 Signed-off-by: Mohit Kumar <113413713+mohitamg@users.noreply.github.com> --- ...pensearch-cross-cluster-replication.release-notes-2.11.0.md | 3 +++ 1 file changed, 3 insertions(+) create mode 100644 release-notes/opensearch-cross-cluster-replication.release-notes-2.11.0.md diff --git a/release-notes/opensearch-cross-cluster-replication.release-notes-2.11.0.md b/release-notes/opensearch-cross-cluster-replication.release-notes-2.11.0.md new file mode 100644 index 00000000..2a075f7f --- /dev/null +++ b/release-notes/opensearch-cross-cluster-replication.release-notes-2.11.0.md @@ -0,0 +1,3 @@ +## Version 2.11.0 Release Notes + +Compatible with OpenSearch 2.11.0 From 24b48c6d0817e8da0f674dbd157289648f4bb199 Mon Sep 17 00:00:00 2001 From: Mohit Kumar <113413713+mohitamg@users.noreply.github.com> Date: Thu, 12 Oct 2023 11:27:04 +0530 Subject: [PATCH 118/157] Rename opensearch-cross-cluster-replication.release-notes-2.11.0.md to opensearch-cross-cluster-replication.release-notes-2.11.0.0.md (#1253) --- ...pensearch-cross-cluster-replication.release-notes-2.11.0.0.md} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename release-notes/{opensearch-cross-cluster-replication.release-notes-2.11.0.md => opensearch-cross-cluster-replication.release-notes-2.11.0.0.md} (100%) diff --git a/release-notes/opensearch-cross-cluster-replication.release-notes-2.11.0.md b/release-notes/opensearch-cross-cluster-replication.release-notes-2.11.0.0.md similarity index 100% rename from release-notes/opensearch-cross-cluster-replication.release-notes-2.11.0.md rename to release-notes/opensearch-cross-cluster-replication.release-notes-2.11.0.0.md From 0bd703bf1468c088b760e274024477df52ac35a1 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Tue, 17 Oct 2023 19:36:54 +0530 Subject: [PATCH 119/157] Container image (#1249) Signed-off-by: monusingh-1 --- .github/workflows/build-and-test.yml | 45 +++++++++++++++--------- .github/workflows/build.yml | 2 +- .github/workflows/security-knn-tests.yml | 35 ++++++++++++++---- 3 files changed, 57 insertions(+), 25 deletions(-) diff --git a/.github/workflows/build-and-test.yml b/.github/workflows/build-and-test.yml index 4a257534..125d13db 100644 --- a/.github/workflows/build-and-test.yml +++ b/.github/workflows/build-and-test.yml @@ -10,28 +10,39 @@ on: # We build for all combinations but run tests only on one combination (linux & latest java) jobs: - build: - continue-on-error: true + Get-CI-Image-Tag: + uses: opensearch-project/opensearch-build/.github/workflows/get-ci-image-tag.yml@main + with: + product: opensearch + + build-test-linux: strategy: matrix: - java: - - 11 - - 17 - # Job name - name: Run integration tests on linux with Java ${{ matrix.java }} + java: [11, 17] + + name: Build CCR Plugin on Linux using Container Image runs-on: ubuntu-latest + needs: Get-CI-Image-Tag + container: + # using the same image which is used by opensearch-build team to build the OpenSearch Distribution + # this image tag is subject to change as more dependencies and updates will arrive over time + image: ${{ needs.Get-CI-Image-Tag.outputs.ci-image-version-linux }} + # need to switch to root so that github actions can install runner binary on container without permission issues. + options: --user root + steps: - # This step uses the setup-java Github action: https://github.com/actions/setup-java - - name: Set Up JDK ${{ matrix.java }} + - name: Checkout CCR + uses: actions/checkout@v2 + - name: Setup Java ${{ matrix.java }} uses: actions/setup-java@v1 with: java-version: ${{ matrix.java }} - # This step uses the checkout Github action: https://github.com/actions/checkout - - name: Checkout Branch - uses: actions/checkout@v2 - - name: Build and run Replication tests + + - name: Run build + # switching the user, as OpenSearch cluster can only be started as root/Administrator on linux-deb/linux-rpm/windows-zip. run: | - ./gradlew --refresh-dependencies clean release -D"build.snapshot=true" + chown -R 1000:1000 `pwd` + su `id -un 1000` -c 'whoami && java -version && ./gradlew --refresh-dependencies clean release -D"build.snapshot=true"' - name: Upload failed logs uses: actions/upload-artifact@v2 if: failure() @@ -45,7 +56,7 @@ jobs: run: | mkdir -p cross-cluster-replication-artifacts cp ./build/distributions/*.zip cross-cluster-replication-artifacts - - name: Uploads coverage + - name: Upload Coverage Report + uses: codecov/codecov-action@v1 with: - fetch-depth: 2 - uses: codecov/codecov-action@v1.2.1 + token: ${{ secrets.CODECOV_TOKEN }} diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index 2e4d54e4..ca44eebf 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -34,4 +34,4 @@ jobs: uses: actions/checkout@v2 - name: Build and run Replication tests run: | - ./gradlew --refresh-dependencies clean release -D"build.snapshot=true" -x test -x IntegTest \ No newline at end of file + ./gradlew --refresh-dependencies clean release -D"build.snapshot=true" -x test -x IntegTest diff --git a/.github/workflows/security-knn-tests.yml b/.github/workflows/security-knn-tests.yml index 45d4e898..a6b3fd9d 100644 --- a/.github/workflows/security-knn-tests.yml +++ b/.github/workflows/security-knn-tests.yml @@ -9,6 +9,11 @@ on: - '*' jobs: + Get-CI-Image-Tag: + uses: opensearch-project/opensearch-build/.github/workflows/get-ci-image-tag.yml@main + with: + product: opensearch + req: # Job name name: plugin check @@ -45,12 +50,19 @@ jobs: cat $GITHUB_OUTPUT fi - build: - needs: req + build-linux: + needs: [req, Get-CI-Image-Tag] if: ${{ 'True' == needs.req.outputs.isSecurityPluginAvailable }} # Job name name: Build and Run Security tests runs-on: ubuntu-latest + container: + # using the same image which is used by opensearch-build team to build the OpenSearch Distribution + # this image tag is subject to change as more dependencies and updates will arrive over time + image: ${{ needs.Get-CI-Image-Tag.outputs.ci-image-version-linux }} + # need to switch to root so that github actions can install runner binary on container without permission issues. + options: --user root + steps: # This step uses the setup-java Github action: https://github.com/actions/setup-java - name: Set Up JDK 17 @@ -62,8 +74,9 @@ jobs: uses: actions/checkout@v2 - name: Build and run Replication tests run: | + chown -R 1000:1000 `pwd` ls -al src/test/resources/security/plugin - ./gradlew --refresh-dependencies clean release -Dbuild.snapshot=true -Psecurity=true + su `id -un 1000` -c "whoami && java -version && ./gradlew --refresh-dependencies clean release -Dbuild.snapshot=true -Psecurity=true" - name: Upload failed logs uses: actions/upload-artifact@v2 if: failure() @@ -82,12 +95,19 @@ jobs: fetch-depth: 2 uses: codecov/codecov-action@v1.2.1 - knn-build: - needs: req + knn-build-linux: + needs: [req, Get-CI-Image-Tag] if: ${{ 'True' == needs.req.outputs.isKnnPluginAvailable }} # Job name name: Build and Run Knn tests runs-on: ubuntu-latest + container: + # using the same image which is used by opensearch-build team to build the OpenSearch Distribution + # this image tag is subject to change as more dependencies and updates will arrive over time + image: ${{ needs.Get-CI-Image-Tag.outputs.ci-image-version-linux }} + # need to switch to root so that github actions can install runner binary on container without permission issues. + options: --user root + steps: # This step uses the setup-java Github action: https://github.com/actions/setup-java - name: Set Up JDK 17 @@ -99,7 +119,8 @@ jobs: uses: actions/checkout@v2 - name: Build and run Replication tests run: | - ./gradlew --refresh-dependencies clean release -Dbuild.snapshot=true -PnumNodes=1 -Dtests.class=org.opensearch.replication.BasicReplicationIT -Dtests.method="test knn index replication" -Pknn=true + chown -R 1000:1000 `pwd` + su `id -un 1000` -c 'whoami && java -version && ./gradlew --refresh-dependencies clean release -Dbuild.snapshot=true -PnumNodes=1 -Dtests.class=org.opensearch.replication.BasicReplicationIT -Dtests.method="test knn index replication" -Pknn=true' - name: Upload failed logs uses: actions/upload-artifact@v2 if: failure() @@ -116,4 +137,4 @@ jobs: - name: Uploads coverage with: fetch-depth: 2 - uses: codecov/codecov-action@v1.2.1 \ No newline at end of file + uses: codecov/codecov-action@v1.2.1 From e1d25969b823e7149f3ab5cff269edec3f839773 Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Wed, 18 Oct 2023 09:51:01 +0530 Subject: [PATCH 120/157] CCR: Route requests to primary for remote store enabled leader clusters Signed-off-by: Ankit Kala --- .../replication/ReplicationPlugin.kt | 15 ++++++-- .../changes/TransportGetChangesAction.kt | 35 ++++++++++++------- .../repository/RemoteClusterRepository.kt | 3 ++ .../replication/util/ValidationUtil.kt | 7 ++++ 4 files changed, 45 insertions(+), 15 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt b/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt index ecc1f888..39ba84eb 100644 --- a/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt +++ b/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt @@ -144,12 +144,14 @@ import java.util.Optional import java.util.function.Supplier import org.opensearch.index.engine.NRTReplicationEngine +import org.opensearch.replication.util.ValidationUtil @OpenForTesting internal class ReplicationPlugin : Plugin(), ActionPlugin, PersistentTaskPlugin, RepositoryPlugin, EnginePlugin { private lateinit var client: Client + private lateinit var clusterService: ClusterService private lateinit var threadPool: ThreadPool private lateinit var replicationMetadataManager: ReplicationMetadataManager private lateinit var replicationSettings: ReplicationSettings @@ -207,6 +209,7 @@ internal class ReplicationPlugin : Plugin(), ActionPlugin, PersistentTaskPlugin, repositoriesService: Supplier): Collection { this.client = client this.threadPool = threadPool + this.clusterService = clusterService this.replicationMetadataManager = ReplicationMetadataManager(clusterService, client, ReplicationMetadataStore(client, clusterService, xContentRegistry)) this.replicationSettings = ReplicationSettings(clusterService) @@ -379,9 +382,15 @@ internal class ReplicationPlugin : Plugin(), ActionPlugin, PersistentTaskPlugin, } override fun getCustomTranslogDeletionPolicyFactory(): Optional { - return Optional.of(TranslogDeletionPolicyFactory{ - indexSettings, retentionLeasesSupplier -> ReplicationTranslogDeletionPolicy(indexSettings, retentionLeasesSupplier) - }) + // We don't need a retention lease translog deletion policy for remote store enabled clusters as + // we fetch the operations directly from lucene in such cases. + return if (ValidationUtil.isRemoteStoreEnabledCluster(clusterService) == false) { + Optional.of(TranslogDeletionPolicyFactory { indexSettings, retentionLeasesSupplier -> + ReplicationTranslogDeletionPolicy(indexSettings, retentionLeasesSupplier) + }) + } else { + Optional.empty() + } } override fun onIndexModule(indexModule: IndexModule) { diff --git a/src/main/kotlin/org/opensearch/replication/action/changes/TransportGetChangesAction.kt b/src/main/kotlin/org/opensearch/replication/action/changes/TransportGetChangesAction.kt index dbee183d..392555b5 100644 --- a/src/main/kotlin/org/opensearch/replication/action/changes/TransportGetChangesAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/changes/TransportGetChangesAction.kt @@ -27,6 +27,7 @@ import org.opensearch.core.common.io.stream.StreamInput import org.opensearch.core.common.io.stream.Writeable import org.opensearch.common.unit.TimeValue import org.opensearch.core.index.shard.ShardId +import org.opensearch.index.shard.IndexShard import org.opensearch.index.translog.Translog import org.opensearch.indices.IndicesService import org.opensearch.replication.ReplicationPlugin.Companion.REPLICATION_INDEX_TRANSLOG_PRUNING_ENABLED_SETTING @@ -34,10 +35,7 @@ import org.opensearch.replication.ReplicationPlugin.Companion.REPLICATION_EXECUT import org.opensearch.replication.seqno.RemoteClusterStats import org.opensearch.replication.seqno.RemoteClusterTranslogService import org.opensearch.replication.seqno.RemoteShardMetric -import org.opensearch.replication.util.completeWith -import org.opensearch.replication.util.coroutineContext -import org.opensearch.replication.util.stackTraceToString -import org.opensearch.replication.util.waitForGlobalCheckpoint +import org.opensearch.replication.util.* import org.opensearch.threadpool.ThreadPool import org.opensearch.transport.TransportActionProxy import org.opensearch.transport.TransportService @@ -79,7 +77,8 @@ class TransportGetChangesAction @Inject constructor(threadPool: ThreadPool, clus indexMetric.lastFetchTime.set(relativeStartNanos) val indexShard = indicesService.indexServiceSafe(shardId.index).getShard(shardId.id) - if (indexShard.lastSyncedGlobalCheckpoint < request.fromSeqNo) { + val isRemoteStoreEnabled = ValidationUtil.isRemoteStoreEnabledCluster(clusterService) + if (lastGlobalCheckpoint(indexShard, isRemoteStoreEnabled) < request.fromSeqNo) { // There are no new operations to sync. Do a long poll and wait for GlobalCheckpoint to advance. If // the checkpoint doesn't advance by the timeout this throws an ESTimeoutException which the caller // should catch and start a new poll. @@ -88,18 +87,18 @@ class TransportGetChangesAction @Inject constructor(threadPool: ThreadPool, clus // At this point indexShard.lastKnownGlobalCheckpoint has advanced but it may not yet have been synced // to the translog, which means we can't return those changes. Return to the caller to retry. // TODO: Figure out a better way to wait for the global checkpoint to be synced to the translog - if (indexShard.lastSyncedGlobalCheckpoint < request.fromSeqNo) { - assert(gcp > indexShard.lastSyncedGlobalCheckpoint) { "Checkpoint didn't advance at all" } + if (lastGlobalCheckpoint(indexShard, isRemoteStoreEnabled) < request.fromSeqNo) { + assert(gcp > lastGlobalCheckpoint(indexShard, isRemoteStoreEnabled)) { "Checkpoint didn't advance at all $gcp ${lastGlobalCheckpoint(indexShard, isRemoteStoreEnabled)}" } throw OpenSearchTimeoutException("global checkpoint not synced. Retry after a few miliseconds...") } } relativeStartNanos = System.nanoTime() // At this point lastSyncedGlobalCheckpoint is at least fromSeqNo - val toSeqNo = min(indexShard.lastSyncedGlobalCheckpoint, request.toSeqNo) + val toSeqNo = min(lastGlobalCheckpoint(indexShard, isRemoteStoreEnabled), request.toSeqNo) var ops: List = listOf() - var fetchFromTranslog = isTranslogPruningByRetentionLeaseEnabled(shardId) + var fetchFromTranslog = isTranslogPruningByRetentionLeaseEnabled(shardId) && isRemoteStoreEnabled == false if(fetchFromTranslog) { try { ops = translogService.getHistoryOfOperations(indexShard, request.fromSeqNo, toSeqNo) @@ -137,12 +136,22 @@ class TransportGetChangesAction @Inject constructor(threadPool: ThreadPool, clus indexMetric.ops.addAndGet(ops.size.toLong()) ops.stream().forEach{op -> indexMetric.bytesRead.addAndGet(op.estimateSize()) } - - GetChangesResponse(ops, request.fromSeqNo, indexShard.maxSeqNoOfUpdatesOrDeletes, indexShard.lastSyncedGlobalCheckpoint) + GetChangesResponse(ops, request.fromSeqNo, indexShard.maxSeqNoOfUpdatesOrDeletes, lastGlobalCheckpoint(indexShard, isRemoteStoreEnabled)) } } } + private fun lastGlobalCheckpoint(indexShard: IndexShard, isRemoteStoreEnabled: Boolean): Long { + // We rely on lastSyncedGlobalCheckpoint as it has been durably written to disk. In case of remote store + // enabled clusters, the semantics are slightly different, and we can't use lastSyncedGlobalCheckpoint. Falling back to + // lastKnownGlobalCheckpoint in such cases. + return if (isRemoteStoreEnabled) { + indexShard.lastKnownGlobalCheckpoint + } else { + indexShard.lastSyncedGlobalCheckpoint + } + } + private fun isTranslogPruningByRetentionLeaseEnabled(shardId: ShardId): Boolean { val enabled = clusterService.state().metadata.indices.get(shardId.indexName) @@ -162,7 +171,9 @@ class TransportGetChangesAction @Inject constructor(threadPool: ThreadPool, clus } override fun shards(state: ClusterState, request: InternalRequest): ShardsIterator { + val shardIt = state.routingTable().shardRoutingTable(request.request().shardId) // Random active shards - return state.routingTable().shardRoutingTable(request.request().shardId).activeInitializingShardsRandomIt() + return if (ValidationUtil.isRemoteStoreEnabledCluster(clusterService)) shardIt.primaryShardIt() + else shardIt.activeInitializingShardsRandomIt() } } \ No newline at end of file diff --git a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt index eb1d4e47..bb03f275 100644 --- a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt +++ b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt @@ -245,6 +245,9 @@ class RemoteClusterRepository(private val repositoryMetadata: RepositoryMetadata // Remove translog pruning for the follower index builder.remove(REPLICATION_INDEX_TRANSLOG_PRUNING_ENABLED_SETTING.key) + builder.remove(IndexMetadata.SETTING_REMOTE_STORE_ENABLED) + builder.remove(IndexMetadata.SETTING_REMOTE_SEGMENT_STORE_REPOSITORY) + builder.remove(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY) val indexMdBuilder = IndexMetadata.builder(indexMetadata).settings(builder) indexMetadata.aliases.values.forEach { diff --git a/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt b/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt index 0e2b4e62..afbcdf60 100644 --- a/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt +++ b/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt @@ -24,8 +24,11 @@ import org.opensearch.env.Environment import org.opensearch.index.IndexNotFoundException import java.io.UnsupportedEncodingException import org.opensearch.cluster.service.ClusterService +import org.opensearch.node.Node +import org.opensearch.node.remotestore.RemoteStoreNodeAttribute import org.opensearch.replication.ReplicationPlugin.Companion.KNN_INDEX_SETTING import org.opensearch.replication.ReplicationPlugin.Companion.KNN_PLUGIN_PRESENT_SETTING +import org.opensearch.replication.action.changes.TransportGetChangesAction import java.nio.file.Files import java.nio.file.Path import java.util.Locale @@ -154,4 +157,8 @@ object ValidationUtil { } + fun isRemoteStoreEnabledCluster(clusterService: ClusterService): Boolean { + return clusterService.settings.getByPrefix(Node.NODE_ATTRIBUTES.key + RemoteStoreNodeAttribute.REMOTE_STORE_NODE_ATTRIBUTE_KEY_PREFIX).isEmpty == false + } + } From 12ebd7e946796fdb83d746c26db1c772cd9ec76e Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Fri, 27 Oct 2023 13:55:00 +0530 Subject: [PATCH 121/157] Fix build failure (#1281) Signed-off-by: monusingh-1 --- .../replication/action/replay/TransportReplayChangesAction.kt | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/main/kotlin/org/opensearch/replication/action/replay/TransportReplayChangesAction.kt b/src/main/kotlin/org/opensearch/replication/action/replay/TransportReplayChangesAction.kt index fbf9e181..66ebc015 100644 --- a/src/main/kotlin/org/opensearch/replication/action/replay/TransportReplayChangesAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/replay/TransportReplayChangesAction.kt @@ -47,11 +47,13 @@ import org.opensearch.common.settings.Settings import org.opensearch.common.xcontent.XContentType import org.opensearch.index.IndexingPressureService import org.opensearch.index.engine.Engine +import org.opensearch.index.engine.Engine.NoOp import org.opensearch.index.mapper.MapperParsingException import org.opensearch.index.shard.IndexShard import org.opensearch.index.translog.Translog import org.opensearch.indices.IndicesService import org.opensearch.indices.SystemIndices +import org.opensearch.telemetry.tracing.noop.NoopTracer import org.opensearch.threadpool.ThreadPool import org.opensearch.transport.TransportService import java.util.function.Function @@ -72,7 +74,7 @@ class TransportReplayChangesAction @Inject constructor(settings: Settings, trans TransportWriteAction( settings, ReplayChangesAction.NAME, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters, Writeable.Reader { inp -> ReplayChangesRequest(inp) }, Writeable.Reader { inp -> ReplayChangesRequest(inp) }, - EXECUTOR_NAME_FUNCTION, false, indexingPressureService, systemIndices) { + EXECUTOR_NAME_FUNCTION, false, indexingPressureService, systemIndices, NoopTracer.INSTANCE) { companion object { private val log = LogManager.getLogger(TransportReplayChangesAction::class.java)!! From abb41a93f492b927db0d6880e81a85a978454e73 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Wed, 22 Nov 2023 17:59:31 +0530 Subject: [PATCH 122/157] Add 2.11.1.0 release notes (#1285) Signed-off-by: monusingh-1 --- ...rch-cross-cluster-replication.release-notes-2.11.1.0.md | 7 +++++++ 1 file changed, 7 insertions(+) create mode 100644 release-notes/opensearch-cross-cluster-replication.release-notes-2.11.1.0.md diff --git a/release-notes/opensearch-cross-cluster-replication.release-notes-2.11.1.0.md b/release-notes/opensearch-cross-cluster-replication.release-notes-2.11.1.0.md new file mode 100644 index 00000000..0df3f9e6 --- /dev/null +++ b/release-notes/opensearch-cross-cluster-replication.release-notes-2.11.1.0.md @@ -0,0 +1,7 @@ +## Version 2.11.1.0 Release Notes + +Compatible with OpenSearch 2.11.1 + + +### Bug Fixes +* Fix CCR compatibility with remote translogs ([#1276](https://github.com/opensearch-project/cross-cluster-replication/pull/1276)) \ No newline at end of file From 577fd6275085d3b50ef816949c3ab178dc805b4c Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Wed, 6 Dec 2023 19:04:29 +0530 Subject: [PATCH 123/157] Add support for SystemIndex (#1290) Signed-off-by: monusingh-1 --- .../org/opensearch/replication/ReplicationPlugin.kt | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt b/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt index 39ba84eb..792ebe27 100644 --- a/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt +++ b/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt @@ -114,6 +114,7 @@ import org.opensearch.plugins.ActionPlugin import org.opensearch.plugins.ActionPlugin.ActionHandler import org.opensearch.plugins.EnginePlugin import org.opensearch.plugins.PersistentTaskPlugin +import org.opensearch.plugins.SystemIndexPlugin import org.opensearch.plugins.Plugin import org.opensearch.plugins.RepositoryPlugin import org.opensearch.replication.action.autofollow.* @@ -144,11 +145,13 @@ import java.util.Optional import java.util.function.Supplier import org.opensearch.index.engine.NRTReplicationEngine +import org.opensearch.indices.SystemIndexDescriptor import org.opensearch.replication.util.ValidationUtil @OpenForTesting -internal class ReplicationPlugin : Plugin(), ActionPlugin, PersistentTaskPlugin, RepositoryPlugin, EnginePlugin { +internal class ReplicationPlugin : Plugin(), ActionPlugin, PersistentTaskPlugin, + RepositoryPlugin, EnginePlugin, SystemIndexPlugin { private lateinit var client: Client private lateinit var clusterService: ClusterService @@ -399,4 +402,7 @@ internal class ReplicationPlugin : Plugin(), ActionPlugin, PersistentTaskPlugin, indexModule.addIndexEventListener(IndexCloseListener) } } + override fun getSystemIndexDescriptors(settings: Settings): Collection { + return listOf(SystemIndexDescriptor(ReplicationMetadataStore.REPLICATION_CONFIG_SYSTEM_INDEX, "System Index for storing cross cluster replication configuration.")) + } } From be24bfaa9a795179662a6660633902a7737b717c Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Thu, 7 Dec 2023 17:06:48 +0530 Subject: [PATCH 124/157] Correct error message when field are not passed when starting replication (#1292) --- .../action/index/ReplicateIndexRequest.kt | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/action/index/ReplicateIndexRequest.kt b/src/main/kotlin/org/opensearch/replication/action/index/ReplicateIndexRequest.kt index 6024798a..3d2bfb1b 100644 --- a/src/main/kotlin/org/opensearch/replication/action/index/ReplicateIndexRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/index/ReplicateIndexRequest.kt @@ -92,10 +92,19 @@ class ReplicateIndexRequest : AcknowledgedRequest, Indice override fun validate(): ActionRequestValidationException? { var validationException = ActionRequestValidationException() - if (!this::leaderAlias.isInitialized || - !this::leaderIndex.isInitialized || - !this::followerIndex.isInitialized) { - validationException.addValidationError("Mandatory params are missing for the request") + val missingFields: MutableList = mutableListOf() + if (!this::leaderAlias.isInitialized){ + missingFields.add("leader_alias") + } + if(!this::leaderIndex.isInitialized){ + missingFields.add("leader_index") + } + if (!this::followerIndex.isInitialized){ + missingFields.add("follower_index") + } + if(missingFields.isNotEmpty()){ + validationException.addValidationError("Mandatory params $missingFields are missing for the request") + return validationException } validateName(leaderIndex, validationException) From 8258ac5ded83398be701721c7199b3d2ea714c70 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Tue, 2 Jan 2024 13:52:24 +0530 Subject: [PATCH 125/157] Use latest version of ipaddress Library (#1302) Use latest version of ipaddress Library, fixes https://nvd.nist.gov/vuln/detail/CVE-2023-50570 Signed-off-by: Monu Singh --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index aa4422a7..ddfea32a 100644 --- a/build.gradle +++ b/build.gradle @@ -138,7 +138,7 @@ dependencies { implementation "org.jetbrains.kotlin:kotlin-stdlib:${kotlin_version}" implementation "org.jetbrains.kotlin:kotlin-stdlib-common:${kotlin_version}" implementation "org.jetbrains:annotations:13.0" - implementation "com.github.seancfoley:ipaddress:5.3.3" + implementation "com.github.seancfoley:ipaddress:5.4.0" implementation "org.jetbrains.kotlinx:kotlinx-coroutines-core:1.6.0" // Moving away from kotlin_version implementation "org.opensearch:common-utils:${common_utils_version}" implementation "org.apache.httpcomponents.client5:httpclient5:5.1.3" From 532a57fe5b4619a1fbe6c576b54161fda67e4506 Mon Sep 17 00:00:00 2001 From: Nishant Goel <113011736+nisgoel-amazon@users.noreply.github.com> Date: Wed, 3 Jan 2024 11:37:36 +0530 Subject: [PATCH 126/157] Ignoring update of final type settings in updateSettings function (#1304) (#1307) Signed-off-by: Nishant Goel --- .../replication/task/index/IndexReplicationTask.kt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt index d0c812a7..48d0f331 100644 --- a/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt +++ b/src/main/kotlin/org/opensearch/replication/task/index/IndexReplicationTask.kt @@ -461,7 +461,7 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript continue } val setting = indexScopedSettings[key] - if (!setting.isPrivateIndex) { + if (!setting.isPrivateIndex && !setting.isFinal) { desiredSettingsBuilder.copy(key, settings); } } @@ -473,7 +473,7 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript if (desiredSettings.get(key) != followerSettings.get(key)) { //Not intended setting on follower side. val setting = indexScopedSettings[key] - if (indexScopedSettings.isPrivateSetting(key)) { + if (indexScopedSettings.isPrivateSetting(key) || setting.isFinal) { continue } if (!setting.isDynamic()) { @@ -486,7 +486,7 @@ open class IndexReplicationTask(id: Long, type: String, action: String, descript for (key in followerSettings.keySet()) { val setting = indexScopedSettings[key] - if (setting == null || setting.isPrivateIndex) { + if (setting == null || setting.isPrivateIndex || setting.isFinal) { continue } From 3ea239b5b308b8c053511fb2da79ac37f12b2340 Mon Sep 17 00:00:00 2001 From: Derek Ho Date: Thu, 18 Jan 2024 23:56:32 -0500 Subject: [PATCH 127/157] Remove references to admin:admin (#1298) * Remove references to admin:admin Signed-off-by: Derek Ho * Try to pass in initial admin password via env variable Signed-off-by: Derek Ho * Try using the env variable Signed-off-by: Derek Ho * Revert file deletion and add logic for integration tests Signed-off-by: Derek Ho * fix env variable Signed-off-by: Derek Ho * Update logic to be the same across all repos Signed-off-by: Derek Ho * Keep old logic Signed-off-by: Derek Ho * Change variable name Signed-off-by: Derek Ho --------- Signed-off-by: Derek Ho --- HANDBOOK.md | 2 +- build.gradle | 4 +++- scripts/integtest.sh | 11 +++++++++-- 3 files changed, 13 insertions(+), 4 deletions(-) mode change 100755 => 100644 scripts/integtest.sh diff --git a/HANDBOOK.md b/HANDBOOK.md index ebb63090..fe1dfa0d 100644 --- a/HANDBOOK.md +++ b/HANDBOOK.md @@ -66,7 +66,7 @@ plugins.security.nodes_dn_dynamic_config_enabled: true Allow connections from follower cluster on the leader as follows ```bash -curl -k -u admin:admin -XPUT "https://${LEADER}/_plugins/_security/api/nodesdn/follower" \ +curl -k -u admin: -XPUT "https://${LEADER}/_plugins/_security/api/nodesdn/follower" \ -H 'Content-type: application/json' \ -d'{"nodes_dn": ["CN=follower.example.com"]}' ``` diff --git a/build.gradle b/build.gradle index ddfea32a..3774dbb9 100644 --- a/build.gradle +++ b/build.gradle @@ -34,6 +34,7 @@ import org.opensearch.gradle.test.RestIntegTestTask buildscript { ext { + System.setProperty("OPENSEARCH_INITIAL_ADMIN_PASSWORD", "myStrongPassword123!") isSnapshot = "true" == System.getProperty("build.snapshot", "true") opensearch_version = System.getProperty("opensearch.version", "3.0.0-SNAPSHOT") buildVersionQualifier = System.getProperty("build.version_qualifier", "") @@ -429,7 +430,7 @@ def configureCluster(OpenSearchCluster cluster, Boolean securityEnabled) { } CrossClusterWaitForHttpResource wait = new CrossClusterWaitForHttpResource(protocol, cluster.getFirstNode().getHttpSocketURI(), cluster.nodes.size()) wait.setUsername("admin") - wait.setPassword("admin") + wait.setPassword(System.getProperty("OPENSEARCH_INITIAL_ADMIN_PASSWORD")) return wait.wait(500) } @@ -682,6 +683,7 @@ clusters.each { name -> if (securityEnabled) { plugin(provider(securityPluginOld)) cliSetup("opensearch-security/install_demo_configuration.sh", "-y") + } // Currently fetching the ARCHIVE distribution fails on mac as it tries to fetch the Mac specific "DARWIN" distribution // for Opensearch which is not publish yet. Changing this to INTEG_TEST to make it work on mac. diff --git a/scripts/integtest.sh b/scripts/integtest.sh old mode 100755 new mode 100644 index 1a0498ae..464108c7 --- a/scripts/integtest.sh +++ b/scripts/integtest.sh @@ -70,9 +70,16 @@ then SECURITY_ENABLED="true" fi +IFS='.' read -ra version_array <<< "$OPENSEARCH_VERSION" + if [ -z "$CREDENTIAL" ] then - CREDENTIAL="admin:admin" + # Starting in 2.12.0, security demo configuration script requires an initial admin password + if (( ${version_array[0]} > 2 || (${version_array[0]} == 2 && ${version_array[1]} >= 12) )); then + CREDENTIAL="admin:myStrongPassword123!" + else + CREDENTIAL="admin:admin" + fi fi USERNAME=`echo $CREDENTIAL | awk -F ':' '{print $1}'` @@ -118,4 +125,4 @@ else TRANSPORT_PORT="9300" fi ./gradlew singleClusterSanityTest -Dfollower.http_host="$BIND_ADDRESS:$BIND_PORT" -Dfollower.transport_host="$BIND_ADDRESS:$TRANSPORT_PORT" -Dsecurity_enabled=$SECURITY_ENABLED -Duser=$USERNAME -Dpassword=$PASSWORD --console=plain -fi +fi \ No newline at end of file From 043a7882a369399b07fead3b1d146aefc359d03a Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Fri, 2 Feb 2024 15:04:16 +0530 Subject: [PATCH 128/157] Fix build failures (#1315) * Fix build failures Signed-off-by: monusingh-1 * Enable xml report Signed-off-by: monusingh-1 --------- Signed-off-by: monusingh-1 --- build.gradle | 9 ++++----- gradle/wrapper/gradle-wrapper.properties | 2 +- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/build.gradle b/build.gradle index 3774dbb9..e7dbd2c5 100644 --- a/build.gradle +++ b/build.gradle @@ -82,7 +82,7 @@ buildscript { } plugins { - id 'nebula.ospackage' version "8.3.0" + id 'com.netflix.nebula.ospackage' version "11.6.0" id "com.dorongold.task-tree" version "1.5" id "jacoco" } @@ -166,15 +166,14 @@ repositories { compileKotlin { kotlinOptions { - // This should be 11, but the OpenSearch logger usage checker tool doesn't like classes > 1.8 - jvmTarget = "1.8" + jvmTarget = "11" freeCompilerArgs = ['-Xjsr305=strict'] // Handle OpenSearch @Nullable annotation correctly } } compileTestKotlin { kotlinOptions { - jvmTarget = "1.8" + jvmTarget = "11" freeCompilerArgs = ['-Xjsr305=strict'] } } @@ -536,7 +535,7 @@ jacocoTestReport { dependsOn test dependsOn integTest reports { - xml.enabled true + xml.required.set(true) } // We're combining the coverage data for both test and integ tests. getExecutionData().setFrom(fileTree(buildDir).include("/jacoco/*.exec")) diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index 68efe1de..8d687878 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -11,6 +11,6 @@ distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists -distributionUrl=https\://services.gradle.org/distributions/gradle-7.6.1-all.zip +distributionUrl=https\://services.gradle.org/distributions/gradle-8.4-all.zip zipStoreBase=GRADLE_USER_HOME zipStorePath=wrapper/dists From 720e0805b4515acbc07a7c192b8db48f17c99a38 Mon Sep 17 00:00:00 2001 From: Mohit Kumar <113413713+mohitamg@users.noreply.github.com> Date: Tue, 6 Feb 2024 21:25:39 +0530 Subject: [PATCH 129/157] Create opensearch-cross-cluster-replication.release-notes-2.12.0.0.md (#1321) Add Release notes for OS 2.12.0 Signed-off-by: Mohit Kumar <113413713+mohitamg@users.noreply.github.com> --- ...h-cross-cluster-replication.release-notes-2.12.0.0.md | 9 +++++++++ 1 file changed, 9 insertions(+) create mode 100644 release-notes/opensearch-cross-cluster-replication.release-notes-2.12.0.0.md diff --git a/release-notes/opensearch-cross-cluster-replication.release-notes-2.12.0.0.md b/release-notes/opensearch-cross-cluster-replication.release-notes-2.12.0.0.md new file mode 100644 index 00000000..e8d5c2d1 --- /dev/null +++ b/release-notes/opensearch-cross-cluster-replication.release-notes-2.12.0.0.md @@ -0,0 +1,9 @@ +## Version 2.12.0 Release Notes + +Compatible with OpenSearch 2.12.0 + +## Bug Fixes + +* Implement getSystemIndexDescriptors to support SystemIndex for replication plugin ([#1290](https://github.com/opensearch-project/cross-cluster-replication/pull/1290)) +* Correct error message including what fields are missing when field are not passed when starting replication ([#1292](https://github.com/opensearch-project/cross-cluster-replication/pull/1292)) +* Ignoring all the final settings to copy from leader to follower as those settings won't be able to apply as those are not updatable ([#1304](https://github.com/opensearch-project/cross-cluster-replication/pull/1304)) From b011467b34e7aae2ada814e2323a2f5ca329d50d Mon Sep 17 00:00:00 2001 From: Darshit Chanpura Date: Mon, 5 Feb 2024 12:37:27 -0500 Subject: [PATCH 130/157] Updates integTest behavior to accept the version and set the password accordingly Signed-off-by: Darshit Chanpura --- scripts/integtest.sh | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/scripts/integtest.sh b/scripts/integtest.sh index 464108c7..0d64d05a 100644 --- a/scripts/integtest.sh +++ b/scripts/integtest.sh @@ -50,7 +50,7 @@ while getopts ":h:b:p:t:e:s:c:v:" arg; do CREDENTIAL=$OPTARG ;; v) - # Do nothing as we're not consuming this param. + OPENSEARCH_VERSION=$OPTARG ;; :) echo "-${OPTARG} requires an argument" @@ -70,15 +70,16 @@ then SECURITY_ENABLED="true" fi -IFS='.' read -ra version_array <<< "$OPENSEARCH_VERSION" +OPENSEARCH_REQUIRED_VERSION="2.12.0" if [ -z "$CREDENTIAL" ] then # Starting in 2.12.0, security demo configuration script requires an initial admin password - if (( ${version_array[0]} > 2 || (${version_array[0]} == 2 && ${version_array[1]} >= 12) )); then - CREDENTIAL="admin:myStrongPassword123!" - else + COMPARE_VERSION=`echo $OPENSEARCH_REQUIRED_VERSION $OPENSEARCH_VERSION | tr ' ' '\n' | sort -V | uniq | head -n 1` + if [ "$COMPARE_VERSION" != "$OPENSEARCH_REQUIRED_VERSION" ]; then CREDENTIAL="admin:admin" + else + CREDENTIAL="admin:myStrongPassword123!" fi fi From 581cff05a3efae001225f1578fa254c5e80369c0 Mon Sep 17 00:00:00 2001 From: Darshit Chanpura Date: Mon, 5 Feb 2024 13:07:55 -0500 Subject: [PATCH 131/157] Updates credentials Signed-off-by: Darshit Chanpura --- scripts/integtest.sh | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/scripts/integtest.sh b/scripts/integtest.sh index 0d64d05a..39d94c73 100644 --- a/scripts/integtest.sh +++ b/scripts/integtest.sh @@ -70,17 +70,9 @@ then SECURITY_ENABLED="true" fi -OPENSEARCH_REQUIRED_VERSION="2.12.0" - if [ -z "$CREDENTIAL" ] then - # Starting in 2.12.0, security demo configuration script requires an initial admin password - COMPARE_VERSION=`echo $OPENSEARCH_REQUIRED_VERSION $OPENSEARCH_VERSION | tr ' ' '\n' | sort -V | uniq | head -n 1` - if [ "$COMPARE_VERSION" != "$OPENSEARCH_REQUIRED_VERSION" ]; then - CREDENTIAL="admin:admin" - else - CREDENTIAL="admin:myStrongPassword123!" - fi + CREDENTIAL="admin:admin" # CCR uses custom setup and hence doesn't require custom password fi USERNAME=`echo $CREDENTIAL | awk -F ':' '{print $1}'` From 2612f572b7c6a91b93a89298d4ef3105574bcab8 Mon Sep 17 00:00:00 2001 From: Darshit Chanpura Date: Mon, 5 Feb 2024 13:39:53 -0500 Subject: [PATCH 132/157] Updates credentials to be admin Signed-off-by: Darshit Chanpura --- build.gradle | 3 +-- scripts/integtest.sh | 10 +++++++++- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/build.gradle b/build.gradle index e7dbd2c5..40819857 100644 --- a/build.gradle +++ b/build.gradle @@ -34,7 +34,6 @@ import org.opensearch.gradle.test.RestIntegTestTask buildscript { ext { - System.setProperty("OPENSEARCH_INITIAL_ADMIN_PASSWORD", "myStrongPassword123!") isSnapshot = "true" == System.getProperty("build.snapshot", "true") opensearch_version = System.getProperty("opensearch.version", "3.0.0-SNAPSHOT") buildVersionQualifier = System.getProperty("build.version_qualifier", "") @@ -429,7 +428,7 @@ def configureCluster(OpenSearchCluster cluster, Boolean securityEnabled) { } CrossClusterWaitForHttpResource wait = new CrossClusterWaitForHttpResource(protocol, cluster.getFirstNode().getHttpSocketURI(), cluster.nodes.size()) wait.setUsername("admin") - wait.setPassword(System.getProperty("OPENSEARCH_INITIAL_ADMIN_PASSWORD")) + wait.setPassword("admin") return wait.wait(500) } diff --git a/scripts/integtest.sh b/scripts/integtest.sh index 39d94c73..0d64d05a 100644 --- a/scripts/integtest.sh +++ b/scripts/integtest.sh @@ -70,9 +70,17 @@ then SECURITY_ENABLED="true" fi +OPENSEARCH_REQUIRED_VERSION="2.12.0" + if [ -z "$CREDENTIAL" ] then - CREDENTIAL="admin:admin" # CCR uses custom setup and hence doesn't require custom password + # Starting in 2.12.0, security demo configuration script requires an initial admin password + COMPARE_VERSION=`echo $OPENSEARCH_REQUIRED_VERSION $OPENSEARCH_VERSION | tr ' ' '\n' | sort -V | uniq | head -n 1` + if [ "$COMPARE_VERSION" != "$OPENSEARCH_REQUIRED_VERSION" ]; then + CREDENTIAL="admin:admin" + else + CREDENTIAL="admin:myStrongPassword123!" + fi fi USERNAME=`echo $CREDENTIAL | awk -F ':' '{print $1}'` From 341e067aa65a405d28f20399eb334a87fbfaec67 Mon Sep 17 00:00:00 2001 From: Darshit Chanpura Date: Thu, 8 Feb 2024 10:41:20 -0500 Subject: [PATCH 133/157] Forces HTTP1 version policy for local clusters Signed-off-by: Darshit Chanpura --- .../org/opensearch/replication/MultiClusterRestTestCase.kt | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt b/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt index eca8750f..090bdf00 100644 --- a/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt +++ b/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt @@ -27,6 +27,7 @@ import org.apache.hc.core5.http.message.BasicHeader import org.apache.hc.core5.http.io.entity.StringEntity import org.apache.hc.core5.ssl.SSLContexts import org.apache.hc.core5.http.io.entity.EntityUtils +import org.apache.hc.core5.http2.HttpVersionPolicy import org.apache.hc.core5.util.Timeout import org.apache.lucene.util.SetOnce import org.opensearch.action.admin.cluster.node.tasks.list.ListTasksRequest @@ -102,6 +103,7 @@ abstract class MultiClusterRestTestCase : OpenSearchTestCase() { val builder = RestClient.builder(*httpHosts.toTypedArray()).setHttpClientConfigCallback { httpAsyncClientBuilder -> httpAsyncClientBuilder.setConnectionManager(connManager) + httpAsyncClientBuilder.setVersionPolicy(HttpVersionPolicy.FORCE_HTTP_1) } configureClient(builder, getClusterSettings(clusterName), securityEnabled) builder.setStrictDeprecationMode(false) From d703887f88f24c444e0513ebeca21e30b93354f8 Mon Sep 17 00:00:00 2001 From: Darshit Chanpura Date: Fri, 9 Feb 2024 10:46:52 -0500 Subject: [PATCH 134/157] Adds more comments Signed-off-by: Darshit Chanpura --- scripts/integtest.sh | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/scripts/integtest.sh b/scripts/integtest.sh index 0d64d05a..f967dc8a 100644 --- a/scripts/integtest.sh +++ b/scripts/integtest.sh @@ -75,8 +75,11 @@ OPENSEARCH_REQUIRED_VERSION="2.12.0" if [ -z "$CREDENTIAL" ] then # Starting in 2.12.0, security demo configuration script requires an initial admin password - COMPARE_VERSION=`echo $OPENSEARCH_REQUIRED_VERSION $OPENSEARCH_VERSION | tr ' ' '\n' | sort -V | uniq | head -n 1` - if [ "$COMPARE_VERSION" != "$OPENSEARCH_REQUIRED_VERSION" ]; then + # Pick the minimum of two versions + VERSION_TO_COMPARE=`echo $OPENSEARCH_REQUIRED_VERSION $OPENSEARCH_VERSION | tr ' ' '\n' | sort -V | uniq | head -n 1` + # Check if the compared version is not equal to the required version. + # If it is not equal, it means the current version is older. + if [ "$VERSION_TO_COMPARE" != "$OPENSEARCH_REQUIRED_VERSION" ]; then CREDENTIAL="admin:admin" else CREDENTIAL="admin:myStrongPassword123!" From 3f4f2163d61ff660d608c39412802dea383a8426 Mon Sep 17 00:00:00 2001 From: Darshit Chanpura <35282393+DarshitChanpura@users.noreply.github.com> Date: Mon, 12 Feb 2024 05:37:24 -0500 Subject: [PATCH 135/157] Updates 2.12 release notes (#1329) Signed-off-by: Darshit Chanpura --- build.gradle | 1 - ...nsearch-cross-cluster-replication.release-notes-2.12.0.0.md | 3 +++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 40819857..2fee9e0e 100644 --- a/build.gradle +++ b/build.gradle @@ -681,7 +681,6 @@ clusters.each { name -> if (securityEnabled) { plugin(provider(securityPluginOld)) cliSetup("opensearch-security/install_demo_configuration.sh", "-y") - } // Currently fetching the ARCHIVE distribution fails on mac as it tries to fetch the Mac specific "DARWIN" distribution // for Opensearch which is not publish yet. Changing this to INTEG_TEST to make it work on mac. diff --git a/release-notes/opensearch-cross-cluster-replication.release-notes-2.12.0.0.md b/release-notes/opensearch-cross-cluster-replication.release-notes-2.12.0.0.md index e8d5c2d1..19a391a5 100644 --- a/release-notes/opensearch-cross-cluster-replication.release-notes-2.12.0.0.md +++ b/release-notes/opensearch-cross-cluster-replication.release-notes-2.12.0.0.md @@ -7,3 +7,6 @@ Compatible with OpenSearch 2.12.0 * Implement getSystemIndexDescriptors to support SystemIndex for replication plugin ([#1290](https://github.com/opensearch-project/cross-cluster-replication/pull/1290)) * Correct error message including what fields are missing when field are not passed when starting replication ([#1292](https://github.com/opensearch-project/cross-cluster-replication/pull/1292)) * Ignoring all the final settings to copy from leader to follower as those settings won't be able to apply as those are not updatable ([#1304](https://github.com/opensearch-project/cross-cluster-replication/pull/1304)) + +## Maintenance +* Updates integTest behavior to accept the version and set the password accordingly, removes admin:admin references and force HTTP1 policy for local clusters([#1298](https://github.com/opensearch-project/cross-cluster-replication/pull/1298))([#1318](https://github.com/opensearch-project/cross-cluster-replication/pull/1318)) From 74dfcd169d202c1b3eab8e17f3b7bff9461b85d6 Mon Sep 17 00:00:00 2001 From: Darshit Chanpura <35282393+DarshitChanpura@users.noreply.github.com> Date: Wed, 14 Feb 2024 23:18:06 -0500 Subject: [PATCH 136/157] Fixes base64 encoded hardcoded admin credentials (#1331) * Fixes base64 encoded hardcoded admin credentials Signed-off-by: Darshit Chanpura * Fixes broken integTest task Signed-off-by: Darshit Chanpura --------- Signed-off-by: Darshit Chanpura --- build.gradle | 1 + .../org/opensearch/replication/MultiClusterRestTestCase.kt | 6 +++++- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 2fee9e0e..e4e20d62 100644 --- a/build.gradle +++ b/build.gradle @@ -450,6 +450,7 @@ int endJmxPort = startJmxPort integTest { useCluster testClusters.leaderCluster useCluster testClusters.followCluster + systemProperty "password", "admin" // setting it to `admin` explicitly since its a custom security setup if(knnEnabled){ nonInputProperties.systemProperty('tests.knn_plugin_enabled', "true") diff --git a/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt b/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt index 090bdf00..185cadc4 100644 --- a/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt +++ b/src/test/kotlin/org/opensearch/replication/MultiClusterRestTestCase.kt @@ -57,12 +57,14 @@ import org.junit.After import org.junit.AfterClass import org.junit.Before import org.junit.BeforeClass +import java.nio.charset.StandardCharsets import java.nio.file.Files import java.security.KeyManagementException import java.security.KeyStore import java.security.KeyStoreException import java.security.NoSuchAlgorithmException import java.security.cert.CertificateException +import java.util.Base64 import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicReference import java.util.Collections @@ -233,8 +235,10 @@ abstract class MultiClusterRestTestCase : OpenSearchTestCase() { for ((key, value) in headers) { defaultHeaders[i++] = BasicHeader(key, value) } + + val creds = System.getProperty("user", "admin") + ":" + System.getProperty("password", "myStrongPassword123!") if(securityEnabled) { - defaultHeaders[i++] = BasicHeader("Authorization", "Basic YWRtaW46YWRtaW4=") + defaultHeaders[i++] = BasicHeader("Authorization", "Basic " + Base64.getEncoder().encodeToString(creds.toByteArray(StandardCharsets.UTF_8))) } builder.setDefaultHeaders(defaultHeaders) From bfbfdc608e0ba668d658c07ff67412d854011721 Mon Sep 17 00:00:00 2001 From: Mohit Kumar <113413713+mohitamg@users.noreply.github.com> Date: Thu, 29 Feb 2024 15:39:56 +0530 Subject: [PATCH 137/157] Fix for CVE-2023-50570(Bumping up to latest version of ipaddress library (#1339) Signed-off-by: Mohit Kumar --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index e4e20d62..4eb4e250 100644 --- a/build.gradle +++ b/build.gradle @@ -138,7 +138,7 @@ dependencies { implementation "org.jetbrains.kotlin:kotlin-stdlib:${kotlin_version}" implementation "org.jetbrains.kotlin:kotlin-stdlib-common:${kotlin_version}" implementation "org.jetbrains:annotations:13.0" - implementation "com.github.seancfoley:ipaddress:5.4.0" + implementation "com.github.seancfoley:ipaddress:5.4.1" implementation "org.jetbrains.kotlinx:kotlinx-coroutines-core:1.6.0" // Moving away from kotlin_version implementation "org.opensearch:common-utils:${common_utils_version}" implementation "org.apache.httpcomponents.client5:httpclient5:5.1.3" From da8746ff50fbf1d8b8f8d4bce3c312df94607f59 Mon Sep 17 00:00:00 2001 From: Monu Singh Date: Mon, 18 Mar 2024 11:33:36 +0530 Subject: [PATCH 138/157] Update Lucene snapshot location (#1351) Signed-off-by: monusingh-1 --- build.gradle | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/build.gradle b/build.gradle index 4eb4e250..c6658dd5 100644 --- a/build.gradle +++ b/build.gradle @@ -68,7 +68,7 @@ buildscript { mavenCentral() maven { url "https://aws.oss.sonatype.org/content/repositories/snapshots" } maven { url "https://plugins.gradle.org/m2/" } - maven { url "https://d1nvenhzbhpy0q.cloudfront.net/snapshots/lucene/" } + maven { url "https://artifacts.opensearch.org/snapshots/lucene/" } } dependencies { @@ -160,7 +160,7 @@ repositories { mavenCentral() maven { url "https://aws.oss.sonatype.org/content/repositories/snapshots" } maven { url "https://plugins.gradle.org/m2/" } - maven { url "https://d1nvenhzbhpy0q.cloudfront.net/snapshots/lucene/" } + maven { url "https://artifacts.opensearch.org/snapshots/lucene/" } } compileKotlin { From 3e35b09ed58695a094512121ee44ad5645ad8076 Mon Sep 17 00:00:00 2001 From: Mohit Kumar <113413713+mohitamg@users.noreply.github.com> Date: Tue, 19 Mar 2024 13:39:28 +0530 Subject: [PATCH 139/157] Added release notes for OS-cross-cluster-replication_2.13.0 (#1353) Signed-off-by: Mohit Kumar --- ...ross-cluster-replication.release-notes-2.13.0.0.md | 11 +++++++++++ 1 file changed, 11 insertions(+) create mode 100644 release-notes/opensearch-cross-cluster-replication.release-notes-2.13.0.0.md diff --git a/release-notes/opensearch-cross-cluster-replication.release-notes-2.13.0.0.md b/release-notes/opensearch-cross-cluster-replication.release-notes-2.13.0.0.md new file mode 100644 index 00000000..5573ce7b --- /dev/null +++ b/release-notes/opensearch-cross-cluster-replication.release-notes-2.13.0.0.md @@ -0,0 +1,11 @@ +## Version 2.13.0 Release Notes + +Compatible with OpenSearch 2.13.0 + +## Security Fixes + +* Bumped up to latest version of ipaddress library ([#1339](https://github.com/opensearch-project/cross-cluster-replication/pull/1339)) + +## Maintenance + +* Updated lucene snapshot location to prevent builds to fail between file deletion and re-upload when snapshots are updated ([#1351](https://github.com/opensearch-project/cross-cluster-replication/pull/1351)) From 90fcb8bc9c498ad69139969d5eafbdbb87362675 Mon Sep 17 00:00:00 2001 From: Mohit Kumar <113413713+mohitamg@users.noreply.github.com> Date: Tue, 16 Apr 2024 16:47:21 +0530 Subject: [PATCH 140/157] Release notes for opensearch-cross-cluster-replication 1.3.16 (#1366) --- ...pensearch-cross-cluster-replication.release-notes-1.3.16.md | 3 +++ 1 file changed, 3 insertions(+) create mode 100644 release-notes/opensearch-cross-cluster-replication.release-notes-1.3.16.md diff --git a/release-notes/opensearch-cross-cluster-replication.release-notes-1.3.16.md b/release-notes/opensearch-cross-cluster-replication.release-notes-1.3.16.md new file mode 100644 index 00000000..83e72919 --- /dev/null +++ b/release-notes/opensearch-cross-cluster-replication.release-notes-1.3.16.md @@ -0,0 +1,3 @@ +## Version 1.3.16 Release Notes + +Compatible with OpenSearch 1.3.16 \ No newline at end of file From ff42012067e0b14e0e15b79c9be542cbdff08031 Mon Sep 17 00:00:00 2001 From: Shubh Sahu Date: Tue, 23 Apr 2024 09:56:21 +0530 Subject: [PATCH 141/157] =?UTF-8?q?Adding=20support=20to=20fetch=20changes?= =?UTF-8?q?=20from=20Lucene=20store=20while=20migrating=20from/to=20r?= =?UTF-8?q?=E2=80=A6=20(#1369)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Shubh Sahu --- build.gradle | 2 + .../replication/ReplicationPlugin.kt | 2 +- .../changes/TransportGetChangesAction.kt | 20 +++---- .../replication/util/ValidationUtil.kt | 12 ++-- .../integ/rest/StartReplicationIT.kt | 59 +++++++++++++++++++ 5 files changed, 80 insertions(+), 15 deletions(-) diff --git a/build.gradle b/build.gradle index c6658dd5..7563e849 100644 --- a/build.gradle +++ b/build.gradle @@ -380,6 +380,8 @@ testClusters { testDistribution = "ARCHIVE" } int debugPort = 5005 + //adding it to test migration + systemProperty('opensearch.experimental.feature.remote_store.migration.enabled','true') if (_numNodes > 1) numberOfNodes = _numNodes //numberOfNodes = 3 diff --git a/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt b/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt index 792ebe27..3790c3c0 100644 --- a/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt +++ b/src/main/kotlin/org/opensearch/replication/ReplicationPlugin.kt @@ -387,7 +387,7 @@ internal class ReplicationPlugin : Plugin(), ActionPlugin, PersistentTaskPlugin, override fun getCustomTranslogDeletionPolicyFactory(): Optional { // We don't need a retention lease translog deletion policy for remote store enabled clusters as // we fetch the operations directly from lucene in such cases. - return if (ValidationUtil.isRemoteStoreEnabledCluster(clusterService) == false) { + return if (ValidationUtil.isRemoteEnabledOrMigrating(clusterService) == false) { Optional.of(TranslogDeletionPolicyFactory { indexSettings, retentionLeasesSupplier -> ReplicationTranslogDeletionPolicy(indexSettings, retentionLeasesSupplier) }) diff --git a/src/main/kotlin/org/opensearch/replication/action/changes/TransportGetChangesAction.kt b/src/main/kotlin/org/opensearch/replication/action/changes/TransportGetChangesAction.kt index 392555b5..4e364906 100644 --- a/src/main/kotlin/org/opensearch/replication/action/changes/TransportGetChangesAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/changes/TransportGetChangesAction.kt @@ -77,8 +77,8 @@ class TransportGetChangesAction @Inject constructor(threadPool: ThreadPool, clus indexMetric.lastFetchTime.set(relativeStartNanos) val indexShard = indicesService.indexServiceSafe(shardId.index).getShard(shardId.id) - val isRemoteStoreEnabled = ValidationUtil.isRemoteStoreEnabledCluster(clusterService) - if (lastGlobalCheckpoint(indexShard, isRemoteStoreEnabled) < request.fromSeqNo) { + val isRemoteEnabledOrMigrating = ValidationUtil.isRemoteEnabledOrMigrating(clusterService) + if (lastGlobalCheckpoint(indexShard, isRemoteEnabledOrMigrating) < request.fromSeqNo) { // There are no new operations to sync. Do a long poll and wait for GlobalCheckpoint to advance. If // the checkpoint doesn't advance by the timeout this throws an ESTimeoutException which the caller // should catch and start a new poll. @@ -87,18 +87,18 @@ class TransportGetChangesAction @Inject constructor(threadPool: ThreadPool, clus // At this point indexShard.lastKnownGlobalCheckpoint has advanced but it may not yet have been synced // to the translog, which means we can't return those changes. Return to the caller to retry. // TODO: Figure out a better way to wait for the global checkpoint to be synced to the translog - if (lastGlobalCheckpoint(indexShard, isRemoteStoreEnabled) < request.fromSeqNo) { - assert(gcp > lastGlobalCheckpoint(indexShard, isRemoteStoreEnabled)) { "Checkpoint didn't advance at all $gcp ${lastGlobalCheckpoint(indexShard, isRemoteStoreEnabled)}" } + if (lastGlobalCheckpoint(indexShard, isRemoteEnabledOrMigrating) < request.fromSeqNo) { + assert(gcp > lastGlobalCheckpoint(indexShard, isRemoteEnabledOrMigrating)) { "Checkpoint didn't advance at all $gcp ${lastGlobalCheckpoint(indexShard, isRemoteEnabledOrMigrating)}" } throw OpenSearchTimeoutException("global checkpoint not synced. Retry after a few miliseconds...") } } relativeStartNanos = System.nanoTime() // At this point lastSyncedGlobalCheckpoint is at least fromSeqNo - val toSeqNo = min(lastGlobalCheckpoint(indexShard, isRemoteStoreEnabled), request.toSeqNo) + val toSeqNo = min(lastGlobalCheckpoint(indexShard, isRemoteEnabledOrMigrating), request.toSeqNo) var ops: List = listOf() - var fetchFromTranslog = isTranslogPruningByRetentionLeaseEnabled(shardId) && isRemoteStoreEnabled == false + var fetchFromTranslog = isTranslogPruningByRetentionLeaseEnabled(shardId) && isRemoteEnabledOrMigrating == false if(fetchFromTranslog) { try { ops = translogService.getHistoryOfOperations(indexShard, request.fromSeqNo, toSeqNo) @@ -136,16 +136,16 @@ class TransportGetChangesAction @Inject constructor(threadPool: ThreadPool, clus indexMetric.ops.addAndGet(ops.size.toLong()) ops.stream().forEach{op -> indexMetric.bytesRead.addAndGet(op.estimateSize()) } - GetChangesResponse(ops, request.fromSeqNo, indexShard.maxSeqNoOfUpdatesOrDeletes, lastGlobalCheckpoint(indexShard, isRemoteStoreEnabled)) + GetChangesResponse(ops, request.fromSeqNo, indexShard.maxSeqNoOfUpdatesOrDeletes, lastGlobalCheckpoint(indexShard, isRemoteEnabledOrMigrating)) } } } - private fun lastGlobalCheckpoint(indexShard: IndexShard, isRemoteStoreEnabled: Boolean): Long { + private fun lastGlobalCheckpoint(indexShard: IndexShard, isRemoteEnabledOrMigrating: Boolean): Long { // We rely on lastSyncedGlobalCheckpoint as it has been durably written to disk. In case of remote store // enabled clusters, the semantics are slightly different, and we can't use lastSyncedGlobalCheckpoint. Falling back to // lastKnownGlobalCheckpoint in such cases. - return if (isRemoteStoreEnabled) { + return if (isRemoteEnabledOrMigrating) { indexShard.lastKnownGlobalCheckpoint } else { indexShard.lastSyncedGlobalCheckpoint @@ -173,7 +173,7 @@ class TransportGetChangesAction @Inject constructor(threadPool: ThreadPool, clus override fun shards(state: ClusterState, request: InternalRequest): ShardsIterator { val shardIt = state.routingTable().shardRoutingTable(request.request().shardId) // Random active shards - return if (ValidationUtil.isRemoteStoreEnabledCluster(clusterService)) shardIt.primaryShardIt() + return if (ValidationUtil.isRemoteEnabledOrMigrating(clusterService)) shardIt.primaryShardIt() else shardIt.activeInitializingShardsRandomIt() } } \ No newline at end of file diff --git a/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt b/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt index afbcdf60..91d012c2 100644 --- a/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt +++ b/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt @@ -17,18 +17,18 @@ import org.opensearch.Version import org.opensearch.cluster.ClusterState import org.opensearch.cluster.metadata.IndexMetadata import org.opensearch.cluster.metadata.MetadataCreateIndexService -import org.opensearch.core.common.Strings +import org.opensearch.cluster.service.ClusterService import org.opensearch.common.ValidationException import org.opensearch.common.settings.Settings +import org.opensearch.core.common.Strings import org.opensearch.env.Environment import org.opensearch.index.IndexNotFoundException -import java.io.UnsupportedEncodingException -import org.opensearch.cluster.service.ClusterService import org.opensearch.node.Node import org.opensearch.node.remotestore.RemoteStoreNodeAttribute +import org.opensearch.node.remotestore.RemoteStoreNodeService import org.opensearch.replication.ReplicationPlugin.Companion.KNN_INDEX_SETTING import org.opensearch.replication.ReplicationPlugin.Companion.KNN_PLUGIN_PRESENT_SETTING -import org.opensearch.replication.action.changes.TransportGetChangesAction +import java.io.UnsupportedEncodingException import java.nio.file.Files import java.nio.file.Path import java.util.Locale @@ -161,4 +161,8 @@ object ValidationUtil { return clusterService.settings.getByPrefix(Node.NODE_ATTRIBUTES.key + RemoteStoreNodeAttribute.REMOTE_STORE_NODE_ATTRIBUTE_KEY_PREFIX).isEmpty == false } + fun isRemoteEnabledOrMigrating(clusterService: ClusterService): Boolean { + return isRemoteStoreEnabledCluster(clusterService) || + clusterService.clusterSettings.get(RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING).equals(RemoteStoreNodeService.CompatibilityMode.MIXED) + } } diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt index 0fa3c5ab..a0238b93 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/StartReplicationIT.kt @@ -79,7 +79,10 @@ import java.nio.file.Files import java.util.* import java.util.concurrent.TimeUnit import org.opensearch.bootstrap.BootstrapInfo +import org.opensearch.cluster.service.ClusterService import org.opensearch.index.mapper.Mapping +import org.opensearch.indices.replication.common.ReplicationType +import org.opensearch.replication.util.ValidationUtil @MultiClusterAnnotations.ClusterConfigurations( @@ -1255,6 +1258,62 @@ class StartReplicationIT: MultiClusterRestTestCase() { ) } + fun `test operations are fetched from lucene when leader is in mixed mode`() { + + val leaderClient = getClientForCluster(LEADER) + val followerClient = getClientForCluster(FOLLOWER) + + // create index on leader cluster + val settings = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .build() + val createIndexResponse = leaderClient.indices().create( + CreateIndexRequest(leaderIndexName).settings(settings), + RequestOptions.DEFAULT + ) + assertThat(createIndexResponse.isAcknowledged).isTrue() + + // Update leader cluster settings to enable mixed mode and set migration direction to remote_store + val leaderClusterUpdateSettingsRequest = Request("PUT", "_cluster/settings") + val entityAsString = """ + { + "persistent": { + "remote_store.compatibility_mode": "mixed", + "migration.direction" : "remote_store" + } + }""".trimMargin() + + leaderClusterUpdateSettingsRequest.entity = StringEntity(entityAsString,ContentType.APPLICATION_JSON) + val updateSettingResponse = leaderClient.lowLevelClient.performRequest(leaderClusterUpdateSettingsRequest) + assertEquals(HttpStatus.SC_OK.toLong(), updateSettingResponse.statusLine.statusCode.toLong()) + + //create connection and start replication + createConnectionBetweenClusters(FOLLOWER, LEADER) + + followerClient.startReplication( + StartReplicationRequest("source", leaderIndexName, followerIndexName), + TimeValue.timeValueSeconds(10), + true + ) + + //Index documents on leader index + val docCount = 50 + for (i in 1..docCount) { + val sourceMap = mapOf("name" to randomAlphaOfLength(5)) + leaderClient.index(IndexRequest(leaderIndexName).id(i.toString()).source(sourceMap), RequestOptions.DEFAULT) + } + + // Verify that all the documents are replicated to follower index and are fetched from lucene + assertBusy({ + val stats = leaderClient.leaderStats() + assertThat(stats.size).isEqualTo(9) + assertThat(stats.getValue("num_replicated_indices").toString()).isEqualTo("1") + assertThat(stats.getValue("operations_read").toString()).isEqualTo(docCount.toString()) + assertThat(stats.getValue("operations_read_lucene").toString()).isEqualTo(docCount.toString()) + assertThat(stats.getValue("operations_read_translog").toString()).isEqualTo("0") + assertThat(stats.containsKey("index_stats")) + }, 60L, TimeUnit.SECONDS) + } private fun excludeAllClusterNodes(clusterName: String) { val transientSettingsRequest = Request("PUT", "_cluster/settings") From d4047421120b0f24a12fd67eded3e698b25d05f8 Mon Sep 17 00:00:00 2001 From: Sanjay Kumar <93825455+skumarp7@users.noreply.github.com> Date: Tue, 23 Apr 2024 10:16:17 +0530 Subject: [PATCH 142/157] Handle respone for deletion of non-existing autofollow replication rule (#1371) Signed-off-by: Sanjay Kumar --- ...ransportAutoFollowClusterManagerNodeAction.kt | 1 + .../integ/rest/UpdateAutoFollowPatternIT.kt | 16 ++++++++++++++++ 2 files changed, 17 insertions(+) diff --git a/src/main/kotlin/org/opensearch/replication/action/autofollow/TransportAutoFollowClusterManagerNodeAction.kt b/src/main/kotlin/org/opensearch/replication/action/autofollow/TransportAutoFollowClusterManagerNodeAction.kt index 140a708d..51a0bd06 100644 --- a/src/main/kotlin/org/opensearch/replication/action/autofollow/TransportAutoFollowClusterManagerNodeAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/autofollow/TransportAutoFollowClusterManagerNodeAction.kt @@ -123,6 +123,7 @@ class TransportAutoFollowClusterManagerNodeAction @Inject constructor(transportS } catch(e: ResourceNotFoundException) { // Log warn as the task is already removed log.warn("Task already stopped for '$clusterAlias:$patternName'", e) + throw OpenSearchException("Autofollow replication rule $clusterAlias:$patternName does not exist") } catch (e: Exception) { log.error("Failed to stop auto follow task for cluster '$clusterAlias:$patternName'", e) throw OpenSearchException(AUTOFOLLOW_EXCEPTION_GENERIC_STRING) diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt index 57593241..78e4ead4 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt @@ -294,6 +294,22 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { .hasMessageContaining(errorMsg) } + fun `test deletion of auto follow pattern`() { + val followerClient = getClientForCluster(FOLLOWER) + createConnectionBetweenClusters(FOLLOWER, LEADER, connectionAlias) + followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, indexPattern) + //Delete a replication rule which does not exist + Assertions.assertThatThrownBy { + followerClient.deleteAutoFollowPattern(connectionAlias, "dummy_conn") + }.isInstanceOf(ResponseException::class.java) + .hasMessageContaining("does not exist") + //Delete a replication rule which exists + Assertions.assertThatCode { + followerClient.deleteAutoFollowPattern(connectionAlias, indexPatternName) + }.doesNotThrowAnyException() + + } + fun `test removing autofollow pattern stop autofollow task`() { val followerClient = getClientForCluster(FOLLOWER) val leaderClient = getClientForCluster(LEADER) From 12d285bbcd25aabb95503c739ed2695bbf0ccc9c Mon Sep 17 00:00:00 2001 From: Sooraj Sinha <81695996+soosinha@users.noreply.github.com> Date: Thu, 2 May 2024 14:14:42 +0530 Subject: [PATCH 143/157] Add release notes for 2.14 (#1376) Signed-off-by: Sooraj Sinha --- ...cross-cluster-replication.release-notes-2.14.0.0.md | 10 ++++++++++ 1 file changed, 10 insertions(+) create mode 100644 release-notes/opensearch-cross-cluster-replication.release-notes-2.14.0.0.md diff --git a/release-notes/opensearch-cross-cluster-replication.release-notes-2.14.0.0.md b/release-notes/opensearch-cross-cluster-replication.release-notes-2.14.0.0.md new file mode 100644 index 00000000..01ec5f51 --- /dev/null +++ b/release-notes/opensearch-cross-cluster-replication.release-notes-2.14.0.0.md @@ -0,0 +1,10 @@ +## Version 2.14.0.0 Release Notes + +Compatible with OpenSearch 2.14.0 + + +### Enhancements +* Support for fetching changes from Lucene store during remote store migration ([#1369](https://github.com/opensearch-project/cross-cluster-replication/pull/1369)) + +### Bug Fixes +* Handle response for deletion of non-existent autofollow replication rule ([#1371](https://github.com/opensearch-project/cross-cluster-replication/pull/1371)) From d287aa2d99894b2b036014e85a52b078534c8d14 Mon Sep 17 00:00:00 2001 From: Shubh Sahu Date: Fri, 3 May 2024 13:36:54 +0530 Subject: [PATCH 144/157] Disabling docrep to remote store migration test for integTestRemote (#1379) Signed-off-by: Shubh Sahu Co-authored-by: Shubh Sahu --- build.gradle | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 7563e849..e862827b 100644 --- a/build.gradle +++ b/build.gradle @@ -940,7 +940,8 @@ task integTestRemote (type: RestIntegTestTask) { } filter { - setExcludePatterns("org.opensearch.replication.bwc.BackwardsCompatibilityIT","org.opensearch.replication.singleCluster.SingleClusterSanityIT") + setExcludePatterns("org.opensearch.replication.bwc.BackwardsCompatibilityIT","org.opensearch.replication.singleCluster.SingleClusterSanityIT", + "org.opensearch.replication.integ.rest.StartReplicationIT.test operations are fetched from lucene when leader is in mixed mode") } } From 94e2dd9311c5470b538f0b4ca3546203631cb87d Mon Sep 17 00:00:00 2001 From: Varun Lodaya Date: Wed, 29 May 2024 15:49:35 +0530 Subject: [PATCH 145/157] Updating security reachout email Signed-off-by: Varun Lodaya --- SECURITY.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/SECURITY.md b/SECURITY.md index 0b85ca04..be4ac746 100644 --- a/SECURITY.md +++ b/SECURITY.md @@ -1,3 +1,3 @@ ## Reporting a Vulnerability -If you discover a potential security issue in this project we ask that you notify AWS/Amazon Security via our [vulnerability reporting page](http://aws.amazon.com/security/vulnerability-reporting/) or directly via email to aws-security@amazon.com. Please do **not** create a public GitHub issue. \ No newline at end of file +If you discover a potential security issue in this project we ask that you notify OpenSearch Security directly via email to security@opensearch.org. Please do **not** create a public GitHub issue. From 32f28ccda5758750c769378c2f835e9251ca9523 Mon Sep 17 00:00:00 2001 From: Mohit Kumar <113413713+mohitamg@users.noreply.github.com> Date: Sat, 15 Jun 2024 00:10:34 +0530 Subject: [PATCH 146/157] Added release notes for OS-cross-cluster-replication_2.15.0 (#1399) Signed-off-by: Mohit Kumar --- ...arch-cross-cluster-replication.release-notes-2.15.0.0.md | 6 ++++++ 1 file changed, 6 insertions(+) create mode 100644 release-notes/opensearch-cross-cluster-replication.release-notes-2.15.0.0.md diff --git a/release-notes/opensearch-cross-cluster-replication.release-notes-2.15.0.0.md b/release-notes/opensearch-cross-cluster-replication.release-notes-2.15.0.0.md new file mode 100644 index 00000000..58935225 --- /dev/null +++ b/release-notes/opensearch-cross-cluster-replication.release-notes-2.15.0.0.md @@ -0,0 +1,6 @@ +## Version 2.15.0.0 Release Notes + +Compatible with OpenSearch 2.15.0 + +### Maintenance +* Disabling docrep to remote store migration test for integTestRemote suite ([#1379](https://github.com/opensearch-project/cross-cluster-replication/pull/1379)) From d6e7636d42ef87cd483f7439d8a2cf43bf397727 Mon Sep 17 00:00:00 2001 From: Nishant Goel Date: Fri, 3 May 2024 21:58:01 +0530 Subject: [PATCH 147/157] Fixing remote integ test for windows by using awk instead of python Signed-off-by: Nishant Goel --- scripts/integtest.sh | 52 ++++++++++++++++++++++++++++++++------------ 1 file changed, 38 insertions(+), 14 deletions(-) mode change 100644 => 100755 scripts/integtest.sh diff --git a/scripts/integtest.sh b/scripts/integtest.sh old mode 100644 new mode 100755 index f967dc8a..840f4c75 --- a/scripts/integtest.sh +++ b/scripts/integtest.sh @@ -103,24 +103,48 @@ then exit 1 fi - data=$(python3 -c "import json; cluster=$ENDPOINT_LIST ; data_nodes=cluster; print(data_nodes[0][\"data_nodes\"][0][\"endpoint\"],':',data_nodes[0][\"data_nodes\"][0][\"port\"],':',data_nodes[0][\"data_nodes\"][0][\"transport\"],',',data_nodes[1][\"data_nodes\"][0][\"endpoint\"],':',data_nodes[1][\"data_nodes\"][0][\"port\"],':',data_nodes[1][\"data_nodes\"][0][\"transport\"])" | tr -d "[:blank:]") + extract_values() { + local cluster_name="$1" + local field="$2" + echo "$ENDPOINT_LIST" | awk -v cluster="$cluster_name" -v field="$field" ' + BEGIN { RS=","; FS=":" } + $1 ~ "\"cluster_name\"" && $2 ~ "\"" cluster "\"" { + while (getline) { + if ($1 ~ "\"" field "\"") { + gsub(/"/, "", $2) + gsub(/ /, "", $2) + print $2 + exit + } + } + } + ' | tr -d '{}' + } + + # Extract values for leader cluster + leader_endpoint=$(extract_values "leader" "endpoint") + leader_port=$(extract_values "leader" "port") + leader_transport=$(extract_values "leader" "transport") + + # Extract values for follower cluster + follower_endpoint=$(extract_values "follower" "endpoint") + follower_port=$(extract_values "follower" "port") + follower_transport=$(extract_values "follower" "transport") + + # Print extracted data + echo "Leader Endpoint: $leader_endpoint" + echo "Leader Port: $leader_port" + echo "Leader Transport: $leader_transport" + echo "Follower Endpoint: $follower_endpoint" + echo "Follower Port: $follower_port" + echo "Follower Transport: $follower_transport" - leader=$(echo $data | cut -d ',' -f1 | cut -d ':' -f1,2 ) - follower=$(echo $data | cut -d ',' -f2 | cut -d ':' -f1,2 ) - echo "leader: $leader" - echo "follower: $follower" - # Get number of nodes, assuming both leader and follower have same number of nodes - numNodes=$((${follower##*:} - ${leader##*:})) + numNodes=$((${follower_port} - ${leader_port})) echo "numNodes: $numNodes" - - LTRANSPORT_PORT=$(echo $data | cut -d ',' -f1 | cut -d ':' -f1,3 ) - FTRANSPORT_PORT=$(echo $data | cut -d ',' -f2 | cut -d ':' -f1,3 ) - echo "LTRANSPORT_PORT: $LTRANSPORT_PORT" - echo "FTRANSPORT_PORT: $FTRANSPORT_PORT" - - eval "./gradlew integTestRemote -Dleader.http_host=\"$leader\" -Dfollower.http_host=\"$follower\" -Dfollower.transport_host=\"$FTRANSPORT_PORT\" -Dleader.transport_host=\"$LTRANSPORT_PORT\" -Dsecurity_enabled=\"$SECURITY_ENABLED\" -Duser=\"$USERNAME\" -Dpassword=\"$PASSWORD\" -PnumNodes=$numNodes --console=plain " + echo './gradlew integTestRemote -Dleader.http_host='"$leader_endpoint:$leader_port"' -Dfollower.http_host='"$follower_endpoint:$follower_port"' -Dfollower.transport_host='"$follower_endpoint:$follower_transport"' -Dleader.transport_host='"$leader_endpoint:$leader_transport"' -Dsecurity_enabled='"$SECURITY_ENABLED"' -Duser='"$USERNAME"' -Dpassword='"$PASSWORD"' -PnumNodes='"$numNodes"' --console=plain' + ./gradlew integTestRemote -Dleader.http_host=$leader_endpoint:$leader_port -Dfollower.http_host=$follower_endpoint:$follower_port -Dfollower.transport_host=$follower_endpoint:$follower_transport -Dleader.transport_host=$leader_endpoint:$leader_transport -Dsecurity_enabled=$SECURITY_ENABLED -Duser=$USERNAME -Dpassword=$PASSWORD -PnumNodes=$numNodes --console=plain else # Single cluster From 754ffd629f1d95f79beef164986ac011a1b5d965 Mon Sep 17 00:00:00 2001 From: dblock Date: Mon, 8 Jul 2024 19:28:15 -0400 Subject: [PATCH 148/157] Update PULL_REQUEST_TEMPLATE to include an API spec change in the checklist. Signed-off-by: dblock --- .github/pull_request_template.md | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/.github/pull_request_template.md b/.github/pull_request_template.md index d7981b51..7507506a 100644 --- a/.github/pull_request_template.md +++ b/.github/pull_request_template.md @@ -1,15 +1,16 @@ ### Description [Describe what this change achieves] - -### Issues Resolved -[List any issues this PR will resolve] - + +### Related Issues +Resolves #[Issue number to be closed when this PR is merged] + + ### Check List - [ ] New functionality includes testing. - - [ ] All tests pass - [ ] New functionality has been documented. - - [ ] New functionality has javadoc added -- [ ] Commits are signed per the DCO using --signoff +- [ ] API changes companion pull request [created](https://github.com/opensearch-project/opensearch-api-specification/blob/main/DEVELOPER_GUIDE.md). +- [ ] Commits are signed per the DCO using `--signoff`. +- [ ] Public documentation issue/PR [created](https://github.com/opensearch-project/documentation-website/issues/new/choose). By submitting this pull request, I confirm that my contribution is made under the terms of the Apache 2.0 license. -For more information on following Developer Certificate of Origin and signing off your commits, please check [here](https://github.com/opensearch-project/OpenSearch/blob/main/CONTRIBUTING.md#developer-certificate-of-origin). +For more information on following Developer Certificate of Origin and signing off your commits, please check [here](https://github.com/opensearch-project/cross-cluster-replication/blob/main/CONTRIBUTING.md#developer-certificate-of-origin). From 94cd53e07574d4a4e98092f3e04500a3a278953b Mon Sep 17 00:00:00 2001 From: Sanjay Kumar Date: Mon, 24 Jun 2024 23:09:18 +0530 Subject: [PATCH 149/157] Handle exception for the updation of a replication rule Signed-off-by: Sanjay Kumar --- ...sportAutoFollowClusterManagerNodeAction.kt | 1 + .../integ/rest/UpdateAutoFollowPatternIT.kt | 76 +++++++++++++------ 2 files changed, 52 insertions(+), 25 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/action/autofollow/TransportAutoFollowClusterManagerNodeAction.kt b/src/main/kotlin/org/opensearch/replication/action/autofollow/TransportAutoFollowClusterManagerNodeAction.kt index 51a0bd06..b0f85838 100644 --- a/src/main/kotlin/org/opensearch/replication/action/autofollow/TransportAutoFollowClusterManagerNodeAction.kt +++ b/src/main/kotlin/org/opensearch/replication/action/autofollow/TransportAutoFollowClusterManagerNodeAction.kt @@ -111,6 +111,7 @@ class TransportAutoFollowClusterManagerNodeAction @Inject constructor(transportS } catch(e: ResourceAlreadyExistsException) { // Log and bail as task is already running log.warn("Task already started for '$clusterAlias:$patternName'", e) + throw OpenSearchException("Exisiting autofollow replication rule cannot be recreated/updated", e) } catch (e: Exception) { log.error("Failed to start auto follow task for cluster '$clusterAlias:$patternName'", e) throw OpenSearchException(AUTOFOLLOW_EXCEPTION_GENERIC_STRING) diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt index 78e4ead4..7b8efee0 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt @@ -115,7 +115,7 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { try { // Set poll duration to 30sec from 60sec (default) val settings = Settings.builder().put(ReplicationPlugin.REPLICATION_AUTOFOLLOW_REMOTE_INDICES_POLL_INTERVAL.key, - TimeValue.timeValueSeconds(30)) + TimeValue.timeValueSeconds(30)) val clusterUpdateSetttingsReq = ClusterUpdateSettingsRequest().persistentSettings(settings) val clusterUpdateResponse = followerClient.cluster().putSettings(clusterUpdateSetttingsReq, RequestOptions.DEFAULT) var lastExecutionTime = 0L @@ -138,12 +138,12 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { } }, 30, TimeUnit.SECONDS) assertBusy({ - var af_stats = stats.get("autofollow_stats")!! as ArrayList> - for (key in af_stats) { - if(key["name"] == indexPatternName) { - Assertions.assertThat(key["last_execution_time"]!! as Long).isNotEqualTo(lastExecutionTime) + var af_stats = stats.get("autofollow_stats")!! as ArrayList> + for (key in af_stats) { + if(key["name"] == indexPatternName) { + Assertions.assertThat(key["last_execution_time"]!! as Long).isNotEqualTo(lastExecutionTime) + } } - } }, 40, TimeUnit.SECONDS) } finally { followerClient.deleteAutoFollowPattern(connectionAlias, indexPatternName) @@ -184,7 +184,7 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { followerClient.deleteAutoFollowPattern(connectionAlias, indexPatternName) } } - + fun `test auto follow stats`() { val indexPatternName2 = "test_pattern2" val indexPattern2 = "lead_index*" @@ -263,35 +263,35 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { val followerClient = getClientForCluster(FOLLOWER) createConnectionBetweenClusters(FOLLOWER, LEADER, connectionAlias) assertPatternNameValidation(followerClient, "testPattern", - "Value testPattern must be lowercase") + "Value testPattern must be lowercase") assertPatternNameValidation(followerClient, "testPattern*", - "Value testPattern* must not contain the following characters") + "Value testPattern* must not contain the following characters") assertPatternNameValidation(followerClient, "test#", - "Value test# must not contain '#' or ':'") + "Value test# must not contain '#' or ':'") assertPatternNameValidation(followerClient, "test:", - "Value test: must not contain '#' or ':'") + "Value test: must not contain '#' or ':'") assertPatternNameValidation(followerClient, ".", - "Value . must not be '.' or '..'") + "Value . must not be '.' or '..'") assertPatternNameValidation(followerClient, "..", - "Value .. must not be '.' or '..'") + "Value .. must not be '.' or '..'") assertPatternNameValidation(followerClient, "_leader", - "Value _leader must not start with '_' or '-' or '+'") + "Value _leader must not start with '_' or '-' or '+'") assertPatternNameValidation(followerClient, "-leader", - "Value -leader must not start with '_' or '-' or '+'") + "Value -leader must not start with '_' or '-' or '+'") assertPatternNameValidation(followerClient, "+leader", - "Value +leader must not start with '_' or '-' or '+'") + "Value +leader must not start with '_' or '-' or '+'") assertPatternNameValidation(followerClient, longIndexPatternName, - "Value $longIndexPatternName must not be longer than ${MetadataCreateIndexService.MAX_INDEX_NAME_BYTES} bytes") + "Value $longIndexPatternName must not be longer than ${MetadataCreateIndexService.MAX_INDEX_NAME_BYTES} bytes") assertPatternNameValidation(followerClient, ".leaderIndex", - "Value .leaderIndex must not start with '.'") + "Value .leaderIndex must not start with '.'") } private fun assertPatternNameValidation(followerClient: RestHighLevelClient, patternName: String, - errorMsg: String) { + errorMsg: String) { Assertions.assertThatThrownBy { followerClient.updateAutoFollowPattern(connectionAlias, patternName, indexPattern) }.isInstanceOf(ResponseException::class.java) - .hasMessageContaining(errorMsg) + .hasMessageContaining(errorMsg) } fun `test deletion of auto follow pattern`() { @@ -308,6 +308,32 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { followerClient.deleteAutoFollowPattern(connectionAlias, indexPatternName) }.doesNotThrowAnyException() + } + fun `test updation of auto follow pattern`() { + val followerClient = getClientForCluster(FOLLOWER) + createConnectionBetweenClusters(FOLLOWER, LEADER, connectionAlias) + followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, indexPattern) + val indexPattern1 = "log*" + //Re-create the same replication rule + Assertions.assertThatThrownBy { + followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, indexPattern) + }.isInstanceOf(ResponseException::class.java) + .hasMessageContaining("autofollow replication rule cannot be recreated/updated") + + //Update the replication rule with different indexpattern + Assertions.assertThatThrownBy { + followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, indexPattern1) + }.isInstanceOf(ResponseException::class.java) + .hasMessageContaining("autofollow replication rule cannot be recreated/updated") + + //Create a new replication rule with same indexpattern but unique rule name + Assertions.assertThatCode { + followerClient.updateAutoFollowPattern(connectionAlias, "unique-rule", indexPattern1) + }.doesNotThrowAnyException() + + followerClient.deleteAutoFollowPattern(connectionAlias, indexPatternName) + followerClient.deleteAutoFollowPattern(connectionAlias, "unique-rule") + } fun `test removing autofollow pattern stop autofollow task`() { @@ -344,12 +370,12 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { try { //modify retry duration to account for autofollow trigger in next retry followerClient.updateAutofollowRetrySetting("1m") + followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, indexPattern) for (repeat in 1..2) { log.info("Current Iteration $repeat") // Add replication start block followerClient.updateReplicationStartBlockSetting(true) createRandomIndex(leaderClient) - followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, indexPattern) sleep(95000) // wait for auto follow trigger in the worst case // verify both index replication tasks and autofollow tasks // Replication shouldn't have been started - (repeat-1) tasks as for current loop index shouldn't be @@ -392,13 +418,13 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { // Verify that existing index matching the pattern are replicated. assertBusy { Assertions.assertThat(followerClient.indices() - .exists(GetIndexRequest(leaderIndexName1), RequestOptions.DEFAULT)) - .isEqualTo(true) + .exists(GetIndexRequest(leaderIndexName1), RequestOptions.DEFAULT)) + .isEqualTo(true) } assertBusy { Assertions.assertThat(followerClient.indices() - .exists(GetIndexRequest(leaderIndexName2), RequestOptions.DEFAULT)) - .isEqualTo(true) + .exists(GetIndexRequest(leaderIndexName2), RequestOptions.DEFAULT)) + .isEqualTo(true) } sleep(30000) // Default poll for auto follow in worst case Assertions.assertThat(getAutoFollowTasks(FOLLOWER).size).isEqualTo(1) From 63fcd31ece594d68eaa49a2b9f847e9219cfad5d Mon Sep 17 00:00:00 2001 From: Sanjay Kumar Date: Tue, 25 Jun 2024 00:31:30 +0530 Subject: [PATCH 150/157] indendation fix Signed-off-by: Sanjay Kumar --- .../integ/rest/UpdateAutoFollowPatternIT.kt | 46 +++++++++---------- 1 file changed, 23 insertions(+), 23 deletions(-) diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt index 7b8efee0..eba1e407 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt @@ -115,7 +115,7 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { try { // Set poll duration to 30sec from 60sec (default) val settings = Settings.builder().put(ReplicationPlugin.REPLICATION_AUTOFOLLOW_REMOTE_INDICES_POLL_INTERVAL.key, - TimeValue.timeValueSeconds(30)) + TimeValue.timeValueSeconds(30)) val clusterUpdateSetttingsReq = ClusterUpdateSettingsRequest().persistentSettings(settings) val clusterUpdateResponse = followerClient.cluster().putSettings(clusterUpdateSetttingsReq, RequestOptions.DEFAULT) var lastExecutionTime = 0L @@ -138,12 +138,12 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { } }, 30, TimeUnit.SECONDS) assertBusy({ - var af_stats = stats.get("autofollow_stats")!! as ArrayList> - for (key in af_stats) { - if(key["name"] == indexPatternName) { - Assertions.assertThat(key["last_execution_time"]!! as Long).isNotEqualTo(lastExecutionTime) - } + var af_stats = stats.get("autofollow_stats")!! as ArrayList> + for (key in af_stats) { + if(key["name"] == indexPatternName) { + Assertions.assertThat(key["last_execution_time"]!! as Long).isNotEqualTo(lastExecutionTime) } + } }, 40, TimeUnit.SECONDS) } finally { followerClient.deleteAutoFollowPattern(connectionAlias, indexPatternName) @@ -263,35 +263,35 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { val followerClient = getClientForCluster(FOLLOWER) createConnectionBetweenClusters(FOLLOWER, LEADER, connectionAlias) assertPatternNameValidation(followerClient, "testPattern", - "Value testPattern must be lowercase") + "Value testPattern must be lowercase") assertPatternNameValidation(followerClient, "testPattern*", - "Value testPattern* must not contain the following characters") + "Value testPattern* must not contain the following characters") assertPatternNameValidation(followerClient, "test#", - "Value test# must not contain '#' or ':'") + "Value test# must not contain '#' or ':'") assertPatternNameValidation(followerClient, "test:", - "Value test: must not contain '#' or ':'") + "Value test: must not contain '#' or ':'") assertPatternNameValidation(followerClient, ".", - "Value . must not be '.' or '..'") + "Value . must not be '.' or '..'") assertPatternNameValidation(followerClient, "..", - "Value .. must not be '.' or '..'") + "Value .. must not be '.' or '..'") assertPatternNameValidation(followerClient, "_leader", - "Value _leader must not start with '_' or '-' or '+'") + "Value _leader must not start with '_' or '-' or '+'") assertPatternNameValidation(followerClient, "-leader", - "Value -leader must not start with '_' or '-' or '+'") + "Value -leader must not start with '_' or '-' or '+'") assertPatternNameValidation(followerClient, "+leader", - "Value +leader must not start with '_' or '-' or '+'") + "Value +leader must not start with '_' or '-' or '+'") assertPatternNameValidation(followerClient, longIndexPatternName, - "Value $longIndexPatternName must not be longer than ${MetadataCreateIndexService.MAX_INDEX_NAME_BYTES} bytes") + "Value $longIndexPatternName must not be longer than ${MetadataCreateIndexService.MAX_INDEX_NAME_BYTES} bytes") assertPatternNameValidation(followerClient, ".leaderIndex", - "Value .leaderIndex must not start with '.'") + "Value .leaderIndex must not start with '.'") } private fun assertPatternNameValidation(followerClient: RestHighLevelClient, patternName: String, - errorMsg: String) { + errorMsg: String) { Assertions.assertThatThrownBy { followerClient.updateAutoFollowPattern(connectionAlias, patternName, indexPattern) }.isInstanceOf(ResponseException::class.java) - .hasMessageContaining(errorMsg) + .hasMessageContaining(errorMsg) } fun `test deletion of auto follow pattern`() { @@ -418,13 +418,13 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { // Verify that existing index matching the pattern are replicated. assertBusy { Assertions.assertThat(followerClient.indices() - .exists(GetIndexRequest(leaderIndexName1), RequestOptions.DEFAULT)) - .isEqualTo(true) + .exists(GetIndexRequest(leaderIndexName1), RequestOptions.DEFAULT)) + .isEqualTo(true) } assertBusy { Assertions.assertThat(followerClient.indices() - .exists(GetIndexRequest(leaderIndexName2), RequestOptions.DEFAULT)) - .isEqualTo(true) + .exists(GetIndexRequest(leaderIndexName2), RequestOptions.DEFAULT)) + .isEqualTo(true) } sleep(30000) // Default poll for auto follow in worst case Assertions.assertThat(getAutoFollowTasks(FOLLOWER).size).isEqualTo(1) From 2b5732702fd7592018f28f11dfab4f750e754b1e Mon Sep 17 00:00:00 2001 From: Sanjay Kumar Date: Tue, 25 Jun 2024 12:54:53 +0530 Subject: [PATCH 151/157] Exclusion of 'test operations are fetched from lucene when leader is in mixed mode' in integTest Signed-off-by: Sanjay Kumar --- build.gradle | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index e862827b..a3817fd1 100644 --- a/build.gradle +++ b/build.gradle @@ -527,7 +527,9 @@ integTest { } } } - + filter { + setExcludePatterns("org.opensearch.replication.integ.rest.StartReplicationIT.test operations are fetched from lucene when leader is in mixed mode") + } systemProperty "build.dir", "${buildDir}" systemProperty "java.security.policy", "file://${projectDir}/src/test/resources/plugin-security.policy" finalizedBy jacocoTestReport From ee4e03fe89fa98f1902444e62f7d27cfcd96b942 Mon Sep 17 00:00:00 2001 From: Sanjay Kumar Date: Tue, 25 Jun 2024 16:03:09 +0530 Subject: [PATCH 152/157] Exclusion of tests in integTest Signed-off-by: Sanjay Kumar --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index a3817fd1..cb586c9a 100644 --- a/build.gradle +++ b/build.gradle @@ -528,7 +528,7 @@ integTest { } } filter { - setExcludePatterns("org.opensearch.replication.integ.rest.StartReplicationIT.test operations are fetched from lucene when leader is in mixed mode") + setExcludePatterns("org.opensearch.replication.bwc.BackwardsCompatibilityIT","org.opensearch.replication.singleCluster.SingleClusterSanityIT","org.opensearch.replication.integ.rest.StartReplicationIT.test operations are fetched from lucene when leader is in mixed mode") } systemProperty "build.dir", "${buildDir}" systemProperty "java.security.policy", "file://${projectDir}/src/test/resources/plugin-security.policy" From 2da20341107bfe707487d93d271df70aed31356d Mon Sep 17 00:00:00 2001 From: Sanjay Kumar Date: Thu, 18 Jul 2024 21:02:19 +0530 Subject: [PATCH 153/157] revert changes for exclusion of tests Signed-off-by: Sanjay Kumar --- build.gradle | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/build.gradle b/build.gradle index cb586c9a..e862827b 100644 --- a/build.gradle +++ b/build.gradle @@ -527,9 +527,7 @@ integTest { } } } - filter { - setExcludePatterns("org.opensearch.replication.bwc.BackwardsCompatibilityIT","org.opensearch.replication.singleCluster.SingleClusterSanityIT","org.opensearch.replication.integ.rest.StartReplicationIT.test operations are fetched from lucene when leader is in mixed mode") - } + systemProperty "build.dir", "${buildDir}" systemProperty "java.security.policy", "file://${projectDir}/src/test/resources/plugin-security.policy" finalizedBy jacocoTestReport From 434d31ad317dd87d0b16e975f88e8feb283b23bc Mon Sep 17 00:00:00 2001 From: Sanjay Kumar Date: Fri, 22 Mar 2024 15:00:01 +0530 Subject: [PATCH 154/157] Added validations for indexPattern in autofollow API Signed-off-by: Sanjay Kumar --- .../UpdateAutoFollowPatternRequest.kt | 8 ++++++-- .../replication/util/ValidationUtil.kt | 19 +++++++++++++++++++ 2 files changed, 25 insertions(+), 2 deletions(-) diff --git a/src/main/kotlin/org/opensearch/replication/action/autofollow/UpdateAutoFollowPatternRequest.kt b/src/main/kotlin/org/opensearch/replication/action/autofollow/UpdateAutoFollowPatternRequest.kt index 165ede8b..d145eda2 100644 --- a/src/main/kotlin/org/opensearch/replication/action/autofollow/UpdateAutoFollowPatternRequest.kt +++ b/src/main/kotlin/org/opensearch/replication/action/autofollow/UpdateAutoFollowPatternRequest.kt @@ -14,6 +14,7 @@ package org.opensearch.replication.action.autofollow import org.opensearch.replication.action.index.ReplicateIndexRequest import org.opensearch.replication.metadata.store.KEY_SETTINGS import org.opensearch.replication.util.ValidationUtil.validateName +import org.opensearch.replication.util.ValidationUtil.validatePattern import org.opensearch.action.ActionRequestValidationException import org.opensearch.action.support.master.AcknowledgedRequest import org.opensearch.core.ParseField @@ -113,8 +114,11 @@ class UpdateAutoFollowPatternRequest: AcknowledgedRequest Date: Tue, 23 Apr 2024 15:37:46 +0530 Subject: [PATCH 155/157] rebased commit Signed-off-by: Sanjay Kumar --- .../integ/rest/UpdateAutoFollowPatternIT.kt | 44 +++++++++++++++++++ 1 file changed, 44 insertions(+) diff --git a/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt b/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt index eba1e407..6782e56c 100644 --- a/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt +++ b/src/test/kotlin/org/opensearch/replication/integ/rest/UpdateAutoFollowPatternIT.kt @@ -482,6 +482,50 @@ class UpdateAutoFollowPatternIT: MultiClusterRestTestCase() { fun getIndexReplicationTasks(clusterName: String): List { return getReplicationTaskList(clusterName, IndexReplicationExecutor.TASK_NAME + "*") } + fun `test auto follow should fail on indexPattern validation failure`() { + val followerClient = getClientForCluster(FOLLOWER) + createConnectionBetweenClusters(FOLLOWER, LEADER, connectionAlias) + assertPatternValidation(followerClient, "testPattern,", + "Autofollow pattern: testPattern, must not contain the following characters") + assertPatternValidation(followerClient, "testPat?", + "Autofollow pattern: testPat? must not contain the following characters") + assertPatternValidation(followerClient, "test#", + "Autofollow pattern: test# must not contain '#' or ':'") + assertPatternValidation(followerClient, "test:", + "Autofollow pattern: test: must not contain '#' or ':'") + assertPatternValidation(followerClient, "_test", + "Autofollow pattern: _test must not start with '_' or '-'") + assertPatternValidation(followerClient, "-leader", + "Autofollow pattern: -leader must not start with '_' or '-'") + assertPatternValidation(followerClient, "", + "Autofollow pattern: must not be empty") + + } + private fun assertPatternValidation(followerClient: RestHighLevelClient, pattern: String, + errorMsg: String) { + Assertions.assertThatThrownBy { + followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, pattern) + }.isInstanceOf(ResponseException::class.java) + .hasMessageContaining(errorMsg) + } + + fun `test auto follow should succeed on valid indexPatterns`() { + val followerClient = getClientForCluster(FOLLOWER) + createConnectionBetweenClusters(FOLLOWER, LEADER, connectionAlias) + assertValidPatternValidation(followerClient, "test-leader") + assertValidPatternValidation(followerClient, "test*") + assertValidPatternValidation(followerClient, "leader-*") + assertValidPatternValidation(followerClient, "leader_test") + assertValidPatternValidation(followerClient, "Leader_Test-*") + assertValidPatternValidation(followerClient, "Leader_*") + + } + + private fun assertValidPatternValidation(followerClient: RestHighLevelClient, pattern: String) { + Assertions.assertThatCode { + followerClient.updateAutoFollowPattern(connectionAlias, indexPatternName, pattern) + }.doesNotThrowAnyException() + } fun createDummyConnection(fromClusterName: String, connectionName: String="source") { val fromCluster = getNamedCluster(fromClusterName) From 6022720794ec37f7770d36d7a9909b61ecc056c6 Mon Sep 17 00:00:00 2001 From: Sanjay Kumar Date: Wed, 17 Jul 2024 10:48:13 +0530 Subject: [PATCH 156/157] empty commit Signed-off-by: Sanjay Kumar --- .../kotlin/org/opensearch/replication/util/ValidationUtil.kt | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt b/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt index 5a08374a..b057e414 100644 --- a/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt +++ b/src/main/kotlin/org/opensearch/replication/util/ValidationUtil.kt @@ -122,6 +122,7 @@ object ValidationUtil { if ((pattern?.startsWith('_') ?: false) || (pattern?.startsWith('-') ?: false)) validationException.addValidationError("Autofollow pattern: $pattern must not start with '_' or '-'") + } /** From 802ae780a79831cbaddd73fcc024e73aec43fa30 Mon Sep 17 00:00:00 2001 From: nandan Date: Thu, 5 Sep 2024 05:44:35 +0000 Subject: [PATCH 157/157] Resolvinfg Merge conflicts Signed-off-by: nandan --- .../repository/RemoteClusterRepository.kt | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt index bb03f275..6ebd2994 100644 --- a/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt +++ b/src/main/kotlin/org/opensearch/replication/repository/RemoteClusterRepository.kt @@ -74,6 +74,10 @@ import java.util.UUID import java.util.function.Consumer import java.util.function.Function import kotlin.collections.ArrayList +<<<<<<< HEAD +======= +import org.opensearch.common.Priority +>>>>>>> 735b1f1 ( Fixing Identation) const val REMOTE_REPOSITORY_PREFIX = "replication-remote-repo-" const val REMOTE_REPOSITORY_TYPE = "replication-remote-repository" @@ -122,6 +126,16 @@ class RemoteClusterRepository(private val repositoryMetadata: RepositoryMetadata throw UnsupportedOperationException("Operation not permitted") } +<<<<<<< HEAD +======= + override fun finalizeSnapshot(shardGenerations: ShardGenerations?, repositoryStateId: Long, clusterMetadata: Metadata?, + snapshotInfo: SnapshotInfo?, repositoryMetaVersion: Version?, + stateTransformer: Function?, repositoryUpdatePriority: Priority, + listener: ActionListener?) { + throw UnsupportedOperationException("Operation not permitted") + } + +>>>>>>> 735b1f1 ( Fixing Identation) override fun deleteSnapshots(snapshotIds: MutableCollection?, repositoryStateId: Long, repositoryMetaVersion: Version?, listener: ActionListener?) { throw UnsupportedOperationException("Operation not permitted")