From a1d8bebab16fb099c85cb199af91fb4980dcafb8 Mon Sep 17 00:00:00 2001 From: Varun Bansal Date: Mon, 28 Aug 2023 14:01:45 +0530 Subject: [PATCH 01/37] Set replication type to SEGMENT in remote store enabled cluster if nothing specified by user (#9264) --------- Signed-off-by: bansvaru Signed-off-by: Varun Bansal --- CHANGELOG.md | 1 + .../RemoteStoreBaseIntegTestCase.java | 12 ++++--- .../metadata/MetadataCreateIndexService.java | 15 +++++---- .../MetadataCreateIndexServiceTests.java | 32 +++++++++++++++++++ 4 files changed, 49 insertions(+), 11 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 96af8741a0de9..441de802f6ea8 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -158,6 +158,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Removing the vec file extension from INDEX_STORE_HYBRID_NIO_EXTENSIONS, to ensure the no performance degradation for vector search via Lucene Engine.([#9528](https://github.com/opensearch-project/OpenSearch/pull/9528))) - Separate request-based and settings-based concurrent segment search controls and introduce AggregatorFactory method to determine concurrent search support ([#9469](https://github.com/opensearch-project/OpenSearch/pull/9469)) - [Remote Store] Rate limiter integration for remote store uploads and downloads([#9448](https://github.com/opensearch-project/OpenSearch/pull/9448/)) +- [Remote Store] Implicitly use replication type SEGMENT for remote store clusters ([#9264](https://github.com/opensearch-project/OpenSearch/pull/9264)) ### Deprecated diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java index 90efafe9423c6..15d4b8d1de58a 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java @@ -169,12 +169,16 @@ public static Settings remoteStoreClusterSettings( } public static Settings remoteStoreClusterSettings(String segmentRepoName, String translogRepoName) { - return Settings.builder() - .put(CLUSTER_REPLICATION_TYPE_SETTING.getKey(), ReplicationType.SEGMENT) + Settings.Builder settingsBuilder = Settings.builder() .put(CLUSTER_REMOTE_STORE_ENABLED_SETTING.getKey(), true) .put(CLUSTER_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING.getKey(), segmentRepoName) - .put(CLUSTER_REMOTE_TRANSLOG_REPOSITORY_SETTING.getKey(), translogRepoName) - .build(); + .put(CLUSTER_REMOTE_TRANSLOG_REPOSITORY_SETTING.getKey(), translogRepoName); + + if (randomBoolean()) { + settingsBuilder.put(CLUSTER_REPLICATION_TYPE_SETTING.getKey(), ReplicationType.SEGMENT); + } + + return settingsBuilder.build(); } private Settings defaultIndexSettings() { diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java index ec63f762bea9f..fd7fe29442eb2 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java @@ -93,6 +93,7 @@ import org.opensearch.indices.InvalidIndexNameException; import org.opensearch.indices.ShardLimitValidator; import org.opensearch.indices.SystemIndices; +import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.threadpool.ThreadPool; import java.io.IOException; @@ -932,15 +933,15 @@ static Settings aggregateIndexSettings( * @param clusterSettings cluster level settings */ private static void updateReplicationStrategy(Settings.Builder settingsBuilder, Settings requestSettings, Settings clusterSettings) { - if (CLUSTER_REPLICATION_TYPE_SETTING.exists(clusterSettings) && INDEX_REPLICATION_TYPE_SETTING.exists(requestSettings) == false) { - settingsBuilder.put(SETTING_REPLICATION_TYPE, CLUSTER_REPLICATION_TYPE_SETTING.get(clusterSettings)); - return; - } - if (INDEX_REPLICATION_TYPE_SETTING.exists(requestSettings) == true) { + if (INDEX_REPLICATION_TYPE_SETTING.exists(requestSettings)) { settingsBuilder.put(SETTING_REPLICATION_TYPE, INDEX_REPLICATION_TYPE_SETTING.get(requestSettings)); - return; + } else if (CLUSTER_REPLICATION_TYPE_SETTING.exists(clusterSettings)) { + settingsBuilder.put(SETTING_REPLICATION_TYPE, CLUSTER_REPLICATION_TYPE_SETTING.get(clusterSettings)); + } else if (CLUSTER_REMOTE_STORE_ENABLED_SETTING.get(clusterSettings)) { + settingsBuilder.put(SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT); + } else { + settingsBuilder.put(SETTING_REPLICATION_TYPE, CLUSTER_REPLICATION_TYPE_SETTING.getDefault(clusterSettings)); } - settingsBuilder.put(SETTING_REPLICATION_TYPE, CLUSTER_REPLICATION_TYPE_SETTING.getDefault(clusterSettings)); } /** diff --git a/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java b/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java index a9f3e97e64e39..ad5fbb18138b3 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java @@ -1245,6 +1245,38 @@ public void testRemoteStoreNoUserOverrideExceptReplicationTypeSegmentIndexSettin ); } + public void testRemoteStoreImplicitOverrideReplicationTypeToSegmentForRemoteStore() { + Settings settings = Settings.builder() + .put(CLUSTER_REMOTE_STORE_ENABLED_SETTING.getKey(), true) + .put(CLUSTER_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING.getKey(), "my-segment-repo-1") + .put(CLUSTER_REMOTE_TRANSLOG_REPOSITORY_SETTING.getKey(), "my-translog-repo-1") + .build(); + FeatureFlagSetter.set(FeatureFlags.REMOTE_STORE); + + request = new CreateIndexClusterStateUpdateRequest("create index", "test", "test"); + final Settings.Builder requestSettings = Settings.builder(); + request.settings(requestSettings.build()); + Settings indexSettings = aggregateIndexSettings( + ClusterState.EMPTY_STATE, + request, + Settings.EMPTY, + null, + settings, + IndexScopedSettings.DEFAULT_SCOPED_SETTINGS, + randomShardLimitService(), + Collections.emptySet(), + clusterSettings + ); + verifyRemoteStoreIndexSettings( + indexSettings, + "true", + "my-segment-repo-1", + "my-translog-repo-1", + ReplicationType.SEGMENT.toString(), + IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL + ); + } + public void testRemoteStoreNoUserOverrideIndexSettings() { Settings settings = Settings.builder() .put(CLUSTER_REPLICATION_TYPE_SETTING.getKey(), ReplicationType.SEGMENT) From 569d5c242dd0b04e814714cc9369ca94124d5260 Mon Sep 17 00:00:00 2001 From: Sachin Kale Date: Mon, 28 Aug 2023 13:08:41 +0000 Subject: [PATCH 02/37] Bugfix: add replica information in remote store restore flow (#8951) Signed-off-by: Sachin Kale --- .../remotestore/RemoteStoreRestoreIT.java | 65 +++++----- .../cluster/routing/IndexRoutingTable.java | 32 ++++- .../cluster/routing/RoutingTable.java | 5 +- .../recovery/RemoteStoreRestoreService.java | 26 ++-- .../opensearch/index/shard/IndexShard.java | 13 +- .../shard/RemoteStoreRefreshListener.java | 8 ++ .../opensearch/index/shard/StoreRecovery.java | 6 +- .../recovery/PeerRecoveryTargetService.java | 2 +- .../cluster/routing/RoutingTableTests.java | 119 +++++++++++++++--- .../RemoteStoreRefreshListenerTests.java | 3 +- 10 files changed, 201 insertions(+), 78 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRestoreIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRestoreIT.java index 60d7eefbb6d9b..85c2514ebf00f 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRestoreIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreRestoreIT.java @@ -41,7 +41,6 @@ public class RemoteStoreRestoreIT extends RemoteStoreBaseIntegTestCase { private static final String INDEX_NAMES = "test-remote-store-1,test-remote-store-2,remote-store-test-index-1,remote-store-test-index-2"; private static final String INDEX_NAMES_WILDCARD = "test-remote-store-*,remote-store-test-index-*"; private static final String TOTAL_OPERATIONS = "total-operations"; - private static final String REFRESHED_OR_FLUSHED_OPERATIONS = "refreshed-or-flushed-operations"; private static final String MAX_SEQ_NO_TOTAL = "max-seq-no-total"; @Override @@ -72,18 +71,26 @@ private void restore(String... indices) { ); } - private void verifyRestoredData(Map indexStats, String indexName) { - // This is required to get updated number from already active shards which were not restored - refresh(indexName); + private void verifyRestoredData(Map indexStats, String indexName) throws Exception { ensureYellowAndNoInitializingShards(indexName); ensureGreen(indexName); - assertHitCount(client().prepareSearch(indexName).setSize(0).get(), indexStats.get(TOTAL_OPERATIONS)); + // This is to ensure that shards that were already assigned will get latest count + refresh(indexName); + assertBusy( + () -> assertHitCount(client().prepareSearch(indexName).setSize(0).get(), indexStats.get(TOTAL_OPERATIONS)), + 30, + TimeUnit.SECONDS + ); IndexResponse response = indexSingleDoc(indexName); if (indexStats.containsKey(MAX_SEQ_NO_TOTAL + "-shard-" + response.getShardId().id())) { assertEquals(indexStats.get(MAX_SEQ_NO_TOTAL + "-shard-" + response.getShardId().id()) + 1, response.getSeqNo()); } refresh(indexName); - assertHitCount(client().prepareSearch(indexName).setSize(0).get(), indexStats.get(TOTAL_OPERATIONS) + 1); + assertBusy( + () -> assertHitCount(client().prepareSearch(indexName).setSize(0).get(), indexStats.get(TOTAL_OPERATIONS) + 1), + 30, + TimeUnit.SECONDS + ); } private void prepareCluster(int numClusterManagerNodes, int numDataOnlyNodes, String indices, int replicaCount, int shardCount) { @@ -100,7 +107,7 @@ private void prepareCluster(int numClusterManagerNodes, int numDataOnlyNodes, St * Simulates all data restored using Remote Translog Store. * @throws IOException IO Exception. */ - public void testRemoteTranslogRestoreWithNoDataPostCommit() throws IOException { + public void testRemoteTranslogRestoreWithNoDataPostCommit() throws Exception { testRestoreFlow(1, true, randomIntBetween(1, 5)); } @@ -108,7 +115,7 @@ public void testRemoteTranslogRestoreWithNoDataPostCommit() throws IOException { * Simulates all data restored using Remote Translog Store. * @throws IOException IO Exception. */ - public void testRemoteTranslogRestoreWithNoDataPostRefresh() throws IOException { + public void testRemoteTranslogRestoreWithNoDataPostRefresh() throws Exception { testRestoreFlow(1, false, randomIntBetween(1, 5)); } @@ -117,7 +124,7 @@ public void testRemoteTranslogRestoreWithNoDataPostRefresh() throws IOException * and unrefreshed data restored using Remote Translog Store. * @throws IOException IO Exception. */ - public void testRemoteTranslogRestoreWithRefreshedData() throws IOException { + public void testRemoteTranslogRestoreWithRefreshedData() throws Exception { testRestoreFlow(randomIntBetween(2, 5), false, randomIntBetween(1, 5)); } @@ -126,7 +133,7 @@ public void testRemoteTranslogRestoreWithRefreshedData() throws IOException { * and unrefreshed data restored using Remote Translog Store. * @throws IOException IO Exception. */ - public void testRemoteTranslogRestoreWithCommittedData() throws IOException { + public void testRemoteTranslogRestoreWithCommittedData() throws Exception { testRestoreFlow(randomIntBetween(2, 5), true, randomIntBetween(1, 5)); } @@ -134,8 +141,7 @@ public void testRemoteTranslogRestoreWithCommittedData() throws IOException { * Simulates all data restored using Remote Translog Store. * @throws IOException IO Exception. */ - @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/8479") - public void testRTSRestoreWithNoDataPostCommitPrimaryReplicaDown() throws IOException { + public void testRTSRestoreWithNoDataPostCommitPrimaryReplicaDown() throws Exception { testRestoreFlowBothPrimaryReplicasDown(1, true, randomIntBetween(1, 5)); } @@ -143,8 +149,7 @@ public void testRTSRestoreWithNoDataPostCommitPrimaryReplicaDown() throws IOExce * Simulates all data restored using Remote Translog Store. * @throws IOException IO Exception. */ - @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/8479") - public void testRTSRestoreWithNoDataPostRefreshPrimaryReplicaDown() throws IOException { + public void testRTSRestoreWithNoDataPostRefreshPrimaryReplicaDown() throws Exception { testRestoreFlowBothPrimaryReplicasDown(1, false, randomIntBetween(1, 5)); } @@ -153,8 +158,7 @@ public void testRTSRestoreWithNoDataPostRefreshPrimaryReplicaDown() throws IOExc * and unrefreshed data restored using Remote Translog Store. * @throws IOException IO Exception. */ - @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/8479") - public void testRTSRestoreWithRefreshedDataPrimaryReplicaDown() throws IOException { + public void testRTSRestoreWithRefreshedDataPrimaryReplicaDown() throws Exception { testRestoreFlowBothPrimaryReplicasDown(randomIntBetween(2, 5), false, randomIntBetween(1, 5)); } @@ -163,12 +167,11 @@ public void testRTSRestoreWithRefreshedDataPrimaryReplicaDown() throws IOExcepti * and unrefreshed data restored using Remote Translog Store. * @throws IOException IO Exception. */ - @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/8479") - public void testRTSRestoreWithCommittedDataPrimaryReplicaDown() throws IOException { + public void testRTSRestoreWithCommittedDataPrimaryReplicaDown() throws Exception { testRestoreFlowBothPrimaryReplicasDown(randomIntBetween(2, 5), true, randomIntBetween(1, 5)); } - private void restoreAndVerify(int shardCount, int replicaCount, Map indexStats) { + private void restoreAndVerify(int shardCount, int replicaCount, Map indexStats) throws Exception { restore(INDEX_NAME); ensureGreen(INDEX_NAME); // This is required to get updated number from already active shards which were not restored @@ -183,7 +186,7 @@ private void restoreAndVerify(int shardCount, int replicaCount, Map indexStats = indexData(numberOfIterations, invokeFlush, INDEX_NAME); assertEquals(shardCount, getNumShards(INDEX_NAME).totalNumShards); @@ -202,10 +205,10 @@ private void testRestoreFlow(int numberOfIterations, boolean invokeFlush, int sh * @param invokeFlush If true, a flush is invoked. Otherwise, a refresh is invoked. * @throws IOException IO Exception. */ - private void testRestoreFlowBothPrimaryReplicasDown(int numberOfIterations, boolean invokeFlush, int shardCount) throws IOException { + private void testRestoreFlowBothPrimaryReplicasDown(int numberOfIterations, boolean invokeFlush, int shardCount) throws Exception { prepareCluster(1, 2, INDEX_NAME, 1, shardCount); Map indexStats = indexData(numberOfIterations, invokeFlush, INDEX_NAME); - assertEquals(shardCount, getNumShards(INDEX_NAME).totalNumShards); + assertEquals(shardCount * 2, getNumShards(INDEX_NAME).totalNumShards); internalCluster().stopRandomNode(InternalTestCluster.nameFilter(replicaNodeName(INDEX_NAME))); internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primaryNodeName(INDEX_NAME))); @@ -221,14 +224,14 @@ private void testRestoreFlowBothPrimaryReplicasDown(int numberOfIterations, bool * @param invokeFlush If true, a flush is invoked. Otherwise, a refresh is invoked. * @throws IOException IO Exception. */ - private void testRestoreFlowMultipleIndices(int numberOfIterations, boolean invokeFlush, int shardCount) throws IOException { + private void testRestoreFlowMultipleIndices(int numberOfIterations, boolean invokeFlush, int shardCount) throws Exception { prepareCluster(1, 3, INDEX_NAMES, 1, shardCount); String[] indices = INDEX_NAMES.split(","); Map> indicesStats = new HashMap<>(); for (String index : indices) { Map indexStats = indexData(numberOfIterations, invokeFlush, index); indicesStats.put(index, indexStats); - assertEquals(shardCount, getNumShards(index).totalNumShards); + assertEquals(shardCount * 2, getNumShards(index).totalNumShards); } for (String index : indices) { @@ -259,7 +262,7 @@ private void testRestoreFlowMultipleIndices(int numberOfIterations, boolean invo ); ensureGreen(indices); for (String index : indices) { - assertEquals(shardCount, getNumShards(index).totalNumShards); + assertEquals(shardCount * 2, getNumShards(index).totalNumShards); verifyRestoredData(indicesStats.get(index), index); } } @@ -280,7 +283,7 @@ public void testRestoreFlowAllShardsNoRedIndex() throws InterruptedException { } } - public void testRestoreFlowNoRedIndex() { + public void testRestoreFlowNoRedIndex() throws Exception { int shardCount = randomIntBetween(1, 5); prepareCluster(0, 3, INDEX_NAME, 0, shardCount); Map indexStats = indexData(randomIntBetween(2, 5), true, INDEX_NAME); @@ -302,7 +305,7 @@ public void testRestoreFlowNoRedIndex() { * @throws IOException IO Exception. */ @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/8480") - public void testRTSRestoreWithCommittedDataMultipleIndicesPatterns() throws IOException { + public void testRTSRestoreWithCommittedDataMultipleIndicesPatterns() throws Exception { testRestoreFlowMultipleIndices(2, true, randomIntBetween(1, 5)); } @@ -313,7 +316,7 @@ public void testRTSRestoreWithCommittedDataMultipleIndicesPatterns() throws IOEx * @throws IOException IO Exception. */ @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/8480") - public void testRTSRestoreWithCommittedDataDefaultAllIndices() throws IOException { + public void testRTSRestoreWithCommittedDataDefaultAllIndices() throws Exception { int shardCount = randomIntBetween(1, 5); prepareCluster(1, 3, INDEX_NAMES, 1, shardCount); String[] indices = INDEX_NAMES.split(","); @@ -354,7 +357,7 @@ public void testRTSRestoreWithCommittedDataDefaultAllIndices() throws IOExceptio * with only some of the remote-enabled red indices requested for the restore. * @throws IOException IO Exception. */ - public void testRTSRestoreWithCommittedDataNotAllRedRemoteIndices() throws IOException { + public void testRTSRestoreWithCommittedDataNotAllRedRemoteIndices() throws Exception { int shardCount = randomIntBetween(1, 5); prepareCluster(1, 3, INDEX_NAMES, 0, shardCount); String[] indices = INDEX_NAMES.split(","); @@ -402,7 +405,7 @@ public void testRTSRestoreWithCommittedDataNotAllRedRemoteIndices() throws IOExc * @throws IOException IO Exception. */ @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/8480") - public void testRTSRestoreWithCommittedDataExcludeIndicesPatterns() throws IOException { + public void testRTSRestoreWithCommittedDataExcludeIndicesPatterns() throws Exception { int shardCount = randomIntBetween(1, 5); prepareCluster(1, 3, INDEX_NAMES, 1, shardCount); String[] indices = INDEX_NAMES.split(","); @@ -451,7 +454,7 @@ public void testRTSRestoreWithCommittedDataExcludeIndicesPatterns() throws IOExc * when the index has no data. * @throws IOException IO Exception. */ - public void testRTSRestoreDataOnlyInTranslog() throws IOException { + public void testRTSRestoreDataOnlyInTranslog() throws Exception { testRestoreFlow(0, true, randomIntBetween(1, 5)); } diff --git a/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java b/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java index e4ffacd708632..fd70d319780c8 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java +++ b/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java @@ -453,7 +453,8 @@ public Builder initializeAsRestore(IndexMetadata indexMetadata, SnapshotRecovery public Builder initializeAsRemoteStoreRestore( IndexMetadata indexMetadata, RemoteStoreRecoverySource recoverySource, - Map activeInitializingShards + Map indexShardRoutingTableMap, + boolean restoreAllShards ) { final UnassignedInfo unassignedInfo = new UnassignedInfo( UnassignedInfo.Reason.EXISTING_INDEX_RESTORED, @@ -465,11 +466,34 @@ public Builder initializeAsRemoteStoreRestore( } for (int shardNumber = 0; shardNumber < indexMetadata.getNumberOfShards(); shardNumber++) { ShardId shardId = new ShardId(index, shardNumber); + if (indexShardRoutingTableMap.containsKey(shardId) == false) { + throw new IllegalStateException("IndexShardRoutingTable is not present for shardId: " + shardId); + } IndexShardRoutingTable.Builder indexShardRoutingBuilder = new IndexShardRoutingTable.Builder(shardId); - if (activeInitializingShards.containsKey(shardId)) { - indexShardRoutingBuilder.addShard(activeInitializingShards.get(shardId)); - } else { + IndexShardRoutingTable indexShardRoutingTable = indexShardRoutingTableMap.get(shardId); + if (restoreAllShards || indexShardRoutingTable.primaryShard().unassigned()) { + // Primary shard to be recovered from remote store. indexShardRoutingBuilder.addShard(ShardRouting.newUnassigned(shardId, true, recoverySource, unassignedInfo)); + // All the replica shards to be recovered from peer recovery. + indexShardRoutingTable.replicaShards() + .forEach( + shardRouting -> indexShardRoutingBuilder.addShard( + ShardRouting.newUnassigned(shardId, false, PeerRecoverySource.INSTANCE, unassignedInfo) + ) + ); + } else { + // Primary is either active or initializing. Do not trigger restore. + indexShardRoutingBuilder.addShard(indexShardRoutingTable.primaryShard()); + // Replica, if unassigned, trigger peer recovery else no action. + for (ShardRouting shardRouting : indexShardRoutingTable.replicaShards()) { + if (shardRouting.unassigned()) { + indexShardRoutingBuilder.addShard( + ShardRouting.newUnassigned(shardId, false, PeerRecoverySource.INSTANCE, unassignedInfo) + ); + } else { + indexShardRoutingBuilder.addShard(shardRouting); + } + } } shards.put(shardNumber, indexShardRoutingBuilder.build()); } diff --git a/server/src/main/java/org/opensearch/cluster/routing/RoutingTable.java b/server/src/main/java/org/opensearch/cluster/routing/RoutingTable.java index 9ca4da0b5a85e..bcfc324b202b9 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/RoutingTable.java +++ b/server/src/main/java/org/opensearch/cluster/routing/RoutingTable.java @@ -575,10 +575,11 @@ public Builder addAsFromOpenToClose(IndexMetadata indexMetadata) { public Builder addAsRemoteStoreRestore( IndexMetadata indexMetadata, RemoteStoreRecoverySource recoverySource, - Map activeInitializingShards + Map indexShardRoutingTableMap, + boolean restoreAllShards ) { IndexRoutingTable.Builder indexRoutingBuilder = new IndexRoutingTable.Builder(indexMetadata.getIndex()) - .initializeAsRemoteStoreRestore(indexMetadata, recoverySource, activeInitializingShards); + .initializeAsRemoteStoreRestore(indexMetadata, recoverySource, indexShardRoutingTableMap, restoreAllShards); add(indexRoutingBuilder); return this; } diff --git a/server/src/main/java/org/opensearch/index/recovery/RemoteStoreRestoreService.java b/server/src/main/java/org/opensearch/index/recovery/RemoteStoreRestoreService.java index 2617898fef491..9fdd2ff9f759d 100644 --- a/server/src/main/java/org/opensearch/index/recovery/RemoteStoreRestoreService.java +++ b/server/src/main/java/org/opensearch/index/recovery/RemoteStoreRestoreService.java @@ -19,7 +19,6 @@ import org.opensearch.cluster.routing.IndexShardRoutingTable; import org.opensearch.cluster.routing.RecoverySource; import org.opensearch.cluster.routing.RoutingTable; -import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.allocation.AllocationService; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.UUIDs; @@ -31,7 +30,6 @@ import org.opensearch.snapshots.RestoreService; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.function.Function; @@ -80,7 +78,6 @@ public ClusterState execute(ClusterState currentState) { } if (currentIndexMetadata.getSettings().getAsBoolean(SETTING_REMOTE_STORE_ENABLED, false)) { IndexMetadata updatedIndexMetadata = currentIndexMetadata; - Map activeInitializingShards = new HashMap<>(); if (request.restoreAllShards()) { if (currentIndexMetadata.getState() != IndexMetadata.State.CLOSE) { throw new IllegalStateException( @@ -97,17 +94,15 @@ public ClusterState execute(ClusterState currentState) { .settingsVersion(1 + currentIndexMetadata.getSettingsVersion()) .aliasesVersion(1 + currentIndexMetadata.getAliasesVersion()) .build(); - } else { - activeInitializingShards = currentState.routingTable() - .index(index) - .shards() - .values() - .stream() - .map(IndexShardRoutingTable::primaryShard) - .filter(shardRouting -> shardRouting.unassigned() == false) - .collect(Collectors.toMap(ShardRouting::shardId, Function.identity())); } + Map indexShardRoutingTableMap = currentState.routingTable() + .index(index) + .shards() + .values() + .stream() + .collect(Collectors.toMap(IndexShardRoutingTable::shardId, Function.identity())); + IndexId indexId = new IndexId(index, updatedIndexMetadata.getIndexUUID()); RecoverySource.RemoteStoreRecoverySource recoverySource = new RecoverySource.RemoteStoreRecoverySource( @@ -115,7 +110,12 @@ public ClusterState execute(ClusterState currentState) { updatedIndexMetadata.getCreationVersion(), indexId ); - rtBuilder.addAsRemoteStoreRestore(updatedIndexMetadata, recoverySource, activeInitializingShards); + rtBuilder.addAsRemoteStoreRestore( + updatedIndexMetadata, + recoverySource, + indexShardRoutingTableMap, + request.restoreAllShards() + ); blocks.updateBlocks(updatedIndexMetadata); mdBuilder.put(updatedIndexMetadata, true); indicesToBeRestored.add(index); diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index 1d0184de9d93c..4a39c3e6c942e 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -2335,7 +2335,7 @@ private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier, b assert currentEngineReference.get() == null : "engine is running"; verifyNotClosed(); if (indexSettings.isRemoteStoreEnabled() && syncFromRemote) { - syncSegmentsFromRemoteSegmentStore(false, true); + syncSegmentsFromRemoteSegmentStore(false); } if (indexSettings.isRemoteTranslogStoreEnabled() && shardRouting.primary()) { if (syncFromRemote) { @@ -4596,7 +4596,7 @@ public void close() throws IOException { }; IOUtils.close(currentEngineReference.getAndSet(readOnlyEngine)); if (indexSettings.isRemoteStoreEnabled()) { - syncSegmentsFromRemoteSegmentStore(false, true); + syncSegmentsFromRemoteSegmentStore(false); } if (indexSettings.isRemoteTranslogStoreEnabled() && shardRouting.primary()) { syncRemoteTranslogAndUpdateGlobalCheckpoint(); @@ -4656,10 +4656,9 @@ public void syncTranslogFilesFromRemoteTranslog() throws IOException { /** * Downloads segments from remote segment store. * @param overrideLocal flag to override local segment files with those in remote store - * @param refreshLevelSegmentSync last refresh checkpoint is used if true, commit checkpoint otherwise * @throws IOException if exception occurs while reading segments from remote store */ - public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, boolean refreshLevelSegmentSync) throws IOException { + public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal) throws IOException { assert indexSettings.isRemoteStoreEnabled(); logger.trace("Downloading segments from remote segment store"); RemoteSegmentStoreDirectory remoteDirectory = getRemoteDirectory(); @@ -4671,9 +4670,7 @@ public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, boolean re .getSegmentsUploadedToRemoteStore() .entrySet() .stream() - // if this is a refresh level sync, ignore any segments_n uploaded to the store, we will commit the received infos bytes - // locally. - .filter(entry -> refreshLevelSegmentSync && entry.getKey().startsWith(IndexFileNames.SEGMENTS) == false) + .filter(entry -> entry.getKey().startsWith(IndexFileNames.SEGMENTS) == false) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); store.incRef(); remoteStore.incRef(); @@ -4694,7 +4691,7 @@ public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, boolean re } copySegmentFiles(storeDirectory, remoteDirectory, null, uploadedSegments, overrideLocal); - if (refreshLevelSegmentSync && remoteSegmentMetadata != null) { + if (remoteSegmentMetadata != null) { final SegmentInfos infosSnapshot = store.buildSegmentInfos( remoteSegmentMetadata.getSegmentInfosBytes(), remoteSegmentMetadata.getGeneration() diff --git a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java index e8a9ec866ac01..bbb8fd1b3d08e 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java @@ -173,6 +173,14 @@ private boolean syncSegments() { indexShard.getReplicationTracker().isPrimaryMode(), indexShard.state() ); + // Following check is required to enable retry and make sure that we do not lose this refresh event + // When primary shard is restored from remote store, the recovery happens first followed by changing + // primaryMode to true. Due to this, the refresh that is triggered post replay of translog will not go through + // if following condition does not exist. The segments created as part of translog replay will not be present + // in the remote store. + if (indexShard.state() == IndexShardState.STARTED && indexShard.getEngine() instanceof InternalEngine) { + return false; + } return true; } ReplicationCheckpoint checkpoint = indexShard.getLatestReplicationCheckpoint(); diff --git a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java index d0c083390ab70..6d675b709e05b 100644 --- a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java @@ -535,11 +535,13 @@ private void recoverFromRemoteStore(IndexShard indexShard) throws IndexShardReco remoteStore.incRef(); try { // Download segments from remote segment store - indexShard.syncSegmentsFromRemoteSegmentStore(true, true); + indexShard.syncSegmentsFromRemoteSegmentStore(true); indexShard.syncTranslogFilesFromRemoteTranslog(); - if (store.directory().listAll().length == 0) { + // On index creation, the only segment file that is created is segments_N. We can safely discard this file + // as there is no data associated with this shard as part of segments. + if (store.directory().listAll().length <= 1) { Path location = indexShard.shardPath().resolveTranslog(); Checkpoint checkpoint = Checkpoint.read(location.resolve(CHECKPOINT_FILE_NAME)); final Path translogFile = location.resolve(Translog.getFilename(checkpoint.getGeneration())); diff --git a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java index 88b4cd063b8a6..df1589b3f29b9 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java @@ -245,7 +245,7 @@ private void doRecovery(final long recoveryId, final StartRecoveryRequest preExi indexShard.prepareForIndexRecovery(); final boolean hasRemoteSegmentStore = indexShard.indexSettings().isRemoteStoreEnabled(); if (hasRemoteSegmentStore) { - indexShard.syncSegmentsFromRemoteSegmentStore(false, true); + indexShard.syncSegmentsFromRemoteSegmentStore(false); } final boolean hasRemoteTranslog = recoveryTarget.state().getPrimary() == false && indexShard.isRemoteTranslogEnabled(); final boolean hasNoTranslog = indexShard.indexSettings().isRemoteSnapshot(); diff --git a/server/src/test/java/org/opensearch/cluster/routing/RoutingTableTests.java b/server/src/test/java/org/opensearch/cluster/routing/RoutingTableTests.java index 6e7583fbd75d5..8542ff53c6ff1 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/RoutingTableTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/RoutingTableTests.java @@ -50,13 +50,18 @@ import org.opensearch.repositories.IndexId; import org.junit.Before; +import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.function.Predicate; import java.util.stream.Collectors; +import static org.opensearch.cluster.routing.ShardRoutingState.INITIALIZING; +import static org.opensearch.cluster.routing.ShardRoutingState.RELOCATING; +import static org.opensearch.cluster.routing.ShardRoutingState.STARTED; import static org.opensearch.cluster.routing.ShardRoutingState.UNASSIGNED; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; @@ -64,6 +69,7 @@ import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.nullValue; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class RoutingTableTests extends OpenSearchAllocationTestCase { @@ -540,8 +546,47 @@ public void testAddAsRecovery() { } } - public void testAddAsRemoteStoreRestore() { - final IndexMetadata indexMetadata = createIndexMetadata(TEST_INDEX_1).state(IndexMetadata.State.OPEN).build(); + private Map getIndexShardRoutingTableMap(Index index, boolean allUnassigned, int numberOfReplicas) { + Map indexShardRoutingTableMap = new HashMap<>(); + List activeInitializingStates = List.of(INITIALIZING, STARTED, RELOCATING); + for (int i = 0; i < this.numberOfShards; i++) { + IndexShardRoutingTable indexShardRoutingTable = mock(IndexShardRoutingTable.class); + ShardRouting primaryShardRouting = mock(ShardRouting.class); + Boolean primaryUnassigned = allUnassigned || randomBoolean(); + when(primaryShardRouting.unassigned()).thenReturn(primaryUnassigned); + if (primaryUnassigned) { + when(primaryShardRouting.state()).thenReturn(UNASSIGNED); + } else { + when(primaryShardRouting.state()).thenReturn( + activeInitializingStates.get(randomIntBetween(0, activeInitializingStates.size() - 1)) + ); + } + when(indexShardRoutingTable.primaryShard()).thenReturn(primaryShardRouting); + List replicaShards = new ArrayList<>(); + for (int j = 0; j < numberOfReplicas; j++) { + ShardRouting replicaShardRouting = mock(ShardRouting.class); + Boolean replicaUnassigned = allUnassigned || randomBoolean(); + when(replicaShardRouting.unassigned()).thenReturn(replicaUnassigned); + if (replicaUnassigned) { + when(replicaShardRouting.state()).thenReturn(UNASSIGNED); + } else { + when(replicaShardRouting.state()).thenReturn( + activeInitializingStates.get(randomIntBetween(0, activeInitializingStates.size() - 1)) + ); + } + replicaShards.add(replicaShardRouting); + } + when(indexShardRoutingTable.replicaShards()).thenReturn(replicaShards); + indexShardRoutingTableMap.put(new ShardId(index, i), indexShardRoutingTable); + } + return indexShardRoutingTableMap; + } + + public void testAddAsRemoteStoreRestoreAllUnassigned() { + int numberOfReplicas = randomIntBetween(0, 5); + final IndexMetadata indexMetadata = createIndexMetadata(TEST_INDEX_1).state(IndexMetadata.State.OPEN) + .numberOfReplicas(numberOfReplicas) + .build(); final RemoteStoreRecoverySource remoteStoreRecoverySource = new RemoteStoreRecoverySource( "restore_uuid", Version.CURRENT, @@ -550,34 +595,78 @@ public void testAddAsRemoteStoreRestore() { final RoutingTable routingTable = new RoutingTable.Builder().addAsRemoteStoreRestore( indexMetadata, remoteStoreRecoverySource, - new HashMap<>() + getIndexShardRoutingTableMap(indexMetadata.getIndex(), true, numberOfReplicas), + false ).build(); assertTrue(routingTable.hasIndex(TEST_INDEX_1)); - assertEquals(this.numberOfShards, routingTable.allShards(TEST_INDEX_1).size()); - assertEquals(this.numberOfShards, routingTable.index(TEST_INDEX_1).shardsWithState(UNASSIGNED).size()); + int numberOfShards = this.numberOfShards * (numberOfReplicas + 1); + assertEquals(numberOfShards, routingTable.allShards(TEST_INDEX_1).size()); + assertEquals(numberOfShards, routingTable.index(TEST_INDEX_1).shardsWithState(UNASSIGNED).size()); } public void testAddAsRemoteStoreRestoreWithActiveShards() { - final IndexMetadata indexMetadata = createIndexMetadata(TEST_INDEX_1).state(IndexMetadata.State.OPEN).build(); + int numberOfReplicas = randomIntBetween(0, 5); + final IndexMetadata indexMetadata = createIndexMetadata(TEST_INDEX_1).state(IndexMetadata.State.OPEN) + .numberOfReplicas(numberOfReplicas) + .build(); final RemoteStoreRecoverySource remoteStoreRecoverySource = new RemoteStoreRecoverySource( "restore_uuid", Version.CURRENT, new IndexId(TEST_INDEX_1, "1") ); - Map activeInitializingShards = new HashMap<>(); - for (int i = 0; i < randomIntBetween(1, this.numberOfShards); i++) { - activeInitializingShards.put(new ShardId(indexMetadata.getIndex(), i), mock(ShardRouting.class)); - } + Map indexShardRoutingTableMap = getIndexShardRoutingTableMap( + indexMetadata.getIndex(), + false, + numberOfReplicas + ); final RoutingTable routingTable = new RoutingTable.Builder().addAsRemoteStoreRestore( indexMetadata, remoteStoreRecoverySource, - activeInitializingShards + indexShardRoutingTableMap, + false ).build(); assertTrue(routingTable.hasIndex(TEST_INDEX_1)); - assertEquals(this.numberOfShards, routingTable.allShards(TEST_INDEX_1).size()); - assertEquals( - this.numberOfShards - activeInitializingShards.size(), - routingTable.index(TEST_INDEX_1).shardsWithState(UNASSIGNED).size() + int numberOfShards = this.numberOfShards * (numberOfReplicas + 1); + assertEquals(numberOfShards, routingTable.allShards(TEST_INDEX_1).size()); + int unassignedShards = 0; + for (IndexShardRoutingTable indexShardRoutingTable : indexShardRoutingTableMap.values()) { + if (indexShardRoutingTable.primaryShard().unassigned()) { + unassignedShards += indexShardRoutingTable.replicaShards().size() + 1; + } else { + for (ShardRouting replicaShardRouting : indexShardRoutingTable.replicaShards()) { + if (replicaShardRouting.unassigned()) { + unassignedShards += 1; + } + } + } + } + assertEquals(unassignedShards, routingTable.index(TEST_INDEX_1).shardsWithState(UNASSIGNED).size()); + } + + public void testAddAsRemoteStoreRestoreShardMismatch() { + int numberOfReplicas = randomIntBetween(0, 5); + final IndexMetadata indexMetadata = createIndexMetadata(TEST_INDEX_1).state(IndexMetadata.State.OPEN) + .numberOfReplicas(numberOfReplicas) + .build(); + final RemoteStoreRecoverySource remoteStoreRecoverySource = new RemoteStoreRecoverySource( + "restore_uuid", + Version.CURRENT, + new IndexId(TEST_INDEX_1, "1") + ); + Map indexShardRoutingTableMap = getIndexShardRoutingTableMap( + indexMetadata.getIndex(), + true, + numberOfReplicas + ); + indexShardRoutingTableMap.remove(indexShardRoutingTableMap.keySet().iterator().next()); + assertThrows( + IllegalStateException.class, + () -> new RoutingTable.Builder().addAsRemoteStoreRestore( + indexMetadata, + remoteStoreRecoverySource, + indexShardRoutingTableMap, + false + ).build() ); } diff --git a/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java b/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java index 83b07e986bcc5..95fe67592d5f8 100644 --- a/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java +++ b/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java @@ -532,10 +532,9 @@ private Tuple mockIndexS new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), threadPool ); - RemoteStorePressureService remoteStorePressureService = new RemoteStorePressureService(clusterService, Settings.EMPTY); + RemoteStorePressureService remoteStorePressureService = indexShard.getRemoteStorePressureService(); when(shard.indexSettings()).thenReturn(indexShard.indexSettings()); when(shard.shardId()).thenReturn(indexShard.shardId()); - remoteStorePressureService.afterIndexShardCreated(shard); RemoteSegmentTransferTracker tracker = remoteStorePressureService.getRemoteRefreshSegmentTracker(indexShard.shardId()); RemoteStoreRefreshListener refreshListener = new RemoteStoreRefreshListener(shard, emptyCheckpointPublisher, tracker); refreshListener.afterRefresh(true); From f4106a474c8503187b1d6bbe47a85a82f5243163 Mon Sep 17 00:00:00 2001 From: Gagan Juneja Date: Mon, 28 Aug 2023 21:43:27 +0530 Subject: [PATCH 03/37] Add support to use trace propagated from client (#9506) * Add support to use trace propagated from client Signed-off-by: Gagan Juneja * Add support to use trace propagated from client Signed-off-by: Gagan Juneja * Refactor code Signed-off-by: Gagan Juneja * Add support to use trace propagated from client Signed-off-by: Gagan Juneja * Add support to use trace propagated from client Signed-off-by: Gagan Juneja * Refactor code Signed-off-by: Gagan Juneja * Refactor code Signed-off-by: Gagan Juneja * Merged CHANGELOG Signed-off-by: Gagan Juneja * Empty-Commit Signed-off-by: Gagan Juneja * Empty-Commit Signed-off-by: Gagan Juneja * Empty-Commit Signed-off-by: Gagan Juneja * Empty-Commit Signed-off-by: Gagan Juneja * Empty-Commit Signed-off-by: Gagan Juneja * Empty-Commit Signed-off-by: Gagan Juneja --------- Signed-off-by: Gagan Juneja Signed-off-by: Gagan Juneja Co-authored-by: Gagan Juneja --- CHANGELOG.md | 1 + .../telemetry/tracing/DefaultTracer.java | 11 +++++- .../opensearch/telemetry/tracing/Tracer.java | 3 +- .../tracing/TracingContextPropagator.java | 12 ++++++- .../telemetry/tracing/http/HttpTracer.java | 34 ++++++++++++++++++ .../telemetry/tracing/http/package-info.java | 12 +++++++ .../telemetry/tracing/noop/NoopTracer.java | 8 +++++ .../telemetry/tracing/DefaultTracerTests.java | 28 ++++++++++++++- .../tracing/TraceableRunnableTests.java | 8 ++--- .../tracing/OTelTracingContextPropagator.java | 36 ++++++++++++++++++- .../OTelTracingContextPropagatorTests.java | 36 ++++++++++++++++++- ...hreadContextBasedTracerContextStorage.java | 3 +- .../telemetry/tracing/WrappedTracer.java | 9 ++++- .../telemetry/tracing/WrappedTracerTests.java | 4 +-- test/telemetry/build.gradle | 1 + .../tracing/MockTracingContextPropagator.java | 23 ++++++++++-- 16 files changed, 212 insertions(+), 17 deletions(-) create mode 100644 libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/http/HttpTracer.java create mode 100644 libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/http/package-info.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 441de802f6ea8..4ff048fdbf1a3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -156,6 +156,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Improve performance of encoding composite keys in multi-term aggregations ([#9412](https://github.com/opensearch-project/OpenSearch/pull/9412)) - Fix sort related ITs for concurrent search ([#9177](https://github.com/opensearch-project/OpenSearch/pull/9466) - Removing the vec file extension from INDEX_STORE_HYBRID_NIO_EXTENSIONS, to ensure the no performance degradation for vector search via Lucene Engine.([#9528](https://github.com/opensearch-project/OpenSearch/pull/9528))) +- Add support to use trace propagated from client ([#9506](https://github.com/opensearch-project/OpenSearch/pull/9506)) - Separate request-based and settings-based concurrent segment search controls and introduce AggregatorFactory method to determine concurrent search support ([#9469](https://github.com/opensearch-project/OpenSearch/pull/9469)) - [Remote Store] Rate limiter integration for remote store uploads and downloads([#9448](https://github.com/opensearch-project/OpenSearch/pull/9448/)) - [Remote Store] Implicitly use replication type SEGMENT for remote store clusters ([#9264](https://github.com/opensearch-project/OpenSearch/pull/9264)) diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/DefaultTracer.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/DefaultTracer.java index 2f3a425f96703..bc1a08e2d3c72 100644 --- a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/DefaultTracer.java +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/DefaultTracer.java @@ -12,6 +12,9 @@ import java.io.Closeable; import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Optional; /** * @@ -44,7 +47,7 @@ public SpanScope startSpan(String spanName) { @Override public SpanScope startSpan(String spanName, Attributes attributes) { - return startSpan(spanName, null, attributes); + return startSpan(spanName, (SpanContext) null, attributes); } @Override @@ -97,4 +100,10 @@ protected void addDefaultAttributes(Span span) { span.addAttribute(THREAD_NAME, Thread.currentThread().getName()); } + @Override + public SpanScope startSpan(String spanName, Map> headers, Attributes attributes) { + Optional propagatedSpan = tracingTelemetry.getContextPropagator().extractFromHeaders(headers); + return startSpan(spanName, propagatedSpan.map(SpanContext::new).orElse(null), attributes); + } + } diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/Tracer.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/Tracer.java index bc55b26abc761..40cc5dfd2d743 100644 --- a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/Tracer.java +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/Tracer.java @@ -9,6 +9,7 @@ package org.opensearch.telemetry.tracing; import org.opensearch.telemetry.tracing.attributes.Attributes; +import org.opensearch.telemetry.tracing.http.HttpTracer; import java.io.Closeable; @@ -18,7 +19,7 @@ * * All methods on the Tracer object are multi-thread safe. */ -public interface Tracer extends Closeable { +public interface Tracer extends HttpTracer, Closeable { /** * Starts the {@link Span} with given name diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/TracingContextPropagator.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/TracingContextPropagator.java index 3e4a377d33a3d..52e272afcd07f 100644 --- a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/TracingContextPropagator.java +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/TracingContextPropagator.java @@ -8,7 +8,9 @@ package org.opensearch.telemetry.tracing; +import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.function.BiConsumer; /** @@ -23,7 +25,15 @@ public interface TracingContextPropagator { * @param props properties * @return current span */ - Span extract(Map props); + Optional extract(Map props); + + /** + * Extracts current span from HTTP headers. + * + * @param headers request headers to extract the context from + * @return current span + */ + Optional extractFromHeaders(Map> headers); /** * Injects tracing context diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/http/HttpTracer.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/http/HttpTracer.java new file mode 100644 index 0000000000000..64ef84335a95b --- /dev/null +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/http/HttpTracer.java @@ -0,0 +1,34 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.telemetry.tracing.http; + +import org.opensearch.telemetry.tracing.Span; +import org.opensearch.telemetry.tracing.SpanScope; +import org.opensearch.telemetry.tracing.attributes.Attributes; + +import java.util.List; +import java.util.Map; + +/** + * HttpTracer helps in creating a {@link Span} which reads the incoming tracing information + * from the HttpRequest header and propagate the span accordingly. + * + * All methods on the Tracer object are multi-thread safe. + */ +public interface HttpTracer { + /** + * Start the span with propagating the tracing info from the HttpRequest header. + * + * @param spanName span name. + * @param header http request header. + * @param attributes span attributes. + * @return scope of the span, must be closed with explicit close or with try-with-resource + */ + SpanScope startSpan(String spanName, Map> header, Attributes attributes); +} diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/http/package-info.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/http/package-info.java new file mode 100644 index 0000000000000..9feb862a4e010 --- /dev/null +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/http/package-info.java @@ -0,0 +1,12 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +/** + * Contains No-op implementations + */ +package org.opensearch.telemetry.tracing.http; diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/NoopTracer.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/NoopTracer.java index 1a37ed0d0f245..2ff50bf3bcb18 100644 --- a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/NoopTracer.java +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/NoopTracer.java @@ -13,6 +13,9 @@ import org.opensearch.telemetry.tracing.Tracer; import org.opensearch.telemetry.tracing.attributes.Attributes; +import java.util.List; +import java.util.Map; + /** * No-op implementation of Tracer * @@ -51,4 +54,9 @@ public SpanContext getCurrentSpan() { public void close() { } + + @Override + public SpanScope startSpan(String spanName, Map> header, Attributes attributes) { + return SpanScope.NO_OP; + } } diff --git a/libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/DefaultTracerTests.java b/libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/DefaultTracerTests.java index 07abd43c8dd7b..150992da06f89 100644 --- a/libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/DefaultTracerTests.java +++ b/libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/DefaultTracerTests.java @@ -17,6 +17,10 @@ import org.junit.Assert; import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; @@ -104,6 +108,28 @@ public void testCreateSpanWithParent() { Assert.assertEquals(parentSpan.getSpan(), defaultTracer.getCurrentSpan().getSpan().getParentSpan()); } + public void testHttpTracer() { + String traceId = "trace_id"; + String spanId = "span_id"; + TracingTelemetry tracingTelemetry = new MockTracingTelemetry(); + + DefaultTracer defaultTracer = new DefaultTracer( + tracingTelemetry, + new ThreadContextBasedTracerContextStorage(new ThreadContext(Settings.EMPTY), tracingTelemetry) + ); + + Map> requestHeaders = new HashMap<>(); + requestHeaders.put("traceparent", Arrays.asList(traceId + "~" + spanId)); + + SpanScope spanScope = defaultTracer.startSpan("test_span", requestHeaders, Attributes.EMPTY); + SpanContext currentSpan = defaultTracer.getCurrentSpan(); + assertNotNull(currentSpan); + assertEquals(traceId, currentSpan.getSpan().getTraceId()); + assertEquals(traceId, currentSpan.getSpan().getParentSpan().getTraceId()); + assertEquals(spanId, currentSpan.getSpan().getParentSpan().getSpanId()); + spanScope.close(); + } + public void testCreateSpanWithNullParent() { TracingTelemetry tracingTelemetry = new MockTracingTelemetry(); DefaultTracer defaultTracer = new DefaultTracer( @@ -111,7 +137,7 @@ public void testCreateSpanWithNullParent() { new ThreadContextBasedTracerContextStorage(new ThreadContext(Settings.EMPTY), tracingTelemetry) ); - defaultTracer.startSpan("span_name", null, Attributes.EMPTY); + defaultTracer.startSpan("span_name"); Assert.assertEquals("span_name", defaultTracer.getCurrentSpan().getSpan().getSpanName()); Assert.assertEquals(null, defaultTracer.getCurrentSpan().getSpan().getParentSpan()); diff --git a/libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/TraceableRunnableTests.java b/libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/TraceableRunnableTests.java index f1df3b24e1c9b..bcd8ffe41a17b 100644 --- a/libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/TraceableRunnableTests.java +++ b/libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/TraceableRunnableTests.java @@ -54,16 +54,16 @@ public void testRunnableWithParent() throws Exception { DefaultTracer defaultTracer = new DefaultTracer(new MockTracingTelemetry(), contextStorage); defaultTracer.startSpan(parentSpanName); SpanContext parentSpan = defaultTracer.getCurrentSpan(); - AtomicReference currrntSpan = new AtomicReference<>(new SpanContext(null)); + AtomicReference currentSpan = new AtomicReference<>(); final AtomicBoolean isRunnableCompleted = new AtomicBoolean(false); TraceableRunnable traceableRunnable = new TraceableRunnable(defaultTracer, spanName, parentSpan, Attributes.EMPTY, () -> { isRunnableCompleted.set(true); - currrntSpan.set(defaultTracer.getCurrentSpan()); + currentSpan.set(defaultTracer.getCurrentSpan()); }); traceableRunnable.run(); assertTrue(isRunnableCompleted.get()); - assertEquals(spanName, currrntSpan.get().getSpan().getSpanName()); - assertEquals(parentSpan.getSpan(), currrntSpan.get().getSpan().getParentSpan()); + assertEquals(spanName, currentSpan.get().getSpan().getSpanName()); + assertEquals(parentSpan.getSpan(), currentSpan.get().getSpan().getParentSpan()); assertEquals(parentSpan.getSpan(), defaultTracer.getCurrentSpan().getSpan()); } } diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTracingContextPropagator.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTracingContextPropagator.java index 739a6367ccb2e..f8fe885ee450c 100644 --- a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTracingContextPropagator.java +++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTracingContextPropagator.java @@ -8,7 +8,12 @@ package org.opensearch.telemetry.tracing; +import org.opensearch.core.common.Strings; + +import java.util.Collections; +import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.function.BiConsumer; import io.opentelemetry.api.OpenTelemetry; @@ -32,8 +37,12 @@ public OTelTracingContextPropagator(OpenTelemetry openTelemetry) { } @Override - public Span extract(Map props) { + public Optional extract(Map props) { Context context = openTelemetry.getPropagators().getTextMapPropagator().extract(Context.current(), props, TEXT_MAP_GETTER); + return Optional.ofNullable(getPropagatedSpan(context)); + } + + private static OTelPropagatedSpan getPropagatedSpan(Context context) { if (context != null) { io.opentelemetry.api.trace.Span span = io.opentelemetry.api.trace.Span.fromContext(context); return new OTelPropagatedSpan(span); @@ -41,6 +50,12 @@ public Span extract(Map props) { return null; } + @Override + public Optional extractFromHeaders(Map> headers) { + Context context = openTelemetry.getPropagators().getTextMapPropagator().extract(Context.current(), headers, HEADER_TEXT_MAP_GETTER); + return Optional.ofNullable(getPropagatedSpan(context)); + } + @Override public void inject(Span currentSpan, BiConsumer setter) { openTelemetry.getPropagators().getTextMapPropagator().inject(context((OTelSpan) currentSpan), setter, TEXT_MAP_SETTER); @@ -72,4 +87,23 @@ public String get(Map headers, String key) { } }; + private static final TextMapGetter>> HEADER_TEXT_MAP_GETTER = new TextMapGetter<>() { + @Override + public Iterable keys(Map> headers) { + if (headers != null) { + return headers.keySet(); + } else { + return Collections.emptySet(); + } + } + + @Override + public String get(Map> headers, String key) { + if (headers != null && headers.containsKey(key)) { + return Strings.collectionToCommaDelimitedString(headers.get(key)); + } + return null; + } + }; + } diff --git a/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/OTelTracingContextPropagatorTests.java b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/OTelTracingContextPropagatorTests.java index fcf7495f331af..16a3ec9493d5d 100644 --- a/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/OTelTracingContextPropagatorTests.java +++ b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/OTelTracingContextPropagatorTests.java @@ -10,7 +10,9 @@ import org.opensearch.test.OpenSearchTestCase; +import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; import io.opentelemetry.api.OpenTelemetry; @@ -19,6 +21,7 @@ import io.opentelemetry.api.trace.TraceFlags; import io.opentelemetry.api.trace.TraceState; import io.opentelemetry.api.trace.propagation.W3CTraceContextPropagator; +import io.opentelemetry.context.Context; import io.opentelemetry.context.propagation.ContextPropagators; import static org.mockito.Mockito.mock; @@ -48,8 +51,39 @@ public void testExtractTracerContextFromHeader() { OpenTelemetry mockOpenTelemetry = mock(OpenTelemetry.class); when(mockOpenTelemetry.getPropagators()).thenReturn(ContextPropagators.create(W3CTraceContextPropagator.getInstance())); TracingContextPropagator tracingContextPropagator = new OTelTracingContextPropagator(mockOpenTelemetry); - org.opensearch.telemetry.tracing.Span span = tracingContextPropagator.extract(requestHeaders); + org.opensearch.telemetry.tracing.Span span = tracingContextPropagator.extract(requestHeaders).orElse(null); assertEquals(TRACE_ID, span.getTraceId()); assertEquals(SPAN_ID, span.getSpanId()); } + + public void testExtractTracerContextFromHttpHeader() { + Map> requestHeaders = new HashMap<>(); + requestHeaders.put("traceparent", Arrays.asList("00-" + TRACE_ID + "-" + SPAN_ID + "-00")); + OpenTelemetry mockOpenTelemetry = mock(OpenTelemetry.class); + when(mockOpenTelemetry.getPropagators()).thenReturn(ContextPropagators.create(W3CTraceContextPropagator.getInstance())); + TracingContextPropagator tracingContextPropagator = new OTelTracingContextPropagator(mockOpenTelemetry); + org.opensearch.telemetry.tracing.Span span = tracingContextPropagator.extractFromHeaders(requestHeaders).get(); + assertEquals(TRACE_ID, span.getTraceId()); + assertEquals(SPAN_ID, span.getSpanId()); + } + + public void testExtractTracerContextFromHttpHeaderNull() { + OpenTelemetry mockOpenTelemetry = mock(OpenTelemetry.class); + when(mockOpenTelemetry.getPropagators()).thenReturn(ContextPropagators.create(W3CTraceContextPropagator.getInstance())); + TracingContextPropagator tracingContextPropagator = new OTelTracingContextPropagator(mockOpenTelemetry); + org.opensearch.telemetry.tracing.Span span = tracingContextPropagator.extractFromHeaders(null).get(); + org.opensearch.telemetry.tracing.Span propagatedSpan = new OTelPropagatedSpan(Span.fromContext(Context.root())); + assertEquals(propagatedSpan.getTraceId(), span.getTraceId()); + assertEquals(propagatedSpan.getSpanId(), span.getSpanId()); + } + + public void testExtractTracerContextFromHttpHeaderEmpty() { + OpenTelemetry mockOpenTelemetry = mock(OpenTelemetry.class); + when(mockOpenTelemetry.getPropagators()).thenReturn(ContextPropagators.create(W3CTraceContextPropagator.getInstance())); + TracingContextPropagator tracingContextPropagator = new OTelTracingContextPropagator(mockOpenTelemetry); + org.opensearch.telemetry.tracing.Span span = tracingContextPropagator.extractFromHeaders(new HashMap<>()).get(); + org.opensearch.telemetry.tracing.Span propagatedSpan = new OTelPropagatedSpan(Span.fromContext(Context.root())); + assertEquals(propagatedSpan.getTraceId(), span.getTraceId()); + assertEquals(propagatedSpan.getSpanId(), span.getSpanId()); + } } diff --git a/server/src/main/java/org/opensearch/telemetry/tracing/ThreadContextBasedTracerContextStorage.java b/server/src/main/java/org/opensearch/telemetry/tracing/ThreadContextBasedTracerContextStorage.java index c009ab2391aab..a32facdc71146 100644 --- a/server/src/main/java/org/opensearch/telemetry/tracing/ThreadContextBasedTracerContextStorage.java +++ b/server/src/main/java/org/opensearch/telemetry/tracing/ThreadContextBasedTracerContextStorage.java @@ -90,6 +90,7 @@ private Optional spanFromThreadContext(String key) { } private Span spanFromHeader() { - return tracingTelemetry.getContextPropagator().extract(threadContext.getHeaders()); + Optional span = tracingTelemetry.getContextPropagator().extract(threadContext.getHeaders()); + return span.orElse(null); } } diff --git a/server/src/main/java/org/opensearch/telemetry/tracing/WrappedTracer.java b/server/src/main/java/org/opensearch/telemetry/tracing/WrappedTracer.java index 466abaac435f3..b699471be7f4c 100644 --- a/server/src/main/java/org/opensearch/telemetry/tracing/WrappedTracer.java +++ b/server/src/main/java/org/opensearch/telemetry/tracing/WrappedTracer.java @@ -13,6 +13,8 @@ import org.opensearch.telemetry.tracing.noop.NoopTracer; import java.io.IOException; +import java.util.List; +import java.util.Map; /** * Wrapper implementation of Tracer. This delegates call to right tracer based on the tracer settings @@ -42,7 +44,7 @@ public SpanScope startSpan(String spanName) { @Override public SpanScope startSpan(String spanName, Attributes attributes) { - return startSpan(spanName, null, attributes); + return startSpan(spanName, (SpanContext) null, attributes); } @Override @@ -66,4 +68,9 @@ public void close() throws IOException { Tracer getDelegateTracer() { return telemetrySettings.isTracingEnabled() ? defaultTracer : NoopTracer.INSTANCE; } + + @Override + public SpanScope startSpan(String spanName, Map> headers, Attributes attributes) { + return defaultTracer.startSpan(spanName, headers, attributes); + } } diff --git a/server/src/test/java/org/opensearch/telemetry/tracing/WrappedTracerTests.java b/server/src/test/java/org/opensearch/telemetry/tracing/WrappedTracerTests.java index f45381e3b4cc4..b70fe81d5f9c4 100644 --- a/server/src/test/java/org/opensearch/telemetry/tracing/WrappedTracerTests.java +++ b/server/src/test/java/org/opensearch/telemetry/tracing/WrappedTracerTests.java @@ -51,7 +51,7 @@ public void testStartSpanWithTracingEnabledInvokesDefaultTracer() throws Excepti wrappedTracer.startSpan("foo"); assertTrue(wrappedTracer.getDelegateTracer() instanceof DefaultTracer); - verify(mockDefaultTracer).startSpan(eq("foo"), eq(null), any(Attributes.class)); + verify(mockDefaultTracer).startSpan(eq("foo"), eq((SpanContext) null), any(Attributes.class)); } } @@ -64,7 +64,7 @@ public void testStartSpanWithTracingEnabledInvokesDefaultTracerWithAttr() throws wrappedTracer.startSpan("foo", attributes); assertTrue(wrappedTracer.getDelegateTracer() instanceof DefaultTracer); - verify(mockDefaultTracer).startSpan("foo", null, attributes); + verify(mockDefaultTracer).startSpan("foo", (SpanContext) null, attributes); } } diff --git a/test/telemetry/build.gradle b/test/telemetry/build.gradle index fbabe43aa5e5a..ca523a9204f4c 100644 --- a/test/telemetry/build.gradle +++ b/test/telemetry/build.gradle @@ -13,6 +13,7 @@ apply plugin: 'opensearch.build' apply plugin: 'opensearch.publish' dependencies { + api project(":libs:opensearch-core") api project(":libs:opensearch-common") api project(":libs:opensearch-telemetry") } diff --git a/test/telemetry/src/main/java/org/opensearch/test/telemetry/tracing/MockTracingContextPropagator.java b/test/telemetry/src/main/java/org/opensearch/test/telemetry/tracing/MockTracingContextPropagator.java index dccf062df5ca5..7525b4424c243 100644 --- a/test/telemetry/src/main/java/org/opensearch/test/telemetry/tracing/MockTracingContextPropagator.java +++ b/test/telemetry/src/main/java/org/opensearch/test/telemetry/tracing/MockTracingContextPropagator.java @@ -8,13 +8,17 @@ package org.opensearch.test.telemetry.tracing; +import org.opensearch.core.common.Strings; import org.opensearch.telemetry.tracing.Span; import org.opensearch.telemetry.tracing.TracingContextPropagator; import org.opensearch.telemetry.tracing.attributes.Attributes; +import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Optional; import java.util.function.BiConsumer; +import java.util.stream.Collectors; /** * Mock {@link TracingContextPropagator} to persist the span for internode communication. @@ -34,18 +38,31 @@ public MockTracingContextPropagator(SpanProcessor spanProcessor) { } @Override - public Span extract(Map props) { + public Optional extract(Map props) { String value = props.get(TRACE_PARENT); if (value != null) { String[] values = value.split(SEPARATOR); String traceId = values[0]; String spanId = values[1]; - return new MockSpan(null, null, traceId, spanId, spanProcessor, Attributes.EMPTY); + return Optional.of(new MockSpan(null, null, traceId, spanId, spanProcessor, Attributes.EMPTY)); } else { - return null; + return Optional.empty(); } } + @Override + public Optional extractFromHeaders(Map> headers) { + if (headers != null) { + Map convertedHeader = headers.entrySet() + .stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> Strings.collectionToCommaDelimitedString(e.getValue()))); + return extract(convertedHeader); + } else { + return Optional.empty(); + } + + } + @Override public void inject(Span currentSpan, BiConsumer setter) { if (currentSpan instanceof MockSpan) { From a08d588691c3b232e65d73b0a0c2fc5c72c870cf Mon Sep 17 00:00:00 2001 From: Stephen Crawford <65832608+scrawfor99@users.noreply.github.com> Date: Mon, 28 Aug 2023 15:42:22 -0400 Subject: [PATCH 04/37] Add Secure Bwc documentation to TESTING.md (#9414) * Update BWC testing documentation Signed-off-by: Stephen Crawford * Fix code block Signed-off-by: Stephen Crawford * Fix code block Signed-off-by: Stephen Crawford * Add placeholder Signed-off-by: Stephen Crawford * Fix typos Signed-off-by: Stephen Crawford * Fix table of contents order Signed-off-by: Stephen Crawford * Add colon Signed-off-by: Stephen Crawford * trigger retry Signed-off-by: Stephen Crawford --------- Signed-off-by: Stephen Crawford --- TESTING.md | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/TESTING.md b/TESTING.md index a76ee07e1faac..1c91d60840d61 100644 --- a/TESTING.md +++ b/TESTING.md @@ -23,6 +23,7 @@ OpenSearch uses [jUnit](https://junit.org/junit5/) for testing, it also uses ran - [Iterating on packaging tests](#iterating-on-packaging-tests) - [Testing backwards compatibility](#testing-backwards-compatibility) - [BWC Testing against a specific remote/branch](#bwc-testing-against-a-specific-remotebranch) + - [BWC Testing with security](#bwc-testing-with-security) - [Skip fetching latest](#skip-fetching-latest) - [How to write good tests?](#how-to-write-good-tests) - [Base classes for test cases](#base-classes-for-test-cases) @@ -406,6 +407,29 @@ Example: Say you need to make a change to `main` and have a BWC layer in `5.x`. You will need to: . Create a branch called `index_req_change` off your remote `${remote}`. This will contain your change. . Create a branch called `index_req_bwc_5.x` off `5.x`. This will contain your bwc layer. . Push both branches to your remote repository. . Run the tests with `./gradlew check -Dbwc.remote=${remote} -Dbwc.refspec.5.x=index_req_bwc_5.x`. +## BWC Testing with security + +You may want to run BWC tests for a secure OpenSearch cluster. In order to do this, you will need to follow a few additional steps: + +1. Clone the OpenSearch Security repository from https://github.com/opensearch-project/security. +2. Get both the old version of the Security plugin (the version you wish to come from) and the new version of the Security plugin (the version you wish to go to). This can be done either by fetching the maven artifact with a command like `wget https://repo1.maven.org/maven2/org/opensearch/plugin/opensearch-security/.0/opensearch-security-.0.zip` or by running `./gradlew assemble` from the base of the Security repository. +3. Move both of the Security artifacts into new directories at the path `/security/bwc-test/src/test/resources/.0`. You should end up with two different directories in `/security/bwc-test/src/test/resources/`, one named the old version and one the new version. +4. Run the following command from the base of the Security repository: + +``` + ./gradlew -p bwc-test clean bwcTestSuite \ + -Dtests.security.manager=false \ + -Dtests.opensearch.http.protocol=https \ + -Dtests.opensearch.username=admin \ + -Dtests.opensearch.password=admin \ + -PcustomDistributionUrl="/OpenSearch/distribution/archives/linux-tar/build/distributions/opensearch-min--SNAPSHOT-linux-x64.tar.gz" \ + -i +``` + +`-Dtests.security.manager=false` handles access issues when attempting to read the certificates from the file system. +`-Dtests.opensearch.http.protocol=https` tells the wait for cluster startup task to do the right thing. +`-PcustomDistributionUrl=...` uses a custom build of the distribution of OpenSearch. This is unnecessary when running against standard/unmodified OpenSearch core distributions. + ### Skip fetching latest For some BWC testing scenarios, you want to use the local clone of the repository without fetching latest. For these use cases, you can set the system property `tests.bwc.git_fetch_latest` to `false` and the BWC builds will skip fetching the latest from the remote. From e5c4f9d2016bfa50c464945dc9533aa4aee49922 Mon Sep 17 00:00:00 2001 From: Sorabh Date: Mon, 28 Aug 2023 15:22:16 -0700 Subject: [PATCH 05/37] For sort request on timeseries field use non concurrent search path (#9562) * For sort request on timeseries field use non concurrent search path Signed-off-by: Sorabh Hamirwasia * Address review feedback Signed-off-by: Sorabh Hamirwasia --------- Signed-off-by: Sorabh Hamirwasia --- CHANGELOG.md | 1 + .../search/DefaultSearchContext.java | 22 ++- .../search/internal/ContextIndexSearcher.java | 30 +--- .../internal/FilteredSearchContext.java | 5 + .../search/internal/SearchContext.java | 2 + .../query/QueryPhaseSearcherWrapper.java | 9 +- .../search/sort/SortAndFormats.java | 10 ++ .../search/DefaultSearchContextTests.java | 159 ++++++++++++++++++ .../opensearch/test/TestSearchContext.java | 9 + 9 files changed, 215 insertions(+), 32 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 4ff048fdbf1a3..db57363ab605f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -160,6 +160,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Separate request-based and settings-based concurrent segment search controls and introduce AggregatorFactory method to determine concurrent search support ([#9469](https://github.com/opensearch-project/OpenSearch/pull/9469)) - [Remote Store] Rate limiter integration for remote store uploads and downloads([#9448](https://github.com/opensearch-project/OpenSearch/pull/9448/)) - [Remote Store] Implicitly use replication type SEGMENT for remote store clusters ([#9264](https://github.com/opensearch-project/OpenSearch/pull/9264)) +- Use non-concurrent path for sort request on timeseries index and field([#9562](https://github.com/opensearch-project/OpenSearch/pull/9562)) ### Deprecated diff --git a/server/src/main/java/org/opensearch/search/DefaultSearchContext.java b/server/src/main/java/org/opensearch/search/DefaultSearchContext.java index ef8a6c9f36b0c..28931bb5a860f 100644 --- a/server/src/main/java/org/opensearch/search/DefaultSearchContext.java +++ b/server/src/main/java/org/opensearch/search/DefaultSearchContext.java @@ -890,11 +890,15 @@ public boolean shouldUseConcurrentSearch() { * Evaluate if parsed request supports concurrent segment search */ public void evaluateRequestShouldUseConcurrentSearch() { - if (aggregations() != null && aggregations().factories() != null) { - requestShouldUseConcurrentSearch.set(aggregations().factories().allFactoriesSupportConcurrentSearch()); - } else { - requestShouldUseConcurrentSearch.set(true); - } + if (sort != null && sort.isSortOnTimeSeriesField()) { + requestShouldUseConcurrentSearch.set(false); + } else if (aggregations() != null + && aggregations().factories() != null + && !aggregations().factories().allFactoriesSupportConcurrentSearch()) { + requestShouldUseConcurrentSearch.set(false); + } else { + requestShouldUseConcurrentSearch.set(true); + } } public void setProfilers(Profilers profilers) { @@ -965,4 +969,12 @@ public int getTargetMaxSliceCount() { } return clusterService.getClusterSettings().get(SearchService.CONCURRENT_SEGMENT_SEARCH_TARGET_MAX_SLICE_COUNT_SETTING); } + + @Override + public boolean shouldUseTimeSeriesDescSortOptimization() { + return indexShard.isTimeSeriesDescSortOptimizationEnabled() + && sort != null + && sort.isSortOnTimeSeriesField() + && sort.sort.getSort()[0].getReverse() == false; + } } diff --git a/server/src/main/java/org/opensearch/search/internal/ContextIndexSearcher.java b/server/src/main/java/org/opensearch/search/internal/ContextIndexSearcher.java index 7974561d6187c..6c3d2bb278bd0 100644 --- a/server/src/main/java/org/opensearch/search/internal/ContextIndexSearcher.java +++ b/server/src/main/java/org/opensearch/search/internal/ContextIndexSearcher.java @@ -64,7 +64,6 @@ import org.apache.lucene.util.Bits; import org.apache.lucene.util.CombinedBitSet; import org.apache.lucene.util.SparseFixedBitSet; -import org.opensearch.cluster.metadata.DataStream; import org.opensearch.common.lease.Releasable; import org.opensearch.common.lucene.search.TopDocsAndMaxScore; import org.opensearch.search.DocValueFormat; @@ -268,10 +267,11 @@ public void search( @Override protected void search(List leaves, Weight weight, Collector collector) throws IOException { - if (shouldReverseLeafReaderContexts()) { - // reverse the segment search order if this flag is true. - // Certain queries can benefit if we reverse the segment read order, - // for example time series based queries if searched for desc sort order. + // Time series based workload by default traverses segments in desc order i.e. latest to the oldest order. + // This is actually beneficial for search queries to start search on latest segments first for time series workload. + // That can slow down ASC order queries on timestamp workload. So to avoid that slowdown, we will reverse leaf + // reader order here. + if (searchContext.shouldUseTimeSeriesDescSortOptimization()) { for (int i = leaves.size() - 1; i >= 0; i--) { searchLeaf(leaves.get(i), weight, collector); } @@ -517,26 +517,6 @@ private boolean canMatchSearchAfter(LeafReaderContext ctx) throws IOException { return true; } - private boolean shouldReverseLeafReaderContexts() { - // Time series based workload by default traverses segments in desc order i.e. latest to the oldest order. - // This is actually beneficial for search queries to start search on latest segments first for time series workload. - // That can slow down ASC order queries on timestamp workload. So to avoid that slowdown, we will reverse leaf - // reader order here. - if (searchContext.indexShard().isTimeSeriesDescSortOptimizationEnabled()) { - // Only reverse order for asc order sort queries - if (searchContext.sort() != null - && searchContext.sort().sort != null - && searchContext.sort().sort.getSort() != null - && searchContext.sort().sort.getSort().length > 0 - && searchContext.sort().sort.getSort()[0].getReverse() == false - && searchContext.sort().sort.getSort()[0].getField() != null - && searchContext.sort().sort.getSort()[0].getField().equals(DataStream.TIMESERIES_FIELDNAME)) { - return true; - } - } - return false; - } - // package-private for testing LeafSlice[] slicesInternal(List leaves, int targetMaxSlice) { LeafSlice[] leafSlices; diff --git a/server/src/main/java/org/opensearch/search/internal/FilteredSearchContext.java b/server/src/main/java/org/opensearch/search/internal/FilteredSearchContext.java index 32de5fc9864ce..151ef97a2a141 100644 --- a/server/src/main/java/org/opensearch/search/internal/FilteredSearchContext.java +++ b/server/src/main/java/org/opensearch/search/internal/FilteredSearchContext.java @@ -569,4 +569,9 @@ public boolean shouldUseConcurrentSearch() { public int getTargetMaxSliceCount() { return in.getTargetMaxSliceCount(); } + + @Override + public boolean shouldUseTimeSeriesDescSortOptimization() { + return in.shouldUseTimeSeriesDescSortOptimization(); + } } diff --git a/server/src/main/java/org/opensearch/search/internal/SearchContext.java b/server/src/main/java/org/opensearch/search/internal/SearchContext.java index 590ce4b077cbc..dce6da897a74b 100644 --- a/server/src/main/java/org/opensearch/search/internal/SearchContext.java +++ b/server/src/main/java/org/opensearch/search/internal/SearchContext.java @@ -487,4 +487,6 @@ public String toString() { public abstract BucketCollectorProcessor bucketCollectorProcessor(); public abstract int getTargetMaxSliceCount(); + + public abstract boolean shouldUseTimeSeriesDescSortOptimization(); } diff --git a/server/src/main/java/org/opensearch/search/query/QueryPhaseSearcherWrapper.java b/server/src/main/java/org/opensearch/search/query/QueryPhaseSearcherWrapper.java index 115f7503631c1..631ace41090d7 100644 --- a/server/src/main/java/org/opensearch/search/query/QueryPhaseSearcherWrapper.java +++ b/server/src/main/java/org/opensearch/search/query/QueryPhaseSearcherWrapper.java @@ -58,9 +58,10 @@ public boolean searchWith( boolean hasTimeout ) throws IOException { if (searchContext.shouldUseConcurrentSearch()) { - LOGGER.info("Using concurrent search over segments (experimental)"); + LOGGER.debug("Using concurrent search over segments (experimental) for request with context id {}", searchContext.id()); return concurrentQueryPhaseSearcher.searchWith(searchContext, searcher, query, collectors, hasFilterCollector, hasTimeout); } else { + LOGGER.debug("Using non-concurrent search over segments for request with context id {}", searchContext.id()); return defaultQueryPhaseSearcher.searchWith(searchContext, searcher, query, collectors, hasFilterCollector, hasTimeout); } } @@ -73,9 +74,13 @@ public boolean searchWith( @Override public AggregationProcessor aggregationProcessor(SearchContext searchContext) { if (searchContext.shouldUseConcurrentSearch()) { - LOGGER.info("Using concurrent search over segments (experimental)"); + LOGGER.debug( + "Using concurrent aggregation processor over segments (experimental) for request with context id {}", + searchContext.id() + ); return concurrentQueryPhaseSearcher.aggregationProcessor(searchContext); } else { + LOGGER.debug("Using non-concurrent aggregation processor over segments for request with context id {}", searchContext.id()); return defaultQueryPhaseSearcher.aggregationProcessor(searchContext); } } diff --git a/server/src/main/java/org/opensearch/search/sort/SortAndFormats.java b/server/src/main/java/org/opensearch/search/sort/SortAndFormats.java index 272b1e9c1dc8d..e65187e558aef 100644 --- a/server/src/main/java/org/opensearch/search/sort/SortAndFormats.java +++ b/server/src/main/java/org/opensearch/search/sort/SortAndFormats.java @@ -32,6 +32,7 @@ package org.opensearch.search.sort; import org.apache.lucene.search.Sort; +import org.opensearch.cluster.metadata.DataStream; import org.opensearch.search.DocValueFormat; /** @@ -52,4 +53,13 @@ public SortAndFormats(Sort sort, DocValueFormat[] formats) { this.formats = formats; } + /** + * @return true: if sort is on timestamp field, false: otherwise + */ + public boolean isSortOnTimeSeriesField() { + return sort.getSort().length > 0 + && sort.getSort()[0].getField() != null + && sort.getSort()[0].getField().equals(DataStream.TIMESERIES_FIELDNAME); + } + } diff --git a/server/src/test/java/org/opensearch/search/DefaultSearchContextTests.java b/server/src/test/java/org/opensearch/search/DefaultSearchContextTests.java index f569fe3b63af0..347011af98c6d 100644 --- a/server/src/test/java/org/opensearch/search/DefaultSearchContextTests.java +++ b/server/src/test/java/org/opensearch/search/DefaultSearchContextTests.java @@ -40,16 +40,21 @@ import org.apache.lucene.search.Query; import org.apache.lucene.search.QueryCachingPolicy; import org.apache.lucene.search.Sort; +import org.apache.lucene.search.SortField; import org.apache.lucene.store.Directory; import org.apache.lucene.tests.index.RandomIndexWriter; import org.opensearch.Version; import org.opensearch.action.OriginalIndices; import org.opensearch.action.search.SearchType; import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.SetOnce; import org.opensearch.common.UUIDs; +import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.BigArrays; +import org.opensearch.common.util.FeatureFlags; import org.opensearch.common.util.MockBigArrays; import org.opensearch.common.util.MockPageCacheRecycler; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; @@ -75,6 +80,7 @@ import org.opensearch.search.rescore.RescoreContext; import org.opensearch.search.slice.SliceBuilder; import org.opensearch.search.sort.SortAndFormats; +import org.opensearch.test.FeatureFlagSetter; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; @@ -547,6 +553,159 @@ protected Engine.Searcher acquireSearcherInternal(String source) { } } + public void testSearchPathEvaluationUsingSortField() throws Exception { + // enable the concurrent set FeatureFlag + FeatureFlagSetter.set(FeatureFlags.CONCURRENT_SEGMENT_SEARCH); + ShardSearchRequest shardSearchRequest = mock(ShardSearchRequest.class); + when(shardSearchRequest.searchType()).thenReturn(SearchType.DEFAULT); + ShardId shardId = new ShardId("index", UUID.randomUUID().toString(), 1); + when(shardSearchRequest.shardId()).thenReturn(shardId); + + ThreadPool threadPool = new TestThreadPool(this.getClass().getName()); + IndexShard indexShard = mock(IndexShard.class); + QueryCachingPolicy queryCachingPolicy = mock(QueryCachingPolicy.class); + when(indexShard.getQueryCachingPolicy()).thenReturn(queryCachingPolicy); + when(indexShard.getThreadPool()).thenReturn(threadPool); + + Settings settings = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 2) + .build(); + + IndexService indexService = mock(IndexService.class); + QueryShardContext queryShardContext = mock(QueryShardContext.class); + when(indexService.newQueryShardContext(eq(shardId.id()), any(), any(), nullable(String.class), anyBoolean())).thenReturn( + queryShardContext + ); + + IndexMetadata indexMetadata = IndexMetadata.builder("index").settings(settings).build(); + IndexSettings indexSettings = new IndexSettings(indexMetadata, Settings.EMPTY); + when(indexService.getIndexSettings()).thenReturn(indexSettings); + + BigArrays bigArrays = new MockBigArrays(new MockPageCacheRecycler(Settings.EMPTY), new NoneCircuitBreakerService()); + + try (Directory dir = newDirectory(); RandomIndexWriter w = new RandomIndexWriter(random(), dir)) { + + final Supplier searcherSupplier = () -> new Engine.SearcherSupplier(Function.identity()) { + @Override + protected void doClose() {} + + @Override + protected Engine.Searcher acquireSearcherInternal(String source) { + try { + IndexReader reader = w.getReader(); + return new Engine.Searcher( + "test", + reader, + IndexSearcher.getDefaultSimilarity(), + IndexSearcher.getDefaultQueryCache(), + IndexSearcher.getDefaultQueryCachingPolicy(), + reader + ); + } catch (IOException exc) { + throw new AssertionError(exc); + } + } + }; + + SearchShardTarget target = new SearchShardTarget("node", shardId, null, OriginalIndices.NONE); + ReaderContext readerContext = new ReaderContext( + newContextId(), + indexService, + indexShard, + searcherSupplier.get(), + randomNonNegativeLong(), + false + ); + + final ClusterService clusterService = mock(ClusterService.class); + final ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + clusterSettings.registerSetting(SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING); + clusterSettings.applySettings( + Settings.builder().put(SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey(), true).build() + ); + when(clusterService.getClusterSettings()).thenReturn(clusterSettings); + DefaultSearchContext context = new DefaultSearchContext( + readerContext, + shardSearchRequest, + target, + null, + bigArrays, + null, + null, + null, + false, + Version.CURRENT, + false, + executor, + null + ); + + // Case1: if sort is on timestamp field, non-concurrent path is used + context.sort( + new SortAndFormats(new Sort(new SortField("@timestamp", SortField.Type.INT)), new DocValueFormat[] { DocValueFormat.RAW }) + ); + context.evaluateRequestShouldUseConcurrentSearch(); + assertFalse(context.shouldUseConcurrentSearch()); + assertThrows(SetOnce.AlreadySetException.class, context::evaluateRequestShouldUseConcurrentSearch); + + // Case2: if sort is on other field, concurrent path is used + context = new DefaultSearchContext( + readerContext, + shardSearchRequest, + target, + clusterService, + bigArrays, + null, + null, + null, + false, + Version.CURRENT, + false, + executor, + null + ); + context.sort( + new SortAndFormats(new Sort(new SortField("test2", SortField.Type.INT)), new DocValueFormat[] { DocValueFormat.RAW }) + ); + context.evaluateRequestShouldUseConcurrentSearch(); + if (executor == null) { + assertFalse(context.shouldUseConcurrentSearch()); + } else { + assertTrue(context.shouldUseConcurrentSearch()); + } + assertThrows(SetOnce.AlreadySetException.class, context::evaluateRequestShouldUseConcurrentSearch); + + // Case 3: With no sort, concurrent path is used + context = new DefaultSearchContext( + readerContext, + shardSearchRequest, + target, + clusterService, + bigArrays, + null, + null, + null, + false, + Version.CURRENT, + false, + executor, + null + ); + context.evaluateRequestShouldUseConcurrentSearch(); + if (executor == null) { + assertFalse(context.shouldUseConcurrentSearch()); + } else { + assertTrue(context.shouldUseConcurrentSearch()); + } + assertThrows(SetOnce.AlreadySetException.class, context::evaluateRequestShouldUseConcurrentSearch); + + // shutdown the threadpool + threadPool.shutdown(); + } + } + private ShardSearchContextId newContextId() { return new ShardSearchContextId(UUIDs.randomBase64UUID(), randomNonNegativeLong()); } diff --git a/test/framework/src/main/java/org/opensearch/test/TestSearchContext.java b/test/framework/src/main/java/org/opensearch/test/TestSearchContext.java index dd4a05b67271c..2fb345f73fb06 100644 --- a/test/framework/src/main/java/org/opensearch/test/TestSearchContext.java +++ b/test/framework/src/main/java/org/opensearch/test/TestSearchContext.java @@ -692,6 +692,15 @@ public int getTargetMaxSliceCount() { return maxSliceCount; } + @Override + public boolean shouldUseTimeSeriesDescSortOptimization() { + return indexShard != null + && indexShard.isTimeSeriesDescSortOptimizationEnabled() + && sort != null + && sort.isSortOnTimeSeriesField() + && sort.sort.getSort()[0].getReverse() == false; + } + /** * Clean the query results by consuming all of it */ From 012c4fa4ec8b719cecd4e163c5fdc0e4f42679d3 Mon Sep 17 00:00:00 2001 From: Marc Handalian Date: Mon, 28 Aug 2023 16:02:18 -0700 Subject: [PATCH 06/37] [Segment Replication] Fix bug where replica shows stale doc count during engine reset. (#9495) * Fix bug where replica shows stale doc count during engine reset. This change fixes an issue where replica shards can temporarily return stale results while converting to a RO engine during an engine reset. This is possible because NRTReplicationEngine did not previously implement flush and the freshest data is only active on the reader. Fixed by implementing flush and also honoring acquireLatestCommit's flushFirst parameter. Signed-off-by: Marc Handalian * Add changelog entry. Signed-off-by: Marc Handalian * Add unit test for search during engine reset. Signed-off-by: Marc Handalian * Remove useless test. Signed-off-by: Marc Handalian --------- Signed-off-by: Marc Handalian --- CHANGELOG.md | 1 + .../replication/SegmentReplicationIT.java | 5 +-- .../index/engine/NRTReplicationEngine.java | 32 +++++++++++++++++-- .../engine/NRTReplicationEngineTests.java | 10 ++++-- .../index/shard/RemoteIndexShardTests.java | 10 +++--- .../SegmentReplicationIndexShardTests.java | 31 ++++++++++++++++++ 6 files changed, 76 insertions(+), 13 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index db57363ab605f..8dee8390cbc5e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -174,6 +174,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Handle null partSize in OnDemandBlockSnapshotIndexInput ([#9291](https://github.com/opensearch-project/OpenSearch/issues/9291)) - Fix condition to remove index create block ([#9437](https://github.com/opensearch-project/OpenSearch/pull/9437)) - Add support to clear archived index setting ([#9019](https://github.com/opensearch-project/OpenSearch/pull/9019)) +- [Segment Replication] Fixed bug where replica shard temporarily serves stale data during an engine reset ([#9495](https://github.com/opensearch-project/OpenSearch/pull/9495)) ### Security diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java index 69cdd80bb5085..5855ed7470559 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java @@ -143,8 +143,9 @@ public void testPrimaryStopped_ReplicaPromoted() throws Exception { final ShardRouting replicaShardRouting = getShardRoutingForNodeName(replica); assertNotNull(replicaShardRouting); assertTrue(replicaShardRouting + " should be promoted as a primary", replicaShardRouting.primary()); - refresh(INDEX_NAME); - assertHitCount(client(replica).prepareSearch(INDEX_NAME).setSize(0).setPreference("_only_local").get(), 2); + final SearchResponse response = client(replica).prepareSearch(INDEX_NAME).setSize(0).setPreference("_only_local").get(); + // new primary should have at least the doc count from the first set of segments. + assertTrue(response.getHits().getTotalHits().value >= 1); // assert we can index into the new primary. client().prepareIndex(INDEX_NAME).setId("3").setSource("bar", "baz").setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE).get(); diff --git a/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngine.java b/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngine.java index 48556cc6b9709..b529dfbe13bf4 100644 --- a/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngine.java +++ b/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngine.java @@ -39,6 +39,8 @@ import java.util.Map; import java.util.Objects; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiFunction; import static org.opensearch.index.seqno.SequenceNumbers.MAX_SEQ_NO; @@ -57,6 +59,7 @@ public class NRTReplicationEngine extends Engine { private final CompletionStatsCache completionStatsCache; private final LocalCheckpointTracker localCheckpointTracker; private final WriteOnlyTranslogManager translogManager; + private final Lock flushLock = new ReentrantLock(); protected final ReplicaFileTracker replicaFileTracker; private volatile long lastReceivedPrimaryGen = SequenceNumbers.NO_OPS_PERFORMED; @@ -156,7 +159,7 @@ public synchronized void updateSegments(final SegmentInfos infos) throws IOExcep // a lower gen from a newly elected primary shard that is behind this shard's last commit gen. // In that case we still commit into the next local generation. if (incomingGeneration != this.lastReceivedPrimaryGen) { - commitSegmentInfos(); + flush(false, true); translogManager.getDeletionPolicy().setLocalCheckpointOfSafeCommit(maxSeqNo); translogManager.rollTranslogGeneration(); } @@ -184,7 +187,7 @@ private void commitSegmentInfos(SegmentInfos infos) throws IOException { translogManager.syncTranslog(); } - protected void commitSegmentInfos() throws IOException { + private void commitSegmentInfos() throws IOException { commitSegmentInfos(getLatestSegmentInfos()); } @@ -351,7 +354,27 @@ public boolean shouldPeriodicallyFlush() { } @Override - public void flush(boolean force, boolean waitIfOngoing) throws EngineException {} + public void flush(boolean force, boolean waitIfOngoing) throws EngineException { + ensureOpen(); + // readLock is held here to wait/block any concurrent close that acquires the writeLock. + try (final ReleasableLock lock = readLock.acquire()) { + ensureOpen(); + if (flushLock.tryLock() == false) { + if (waitIfOngoing == false) { + return; + } + flushLock.lock(); + } + // we are now locked. + try { + commitSegmentInfos(); + } catch (IOException e) { + throw new FlushFailedEngineException(shardId, e); + } finally { + flushLock.unlock(); + } + } + } @Override public void forceMerge( @@ -365,6 +388,9 @@ public void forceMerge( @Override public GatedCloseable acquireLastIndexCommit(boolean flushFirst) throws EngineException { + if (flushFirst) { + flush(false, true); + } try { final IndexCommit indexCommit = Lucene.getIndexCommit(lastCommittedSegmentInfos, store.directory()); return new GatedCloseable<>(indexCommit, () -> {}); diff --git a/server/src/test/java/org/opensearch/index/engine/NRTReplicationEngineTests.java b/server/src/test/java/org/opensearch/index/engine/NRTReplicationEngineTests.java index cb93d3a8db20e..22eb5195af507 100644 --- a/server/src/test/java/org/opensearch/index/engine/NRTReplicationEngineTests.java +++ b/server/src/test/java/org/opensearch/index/engine/NRTReplicationEngineTests.java @@ -152,7 +152,7 @@ public void testUpdateSegments_replicaReceivesSISWithLowerGen() throws IOExcepti assertEquals(2, nrtEngine.getLastCommittedSegmentInfos().getGeneration()); // commit the infos to push us to segments_3. - nrtEngine.commitSegmentInfos(); + nrtEngine.flush(); assertEquals(3, nrtEngine.getLastCommittedSegmentInfos().getGeneration()); assertEquals(3, nrtEngine.getLatestSegmentInfos().getGeneration()); @@ -283,7 +283,7 @@ public void testTrimTranslogOps() throws Exception { } } - public void testCommitSegmentInfos() throws Exception { + public void testFlush() throws Exception { // This test asserts that NRTReplication#commitSegmentInfos creates a new commit point with the latest checkpoints // stored in user data. final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); @@ -304,7 +304,7 @@ public void testCommitSegmentInfos() throws Exception { LocalCheckpointTracker localCheckpointTracker = nrtEngine.getLocalCheckpointTracker(); final long maxSeqNo = localCheckpointTracker.getMaxSeqNo(); final long processedCheckpoint = localCheckpointTracker.getProcessedCheckpoint(); - nrtEngine.commitSegmentInfos(); + nrtEngine.flush(); // ensure getLatestSegmentInfos returns an updated infos ref with correct userdata. final SegmentInfos latestSegmentInfos = nrtEngine.getLatestSegmentInfos(); @@ -322,6 +322,10 @@ public void testCommitSegmentInfos() throws Exception { userData = committedInfos.getUserData(); assertEquals(processedCheckpoint, Long.parseLong(userData.get(LOCAL_CHECKPOINT_KEY))); assertEquals(maxSeqNo, Long.parseLong(userData.get(MAX_SEQ_NO))); + + try (final GatedCloseable indexCommit = nrtEngine.acquireLastIndexCommit(true)) { + assertEquals(committedInfos.getGeneration() + 1, indexCommit.get().getGeneration()); + } } } diff --git a/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java index 8e27c9ff9ae1a..ead9c1c22c931 100644 --- a/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java @@ -236,7 +236,7 @@ public void testReplicaCommitsInfosBytesOnRecovery() throws Exception { MatcherAssert.assertThat( "Replica commits infos bytes referencing latest refresh point", latestReplicaCommit.files(true), - containsInAnyOrder("_0.cfe", "_0.si", "_0.cfs", "segments_5") + containsInAnyOrder("_0.cfe", "_0.si", "_0.cfs", "segments_6") ); MatcherAssert.assertThat( "Segments are referenced in memory", @@ -294,20 +294,20 @@ public void testRepicaCleansUpOldCommitsWhenReceivingNew() throws Exception { replicateSegments(primary, shards.getReplicas()); assertDocCount(primary, 1); assertDocCount(replica, 1); - assertEquals("segments_4", replica.store().readLastCommittedSegmentsInfo().getSegmentsFileName()); - assertSingleSegmentFile(replica, "segments_4"); + assertEquals("segments_5", replica.store().readLastCommittedSegmentsInfo().getSegmentsFileName()); + assertSingleSegmentFile(replica, "segments_5"); shards.indexDocs(1); primary.refresh("test"); replicateSegments(primary, shards.getReplicas()); assertDocCount(replica, 2); - assertSingleSegmentFile(replica, "segments_4"); + assertSingleSegmentFile(replica, "segments_5"); shards.indexDocs(1); flushShard(primary); replicateSegments(primary, shards.getReplicas()); assertDocCount(replica, 3); - assertSingleSegmentFile(replica, "segments_5"); + assertSingleSegmentFile(replica, "segments_6"); final Store.RecoveryDiff diff = Store.segmentReplicationDiff(primary.getSegmentMetadataMap(), replica.getSegmentMetadataMap()); assertTrue(diff.missing.isEmpty()); diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java index 807b4a9cd7482..e8220830063ee 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java @@ -34,6 +34,7 @@ import org.opensearch.index.engine.InternalEngineFactory; import org.opensearch.index.engine.NRTReplicationEngine; import org.opensearch.index.engine.NRTReplicationEngineFactory; +import org.opensearch.index.engine.ReadOnlyEngine; import org.opensearch.index.mapper.MapperService; import org.opensearch.index.replication.OpenSearchIndexLevelReplicationTestCase; import org.opensearch.index.replication.TestReplicationSource; @@ -71,6 +72,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import static org.opensearch.index.engine.EngineTestCase.assertAtMostOneLuceneDocumentPerSequenceNumber; @@ -773,6 +775,35 @@ public void testNoDuplicateSeqNo() throws Exception { } } + public void testQueryDuringEngineResetShowsDocs() throws Exception { + final NRTReplicationEngineFactory engineFactory = new NRTReplicationEngineFactory(); + final NRTReplicationEngineFactory spy = spy(engineFactory); + try (ReplicationGroup shards = createGroup(1, settings, indexMapping, spy, createTempDir())) { + final IndexShard primaryShard = shards.getPrimary(); + final IndexShard replicaShard = shards.getReplicas().get(0); + shards.startAll(); + shards.indexDocs(10); + shards.refresh("test"); + replicateSegments(primaryShard, shards.getReplicas()); + shards.assertAllEqual(10); + + final AtomicReference failed = new AtomicReference<>(); + doAnswer(ans -> { + try { + final Engine engineOrNull = replicaShard.getEngineOrNull(); + assertNotNull(engineOrNull); + assertTrue(engineOrNull instanceof ReadOnlyEngine); + shards.assertAllEqual(10); + } catch (Throwable e) { + failed.set(e); + } + return ans.callRealMethod(); + }).when(spy).newReadWriteEngine(any()); + shards.promoteReplicaToPrimary(replicaShard).get(); + assertNull("Expected correct doc count during engine reset", failed.get()); + } + } + /** * Assert persisted and searchable doc counts. This method should not be used while docs are concurrently indexed because * it asserts point in time seqNos are relative to the doc counts. From 61d4d43c8a9af909dbdd44058b845e91b8a98f4b Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Mon, 28 Aug 2023 17:36:06 -0700 Subject: [PATCH 07/37] [Segment Replication] Add ClusterState utility to identify SEGMENT replication (#9593) * [Segment Replication] Add ClusterState utility to identify SEGMENT replication Signed-off-by: Suraj Singh * Address review comment Signed-off-by: Suraj Singh * Address review comments Signed-off-by: Suraj Singh --------- Signed-off-by: Suraj Singh --- .../action/get/TransportGetAction.java | 14 +------ .../org/opensearch/cluster/ClusterState.java | 16 ++++++++ .../opensearch/cluster/ClusterStateTests.java | 37 ++++++++++++++++++- 3 files changed, 53 insertions(+), 14 deletions(-) diff --git a/server/src/main/java/org/opensearch/action/get/TransportGetAction.java b/server/src/main/java/org/opensearch/action/get/TransportGetAction.java index 583815b91ae68..0c444732fb12b 100644 --- a/server/src/main/java/org/opensearch/action/get/TransportGetAction.java +++ b/server/src/main/java/org/opensearch/action/get/TransportGetAction.java @@ -36,7 +36,6 @@ import org.opensearch.action.support.ActionFilters; import org.opensearch.action.support.single.shard.TransportSingleShardAction; import org.opensearch.cluster.ClusterState; -import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.IndexNameExpressionResolver; import org.opensearch.cluster.routing.Preference; import org.opensearch.cluster.routing.ShardIterator; @@ -49,12 +48,10 @@ import org.opensearch.index.get.GetResult; import org.opensearch.index.shard.IndexShard; import org.opensearch.indices.IndicesService; -import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; import java.io.IOException; -import java.util.Optional; /** * Performs the get operation. @@ -92,20 +89,11 @@ protected boolean resolveIndex(GetRequest request) { return true; } - static boolean isSegmentReplicationEnabled(ClusterState state, String indexName) { - return Optional.ofNullable(state.getMetadata().index(indexName)) - .map( - indexMetadata -> ReplicationType.parseString(indexMetadata.getSettings().get(IndexMetadata.SETTING_REPLICATION_TYPE)) - .equals(ReplicationType.SEGMENT) - ) - .orElse(false); - } - /** * Returns true if GET request should be routed to primary shards, else false. */ protected static boolean shouldForcePrimaryRouting(ClusterState state, boolean realtime, String preference, String indexName) { - return isSegmentReplicationEnabled(state, indexName) && realtime && preference == null; + return state.isSegmentReplicationEnabled(indexName) && realtime && preference == null; } @Override diff --git a/server/src/main/java/org/opensearch/cluster/ClusterState.java b/server/src/main/java/org/opensearch/cluster/ClusterState.java index 1b87a60c2ccf5..2fd58d3db4975 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterState.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterState.java @@ -61,6 +61,7 @@ import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.discovery.Discovery; +import org.opensearch.indices.replication.common.ReplicationType; import java.io.IOException; import java.util.Collections; @@ -409,6 +410,21 @@ public boolean supersedes(ClusterState other) { } + /** + * Utility to identify whether input index belongs to SEGMENT replication in established cluster state. + * + * @param indexName Index name + * @return true if index belong SEGMENT replication, false otherwise + */ + public boolean isSegmentReplicationEnabled(String indexName) { + return Optional.ofNullable(this.getMetadata().index(indexName)) + .map( + indexMetadata -> ReplicationType.parseString(indexMetadata.getSettings().get(IndexMetadata.SETTING_REPLICATION_TYPE)) + .equals(ReplicationType.SEGMENT) + ) + .orElse(false); + } + /** * Metrics for cluster state. * diff --git a/server/src/test/java/org/opensearch/cluster/ClusterStateTests.java b/server/src/test/java/org/opensearch/cluster/ClusterStateTests.java index 63fe65d70d020..c4fb3271ae3ce 100644 --- a/server/src/test/java/org/opensearch/cluster/ClusterStateTests.java +++ b/server/src/test/java/org/opensearch/cluster/ClusterStateTests.java @@ -57,6 +57,7 @@ import org.opensearch.core.rest.RestStatus; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.test.TestCustomMetadata; @@ -73,6 +74,7 @@ import static java.util.Collections.emptyMap; import static java.util.Collections.emptySet; import static java.util.Collections.singletonMap; +import static org.opensearch.cluster.ClusterName.CLUSTER_NAME_SETTING; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_VERSION_CREATED; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; @@ -84,7 +86,7 @@ public void testSupersedes() { final DiscoveryNode node1 = new DiscoveryNode("node1", buildNewFakeTransportAddress(), emptyMap(), emptySet(), version); final DiscoveryNode node2 = new DiscoveryNode("node2", buildNewFakeTransportAddress(), emptyMap(), emptySet(), version); final DiscoveryNodes nodes = DiscoveryNodes.builder().add(node1).add(node2).build(); - ClusterName name = ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY); + ClusterName name = CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY); ClusterState noClusterManager1 = ClusterState.builder(name).version(randomInt(5)).nodes(nodes).build(); ClusterState noClusterManager2 = ClusterState.builder(name).version(randomInt(5)).nodes(nodes).build(); ClusterState withClusterManager1a = ClusterState.builder(name) @@ -115,6 +117,39 @@ public void testSupersedes() { ); } + public void testIsSegmentReplicationEnabled() { + final String indexName = "test"; + ClusterState clusterState = ClusterState.builder(CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).build(); + Settings.Builder builder = settings(Version.CURRENT).put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT); + IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(indexName) + .settings(builder) + .numberOfShards(1) + .numberOfReplicas(1); + Metadata.Builder metadataBuilder = Metadata.builder().put(indexMetadataBuilder); + RoutingTable.Builder routingTableBuilder = RoutingTable.builder().addAsNew(indexMetadataBuilder.build()); + clusterState = ClusterState.builder(clusterState) + .metadata(metadataBuilder.build()) + .routingTable(routingTableBuilder.build()) + .build(); + assertTrue(clusterState.isSegmentReplicationEnabled(indexName)); + } + + public void testIsSegmentReplicationDisabled() { + final String indexName = "test"; + ClusterState clusterState = ClusterState.builder(CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).build(); + IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(indexName) + .settings(settings(Version.CURRENT)) + .numberOfShards(1) + .numberOfReplicas(1); + Metadata.Builder metadataBuilder = Metadata.builder().put(indexMetadataBuilder); + RoutingTable.Builder routingTableBuilder = RoutingTable.builder().addAsNew(indexMetadataBuilder.build()); + clusterState = ClusterState.builder(clusterState) + .metadata(metadataBuilder.build()) + .routingTable(routingTableBuilder.build()) + .build(); + assertFalse(clusterState.isSegmentReplicationEnabled(indexName)); + } + public void testBuilderRejectsNullCustom() { final ClusterState.Builder builder = ClusterState.builder(ClusterName.DEFAULT); final String key = randomAlphaOfLength(10); From bb7d23ca3c31140b03899d83648948dbe7229cf2 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 28 Aug 2023 20:58:31 -0400 Subject: [PATCH 08/37] Bump io.grpc:grpc-api from 1.57.1 to 1.57.2 in /plugins/repository-gcs (#9578) * Bump io.grpc:grpc-api from 1.57.1 to 1.57.2 in /plugins/repository-gcs Bumps [io.grpc:grpc-api](https://github.com/grpc/grpc-java) from 1.57.1 to 1.57.2. - [Release notes](https://github.com/grpc/grpc-java/releases) - [Commits](https://github.com/grpc/grpc-java/compare/v1.57.1...v1.57.2) --- updated-dependencies: - dependency-name: io.grpc:grpc-api dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] * Updating SHAs Signed-off-by: dependabot[bot] * Update changelog Signed-off-by: dependabot[bot] --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] --- CHANGELOG.md | 3 ++- plugins/repository-gcs/build.gradle | 2 +- plugins/repository-gcs/licenses/grpc-api-1.57.1.jar.sha1 | 1 - plugins/repository-gcs/licenses/grpc-api-1.57.2.jar.sha1 | 1 + 4 files changed, 4 insertions(+), 3 deletions(-) delete mode 100644 plugins/repository-gcs/licenses/grpc-api-1.57.1.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/grpc-api-1.57.2.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index 8dee8390cbc5e..6c26302cb2272 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -122,6 +122,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `actions/setup-java` from 2 to 3 ([#9457](https://github.com/opensearch-project/OpenSearch/pull/9457)) - Bump `com.google.api:gax` from 2.27.0 to 2.32.0 ([#9300](https://github.com/opensearch-project/OpenSearch/pull/9300)) - Bump `netty` from 4.1.96.Final to 4.1.97.Final ([#9553](https://github.com/opensearch-project/OpenSearch/pull/9553)) +- Bump `io.grpc:grpc-api` from 1.57.1 to 1.57.2 ([#9578](https://github.com/opensearch-project/OpenSearch/pull/9578)) ### Changed - Default to mmapfs within hybridfs ([#8508](https://github.com/opensearch-project/OpenSearch/pull/8508)) @@ -179,4 +180,4 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ### Security [Unreleased 3.0]: https://github.com/opensearch-project/OpenSearch/compare/2.x...HEAD -[Unreleased 2.x]: https://github.com/opensearch-project/OpenSearch/compare/2.10...2.x +[Unreleased 2.x]: https://github.com/opensearch-project/OpenSearch/compare/2.10...2.x \ No newline at end of file diff --git a/plugins/repository-gcs/build.gradle b/plugins/repository-gcs/build.gradle index 6d4e56cb81a11..e7ee980114e5e 100644 --- a/plugins/repository-gcs/build.gradle +++ b/plugins/repository-gcs/build.gradle @@ -86,7 +86,7 @@ dependencies { api "org.apache.logging.log4j:log4j-1.2-api:${versions.log4j}" api "commons-codec:commons-codec:${versions.commonscodec}" api 'org.threeten:threetenbp:1.4.4' - api 'io.grpc:grpc-api:1.57.1' + api 'io.grpc:grpc-api:1.57.2' api 'io.opencensus:opencensus-api:0.31.1' api 'io.opencensus:opencensus-contrib-http-util:0.31.1' diff --git a/plugins/repository-gcs/licenses/grpc-api-1.57.1.jar.sha1 b/plugins/repository-gcs/licenses/grpc-api-1.57.1.jar.sha1 deleted file mode 100644 index c52d208334070..0000000000000 --- a/plugins/repository-gcs/licenses/grpc-api-1.57.1.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -2a7f06d11b65839cf222159b4e947a22eddc59e6 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/grpc-api-1.57.2.jar.sha1 b/plugins/repository-gcs/licenses/grpc-api-1.57.2.jar.sha1 new file mode 100644 index 0000000000000..8b320fdd2f9cc --- /dev/null +++ b/plugins/repository-gcs/licenses/grpc-api-1.57.2.jar.sha1 @@ -0,0 +1 @@ +c71a006b81ddae7bc4b7cb1d2da78c1b173761f4 \ No newline at end of file From 60787b869cb800eab53417e87a6399ed680de060 Mon Sep 17 00:00:00 2001 From: Austin Lee Date: Tue, 29 Aug 2023 14:39:37 -0700 Subject: [PATCH 09/37] Add SearchExtBuilders to SearchResponse (#9379) * Add SearchExtBuilders to SearchResponse. [Issue #9328](https://github.com/opensearch-project/OpenSearch/issues/9328) Signed-off-by: Austin Lee * Keep SearchResponse immutable, add a constructor to take a List of SearchExtBuilders. Signed-off-by: Austin Lee * Fix spotlessJavaCheck findings. Signed-off-by: Austin Lee * Move SearchExtBuilders into SearchResponseSections, fix indenting in SearchRequest. Signed-off-by: Austin Lee * Updated changelog (mixed minor formatting issues), added version checks on serialization/deserialization, added a Builder for making copies of SearchResponse easier. Signed-off-by: Austin Lee * Add GenericSearchExtBuilder as a catch-all for SearchExtBuilders not registered in xcontent registry. Signed-off-by: Austin Lee * Simplify GenericSearchExtBuilder using a single Object member. Signed-off-by: Austin Lee * Address additional review comments. Signed-off-by: Austin Lee * Add Javadocs. Signed-off-by: Austin Lee --------- Signed-off-by: Austin Lee --- CHANGELOG.md | 11 +- .../action/search/SearchResponse.java | 36 +- .../action/search/SearchResponseSections.java | 33 ++ .../search/GenericSearchExtBuilder.java | 165 +++++++ .../internal/InternalSearchResponse.java | 33 +- .../action/search/SearchResponseTests.java | 209 ++++++++- .../search/GenericSearchExtBuilderTests.java | 422 ++++++++++++++++++ 7 files changed, 893 insertions(+), 16 deletions(-) create mode 100644 server/src/main/java/org/opensearch/search/GenericSearchExtBuilder.java create mode 100644 server/src/test/java/org/opensearch/search/GenericSearchExtBuilderTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 6c26302cb2272..eabc17f81917f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -78,17 +78,18 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ## [Unreleased 2.x] ### Added - Add server version as REST response header [#6583](https://github.com/opensearch-project/OpenSearch/issues/6583) -- Start replication checkpointTimers on primary before segments upload to remote store. ([#8221]()https://github.com/opensearch-project/OpenSearch/pull/8221) -- [distribution/archives] [Linux] [x64] Provide the variant of the distributions bundled with JRE ([#8195]()https://github.com/opensearch-project/OpenSearch/pull/8195) +- Start replication checkpointTimers on primary before segments upload to remote store. ([#8221](https://github.com/opensearch-project/OpenSearch/pull/8221)) +- [distribution/archives] [Linux] [x64] Provide the variant of the distributions bundled with JRE ([#8195](https://github.com/opensearch-project/OpenSearch/pull/8195)) - Add configuration for file cache size to max remote data ratio to prevent oversubscription of file cache ([#8606](https://github.com/opensearch-project/OpenSearch/pull/8606)) -- Disallow compression level to be set for default and best_compression index codecs ([#8737]()https://github.com/opensearch-project/OpenSearch/pull/8737) +- Disallow compression level to be set for default and best_compression index codecs ([#8737](https://github.com/opensearch-project/OpenSearch/pull/8737)) - Prioritize replica shard movement during shard relocation ([#8875](https://github.com/opensearch-project/OpenSearch/pull/8875)) -- Introducing Default and Best Compression codecs as their algorithm name ([#9123]()https://github.com/opensearch-project/OpenSearch/pull/9123) -- Make SearchTemplateRequest implement IndicesRequest.Replaceable ([#9122]()https://github.com/opensearch-project/OpenSearch/pull/9122) +- Introducing Default and Best Compression codecs as their algorithm name ([#9123](https://github.com/opensearch-project/OpenSearch/pull/9123)) +- Make SearchTemplateRequest implement IndicesRequest.Replaceable ([#9122](https://github.com/opensearch-project/OpenSearch/pull/9122)) - [BWC and API enforcement] Define the initial set of annotations, their meaning and relations between them ([#9223](https://github.com/opensearch-project/OpenSearch/pull/9223)) - [Segment Replication] Support realtime reads for GET requests ([#9212](https://github.com/opensearch-project/OpenSearch/pull/9212)) - [Feature] Expose term frequency in Painless script score context ([#9081](https://github.com/opensearch-project/OpenSearch/pull/9081)) - Add support for reading partial files to HDFS repository ([#9513](https://github.com/opensearch-project/OpenSearch/issues/9513)) +- Add support for extensions to search responses using SearchExtBuilder ([#9379](https://github.com/opensearch-project/OpenSearch/pull/9379)) ### Dependencies - Bump `org.apache.logging.log4j:log4j-core` from 2.17.1 to 2.20.0 ([#8307](https://github.com/opensearch-project/OpenSearch/pull/8307)) diff --git a/server/src/main/java/org/opensearch/action/search/SearchResponse.java b/server/src/main/java/org/opensearch/action/search/SearchResponse.java index cfdbe5647df5a..a546311a1f668 100644 --- a/server/src/main/java/org/opensearch/action/search/SearchResponse.java +++ b/server/src/main/java/org/opensearch/action/search/SearchResponse.java @@ -46,9 +46,12 @@ import org.opensearch.core.xcontent.MediaTypeRegistry; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParseException; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.core.xcontent.XContentParser.Token; import org.opensearch.rest.action.RestActions; +import org.opensearch.search.GenericSearchExtBuilder; +import org.opensearch.search.SearchExtBuilder; import org.opensearch.search.SearchHit; import org.opensearch.search.SearchHits; import org.opensearch.search.aggregations.Aggregations; @@ -65,6 +68,7 @@ import java.util.Objects; import java.util.function.Supplier; +import static org.opensearch.action.search.SearchResponseSections.EXT_FIELD; import static org.opensearch.core.xcontent.XContentParserUtils.ensureExpectedToken; /** @@ -312,6 +316,7 @@ public XContentBuilder innerToXContent(XContentBuilder builder, Params params) t ); clusters.toXContent(builder, params); internalResponse.toXContent(builder, params); + return builder; } @@ -339,6 +344,7 @@ public static SearchResponse innerFromXContent(XContentParser parser) throws IOE String searchContextId = null; List failures = new ArrayList<>(); Clusters clusters = Clusters.EMPTY; + List extBuilders = new ArrayList<>(); for (Token token = parser.nextToken(); token != Token.END_OBJECT; token = parser.nextToken()) { if (token == Token.FIELD_NAME) { currentFieldName = parser.currentName(); @@ -417,6 +423,33 @@ public static SearchResponse innerFromXContent(XContentParser parser) throws IOE } } clusters = new Clusters(total, successful, skipped); + } else if (EXT_FIELD.match(currentFieldName, parser.getDeprecationHandler())) { + String extSectionName = null; + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + extSectionName = parser.currentName(); + } else { + SearchExtBuilder searchExtBuilder; + try { + searchExtBuilder = parser.namedObject(SearchExtBuilder.class, extSectionName, null); + if (!searchExtBuilder.getWriteableName().equals(extSectionName)) { + throw new IllegalStateException( + "The parsed [" + + searchExtBuilder.getClass().getName() + + "] object has a " + + "different writeable name compared to the name of the section that it was parsed from: found [" + + searchExtBuilder.getWriteableName() + + "] expected [" + + extSectionName + + "]" + ); + } + } catch (XContentParseException e) { + searchExtBuilder = GenericSearchExtBuilder.fromXContent(parser); + } + extBuilders.add(searchExtBuilder); + } + } } else { parser.skipChildren(); } @@ -429,7 +462,8 @@ public static SearchResponse innerFromXContent(XContentParser parser) throws IOE timedOut, terminatedEarly, profile, - numReducePhases + numReducePhases, + extBuilders ); return new SearchResponse( searchResponseSections, diff --git a/server/src/main/java/org/opensearch/action/search/SearchResponseSections.java b/server/src/main/java/org/opensearch/action/search/SearchResponseSections.java index 214bc0448b90c..2e447abd125c5 100644 --- a/server/src/main/java/org/opensearch/action/search/SearchResponseSections.java +++ b/server/src/main/java/org/opensearch/action/search/SearchResponseSections.java @@ -32,9 +32,11 @@ package org.opensearch.action.search; +import org.opensearch.core.ParseField; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.search.SearchExtBuilder; import org.opensearch.search.SearchHits; import org.opensearch.search.aggregations.Aggregations; import org.opensearch.search.profile.ProfileShardResult; @@ -42,8 +44,11 @@ import org.opensearch.search.suggest.Suggest; import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; +import java.util.List; import java.util.Map; +import java.util.Objects; /** * Base class that holds the various sections which a search response is @@ -57,6 +62,8 @@ */ public class SearchResponseSections implements ToXContentFragment { + public static final ParseField EXT_FIELD = new ParseField("ext"); + protected final SearchHits hits; protected final Aggregations aggregations; protected final Suggest suggest; @@ -64,6 +71,7 @@ public class SearchResponseSections implements ToXContentFragment { protected final boolean timedOut; protected final Boolean terminatedEarly; protected final int numReducePhases; + protected final List searchExtBuilders = new ArrayList<>(); public SearchResponseSections( SearchHits hits, @@ -73,6 +81,19 @@ public SearchResponseSections( Boolean terminatedEarly, SearchProfileShardResults profileResults, int numReducePhases + ) { + this(hits, aggregations, suggest, timedOut, terminatedEarly, profileResults, numReducePhases, Collections.emptyList()); + } + + public SearchResponseSections( + SearchHits hits, + Aggregations aggregations, + Suggest suggest, + boolean timedOut, + Boolean terminatedEarly, + SearchProfileShardResults profileResults, + int numReducePhases, + List searchExtBuilders ) { this.hits = hits; this.aggregations = aggregations; @@ -81,6 +102,7 @@ public SearchResponseSections( this.timedOut = timedOut; this.terminatedEarly = terminatedEarly; this.numReducePhases = numReducePhases; + this.searchExtBuilders.addAll(Objects.requireNonNull(searchExtBuilders, "searchExtBuilders must not be null")); } public final boolean timedOut() { @@ -135,9 +157,20 @@ public final XContentBuilder toXContent(XContentBuilder builder, Params params) if (profileResults != null) { profileResults.toXContent(builder, params); } + if (!searchExtBuilders.isEmpty()) { + builder.startObject(EXT_FIELD.getPreferredName()); + for (SearchExtBuilder searchExtBuilder : searchExtBuilders) { + searchExtBuilder.toXContent(builder, params); + } + builder.endObject(); + } return builder; } + public List getSearchExtBuilders() { + return Collections.unmodifiableList(this.searchExtBuilders); + } + protected void writeTo(StreamOutput out) throws IOException { throw new UnsupportedOperationException(); } diff --git a/server/src/main/java/org/opensearch/search/GenericSearchExtBuilder.java b/server/src/main/java/org/opensearch/search/GenericSearchExtBuilder.java new file mode 100644 index 0000000000000..35e68f78774e3 --- /dev/null +++ b/server/src/main/java/org/opensearch/search/GenericSearchExtBuilder.java @@ -0,0 +1,165 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ +package org.opensearch.search; + +import org.opensearch.core.ParseField; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParseException; +import org.opensearch.core.xcontent.XContentParser; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * This is a catch-all SearchExtBuilder implementation that is used when an appropriate SearchExtBuilder + * is not found during SearchResponse's fromXContent operation. + */ +public final class GenericSearchExtBuilder extends SearchExtBuilder { + + public final static ParseField EXT_BUILDER_NAME = new ParseField("generic_ext"); + + private final Object genericObj; + private final ValueType valueType; + + enum ValueType { + SIMPLE(0), + MAP(1), + LIST(2); + + private final int value; + + ValueType(int value) { + this.value = value; + } + + public int getValue() { + return value; + } + + static ValueType fromInt(int value) { + switch (value) { + case 0: + return SIMPLE; + case 1: + return MAP; + case 2: + return LIST; + default: + throw new IllegalArgumentException("Unsupported value: " + value); + } + } + } + + public GenericSearchExtBuilder(Object genericObj, ValueType valueType) { + this.genericObj = genericObj; + this.valueType = valueType; + } + + public GenericSearchExtBuilder(StreamInput in) throws IOException { + valueType = ValueType.fromInt(in.readInt()); + switch (valueType) { + case SIMPLE: + genericObj = in.readGenericValue(); + break; + case MAP: + genericObj = in.readMap(); + break; + case LIST: + genericObj = in.readList(r -> r.readGenericValue()); + break; + default: + throw new IllegalStateException("Unable to construct GenericSearchExtBuilder from incoming stream."); + } + } + + public static GenericSearchExtBuilder fromXContent(XContentParser parser) throws IOException { + // Look at the parser's next token. + // If it's START_OBJECT, parse as map, if it's START_ARRAY, parse as list, else + // parse as simpleVal + XContentParser.Token token = parser.currentToken(); + ValueType valueType; + Object genericObj; + if (token == XContentParser.Token.START_OBJECT) { + genericObj = parser.map(); + valueType = ValueType.MAP; + } else if (token == XContentParser.Token.START_ARRAY) { + genericObj = parser.list(); + valueType = ValueType.LIST; + } else if (token.isValue()) { + genericObj = parser.objectText(); + valueType = ValueType.SIMPLE; + } else { + throw new XContentParseException("Unknown token: " + token); + } + + return new GenericSearchExtBuilder(genericObj, valueType); + } + + @Override + public String getWriteableName() { + return EXT_BUILDER_NAME.getPreferredName(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeInt(valueType.getValue()); + switch (valueType) { + case SIMPLE: + out.writeGenericValue(genericObj); + break; + case MAP: + out.writeMap((Map) genericObj); + break; + case LIST: + out.writeCollection((List) genericObj, StreamOutput::writeGenericValue); + break; + default: + throw new IllegalStateException("Unknown valueType: " + valueType); + } + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + switch (valueType) { + case SIMPLE: + return builder.field(EXT_BUILDER_NAME.getPreferredName(), genericObj); + case MAP: + return builder.field(EXT_BUILDER_NAME.getPreferredName(), (Map) genericObj); + case LIST: + return builder.field(EXT_BUILDER_NAME.getPreferredName(), (List) genericObj); + default: + return null; + } + } + + // We need this for the equals method. + Object getValue() { + return genericObj; + } + + @Override + public int hashCode() { + return Objects.hash(this.valueType, this.genericObj); + } + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return false; + } + if (!(obj instanceof GenericSearchExtBuilder)) { + return false; + } + return Objects.equals(getValue(), ((GenericSearchExtBuilder) obj).getValue()) + && Objects.equals(valueType, ((GenericSearchExtBuilder) obj).valueType); + } +} diff --git a/server/src/main/java/org/opensearch/search/internal/InternalSearchResponse.java b/server/src/main/java/org/opensearch/search/internal/InternalSearchResponse.java index 1561d18f3040a..8e3979045f857 100644 --- a/server/src/main/java/org/opensearch/search/internal/InternalSearchResponse.java +++ b/server/src/main/java/org/opensearch/search/internal/InternalSearchResponse.java @@ -32,17 +32,21 @@ package org.opensearch.search.internal; +import org.opensearch.Version; import org.opensearch.action.search.SearchResponseSections; 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.ToXContentFragment; +import org.opensearch.search.SearchExtBuilder; import org.opensearch.search.SearchHits; import org.opensearch.search.aggregations.InternalAggregations; import org.opensearch.search.profile.SearchProfileShardResults; import org.opensearch.search.suggest.Suggest; import java.io.IOException; +import java.util.Collections; +import java.util.List; /** * {@link SearchResponseSections} subclass that can be serialized over the wire. @@ -67,7 +71,20 @@ public InternalSearchResponse( Boolean terminatedEarly, int numReducePhases ) { - super(hits, aggregations, suggest, timedOut, terminatedEarly, profileResults, numReducePhases); + this(hits, aggregations, suggest, profileResults, timedOut, terminatedEarly, numReducePhases, Collections.emptyList()); + } + + public InternalSearchResponse( + SearchHits hits, + InternalAggregations aggregations, + Suggest suggest, + SearchProfileShardResults profileResults, + boolean timedOut, + Boolean terminatedEarly, + int numReducePhases, + List searchExtBuilderList + ) { + super(hits, aggregations, suggest, timedOut, terminatedEarly, profileResults, numReducePhases, searchExtBuilderList); } public InternalSearchResponse(StreamInput in) throws IOException { @@ -78,7 +95,8 @@ public InternalSearchResponse(StreamInput in) throws IOException { in.readBoolean(), in.readOptionalBoolean(), in.readOptionalWriteable(SearchProfileShardResults::new), - in.readVInt() + in.readVInt(), + readSearchExtBuildersOnOrAfter(in) ); } @@ -91,5 +109,16 @@ public void writeTo(StreamOutput out) throws IOException { out.writeOptionalBoolean(terminatedEarly); out.writeOptionalWriteable(profileResults); out.writeVInt(numReducePhases); + writeSearchExtBuildersOnOrAfter(out, searchExtBuilders); + } + + private static List readSearchExtBuildersOnOrAfter(StreamInput in) throws IOException { + return (in.getVersion().onOrAfter(Version.V_3_0_0)) ? in.readNamedWriteableList(SearchExtBuilder.class) : Collections.emptyList(); + } + + private static void writeSearchExtBuildersOnOrAfter(StreamOutput out, List searchExtBuilders) throws IOException { + if (out.getVersion().onOrAfter(Version.V_3_0_0)) { + out.writeNamedWriteableList(searchExtBuilders); + } } } diff --git a/server/src/test/java/org/opensearch/action/search/SearchResponseTests.java b/server/src/test/java/org/opensearch/action/search/SearchResponseTests.java index c35bdf9c14587..097e922147698 100644 --- a/server/src/test/java/org/opensearch/action/search/SearchResponseTests.java +++ b/server/src/test/java/org/opensearch/action/search/SearchResponseTests.java @@ -37,9 +37,13 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.settings.Settings; import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.ParseField; +import org.opensearch.core.common.ParsingException; import org.opensearch.core.common.Strings; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.xcontent.MediaType; import org.opensearch.core.xcontent.MediaTypeRegistry; import org.opensearch.core.xcontent.NamedXContentRegistry; @@ -47,7 +51,10 @@ import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentHelper; import org.opensearch.core.xcontent.XContentParser; +import org.opensearch.plugins.SearchPlugin; import org.opensearch.rest.action.search.RestSearchAction; +import org.opensearch.search.GenericSearchExtBuilder; +import org.opensearch.search.SearchExtBuilder; import org.opensearch.search.SearchHit; import org.opensearch.search.SearchHits; import org.opensearch.search.SearchHitsTests; @@ -68,8 +75,8 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.UUID; -import static java.util.Collections.emptyList; import static java.util.Collections.singletonMap; import static org.opensearch.test.XContentTestUtils.insertRandomFields; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertToXContentEquivalent; @@ -80,11 +87,25 @@ public class SearchResponseTests extends OpenSearchTestCase { static { List namedXContents = new ArrayList<>(InternalAggregationTestCase.getDefaultNamedXContents()); namedXContents.addAll(SuggestTests.getDefaultNamedXContents()); + namedXContents.add( + new NamedXContentRegistry.Entry(SearchExtBuilder.class, DummySearchExtBuilder.DUMMY_FIELD, DummySearchExtBuilder::parse) + ); xContentRegistry = new NamedXContentRegistry(namedXContents); } private final NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry( - new SearchModule(Settings.EMPTY, emptyList()).getNamedWriteables() + new SearchModule(Settings.EMPTY, List.of(new SearchPlugin() { + @Override + public List> getSearchExts() { + return List.of( + new SearchExtSpec<>( + DummySearchExtBuilder.DUMMY_FIELD, + DummySearchExtBuilder::new, + parser -> DummySearchExtBuilder.parse(parser) + ) + ); + } + })).getNamedWriteables() ); private AggregationsTests aggregationsTests = new AggregationsTests(); @@ -119,6 +140,14 @@ private SearchResponse createMinimalTestItem() { * if minimal is set, don't include search hits, aggregations, suggest etc... to make test simpler */ private SearchResponse createTestItem(boolean minimal, ShardSearchFailure... shardSearchFailures) { + return createTestItem(minimal, Collections.emptyList(), shardSearchFailures); + } + + public SearchResponse createTestItem( + boolean minimal, + List searchExtBuilders, + ShardSearchFailure... shardSearchFailures + ) { boolean timedOut = randomBoolean(); Boolean terminatedEarly = randomBoolean() ? null : randomBoolean(); int numReducePhases = randomIntBetween(1, 10); @@ -139,7 +168,8 @@ private SearchResponse createTestItem(boolean minimal, ShardSearchFailure... sha profileShardResults, timedOut, terminatedEarly, - numReducePhases + numReducePhases, + searchExtBuilders ); } else { internalSearchResponse = InternalSearchResponse.empty(); @@ -153,7 +183,8 @@ private SearchResponse createTestItem(boolean minimal, ShardSearchFailure... sha skippedShards, tookInMillis, shardSearchFailures, - randomBoolean() ? randomClusters() : SearchResponse.Clusters.EMPTY + randomBoolean() ? randomClusters() : SearchResponse.Clusters.EMPTY, + null ); } @@ -172,6 +203,32 @@ public void testFromXContent() throws IOException { doFromXContentTestWithRandomFields(createTestItem(), false); } + public void testFromXContentWithSearchExtBuilders() throws IOException { + doFromXContentTestWithRandomFields(createTestItem(false, List.of(new DummySearchExtBuilder(UUID.randomUUID().toString()))), false); + } + + public void testFromXContentWithUnregisteredSearchExtBuilders() throws IOException { + List namedXContents = new ArrayList<>(InternalAggregationTestCase.getDefaultNamedXContents()); + namedXContents.addAll(SuggestTests.getDefaultNamedXContents()); + String dummyId = UUID.randomUUID().toString(); + String fakeId = UUID.randomUUID().toString(); + List extBuilders = List.of(new DummySearchExtBuilder(dummyId), new FakeSearchExtBuilder(fakeId)); + SearchResponse response = createTestItem(false, extBuilders); + MediaType xcontentType = randomFrom(XContentType.values()); + boolean humanReadable = randomBoolean(); + final ToXContent.Params params = new ToXContent.MapParams(singletonMap(RestSearchAction.TYPED_KEYS_PARAM, "true")); + BytesReference originalBytes = toShuffledXContent(response, xcontentType, params, humanReadable); + XContentParser parser = createParser(new NamedXContentRegistry(namedXContents), xcontentType.xContent(), originalBytes); + SearchResponse parsed = SearchResponse.fromXContent(parser); + assertEquals(extBuilders.size(), response.getInternalResponse().getSearchExtBuilders().size()); + + List actual = parsed.getInternalResponse().getSearchExtBuilders(); + assertEquals(extBuilders.size(), actual.size()); + for (int i = 0; i < actual.size(); i++) { + assertTrue(actual.get(0) instanceof GenericSearchExtBuilder); + } + } + /** * This test adds random fields and objects to the xContent rendered out to * ensure we can parse it back to be forward compatible with additions to @@ -182,7 +239,7 @@ public void testFromXContentWithRandomFields() throws IOException { doFromXContentTestWithRandomFields(createMinimalTestItem(), true); } - private void doFromXContentTestWithRandomFields(SearchResponse response, boolean addRandomFields) throws IOException { + public void doFromXContentTestWithRandomFields(SearchResponse response, boolean addRandomFields) throws IOException { MediaType xcontentType = randomFrom(XContentType.values()); boolean humanReadable = randomBoolean(); final ToXContent.Params params = new ToXContent.MapParams(singletonMap(RestSearchAction.TYPED_KEYS_PARAM, "true")); @@ -245,6 +302,7 @@ public void testToXContent() { SearchHit hit = new SearchHit(1, "id1", Collections.emptyMap(), Collections.emptyMap()); hit.score(2.0f); SearchHit[] hits = new SearchHit[] { hit }; + String dummyId = UUID.randomUUID().toString(); { SearchResponse response = new SearchResponse( new InternalSearchResponse( @@ -254,7 +312,8 @@ public void testToXContent() { null, false, null, - 1 + 1, + List.of(new DummySearchExtBuilder(dummyId)) ), null, 0, @@ -262,7 +321,8 @@ public void testToXContent() { 0, 0, ShardSearchFailure.EMPTY_ARRAY, - SearchResponse.Clusters.EMPTY + SearchResponse.Clusters.EMPTY, + null ); StringBuilder expectedString = new StringBuilder(); expectedString.append("{"); @@ -280,11 +340,17 @@ public void testToXContent() { { expectedString.append("{\"total\":{\"value\":100,\"relation\":\"eq\"},"); expectedString.append("\"max_score\":1.5,"); - expectedString.append("\"hits\":[{\"_id\":\"id1\",\"_score\":2.0}]}"); + expectedString.append("\"hits\":[{\"_id\":\"id1\",\"_score\":2.0}]},"); + } + expectedString.append("\"ext\":"); + { + expectedString.append("{\"dummy\":\"" + dummyId + "\"}"); } } expectedString.append("}"); assertEquals(expectedString.toString(), Strings.toString(MediaTypeRegistry.JSON, response)); + List searchExtBuilders = response.getInternalResponse().getSearchExtBuilders(); + assertEquals(1, searchExtBuilders.size()); } { SearchResponse response = new SearchResponse( @@ -352,6 +418,48 @@ public void testSerialization() throws IOException { assertEquals(searchResponse.getClusters(), deserialized.getClusters()); } + public void testSerializationWithSearchExtBuilders() throws IOException { + String id = UUID.randomUUID().toString(); + SearchResponse searchResponse = createTestItem(false, List.of(new DummySearchExtBuilder(id))); + SearchResponse deserialized = copyWriteable(searchResponse, namedWriteableRegistry, SearchResponse::new, Version.CURRENT); + if (searchResponse.getHits().getTotalHits() == null) { + assertNull(deserialized.getHits().getTotalHits()); + } else { + assertEquals(searchResponse.getHits().getTotalHits().value, deserialized.getHits().getTotalHits().value); + assertEquals(searchResponse.getHits().getTotalHits().relation, deserialized.getHits().getTotalHits().relation); + } + assertEquals(searchResponse.getHits().getHits().length, deserialized.getHits().getHits().length); + assertEquals(searchResponse.getNumReducePhases(), deserialized.getNumReducePhases()); + assertEquals(searchResponse.getFailedShards(), deserialized.getFailedShards()); + assertEquals(searchResponse.getTotalShards(), deserialized.getTotalShards()); + assertEquals(searchResponse.getSkippedShards(), deserialized.getSkippedShards()); + assertEquals(searchResponse.getClusters(), deserialized.getClusters()); + assertEquals( + searchResponse.getInternalResponse().getSearchExtBuilders().get(0), + deserialized.getInternalResponse().getSearchExtBuilders().get(0) + ); + } + + public void testSerializationWithSearchExtBuildersOnUnsupportedWriterVersion() throws IOException { + String id = UUID.randomUUID().toString(); + SearchResponse searchResponse = createTestItem(false, List.of(new DummySearchExtBuilder(id))); + SearchResponse deserialized = copyWriteable(searchResponse, namedWriteableRegistry, SearchResponse::new, Version.V_2_9_0); + if (searchResponse.getHits().getTotalHits() == null) { + assertNull(deserialized.getHits().getTotalHits()); + } else { + assertEquals(searchResponse.getHits().getTotalHits().value, deserialized.getHits().getTotalHits().value); + assertEquals(searchResponse.getHits().getTotalHits().relation, deserialized.getHits().getTotalHits().relation); + } + assertEquals(searchResponse.getHits().getHits().length, deserialized.getHits().getHits().length); + assertEquals(searchResponse.getNumReducePhases(), deserialized.getNumReducePhases()); + assertEquals(searchResponse.getFailedShards(), deserialized.getFailedShards()); + assertEquals(searchResponse.getTotalShards(), deserialized.getTotalShards()); + assertEquals(searchResponse.getSkippedShards(), deserialized.getSkippedShards()); + assertEquals(searchResponse.getClusters(), deserialized.getClusters()); + assertEquals(1, searchResponse.getInternalResponse().getSearchExtBuilders().size()); + assertTrue(deserialized.getInternalResponse().getSearchExtBuilders().isEmpty()); + } + public void testToXContentEmptyClusters() throws IOException { SearchResponse searchResponse = new SearchResponse( InternalSearchResponse.empty(), @@ -368,4 +476,89 @@ public void testToXContentEmptyClusters() throws IOException { deserialized.getClusters().toXContent(builder, ToXContent.EMPTY_PARAMS); assertEquals(0, builder.toString().length()); } + + static class DummySearchExtBuilder extends SearchExtBuilder { + + static ParseField DUMMY_FIELD = new ParseField("dummy"); + + protected final String id; + + public DummySearchExtBuilder(String id) { + assertNotNull(id); + this.id = id; + } + + public DummySearchExtBuilder(StreamInput in) throws IOException { + this.id = in.readString(); + } + + public String getId() { + return this.id; + } + + @Override + public String getWriteableName() { + return DUMMY_FIELD.getPreferredName(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(this.id); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return builder.field("dummy", id); + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return false; + } + + if (!(obj instanceof DummySearchExtBuilder)) { + return false; + } + + return this.id.equals(((DummySearchExtBuilder) obj).getId()); + } + + public static DummySearchExtBuilder parse(XContentParser parser) throws IOException { + String id; + XContentParser.Token token = parser.currentToken(); + if (token == XContentParser.Token.VALUE_STRING) { + id = parser.text(); + } else { + throw new ParsingException(parser.getTokenLocation(), "Expected a VALUE_STRING but got " + token); + } + if (id == null) { + throw new ParsingException(parser.getTokenLocation(), "no id specified for " + DUMMY_FIELD.getPreferredName()); + } + return new DummySearchExtBuilder(id); + } + } + + static class FakeSearchExtBuilder extends DummySearchExtBuilder { + static ParseField DUMMY_FIELD = new ParseField("fake"); + + public FakeSearchExtBuilder(String id) { + super(id); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(DUMMY_FIELD.getPreferredName()); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return builder.field(DUMMY_FIELD.getPreferredName(), id); + } + } } diff --git a/server/src/test/java/org/opensearch/search/GenericSearchExtBuilderTests.java b/server/src/test/java/org/opensearch/search/GenericSearchExtBuilderTests.java new file mode 100644 index 0000000000000..8fb1814962155 --- /dev/null +++ b/server/src/test/java/org/opensearch/search/GenericSearchExtBuilderTests.java @@ -0,0 +1,422 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ +package org.opensearch.search; + +import org.opensearch.Version; +import org.opensearch.action.search.SearchResponse; +import org.opensearch.action.search.SearchResponseTests; +import org.opensearch.action.search.ShardSearchFailure; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.ParseField; +import org.opensearch.core.common.ParsingException; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.common.io.stream.NamedWriteableRegistry; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.xcontent.MediaType; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; +import org.opensearch.plugins.SearchPlugin; +import org.opensearch.rest.action.search.RestSearchAction; +import org.opensearch.search.aggregations.AggregationsTests; +import org.opensearch.search.aggregations.InternalAggregations; +import org.opensearch.search.internal.InternalSearchResponse; +import org.opensearch.search.profile.SearchProfileShardResults; +import org.opensearch.search.profile.SearchProfileShardResultsTests; +import org.opensearch.search.suggest.Suggest; +import org.opensearch.search.suggest.SuggestTests; +import org.opensearch.test.InternalAggregationTestCase; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.After; +import org.junit.Before; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.UUID; + +import static java.util.Collections.singletonMap; + +public class GenericSearchExtBuilderTests extends OpenSearchTestCase { + + private static final NamedXContentRegistry xContentRegistry; + static { + List namedXContents = new ArrayList<>(InternalAggregationTestCase.getDefaultNamedXContents()); + namedXContents.addAll(SuggestTests.getDefaultNamedXContents()); + namedXContents.add( + new NamedXContentRegistry.Entry( + SearchExtBuilder.class, + GenericSearchExtBuilder.EXT_BUILDER_NAME, + GenericSearchExtBuilder::fromXContent + ) + ); + xContentRegistry = new NamedXContentRegistry(namedXContents); + } + + private final NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry( + new SearchModule(Settings.EMPTY, List.of(new SearchPlugin() { + @Override + public List> getSearchExts() { + return List.of( + new SearchExtSpec<>( + GenericSearchExtBuilder.EXT_BUILDER_NAME, + GenericSearchExtBuilder::new, + GenericSearchExtBuilder::fromXContent + ) + ); + } + })).getNamedWriteables() + ); + + @Override + protected NamedXContentRegistry xContentRegistry() { + return xContentRegistry; + } + + SearchResponseTests srt = new SearchResponseTests(); + private AggregationsTests aggregationsTests = new AggregationsTests(); + + @Before + public void init() throws Exception { + aggregationsTests.init(); + } + + @After + public void cleanUp() throws Exception { + aggregationsTests.cleanUp(); + } + + public void testFromXContentWithUnregisteredSearchExtBuilders() throws IOException { + List namedXContents = new ArrayList<>(InternalAggregationTestCase.getDefaultNamedXContents()); + namedXContents.addAll(SuggestTests.getDefaultNamedXContents()); + String dummyId = UUID.randomUUID().toString(); + List extBuilders = List.of( + new SimpleValueSearchExtBuilder(dummyId), + new MapSearchExtBuilder(Map.of("x", "y", "a", "b")), + new ListSearchExtBuilder(List.of("1", "2", "3")) + ); + SearchResponse response = srt.createTestItem(false, extBuilders); + MediaType xcontentType = randomFrom(XContentType.values()); + boolean humanReadable = randomBoolean(); + final ToXContent.Params params = new ToXContent.MapParams(singletonMap(RestSearchAction.TYPED_KEYS_PARAM, "true")); + BytesReference originalBytes = toShuffledXContent(response, xcontentType, params, humanReadable); + XContentParser parser = createParser(new NamedXContentRegistry(namedXContents), xcontentType.xContent(), originalBytes); + SearchResponse parsed = SearchResponse.fromXContent(parser); + assertEquals(extBuilders.size(), response.getInternalResponse().getSearchExtBuilders().size()); + + List actual = parsed.getInternalResponse().getSearchExtBuilders(); + assertEquals(extBuilders.size(), actual.size()); + for (int i = 0; i < actual.size(); i++) { + assertTrue(actual.get(0) instanceof GenericSearchExtBuilder); + } + } + + // This test case fails because GenericSearchExtBuilder does not retain the name of the SearchExtBuilder that it is replacing. + // GenericSearchExtBuilder has its own "generic_ext" section name. + // public void testFromXContentWithSearchExtBuilders() throws IOException { + // String dummyId = UUID.randomUUID().toString(); + // srt.doFromXContentTestWithRandomFields(createTestItem(false, List.of(new SimpleValueSearchExtBuilder(dummyId))), false); + // } + + public void testFromXContentWithGenericSearchExtBuildersForSimpleValues() throws IOException { + String dummyId = UUID.randomUUID().toString(); + srt.doFromXContentTestWithRandomFields( + createTestItem(false, List.of(new GenericSearchExtBuilder(dummyId, GenericSearchExtBuilder.ValueType.SIMPLE))), + false + ); + } + + public void testFromXContentWithGenericSearchExtBuildersForMapValues() throws IOException { + srt.doFromXContentTestWithRandomFields( + createTestItem(false, List.of(new GenericSearchExtBuilder(Map.of("x", "y", "a", "b"), GenericSearchExtBuilder.ValueType.MAP))), + false + ); + } + + public void testFromXContentWithGenericSearchExtBuildersForListValues() throws IOException { + String dummyId = UUID.randomUUID().toString(); + srt.doFromXContentTestWithRandomFields( + createTestItem(false, List.of(new GenericSearchExtBuilder(List.of("1", "2", "3"), GenericSearchExtBuilder.ValueType.LIST))), + false + ); + } + + public void testSerializationWithGenericSearchExtBuildersForSimpleValues() throws IOException { + String id = UUID.randomUUID().toString(); + SearchResponse searchResponse = createTestItem( + false, + List.of(new GenericSearchExtBuilder(id, GenericSearchExtBuilder.ValueType.SIMPLE)) + ); + SearchResponse deserialized = copyWriteable(searchResponse, namedWriteableRegistry, SearchResponse::new, Version.CURRENT); + if (searchResponse.getHits().getTotalHits() == null) { + assertNull(deserialized.getHits().getTotalHits()); + } else { + assertEquals(searchResponse.getHits().getTotalHits().value, deserialized.getHits().getTotalHits().value); + assertEquals(searchResponse.getHits().getTotalHits().relation, deserialized.getHits().getTotalHits().relation); + } + assertEquals(searchResponse.getHits().getHits().length, deserialized.getHits().getHits().length); + assertEquals(searchResponse.getNumReducePhases(), deserialized.getNumReducePhases()); + assertEquals(searchResponse.getFailedShards(), deserialized.getFailedShards()); + assertEquals(searchResponse.getTotalShards(), deserialized.getTotalShards()); + assertEquals(searchResponse.getSkippedShards(), deserialized.getSkippedShards()); + assertEquals(searchResponse.getClusters(), deserialized.getClusters()); + assertEquals( + searchResponse.getInternalResponse().getSearchExtBuilders().get(0), + deserialized.getInternalResponse().getSearchExtBuilders().get(0) + ); + } + + public void testSerializationWithGenericSearchExtBuildersForMapValues() throws IOException { + SearchResponse searchResponse = createTestItem( + false, + List.of(new GenericSearchExtBuilder(Map.of("x", "y", "a", "b"), GenericSearchExtBuilder.ValueType.MAP)) + ); + SearchResponse deserialized = copyWriteable(searchResponse, namedWriteableRegistry, SearchResponse::new, Version.CURRENT); + if (searchResponse.getHits().getTotalHits() == null) { + assertNull(deserialized.getHits().getTotalHits()); + } else { + assertEquals(searchResponse.getHits().getTotalHits().value, deserialized.getHits().getTotalHits().value); + assertEquals(searchResponse.getHits().getTotalHits().relation, deserialized.getHits().getTotalHits().relation); + } + assertEquals(searchResponse.getHits().getHits().length, deserialized.getHits().getHits().length); + assertEquals(searchResponse.getNumReducePhases(), deserialized.getNumReducePhases()); + assertEquals(searchResponse.getFailedShards(), deserialized.getFailedShards()); + assertEquals(searchResponse.getTotalShards(), deserialized.getTotalShards()); + assertEquals(searchResponse.getSkippedShards(), deserialized.getSkippedShards()); + assertEquals(searchResponse.getClusters(), deserialized.getClusters()); + assertEquals( + searchResponse.getInternalResponse().getSearchExtBuilders().get(0), + deserialized.getInternalResponse().getSearchExtBuilders().get(0) + ); + } + + public void testSerializationWithGenericSearchExtBuildersForListValues() throws IOException { + SearchResponse searchResponse = createTestItem( + false, + List.of(new GenericSearchExtBuilder(List.of("1", "2", "3"), GenericSearchExtBuilder.ValueType.LIST)) + ); + SearchResponse deserialized = copyWriteable(searchResponse, namedWriteableRegistry, SearchResponse::new, Version.CURRENT); + if (searchResponse.getHits().getTotalHits() == null) { + assertNull(deserialized.getHits().getTotalHits()); + } else { + assertEquals(searchResponse.getHits().getTotalHits().value, deserialized.getHits().getTotalHits().value); + assertEquals(searchResponse.getHits().getTotalHits().relation, deserialized.getHits().getTotalHits().relation); + } + assertEquals(searchResponse.getHits().getHits().length, deserialized.getHits().getHits().length); + assertEquals(searchResponse.getNumReducePhases(), deserialized.getNumReducePhases()); + assertEquals(searchResponse.getFailedShards(), deserialized.getFailedShards()); + assertEquals(searchResponse.getTotalShards(), deserialized.getTotalShards()); + assertEquals(searchResponse.getSkippedShards(), deserialized.getSkippedShards()); + assertEquals(searchResponse.getClusters(), deserialized.getClusters()); + assertEquals( + searchResponse.getInternalResponse().getSearchExtBuilders().get(0), + deserialized.getInternalResponse().getSearchExtBuilders().get(0) + ); + } + + public SearchResponse createTestItem( + boolean minimal, + List searchExtBuilders, + ShardSearchFailure... shardSearchFailures + ) { + boolean timedOut = randomBoolean(); + Boolean terminatedEarly = randomBoolean() ? null : randomBoolean(); + int numReducePhases = randomIntBetween(1, 10); + long tookInMillis = randomNonNegativeLong(); + int totalShards = randomIntBetween(1, Integer.MAX_VALUE); + int successfulShards = randomIntBetween(0, totalShards); + int skippedShards = randomIntBetween(0, totalShards); + InternalSearchResponse internalSearchResponse; + if (minimal == false) { + SearchHits hits = SearchHitsTests.createTestItem(true, true); + InternalAggregations aggregations = aggregationsTests.createTestInstance(); + Suggest suggest = SuggestTests.createTestItem(); + SearchProfileShardResults profileShardResults = SearchProfileShardResultsTests.createTestItem(); + internalSearchResponse = new InternalSearchResponse( + hits, + aggregations, + suggest, + profileShardResults, + timedOut, + terminatedEarly, + numReducePhases, + searchExtBuilders + ); + } else { + internalSearchResponse = InternalSearchResponse.empty(); + } + + return new SearchResponse( + internalSearchResponse, + null, + totalShards, + successfulShards, + skippedShards, + tookInMillis, + shardSearchFailures, + randomBoolean() ? randomClusters() : SearchResponse.Clusters.EMPTY, + null + ); + } + + static SearchResponse.Clusters randomClusters() { + int totalClusters = randomIntBetween(0, 10); + int successfulClusters = randomIntBetween(0, totalClusters); + int skippedClusters = totalClusters - successfulClusters; + return new SearchResponse.Clusters(totalClusters, successfulClusters, skippedClusters); + } + + static class SimpleValueSearchExtBuilder extends SearchExtBuilder { + + static ParseField FIELD = new ParseField("simple_value"); + + private final String id; + + public SimpleValueSearchExtBuilder(String id) { + assertNotNull(id); + this.id = id; + } + + public SimpleValueSearchExtBuilder(StreamInput in) throws IOException { + this.id = in.readString(); + } + + public String getId() { + return this.id; + } + + @Override + public String getWriteableName() { + return FIELD.getPreferredName(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(this.id); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return builder.field(FIELD.getPreferredName(), id); + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return false; + } + + if (!(obj instanceof SimpleValueSearchExtBuilder)) { + return false; + } + + return this.id.equals(((SimpleValueSearchExtBuilder) obj).getId()); + } + + public static SimpleValueSearchExtBuilder parse(XContentParser parser) throws IOException { + String id; + XContentParser.Token token = parser.currentToken(); + if (token == XContentParser.Token.VALUE_STRING) { + id = parser.text(); + } else { + throw new ParsingException(parser.getTokenLocation(), "Expected a VALUE_STRING but got " + token); + } + if (id == null) { + throw new ParsingException(parser.getTokenLocation(), "no id specified for " + FIELD.getPreferredName()); + } + return new SimpleValueSearchExtBuilder(id); + } + } + + static class MapSearchExtBuilder extends SearchExtBuilder { + + private final static String EXT_FIELD = "map0"; + + private final Map map; + + public MapSearchExtBuilder(Map map) { + this.map = new HashMap<>(); + for (Map.Entry e : map.entrySet()) { + this.map.put(e.getKey(), e.getValue()); + } + } + + @Override + public String getWriteableName() { + return EXT_FIELD; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeMap(this.map); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return builder.field(EXT_FIELD, this.map); + } + + @Override + public int hashCode() { + return Objects.hash(this.getClass(), this.map); + } + + @Override + public boolean equals(Object obj) { + return false; + } + } + + static class ListSearchExtBuilder extends SearchExtBuilder { + + private final static String EXT_FIELD = "list0"; + + private final List list; + + public ListSearchExtBuilder(List list) { + this.list = new ArrayList<>(); + list.forEach(e -> this.list.add(e)); + } + + @Override + public String getWriteableName() { + return EXT_FIELD; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeCollection(this.list, StreamOutput::writeString); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return builder.field(EXT_FIELD, this.list); + } + + @Override + public int hashCode() { + return Objects.hash(this.getClass(), this.list); + } + + @Override + public boolean equals(Object obj) { + return false; + } + } +} From 81c7b9773cbb862bbb3e36695077c62bcfdaa7b6 Mon Sep 17 00:00:00 2001 From: Sayali Gaikawad <61760125+gaiksaya@users.noreply.github.com> Date: Tue, 29 Aug 2023 16:19:24 -0700 Subject: [PATCH 10/37] Fix GH runners memory issue by increasing swapfile (#9596) Signed-off-by: Sayali Gaikawad --- .github/workflows/check-compatibility.yml | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/.github/workflows/check-compatibility.yml b/.github/workflows/check-compatibility.yml index b5f2ccbae6917..dab8afd4ec1f2 100644 --- a/.github/workflows/check-compatibility.yml +++ b/.github/workflows/check-compatibility.yml @@ -15,6 +15,15 @@ jobs: with: ref: ${{ github.event.pull_request.head.sha }} + - name: Increase swapfile + run: | + sudo swapoff -a + sudo fallocate -l 10G /swapfile + sudo chmod 600 /swapfile + sudo mkswap /swapfile + sudo swapon /swapfile + sudo swapon --show + - name: Run compatibility task run: ./gradlew checkCompatibility -i | tee $HOME/gradlew-check.out From 8324b889b9da286f506451ff8133150e3a0b757b Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Tue, 29 Aug 2023 20:42:55 -0700 Subject: [PATCH 11/37] [Remote Store] Retry RemoteIndexShardTests flaky tests (#9597) Signed-off-by: Suraj Singh --- build.gradle | 1 + 1 file changed, 1 insertion(+) diff --git a/build.gradle b/build.gradle index eef5d6d5b442c..9d62e942a4431 100644 --- a/build.gradle +++ b/build.gradle @@ -501,6 +501,7 @@ subprojects { includeClasses.add("org.opensearch.index.reindex.DeleteByQueryBasicTests") includeClasses.add("org.opensearch.index.reindex.UpdateByQueryBasicTests") includeClasses.add("org.opensearch.index.shard.IndexShardIT") + includeClasses.add("org.opensearch.index.shard.RemoteIndexShardTests") includeClasses.add("org.opensearch.index.shard.RemoteStoreRefreshListenerTests") includeClasses.add("org.opensearch.index.translog.RemoteFSTranslogTests") includeClasses.add("org.opensearch.indices.DateMathIndexExpressionsIntegrationIT") From 78eea275a3e1812a008e815d846ca871d7f09d20 Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Wed, 30 Aug 2023 11:56:43 -0400 Subject: [PATCH 12/37] [BWC and API enforcement] Decorate the existing APIs with proper annotations (part 1) (#9520) * [BWC and API enforcement] Decorate the existing APIs with proper annotations (part 1) Signed-off-by: Andriy Redko * Address code review comments Signed-off-by: Andriy Redko --------- Signed-off-by: Andriy Redko --- CHANGELOG.md | 1 + .../opensearch/common/CheckedConsumer.java | 3 ++ .../common/action/ActionFuture.java | 2 ++ .../common/lifecycle/Lifecycle.java | 8 ++++-- .../common/lifecycle/LifecycleComponent.java | 4 ++- .../org/opensearch/common/unit/TimeValue.java | 3 ++ .../src/main/java/org/opensearch/Version.java | 2 ++ .../java/org/opensearch/core/ParseField.java | 5 ++++ .../core/action/ActionListener.java | 2 ++ .../core/common/bytes/BytesReference.java | 4 ++- .../io/stream/NamedWriteableRegistry.java | 8 ++++-- .../core/common/io/stream/StreamInput.java | 4 ++- .../core/common/io/stream/StreamOutput.java | 4 ++- .../core/common/settings/SecureString.java | 5 +++- .../core/common/unit/ByteSizeUnit.java | 4 ++- .../core/common/unit/ByteSizeValue.java | 4 ++- .../java/org/opensearch/core/index/Index.java | 4 ++- .../opensearch/core/index/shard/ShardId.java | 4 ++- .../org/opensearch/core/rest/RestStatus.java | 2 ++ .../org/opensearch/core/tasks/TaskId.java | 4 ++- .../opensearch/core/xcontent/MediaType.java | 4 +++ .../core/xcontent/NamedXContentRegistry.java | 5 +++- .../core/xcontent/XContentBuilder.java | 2 ++ .../opensearch/bootstrap/BootstrapCheck.java | 6 +++- .../bootstrap/BootstrapContext.java | 4 ++- .../org/opensearch/client/AdminClient.java | 5 +++- .../java/org/opensearch/client/Client.java | 4 ++- .../opensearch/client/ClusterAdminClient.java | 4 ++- .../opensearch/client/IndicesAdminClient.java | 4 ++- .../metadata/IndexNameExpressionResolver.java | 4 ++- .../metadata/IndexTemplateMetadata.java | 4 ++- .../cluster/node/DiscoveryNode.java | 4 ++- .../cluster/node/DiscoveryNodeRole.java | 4 ++- .../cluster/service/ClusterService.java | 4 ++- .../org/opensearch/common/inject/Module.java | 5 +++- .../common/settings/ClusterSettings.java | 4 ++- .../common/settings/SecureSettings.java | 4 ++- .../opensearch/common/settings/Setting.java | 28 +++++++++++++------ .../common/settings/SettingUpgrader.java | 5 +++- .../opensearch/common/settings/Settings.java | 7 +++-- .../common/settings/SettingsException.java | 4 ++- .../java/org/opensearch/env/Environment.java | 4 ++- .../org/opensearch/env/NodeEnvironment.java | 7 +++-- .../java/org/opensearch/env/ShardLock.java | 4 ++- .../env/ShardLockObtainFailedException.java | 4 ++- .../org/opensearch/index/IndexModule.java | 4 ++- .../org/opensearch/index/IndexSettings.java | 4 ++- .../index/shard/IndexSettingProvider.java | 4 ++- .../java/org/opensearch/plugins/Plugin.java | 2 ++ .../repositories/RepositoriesService.java | 4 ++- .../java/org/opensearch/script/Script.java | 4 ++- .../org/opensearch/script/ScriptContext.java | 4 ++- .../org/opensearch/script/ScriptEngine.java | 5 +++- .../org/opensearch/script/ScriptService.java | 4 ++- .../org/opensearch/script/ScriptType.java | 4 ++- .../org/opensearch/threadpool/ThreadPool.java | 4 ++- .../watcher/ResourceWatcherService.java | 4 ++- 57 files changed, 200 insertions(+), 58 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index eabc17f81917f..1d11c28ec2429 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -90,6 +90,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [Feature] Expose term frequency in Painless script score context ([#9081](https://github.com/opensearch-project/OpenSearch/pull/9081)) - Add support for reading partial files to HDFS repository ([#9513](https://github.com/opensearch-project/OpenSearch/issues/9513)) - Add support for extensions to search responses using SearchExtBuilder ([#9379](https://github.com/opensearch-project/OpenSearch/pull/9379)) +- [BWC and API enforcement] Decorate the existing APIs with proper annotations (part 1) ([#9520](https://github.com/opensearch-project/OpenSearch/pull/9520)) ### Dependencies - Bump `org.apache.logging.log4j:log4j-core` from 2.17.1 to 2.20.0 ([#8307](https://github.com/opensearch-project/OpenSearch/pull/8307)) diff --git a/libs/common/src/main/java/org/opensearch/common/CheckedConsumer.java b/libs/common/src/main/java/org/opensearch/common/CheckedConsumer.java index dede06d0e207d..07b4973c3a340 100644 --- a/libs/common/src/main/java/org/opensearch/common/CheckedConsumer.java +++ b/libs/common/src/main/java/org/opensearch/common/CheckedConsumer.java @@ -32,6 +32,8 @@ package org.opensearch.common; +import org.opensearch.common.annotation.PublicApi; + import java.util.function.Consumer; /** @@ -39,6 +41,7 @@ * * @opensearch.api */ +@PublicApi(since = "1.0.0") @FunctionalInterface public interface CheckedConsumer { void accept(T t) throws E; diff --git a/libs/common/src/main/java/org/opensearch/common/action/ActionFuture.java b/libs/common/src/main/java/org/opensearch/common/action/ActionFuture.java index 5bb8111ef089f..7f9dd096667e9 100644 --- a/libs/common/src/main/java/org/opensearch/common/action/ActionFuture.java +++ b/libs/common/src/main/java/org/opensearch/common/action/ActionFuture.java @@ -32,6 +32,7 @@ package org.opensearch.common.action; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.unit.TimeValue; import java.util.concurrent.Future; @@ -42,6 +43,7 @@ * * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface ActionFuture extends Future { /** diff --git a/libs/common/src/main/java/org/opensearch/common/lifecycle/Lifecycle.java b/libs/common/src/main/java/org/opensearch/common/lifecycle/Lifecycle.java index e76d49cbf49e8..c1cf9b2998a13 100644 --- a/libs/common/src/main/java/org/opensearch/common/lifecycle/Lifecycle.java +++ b/libs/common/src/main/java/org/opensearch/common/lifecycle/Lifecycle.java @@ -32,6 +32,8 @@ package org.opensearch.common.lifecycle; +import org.opensearch.common.annotation.PublicApi; + /** * Lifecycle state. Allows the following transitions: *
    @@ -73,15 +75,17 @@ * } * * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class Lifecycle { /** * State in the lifecycle * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public enum State { INITIALIZED, STOPPED, diff --git a/libs/common/src/main/java/org/opensearch/common/lifecycle/LifecycleComponent.java b/libs/common/src/main/java/org/opensearch/common/lifecycle/LifecycleComponent.java index f343f9ada01ef..781c276fefe13 100644 --- a/libs/common/src/main/java/org/opensearch/common/lifecycle/LifecycleComponent.java +++ b/libs/common/src/main/java/org/opensearch/common/lifecycle/LifecycleComponent.java @@ -32,13 +32,15 @@ package org.opensearch.common.lifecycle; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.lease.Releasable; /** * Base interface for a lifecycle component. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface LifecycleComponent extends Releasable { Lifecycle.State lifecycleState(); diff --git a/libs/common/src/main/java/org/opensearch/common/unit/TimeValue.java b/libs/common/src/main/java/org/opensearch/common/unit/TimeValue.java index 670275397893c..a3fcffb1d6a4c 100644 --- a/libs/common/src/main/java/org/opensearch/common/unit/TimeValue.java +++ b/libs/common/src/main/java/org/opensearch/common/unit/TimeValue.java @@ -32,6 +32,8 @@ package org.opensearch.common.unit; +import org.opensearch.common.annotation.PublicApi; + import java.util.Locale; import java.util.Objects; import java.util.concurrent.TimeUnit; @@ -41,6 +43,7 @@ * * @opensearch.api */ +@PublicApi(since = "1.0.0") public class TimeValue implements Comparable { /** How many nano-seconds in one milli-second */ diff --git a/libs/core/src/main/java/org/opensearch/Version.java b/libs/core/src/main/java/org/opensearch/Version.java index 3f83282245fd8..b05a069ba971c 100644 --- a/libs/core/src/main/java/org/opensearch/Version.java +++ b/libs/core/src/main/java/org/opensearch/Version.java @@ -33,6 +33,7 @@ package org.opensearch; import org.opensearch.common.SuppressForbidden; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; @@ -50,6 +51,7 @@ * * @opensearch.api */ +@PublicApi(since = "1.0.0") public class Version implements Comparable, ToXContentFragment { /* * The logic for ID is: XXYYZZAA, where XX is major version, YY is minor version, ZZ is revision, and AA is alpha/beta/rc indicator AA diff --git a/libs/core/src/main/java/org/opensearch/core/ParseField.java b/libs/core/src/main/java/org/opensearch/core/ParseField.java index 5741f97d1d335..171b8eaf5c397 100644 --- a/libs/core/src/main/java/org/opensearch/core/ParseField.java +++ b/libs/core/src/main/java/org/opensearch/core/ParseField.java @@ -31,6 +31,7 @@ package org.opensearch.core; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.xcontent.DeprecationHandler; import org.opensearch.core.xcontent.XContentLocation; @@ -43,7 +44,11 @@ /** * Holds a field that can be found in a request while parsing and its different * variants, which may be deprecated. + * + * @opensearch.api + * */ +@PublicApi(since = "1.0.0") public class ParseField { private final String name; private final String[] deprecatedNames; diff --git a/libs/core/src/main/java/org/opensearch/core/action/ActionListener.java b/libs/core/src/main/java/org/opensearch/core/action/ActionListener.java index 1127e0151145a..119e56cfe0bf2 100644 --- a/libs/core/src/main/java/org/opensearch/core/action/ActionListener.java +++ b/libs/core/src/main/java/org/opensearch/core/action/ActionListener.java @@ -37,6 +37,7 @@ import org.opensearch.common.CheckedFunction; import org.opensearch.common.CheckedRunnable; import org.opensearch.common.CheckedSupplier; +import org.opensearch.common.annotation.PublicApi; import java.util.ArrayList; import java.util.List; @@ -48,6 +49,7 @@ * * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface ActionListener { /** * Handle action response. This response may constitute a failure or a diff --git a/libs/core/src/main/java/org/opensearch/core/common/bytes/BytesReference.java b/libs/core/src/main/java/org/opensearch/core/common/bytes/BytesReference.java index fc8e62c914e27..9d24d3653397b 100644 --- a/libs/core/src/main/java/org/opensearch/core/common/bytes/BytesReference.java +++ b/libs/core/src/main/java/org/opensearch/core/common/bytes/BytesReference.java @@ -35,6 +35,7 @@ import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefIterator; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.BytesStream; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.util.ByteArray; @@ -50,8 +51,9 @@ /** * A reference to bytes. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface BytesReference extends Comparable, ToXContentFragment { /** diff --git a/libs/core/src/main/java/org/opensearch/core/common/io/stream/NamedWriteableRegistry.java b/libs/core/src/main/java/org/opensearch/core/common/io/stream/NamedWriteableRegistry.java index ec707f147cade..abac76c8b6c27 100644 --- a/libs/core/src/main/java/org/opensearch/core/common/io/stream/NamedWriteableRegistry.java +++ b/libs/core/src/main/java/org/opensearch/core/common/io/stream/NamedWriteableRegistry.java @@ -32,6 +32,8 @@ package org.opensearch.core.common.io.stream; +import org.opensearch.common.annotation.PublicApi; + import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -45,15 +47,17 @@ * The registration is keyed by the combination of the category class of {@link NamedWriteable}, and a name unique * to that category. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class NamedWriteableRegistry { /** * An entry in the registry, made up of a category class and name, and a reader for that category class. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public static class Entry { /** The superclass of a {@link NamedWriteable} which will be read by {@link #reader}. */ diff --git a/libs/core/src/main/java/org/opensearch/core/common/io/stream/StreamInput.java b/libs/core/src/main/java/org/opensearch/core/common/io/stream/StreamInput.java index 6681bc6035d7a..ece2012302919 100644 --- a/libs/core/src/main/java/org/opensearch/core/common/io/stream/StreamInput.java +++ b/libs/core/src/main/java/org/opensearch/core/common/io/stream/StreamInput.java @@ -46,6 +46,7 @@ import org.opensearch.Version; import org.opensearch.common.CharArrays; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.unit.TimeValue; import org.opensearch.core.common.Strings; import org.opensearch.core.common.bytes.BytesArray; @@ -104,8 +105,9 @@ * lists, either by storing {@code List}s internally or just converting to and from a {@code List} when calling. This comment is repeated * on {@link StreamInput}. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public abstract class StreamInput extends InputStream { private Version version = Version.CURRENT; diff --git a/libs/core/src/main/java/org/opensearch/core/common/io/stream/StreamOutput.java b/libs/core/src/main/java/org/opensearch/core/common/io/stream/StreamOutput.java index a61278c0cc4de..94b813246bc7e 100644 --- a/libs/core/src/main/java/org/opensearch/core/common/io/stream/StreamOutput.java +++ b/libs/core/src/main/java/org/opensearch/core/common/io/stream/StreamOutput.java @@ -45,6 +45,7 @@ import org.opensearch.Version; import org.opensearch.common.CharArrays; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.unit.TimeValue; import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.core.common.bytes.BytesReference; @@ -96,8 +97,9 @@ * lists, either by storing {@code List}s internally or just converting to and from a {@code List} when calling. This comment is repeated * on {@link StreamInput}. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public abstract class StreamOutput extends OutputStream { private static final int MAX_NESTED_EXCEPTION_LEVEL = 100; diff --git a/libs/core/src/main/java/org/opensearch/core/common/settings/SecureString.java b/libs/core/src/main/java/org/opensearch/core/common/settings/SecureString.java index f5529bcebc82f..322300a554284 100644 --- a/libs/core/src/main/java/org/opensearch/core/common/settings/SecureString.java +++ b/libs/core/src/main/java/org/opensearch/core/common/settings/SecureString.java @@ -32,6 +32,8 @@ package org.opensearch.core.common.settings; +import org.opensearch.common.annotation.PublicApi; + import java.io.Closeable; import java.util.Arrays; import java.util.Objects; @@ -39,8 +41,9 @@ /** * A String implementations which allows clearing the underlying char array. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class SecureString implements CharSequence, Closeable { private char[] chars; diff --git a/libs/core/src/main/java/org/opensearch/core/common/unit/ByteSizeUnit.java b/libs/core/src/main/java/org/opensearch/core/common/unit/ByteSizeUnit.java index 68486dd7c975f..1f49a3531986c 100644 --- a/libs/core/src/main/java/org/opensearch/core/common/unit/ByteSizeUnit.java +++ b/libs/core/src/main/java/org/opensearch/core/common/unit/ByteSizeUnit.java @@ -32,6 +32,7 @@ package org.opensearch.core.common.unit; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; @@ -45,8 +46,9 @@ * helps organize and use size representations that may be maintained * separately across various contexts. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public enum ByteSizeUnit implements Writeable { BYTES { @Override diff --git a/libs/core/src/main/java/org/opensearch/core/common/unit/ByteSizeValue.java b/libs/core/src/main/java/org/opensearch/core/common/unit/ByteSizeValue.java index 529501226f5e3..1ed6d2d204a99 100644 --- a/libs/core/src/main/java/org/opensearch/core/common/unit/ByteSizeValue.java +++ b/libs/core/src/main/java/org/opensearch/core/common/unit/ByteSizeValue.java @@ -33,6 +33,7 @@ package org.opensearch.core.common.unit; import org.opensearch.OpenSearchParseException; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -47,8 +48,9 @@ /** * A byte size value * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class ByteSizeValue implements Writeable, Comparable, ToXContentFragment { public static final ByteSizeValue ZERO = new ByteSizeValue(0, ByteSizeUnit.BYTES); diff --git a/libs/core/src/main/java/org/opensearch/core/index/Index.java b/libs/core/src/main/java/org/opensearch/core/index/Index.java index c7b680dd1f753..fdff43f3c9139 100644 --- a/libs/core/src/main/java/org/opensearch/core/index/Index.java +++ b/libs/core/src/main/java/org/opensearch/core/index/Index.java @@ -32,6 +32,7 @@ package org.opensearch.core.index; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.ParseField; import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; @@ -48,8 +49,9 @@ /** * A value class representing the basic required properties of an OpenSearch index. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class Index implements Writeable, ToXContentObject { public static final Index[] EMPTY_ARRAY = new Index[0]; diff --git a/libs/core/src/main/java/org/opensearch/core/index/shard/ShardId.java b/libs/core/src/main/java/org/opensearch/core/index/shard/ShardId.java index f6980be94ca49..adea6cd8f0687 100644 --- a/libs/core/src/main/java/org/opensearch/core/index/shard/ShardId.java +++ b/libs/core/src/main/java/org/opensearch/core/index/shard/ShardId.java @@ -32,6 +32,7 @@ package org.opensearch.core.index.shard; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -45,8 +46,9 @@ /** * Allows for shard level components to be injected with the shard id. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class ShardId implements Comparable, ToXContentFragment, Writeable { private final Index index; diff --git a/libs/core/src/main/java/org/opensearch/core/rest/RestStatus.java b/libs/core/src/main/java/org/opensearch/core/rest/RestStatus.java index ae4f4c65b28d2..313bc23bedc90 100644 --- a/libs/core/src/main/java/org/opensearch/core/rest/RestStatus.java +++ b/libs/core/src/main/java/org/opensearch/core/rest/RestStatus.java @@ -32,6 +32,7 @@ package org.opensearch.core.rest; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.action.ShardOperationFailedException; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -47,6 +48,7 @@ * * @opensearch.api */ +@PublicApi(since = "1.0.0") public enum RestStatus { /** * The client SHOULD continue with its request. This interim response is used to inform the client that the diff --git a/libs/core/src/main/java/org/opensearch/core/tasks/TaskId.java b/libs/core/src/main/java/org/opensearch/core/tasks/TaskId.java index 97b0231613c73..d34d4acf00e6e 100644 --- a/libs/core/src/main/java/org/opensearch/core/tasks/TaskId.java +++ b/libs/core/src/main/java/org/opensearch/core/tasks/TaskId.java @@ -33,6 +33,7 @@ package org.opensearch.core.tasks; import org.opensearch.OpenSearchParseException; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -45,8 +46,9 @@ /** * Task id that consists of node id and id of the task on the node * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class TaskId implements Writeable { public static final TaskId EMPTY_TASK_ID = new TaskId(); diff --git a/libs/core/src/main/java/org/opensearch/core/xcontent/MediaType.java b/libs/core/src/main/java/org/opensearch/core/xcontent/MediaType.java index 8e3c115c7ba58..c58b3e80d98b5 100644 --- a/libs/core/src/main/java/org/opensearch/core/xcontent/MediaType.java +++ b/libs/core/src/main/java/org/opensearch/core/xcontent/MediaType.java @@ -32,6 +32,7 @@ package org.opensearch.core.xcontent; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.Writeable; import java.io.IOException; @@ -42,7 +43,10 @@ * Abstracts a Media Type and a format parameter. * Media types are used as values on Content-Type and Accept headers * format is an URL parameter, specifies response media type. + * + * @opensearch.api */ +@PublicApi(since = "2.1.0") public interface MediaType extends Writeable { /** * Returns a type part of a MediaType diff --git a/libs/core/src/main/java/org/opensearch/core/xcontent/NamedXContentRegistry.java b/libs/core/src/main/java/org/opensearch/core/xcontent/NamedXContentRegistry.java index 10718ba98fe17..9d876825c5196 100644 --- a/libs/core/src/main/java/org/opensearch/core/xcontent/NamedXContentRegistry.java +++ b/libs/core/src/main/java/org/opensearch/core/xcontent/NamedXContentRegistry.java @@ -33,6 +33,7 @@ package org.opensearch.core.xcontent; import org.opensearch.common.CheckedFunction; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.ParseField; import java.io.IOException; @@ -49,8 +50,9 @@ /** * Main registry for serializable content (e.g., field mappers, aggregations) * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class NamedXContentRegistry { /** * The empty {@link NamedXContentRegistry} for use when you are sure that you aren't going to call @@ -64,6 +66,7 @@ public class NamedXContentRegistry { /** * An entry in the {@linkplain NamedXContentRegistry} containing the name of the object and the parser that can parse it. */ + @PublicApi(since = "1.0.0") public static class Entry { /** The class that this entry can read. */ public final Class categoryClass; diff --git a/libs/core/src/main/java/org/opensearch/core/xcontent/XContentBuilder.java b/libs/core/src/main/java/org/opensearch/core/xcontent/XContentBuilder.java index dfd1449ef0e0b..a38bdd049ee88 100644 --- a/libs/core/src/main/java/org/opensearch/core/xcontent/XContentBuilder.java +++ b/libs/core/src/main/java/org/opensearch/core/xcontent/XContentBuilder.java @@ -32,6 +32,7 @@ package org.opensearch.core.xcontent; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.bytes.BytesReference; import java.io.ByteArrayOutputStream; @@ -61,6 +62,7 @@ /** * A utility to build XContent (ie json). */ +@PublicApi(since = "1.0.0") public final class XContentBuilder implements Closeable, Flushable { /** diff --git a/server/src/main/java/org/opensearch/bootstrap/BootstrapCheck.java b/server/src/main/java/org/opensearch/bootstrap/BootstrapCheck.java index 429612ba1b93d..a695486bd084c 100644 --- a/server/src/main/java/org/opensearch/bootstrap/BootstrapCheck.java +++ b/server/src/main/java/org/opensearch/bootstrap/BootstrapCheck.java @@ -32,18 +32,22 @@ package org.opensearch.bootstrap; +import org.opensearch.common.annotation.PublicApi; + import java.util.Objects; /** * Encapsulates a bootstrap check. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface BootstrapCheck { /** * Encapsulate the result of a bootstrap check. */ + @PublicApi(since = "1.0.0") final class BootstrapCheckResult { private final String message; diff --git a/server/src/main/java/org/opensearch/bootstrap/BootstrapContext.java b/server/src/main/java/org/opensearch/bootstrap/BootstrapContext.java index 1cfd8bf6dfc35..a7ffd701d07f0 100644 --- a/server/src/main/java/org/opensearch/bootstrap/BootstrapContext.java +++ b/server/src/main/java/org/opensearch/bootstrap/BootstrapContext.java @@ -32,14 +32,16 @@ package org.opensearch.bootstrap; import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.settings.Settings; import org.opensearch.env.Environment; /** * Context that is passed to every bootstrap check to make decisions on. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class BootstrapContext { /** * The node's environment diff --git a/server/src/main/java/org/opensearch/client/AdminClient.java b/server/src/main/java/org/opensearch/client/AdminClient.java index 0c6c97b795983..1a5a39be4241a 100644 --- a/server/src/main/java/org/opensearch/client/AdminClient.java +++ b/server/src/main/java/org/opensearch/client/AdminClient.java @@ -32,13 +32,16 @@ package org.opensearch.client; +import org.opensearch.common.annotation.PublicApi; + /** * Administrative actions/operations against the cluster or the indices. * * @see org.opensearch.client.Client#admin() * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface AdminClient { /** diff --git a/server/src/main/java/org/opensearch/client/Client.java b/server/src/main/java/org/opensearch/client/Client.java index 551c64ad1c835..f4ae383249f61 100644 --- a/server/src/main/java/org/opensearch/client/Client.java +++ b/server/src/main/java/org/opensearch/client/Client.java @@ -83,6 +83,7 @@ import org.opensearch.action.update.UpdateResponse; import org.opensearch.common.Nullable; import org.opensearch.common.action.ActionFuture; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.lease.Releasable; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; @@ -102,8 +103,9 @@ * * @see org.opensearch.node.Node#client() * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface Client extends OpenSearchClient, Releasable { Setting CLIENT_TYPE_SETTING_S = new Setting<>("client.type", "node", (s) -> { diff --git a/server/src/main/java/org/opensearch/client/ClusterAdminClient.java b/server/src/main/java/org/opensearch/client/ClusterAdminClient.java index 0b511fa95b9d0..05f09c1a6e661 100644 --- a/server/src/main/java/org/opensearch/client/ClusterAdminClient.java +++ b/server/src/main/java/org/opensearch/client/ClusterAdminClient.java @@ -157,6 +157,7 @@ import org.opensearch.action.search.PutSearchPipelineRequest; import org.opensearch.action.support.master.AcknowledgedResponse; import org.opensearch.common.action.ActionFuture; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.tasks.TaskId; @@ -167,8 +168,9 @@ * * @see AdminClient#cluster() * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface ClusterAdminClient extends OpenSearchClient { /** diff --git a/server/src/main/java/org/opensearch/client/IndicesAdminClient.java b/server/src/main/java/org/opensearch/client/IndicesAdminClient.java index 72b986ee25a31..20dab1caa36c4 100644 --- a/server/src/main/java/org/opensearch/client/IndicesAdminClient.java +++ b/server/src/main/java/org/opensearch/client/IndicesAdminClient.java @@ -129,6 +129,7 @@ import org.opensearch.cluster.metadata.IndexMetadata.APIBlock; import org.opensearch.common.Nullable; import org.opensearch.common.action.ActionFuture; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.action.ActionListener; /** @@ -136,8 +137,9 @@ * * @see AdminClient#indices() * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface IndicesAdminClient extends OpenSearchClient { /** diff --git a/server/src/main/java/org/opensearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/opensearch/cluster/metadata/IndexNameExpressionResolver.java index 06de4d6929f0b..9a3b569a7ac3d 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexNameExpressionResolver.java @@ -38,6 +38,7 @@ import org.opensearch.cluster.ClusterState; import org.opensearch.common.Booleans; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.collect.Tuple; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.regex.Regex; @@ -76,8 +77,9 @@ /** * Resolves index name from an expression * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class IndexNameExpressionResolver { private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(IndexNameExpressionResolver.class); diff --git a/server/src/main/java/org/opensearch/cluster/metadata/IndexTemplateMetadata.java b/server/src/main/java/org/opensearch/cluster/metadata/IndexTemplateMetadata.java index 3074719ffa179..272bb132197af 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexTemplateMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexTemplateMetadata.java @@ -35,6 +35,7 @@ import org.opensearch.cluster.AbstractDiffable; import org.opensearch.cluster.Diff; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.collect.MapBuilder; import org.opensearch.common.compress.CompressedXContent; import org.opensearch.common.logging.DeprecationLogger; @@ -63,8 +64,9 @@ /** * Metadata for Index Templates * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class IndexTemplateMetadata extends AbstractDiffable { private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(IndexTemplateMetadata.class); diff --git a/server/src/main/java/org/opensearch/cluster/node/DiscoveryNode.java b/server/src/main/java/org/opensearch/cluster/node/DiscoveryNode.java index f68ac406aa01e..a04b0d9de912d 100644 --- a/server/src/main/java/org/opensearch/cluster/node/DiscoveryNode.java +++ b/server/src/main/java/org/opensearch/cluster/node/DiscoveryNode.java @@ -34,6 +34,7 @@ import org.opensearch.Version; import org.opensearch.common.UUIDs; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.core.common.io.stream.StreamInput; @@ -64,8 +65,9 @@ /** * A discovery node represents a node that is part of the cluster. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class DiscoveryNode implements Writeable, ToXContentFragment { static final String COORDINATING_ONLY = "coordinating_only"; diff --git a/server/src/main/java/org/opensearch/cluster/node/DiscoveryNodeRole.java b/server/src/main/java/org/opensearch/cluster/node/DiscoveryNodeRole.java index 07d70b2c6c1b2..0d2b08656c38d 100644 --- a/server/src/main/java/org/opensearch/cluster/node/DiscoveryNodeRole.java +++ b/server/src/main/java/org/opensearch/cluster/node/DiscoveryNodeRole.java @@ -35,6 +35,7 @@ import org.opensearch.LegacyESVersion; import org.opensearch.Version; import org.opensearch.common.Booleans; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; @@ -52,8 +53,9 @@ /** * Represents a node role. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public abstract class DiscoveryNodeRole implements Comparable { private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(DiscoveryNodeRole.class); diff --git a/server/src/main/java/org/opensearch/cluster/service/ClusterService.java b/server/src/main/java/org/opensearch/cluster/service/ClusterService.java index e097803d86b48..aa7766979e851 100644 --- a/server/src/main/java/org/opensearch/cluster/service/ClusterService.java +++ b/server/src/main/java/org/opensearch/cluster/service/ClusterService.java @@ -45,6 +45,7 @@ import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.routing.OperationRouting; import org.opensearch.cluster.routing.RerouteService; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.lifecycle.AbstractLifecycleComponent; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; @@ -60,8 +61,9 @@ /** * Main Cluster Service * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class ClusterService extends AbstractLifecycleComponent { private final ClusterManagerService clusterManagerService; diff --git a/server/src/main/java/org/opensearch/common/inject/Module.java b/server/src/main/java/org/opensearch/common/inject/Module.java index b1fc031192ea0..e66044ff26c40 100644 --- a/server/src/main/java/org/opensearch/common/inject/Module.java +++ b/server/src/main/java/org/opensearch/common/inject/Module.java @@ -29,6 +29,8 @@ package org.opensearch.common.inject; +import org.opensearch.common.annotation.PublicApi; + /** * A module contributes configuration information, typically interface * bindings, which will be used to create an {@link Injector}. A Guice-based @@ -43,8 +45,9 @@ * Use scope and binding annotations on these methods to configure the * bindings. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface Module { /** diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index 32d14a3519659..c89ae8088f1be 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -78,6 +78,7 @@ import org.opensearch.cluster.service.ClusterManagerService; import org.opensearch.cluster.service.ClusterManagerTaskThrottler; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.logging.Loggers; import org.opensearch.common.network.NetworkModule; import org.opensearch.common.network.NetworkService; @@ -166,8 +167,9 @@ /** * Encapsulates all valid cluster level settings. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class ClusterSettings extends AbstractScopedSettings { public ClusterSettings(final Settings nodeSettings, final Set> settingsSet) { diff --git a/server/src/main/java/org/opensearch/common/settings/SecureSettings.java b/server/src/main/java/org/opensearch/common/settings/SecureSettings.java index 2fe7d4834c92a..3732478243dab 100644 --- a/server/src/main/java/org/opensearch/common/settings/SecureSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/SecureSettings.java @@ -32,6 +32,7 @@ package org.opensearch.common.settings; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.settings.SecureString; import java.io.Closeable; @@ -43,8 +44,9 @@ /** * An accessor for settings which are securely stored. See {@link SecureSetting}. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface SecureSettings extends Closeable { /** Returns true iff the settings are loaded and retrievable. */ diff --git a/server/src/main/java/org/opensearch/common/settings/Setting.java b/server/src/main/java/org/opensearch/common/settings/Setting.java index c43e0f26f9138..0e96edff0681c 100644 --- a/server/src/main/java/org/opensearch/common/settings/Setting.java +++ b/server/src/main/java/org/opensearch/common/settings/Setting.java @@ -38,6 +38,7 @@ import org.opensearch.Version; import org.opensearch.common.Booleans; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.collect.Tuple; import org.opensearch.common.regex.Regex; import org.opensearch.common.unit.MemorySizeValue; @@ -102,15 +103,17 @@ * } * * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class Setting implements ToXContentObject { /** * Property of the setting * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public enum Property { /** * should be filtered in some api (mask password/credentials) @@ -635,8 +638,9 @@ public Setting getConcreteSetting(String key) { * Allows a setting to declare a dependency on another setting being set. Optionally, a setting can validate the value of the dependent * setting. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public interface SettingDependency { /** @@ -784,8 +788,9 @@ public String toString() { /** * Allows an affix setting to declare a dependency on another affix setting. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public interface AffixSettingDependency extends SettingDependency { @Override @@ -796,8 +801,9 @@ public interface AffixSettingDependency extends SettingDependency { /** * An affix setting * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public static class AffixSetting extends Setting { private final AffixKey key; private final BiFunction> delegateFactory; @@ -1026,9 +1032,10 @@ public Map getAsMap(Settings settings) { * * @param the type of the {@link Setting} * - * @opensearch.internal + * @opensearch.api */ @FunctionalInterface + @PublicApi(since = "1.0.0") public interface Validator { /** @@ -2834,8 +2841,9 @@ private static AffixSetting affixKeySetting( /** * Key for the setting * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public interface Key { boolean match(String key); } @@ -2843,8 +2851,9 @@ public interface Key { /** * A simple key for a setting * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public static class SimpleKey implements Key { protected final String key; @@ -2918,8 +2927,9 @@ public boolean match(String toTest) { * A key that allows for static pre and suffix. This is used for settings * that have dynamic namespaces like for different accounts etc. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public static final class AffixKey implements Key { private final Pattern pattern; private final String prefix; diff --git a/server/src/main/java/org/opensearch/common/settings/SettingUpgrader.java b/server/src/main/java/org/opensearch/common/settings/SettingUpgrader.java index 1dabf020d8398..dac0b9b867768 100644 --- a/server/src/main/java/org/opensearch/common/settings/SettingUpgrader.java +++ b/server/src/main/java/org/opensearch/common/settings/SettingUpgrader.java @@ -32,6 +32,8 @@ package org.opensearch.common.settings; +import org.opensearch.common.annotation.PublicApi; + import java.util.List; /** @@ -39,8 +41,9 @@ * * @param the type of the underlying setting * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface SettingUpgrader { /** diff --git a/server/src/main/java/org/opensearch/common/settings/Settings.java b/server/src/main/java/org/opensearch/common/settings/Settings.java index ae10f38943e73..91e39e38f0379 100644 --- a/server/src/main/java/org/opensearch/common/settings/Settings.java +++ b/server/src/main/java/org/opensearch/common/settings/Settings.java @@ -38,6 +38,7 @@ import org.opensearch.Version; import org.opensearch.common.Booleans; import org.opensearch.common.SetOnce; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.logging.LogConfigurator; import org.opensearch.common.unit.MemorySizeValue; @@ -95,8 +96,9 @@ /** * An immutable settings implementation. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class Settings implements ToXContentFragment { public static final Settings EMPTY = new Builder().build(); @@ -750,8 +752,9 @@ public Set keySet() { * settings implementation. Use {@link Settings#builder()} in order to * construct it. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public static class Builder { public static final Settings EMPTY_SETTINGS = new Builder().build(); diff --git a/server/src/main/java/org/opensearch/common/settings/SettingsException.java b/server/src/main/java/org/opensearch/common/settings/SettingsException.java index d1b924827a651..5e1d2ada2529d 100644 --- a/server/src/main/java/org/opensearch/common/settings/SettingsException.java +++ b/server/src/main/java/org/opensearch/common/settings/SettingsException.java @@ -33,6 +33,7 @@ package org.opensearch.common.settings; import org.opensearch.OpenSearchException; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.rest.RestStatus; @@ -41,8 +42,9 @@ /** * A generic failure to handle settings. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class SettingsException extends OpenSearchException { public SettingsException(String message) { diff --git a/server/src/main/java/org/opensearch/env/Environment.java b/server/src/main/java/org/opensearch/env/Environment.java index a1e467ad1ba48..3b87c756ffdae 100644 --- a/server/src/main/java/org/opensearch/env/Environment.java +++ b/server/src/main/java/org/opensearch/env/Environment.java @@ -33,6 +33,7 @@ package org.opensearch.env; import org.opensearch.common.SuppressForbidden; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.io.PathUtils; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; @@ -56,8 +57,9 @@ /** * The environment of where things exists. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") @SuppressForbidden(reason = "configures paths for the system") // TODO: move PathUtils to be package-private here instead of // public+forbidden api! diff --git a/server/src/main/java/org/opensearch/env/NodeEnvironment.java b/server/src/main/java/org/opensearch/env/NodeEnvironment.java index 1d58351b98b67..3c5ab5ba98875 100644 --- a/server/src/main/java/org/opensearch/env/NodeEnvironment.java +++ b/server/src/main/java/org/opensearch/env/NodeEnvironment.java @@ -53,6 +53,7 @@ import org.opensearch.common.Randomness; import org.opensearch.common.SuppressForbidden; import org.opensearch.common.UUIDs; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.collect.Tuple; import org.opensearch.common.lease.Releasable; import org.opensearch.common.settings.Setting; @@ -108,14 +109,16 @@ /** * A component that holds all data paths for a single node. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class NodeEnvironment implements Closeable { /** * A node path. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public static class NodePath { /* ${data.paths}/nodes/{node.id} */ public final Path path; diff --git a/server/src/main/java/org/opensearch/env/ShardLock.java b/server/src/main/java/org/opensearch/env/ShardLock.java index dd34eb3275f68..76afc0ec0329a 100644 --- a/server/src/main/java/org/opensearch/env/ShardLock.java +++ b/server/src/main/java/org/opensearch/env/ShardLock.java @@ -32,6 +32,7 @@ package org.opensearch.env; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.index.shard.ShardId; import java.io.Closeable; @@ -44,8 +45,9 @@ * * @see NodeEnvironment * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public abstract class ShardLock implements Closeable { private final ShardId shardId; diff --git a/server/src/main/java/org/opensearch/env/ShardLockObtainFailedException.java b/server/src/main/java/org/opensearch/env/ShardLockObtainFailedException.java index 525d8a76c9699..ae77d942356b5 100644 --- a/server/src/main/java/org/opensearch/env/ShardLockObtainFailedException.java +++ b/server/src/main/java/org/opensearch/env/ShardLockObtainFailedException.java @@ -33,6 +33,7 @@ package org.opensearch.env; import org.opensearch.OpenSearchException; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.index.shard.ShardId; @@ -41,8 +42,9 @@ /** * Exception used when the in-memory lock for a shard cannot be obtained * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class ShardLockObtainFailedException extends OpenSearchException { public ShardLockObtainFailedException(ShardId shardId, String message) { diff --git a/server/src/main/java/org/opensearch/index/IndexModule.java b/server/src/main/java/org/opensearch/index/IndexModule.java index 131e2a867ac8b..d1e071eedb39e 100644 --- a/server/src/main/java/org/opensearch/index/IndexModule.java +++ b/server/src/main/java/org/opensearch/index/IndexModule.java @@ -48,6 +48,7 @@ import org.opensearch.common.CheckedFunction; import org.opensearch.common.SetOnce; import org.opensearch.common.TriFunction; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; @@ -119,8 +120,9 @@ * {@link #addSettingsUpdateConsumer(Setting, Consumer)} *
* - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class IndexModule { public static final Setting NODE_STORE_ALLOW_MMAP = Setting.boolSetting("node.store.allow_mmap", true, Property.NodeScope); diff --git a/server/src/main/java/org/opensearch/index/IndexSettings.java b/server/src/main/java/org/opensearch/index/IndexSettings.java index ec719c99e163f..4609b1f994737 100644 --- a/server/src/main/java/org/opensearch/index/IndexSettings.java +++ b/server/src/main/java/org/opensearch/index/IndexSettings.java @@ -36,6 +36,7 @@ import org.apache.lucene.sandbox.index.MergeOnFlushMergePolicy; import org.opensearch.Version; import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.logging.Loggers; import org.opensearch.common.settings.IndexScopedSettings; import org.opensearch.common.settings.Setting; @@ -77,8 +78,9 @@ * a settings consumer at index creation via {@link IndexModule#addSettingsUpdateConsumer(Setting, Consumer)} that will * be called for each settings update. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class IndexSettings { private static final String MERGE_ON_FLUSH_DEFAULT_POLICY = "default"; private static final String MERGE_ON_FLUSH_MERGE_POLICY = "merge-on-flush"; diff --git a/server/src/main/java/org/opensearch/index/shard/IndexSettingProvider.java b/server/src/main/java/org/opensearch/index/shard/IndexSettingProvider.java index 441a9a6413ffc..861a325c45d4b 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexSettingProvider.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexSettingProvider.java @@ -32,14 +32,16 @@ package org.opensearch.index.shard; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.settings.Settings; /** * An {@link IndexSettingProvider} is a provider for index level settings that can be set * explicitly as a default value (so they show up as "set" for newly created indices) * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface IndexSettingProvider { /** * Returns explicitly set default index {@link Settings} for the given index. This should not diff --git a/server/src/main/java/org/opensearch/plugins/Plugin.java b/server/src/main/java/org/opensearch/plugins/Plugin.java index 998741a098792..0743cd3807eff 100644 --- a/server/src/main/java/org/opensearch/plugins/Plugin.java +++ b/server/src/main/java/org/opensearch/plugins/Plugin.java @@ -40,6 +40,7 @@ import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.node.DiscoveryNodeRole; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Module; import org.opensearch.common.lifecycle.LifecycleComponent; import org.opensearch.common.settings.Setting; @@ -89,6 +90,7 @@ * * @opensearch.api */ +@PublicApi(since = "1.0.0") public abstract class Plugin implements Closeable { /** diff --git a/server/src/main/java/org/opensearch/repositories/RepositoriesService.java b/server/src/main/java/org/opensearch/repositories/RepositoriesService.java index f18dc63013abf..f00bf3942c9a9 100644 --- a/server/src/main/java/org/opensearch/repositories/RepositoriesService.java +++ b/server/src/main/java/org/opensearch/repositories/RepositoriesService.java @@ -56,6 +56,7 @@ import org.opensearch.cluster.service.ClusterManagerTaskKeys; import org.opensearch.cluster.service.ClusterManagerTaskThrottler; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.lifecycle.AbstractLifecycleComponent; import org.opensearch.common.regex.Regex; import org.opensearch.common.settings.Setting; @@ -86,8 +87,9 @@ /** * Service responsible for maintaining and providing access to snapshot repositories on nodes. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class RepositoriesService extends AbstractLifecycleComponent implements ClusterStateApplier { private static final Logger logger = LogManager.getLogger(RepositoriesService.class); diff --git a/server/src/main/java/org/opensearch/script/Script.java b/server/src/main/java/org/opensearch/script/Script.java index ed88737a5b87e..a611e71c3bf3f 100644 --- a/server/src/main/java/org/opensearch/script/Script.java +++ b/server/src/main/java/org/opensearch/script/Script.java @@ -33,6 +33,7 @@ package org.opensearch.script; import org.opensearch.OpenSearchParseException; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.settings.Settings; import org.opensearch.common.xcontent.LoggingDeprecationHandler; @@ -96,8 +97,9 @@ * * * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class Script implements ToXContentObject, Writeable { private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(Script.class); diff --git a/server/src/main/java/org/opensearch/script/ScriptContext.java b/server/src/main/java/org/opensearch/script/ScriptContext.java index 27ad1f3ce03c8..71ced303b062e 100644 --- a/server/src/main/java/org/opensearch/script/ScriptContext.java +++ b/server/src/main/java/org/opensearch/script/ScriptContext.java @@ -32,6 +32,7 @@ package org.opensearch.script; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.collect.Tuple; import org.opensearch.common.unit.TimeValue; @@ -70,8 +71,9 @@ * If the variable name starts with an underscore, for example, {@code _score}, the needs method would * be {@code boolean needs_score()}. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class ScriptContext { /** A unique identifier for this context. */ diff --git a/server/src/main/java/org/opensearch/script/ScriptEngine.java b/server/src/main/java/org/opensearch/script/ScriptEngine.java index 418fbed52da30..560727bc8fa97 100644 --- a/server/src/main/java/org/opensearch/script/ScriptEngine.java +++ b/server/src/main/java/org/opensearch/script/ScriptEngine.java @@ -32,6 +32,8 @@ package org.opensearch.script; +import org.opensearch.common.annotation.PublicApi; + import java.io.Closeable; import java.io.IOException; import java.util.Map; @@ -40,8 +42,9 @@ /** * A script language implementation. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface ScriptEngine extends Closeable { /** diff --git a/server/src/main/java/org/opensearch/script/ScriptService.java b/server/src/main/java/org/opensearch/script/ScriptService.java index f0e6bd5d54422..d3c8861dbc5d7 100644 --- a/server/src/main/java/org/opensearch/script/ScriptService.java +++ b/server/src/main/java/org/opensearch/script/ScriptService.java @@ -46,6 +46,7 @@ import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.service.ClusterManagerTaskThrottler; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; @@ -75,8 +76,9 @@ /** * Service for scripting * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class ScriptService implements Closeable, ClusterStateApplier { private static final Logger logger = LogManager.getLogger(ScriptService.class); diff --git a/server/src/main/java/org/opensearch/script/ScriptType.java b/server/src/main/java/org/opensearch/script/ScriptType.java index 5f505c781bd0a..c39edcbcb12c4 100644 --- a/server/src/main/java/org/opensearch/script/ScriptType.java +++ b/server/src/main/java/org/opensearch/script/ScriptType.java @@ -32,6 +32,7 @@ package org.opensearch.script; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.ParseField; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -45,8 +46,9 @@ * It's also used to by {@link ScriptService} to determine whether or not a {@link Script} is * allowed to be executed based on both default and user-defined settings. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public enum ScriptType implements Writeable { /** diff --git a/server/src/main/java/org/opensearch/threadpool/ThreadPool.java b/server/src/main/java/org/opensearch/threadpool/ThreadPool.java index af69698536420..6ddf3ff6b2f6a 100644 --- a/server/src/main/java/org/opensearch/threadpool/ThreadPool.java +++ b/server/src/main/java/org/opensearch/threadpool/ThreadPool.java @@ -37,6 +37,7 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.Version; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.SizeValue; @@ -78,8 +79,9 @@ /** * The OpenSearch threadpool class * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class ThreadPool implements ReportingService, Scheduler { private static final Logger logger = LogManager.getLogger(ThreadPool.class); diff --git a/server/src/main/java/org/opensearch/watcher/ResourceWatcherService.java b/server/src/main/java/org/opensearch/watcher/ResourceWatcherService.java index a7c7a248ce417..9b9c00cd4252f 100644 --- a/server/src/main/java/org/opensearch/watcher/ResourceWatcherService.java +++ b/server/src/main/java/org/opensearch/watcher/ResourceWatcherService.java @@ -33,6 +33,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; import org.opensearch.common.settings.Settings; @@ -54,8 +55,9 @@ * registered watcher periodically. The frequency of checks can be specified using {@code resource.reload.interval} setting, which * defaults to {@code 60s}. The service can be disabled by setting {@code resource.reload.enabled} setting to {@code false}. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class ResourceWatcherService implements Closeable { private static final Logger logger = LogManager.getLogger(ResourceWatcherService.class); From 6cd576f2d69b9c7d05d22aecff3fd9a6e6d335c9 Mon Sep 17 00:00:00 2001 From: Marc Handalian Date: Wed, 30 Aug 2023 10:02:29 -0700 Subject: [PATCH 13/37] Fix SegmentReplicationUsingRemoteStoreIT#testDropPrimaryDuringReplication. (#9471) * Fix SegmentReplicationUsingRemoteStoreIT#testDropPrimaryDuringReplication. This test is failing because a concurrent flush can wipe out an old commit file while we are in the remote store refresh listener. The listener will fetch the latest infos from the reader which will reference a segments_n tht has been deleted by an incoming flush. To fix this, InternalEngine will preserve the latest commit until a new commit is loaded onto the readerManager. Signed-off-by: Marc Handalian * update InternalEngine to preserve commit file until a new commit is refreshed on. Signed-off-by: Marc Handalian * Update ReadOnlyEngine inside of resetEngineToGlobalCheckpoint to implement getSegmentInfosSnapshot. This ensures access to this function is not permitted on the ReadOnlyEngine and is delegated to the new IE once opened. Signed-off-by: Marc Handalian * Update javadoc. Signed-off-by: Marc Handalian * spotless. Signed-off-by: Marc Handalian --------- Signed-off-by: Marc Handalian --- .../SegmentReplicationUsingRemoteStoreIT.java | 11 --- .../index/engine/InternalEngine.java | 47 +++++----- .../opensearch/index/shard/IndexShard.java | 10 +++ .../indices/replication/common/CopyState.java | 7 -- .../index/engine/InternalEngineTests.java | 90 +++++++++++++++++-- .../index/shard/RemoteIndexShardTests.java | 28 +++--- .../RemoteStoreRefreshListenerTests.java | 7 ++ .../SegmentReplicationIndexShardTests.java | 16 +++- .../SegmentReplicationSourceHandlerTests.java | 1 + 9 files changed, 154 insertions(+), 63 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationUsingRemoteStoreIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationUsingRemoteStoreIT.java index ecb1c9b0f86db..22250c3b793cf 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationUsingRemoteStoreIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationUsingRemoteStoreIT.java @@ -62,15 +62,4 @@ public void setup() { public void teardown() { assertAcked(clusterAdmin().prepareDeleteRepository(REPOSITORY_NAME)); } - - @Override - public void testPressureServiceStats() throws Exception { - super.testPressureServiceStats(); - } - - @Override - @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/8059") - public void testDropPrimaryDuringReplication() throws Exception { - super.testDropPrimaryDuringReplication(); - } } diff --git a/server/src/main/java/org/opensearch/index/engine/InternalEngine.java b/server/src/main/java/org/opensearch/index/engine/InternalEngine.java index e8e5042cfe944..bfa96445f5b2d 100644 --- a/server/src/main/java/org/opensearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/opensearch/index/engine/InternalEngine.java @@ -1862,6 +1862,13 @@ public void flush(boolean force, boolean waitIfOngoing) throws EngineException { try { translogManager.rollTranslogGeneration(); logger.trace("starting commit for flush; commitTranslog=true"); + // with Segment Replication we need to hold the latest commit before a new one is created and ensure it is released + // only after the active reader is updated. This ensures that a flush does not wipe out a required commit point file + // while we are + // in refresh listeners. + final GatedCloseable latestCommit = engineConfig.getIndexSettings().isSegRepEnabled() + ? acquireLastIndexCommit(false) + : null; commitIndexWriter(indexWriter, translogManager.getTranslogUUID()); logger.trace("finished commit for flush"); @@ -1875,6 +1882,11 @@ public void flush(boolean force, boolean waitIfOngoing) throws EngineException { // we need to refresh in order to clear older version values refresh("version_table_flush", SearcherScope.INTERNAL, true); + + if (latestCommit != null) { + latestCommit.close(); + } + translogManager.trimUnreferencedReaders(); } catch (AlreadyClosedException e) { failOnTragicEvent(e); @@ -2134,41 +2146,32 @@ protected SegmentInfos getLastCommittedSegmentInfos() { @Override protected SegmentInfos getLatestSegmentInfos() { - OpenSearchDirectoryReader reader = null; - try { - reader = internalReaderManager.acquire(); - return ((StandardDirectoryReader) reader.getDelegate()).getSegmentInfos(); + try (final GatedCloseable snapshot = getSegmentInfosSnapshot()) { + return snapshot.get(); } catch (IOException e) { throw new EngineException(shardId, e.getMessage(), e); - } finally { - try { - internalReaderManager.release(reader); - } catch (IOException e) { - throw new EngineException(shardId, e.getMessage(), e); - } } } /** - * Fetch the latest {@link SegmentInfos} object via {@link #getLatestSegmentInfos()} - * but also increment the ref-count to ensure that these segment files are retained - * until the reference is closed. On close, the ref-count is decremented. + * Fetch the latest {@link SegmentInfos} from the current ReaderManager's active DirectoryReader. + * This method will hold the reader reference until the returned {@link GatedCloseable} is closed. */ @Override public GatedCloseable getSegmentInfosSnapshot() { - final SegmentInfos segmentInfos = getLatestSegmentInfos(); + final OpenSearchDirectoryReader reader; try { - indexWriter.incRefDeleter(segmentInfos); + reader = internalReaderManager.acquire(); + return new GatedCloseable<>(((StandardDirectoryReader) reader.getDelegate()).getSegmentInfos(), () -> { + try { + internalReaderManager.release(reader); + } catch (AlreadyClosedException e) { + logger.warn("Engine is already closed.", e); + } + }); } catch (IOException e) { throw new EngineException(shardId, e.getMessage(), e); } - return new GatedCloseable<>(segmentInfos, () -> { - try { - indexWriter.decRefDeleter(segmentInfos); - } catch (AlreadyClosedException e) { - logger.warn("Engine is already closed.", e); - } - }); } @Override diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index 4a39c3e6c942e..1c1d7fed5e4f5 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -4582,6 +4582,16 @@ public GatedCloseable acquireSafeIndexCommit() { } } + @Override + public GatedCloseable getSegmentInfosSnapshot() { + synchronized (engineMutex) { + if (newEngineReference.get() == null) { + throw new AlreadyClosedException("engine was closed"); + } + return newEngineReference.get().getSegmentInfosSnapshot(); + } + } + @Override public void close() throws IOException { assert Thread.holdsLock(engineMutex); diff --git a/server/src/main/java/org/opensearch/indices/replication/common/CopyState.java b/server/src/main/java/org/opensearch/indices/replication/common/CopyState.java index a6aa39e7cb074..3b7ae2af80ca0 100644 --- a/server/src/main/java/org/opensearch/indices/replication/common/CopyState.java +++ b/server/src/main/java/org/opensearch/indices/replication/common/CopyState.java @@ -8,7 +8,6 @@ package org.opensearch.indices.replication.common; -import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.store.ByteBuffersDataOutput; import org.apache.lucene.store.ByteBuffersIndexOutput; @@ -38,7 +37,6 @@ public class CopyState extends AbstractRefCounted { private final ReplicationCheckpoint replicationCheckpoint; private final Map metadataMap; private final byte[] infosBytes; - private GatedCloseable commitRef; private final IndexShard shard; public CopyState(ReplicationCheckpoint requestedReplicationCheckpoint, IndexShard shard) throws IOException { @@ -51,7 +49,6 @@ public CopyState(ReplicationCheckpoint requestedReplicationCheckpoint, IndexShar this.replicationCheckpoint = latestSegmentInfosAndCheckpoint.v2(); SegmentInfos segmentInfos = this.segmentInfosRef.get(); this.metadataMap = shard.store().getSegmentMetadataMap(segmentInfos); - this.commitRef = shard.acquireLastIndexCommit(false); ByteBuffersDataOutput buffer = new ByteBuffersDataOutput(); // resource description and name are not used, but resource description cannot be null @@ -65,10 +62,6 @@ public CopyState(ReplicationCheckpoint requestedReplicationCheckpoint, IndexShar protected void closeInternal() { try { segmentInfosRef.close(); - // commitRef may be null if there were no pending delete files - if (commitRef != null) { - commitRef.close(); - } } catch (IOException e) { throw new UncheckedIOException(e); } diff --git a/server/src/test/java/org/opensearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/opensearch/index/engine/InternalEngineTests.java index 0f7a571987df0..e93d65518ffbb 100644 --- a/server/src/test/java/org/opensearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/opensearch/index/engine/InternalEngineTests.java @@ -156,6 +156,7 @@ import org.opensearch.threadpool.ThreadPool; import org.hamcrest.MatcherAssert; import org.hamcrest.Matchers; +import org.junit.Assert; import java.io.Closeable; import java.io.IOException; @@ -165,6 +166,7 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; @@ -7530,16 +7532,86 @@ public void testMaxDocsOnReplica() throws Exception { } } - public void testGetSegmentInfosSnapshot() throws IOException { + public void testGetSegmentInfosSnapshot_AllSnapshotFilesPreservedAcrossCommit() throws Exception { IOUtils.close(store, engine); - Store store = createStore(); - InternalEngine engine = spy(createEngine(store, createTempDir())); - GatedCloseable segmentInfosSnapshot = engine.getSegmentInfosSnapshot(); - assertNotNull(segmentInfosSnapshot); - assertNotNull(segmentInfosSnapshot.get()); - verify(engine, times(1)).getLatestSegmentInfos(); - store.close(); - engine.close(); + store = createStore(); + engine = createEngine(store, createTempDir()); + List operations = generateHistoryOnReplica( + randomIntBetween(1, 100), + randomBoolean(), + randomBoolean(), + randomBoolean() + ); + for (Engine.Operation op : operations) { + applyOperation(engine, op); + } + engine.refresh("test"); + try (GatedCloseable snapshot = engine.getSegmentInfosSnapshot()) { + Collection files = snapshot.get().files(true); + Set localFiles = Set.of(store.directory().listAll()); + for (String file : files) { + assertTrue("Local directory contains file " + file, localFiles.contains(file)); + } + + engine.flush(true, true); + + try ( + final GatedCloseable snapshotAfterFlush = engine.getSegmentInfosSnapshot(); + final GatedCloseable commit = engine.acquireLastIndexCommit(false) + ) { + final SegmentInfos segmentInfos = snapshotAfterFlush.get(); + assertNotEquals(segmentInfos.getSegmentsFileName(), snapshot.get().getSegmentsFileName()); + assertEquals(commit.get().getSegmentsFileName(), segmentInfos.getSegmentsFileName()); + } + + // original files are preserved. + localFiles = Set.of(store.directory().listAll()); + for (String file : files) { + assertTrue("Local directory contains file " + file, localFiles.contains(file)); + } + } + } + + public void testGetSegmentInfosSnapshot_LatestCommitOnDiskHasHigherGenThanReader() throws Exception { + IOUtils.close(store, engine); + store = createStore(); + engine = createEngine(store, createTempDir()); + // to simulate this we need concurrent flush/refresh. + AtomicBoolean run = new AtomicBoolean(true); + AtomicInteger docId = new AtomicInteger(0); + Thread refresher = new Thread(() -> { + while (run.get()) { + try { + engine.index(indexForDoc(createParsedDoc(Integer.toString(docId.getAndIncrement()), null))); + engine.refresh("test"); + getSnapshotAndAssertFilesExistLocally(); + } catch (Exception e) { + Assert.fail(); + } + } + }); + refresher.start(); + try { + for (int i = 0; i < 10; i++) { + engine.flush(true, true); + getSnapshotAndAssertFilesExistLocally(); + } + } catch (Exception e) { + Assert.fail(); + } finally { + run.set(false); + refresher.join(); + } + } + + private void getSnapshotAndAssertFilesExistLocally() throws IOException { + try (GatedCloseable snapshot = engine.getSegmentInfosSnapshot()) { + Collection files = snapshot.get().files(true); + Set localFiles = Set.of(store.directory().listAll()); + for (String file : files) { + assertTrue("Local directory contains file " + file, localFiles.contains(file)); + } + } } public void testGetProcessedLocalCheckpoint() throws IOException { diff --git a/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java index ead9c1c22c931..9dcecbe1059b6 100644 --- a/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java @@ -12,6 +12,7 @@ import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.util.Version; import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.concurrent.GatedCloseable; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.FeatureFlags; import org.opensearch.index.engine.DocIdSeqNoAndSource; @@ -204,11 +205,14 @@ public void testReplicaCommitsInfosBytesOnRecovery() throws Exception { Set.of("segments_3"), primary.remoteStore().readLastCommittedSegmentsInfo().files(true) ); - MatcherAssert.assertThat( - "Segments are referenced in memory only", - primaryEngine.getSegmentInfosSnapshot().get().files(false), - containsInAnyOrder("_0.cfe", "_0.si", "_0.cfs") - ); + + try (final GatedCloseable segmentInfosSnapshot = primaryEngine.getSegmentInfosSnapshot()) { + MatcherAssert.assertThat( + "Segments are referenced in memory only", + segmentInfosSnapshot.get().files(false), + containsInAnyOrder("_0.cfe", "_0.si", "_0.cfs") + ); + } final IndexShard replica = shards.addReplica(remotePath); replica.store().createEmpty(Version.LATEST); @@ -238,11 +242,15 @@ public void testReplicaCommitsInfosBytesOnRecovery() throws Exception { latestReplicaCommit.files(true), containsInAnyOrder("_0.cfe", "_0.si", "_0.cfs", "segments_6") ); - MatcherAssert.assertThat( - "Segments are referenced in memory", - replicaEngine.getSegmentInfosSnapshot().get().files(false), - containsInAnyOrder("_0.cfe", "_0.si", "_0.cfs") - ); + + try (final GatedCloseable segmentInfosSnapshot = replicaEngine.getSegmentInfosSnapshot()) { + MatcherAssert.assertThat( + "Segments are referenced in memory", + segmentInfosSnapshot.get().files(false), + containsInAnyOrder("_0.cfe", "_0.si", "_0.cfs") + ); + } + final Store.RecoveryDiff recoveryDiff = Store.segmentReplicationDiff( primary.getSegmentMetadataMap(), replica.getSegmentMetadataMap() diff --git a/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java b/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java index 95fe67592d5f8..e05f8dc6e4e57 100644 --- a/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java +++ b/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java @@ -508,6 +508,13 @@ private Tuple mockIndexS return indexShard.getSegmentInfosSnapshot(); }).when(shard).getSegmentInfosSnapshot(); + doAnswer((invocation -> { + if (counter.incrementAndGet() <= succeedOnAttempt) { + throw new RuntimeException("Inducing failure in upload"); + } + return indexShard.getLatestSegmentInfosAndCheckpoint(); + })).when(shard).getLatestSegmentInfosAndCheckpoint(); + doAnswer(invocation -> { if (Objects.nonNull(successLatch)) { successLatch.countDown(); diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java index e8220830063ee..29daa3936e8bb 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java @@ -63,6 +63,7 @@ import org.junit.Assert; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -817,17 +818,24 @@ protected void assertDocCounts(IndexShard indexShard, int expectedPersistedDocCo } protected void resolveCheckpointInfoResponseListener(ActionListener listener, IndexShard primary) { + final CopyState copyState; try { - final CopyState copyState = new CopyState( + copyState = new CopyState( ReplicationCheckpoint.empty(primary.shardId, primary.getLatestReplicationCheckpoint().getCodec()), primary ); - listener.onResponse( - new CheckpointInfoResponse(copyState.getCheckpoint(), copyState.getMetadataMap(), copyState.getInfosBytes()) - ); } catch (IOException e) { logger.error("Unexpected error computing CopyState", e); Assert.fail("Failed to compute copyState"); + throw new UncheckedIOException(e); + } + + try { + listener.onResponse( + new CheckpointInfoResponse(copyState.getCheckpoint(), copyState.getMetadataMap(), copyState.getInfosBytes()) + ); + } finally { + copyState.decRef(); } } diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationSourceHandlerTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationSourceHandlerTests.java index dfdb0543daf2a..d586767290797 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationSourceHandlerTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationSourceHandlerTests.java @@ -180,6 +180,7 @@ public void onFailure(Exception e) { assertEquals(e.getClass(), OpenSearchException.class); } }); + copyState.decRef(); } public void testReplicationAlreadyRunning() throws IOException { From bb38ed4836496ac70258c2472668325a012ea3ed Mon Sep 17 00:00:00 2001 From: Austin Lee Date: Wed, 30 Aug 2023 12:00:22 -0700 Subject: [PATCH 14/37] Update the minimum version check on SearchExtBuilder support in SearchResponse (Issue # 9328) (#9641) Signed-off-by: Austin Lee --- .../opensearch/search/internal/InternalSearchResponse.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/opensearch/search/internal/InternalSearchResponse.java b/server/src/main/java/org/opensearch/search/internal/InternalSearchResponse.java index 8e3979045f857..3af8fc3854cf1 100644 --- a/server/src/main/java/org/opensearch/search/internal/InternalSearchResponse.java +++ b/server/src/main/java/org/opensearch/search/internal/InternalSearchResponse.java @@ -113,11 +113,11 @@ public void writeTo(StreamOutput out) throws IOException { } private static List readSearchExtBuildersOnOrAfter(StreamInput in) throws IOException { - return (in.getVersion().onOrAfter(Version.V_3_0_0)) ? in.readNamedWriteableList(SearchExtBuilder.class) : Collections.emptyList(); + return (in.getVersion().onOrAfter(Version.V_2_10_0)) ? in.readNamedWriteableList(SearchExtBuilder.class) : Collections.emptyList(); } private static void writeSearchExtBuildersOnOrAfter(StreamOutput out, List searchExtBuilders) throws IOException { - if (out.getVersion().onOrAfter(Version.V_3_0_0)) { + if (out.getVersion().onOrAfter(Version.V_2_10_0)) { out.writeNamedWriteableList(searchExtBuilders); } } From 0274095defad72f17993d052b3505808b342b152 Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Wed, 30 Aug 2023 16:22:38 -0400 Subject: [PATCH 15/37] Allow MockTracingTelemetry to await for asynchronous tasks termination before validating spans (#9561) * Allow MockTracingTelemetry to await for asynchronous tasks termination before validating spans Signed-off-by: Andriy Redko * Address code review comments Signed-off-by: Andriy Redko --------- Signed-off-by: Andriy Redko --- .../test/telemetry/MockTelemetry.java | 29 ++++++++++++--- .../test/telemetry/MockTelemetryPlugin.java | 36 ++++++++++++++++++- .../tracing/MockTracingTelemetry.java | 13 +++++++ 3 files changed, 73 insertions(+), 5 deletions(-) diff --git a/test/framework/src/main/java/org/opensearch/test/telemetry/MockTelemetry.java b/test/framework/src/main/java/org/opensearch/test/telemetry/MockTelemetry.java index de24ea0de77bb..894e8a67cea1f 100644 --- a/test/framework/src/main/java/org/opensearch/test/telemetry/MockTelemetry.java +++ b/test/framework/src/main/java/org/opensearch/test/telemetry/MockTelemetry.java @@ -13,25 +13,46 @@ import org.opensearch.telemetry.metrics.MetricsTelemetry; import org.opensearch.telemetry.tracing.TracingTelemetry; import org.opensearch.test.telemetry.tracing.MockTracingTelemetry; +import org.opensearch.threadpool.ThreadPool; + +import java.util.concurrent.TimeUnit; /** * Mock {@link Telemetry} implementation for testing. */ public class MockTelemetry implements Telemetry { - - private final TelemetrySettings settings; + private final ThreadPool threadPool; /** * Constructor with settings. * @param settings telemetry settings. */ public MockTelemetry(TelemetrySettings settings) { - this.settings = settings; + this(settings, null); + } + + /** + * Constructor with settings. + * @param settings telemetry settings. + * @param threadPool thread pool to watch for termination + */ + public MockTelemetry(TelemetrySettings settings, ThreadPool threadPool) { + this.threadPool = threadPool; } @Override public TracingTelemetry getTracingTelemetry() { - return new MockTracingTelemetry(); + return new MockTracingTelemetry(() -> { + // There could be some asynchronous tasks running that we should await for before the closing + // up the tracer instance. + if (threadPool != null) { + try { + threadPool.awaitTermination(10, TimeUnit.SECONDS); + } catch (final InterruptedException ex) { + Thread.currentThread().interrupt(); + } + } + }); } @Override diff --git a/test/framework/src/main/java/org/opensearch/test/telemetry/MockTelemetryPlugin.java b/test/framework/src/main/java/org/opensearch/test/telemetry/MockTelemetryPlugin.java index 4f483098caf82..ebba9857aa8f1 100644 --- a/test/framework/src/main/java/org/opensearch/test/telemetry/MockTelemetryPlugin.java +++ b/test/framework/src/main/java/org/opensearch/test/telemetry/MockTelemetryPlugin.java @@ -8,18 +8,34 @@ package org.opensearch.test.telemetry; +import org.opensearch.client.Client; +import org.opensearch.cluster.metadata.IndexNameExpressionResolver; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.SetOnce; +import org.opensearch.core.common.io.stream.NamedWriteableRegistry; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.env.Environment; +import org.opensearch.env.NodeEnvironment; import org.opensearch.plugins.Plugin; import org.opensearch.plugins.TelemetryPlugin; +import org.opensearch.repositories.RepositoriesService; +import org.opensearch.script.ScriptService; import org.opensearch.telemetry.Telemetry; import org.opensearch.telemetry.TelemetrySettings; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.watcher.ResourceWatcherService; +import java.util.Collection; +import java.util.Collections; import java.util.Optional; +import java.util.function.Supplier; /** * Mock {@link TelemetryPlugin} implementation for testing. */ public class MockTelemetryPlugin extends Plugin implements TelemetryPlugin { private static final String MOCK_TRACER_NAME = "mock"; + private final SetOnce threadPool = new SetOnce<>(); /** * Base constructor. @@ -28,9 +44,27 @@ public MockTelemetryPlugin() { } + @Override + public Collection createComponents( + Client client, + ClusterService clusterService, + ThreadPool threadPool, + ResourceWatcherService resourceWatcherService, + ScriptService scriptService, + NamedXContentRegistry xContentRegistry, + Environment environment, + NodeEnvironment nodeEnvironment, + NamedWriteableRegistry namedWriteableRegistry, + IndexNameExpressionResolver indexNameExpressionResolver, + Supplier repositoriesServiceSupplier + ) { + this.threadPool.set(threadPool); + return Collections.emptyList(); + } + @Override public Optional getTelemetry(TelemetrySettings settings) { - return Optional.of(new MockTelemetry(settings)); + return Optional.of(new MockTelemetry(settings, threadPool.get())); } @Override diff --git a/test/telemetry/src/main/java/org/opensearch/test/telemetry/tracing/MockTracingTelemetry.java b/test/telemetry/src/main/java/org/opensearch/test/telemetry/tracing/MockTracingTelemetry.java index 9b958bbb40f84..a5e51dd27541b 100644 --- a/test/telemetry/src/main/java/org/opensearch/test/telemetry/tracing/MockTracingTelemetry.java +++ b/test/telemetry/src/main/java/org/opensearch/test/telemetry/tracing/MockTracingTelemetry.java @@ -24,12 +24,22 @@ public class MockTracingTelemetry implements TracingTelemetry { private final SpanProcessor spanProcessor = new StrictCheckSpanProcessor(); + private final Runnable onClose; /** * Base constructor. */ public MockTracingTelemetry() { + this(() -> {}); + } + /** + * Base constructor. + * + * @param onClose on close hook + */ + public MockTracingTelemetry(final Runnable onClose) { + this.onClose = onClose; } @Override @@ -46,6 +56,9 @@ public TracingContextPropagator getContextPropagator() { @Override public void close() { + // Run onClose hook + onClose.run(); + List spanData = ((StrictCheckSpanProcessor) spanProcessor).getFinishedSpanItems(); if (spanData.size() != 0) { TelemetryValidators validators = new TelemetryValidators( From cc007e4511dfb7faec70eff656c02e53a1c410f7 Mon Sep 17 00:00:00 2001 From: Kiran Kumar Muniswamy Reddy Date: Wed, 30 Aug 2023 15:21:46 -0700 Subject: [PATCH 16/37] Add benchmark to measure performance of CustomBinaryDocValuesField (#9426) Benchmarks show that ArrayList performs better than TreeSet. Added a comment on where to find the results and benchmark. Signed-off-by: Kiran Reddy --- .../CustomBinaryDocValuesFieldBenchmark.java | 81 +++++++++++++++++++ .../index/mapper/BinaryFieldMapper.java | 4 + 2 files changed, 85 insertions(+) create mode 100644 benchmarks/src/main/java/org/opensearch/benchmark/index/mapper/CustomBinaryDocValuesFieldBenchmark.java diff --git a/benchmarks/src/main/java/org/opensearch/benchmark/index/mapper/CustomBinaryDocValuesFieldBenchmark.java b/benchmarks/src/main/java/org/opensearch/benchmark/index/mapper/CustomBinaryDocValuesFieldBenchmark.java new file mode 100644 index 0000000000000..7307bec088d02 --- /dev/null +++ b/benchmarks/src/main/java/org/opensearch/benchmark/index/mapper/CustomBinaryDocValuesFieldBenchmark.java @@ -0,0 +1,81 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.benchmark.index.mapper; + +import org.apache.lucene.util.BytesRef; +import org.opensearch.index.mapper.BinaryFieldMapper; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; + +@Warmup(iterations = 1) +@Measurement(iterations = 1) +@Fork(1) +@BenchmarkMode(Mode.Throughput) +@OutputTimeUnit(TimeUnit.MILLISECONDS) +@State(Scope.Thread) +@SuppressWarnings("unused") // invoked by benchmarking framework +public class CustomBinaryDocValuesFieldBenchmark { + + static final String FIELD_NAME = "dummy"; + static final String SEED_VALUE = "seed"; + + @Benchmark + public void add(CustomBinaryDocValuesFieldBenchmark.BenchmarkParameters parameters, Blackhole blackhole) { + // Don't use the parameter binary doc values object. + // Start with a fresh object every call and add maximum number of entries + BinaryFieldMapper.CustomBinaryDocValuesField customBinaryDocValuesField = new BinaryFieldMapper.CustomBinaryDocValuesField( + FIELD_NAME, + new BytesRef(SEED_VALUE).bytes + ); + for (int i = 0; i < parameters.maximumNumberOfEntries; ++i) { + ThreadLocalRandom.current().nextBytes(parameters.bytes); + customBinaryDocValuesField.add(parameters.bytes); + } + } + + @Benchmark + public void binaryValue(CustomBinaryDocValuesFieldBenchmark.BenchmarkParameters parameters, Blackhole blackhole) { + blackhole.consume(parameters.customBinaryDocValuesField.binaryValue()); + } + + @State(Scope.Benchmark) + public static class BenchmarkParameters { + @Param({ "8", "32", "128", "512" }) + int maximumNumberOfEntries; + + @Param({ "8", "32", "128", "512" }) + int entrySize; + + BinaryFieldMapper.CustomBinaryDocValuesField customBinaryDocValuesField; + byte[] bytes; + + @Setup + public void setup() { + customBinaryDocValuesField = new BinaryFieldMapper.CustomBinaryDocValuesField(FIELD_NAME, new BytesRef(SEED_VALUE).bytes); + bytes = new byte[entrySize]; + for (int i = 0; i < maximumNumberOfEntries; ++i) { + ThreadLocalRandom.current().nextBytes(bytes); + customBinaryDocValuesField.add(bytes); + } + } + } +} diff --git a/server/src/main/java/org/opensearch/index/mapper/BinaryFieldMapper.java b/server/src/main/java/org/opensearch/index/mapper/BinaryFieldMapper.java index 62a4af247e0fa..040491f775357 100644 --- a/server/src/main/java/org/opensearch/index/mapper/BinaryFieldMapper.java +++ b/server/src/main/java/org/opensearch/index/mapper/BinaryFieldMapper.java @@ -242,6 +242,10 @@ protected String contentType() { */ public static class CustomBinaryDocValuesField extends CustomDocValuesField { + // We considered using a TreeSet instead of an ArrayList here. + // Benchmarks show that ArrayList performs much better + // For details, see: https://github.com/opensearch-project/OpenSearch/pull/9426 + // Benchmarks are in CustomBinaryDocValuesFiledBenchmark private final ArrayList bytesList; public CustomBinaryDocValuesField(String name, byte[] bytes) { From e563a0c307ade2b3ac339374ddbc2638166f7b49 Mon Sep 17 00:00:00 2001 From: Jay Deng Date: Wed, 30 Aug 2023 19:48:38 -0400 Subject: [PATCH 17/37] Adding concurrent search versions of query count and time metrics (#9622) Signed-off-by: Jay Deng --- CHANGELOG.md | 5 +- .../test/cat.shards/10_basic.yml | 97 +++++++++++- .../search/stats/ConcurrentSearchStatsIT.java | 145 ++++++++++++++++++ .../index/search/stats/SearchStats.java | 54 +++++++ .../index/search/stats/ShardSearchStats.java | 17 ++ .../rest/action/cat/RestIndicesAction.java | 26 ++++ .../rest/action/cat/RestNodesAction.java | 15 ++ .../rest/action/cat/RestShardsAction.java | 15 ++ .../index/search/stats/SearchStatsTests.java | 9 +- .../action/cat/RestShardsActionTests.java | 4 +- 10 files changed, 377 insertions(+), 10 deletions(-) create mode 100644 server/src/internalClusterTest/java/org/opensearch/search/stats/ConcurrentSearchStatsIT.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 1d11c28ec2429..e886ee5ffc97c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -89,8 +89,9 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [Segment Replication] Support realtime reads for GET requests ([#9212](https://github.com/opensearch-project/OpenSearch/pull/9212)) - [Feature] Expose term frequency in Painless script score context ([#9081](https://github.com/opensearch-project/OpenSearch/pull/9081)) - Add support for reading partial files to HDFS repository ([#9513](https://github.com/opensearch-project/OpenSearch/issues/9513)) -- Add support for extensions to search responses using SearchExtBuilder ([#9379](https://github.com/opensearch-project/OpenSearch/pull/9379)) +- Add support for extensions to search responses using SearchExtBuilder ([#9379](https://github.com/opensearch-project/OpenSearch/pull/9379)) - [BWC and API enforcement] Decorate the existing APIs with proper annotations (part 1) ([#9520](https://github.com/opensearch-project/OpenSearch/pull/9520)) +- Add concurrent segment search related metrics to node and index stats ([#9622](https://github.com/opensearch-project/OpenSearch/issues/9622)) ### Dependencies - Bump `org.apache.logging.log4j:log4j-core` from 2.17.1 to 2.20.0 ([#8307](https://github.com/opensearch-project/OpenSearch/pull/8307)) @@ -182,4 +183,4 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ### Security [Unreleased 3.0]: https://github.com/opensearch-project/OpenSearch/compare/2.x...HEAD -[Unreleased 2.x]: https://github.com/opensearch-project/OpenSearch/compare/2.10...2.x \ No newline at end of file +[Unreleased 2.x]: https://github.com/opensearch-project/OpenSearch/compare/2.10...2.x diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.shards/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.shards/10_basic.yml index 189215b6562a3..d9f79124c58d9 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/cat.shards/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/cat.shards/10_basic.yml @@ -1,14 +1,105 @@ ---- "Help": - skip: - version: " - 2.3.99" + version: " - 2.99.99" + reason: concurrent search stats were added in 3.0.0 + features: node_selector + - do: + cat.shards: + help: true + node_selector: + version: "3.0.0 - " + + - match: + $body: | + /^ index .+ \n + shard .+ \n + prirep .+ \n + state .+ \n + docs .+ \n + store .+ \n + ip .+ \n + id .+ \n + node .+ \n + sync_id .+ \n + unassigned.reason .+ \n + unassigned.at .+ \n + unassigned.for .+ \n + unassigned.details .+ \n + recoverysource.type .+ \n + completion.size .+ \n + fielddata.memory_size .+ \n + fielddata.evictions .+ \n + query_cache.memory_size .+ \n + query_cache.evictions .+ \n + flush.total .+ \n + flush.total_time .+ \n + get.current .+ \n + get.time .+ \n + get.total .+ \n + get.exists_time .+ \n + get.exists_total .+ \n + get.missing_time .+ \n + get.missing_total .+ \n + indexing.delete_current .+ \n + indexing.delete_time .+ \n + indexing.delete_total .+ \n + indexing.index_current .+ \n + indexing.index_time .+ \n + indexing.index_total .+ \n + indexing.index_failed .+ \n + merges.current .+ \n + merges.current_docs .+ \n + merges.current_size .+ \n + merges.total .+ \n + merges.total_docs .+ \n + merges.total_size .+ \n + merges.total_time .+ \n + refresh.total .+ \n + refresh.time .+ \n + refresh.external_total .+ \n + refresh.external_time .+ \n + refresh.listeners .+ \n + search.fetch_current .+ \n + search.fetch_time .+ \n + search.fetch_total .+ \n + search.open_contexts .+ \n + search.query_current .+ \n + search.query_time .+ \n + search.query_total .+ \n + search.concurrent_query_current .+ \n + search.concurrent_query_time .+ \n + search.concurrent_query_total .+ \n + search.scroll_current .+ \n + search.scroll_time .+ \n + search.scroll_total .+ \n + search.point_in_time_current .+ \n + search.point_in_time_time .+ \n + search.point_in_time_total .+ \n + segments.count .+ \n + segments.memory .+ \n + segments.index_writer_memory .+ \n + segments.version_map_memory .+ \n + segments.fixed_bitset_memory .+ \n + seq_no.max .+ \n + seq_no.local_checkpoint .+ \n + seq_no.global_checkpoint .+ \n + warmer.current .+ \n + warmer.total .+ \n + warmer.total_time .+ \n + path.data .+ \n + path.state .+ \n + $/ +--- +"Help between 2.4.0 - 2.99.99": + - skip: + version: " - 2.3.99 , 3.0.0 - " reason: point in time stats were added in 2.4.0 features: node_selector - do: cat.shards: help: true node_selector: - version: "2.4.0 - " + version: "2.4.0 - 2.99.99" - match: $body: | diff --git a/server/src/internalClusterTest/java/org/opensearch/search/stats/ConcurrentSearchStatsIT.java b/server/src/internalClusterTest/java/org/opensearch/search/stats/ConcurrentSearchStatsIT.java new file mode 100644 index 0000000000000..352fe78b2680d --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/search/stats/ConcurrentSearchStatsIT.java @@ -0,0 +1,145 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.search.stats; + +import org.opensearch.action.admin.indices.stats.IndicesStatsRequestBuilder; +import org.opensearch.action.admin.indices.stats.IndicesStatsResponse; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.FeatureFlags; +import org.opensearch.index.IndexModule; +import org.opensearch.index.IndexSettings; +import org.opensearch.indices.IndicesQueryCache; +import org.opensearch.indices.IndicesService; +import org.opensearch.plugins.Plugin; +import org.opensearch.script.MockScriptPlugin; +import org.opensearch.script.Script; +import org.opensearch.script.ScriptType; +import org.opensearch.test.InternalSettingsPlugin; +import org.opensearch.test.OpenSearchIntegTestCase; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import java.util.function.Function; + +import static org.opensearch.index.query.QueryBuilders.scriptQuery; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.lessThan; + +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.SUITE, numDataNodes = 2, numClientNodes = 0) +public class ConcurrentSearchStatsIT extends OpenSearchIntegTestCase { + + @Override + protected Collection> nodePlugins() { + return Arrays.asList(ScriptedDelayedPlugin.class, InternalSettingsPlugin.class); + } + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + // Filter/Query cache is cleaned periodically, default is 60s, so make sure it runs often. Thread.sleep for 60s is bad + return Settings.builder() + .put(super.nodeSettings(nodeOrdinal)) + .put(IndicesService.INDICES_CACHE_CLEAN_INTERVAL_SETTING.getKey(), "1ms") + .put(IndicesQueryCache.INDICES_QUERIES_CACHE_ALL_SEGMENTS_SETTING.getKey(), true) + .build(); + } + + @Override + public Settings indexSettings() { + return Settings.builder() + .put(super.indexSettings()) + .put(IndexModule.INDEX_QUERY_CACHE_EVERYTHING_SETTING.getKey(), false) + .put(IndexModule.INDEX_QUERY_CACHE_ENABLED_SETTING.getKey(), false) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0) + .build(); + } + + @Override + protected Settings featureFlagSettings() { + return Settings.builder().put(super.featureFlagSettings()).put(FeatureFlags.CONCURRENT_SEGMENT_SEARCH, "true").build(); + } + + public void testConcurrentQueryCount() throws Exception { + int NUM_SHARDS = randomIntBetween(1, 5); + createIndex( + "test1", + Settings.builder() + .put(indexSettings()) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, NUM_SHARDS) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .build() + ); + createIndex( + "test2", + Settings.builder() + .put(indexSettings()) + .put("search.concurrent_segment_search.enabled", false) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, NUM_SHARDS) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .build() + ); + + ensureGreen(); + + indexRandom( + false, + true, + client().prepareIndex("test1").setId("1").setSource("foo", "bar"), + client().prepareIndex("test1").setId("2").setSource("foo", "baz"), + client().prepareIndex("test2").setId("1").setSource("foo", "bar"), + client().prepareIndex("test2").setId("2").setSource("foo", "baz") + ); + + refresh(); + + // Search with custom plugin to ensure that queryTime is significant + client().prepareSearch("_all") + .setQuery(scriptQuery(new Script(ScriptType.INLINE, "mockscript", ScriptedDelayedPlugin.SCRIPT_NAME, Collections.emptyMap()))) + .execute() + .actionGet(); + client().prepareSearch("test1") + .setQuery(scriptQuery(new Script(ScriptType.INLINE, "mockscript", ScriptedDelayedPlugin.SCRIPT_NAME, Collections.emptyMap()))) + .execute() + .actionGet(); + client().prepareSearch("test2") + .setQuery(scriptQuery(new Script(ScriptType.INLINE, "mockscript", ScriptedDelayedPlugin.SCRIPT_NAME, Collections.emptyMap()))) + .execute() + .actionGet(); + + IndicesStatsRequestBuilder builder = client().admin().indices().prepareStats(); + IndicesStatsResponse stats = builder.execute().actionGet(); + + assertEquals(4 * NUM_SHARDS, stats.getTotal().search.getTotal().getQueryCount()); + assertEquals(2 * NUM_SHARDS, stats.getTotal().search.getTotal().getConcurrentQueryCount()); + assertThat(stats.getTotal().search.getTotal().getQueryTimeInMillis(), greaterThan(0L)); + assertThat(stats.getTotal().search.getTotal().getConcurrentQueryTimeInMillis(), greaterThan(0L)); + assertThat( + stats.getTotal().search.getTotal().getConcurrentQueryTimeInMillis(), + lessThan(stats.getTotal().search.getTotal().getQueryTimeInMillis()) + ); + } + + public static class ScriptedDelayedPlugin extends MockScriptPlugin { + static final String SCRIPT_NAME = "search_timeout"; + + @Override + public Map, Object>> pluginScripts() { + return Collections.singletonMap(SCRIPT_NAME, params -> { + try { + Thread.sleep(500); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + return true; + }); + } + } +} diff --git a/server/src/main/java/org/opensearch/index/search/stats/SearchStats.java b/server/src/main/java/org/opensearch/index/search/stats/SearchStats.java index e41deb514fbef..a04a0c472db19 100644 --- a/server/src/main/java/org/opensearch/index/search/stats/SearchStats.java +++ b/server/src/main/java/org/opensearch/index/search/stats/SearchStats.java @@ -67,6 +67,10 @@ public static class Stats implements Writeable, ToXContentFragment { private long queryTimeInMillis; private long queryCurrent; + private long concurrentQueryCount; + private long concurrentQueryTimeInMillis; + private long concurrentQueryCurrent; + private long fetchCount; private long fetchTimeInMillis; private long fetchCurrent; @@ -91,6 +95,9 @@ public Stats( long queryCount, long queryTimeInMillis, long queryCurrent, + long concurrentQueryCount, + long concurrentQueryTimeInMillis, + long concurrentQueryCurrent, long fetchCount, long fetchTimeInMillis, long fetchCurrent, @@ -108,6 +115,10 @@ public Stats( this.queryTimeInMillis = queryTimeInMillis; this.queryCurrent = queryCurrent; + this.concurrentQueryCount = concurrentQueryCount; + this.concurrentQueryTimeInMillis = concurrentQueryTimeInMillis; + this.concurrentQueryCurrent = concurrentQueryCurrent; + this.fetchCount = fetchCount; this.fetchTimeInMillis = fetchTimeInMillis; this.fetchCurrent = fetchCurrent; @@ -147,6 +158,12 @@ private Stats(StreamInput in) throws IOException { pitTimeInMillis = in.readVLong(); pitCurrent = in.readVLong(); } + + if (in.getVersion().onOrAfter(Version.V_3_0_0)) { + concurrentQueryCount = in.readVLong(); + concurrentQueryTimeInMillis = in.readVLong(); + concurrentQueryCurrent = in.readVLong(); + } } public void add(Stats stats) { @@ -154,6 +171,10 @@ public void add(Stats stats) { queryTimeInMillis += stats.queryTimeInMillis; queryCurrent += stats.queryCurrent; + concurrentQueryCount += stats.concurrentQueryCount; + concurrentQueryTimeInMillis += stats.concurrentQueryTimeInMillis; + concurrentQueryCurrent += stats.concurrentQueryCurrent; + fetchCount += stats.fetchCount; fetchTimeInMillis += stats.fetchTimeInMillis; fetchCurrent += stats.fetchCurrent; @@ -175,6 +196,9 @@ public void addForClosingShard(Stats stats) { queryCount += stats.queryCount; queryTimeInMillis += stats.queryTimeInMillis; + concurrentQueryCount += stats.concurrentQueryCount; + concurrentQueryTimeInMillis += stats.concurrentQueryTimeInMillis; + fetchCount += stats.fetchCount; fetchTimeInMillis += stats.fetchTimeInMillis; @@ -207,6 +231,22 @@ public long getQueryCurrent() { return queryCurrent; } + public long getConcurrentQueryCount() { + return concurrentQueryCount; + } + + public TimeValue getConcurrentQueryTime() { + return new TimeValue(concurrentQueryTimeInMillis); + } + + public long getConcurrentQueryTimeInMillis() { + return concurrentQueryTimeInMillis; + } + + public long getConcurrentQueryCurrent() { + return concurrentQueryCurrent; + } + public long getFetchCount() { return fetchCount; } @@ -298,6 +338,12 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVLong(pitTimeInMillis); out.writeVLong(pitCurrent); } + + if (out.getVersion().onOrAfter(Version.V_3_0_0)) { + out.writeVLong(concurrentQueryCount); + out.writeVLong(concurrentQueryTimeInMillis); + out.writeVLong(concurrentQueryCurrent); + } } @Override @@ -306,6 +352,10 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.humanReadableField(Fields.QUERY_TIME_IN_MILLIS, Fields.QUERY_TIME, getQueryTime()); builder.field(Fields.QUERY_CURRENT, queryCurrent); + builder.field(Fields.CONCURRENT_QUERY_TOTAL, concurrentQueryCount); + builder.humanReadableField(Fields.CONCURRENT_QUERY_TIME_IN_MILLIS, Fields.CONCURRENT_QUERY_TIME, getConcurrentQueryTime()); + builder.field(Fields.CONCURRENT_QUERY_CURRENT, concurrentQueryCurrent); + builder.field(Fields.FETCH_TOTAL, fetchCount); builder.humanReadableField(Fields.FETCH_TIME_IN_MILLIS, Fields.FETCH_TIME, getFetchTime()); builder.field(Fields.FETCH_CURRENT, fetchCurrent); @@ -430,6 +480,10 @@ static final class Fields { static final String QUERY_TIME = "query_time"; static final String QUERY_TIME_IN_MILLIS = "query_time_in_millis"; static final String QUERY_CURRENT = "query_current"; + static final String CONCURRENT_QUERY_TOTAL = "concurrent_query_total"; + static final String CONCURRENT_QUERY_TIME = "concurrent_query_time"; + static final String CONCURRENT_QUERY_TIME_IN_MILLIS = "concurrent_query_time_in_millis"; + static final String CONCURRENT_QUERY_CURRENT = "concurrent_query_current"; static final String FETCH_TOTAL = "fetch_total"; static final String FETCH_TIME = "fetch_time"; static final String FETCH_TIME_IN_MILLIS = "fetch_time_in_millis"; diff --git a/server/src/main/java/org/opensearch/index/search/stats/ShardSearchStats.java b/server/src/main/java/org/opensearch/index/search/stats/ShardSearchStats.java index 6f6ebd5545c7a..c0d4669413de0 100644 --- a/server/src/main/java/org/opensearch/index/search/stats/ShardSearchStats.java +++ b/server/src/main/java/org/opensearch/index/search/stats/ShardSearchStats.java @@ -91,6 +91,9 @@ public void onPreQueryPhase(SearchContext searchContext) { statsHolder.suggestCurrent.inc(); } else { statsHolder.queryCurrent.inc(); + if (searchContext.shouldUseConcurrentSearch()) { + statsHolder.concurrentQueryCurrent.inc(); + } } }); } @@ -104,6 +107,10 @@ public void onFailedQueryPhase(SearchContext searchContext) { } else { statsHolder.queryCurrent.dec(); assert statsHolder.queryCurrent.count() >= 0; + if (searchContext.shouldUseConcurrentSearch()) { + statsHolder.concurrentQueryCurrent.dec(); + assert statsHolder.concurrentQueryCurrent.count() >= 0; + } } }); } @@ -119,6 +126,11 @@ public void onQueryPhase(SearchContext searchContext, long tookInNanos) { statsHolder.queryMetric.inc(tookInNanos); statsHolder.queryCurrent.dec(); assert statsHolder.queryCurrent.count() >= 0; + if (searchContext.shouldUseConcurrentSearch()) { + statsHolder.concurrentQueryMetric.inc(tookInNanos); + statsHolder.concurrentQueryCurrent.dec(); + assert statsHolder.concurrentQueryCurrent.count() >= 0; + } } }); } @@ -206,6 +218,7 @@ public void onFreePitContext(ReaderContext readerContext) { */ static final class StatsHolder { final MeanMetric queryMetric = new MeanMetric(); + final MeanMetric concurrentQueryMetric = new MeanMetric(); final MeanMetric fetchMetric = new MeanMetric(); /* We store scroll statistics in microseconds because with nanoseconds we run the risk of overflowing the total stats if there are * many scrolls. For example, on a system with 2^24 scrolls that have been executed, each executing for 2^10 seconds, then using @@ -218,6 +231,7 @@ static final class StatsHolder { final MeanMetric pitMetric = new MeanMetric(); final MeanMetric suggestMetric = new MeanMetric(); final CounterMetric queryCurrent = new CounterMetric(); + final CounterMetric concurrentQueryCurrent = new CounterMetric(); final CounterMetric fetchCurrent = new CounterMetric(); final CounterMetric scrollCurrent = new CounterMetric(); final CounterMetric pitCurrent = new CounterMetric(); @@ -228,6 +242,9 @@ SearchStats.Stats stats() { queryMetric.count(), TimeUnit.NANOSECONDS.toMillis(queryMetric.sum()), queryCurrent.count(), + concurrentQueryMetric.count(), + TimeUnit.NANOSECONDS.toMillis(concurrentQueryMetric.sum()), + concurrentQueryCurrent.count(), fetchMetric.count(), TimeUnit.NANOSECONDS.toMillis(fetchMetric.sum()), fetchCurrent.count(), diff --git a/server/src/main/java/org/opensearch/rest/action/cat/RestIndicesAction.java b/server/src/main/java/org/opensearch/rest/action/cat/RestIndicesAction.java index 2d2dde12dd603..ff8d2051a84bf 100644 --- a/server/src/main/java/org/opensearch/rest/action/cat/RestIndicesAction.java +++ b/server/src/main/java/org/opensearch/rest/action/cat/RestIndicesAction.java @@ -581,6 +581,23 @@ protected Table getTableWithHeader(final RestRequest request) { "sibling:pri;alias:sqto,searchQueryTotal;default:false;text-align:right;desc:total query phase ops" ); table.addCell("pri.search.query_total", "default:false;text-align:right;desc:total query phase ops"); + table.addCell( + "search.concurrent_query_current", + "sibling:pri;alias:scqc,searchConcurrentQueryCurrent;default:false;text-align:right;desc:current concurrent query phase ops" + ); + table.addCell("pri.search.concurrent_query_current", "default:false;text-align:right;desc:current concurrent query phase ops"); + + table.addCell( + "search.concurrent_query_time", + "sibling:pri;alias:scqti,searchConcurrentQueryTime;default:false;text-align:right;desc:time spent in concurrent query phase" + ); + table.addCell("pri.search.concurrent_query_time", "default:false;text-align:right;desc:time spent in concurrent query phase"); + + table.addCell( + "search.concurrent_query_total", + "sibling:pri;alias:scqto,searchConcurrentQueryTotal;default:false;text-align:right;desc:total query phase ops" + ); + table.addCell("pri.search.concurrent_query_total", "default:false;text-align:right;desc:total query phase ops"); table.addCell( "search.scroll_current", @@ -890,6 +907,15 @@ Table buildTable( table.addCell(totalStats.getSearch() == null ? null : totalStats.getSearch().getTotal().getQueryCount()); table.addCell(primaryStats.getSearch() == null ? null : primaryStats.getSearch().getTotal().getQueryCount()); + table.addCell(totalStats.getSearch() == null ? null : totalStats.getSearch().getTotal().getConcurrentQueryCurrent()); + table.addCell(primaryStats.getSearch() == null ? null : primaryStats.getSearch().getTotal().getConcurrentQueryCurrent()); + + table.addCell(totalStats.getSearch() == null ? null : totalStats.getSearch().getTotal().getConcurrentQueryTime()); + table.addCell(primaryStats.getSearch() == null ? null : primaryStats.getSearch().getTotal().getConcurrentQueryTime()); + + table.addCell(totalStats.getSearch() == null ? null : totalStats.getSearch().getTotal().getConcurrentQueryCount()); + table.addCell(primaryStats.getSearch() == null ? null : primaryStats.getSearch().getTotal().getConcurrentQueryCount()); + table.addCell(totalStats.getSearch() == null ? null : totalStats.getSearch().getTotal().getScrollCurrent()); table.addCell(primaryStats.getSearch() == null ? null : primaryStats.getSearch().getTotal().getScrollCurrent()); diff --git a/server/src/main/java/org/opensearch/rest/action/cat/RestNodesAction.java b/server/src/main/java/org/opensearch/rest/action/cat/RestNodesAction.java index 5a0f85fea8e5d..6aefec817e897 100644 --- a/server/src/main/java/org/opensearch/rest/action/cat/RestNodesAction.java +++ b/server/src/main/java/org/opensearch/rest/action/cat/RestNodesAction.java @@ -303,6 +303,18 @@ protected Table getTableWithHeader(final RestRequest request) { table.addCell("search.query_current", "alias:sqc,searchQueryCurrent;default:false;text-align:right;desc:current query phase ops"); table.addCell("search.query_time", "alias:sqti,searchQueryTime;default:false;text-align:right;desc:time spent in query phase"); table.addCell("search.query_total", "alias:sqto,searchQueryTotal;default:false;text-align:right;desc:total query phase ops"); + table.addCell( + "search.concurrent_query_current", + "alias:scqc,searchConcurrentQueryCurrent;default:false;text-align:right;desc:current concurrent query phase ops" + ); + table.addCell( + "search.concurrent_query_time", + "alias:scqti,searchConcurrentQueryTime;default:false;text-align:right;desc:time spent in concurrent query phase" + ); + table.addCell( + "search.concurrent_query_total", + "alias:scqto,searchConcurrentQueryTotal;default:false;text-align:right;desc:total concurrent query phase ops" + ); table.addCell("search.scroll_current", "alias:scc,searchScrollCurrent;default:false;text-align:right;desc:open scroll contexts"); table.addCell( "search.scroll_time", @@ -529,6 +541,9 @@ Table buildTable( table.addCell(searchStats == null ? null : searchStats.getTotal().getQueryCurrent()); table.addCell(searchStats == null ? null : searchStats.getTotal().getQueryTime()); table.addCell(searchStats == null ? null : searchStats.getTotal().getQueryCount()); + table.addCell(searchStats == null ? null : searchStats.getTotal().getConcurrentQueryCurrent()); + table.addCell(searchStats == null ? null : searchStats.getTotal().getConcurrentQueryTime()); + table.addCell(searchStats == null ? null : searchStats.getTotal().getConcurrentQueryCount()); table.addCell(searchStats == null ? null : searchStats.getTotal().getScrollCurrent()); table.addCell(searchStats == null ? null : searchStats.getTotal().getScrollTime()); table.addCell(searchStats == null ? null : searchStats.getTotal().getScrollCount()); diff --git a/server/src/main/java/org/opensearch/rest/action/cat/RestShardsAction.java b/server/src/main/java/org/opensearch/rest/action/cat/RestShardsAction.java index c5750ef4093c5..b3fd66054c03d 100644 --- a/server/src/main/java/org/opensearch/rest/action/cat/RestShardsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/cat/RestShardsAction.java @@ -219,6 +219,18 @@ protected Table getTableWithHeader(final RestRequest request) { table.addCell("search.query_current", "alias:sqc,searchQueryCurrent;default:false;text-align:right;desc:current query phase ops"); table.addCell("search.query_time", "alias:sqti,searchQueryTime;default:false;text-align:right;desc:time spent in query phase"); table.addCell("search.query_total", "alias:sqto,searchQueryTotal;default:false;text-align:right;desc:total query phase ops"); + table.addCell( + "search.concurrent_query_current", + "alias:scqc,searchConcurrentQueryCurrent;default:false;text-align:right;desc:current concurrent query phase ops" + ); + table.addCell( + "search.concurrent_query_time", + "alias:scqti,searchConcurrentQueryTime;default:false;text-align:right;desc:time spent in concurrent query phase" + ); + table.addCell( + "search.concurrent_query_total", + "alias:scqto,searchConcurrentQueryTotal;default:false;text-align:right;desc:total concurrent query phase ops" + ); table.addCell("search.scroll_current", "alias:scc,searchScrollCurrent;default:false;text-align:right;desc:open scroll contexts"); table.addCell( "search.scroll_time", @@ -399,6 +411,9 @@ Table buildTable(RestRequest request, ClusterStateResponse state, IndicesStatsRe table.addCell(getOrNull(commonStats, CommonStats::getSearch, i -> i.getTotal().getQueryCurrent())); table.addCell(getOrNull(commonStats, CommonStats::getSearch, i -> i.getTotal().getQueryTime())); table.addCell(getOrNull(commonStats, CommonStats::getSearch, i -> i.getTotal().getQueryCount())); + table.addCell(getOrNull(commonStats, CommonStats::getSearch, i -> i.getTotal().getConcurrentQueryCurrent())); + table.addCell(getOrNull(commonStats, CommonStats::getSearch, i -> i.getTotal().getConcurrentQueryTime())); + table.addCell(getOrNull(commonStats, CommonStats::getSearch, i -> i.getTotal().getConcurrentQueryCount())); table.addCell(getOrNull(commonStats, CommonStats::getSearch, i -> i.getTotal().getScrollCurrent())); table.addCell(getOrNull(commonStats, CommonStats::getSearch, i -> i.getTotal().getScrollTime())); table.addCell(getOrNull(commonStats, CommonStats::getSearch, i -> i.getTotal().getScrollCount())); diff --git a/server/src/test/java/org/opensearch/index/search/stats/SearchStatsTests.java b/server/src/test/java/org/opensearch/index/search/stats/SearchStatsTests.java index 7d2d8e38d066e..e56fc28235182 100644 --- a/server/src/test/java/org/opensearch/index/search/stats/SearchStatsTests.java +++ b/server/src/test/java/org/opensearch/index/search/stats/SearchStatsTests.java @@ -45,9 +45,9 @@ public void testShardLevelSearchGroupStats() throws Exception { // let's create two dummy search stats with groups Map groupStats1 = new HashMap<>(); Map groupStats2 = new HashMap<>(); - groupStats2.put("group1", new Stats(1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1)); - SearchStats searchStats1 = new SearchStats(new Stats(1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1), 0, groupStats1); - SearchStats searchStats2 = new SearchStats(new Stats(1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1), 0, groupStats2); + groupStats2.put("group1", new Stats(1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1)); + SearchStats searchStats1 = new SearchStats(new Stats(1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1), 0, groupStats1); + SearchStats searchStats2 = new SearchStats(new Stats(1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1), 0, groupStats2); // adding these two search stats and checking group stats are correct searchStats1.add(searchStats2); @@ -69,6 +69,9 @@ private static void assertStats(Stats stats, long equalTo) { assertEquals(equalTo, stats.getQueryCount()); assertEquals(equalTo, stats.getQueryTimeInMillis()); assertEquals(equalTo, stats.getQueryCurrent()); + assertEquals(equalTo, stats.getConcurrentQueryCount()); + assertEquals(equalTo, stats.getConcurrentQueryTimeInMillis()); + assertEquals(equalTo, stats.getConcurrentQueryCurrent()); assertEquals(equalTo, stats.getFetchCount()); assertEquals(equalTo, stats.getFetchTimeInMillis()); assertEquals(equalTo, stats.getFetchCurrent()); diff --git a/server/src/test/java/org/opensearch/rest/action/cat/RestShardsActionTests.java b/server/src/test/java/org/opensearch/rest/action/cat/RestShardsActionTests.java index a8679a087216d..3895a64474cc4 100644 --- a/server/src/test/java/org/opensearch/rest/action/cat/RestShardsActionTests.java +++ b/server/src/test/java/org/opensearch/rest/action/cat/RestShardsActionTests.java @@ -134,8 +134,8 @@ public void testBuildTable() { assertThat(row.get(3).value, equalTo(shardRouting.state())); assertThat(row.get(6).value, equalTo(localNode.getHostAddress())); assertThat(row.get(7).value, equalTo(localNode.getId())); - assertThat(row.get(72).value, equalTo(shardStats.getDataPath())); - assertThat(row.get(73).value, equalTo(shardStats.getStatePath())); + assertThat(row.get(75).value, equalTo(shardStats.getDataPath())); + assertThat(row.get(76).value, equalTo(shardStats.getStatePath())); } } } From ff654038a499da001fa1c2df0ef5b9d10ed04458 Mon Sep 17 00:00:00 2001 From: Poojita Raj Date: Wed, 30 Aug 2023 17:28:38 -0700 Subject: [PATCH 18/37] [Segment Replication] Handle failover in mixed cluster mode (#9536) * pick oldest OS version replica to promote as primary Signed-off-by: Poojita Raj * add test Signed-off-by: Poojita Raj * refactor Signed-off-by: Poojita Raj * refactor to avoid coupling Signed-off-by: Poojita Raj * add comments Signed-off-by: Poojita Raj --------- Signed-off-by: Poojita Raj --- .../action/get/TransportGetAction.java | 12 ++- .../action/get/TransportMultiGetAction.java | 9 +- .../org/opensearch/cluster/ClusterState.java | 16 ---- .../opensearch/cluster/metadata/Metadata.java | 17 ++++ .../cluster/routing/RoutingNodes.java | 39 ++++++++- .../action/get/TransportGetActionTests.java | 14 +-- .../get/TransportMultiGetActionTests.java | 85 +++++++++++++------ .../opensearch/cluster/ClusterStateTests.java | 34 -------- .../cluster/metadata/MetadataTests.java | 24 ++++++ .../allocation/FailedNodeRoutingTests.java | 37 ++++++-- .../allocation/FailedShardsRoutingTests.java | 57 ++++++++++--- 11 files changed, 233 insertions(+), 111 deletions(-) diff --git a/server/src/main/java/org/opensearch/action/get/TransportGetAction.java b/server/src/main/java/org/opensearch/action/get/TransportGetAction.java index 0c444732fb12b..00a795c86356f 100644 --- a/server/src/main/java/org/opensearch/action/get/TransportGetAction.java +++ b/server/src/main/java/org/opensearch/action/get/TransportGetAction.java @@ -37,6 +37,7 @@ import org.opensearch.action.support.single.shard.TransportSingleShardAction; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexNameExpressionResolver; +import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.routing.Preference; import org.opensearch.cluster.routing.ShardIterator; import org.opensearch.cluster.service.ClusterService; @@ -92,8 +93,8 @@ protected boolean resolveIndex(GetRequest request) { /** * Returns true if GET request should be routed to primary shards, else false. */ - protected static boolean shouldForcePrimaryRouting(ClusterState state, boolean realtime, String preference, String indexName) { - return state.isSegmentReplicationEnabled(indexName) && realtime && preference == null; + protected static boolean shouldForcePrimaryRouting(Metadata metadata, boolean realtime, String preference, String indexName) { + return metadata.isSegmentReplicationEnabled(indexName) && realtime && preference == null; } @Override @@ -101,7 +102,12 @@ protected ShardIterator shards(ClusterState state, InternalRequest request) { final String preference; // route realtime GET requests when segment replication is enabled to primary shards, // iff there are no other preferences/routings enabled for routing to a specific shard - if (shouldForcePrimaryRouting(state, request.request().realtime, request.request().preference(), request.concreteIndex())) { + if (shouldForcePrimaryRouting( + state.getMetadata(), + request.request().realtime, + request.request().preference(), + request.concreteIndex() + )) { preference = Preference.PRIMARY.type(); } else { preference = request.request().preference(); diff --git a/server/src/main/java/org/opensearch/action/get/TransportMultiGetAction.java b/server/src/main/java/org/opensearch/action/get/TransportMultiGetAction.java index a1a74208dc725..8bbfef381aea8 100644 --- a/server/src/main/java/org/opensearch/action/get/TransportMultiGetAction.java +++ b/server/src/main/java/org/opensearch/action/get/TransportMultiGetAction.java @@ -38,6 +38,7 @@ import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.block.ClusterBlockLevel; import org.opensearch.cluster.metadata.IndexNameExpressionResolver; +import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.routing.Preference; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.inject.Inject; @@ -51,8 +52,6 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; -import static org.opensearch.action.get.TransportGetAction.shouldForcePrimaryRouting; - /** * Perform the multi get action. * @@ -78,6 +77,10 @@ public TransportMultiGetAction( this.indexNameExpressionResolver = resolver; } + protected static boolean shouldForcePrimaryRouting(Metadata metadata, boolean realtime, String preference, String indexName) { + return metadata.isSegmentReplicationEnabled(indexName) && realtime && preference == null; + } + @Override protected void doExecute(Task task, final MultiGetRequest request, final ActionListener listener) { ClusterState clusterState = clusterService.state(); @@ -112,7 +115,7 @@ protected void doExecute(Task task, final MultiGetRequest request, final ActionL MultiGetShardRequest shardRequest = shardRequests.get(shardId); if (shardRequest == null) { - if (shouldForcePrimaryRouting(clusterState, request.realtime, request.preference, concreteSingleIndex)) { + if (shouldForcePrimaryRouting(clusterState.getMetadata(), request.realtime(), request.preference(), concreteSingleIndex)) { request.preference(Preference.PRIMARY.type()); } shardRequest = new MultiGetShardRequest(request, shardId.getIndexName(), shardId.getId()); diff --git a/server/src/main/java/org/opensearch/cluster/ClusterState.java b/server/src/main/java/org/opensearch/cluster/ClusterState.java index 2fd58d3db4975..1b87a60c2ccf5 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterState.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterState.java @@ -61,7 +61,6 @@ import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.discovery.Discovery; -import org.opensearch.indices.replication.common.ReplicationType; import java.io.IOException; import java.util.Collections; @@ -410,21 +409,6 @@ public boolean supersedes(ClusterState other) { } - /** - * Utility to identify whether input index belongs to SEGMENT replication in established cluster state. - * - * @param indexName Index name - * @return true if index belong SEGMENT replication, false otherwise - */ - public boolean isSegmentReplicationEnabled(String indexName) { - return Optional.ofNullable(this.getMetadata().index(indexName)) - .map( - indexMetadata -> ReplicationType.parseString(indexMetadata.getSettings().get(IndexMetadata.SETTING_REPLICATION_TYPE)) - .equals(ReplicationType.SEGMENT) - ) - .orElse(false); - } - /** * Metrics for cluster state. * diff --git a/server/src/main/java/org/opensearch/cluster/metadata/Metadata.java b/server/src/main/java/org/opensearch/cluster/metadata/Metadata.java index 13a27f76a181c..146193b8d22c4 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/Metadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/Metadata.java @@ -66,6 +66,7 @@ import org.opensearch.core.xcontent.XContentParser; import org.opensearch.gateway.MetadataStateFormat; import org.opensearch.index.IndexNotFoundException; +import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.plugins.MapperPlugin; import java.io.IOException; @@ -107,6 +108,22 @@ public class Metadata implements Iterable, Diffable, To public static final String UNKNOWN_CLUSTER_UUID = Strings.UNKNOWN_UUID_VALUE; public static final Pattern NUMBER_PATTERN = Pattern.compile("[0-9]+$"); + /** + * Utility to identify whether input index uses SEGMENT replication strategy in established cluster state metadata. + * Note: Method intended for use by other plugins as well. + * + * @param indexName Index name + * @return true if index uses SEGMENT replication, false otherwise + */ + public boolean isSegmentReplicationEnabled(String indexName) { + return Optional.ofNullable(index(indexName)) + .map( + indexMetadata -> ReplicationType.parseString(indexMetadata.getSettings().get(IndexMetadata.SETTING_REPLICATION_TYPE)) + .equals(ReplicationType.SEGMENT) + ) + .orElse(false); + } + /** * Context of the XContent. * diff --git a/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java b/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java index 981e21537c078..4f7b935f15f93 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java +++ b/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java @@ -82,6 +82,7 @@ * @opensearch.internal */ public class RoutingNodes implements Iterable { + private final Metadata metadata; private final Map nodesToShards = new HashMap<>(); @@ -107,6 +108,7 @@ public RoutingNodes(ClusterState clusterState) { } public RoutingNodes(ClusterState clusterState, boolean readOnly) { + this.metadata = clusterState.getMetadata(); this.readOnly = readOnly; final RoutingTable routingTable = clusterState.routingTable(); this.nodesPerAttributeNames = Collections.synchronizedMap(new HashMap<>()); @@ -368,9 +370,9 @@ public ShardRouting activePrimary(ShardId shardId) { * Returns one active replica shard for the given shard id or null if * no active replica is found. * - * Since replicas could possibly be on nodes with a older version of OpenSearch than - * the primary is, this will return replicas on the highest version of OpenSearch. - * + * Since replicas could possibly be on nodes with an older version of OpenSearch than + * the primary is, this will return replicas on the highest version of OpenSearch when document + * replication is enabled. */ public ShardRouting activeReplicaWithHighestVersion(ShardId shardId) { // It's possible for replicaNodeVersion to be null, when disassociating dead nodes @@ -390,6 +392,30 @@ public ShardRouting activeReplicaWithHighestVersion(ShardId shardId) { .orElse(null); } + /** + * Returns one active replica shard for the given shard id or null if + * no active replica is found. + * + * Since replicas could possibly be on nodes with a higher version of OpenSearch than + * the primary is, this will return replicas on the oldest version of OpenSearch when segment + * replication is enabled to allow for replica to read segments from primary. + * + */ + public ShardRouting activeReplicaWithOldestVersion(ShardId shardId) { + // It's possible for replicaNodeVersion to be null. Therefore, we need to protect against the version being null + // (meaning the node will be going away). + return assignedShards(shardId).stream() + .filter(shr -> !shr.primary() && shr.active()) + .filter(shr -> node(shr.currentNodeId()) != null) + .min( + Comparator.comparing( + shr -> node(shr.currentNodeId()).node(), + Comparator.nullsFirst(Comparator.comparing(DiscoveryNode::getVersion)) + ) + ) + .orElse(null); + } + /** * Returns true iff all replicas are active for the given shard routing. Otherwise false */ @@ -724,7 +750,12 @@ private void unassignPrimaryAndPromoteActiveReplicaIfExists( RoutingChangesObserver routingChangesObserver ) { assert failedShard.primary(); - ShardRouting activeReplica = activeReplicaWithHighestVersion(failedShard.shardId()); + ShardRouting activeReplica; + if (metadata.isSegmentReplicationEnabled(failedShard.getIndexName())) { + activeReplica = activeReplicaWithOldestVersion(failedShard.shardId()); + } else { + activeReplica = activeReplicaWithHighestVersion(failedShard.shardId()); + } if (activeReplica == null) { moveToUnassigned(failedShard, unassignedInfo); } else { diff --git a/server/src/test/java/org/opensearch/action/get/TransportGetActionTests.java b/server/src/test/java/org/opensearch/action/get/TransportGetActionTests.java index 2eca49fb3032f..9565e219d1a78 100644 --- a/server/src/test/java/org/opensearch/action/get/TransportGetActionTests.java +++ b/server/src/test/java/org/opensearch/action/get/TransportGetActionTests.java @@ -67,24 +67,24 @@ private static ClusterState clusterState(ReplicationType replicationType) { public void testShouldForcePrimaryRouting() { - ClusterState clusterState = clusterState(ReplicationType.SEGMENT); + Metadata metadata = clusterState(ReplicationType.SEGMENT).getMetadata(); // should return false since preference is set for request - assertFalse(TransportGetAction.shouldForcePrimaryRouting(clusterState, true, Preference.REPLICA.type(), "index1")); + assertFalse(TransportGetAction.shouldForcePrimaryRouting(metadata, true, Preference.REPLICA.type(), "index1")); // should return false since request is not realtime - assertFalse(TransportGetAction.shouldForcePrimaryRouting(clusterState, false, null, "index1")); + assertFalse(TransportGetAction.shouldForcePrimaryRouting(metadata, false, null, "index1")); // should return true since segment replication is enabled - assertTrue(TransportGetAction.shouldForcePrimaryRouting(clusterState, true, null, "index1")); + assertTrue(TransportGetAction.shouldForcePrimaryRouting(metadata, true, null, "index1")); // should return false since index doesn't exist - assertFalse(TransportGetAction.shouldForcePrimaryRouting(clusterState, true, null, "index3")); + assertFalse(TransportGetAction.shouldForcePrimaryRouting(metadata, true, null, "index3")); - clusterState = clusterState(ReplicationType.DOCUMENT); + metadata = clusterState(ReplicationType.DOCUMENT).getMetadata(); // should fail since document replication enabled - assertFalse(TransportGetAction.shouldForcePrimaryRouting(clusterState, true, null, "index1")); + assertFalse(TransportGetAction.shouldForcePrimaryRouting(metadata, true, null, "index1")); } diff --git a/server/src/test/java/org/opensearch/action/get/TransportMultiGetActionTests.java b/server/src/test/java/org/opensearch/action/get/TransportMultiGetActionTests.java index c9f40e0acc56c..0503bb39427a1 100644 --- a/server/src/test/java/org/opensearch/action/get/TransportMultiGetActionTests.java +++ b/server/src/test/java/org/opensearch/action/get/TransportMultiGetActionTests.java @@ -44,6 +44,7 @@ import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.routing.OperationRouting; +import org.opensearch.cluster.routing.Preference; import org.opensearch.cluster.routing.ShardIterator; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.Settings; @@ -58,6 +59,7 @@ import org.opensearch.core.tasks.TaskId; import org.opensearch.core.xcontent.MediaTypeRegistry; import org.opensearch.indices.IndicesService; +import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.tasks.Task; import org.opensearch.tasks.TaskManager; import org.opensearch.test.OpenSearchTestCase; @@ -68,6 +70,7 @@ import org.junit.AfterClass; import org.junit.BeforeClass; +import java.io.IOException; import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -91,32 +94,8 @@ public class TransportMultiGetActionTests extends OpenSearchTestCase { private static TransportMultiGetAction transportAction; private static TransportShardMultiGetAction shardAction; - @BeforeClass - public static void beforeClass() throws Exception { - threadPool = new TestThreadPool(TransportMultiGetActionTests.class.getSimpleName()); - - transportService = new TransportService( - Settings.EMPTY, - mock(Transport.class), - threadPool, - TransportService.NOOP_TRANSPORT_INTERCEPTOR, - boundAddress -> DiscoveryNode.createLocal( - Settings.builder().put("node.name", "node1").build(), - boundAddress.publishAddress(), - randomBase64UUID() - ), - null, - emptySet() - ) { - @Override - public TaskManager getTaskManager() { - return taskManager; - } - }; - - final Index index1 = new Index("index1", randomBase64UUID()); - final Index index2 = new Index("index2", randomBase64UUID()); - final ClusterState clusterState = ClusterState.builder(new ClusterName(TransportMultiGetActionTests.class.getSimpleName())) + private static ClusterState clusterState(ReplicationType replicationType, Index index1, Index index2) throws IOException { + return ClusterState.builder(new ClusterName(TransportMultiGetActionTests.class.getSimpleName())) .metadata( new Metadata.Builder().put( new IndexMetadata.Builder(index1.getName()).settings( @@ -124,6 +103,7 @@ public TaskManager getTaskManager() { .put("index.version.created", Version.CURRENT) .put("index.number_of_shards", 1) .put("index.number_of_replicas", 1) + .put(IndexMetadata.SETTING_REPLICATION_TYPE, replicationType) .put(IndexMetadata.SETTING_INDEX_UUID, index1.getUUID()) ) .putMapping( @@ -149,6 +129,7 @@ public TaskManager getTaskManager() { .put("index.version.created", Version.CURRENT) .put("index.number_of_shards", 1) .put("index.number_of_replicas", 1) + .put(IndexMetadata.SETTING_REPLICATION_TYPE, replicationType) .put(IndexMetadata.SETTING_INDEX_UUID, index1.getUUID()) ) .putMapping( @@ -170,6 +151,34 @@ public TaskManager getTaskManager() { ) ) .build(); + } + + @BeforeClass + public static void beforeClass() throws Exception { + threadPool = new TestThreadPool(TransportMultiGetActionTests.class.getSimpleName()); + + transportService = new TransportService( + Settings.EMPTY, + mock(Transport.class), + threadPool, + TransportService.NOOP_TRANSPORT_INTERCEPTOR, + boundAddress -> DiscoveryNode.createLocal( + Settings.builder().put("node.name", "node1").build(), + boundAddress.publishAddress(), + randomBase64UUID() + ), + null, + emptySet() + ) { + @Override + public TaskManager getTaskManager() { + return taskManager; + } + }; + + final Index index1 = new Index("index1", randomBase64UUID()); + final Index index2 = new Index("index2", randomBase64UUID()); + ClusterState clusterState = clusterState(randomBoolean() ? ReplicationType.SEGMENT : ReplicationType.DOCUMENT, index1, index2); final ShardIterator index1ShardIterator = mock(ShardIterator.class); when(index1ShardIterator.shardId()).thenReturn(new ShardId(index1, randomInt())); @@ -285,6 +294,30 @@ protected void executeShardAction( } + public void testShouldForcePrimaryRouting() throws IOException { + final Index index1 = new Index("index1", randomBase64UUID()); + final Index index2 = new Index("index2", randomBase64UUID()); + Metadata metadata = clusterState(ReplicationType.SEGMENT, index1, index2).getMetadata(); + + // should return false since preference is set for request + assertFalse(TransportMultiGetAction.shouldForcePrimaryRouting(metadata, true, Preference.REPLICA.type(), "index1")); + + // should return false since request is not realtime + assertFalse(TransportMultiGetAction.shouldForcePrimaryRouting(metadata, false, null, "index2")); + + // should return true since segment replication is enabled + assertTrue(TransportMultiGetAction.shouldForcePrimaryRouting(metadata, true, null, "index1")); + + // should return false since index doesn't exist + assertFalse(TransportMultiGetAction.shouldForcePrimaryRouting(metadata, true, null, "index3")); + + metadata = clusterState(ReplicationType.DOCUMENT, index1, index2).getMetadata(); + + // should fail since document replication enabled + assertFalse(TransportGetAction.shouldForcePrimaryRouting(metadata, true, null, "index1")); + + } + private static Task createTask() { return new Task( randomLong(), diff --git a/server/src/test/java/org/opensearch/cluster/ClusterStateTests.java b/server/src/test/java/org/opensearch/cluster/ClusterStateTests.java index c4fb3271ae3ce..457bdac1809ef 100644 --- a/server/src/test/java/org/opensearch/cluster/ClusterStateTests.java +++ b/server/src/test/java/org/opensearch/cluster/ClusterStateTests.java @@ -57,7 +57,6 @@ import org.opensearch.core.rest.RestStatus; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentBuilder; -import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.test.TestCustomMetadata; @@ -117,39 +116,6 @@ public void testSupersedes() { ); } - public void testIsSegmentReplicationEnabled() { - final String indexName = "test"; - ClusterState clusterState = ClusterState.builder(CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).build(); - Settings.Builder builder = settings(Version.CURRENT).put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT); - IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(indexName) - .settings(builder) - .numberOfShards(1) - .numberOfReplicas(1); - Metadata.Builder metadataBuilder = Metadata.builder().put(indexMetadataBuilder); - RoutingTable.Builder routingTableBuilder = RoutingTable.builder().addAsNew(indexMetadataBuilder.build()); - clusterState = ClusterState.builder(clusterState) - .metadata(metadataBuilder.build()) - .routingTable(routingTableBuilder.build()) - .build(); - assertTrue(clusterState.isSegmentReplicationEnabled(indexName)); - } - - public void testIsSegmentReplicationDisabled() { - final String indexName = "test"; - ClusterState clusterState = ClusterState.builder(CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).build(); - IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(indexName) - .settings(settings(Version.CURRENT)) - .numberOfShards(1) - .numberOfReplicas(1); - Metadata.Builder metadataBuilder = Metadata.builder().put(indexMetadataBuilder); - RoutingTable.Builder routingTableBuilder = RoutingTable.builder().addAsNew(indexMetadataBuilder.build()); - clusterState = ClusterState.builder(clusterState) - .metadata(metadataBuilder.build()) - .routingTable(routingTableBuilder.build()) - .build(); - assertFalse(clusterState.isSegmentReplicationEnabled(indexName)); - } - public void testBuilderRejectsNullCustom() { final ClusterState.Builder builder = ClusterState.builder(ClusterName.DEFAULT); final String key = randomAlphaOfLength(10); diff --git a/server/src/test/java/org/opensearch/cluster/metadata/MetadataTests.java b/server/src/test/java/org/opensearch/cluster/metadata/MetadataTests.java index fff39d14e9702..40eefa6cdbf03 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/MetadataTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/MetadataTests.java @@ -52,6 +52,7 @@ import org.opensearch.core.index.Index; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; +import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.plugins.MapperPlugin; import org.opensearch.test.OpenSearchTestCase; @@ -1425,6 +1426,29 @@ public void testMetadataBuildInvocations() { compareMetadata(previousMetadata, builtMetadata, false, true, true); } + public void testIsSegmentReplicationEnabled() { + final String indexName = "test"; + Settings.Builder builder = settings(Version.CURRENT).put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT); + IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(indexName) + .settings(builder) + .numberOfShards(1) + .numberOfReplicas(1); + Metadata.Builder metadataBuilder = Metadata.builder().put(indexMetadataBuilder); + Metadata metadata = metadataBuilder.build(); + assertTrue(metadata.isSegmentReplicationEnabled(indexName)); + } + + public void testIsSegmentReplicationDisabled() { + final String indexName = "test"; + IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(indexName) + .settings(settings(Version.CURRENT)) + .numberOfShards(1) + .numberOfReplicas(1); + Metadata.Builder metadataBuilder = Metadata.builder().put(indexMetadataBuilder); + Metadata metadata = metadataBuilder.build(); + assertFalse(metadata.isSegmentReplicationEnabled(indexName)); + } + public static Metadata randomMetadata() { Metadata.Builder md = Metadata.builder() .put(buildIndexMetadata("index", "alias", randomBoolean() ? null : randomBoolean()).build(), randomBoolean()) diff --git a/server/src/test/java/org/opensearch/cluster/routing/allocation/FailedNodeRoutingTests.java b/server/src/test/java/org/opensearch/cluster/routing/allocation/FailedNodeRoutingTests.java index 80afc1d9b0b0f..c245e608edbec 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/allocation/FailedNodeRoutingTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/allocation/FailedNodeRoutingTests.java @@ -53,6 +53,7 @@ import org.opensearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider; import org.opensearch.common.settings.Settings; import org.opensearch.indices.cluster.ClusterStateChanges; +import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.test.VersionUtils; import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; @@ -69,6 +70,7 @@ import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE; import static org.opensearch.cluster.routing.ShardRoutingState.INITIALIZING; import static org.opensearch.cluster.routing.ShardRoutingState.STARTED; import static org.hamcrest.Matchers.equalTo; @@ -137,7 +139,15 @@ public void testSimpleFailedNodeTest() { } } + public void testRandomClusterPromotesOldestReplica() throws InterruptedException { + testRandomClusterPromotesReplica(true); + } + public void testRandomClusterPromotesNewestReplica() throws InterruptedException { + testRandomClusterPromotesReplica(false); + } + + void testRandomClusterPromotesReplica(boolean isSegmentReplicationEnabled) throws InterruptedException { ThreadPool threadPool = new TestThreadPool(getClass().getName()); ClusterStateChanges cluster = new ClusterStateChanges(xContentRegistry(), threadPool); @@ -164,6 +174,9 @@ public void testRandomClusterPromotesNewestReplica() throws InterruptedException Settings.Builder settingsBuilder = Settings.builder() .put(SETTING_NUMBER_OF_SHARDS, randomIntBetween(1, 4)) .put(SETTING_NUMBER_OF_REPLICAS, randomIntBetween(2, 4)); + if (isSegmentReplicationEnabled) { + settingsBuilder.put(SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT); + } CreateIndexRequest request = new CreateIndexRequest(name, settingsBuilder.build()).waitForActiveShards(ActiveShardCount.NONE); state = cluster.createIndex(state, request); assertTrue(state.metadata().hasIndex(name)); @@ -206,13 +219,23 @@ public void testRandomClusterPromotesNewestReplica() throws InterruptedException Version candidateVer = getNodeVersion(sr, compareState); if (candidateVer != null) { logger.info("--> candidate on {} node; shard routing: {}", candidateVer, sr); - assertTrue( - "candidate was not on the newest version, new primary is on " - + newPrimaryVersion - + " and there is a candidate on " - + candidateVer, - candidateVer.onOrBefore(newPrimaryVersion) - ); + if (isSegmentReplicationEnabled) { + assertTrue( + "candidate was not on the oldest version, new primary is on " + + newPrimaryVersion + + " and there is a candidate on " + + candidateVer, + candidateVer.onOrAfter(newPrimaryVersion) + ); + } else { + assertTrue( + "candidate was not on the newest version, new primary is on " + + newPrimaryVersion + + " and there is a candidate on " + + candidateVer, + candidateVer.onOrBefore(newPrimaryVersion) + ); + } } }); }); diff --git a/server/src/test/java/org/opensearch/cluster/routing/allocation/FailedShardsRoutingTests.java b/server/src/test/java/org/opensearch/cluster/routing/allocation/FailedShardsRoutingTests.java index f2dc745ad33bf..db4cedbbbe7b5 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/allocation/FailedShardsRoutingTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/allocation/FailedShardsRoutingTests.java @@ -49,6 +49,7 @@ import org.opensearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider; import org.opensearch.common.settings.Settings; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.test.VersionUtils; import java.util.ArrayList; @@ -647,10 +648,21 @@ public void testFailAllReplicasInitializingOnPrimaryFailWhileHavingAReplicaToEle } public void testReplicaOnNewestVersionIsPromoted() { + testReplicaIsPromoted(false); + } + + public void testReplicaOnOldestVersionIsPromoted() { + testReplicaIsPromoted(true); + } + + private void testReplicaIsPromoted(boolean isSegmentReplicationEnabled) { AllocationService allocation = createAllocationService(Settings.builder().build()); + Settings.Builder settingsBuilder = isSegmentReplicationEnabled + ? settings(Version.CURRENT).put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) + : settings(Version.CURRENT); Metadata metadata = Metadata.builder() - .put(IndexMetadata.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(3)) + .put(IndexMetadata.builder("test").settings(settingsBuilder).numberOfShards(1).numberOfReplicas(3)) .build(); RoutingTable initialRoutingTable = RoutingTable.builder().addAsNew(metadata.index("test")).build(); @@ -714,7 +726,12 @@ public void testReplicaOnNewestVersionIsPromoted() { assertThat(clusterState.getRoutingNodes().shardsWithState(STARTED).size(), equalTo(4)); assertThat(clusterState.getRoutingNodes().shardsWithState(UNASSIGNED).size(), equalTo(0)); - ShardRouting startedReplica = clusterState.getRoutingNodes().activeReplicaWithHighestVersion(shardId); + ShardRouting startedReplica; + if (isSegmentReplicationEnabled) { + startedReplica = clusterState.getRoutingNodes().activeReplicaWithOldestVersion(shardId); + } else { + startedReplica = clusterState.getRoutingNodes().activeReplicaWithHighestVersion(shardId); + } logger.info("--> all shards allocated, replica that should be promoted: {}", startedReplica); // fail the primary shard again and make sure the correct replica is promoted @@ -739,13 +756,24 @@ public void testReplicaOnNewestVersionIsPromoted() { continue; } Version nodeVer = cursor.getVersion(); - assertTrue( - "expected node [" + cursor.getId() + "] with version " + nodeVer + " to be before " + replicaNodeVersion, - replicaNodeVersion.onOrAfter(nodeVer) - ); + if (isSegmentReplicationEnabled) { + assertTrue( + "expected node [" + cursor.getId() + "] with version " + nodeVer + " to be after " + replicaNodeVersion, + replicaNodeVersion.onOrBefore(nodeVer) + ); + } else { + assertTrue( + "expected node [" + cursor.getId() + "] with version " + nodeVer + " to be before " + replicaNodeVersion, + replicaNodeVersion.onOrAfter(nodeVer) + ); + } } - startedReplica = clusterState.getRoutingNodes().activeReplicaWithHighestVersion(shardId); + if (isSegmentReplicationEnabled) { + startedReplica = clusterState.getRoutingNodes().activeReplicaWithOldestVersion(shardId); + } else { + startedReplica = clusterState.getRoutingNodes().activeReplicaWithHighestVersion(shardId); + } logger.info("--> failing primary shard a second time, should select: {}", startedReplica); // fail the primary shard again, and ensure the same thing happens @@ -771,10 +799,17 @@ public void testReplicaOnNewestVersionIsPromoted() { continue; } Version nodeVer = cursor.getVersion(); - assertTrue( - "expected node [" + cursor.getId() + "] with version " + nodeVer + " to be before " + replicaNodeVersion, - replicaNodeVersion.onOrAfter(nodeVer) - ); + if (isSegmentReplicationEnabled) { + assertTrue( + "expected node [" + cursor.getId() + "] with version " + nodeVer + " to be after " + replicaNodeVersion, + replicaNodeVersion.onOrBefore(nodeVer) + ); + } else { + assertTrue( + "expected node [" + cursor.getId() + "] with version " + nodeVer + " to be before " + replicaNodeVersion, + replicaNodeVersion.onOrAfter(nodeVer) + ); + } } } } From c294c918c7415a451e6494b9f8a4d75a629dccd0 Mon Sep 17 00:00:00 2001 From: Bhumika Saini Date: Thu, 31 Aug 2023 10:31:32 +0530 Subject: [PATCH 19/37] [Remote Store] Add tracker factory to manage remote store stats trackers (#9546) --------- Signed-off-by: Bhumika Saini --- .../TransportRemoteStoreStatsAction.java | 10 +- .../common/settings/ClusterSettings.java | 7 +- .../org/opensearch/index/IndexService.java | 6 +- .../remote/RemoteSegmentTransferTracker.java | 44 +- .../remote/RemoteStorePressureService.java | 76 +-- .../remote/RemoteStorePressureSettings.java | 86 +-- .../RemoteStoreStatsTrackerFactory.java | 100 +++ .../opensearch/index/shard/IndexShard.java | 16 +- .../org/opensearch/indices/IndicesModule.java | 2 + .../opensearch/indices/IndicesService.java | 6 +- .../cluster/IndicesClusterStateService.java | 19 +- .../TransportRemoteStoreStatsActionTests.java | 10 +- .../RemoteSegmentTransferTrackerTests.java | 594 ++++++++---------- .../RemoteStorePressureServiceTests.java | 57 +- .../RemoteStorePressureSettingsTests.java | 97 --- .../RemoteStoreStatsTrackerFactoryTests.java | 119 ++++ .../index/remote/RemoteStoreTestsHelper.java | 40 ++ .../index/shard/IndexShardTests.java | 4 +- .../RemoteStoreRefreshListenerTests.java | 49 +- ...actIndicesClusterStateServiceTestCase.java | 4 +- .../snapshots/SnapshotResiliencyTests.java | 3 +- .../index/shard/IndexShardTestCase.java | 12 +- 22 files changed, 630 insertions(+), 731 deletions(-) create mode 100644 server/src/main/java/org/opensearch/index/remote/RemoteStoreStatsTrackerFactory.java create mode 100644 server/src/test/java/org/opensearch/index/remote/RemoteStoreStatsTrackerFactoryTests.java create mode 100644 server/src/test/java/org/opensearch/index/remote/RemoteStoreTestsHelper.java diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/stats/TransportRemoteStoreStatsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/stats/TransportRemoteStoreStatsAction.java index d05879aa1ae78..2595f783b4cf3 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/stats/TransportRemoteStoreStatsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/stats/TransportRemoteStoreStatsAction.java @@ -24,7 +24,7 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.index.IndexService; import org.opensearch.index.remote.RemoteSegmentTransferTracker; -import org.opensearch.index.remote.RemoteStorePressureService; +import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.ShardNotFoundException; import org.opensearch.indices.IndicesService; @@ -50,7 +50,7 @@ public class TransportRemoteStoreStatsAction extends TransportBroadcastByNodeAct private final IndicesService indicesService; - private final RemoteStorePressureService remoteStorePressureService; + private final RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory; @Inject public TransportRemoteStoreStatsAction( @@ -59,7 +59,7 @@ public TransportRemoteStoreStatsAction( IndicesService indicesService, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, - RemoteStorePressureService remoteStorePressureService + RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory ) { super( RemoteStoreStatsAction.NAME, @@ -71,7 +71,7 @@ public TransportRemoteStoreStatsAction( ThreadPool.Names.MANAGEMENT ); this.indicesService = indicesService; - this.remoteStorePressureService = remoteStorePressureService; + this.remoteStoreStatsTrackerFactory = remoteStoreStatsTrackerFactory; } /** @@ -153,7 +153,7 @@ protected RemoteStoreStats shardOperation(RemoteStoreStatsRequest request, Shard throw new ShardNotFoundException(indexShard.shardId()); } - RemoteSegmentTransferTracker remoteSegmentTransferTracker = remoteStorePressureService.getRemoteRefreshSegmentTracker( + RemoteSegmentTransferTracker remoteSegmentTransferTracker = remoteStoreStatsTrackerFactory.getRemoteSegmentTransferTracker( indexShard.shardId() ); assert Objects.nonNull(remoteSegmentTransferTracker); diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index c89ae8088f1be..de3f13f2e1a1d 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -106,6 +106,7 @@ import org.opensearch.index.ShardIndexingPressureSettings; import org.opensearch.index.ShardIndexingPressureStore; import org.opensearch.index.remote.RemoteStorePressureSettings; +import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory; import org.opensearch.index.store.remote.filecache.FileCache; import org.opensearch.indices.IndexingMemoryController; import org.opensearch.indices.IndicesQueryCache; @@ -656,9 +657,9 @@ public void apply(Settings value, Settings current, Settings previous) { RemoteStorePressureSettings.BYTES_LAG_VARIANCE_FACTOR, RemoteStorePressureSettings.UPLOAD_TIME_LAG_VARIANCE_FACTOR, RemoteStorePressureSettings.MIN_CONSECUTIVE_FAILURES_LIMIT, - RemoteStorePressureSettings.UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE, - RemoteStorePressureSettings.UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE, - RemoteStorePressureSettings.UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE, + + // Settings related to Remote Store stats + RemoteStoreStatsTrackerFactory.MOVING_AVERAGE_WINDOW_SIZE, // Related to monitoring of task cancellation TaskCancellationMonitoringSettings.IS_ENABLED_SETTING, diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index 811768fc1540e..e5028ff2ecff9 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -78,7 +78,7 @@ import org.opensearch.index.mapper.MapperService; import org.opensearch.index.query.QueryShardContext; import org.opensearch.index.query.SearchIndexNameMatcher; -import org.opensearch.index.remote.RemoteStorePressureService; +import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory; import org.opensearch.index.seqno.RetentionLeaseSyncer; import org.opensearch.index.shard.IndexEventListener; import org.opensearch.index.shard.IndexShard; @@ -443,7 +443,7 @@ public synchronized IndexShard createShard( final Consumer globalCheckpointSyncer, final RetentionLeaseSyncer retentionLeaseSyncer, final SegmentReplicationCheckpointPublisher checkpointPublisher, - final RemoteStorePressureService remoteStorePressureService + final RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory ) throws IOException { Objects.requireNonNull(retentionLeaseSyncer); /* @@ -512,7 +512,7 @@ public synchronized IndexShard createShard( translogFactorySupplier, this.indexSettings.isSegRepEnabled() ? checkpointPublisher : null, remoteStore, - remoteStorePressureService + remoteStoreStatsTrackerFactory ); eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created"); eventListener.afterIndexShardCreated(indexShard); diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteSegmentTransferTracker.java b/server/src/main/java/org/opensearch/index/remote/RemoteSegmentTransferTracker.java index 95902fd375145..564fdfbc0761d 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteSegmentTransferTracker.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteSegmentTransferTracker.java @@ -194,9 +194,7 @@ public class RemoteSegmentTransferTracker { public RemoteSegmentTransferTracker( ShardId shardId, DirectoryFileTransferTracker directoryFileTransferTracker, - int uploadBytesMovingAverageWindowSize, - int uploadBytesPerSecMovingAverageWindowSize, - int uploadTimeMsMovingAverageWindowSize + int movingAverageWindowSize ) { logger = Loggers.getLogger(getClass(), shardId); this.shardId = shardId; @@ -207,9 +205,9 @@ public RemoteSegmentTransferTracker( remoteRefreshTimeMs = currentTimeMs; localRefreshClockTimeMs = currentClockTimeMs; remoteRefreshClockTimeMs = currentClockTimeMs; - uploadBytesMovingAverageReference = new AtomicReference<>(new MovingAverage(uploadBytesMovingAverageWindowSize)); - uploadBytesPerSecMovingAverageReference = new AtomicReference<>(new MovingAverage(uploadBytesPerSecMovingAverageWindowSize)); - uploadTimeMsMovingAverageReference = new AtomicReference<>(new MovingAverage(uploadTimeMsMovingAverageWindowSize)); + uploadBytesMovingAverageReference = new AtomicReference<>(new MovingAverage(movingAverageWindowSize)); + uploadBytesPerSecMovingAverageReference = new AtomicReference<>(new MovingAverage(movingAverageWindowSize)); + uploadTimeMsMovingAverageReference = new AtomicReference<>(new MovingAverage(movingAverageWindowSize)); this.directoryFileTransferTracker = directoryFileTransferTracker; } @@ -470,14 +468,22 @@ public void addUploadBytes(long size) { } /** - * Updates the window size for data collection of upload bytes. This also resets any data collected so far. + * Updates the window size for data collection. This also resets any data collected so far. * * @param updatedSize the updated size */ - void updateUploadBytesMovingAverageWindowSize(int updatedSize) { + void updateMovingAverageWindowSize(int updatedSize) { synchronized (uploadBytesMutex) { this.uploadBytesMovingAverageReference.set(this.uploadBytesMovingAverageReference.get().copyWithSize(updatedSize)); } + + synchronized (uploadBytesPerSecMutex) { + this.uploadBytesPerSecMovingAverageReference.set(this.uploadBytesPerSecMovingAverageReference.get().copyWithSize(updatedSize)); + } + + synchronized (uploadTimeMsMutex) { + this.uploadTimeMsMovingAverageReference.set(this.uploadTimeMsMovingAverageReference.get().copyWithSize(updatedSize)); + } } boolean isUploadBytesPerSecAverageReady() { @@ -494,17 +500,6 @@ public void addUploadBytesPerSec(long bytesPerSec) { } } - /** - * Updates the window size for data collection of upload bytes per second. This also resets any data collected so far. - * - * @param updatedSize the updated size - */ - void updateUploadBytesPerSecMovingAverageWindowSize(int updatedSize) { - synchronized (uploadBytesPerSecMutex) { - this.uploadBytesPerSecMovingAverageReference.set(this.uploadBytesPerSecMovingAverageReference.get().copyWithSize(updatedSize)); - } - } - boolean isUploadTimeMsAverageReady() { return uploadTimeMsMovingAverageReference.get().isReady(); } @@ -519,17 +514,6 @@ public void addTimeForCompletedUploadSync(long timeMs) { } } - /** - * Updates the window size for data collection of upload time (ms). This also resets any data collected so far. - * - * @param updatedSize the updated size - */ - void updateUploadTimeMsMovingAverageWindowSize(int updatedSize) { - synchronized (uploadTimeMsMutex) { - this.uploadTimeMsMovingAverageReference.set(this.uploadTimeMsMovingAverageReference.get().copyWithSize(updatedSize)); - } - } - public void addTotalUploadTimeInMs(long fileUploadTimeInMs) { this.totalUploadTimeInMs.addAndGet(fileUploadTimeInMs); } diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStorePressureService.java b/server/src/main/java/org/opensearch/index/remote/RemoteStorePressureService.java index 427304935259b..3722be7dadbfb 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStorePressureService.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStorePressureService.java @@ -13,32 +13,22 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.inject.Inject; import org.opensearch.common.settings.Settings; -import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.core.index.shard.ShardId; -import org.opensearch.index.shard.IndexEventListener; -import org.opensearch.index.shard.IndexShard; import java.util.Arrays; import java.util.List; import java.util.Locale; -import java.util.Map; -import java.util.function.BiConsumer; /** * Service used to validate if the incoming indexing request should be rejected based on the {@link RemoteSegmentTransferTracker}. * * @opensearch.internal */ -public class RemoteStorePressureService implements IndexEventListener { +public class RemoteStorePressureService { private static final Logger logger = LogManager.getLogger(RemoteStorePressureService.class); - /** - * Keeps map of remote-backed index shards and their corresponding backpressure tracker. - */ - private final Map trackerMap = ConcurrentCollections.newConcurrentMap(); - /** * Remote refresh segment pressure settings which is used for creation of the backpressure tracker and as well as rejection. */ @@ -46,51 +36,21 @@ public class RemoteStorePressureService implements IndexEventListener { private final List lagValidators; + private final RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory; + @Inject - public RemoteStorePressureService(ClusterService clusterService, Settings settings) { + public RemoteStorePressureService( + ClusterService clusterService, + Settings settings, + RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory + ) { pressureSettings = new RemoteStorePressureSettings(clusterService, settings, this); lagValidators = Arrays.asList( new ConsecutiveFailureValidator(pressureSettings), new BytesLagValidator(pressureSettings), new TimeLagValidator(pressureSettings) ); - } - - /** - * Get {@code RemoteSegmentTransferTracker} only if the underlying Index has remote segments integration enabled. - * - * @param shardId shard id - * @return the tracker if index is remote-backed, else null. - */ - public RemoteSegmentTransferTracker getRemoteRefreshSegmentTracker(ShardId shardId) { - return trackerMap.get(shardId); - } - - @Override - public void afterIndexShardCreated(IndexShard indexShard) { - if (indexShard.indexSettings().isRemoteStoreEnabled() == false) { - return; - } - ShardId shardId = indexShard.shardId(); - trackerMap.put( - shardId, - new RemoteSegmentTransferTracker( - shardId, - indexShard.store().getDirectoryFileTransferTracker(), - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() - ) - ); - logger.trace("Created tracker for shardId={}", shardId); - } - - @Override - public void afterIndexShardClosed(ShardId shardId, IndexShard indexShard, Settings indexSettings) { - RemoteSegmentTransferTracker remoteSegmentTransferTracker = trackerMap.remove(shardId); - if (remoteSegmentTransferTracker != null) { - logger.trace("Deleted tracker for shardId={}", shardId); - } + this.remoteStoreStatsTrackerFactory = remoteStoreStatsTrackerFactory; } /** @@ -108,7 +68,7 @@ public boolean isSegmentsUploadBackpressureEnabled() { * @param shardId shardId for which the validation needs to be done. */ public void validateSegmentsUploadLag(ShardId shardId) { - RemoteSegmentTransferTracker remoteSegmentTransferTracker = getRemoteRefreshSegmentTracker(shardId); + RemoteSegmentTransferTracker remoteSegmentTransferTracker = remoteStoreStatsTrackerFactory.getRemoteSegmentTransferTracker(shardId); // condition 1 - This will be null for non-remote backed indexes // condition 2 - This will be zero if the remote store is if (remoteSegmentTransferTracker == null || remoteSegmentTransferTracker.getRefreshSeqNoLag() == 0) { @@ -123,22 +83,6 @@ public void validateSegmentsUploadLag(ShardId shardId) { } } - void updateUploadBytesMovingAverageWindowSize(int updatedSize) { - updateMovingAverageWindowSize(RemoteSegmentTransferTracker::updateUploadBytesMovingAverageWindowSize, updatedSize); - } - - void updateUploadBytesPerSecMovingAverageWindowSize(int updatedSize) { - updateMovingAverageWindowSize(RemoteSegmentTransferTracker::updateUploadBytesPerSecMovingAverageWindowSize, updatedSize); - } - - void updateUploadTimeMsMovingAverageWindowSize(int updatedSize) { - updateMovingAverageWindowSize(RemoteSegmentTransferTracker::updateUploadTimeMsMovingAverageWindowSize, updatedSize); - } - - void updateMovingAverageWindowSize(BiConsumer biConsumer, int updatedSize) { - trackerMap.values().forEach(tracker -> biConsumer.accept(tracker, updatedSize)); - } - /** * Abstract class for validating if lag is acceptable or not. * diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStorePressureSettings.java b/server/src/main/java/org/opensearch/index/remote/RemoteStorePressureSettings.java index 3f665890d43e9..af2e453f8107d 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStorePressureSettings.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStorePressureSettings.java @@ -20,16 +20,12 @@ */ public class RemoteStorePressureSettings { - private static class Defaults { + static class Defaults { private static final double BYTES_LAG_VARIANCE_FACTOR = 10.0; private static final double UPLOAD_TIME_LAG_VARIANCE_FACTOR = 10.0; private static final double VARIANCE_FACTOR_MIN_VALUE = 1.0; private static final int MIN_CONSECUTIVE_FAILURES_LIMIT = 5; private static final int MIN_CONSECUTIVE_FAILURES_LIMIT_MIN_VALUE = 1; - private static final int UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE = 20; - private static final int UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE = 20; - private static final int UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE = 20; - private static final int MOVING_AVERAGE_WINDOW_SIZE_MIN_VALUE = 5; } public static final Setting REMOTE_REFRESH_SEGMENT_PRESSURE_ENABLED = Setting.boolSetting( @@ -63,30 +59,6 @@ private static class Defaults { Setting.Property.NodeScope ); - public static final Setting UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE = Setting.intSetting( - "remote_store.segment.pressure.upload_bytes_moving_average_window_size", - Defaults.UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE, - Defaults.MOVING_AVERAGE_WINDOW_SIZE_MIN_VALUE, - Setting.Property.Dynamic, - Setting.Property.NodeScope - ); - - public static final Setting UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE = Setting.intSetting( - "remote_store.segment.pressure.upload_bytes_per_sec_moving_average_window_size", - Defaults.UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE, - Defaults.MOVING_AVERAGE_WINDOW_SIZE_MIN_VALUE, - Setting.Property.Dynamic, - Setting.Property.NodeScope - ); - - public static final Setting UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE = Setting.intSetting( - "remote_store.segment.pressure.upload_time_moving_average_window_size", - Defaults.UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE, - Defaults.MOVING_AVERAGE_WINDOW_SIZE_MIN_VALUE, - Setting.Property.Dynamic, - Setting.Property.NodeScope - ); - private volatile boolean remoteRefreshSegmentPressureEnabled; private volatile long minRefreshSeqNoLagLimit; @@ -97,16 +69,10 @@ private static class Defaults { private volatile int minConsecutiveFailuresLimit; - private volatile int uploadBytesMovingAverageWindowSize; - - private volatile int uploadBytesPerSecMovingAverageWindowSize; - - private volatile int uploadTimeMovingAverageWindowSize; - public RemoteStorePressureSettings( ClusterService clusterService, Settings settings, - RemoteStorePressureService remoteUploadPressureService + RemoteStorePressureService remoteStorePressureService ) { ClusterSettings clusterSettings = clusterService.getClusterSettings(); @@ -121,30 +87,6 @@ public RemoteStorePressureSettings( this.minConsecutiveFailuresLimit = MIN_CONSECUTIVE_FAILURES_LIMIT.get(settings); clusterSettings.addSettingsUpdateConsumer(MIN_CONSECUTIVE_FAILURES_LIMIT, this::setMinConsecutiveFailuresLimit); - - this.uploadBytesMovingAverageWindowSize = UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE.get(settings); - clusterSettings.addSettingsUpdateConsumer( - UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE, - remoteUploadPressureService::updateUploadBytesMovingAverageWindowSize - ); - clusterSettings.addSettingsUpdateConsumer(UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE, this::setUploadBytesMovingAverageWindowSize); - - this.uploadBytesPerSecMovingAverageWindowSize = UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE.get(settings); - clusterSettings.addSettingsUpdateConsumer( - UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE, - remoteUploadPressureService::updateUploadBytesPerSecMovingAverageWindowSize - ); - clusterSettings.addSettingsUpdateConsumer( - UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE, - this::setUploadBytesPerSecMovingAverageWindowSize - ); - - this.uploadTimeMovingAverageWindowSize = UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE.get(settings); - clusterSettings.addSettingsUpdateConsumer( - UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE, - remoteUploadPressureService::updateUploadTimeMsMovingAverageWindowSize - ); - clusterSettings.addSettingsUpdateConsumer(UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE, this::setUploadTimeMovingAverageWindowSize); } public boolean isRemoteRefreshSegmentPressureEnabled() { @@ -186,28 +128,4 @@ public int getMinConsecutiveFailuresLimit() { public void setMinConsecutiveFailuresLimit(int minConsecutiveFailuresLimit) { this.minConsecutiveFailuresLimit = minConsecutiveFailuresLimit; } - - public int getUploadBytesMovingAverageWindowSize() { - return uploadBytesMovingAverageWindowSize; - } - - public void setUploadBytesMovingAverageWindowSize(int uploadBytesMovingAverageWindowSize) { - this.uploadBytesMovingAverageWindowSize = uploadBytesMovingAverageWindowSize; - } - - public int getUploadBytesPerSecMovingAverageWindowSize() { - return uploadBytesPerSecMovingAverageWindowSize; - } - - public void setUploadBytesPerSecMovingAverageWindowSize(int uploadBytesPerSecMovingAverageWindowSize) { - this.uploadBytesPerSecMovingAverageWindowSize = uploadBytesPerSecMovingAverageWindowSize; - } - - public int getUploadTimeMovingAverageWindowSize() { - return uploadTimeMovingAverageWindowSize; - } - - public void setUploadTimeMovingAverageWindowSize(int uploadTimeMovingAverageWindowSize) { - this.uploadTimeMovingAverageWindowSize = uploadTimeMovingAverageWindowSize; - } } diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStoreStatsTrackerFactory.java b/server/src/main/java/org/opensearch/index/remote/RemoteStoreStatsTrackerFactory.java new file mode 100644 index 0000000000000..783f4195be156 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStoreStatsTrackerFactory.java @@ -0,0 +1,100 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.remote; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.inject.Inject; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Setting; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.concurrent.ConcurrentCollections; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.shard.IndexEventListener; +import org.opensearch.index.shard.IndexShard; + +import java.util.Map; + +/** + * Factory to manage stats trackers for Remote Store operations + * + * @opensearch.internal + */ +public class RemoteStoreStatsTrackerFactory implements IndexEventListener { + static class Defaults { + static final int MOVING_AVERAGE_WINDOW_SIZE = 20; + static final int MOVING_AVERAGE_WINDOW_SIZE_MIN_VALUE = 5; + } + + public static final Setting MOVING_AVERAGE_WINDOW_SIZE = Setting.intSetting( + "remote_store.moving_average_window_size", + Defaults.MOVING_AVERAGE_WINDOW_SIZE, + Defaults.MOVING_AVERAGE_WINDOW_SIZE_MIN_VALUE, + Setting.Property.Dynamic, + Setting.Property.NodeScope + ); + + private static final Logger logger = LogManager.getLogger(RemoteStoreStatsTrackerFactory.class); + + /** + * Number of data points to consider for a moving average statistic + */ + private volatile int movingAverageWindowSize; + + /** + * Keeps map of remote-backed index shards and their corresponding stats tracker. + */ + private final Map remoteSegmentTrackerMap = ConcurrentCollections.newConcurrentMap(); + + @Inject + public RemoteStoreStatsTrackerFactory(ClusterService clusterService, Settings settings) { + ClusterSettings clusterSettings = clusterService.getClusterSettings(); + + this.movingAverageWindowSize = MOVING_AVERAGE_WINDOW_SIZE.get(settings); + clusterSettings.addSettingsUpdateConsumer(MOVING_AVERAGE_WINDOW_SIZE, this::updateMovingAverageWindowSize); + } + + @Override + public void afterIndexShardCreated(IndexShard indexShard) { + if (indexShard.indexSettings().isRemoteStoreEnabled() == false) { + return; + } + ShardId shardId = indexShard.shardId(); + remoteSegmentTrackerMap.put( + shardId, + new RemoteSegmentTransferTracker(shardId, indexShard.store().getDirectoryFileTransferTracker(), movingAverageWindowSize) + ); + logger.trace("Created RemoteSegmentTransferTracker for shardId={}", shardId); + } + + @Override + public void afterIndexShardClosed(ShardId shardId, IndexShard indexShard, Settings indexSettings) { + RemoteSegmentTransferTracker remoteSegmentTransferTracker = remoteSegmentTrackerMap.remove(shardId); + if (remoteSegmentTransferTracker != null) { + logger.trace("Deleted RemoteSegmentTransferTracker for shardId={}", shardId); + } + } + + void updateMovingAverageWindowSize(int updatedSize) { + remoteSegmentTrackerMap.values().forEach(tracker -> tracker.updateMovingAverageWindowSize(updatedSize)); + + // Update movingAverageWindowSize only if the trackers were successfully updated + movingAverageWindowSize = updatedSize; + } + + public RemoteSegmentTransferTracker getRemoteSegmentTransferTracker(ShardId shardId) { + return remoteSegmentTrackerMap.get(shardId); + } + + // visible for testing + int getMovingAverageWindowSize() { + return movingAverageWindowSize; + } +} diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index 1c1d7fed5e4f5..ba584b06cff71 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -147,7 +147,7 @@ import org.opensearch.index.recovery.RecoveryStats; import org.opensearch.index.refresh.RefreshStats; import org.opensearch.index.remote.RemoteSegmentStats; -import org.opensearch.index.remote.RemoteStorePressureService; +import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory; import org.opensearch.index.search.stats.SearchStats; import org.opensearch.index.search.stats.ShardSearchStats; import org.opensearch.index.seqno.ReplicationTracker; @@ -333,7 +333,7 @@ Runnable getGlobalCheckpointSyncer() { private final Store remoteStore; private final BiFunction translogFactorySupplier; private final boolean isTimeSeriesIndex; - private final RemoteStorePressureService remoteStorePressureService; + private final RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory; private final List internalRefreshListener = new ArrayList<>(); @@ -361,7 +361,7 @@ public IndexShard( final BiFunction translogFactorySupplier, @Nullable final SegmentReplicationCheckpointPublisher checkpointPublisher, @Nullable final Store remoteStore, - final RemoteStorePressureService remoteStorePressureService + final RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory ) throws IOException { super(shardRouting.shardId(), indexSettings); assert shardRouting.initializing(); @@ -456,7 +456,7 @@ public boolean shouldCache(Query query) { this.isTimeSeriesIndex = (mapperService == null || mapperService.documentMapper() == null) ? false : mapperService.documentMapper().mappers().containsTimeStampField(); - this.remoteStorePressureService = remoteStorePressureService; + this.remoteStoreStatsTrackerFactory = remoteStoreStatsTrackerFactory; } public ThreadPool getThreadPool() { @@ -546,8 +546,8 @@ public QueryCachingPolicy getQueryCachingPolicy() { } /** Only used for testing **/ - protected RemoteStorePressureService getRemoteStorePressureService() { - return remoteStorePressureService; + protected RemoteStoreStatsTrackerFactory getRemoteStoreStatsTrackerFactory() { + return remoteStoreStatsTrackerFactory; } @Override @@ -1386,7 +1386,7 @@ public SegmentsStats segmentStats(boolean includeSegmentFileSizes, boolean inclu // Populate remote_store stats only if the index is remote store backed if (indexSettings.isRemoteStoreEnabled()) { segmentsStats.addRemoteSegmentStats( - new RemoteSegmentStats(remoteStorePressureService.getRemoteRefreshSegmentTracker(shardId).stats()) + new RemoteSegmentStats(remoteStoreStatsTrackerFactory.getRemoteSegmentTransferTracker(shardId).stats()) ); } return segmentsStats; @@ -3696,7 +3696,7 @@ private EngineConfig newEngineConfig(LongSupplier globalCheckpointSupplier) thro this, // Add the checkpoint publisher if the Segment Replciation via remote store is enabled. indexSettings.isSegRepWithRemoteEnabled() ? this.checkpointPublisher : SegmentReplicationCheckpointPublisher.EMPTY, - remoteStorePressureService.getRemoteRefreshSegmentTracker(shardId()) + remoteStoreStatsTrackerFactory.getRemoteSegmentTransferTracker(shardId()) ) ); } diff --git a/server/src/main/java/org/opensearch/indices/IndicesModule.java b/server/src/main/java/org/opensearch/indices/IndicesModule.java index c6ae8b988aed0..d902323401c07 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesModule.java +++ b/server/src/main/java/org/opensearch/indices/IndicesModule.java @@ -71,6 +71,7 @@ import org.opensearch.index.mapper.TextFieldMapper; import org.opensearch.index.mapper.VersionFieldMapper; import org.opensearch.index.remote.RemoteStorePressureService; +import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory; import org.opensearch.index.seqno.GlobalCheckpointSyncAction; import org.opensearch.index.seqno.RetentionLeaseBackgroundSyncAction; import org.opensearch.index.seqno.RetentionLeaseSyncAction; @@ -289,6 +290,7 @@ protected void configure() { bind(SegmentReplicationCheckpointPublisher.class).asEagerSingleton(); bind(SegmentReplicationPressureService.class).asEagerSingleton(); if (FeatureFlags.isEnabled(FeatureFlags.REMOTE_STORE)) { + bind(RemoteStoreStatsTrackerFactory.class).asEagerSingleton(); bind(RemoteStorePressureService.class).asEagerSingleton(); } } diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index 297beff981722..13a82ce5bdf0d 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -121,7 +121,7 @@ import org.opensearch.index.query.QueryRewriteContext; import org.opensearch.index.recovery.RecoveryStats; import org.opensearch.index.refresh.RefreshStats; -import org.opensearch.index.remote.RemoteStorePressureService; +import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory; import org.opensearch.index.search.stats.SearchStats; import org.opensearch.index.seqno.RetentionLeaseStats; import org.opensearch.index.seqno.RetentionLeaseSyncer; @@ -978,7 +978,7 @@ public IndexShard createShard( final RetentionLeaseSyncer retentionLeaseSyncer, final DiscoveryNode targetNode, final DiscoveryNode sourceNode, - final RemoteStorePressureService remoteStorePressureService + final RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory ) throws IOException { Objects.requireNonNull(retentionLeaseSyncer); ensureChangesAllowed(); @@ -990,7 +990,7 @@ public IndexShard createShard( globalCheckpointSyncer, retentionLeaseSyncer, checkpointPublisher, - remoteStorePressureService + remoteStoreStatsTrackerFactory ); indexShard.addShardFailureCallback(onShardFailure); indexShard.startRecovery(recoveryState, recoveryTargetService, recoveryListener, repositoriesService, mapping -> { diff --git a/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java b/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java index d8fe5a9d07f3d..0904dc5fa18a4 100644 --- a/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java +++ b/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java @@ -66,7 +66,7 @@ import org.opensearch.index.IndexComponent; import org.opensearch.index.IndexService; import org.opensearch.index.IndexSettings; -import org.opensearch.index.remote.RemoteStorePressureService; +import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory; import org.opensearch.index.seqno.GlobalCheckpointSyncAction; import org.opensearch.index.seqno.ReplicationTracker; import org.opensearch.index.seqno.RetentionLeaseSyncer; @@ -149,7 +149,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple private final SegmentReplicationCheckpointPublisher checkpointPublisher; - private final RemoteStorePressureService remoteStorePressureService; + private final RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory; @Inject public IndicesClusterStateService( @@ -170,7 +170,7 @@ public IndicesClusterStateService( final GlobalCheckpointSyncAction globalCheckpointSyncAction, final RetentionLeaseSyncer retentionLeaseSyncer, final SegmentReplicationCheckpointPublisher checkpointPublisher, - final RemoteStorePressureService remoteStorePressureService + final RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory ) { this( settings, @@ -190,7 +190,7 @@ public IndicesClusterStateService( primaryReplicaSyncer, globalCheckpointSyncAction::updateGlobalCheckpointForShard, retentionLeaseSyncer, - remoteStorePressureService + remoteStoreStatsTrackerFactory ); } @@ -213,7 +213,7 @@ public IndicesClusterStateService( final PrimaryReplicaSyncer primaryReplicaSyncer, final Consumer globalCheckpointSyncer, final RetentionLeaseSyncer retentionLeaseSyncer, - final RemoteStorePressureService remoteStorePressureService + final RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory ) { this.settings = settings; this.checkpointPublisher = checkpointPublisher; @@ -225,7 +225,7 @@ public IndicesClusterStateService( indexEventListeners.add(segmentReplicationSourceService); // if remote store feature is not enabled, do not wire the remote upload pressure service as an IndexEventListener. if (FeatureFlags.isEnabled(FeatureFlags.REMOTE_STORE)) { - indexEventListeners.add(remoteStorePressureService); + indexEventListeners.add(remoteStoreStatsTrackerFactory); } this.segmentReplicationTargetService = segmentReplicationTargetService; this.builtInIndexListener = Collections.unmodifiableList(indexEventListeners); @@ -240,7 +240,7 @@ public IndicesClusterStateService( this.globalCheckpointSyncer = globalCheckpointSyncer; this.retentionLeaseSyncer = Objects.requireNonNull(retentionLeaseSyncer); this.sendRefreshMapping = settings.getAsBoolean("indices.cluster.send_refresh_mapping", true); - this.remoteStorePressureService = remoteStorePressureService; + this.remoteStoreStatsTrackerFactory = remoteStoreStatsTrackerFactory; } @Override @@ -683,7 +683,7 @@ private void createShard(DiscoveryNodes nodes, RoutingTable routingTable, ShardR retentionLeaseSyncer, nodes.getLocalNode(), sourceNode, - remoteStorePressureService + remoteStoreStatsTrackerFactory ); } catch (Exception e) { failAndRemoveShard(shardRouting, true, "failed to create shard", e, state); @@ -1028,6 +1028,7 @@ U createIndex(IndexMetadata indexMetadata, List builtInIndex * @param retentionLeaseSyncer a callback when this shard syncs retention leases * @param targetNode the node where this shard will be recovered * @param sourceNode the source node to recover this shard from (it might be null) + * @param remoteStoreStatsTrackerFactory factory for remote store stats trackers * @return a new shard * @throws IOException if an I/O exception occurs when creating the shard */ @@ -1042,7 +1043,7 @@ T createShard( RetentionLeaseSyncer retentionLeaseSyncer, DiscoveryNode targetNode, @Nullable DiscoveryNode sourceNode, - RemoteStorePressureService remoteStorePressureService + RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory ) throws IOException; /** diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/remotestore/stats/TransportRemoteStoreStatsActionTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/remotestore/stats/TransportRemoteStoreStatsActionTests.java index c34f55d62fe89..75707f2a7853a 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/remotestore/stats/TransportRemoteStoreStatsActionTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/remotestore/stats/TransportRemoteStoreStatsActionTests.java @@ -28,7 +28,7 @@ import org.opensearch.index.IndexService; import org.opensearch.index.IndexSettings; import org.opensearch.index.remote.RemoteSegmentTransferTracker; -import org.opensearch.index.remote.RemoteStorePressureService; +import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory; import org.opensearch.index.shard.IndexShardTestCase; import org.opensearch.indices.IndicesService; import org.opensearch.indices.replication.common.ReplicationType; @@ -53,7 +53,7 @@ public class TransportRemoteStoreStatsActionTests extends IndexShardTestCase { private IndicesService indicesService; - private RemoteStorePressureService pressureService; + private RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory; private IndexMetadata remoteStoreIndexMetadata; private TransportService transportService; private ClusterService clusterService; @@ -67,7 +67,7 @@ public void setUp() throws Exception { indicesService = mock(IndicesService.class); IndexService indexService = mock(IndexService.class); clusterService = mock(ClusterService.class); - pressureService = mock(RemoteStorePressureService.class); + remoteStoreStatsTrackerFactory = mock(RemoteStoreStatsTrackerFactory.class); MockTransport mockTransport = new MockTransport(); localNode = new DiscoveryNode("node0", buildNewFakeTransportAddress(), Version.CURRENT); remoteStoreIndexMetadata = IndexMetadata.builder(INDEX.getName()) @@ -90,7 +90,7 @@ public void setUp() throws Exception { Collections.emptySet() ); - when(pressureService.getRemoteRefreshSegmentTracker(any())).thenReturn(mock(RemoteSegmentTransferTracker.class)); + when(remoteStoreStatsTrackerFactory.getRemoteSegmentTransferTracker(any())).thenReturn(mock(RemoteSegmentTransferTracker.class)); when(indicesService.indexService(INDEX)).thenReturn(indexService); when(indexService.getIndexSettings()).thenReturn(new IndexSettings(remoteStoreIndexMetadata, Settings.EMPTY)); statsAction = new TransportRemoteStoreStatsAction( @@ -99,7 +99,7 @@ public void setUp() throws Exception { indicesService, mock(ActionFilters.class), mock(IndexNameExpressionResolver.class), - pressureService + remoteStoreStatsTrackerFactory ); } diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteSegmentTransferTrackerTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteSegmentTransferTrackerTests.java index 10fe3f95ab47c..e0a05e8d6b49e 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteSegmentTransferTrackerTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteSegmentTransferTrackerTests.java @@ -23,19 +23,15 @@ import java.util.HashMap; import java.util.Map; -import static org.mockito.Mockito.mock; - public class RemoteSegmentTransferTrackerTests extends OpenSearchTestCase { - - private RemoteStorePressureSettings pressureSettings; - + private RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory; private ClusterService clusterService; private ThreadPool threadPool; private ShardId shardId; - private RemoteSegmentTransferTracker pressureTracker; + private RemoteSegmentTransferTracker transferTracker; private DirectoryFileTransferTracker directoryFileTransferTracker; @@ -48,7 +44,7 @@ public void setUp() throws Exception { new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), threadPool ); - pressureSettings = new RemoteStorePressureSettings(clusterService, Settings.EMPTY, mock(RemoteStorePressureService.class)); + remoteStoreStatsTrackerFactory = new RemoteStoreStatsTrackerFactory(clusterService, Settings.EMPTY); shardId = new ShardId("index", "uuid", 0); directoryFileTransferTracker = new DirectoryFileTransferTracker(); } @@ -60,545 +56,475 @@ public void tearDown() throws Exception { } public void testGetShardId() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); - assertEquals(shardId, pressureTracker.getShardId()); + assertEquals(shardId, transferTracker.getShardId()); } public void testUpdateLocalRefreshSeqNo() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); long refreshSeqNo = 2; - pressureTracker.updateLocalRefreshSeqNo(refreshSeqNo); - assertEquals(refreshSeqNo, pressureTracker.getLocalRefreshSeqNo()); + transferTracker.updateLocalRefreshSeqNo(refreshSeqNo); + assertEquals(refreshSeqNo, transferTracker.getLocalRefreshSeqNo()); } public void testUpdateRemoteRefreshSeqNo() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); long refreshSeqNo = 4; - pressureTracker.updateRemoteRefreshSeqNo(refreshSeqNo); - assertEquals(refreshSeqNo, pressureTracker.getRemoteRefreshSeqNo()); + transferTracker.updateRemoteRefreshSeqNo(refreshSeqNo); + assertEquals(refreshSeqNo, transferTracker.getRemoteRefreshSeqNo()); } public void testUpdateLocalRefreshTimeMs() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); long refreshTimeMs = System.nanoTime() / 1_000_000L + randomIntBetween(10, 100); - pressureTracker.updateLocalRefreshTimeMs(refreshTimeMs); - assertEquals(refreshTimeMs, pressureTracker.getLocalRefreshTimeMs()); + transferTracker.updateLocalRefreshTimeMs(refreshTimeMs); + assertEquals(refreshTimeMs, transferTracker.getLocalRefreshTimeMs()); } public void testUpdateRemoteRefreshTimeMs() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); long refreshTimeMs = System.nanoTime() / 1_000_000 + randomIntBetween(10, 100); - pressureTracker.updateRemoteRefreshTimeMs(refreshTimeMs); - assertEquals(refreshTimeMs, pressureTracker.getRemoteRefreshTimeMs()); + transferTracker.updateRemoteRefreshTimeMs(refreshTimeMs); + assertEquals(refreshTimeMs, transferTracker.getRemoteRefreshTimeMs()); } public void testLastDownloadTimestampMs() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); long currentTimeInMs = System.currentTimeMillis(); - pressureTracker.getDirectoryFileTransferTracker().updateLastTransferTimestampMs(currentTimeInMs); - assertEquals(currentTimeInMs, pressureTracker.getDirectoryFileTransferTracker().getLastTransferTimestampMs()); + transferTracker.getDirectoryFileTransferTracker().updateLastTransferTimestampMs(currentTimeInMs); + assertEquals(currentTimeInMs, transferTracker.getDirectoryFileTransferTracker().getLastTransferTimestampMs()); } public void testComputeSeqNoLagOnUpdate() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); int localRefreshSeqNo = randomIntBetween(50, 100); int remoteRefreshSeqNo = randomIntBetween(20, 50); - pressureTracker.updateLocalRefreshSeqNo(localRefreshSeqNo); - assertEquals(localRefreshSeqNo, pressureTracker.getRefreshSeqNoLag()); - pressureTracker.updateRemoteRefreshSeqNo(remoteRefreshSeqNo); - assertEquals(localRefreshSeqNo - remoteRefreshSeqNo, pressureTracker.getRefreshSeqNoLag()); + transferTracker.updateLocalRefreshSeqNo(localRefreshSeqNo); + assertEquals(localRefreshSeqNo, transferTracker.getRefreshSeqNoLag()); + transferTracker.updateRemoteRefreshSeqNo(remoteRefreshSeqNo); + assertEquals(localRefreshSeqNo - remoteRefreshSeqNo, transferTracker.getRefreshSeqNoLag()); } public void testComputeTimeLagOnUpdate() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); - long currentLocalRefreshTimeMs = pressureTracker.getLocalRefreshTimeMs(); + long currentLocalRefreshTimeMs = transferTracker.getLocalRefreshTimeMs(); long currentTimeMs = System.nanoTime() / 1_000_000L; long localRefreshTimeMs = currentTimeMs + randomIntBetween(100, 500); long remoteRefreshTimeMs = currentTimeMs + randomIntBetween(50, 99); - pressureTracker.updateLocalRefreshTimeMs(localRefreshTimeMs); - assertEquals(localRefreshTimeMs - currentLocalRefreshTimeMs, pressureTracker.getTimeMsLag()); - pressureTracker.updateRemoteRefreshTimeMs(remoteRefreshTimeMs); - assertEquals(localRefreshTimeMs - remoteRefreshTimeMs, pressureTracker.getTimeMsLag()); + transferTracker.updateLocalRefreshTimeMs(localRefreshTimeMs); + assertEquals(localRefreshTimeMs - currentLocalRefreshTimeMs, transferTracker.getTimeMsLag()); + transferTracker.updateRemoteRefreshTimeMs(remoteRefreshTimeMs); + assertEquals(localRefreshTimeMs - remoteRefreshTimeMs, transferTracker.getTimeMsLag()); } public void testAddUploadBytesStarted() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); long bytesToAdd = randomLongBetween(1000, 1000000); - pressureTracker.addUploadBytesStarted(bytesToAdd); - assertEquals(bytesToAdd, pressureTracker.getUploadBytesStarted()); + transferTracker.addUploadBytesStarted(bytesToAdd); + assertEquals(bytesToAdd, transferTracker.getUploadBytesStarted()); long moreBytesToAdd = randomLongBetween(1000, 10000); - pressureTracker.addUploadBytesStarted(moreBytesToAdd); - assertEquals(bytesToAdd + moreBytesToAdd, pressureTracker.getUploadBytesStarted()); + transferTracker.addUploadBytesStarted(moreBytesToAdd); + assertEquals(bytesToAdd + moreBytesToAdd, transferTracker.getUploadBytesStarted()); } public void testAddUploadBytesFailed() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); long bytesToAdd = randomLongBetween(1000, 1000000); - pressureTracker.addUploadBytesFailed(bytesToAdd); - assertEquals(bytesToAdd, pressureTracker.getUploadBytesFailed()); + transferTracker.addUploadBytesFailed(bytesToAdd); + assertEquals(bytesToAdd, transferTracker.getUploadBytesFailed()); long moreBytesToAdd = randomLongBetween(1000, 10000); - pressureTracker.addUploadBytesFailed(moreBytesToAdd); - assertEquals(bytesToAdd + moreBytesToAdd, pressureTracker.getUploadBytesFailed()); + transferTracker.addUploadBytesFailed(moreBytesToAdd); + assertEquals(bytesToAdd + moreBytesToAdd, transferTracker.getUploadBytesFailed()); } public void testAddUploadBytesSucceeded() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); long bytesToAdd = randomLongBetween(1000, 1000000); - pressureTracker.addUploadBytesSucceeded(bytesToAdd); - assertEquals(bytesToAdd, pressureTracker.getUploadBytesSucceeded()); + transferTracker.addUploadBytesSucceeded(bytesToAdd); + assertEquals(bytesToAdd, transferTracker.getUploadBytesSucceeded()); long moreBytesToAdd = randomLongBetween(1000, 10000); - pressureTracker.addUploadBytesSucceeded(moreBytesToAdd); - assertEquals(bytesToAdd + moreBytesToAdd, pressureTracker.getUploadBytesSucceeded()); + transferTracker.addUploadBytesSucceeded(moreBytesToAdd); + assertEquals(bytesToAdd + moreBytesToAdd, transferTracker.getUploadBytesSucceeded()); } public void testAddDownloadBytesStarted() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); long bytesToAdd = randomLongBetween(1000, 1000000); - pressureTracker.getDirectoryFileTransferTracker().addTransferredBytesStarted(bytesToAdd); - assertEquals(bytesToAdd, pressureTracker.getDirectoryFileTransferTracker().getTransferredBytesStarted()); + transferTracker.getDirectoryFileTransferTracker().addTransferredBytesStarted(bytesToAdd); + assertEquals(bytesToAdd, transferTracker.getDirectoryFileTransferTracker().getTransferredBytesStarted()); long moreBytesToAdd = randomLongBetween(1000, 10000); - pressureTracker.getDirectoryFileTransferTracker().addTransferredBytesStarted(moreBytesToAdd); - assertEquals(bytesToAdd + moreBytesToAdd, pressureTracker.getDirectoryFileTransferTracker().getTransferredBytesStarted()); + transferTracker.getDirectoryFileTransferTracker().addTransferredBytesStarted(moreBytesToAdd); + assertEquals(bytesToAdd + moreBytesToAdd, transferTracker.getDirectoryFileTransferTracker().getTransferredBytesStarted()); } public void testAddDownloadBytesFailed() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); long bytesToAdd = randomLongBetween(1000, 1000000); - pressureTracker.getDirectoryFileTransferTracker().addTransferredBytesFailed(bytesToAdd, System.currentTimeMillis()); - assertEquals(bytesToAdd, pressureTracker.getDirectoryFileTransferTracker().getTransferredBytesFailed()); + transferTracker.getDirectoryFileTransferTracker().addTransferredBytesFailed(bytesToAdd, System.currentTimeMillis()); + assertEquals(bytesToAdd, transferTracker.getDirectoryFileTransferTracker().getTransferredBytesFailed()); long moreBytesToAdd = randomLongBetween(1000, 10000); - pressureTracker.getDirectoryFileTransferTracker().addTransferredBytesFailed(moreBytesToAdd, System.currentTimeMillis()); - assertEquals(bytesToAdd + moreBytesToAdd, pressureTracker.getDirectoryFileTransferTracker().getTransferredBytesFailed()); + transferTracker.getDirectoryFileTransferTracker().addTransferredBytesFailed(moreBytesToAdd, System.currentTimeMillis()); + assertEquals(bytesToAdd + moreBytesToAdd, transferTracker.getDirectoryFileTransferTracker().getTransferredBytesFailed()); } public void testAddDownloadBytesSucceeded() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); long bytesToAdd = randomLongBetween(1000, 1000000); - pressureTracker.getDirectoryFileTransferTracker().addTransferredBytesSucceeded(bytesToAdd, System.currentTimeMillis()); - assertEquals(bytesToAdd, pressureTracker.getDirectoryFileTransferTracker().getTransferredBytesSucceeded()); + transferTracker.getDirectoryFileTransferTracker().addTransferredBytesSucceeded(bytesToAdd, System.currentTimeMillis()); + assertEquals(bytesToAdd, transferTracker.getDirectoryFileTransferTracker().getTransferredBytesSucceeded()); long moreBytesToAdd = randomLongBetween(1000, 10000); - pressureTracker.getDirectoryFileTransferTracker().addTransferredBytesSucceeded(moreBytesToAdd, System.currentTimeMillis()); - assertEquals(bytesToAdd + moreBytesToAdd, pressureTracker.getDirectoryFileTransferTracker().getTransferredBytesSucceeded()); + transferTracker.getDirectoryFileTransferTracker().addTransferredBytesSucceeded(moreBytesToAdd, System.currentTimeMillis()); + assertEquals(bytesToAdd + moreBytesToAdd, transferTracker.getDirectoryFileTransferTracker().getTransferredBytesSucceeded()); } public void testGetInflightUploadBytes() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); long bytesStarted = randomLongBetween(10000, 100000); long bytesSucceeded = randomLongBetween(1000, 10000); long bytesFailed = randomLongBetween(100, 1000); - pressureTracker.addUploadBytesStarted(bytesStarted); - pressureTracker.addUploadBytesSucceeded(bytesSucceeded); - pressureTracker.addUploadBytesFailed(bytesFailed); - assertEquals(bytesStarted - bytesSucceeded - bytesFailed, pressureTracker.getInflightUploadBytes()); + transferTracker.addUploadBytesStarted(bytesStarted); + transferTracker.addUploadBytesSucceeded(bytesSucceeded); + transferTracker.addUploadBytesFailed(bytesFailed); + assertEquals(bytesStarted - bytesSucceeded - bytesFailed, transferTracker.getInflightUploadBytes()); } public void testIncrementTotalUploadsStarted() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); - pressureTracker.incrementTotalUploadsStarted(); - assertEquals(1, pressureTracker.getTotalUploadsStarted()); - pressureTracker.incrementTotalUploadsStarted(); - assertEquals(2, pressureTracker.getTotalUploadsStarted()); + transferTracker.incrementTotalUploadsStarted(); + assertEquals(1, transferTracker.getTotalUploadsStarted()); + transferTracker.incrementTotalUploadsStarted(); + assertEquals(2, transferTracker.getTotalUploadsStarted()); } public void testIncrementTotalUploadsFailed() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); - pressureTracker.incrementTotalUploadsFailed(); - assertEquals(1, pressureTracker.getTotalUploadsFailed()); - pressureTracker.incrementTotalUploadsFailed(); - assertEquals(2, pressureTracker.getTotalUploadsFailed()); + transferTracker.incrementTotalUploadsFailed(); + assertEquals(1, transferTracker.getTotalUploadsFailed()); + transferTracker.incrementTotalUploadsFailed(); + assertEquals(2, transferTracker.getTotalUploadsFailed()); } public void testIncrementTotalUploadSucceeded() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); - pressureTracker.incrementTotalUploadsSucceeded(); - assertEquals(1, pressureTracker.getTotalUploadsSucceeded()); - pressureTracker.incrementTotalUploadsSucceeded(); - assertEquals(2, pressureTracker.getTotalUploadsSucceeded()); + transferTracker.incrementTotalUploadsSucceeded(); + assertEquals(1, transferTracker.getTotalUploadsSucceeded()); + transferTracker.incrementTotalUploadsSucceeded(); + assertEquals(2, transferTracker.getTotalUploadsSucceeded()); } public void testGetInflightUploads() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); - pressureTracker.incrementTotalUploadsStarted(); - assertEquals(1, pressureTracker.getInflightUploads()); - pressureTracker.incrementTotalUploadsStarted(); - assertEquals(2, pressureTracker.getInflightUploads()); - pressureTracker.incrementTotalUploadsSucceeded(); - assertEquals(1, pressureTracker.getInflightUploads()); - pressureTracker.incrementTotalUploadsFailed(); - assertEquals(0, pressureTracker.getInflightUploads()); + transferTracker.incrementTotalUploadsStarted(); + assertEquals(1, transferTracker.getInflightUploads()); + transferTracker.incrementTotalUploadsStarted(); + assertEquals(2, transferTracker.getInflightUploads()); + transferTracker.incrementTotalUploadsSucceeded(); + assertEquals(1, transferTracker.getInflightUploads()); + transferTracker.incrementTotalUploadsFailed(); + assertEquals(0, transferTracker.getInflightUploads()); } public void testIncrementRejectionCount() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); - pressureTracker.incrementRejectionCount(); - assertEquals(1, pressureTracker.getRejectionCount()); - pressureTracker.incrementRejectionCount(); - assertEquals(2, pressureTracker.getRejectionCount()); + transferTracker.incrementRejectionCount(); + assertEquals(1, transferTracker.getRejectionCount()); + transferTracker.incrementRejectionCount(); + assertEquals(2, transferTracker.getRejectionCount()); } public void testGetConsecutiveFailureCount() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); - pressureTracker.incrementTotalUploadsFailed(); - assertEquals(1, pressureTracker.getConsecutiveFailureCount()); - pressureTracker.incrementTotalUploadsFailed(); - assertEquals(2, pressureTracker.getConsecutiveFailureCount()); - pressureTracker.incrementTotalUploadsSucceeded(); - assertEquals(0, pressureTracker.getConsecutiveFailureCount()); + transferTracker.incrementTotalUploadsFailed(); + assertEquals(1, transferTracker.getConsecutiveFailureCount()); + transferTracker.incrementTotalUploadsFailed(); + assertEquals(2, transferTracker.getConsecutiveFailureCount()); + transferTracker.incrementTotalUploadsSucceeded(); + assertEquals(0, transferTracker.getConsecutiveFailureCount()); } public void testComputeBytesLag() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); // Create local file size map Map fileSizeMap = new HashMap<>(); fileSizeMap.put("a", 100L); fileSizeMap.put("b", 105L); - pressureTracker.updateLatestLocalFileNameLengthMap(fileSizeMap.keySet(), fileSizeMap::get); - assertEquals(205L, pressureTracker.getBytesLag()); + transferTracker.updateLatestLocalFileNameLengthMap(fileSizeMap.keySet(), fileSizeMap::get); + assertEquals(205L, transferTracker.getBytesLag()); - pressureTracker.addToLatestUploadedFiles("a"); - assertEquals(105L, pressureTracker.getBytesLag()); + transferTracker.addToLatestUploadedFiles("a"); + assertEquals(105L, transferTracker.getBytesLag()); fileSizeMap.put("c", 115L); - pressureTracker.updateLatestLocalFileNameLengthMap(fileSizeMap.keySet(), fileSizeMap::get); - assertEquals(220L, pressureTracker.getBytesLag()); + transferTracker.updateLatestLocalFileNameLengthMap(fileSizeMap.keySet(), fileSizeMap::get); + assertEquals(220L, transferTracker.getBytesLag()); - pressureTracker.addToLatestUploadedFiles("b"); - assertEquals(115L, pressureTracker.getBytesLag()); + transferTracker.addToLatestUploadedFiles("b"); + assertEquals(115L, transferTracker.getBytesLag()); - pressureTracker.addToLatestUploadedFiles("c"); - assertEquals(0L, pressureTracker.getBytesLag()); + transferTracker.addToLatestUploadedFiles("c"); + assertEquals(0L, transferTracker.getBytesLag()); } public void testIsUploadBytesAverageReady() { - int uploadBytesMovingAverageWindowSize = pressureSettings.getUploadBytesMovingAverageWindowSize(); - pressureTracker = new RemoteSegmentTransferTracker( - shardId, - directoryFileTransferTracker, - uploadBytesMovingAverageWindowSize, - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() - ); - assertFalse(pressureTracker.isUploadBytesAverageReady()); + int movingAverageWindowSize = remoteStoreStatsTrackerFactory.getMovingAverageWindowSize(); + transferTracker = new RemoteSegmentTransferTracker(shardId, directoryFileTransferTracker, movingAverageWindowSize); + assertFalse(transferTracker.isUploadBytesAverageReady()); long sum = 0; - for (int i = 1; i < uploadBytesMovingAverageWindowSize; i++) { - pressureTracker.addUploadBytes(i); + for (int i = 1; i < movingAverageWindowSize; i++) { + transferTracker.addUploadBytes(i); sum += i; - assertFalse(pressureTracker.isUploadBytesAverageReady()); - assertEquals((double) sum / i, pressureTracker.getUploadBytesAverage(), 0.0d); + assertFalse(transferTracker.isUploadBytesAverageReady()); + assertEquals((double) sum / i, transferTracker.getUploadBytesAverage(), 0.0d); } - pressureTracker.addUploadBytes(uploadBytesMovingAverageWindowSize); - sum += uploadBytesMovingAverageWindowSize; - assertTrue(pressureTracker.isUploadBytesAverageReady()); - assertEquals((double) sum / uploadBytesMovingAverageWindowSize, pressureTracker.getUploadBytesAverage(), 0.0d); + transferTracker.addUploadBytes(movingAverageWindowSize); + sum += movingAverageWindowSize; + assertTrue(transferTracker.isUploadBytesAverageReady()); + assertEquals((double) sum / movingAverageWindowSize, transferTracker.getUploadBytesAverage(), 0.0d); - pressureTracker.addUploadBytes(100); + transferTracker.addUploadBytes(100); sum = sum + 100 - 1; - assertEquals((double) sum / uploadBytesMovingAverageWindowSize, pressureTracker.getUploadBytesAverage(), 0.0d); + assertEquals((double) sum / movingAverageWindowSize, transferTracker.getUploadBytesAverage(), 0.0d); } public void testIsUploadBytesPerSecAverageReady() { - int uploadBytesPerSecMovingAverageWindowSize = pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(); - pressureTracker = new RemoteSegmentTransferTracker( - shardId, - directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - uploadBytesPerSecMovingAverageWindowSize, - pressureSettings.getUploadTimeMovingAverageWindowSize() - ); - assertFalse(pressureTracker.isUploadBytesPerSecAverageReady()); + int movingAverageWindowSize = remoteStoreStatsTrackerFactory.getMovingAverageWindowSize(); + transferTracker = new RemoteSegmentTransferTracker(shardId, directoryFileTransferTracker, movingAverageWindowSize); + assertFalse(transferTracker.isUploadBytesPerSecAverageReady()); long sum = 0; - for (int i = 1; i < uploadBytesPerSecMovingAverageWindowSize; i++) { - pressureTracker.addUploadBytesPerSec(i); + for (int i = 1; i < movingAverageWindowSize; i++) { + transferTracker.addUploadBytesPerSec(i); sum += i; - assertFalse(pressureTracker.isUploadBytesPerSecAverageReady()); - assertEquals((double) sum / i, pressureTracker.getUploadBytesPerSecAverage(), 0.0d); + assertFalse(transferTracker.isUploadBytesPerSecAverageReady()); + assertEquals((double) sum / i, transferTracker.getUploadBytesPerSecAverage(), 0.0d); } - pressureTracker.addUploadBytesPerSec(uploadBytesPerSecMovingAverageWindowSize); - sum += uploadBytesPerSecMovingAverageWindowSize; - assertTrue(pressureTracker.isUploadBytesPerSecAverageReady()); - assertEquals((double) sum / uploadBytesPerSecMovingAverageWindowSize, pressureTracker.getUploadBytesPerSecAverage(), 0.0d); + transferTracker.addUploadBytesPerSec(movingAverageWindowSize); + sum += movingAverageWindowSize; + assertTrue(transferTracker.isUploadBytesPerSecAverageReady()); + assertEquals((double) sum / movingAverageWindowSize, transferTracker.getUploadBytesPerSecAverage(), 0.0d); - pressureTracker.addUploadBytesPerSec(100); + transferTracker.addUploadBytesPerSec(100); sum = sum + 100 - 1; - assertEquals((double) sum / uploadBytesPerSecMovingAverageWindowSize, pressureTracker.getUploadBytesPerSecAverage(), 0.0d); + assertEquals((double) sum / movingAverageWindowSize, transferTracker.getUploadBytesPerSecAverage(), 0.0d); } public void testIsUploadTimeMsAverageReady() { - int uploadTimeMovingAverageWindowSize = pressureSettings.getUploadTimeMovingAverageWindowSize(); - pressureTracker = new RemoteSegmentTransferTracker( - shardId, - directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - uploadTimeMovingAverageWindowSize - ); - assertFalse(pressureTracker.isUploadTimeMsAverageReady()); + int movingAverageWindowSize = remoteStoreStatsTrackerFactory.getMovingAverageWindowSize(); + transferTracker = new RemoteSegmentTransferTracker(shardId, directoryFileTransferTracker, movingAverageWindowSize); + assertFalse(transferTracker.isUploadTimeMsAverageReady()); long sum = 0; - for (int i = 1; i < uploadTimeMovingAverageWindowSize; i++) { - pressureTracker.addTimeForCompletedUploadSync(i); + for (int i = 1; i < movingAverageWindowSize; i++) { + transferTracker.addTimeForCompletedUploadSync(i); sum += i; - assertFalse(pressureTracker.isUploadTimeMsAverageReady()); - assertEquals((double) sum / i, pressureTracker.getUploadTimeMsAverage(), 0.0d); + assertFalse(transferTracker.isUploadTimeMsAverageReady()); + assertEquals((double) sum / i, transferTracker.getUploadTimeMsAverage(), 0.0d); } - pressureTracker.addTimeForCompletedUploadSync(uploadTimeMovingAverageWindowSize); - sum += uploadTimeMovingAverageWindowSize; - assertTrue(pressureTracker.isUploadTimeMsAverageReady()); - assertEquals((double) sum / uploadTimeMovingAverageWindowSize, pressureTracker.getUploadTimeMsAverage(), 0.0d); + transferTracker.addTimeForCompletedUploadSync(movingAverageWindowSize); + sum += movingAverageWindowSize; + assertTrue(transferTracker.isUploadTimeMsAverageReady()); + assertEquals((double) sum / movingAverageWindowSize, transferTracker.getUploadTimeMsAverage(), 0.0d); - pressureTracker.addTimeForCompletedUploadSync(100); + transferTracker.addTimeForCompletedUploadSync(100); sum = sum + 100 - 1; - assertEquals((double) sum / uploadTimeMovingAverageWindowSize, pressureTracker.getUploadTimeMsAverage(), 0.0d); + assertEquals((double) sum / movingAverageWindowSize, transferTracker.getUploadTimeMsAverage(), 0.0d); } public void testIsDownloadBytesAverageReady() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); - assertFalse(pressureTracker.getDirectoryFileTransferTracker().isTransferredBytesAverageReady()); + assertFalse(transferTracker.getDirectoryFileTransferTracker().isTransferredBytesAverageReady()); long sum = 0; for (int i = 1; i < 20; i++) { - pressureTracker.getDirectoryFileTransferTracker().updateSuccessfulTransferSize(i); + transferTracker.getDirectoryFileTransferTracker().updateSuccessfulTransferSize(i); sum += i; - assertFalse(pressureTracker.getDirectoryFileTransferTracker().isTransferredBytesAverageReady()); - assertEquals((double) sum / i, pressureTracker.getDirectoryFileTransferTracker().getTransferredBytesAverage(), 0.0d); + assertFalse(transferTracker.getDirectoryFileTransferTracker().isTransferredBytesAverageReady()); + assertEquals((double) sum / i, transferTracker.getDirectoryFileTransferTracker().getTransferredBytesAverage(), 0.0d); } - pressureTracker.getDirectoryFileTransferTracker().updateSuccessfulTransferSize(20); + transferTracker.getDirectoryFileTransferTracker().updateSuccessfulTransferSize(20); sum += 20; - assertTrue(pressureTracker.getDirectoryFileTransferTracker().isTransferredBytesAverageReady()); - assertEquals((double) sum / 20, pressureTracker.getDirectoryFileTransferTracker().getTransferredBytesAverage(), 0.0d); + assertTrue(transferTracker.getDirectoryFileTransferTracker().isTransferredBytesAverageReady()); + assertEquals((double) sum / 20, transferTracker.getDirectoryFileTransferTracker().getTransferredBytesAverage(), 0.0d); - pressureTracker.getDirectoryFileTransferTracker().updateSuccessfulTransferSize(100); + transferTracker.getDirectoryFileTransferTracker().updateSuccessfulTransferSize(100); sum = sum + 100 - 1; - assertEquals((double) sum / 20, pressureTracker.getDirectoryFileTransferTracker().getTransferredBytesAverage(), 0.0d); + assertEquals((double) sum / 20, transferTracker.getDirectoryFileTransferTracker().getTransferredBytesAverage(), 0.0d); } public void testIsDownloadBytesPerSecAverageReady() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); - assertFalse(pressureTracker.getDirectoryFileTransferTracker().isTransferredBytesPerSecAverageReady()); + assertFalse(transferTracker.getDirectoryFileTransferTracker().isTransferredBytesPerSecAverageReady()); long sum = 0; for (int i = 1; i < 20; i++) { - pressureTracker.getDirectoryFileTransferTracker().addTransferredBytesPerSec(i); + transferTracker.getDirectoryFileTransferTracker().addTransferredBytesPerSec(i); sum += i; - assertFalse(pressureTracker.getDirectoryFileTransferTracker().isTransferredBytesPerSecAverageReady()); - assertEquals((double) sum / i, pressureTracker.getDirectoryFileTransferTracker().getTransferredBytesPerSecAverage(), 0.0d); + assertFalse(transferTracker.getDirectoryFileTransferTracker().isTransferredBytesPerSecAverageReady()); + assertEquals((double) sum / i, transferTracker.getDirectoryFileTransferTracker().getTransferredBytesPerSecAverage(), 0.0d); } - pressureTracker.getDirectoryFileTransferTracker().addTransferredBytesPerSec(20); + transferTracker.getDirectoryFileTransferTracker().addTransferredBytesPerSec(20); sum += 20; - assertTrue(pressureTracker.getDirectoryFileTransferTracker().isTransferredBytesPerSecAverageReady()); - assertEquals((double) sum / 20, pressureTracker.getDirectoryFileTransferTracker().getTransferredBytesPerSecAverage(), 0.0d); + assertTrue(transferTracker.getDirectoryFileTransferTracker().isTransferredBytesPerSecAverageReady()); + assertEquals((double) sum / 20, transferTracker.getDirectoryFileTransferTracker().getTransferredBytesPerSecAverage(), 0.0d); - pressureTracker.getDirectoryFileTransferTracker().addTransferredBytesPerSec(100); + transferTracker.getDirectoryFileTransferTracker().addTransferredBytesPerSec(100); sum = sum + 100 - 1; - assertEquals((double) sum / 20, pressureTracker.getDirectoryFileTransferTracker().getTransferredBytesPerSecAverage(), 0.0d); + assertEquals((double) sum / 20, transferTracker.getDirectoryFileTransferTracker().getTransferredBytesPerSecAverage(), 0.0d); } public void testAddTotalUploadTimeInMs() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); long timeToAdd = randomLongBetween(100, 200); - pressureTracker.addTotalUploadTimeInMs(timeToAdd); - assertEquals(timeToAdd, pressureTracker.getTotalUploadTimeInMs()); + transferTracker.addTotalUploadTimeInMs(timeToAdd); + assertEquals(timeToAdd, transferTracker.getTotalUploadTimeInMs()); long moreTimeToAdd = randomLongBetween(100, 200); - pressureTracker.addTotalUploadTimeInMs(moreTimeToAdd); - assertEquals(timeToAdd + moreTimeToAdd, pressureTracker.getTotalUploadTimeInMs()); + transferTracker.addTotalUploadTimeInMs(moreTimeToAdd); + assertEquals(timeToAdd + moreTimeToAdd, transferTracker.getTotalUploadTimeInMs()); } public void testAddTotalTransferTimeMs() { - pressureTracker = new RemoteSegmentTransferTracker( + transferTracker = new RemoteSegmentTransferTracker( shardId, directoryFileTransferTracker, - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); long timeToAdd = randomLongBetween(100, 200); - pressureTracker.getDirectoryFileTransferTracker().addTotalTransferTimeInMs(timeToAdd); - assertEquals(timeToAdd, pressureTracker.getDirectoryFileTransferTracker().getTotalTransferTimeInMs()); + transferTracker.getDirectoryFileTransferTracker().addTotalTransferTimeInMs(timeToAdd); + assertEquals(timeToAdd, transferTracker.getDirectoryFileTransferTracker().getTotalTransferTimeInMs()); long moreTimeToAdd = randomLongBetween(100, 200); - pressureTracker.getDirectoryFileTransferTracker().addTotalTransferTimeInMs(moreTimeToAdd); - assertEquals(timeToAdd + moreTimeToAdd, pressureTracker.getDirectoryFileTransferTracker().getTotalTransferTimeInMs()); + transferTracker.getDirectoryFileTransferTracker().addTotalTransferTimeInMs(moreTimeToAdd); + assertEquals(timeToAdd + moreTimeToAdd, transferTracker.getDirectoryFileTransferTracker().getTotalTransferTimeInMs()); } /** * Tests whether RemoteSegmentTransferTracker.Stats object generated correctly from RemoteSegmentTransferTracker. * */ public void testStatsObjectCreation() { - pressureTracker = constructTracker(); - RemoteSegmentTransferTracker.Stats pressureTrackerStats = pressureTracker.stats(); - assertEquals(pressureTracker.getShardId(), pressureTrackerStats.shardId); - assertEquals(pressureTracker.getTimeMsLag(), (int) pressureTrackerStats.refreshTimeLagMs); - assertEquals(pressureTracker.getLocalRefreshSeqNo(), (int) pressureTrackerStats.localRefreshNumber); - assertEquals(pressureTracker.getRemoteRefreshSeqNo(), (int) pressureTrackerStats.remoteRefreshNumber); - assertEquals(pressureTracker.getBytesLag(), (int) pressureTrackerStats.bytesLag); - assertEquals(pressureTracker.getRejectionCount(), (int) pressureTrackerStats.rejectionCount); - assertEquals(pressureTracker.getConsecutiveFailureCount(), (int) pressureTrackerStats.consecutiveFailuresCount); - assertEquals(pressureTracker.getUploadBytesStarted(), (int) pressureTrackerStats.uploadBytesStarted); - assertEquals(pressureTracker.getUploadBytesSucceeded(), (int) pressureTrackerStats.uploadBytesSucceeded); - assertEquals(pressureTracker.getUploadBytesFailed(), (int) pressureTrackerStats.uploadBytesFailed); - assertEquals(pressureTracker.getUploadBytesAverage(), pressureTrackerStats.uploadBytesMovingAverage, 0); - assertEquals(pressureTracker.getUploadBytesPerSecAverage(), pressureTrackerStats.uploadBytesPerSecMovingAverage, 0); - assertEquals(pressureTracker.getUploadTimeMsAverage(), pressureTrackerStats.uploadTimeMovingAverage, 0); - assertEquals(pressureTracker.getTotalUploadsStarted(), (int) pressureTrackerStats.totalUploadsStarted); - assertEquals(pressureTracker.getTotalUploadsSucceeded(), (int) pressureTrackerStats.totalUploadsSucceeded); - assertEquals(pressureTracker.getTotalUploadsFailed(), (int) pressureTrackerStats.totalUploadsFailed); + transferTracker = constructTracker(); + RemoteSegmentTransferTracker.Stats transferTrackerStats = transferTracker.stats(); + assertEquals(transferTracker.getShardId(), transferTrackerStats.shardId); + assertEquals(transferTracker.getTimeMsLag(), (int) transferTrackerStats.refreshTimeLagMs); + assertEquals(transferTracker.getLocalRefreshSeqNo(), (int) transferTrackerStats.localRefreshNumber); + assertEquals(transferTracker.getRemoteRefreshSeqNo(), (int) transferTrackerStats.remoteRefreshNumber); + assertEquals(transferTracker.getBytesLag(), (int) transferTrackerStats.bytesLag); + assertEquals(transferTracker.getRejectionCount(), (int) transferTrackerStats.rejectionCount); + assertEquals(transferTracker.getConsecutiveFailureCount(), (int) transferTrackerStats.consecutiveFailuresCount); + assertEquals(transferTracker.getUploadBytesStarted(), (int) transferTrackerStats.uploadBytesStarted); + assertEquals(transferTracker.getUploadBytesSucceeded(), (int) transferTrackerStats.uploadBytesSucceeded); + assertEquals(transferTracker.getUploadBytesFailed(), (int) transferTrackerStats.uploadBytesFailed); + assertEquals(transferTracker.getUploadBytesAverage(), transferTrackerStats.uploadBytesMovingAverage, 0); + assertEquals(transferTracker.getUploadBytesPerSecAverage(), transferTrackerStats.uploadBytesPerSecMovingAverage, 0); + assertEquals(transferTracker.getUploadTimeMsAverage(), transferTrackerStats.uploadTimeMovingAverage, 0); + assertEquals(transferTracker.getTotalUploadsStarted(), (int) transferTrackerStats.totalUploadsStarted); + assertEquals(transferTracker.getTotalUploadsSucceeded(), (int) transferTrackerStats.totalUploadsSucceeded); + assertEquals(transferTracker.getTotalUploadsFailed(), (int) transferTrackerStats.totalUploadsFailed); } /** @@ -606,64 +532,62 @@ public void testStatsObjectCreation() { * This comes into play during internode data transfer. */ public void testStatsObjectCreationViaStream() throws IOException { - pressureTracker = constructTracker(); - RemoteSegmentTransferTracker.Stats pressureTrackerStats = pressureTracker.stats(); + transferTracker = constructTracker(); + RemoteSegmentTransferTracker.Stats transferTrackerStats = transferTracker.stats(); try (BytesStreamOutput out = new BytesStreamOutput()) { - pressureTrackerStats.writeTo(out); + transferTrackerStats.writeTo(out); try (StreamInput in = out.bytes().streamInput()) { RemoteSegmentTransferTracker.Stats deserializedStats = new RemoteSegmentTransferTracker.Stats(in); - assertEquals(deserializedStats.shardId, pressureTrackerStats.shardId); - assertEquals((int) deserializedStats.refreshTimeLagMs, (int) pressureTrackerStats.refreshTimeLagMs); - assertEquals((int) deserializedStats.localRefreshNumber, (int) pressureTrackerStats.localRefreshNumber); - assertEquals((int) deserializedStats.remoteRefreshNumber, (int) pressureTrackerStats.remoteRefreshNumber); - assertEquals((int) deserializedStats.bytesLag, (int) pressureTrackerStats.bytesLag); - assertEquals((int) deserializedStats.rejectionCount, (int) pressureTrackerStats.rejectionCount); - assertEquals((int) deserializedStats.consecutiveFailuresCount, (int) pressureTrackerStats.consecutiveFailuresCount); - assertEquals((int) deserializedStats.uploadBytesStarted, (int) pressureTrackerStats.uploadBytesStarted); - assertEquals((int) deserializedStats.uploadBytesSucceeded, (int) pressureTrackerStats.uploadBytesSucceeded); - assertEquals((int) deserializedStats.uploadBytesFailed, (int) pressureTrackerStats.uploadBytesFailed); - assertEquals((int) deserializedStats.uploadBytesMovingAverage, pressureTrackerStats.uploadBytesMovingAverage, 0); + assertEquals(deserializedStats.shardId, transferTrackerStats.shardId); + assertEquals((int) deserializedStats.refreshTimeLagMs, (int) transferTrackerStats.refreshTimeLagMs); + assertEquals((int) deserializedStats.localRefreshNumber, (int) transferTrackerStats.localRefreshNumber); + assertEquals((int) deserializedStats.remoteRefreshNumber, (int) transferTrackerStats.remoteRefreshNumber); + assertEquals((int) deserializedStats.bytesLag, (int) transferTrackerStats.bytesLag); + assertEquals((int) deserializedStats.rejectionCount, (int) transferTrackerStats.rejectionCount); + assertEquals((int) deserializedStats.consecutiveFailuresCount, (int) transferTrackerStats.consecutiveFailuresCount); + assertEquals((int) deserializedStats.uploadBytesStarted, (int) transferTrackerStats.uploadBytesStarted); + assertEquals((int) deserializedStats.uploadBytesSucceeded, (int) transferTrackerStats.uploadBytesSucceeded); + assertEquals((int) deserializedStats.uploadBytesFailed, (int) transferTrackerStats.uploadBytesFailed); + assertEquals((int) deserializedStats.uploadBytesMovingAverage, transferTrackerStats.uploadBytesMovingAverage, 0); assertEquals( (int) deserializedStats.uploadBytesPerSecMovingAverage, - pressureTrackerStats.uploadBytesPerSecMovingAverage, + transferTrackerStats.uploadBytesPerSecMovingAverage, 0 ); - assertEquals((int) deserializedStats.uploadTimeMovingAverage, pressureTrackerStats.uploadTimeMovingAverage, 0); - assertEquals((int) deserializedStats.totalUploadsStarted, (int) pressureTrackerStats.totalUploadsStarted); - assertEquals((int) deserializedStats.totalUploadsSucceeded, (int) pressureTrackerStats.totalUploadsSucceeded); - assertEquals((int) deserializedStats.totalUploadsFailed, (int) pressureTrackerStats.totalUploadsFailed); + assertEquals((int) deserializedStats.uploadTimeMovingAverage, transferTrackerStats.uploadTimeMovingAverage, 0); + assertEquals((int) deserializedStats.totalUploadsStarted, (int) transferTrackerStats.totalUploadsStarted); + assertEquals((int) deserializedStats.totalUploadsSucceeded, (int) transferTrackerStats.totalUploadsSucceeded); + assertEquals((int) deserializedStats.totalUploadsFailed, (int) transferTrackerStats.totalUploadsFailed); assertEquals( (int) deserializedStats.directoryFileTransferTrackerStats.transferredBytesStarted, - (int) pressureTrackerStats.directoryFileTransferTrackerStats.transferredBytesStarted + (int) transferTrackerStats.directoryFileTransferTrackerStats.transferredBytesStarted ); assertEquals( (int) deserializedStats.directoryFileTransferTrackerStats.transferredBytesSucceeded, - (int) pressureTrackerStats.directoryFileTransferTrackerStats.transferredBytesSucceeded + (int) transferTrackerStats.directoryFileTransferTrackerStats.transferredBytesSucceeded ); assertEquals( (int) deserializedStats.directoryFileTransferTrackerStats.transferredBytesPerSecMovingAverage, - (int) pressureTrackerStats.directoryFileTransferTrackerStats.transferredBytesPerSecMovingAverage + (int) transferTrackerStats.directoryFileTransferTrackerStats.transferredBytesPerSecMovingAverage ); } } } private RemoteSegmentTransferTracker constructTracker() { - RemoteSegmentTransferTracker segmentPressureTracker = new RemoteSegmentTransferTracker( + RemoteSegmentTransferTracker transferTracker = new RemoteSegmentTransferTracker( shardId, new DirectoryFileTransferTracker(), - pressureSettings.getUploadBytesMovingAverageWindowSize(), - pressureSettings.getUploadBytesPerSecMovingAverageWindowSize(), - pressureSettings.getUploadTimeMovingAverageWindowSize() + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() ); - segmentPressureTracker.incrementTotalUploadsFailed(); - segmentPressureTracker.addTimeForCompletedUploadSync(System.nanoTime() / 1_000_000L + randomIntBetween(10, 100)); - segmentPressureTracker.addUploadBytes(99); - segmentPressureTracker.updateRemoteRefreshTimeMs(System.nanoTime() / 1_000_000L + randomIntBetween(10, 100)); - segmentPressureTracker.incrementRejectionCount(); - segmentPressureTracker.getDirectoryFileTransferTracker().addTransferredBytesStarted(10); - segmentPressureTracker.getDirectoryFileTransferTracker().addTransferredBytesSucceeded(10, System.currentTimeMillis()); - segmentPressureTracker.getDirectoryFileTransferTracker().addTransferredBytesPerSec(5); - return segmentPressureTracker; + transferTracker.incrementTotalUploadsFailed(); + transferTracker.addTimeForCompletedUploadSync(System.nanoTime() / 1_000_000L + randomIntBetween(10, 100)); + transferTracker.addUploadBytes(99); + transferTracker.updateRemoteRefreshTimeMs(System.nanoTime() / 1_000_000L + randomIntBetween(10, 100)); + transferTracker.incrementRejectionCount(); + transferTracker.getDirectoryFileTransferTracker().addTransferredBytesStarted(10); + transferTracker.getDirectoryFileTransferTracker().addTransferredBytesSucceeded(10, System.currentTimeMillis()); + transferTracker.getDirectoryFileTransferTracker().addTransferredBytesPerSec(5); + return transferTracker; } } diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteStorePressureServiceTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteStorePressureServiceTests.java index e164269d96a3d..355333e74f826 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteStorePressureServiceTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteStorePressureServiceTests.java @@ -8,17 +8,12 @@ package org.opensearch.index.remote; -import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.core.index.shard.ShardId; -import org.opensearch.index.IndexSettings; import org.opensearch.index.shard.IndexShard; -import org.opensearch.index.store.Store; -import org.opensearch.indices.replication.common.ReplicationType; -import org.opensearch.test.IndexSettingsModule; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; @@ -28,8 +23,7 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.stream.IntStream; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; +import static org.opensearch.index.remote.RemoteStoreTestsHelper.createIndexShard; public class RemoteStorePressureServiceTests extends OpenSearchTestCase { @@ -41,6 +35,8 @@ public class RemoteStorePressureServiceTests extends OpenSearchTestCase { private RemoteStorePressureService pressureService; + private RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory; + @Override public void setUp() throws Exception { super.setUp(); @@ -60,7 +56,8 @@ public void tearDown() throws Exception { } public void testIsSegmentsUploadBackpressureEnabled() { - pressureService = new RemoteStorePressureService(clusterService, Settings.EMPTY); + remoteStoreStatsTrackerFactory = new RemoteStoreStatsTrackerFactory(clusterService, Settings.EMPTY); + pressureService = new RemoteStorePressureService(clusterService, Settings.EMPTY, remoteStoreStatsTrackerFactory); assertFalse(pressureService.isSegmentsUploadBackpressureEnabled()); Settings newSettings = Settings.builder() @@ -71,37 +68,14 @@ public void testIsSegmentsUploadBackpressureEnabled() { assertTrue(pressureService.isSegmentsUploadBackpressureEnabled()); } - public void testAfterIndexShardCreatedForRemoteBackedIndex() { - IndexShard indexShard = createIndexShard(shardId, true); - pressureService = new RemoteStorePressureService(clusterService, Settings.EMPTY); - pressureService.afterIndexShardCreated(indexShard); - assertNotNull(pressureService.getRemoteRefreshSegmentTracker(indexShard.shardId())); - } - - public void testAfterIndexShardCreatedForNonRemoteBackedIndex() { - IndexShard indexShard = createIndexShard(shardId, false); - pressureService = new RemoteStorePressureService(clusterService, Settings.EMPTY); - pressureService.afterIndexShardCreated(indexShard); - assertNull(pressureService.getRemoteRefreshSegmentTracker(indexShard.shardId())); - } - - public void testAfterIndexShardClosed() { - IndexShard indexShard = createIndexShard(shardId, true); - pressureService = new RemoteStorePressureService(clusterService, Settings.EMPTY); - pressureService.afterIndexShardCreated(indexShard); - assertNotNull(pressureService.getRemoteRefreshSegmentTracker(shardId)); - - pressureService.afterIndexShardClosed(shardId, indexShard, indexShard.indexSettings().getSettings()); - assertNull(pressureService.getRemoteRefreshSegmentTracker(shardId)); - } - public void testValidateSegmentUploadLag() { // Create the pressure tracker IndexShard indexShard = createIndexShard(shardId, true); - pressureService = new RemoteStorePressureService(clusterService, Settings.EMPTY); - pressureService.afterIndexShardCreated(indexShard); + remoteStoreStatsTrackerFactory = new RemoteStoreStatsTrackerFactory(clusterService, Settings.EMPTY); + pressureService = new RemoteStorePressureService(clusterService, Settings.EMPTY, remoteStoreStatsTrackerFactory); + remoteStoreStatsTrackerFactory.afterIndexShardCreated(indexShard); - RemoteSegmentTransferTracker pressureTracker = pressureService.getRemoteRefreshSegmentTracker(shardId); + RemoteSegmentTransferTracker pressureTracker = remoteStoreStatsTrackerFactory.getRemoteSegmentTransferTracker(shardId); pressureTracker.updateLocalRefreshSeqNo(6); // 1. time lag more than dynamic threshold @@ -152,17 +126,4 @@ public void testValidateSegmentUploadLag() { pressureService.validateSegmentsUploadLag(shardId); } - private static IndexShard createIndexShard(ShardId shardId, boolean remoteStoreEnabled) { - Settings settings = Settings.builder() - .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) - .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, String.valueOf(remoteStoreEnabled)) - .build(); - IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test_index", settings); - Store store = mock(Store.class); - IndexShard indexShard = mock(IndexShard.class); - when(indexShard.indexSettings()).thenReturn(indexSettings); - when(indexShard.shardId()).thenReturn(shardId); - when(indexShard.store()).thenReturn(store); - return indexShard; - } } diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteStorePressureSettingsTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteStorePressureSettingsTests.java index 9c5ec69cf6be9..f5514b8936a2f 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteStorePressureSettingsTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteStorePressureSettingsTests.java @@ -15,10 +15,6 @@ import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; -import java.util.concurrent.atomic.AtomicInteger; - -import static org.mockito.ArgumentMatchers.anyInt; -import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; public class RemoteStorePressureSettingsTests extends OpenSearchTestCase { @@ -62,15 +58,6 @@ public void testGetDefaultSettings() { // Check minimum consecutive failures limit default value assertEquals(5, pressureSettings.getMinConsecutiveFailuresLimit()); - - // Check upload bytes moving average window size default value - assertEquals(20, pressureSettings.getUploadBytesMovingAverageWindowSize()); - - // Check upload bytes per sec moving average window size default value - assertEquals(20, pressureSettings.getUploadBytesPerSecMovingAverageWindowSize()); - - // Check upload time moving average window size default value - assertEquals(20, pressureSettings.getUploadTimeMovingAverageWindowSize()); } public void testGetConfiguredSettings() { @@ -79,9 +66,6 @@ public void testGetConfiguredSettings() { .put(RemoteStorePressureSettings.BYTES_LAG_VARIANCE_FACTOR.getKey(), 50.0) .put(RemoteStorePressureSettings.UPLOAD_TIME_LAG_VARIANCE_FACTOR.getKey(), 60.0) .put(RemoteStorePressureSettings.MIN_CONSECUTIVE_FAILURES_LIMIT.getKey(), 121) - .put(RemoteStorePressureSettings.UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 102) - .put(RemoteStorePressureSettings.UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 103) - .put(RemoteStorePressureSettings.UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 104) .build(); RemoteStorePressureSettings pressureSettings = new RemoteStorePressureSettings( clusterService, @@ -100,15 +84,6 @@ public void testGetConfiguredSettings() { // Check minimum consecutive failures limit configured value assertEquals(121, pressureSettings.getMinConsecutiveFailuresLimit()); - - // Check upload bytes moving average window size configured value - assertEquals(102, pressureSettings.getUploadBytesMovingAverageWindowSize()); - - // Check upload bytes per sec moving average window size configured value - assertEquals(103, pressureSettings.getUploadBytesPerSecMovingAverageWindowSize()); - - // Check upload time moving average window size configured value - assertEquals(104, pressureSettings.getUploadTimeMovingAverageWindowSize()); } public void testUpdateAfterGetDefaultSettings() { @@ -123,9 +98,6 @@ public void testUpdateAfterGetDefaultSettings() { .put(RemoteStorePressureSettings.BYTES_LAG_VARIANCE_FACTOR.getKey(), 50.0) .put(RemoteStorePressureSettings.UPLOAD_TIME_LAG_VARIANCE_FACTOR.getKey(), 60.0) .put(RemoteStorePressureSettings.MIN_CONSECUTIVE_FAILURES_LIMIT.getKey(), 121) - .put(RemoteStorePressureSettings.UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 102) - .put(RemoteStorePressureSettings.UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 103) - .put(RemoteStorePressureSettings.UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 104) .build(); clusterService.getClusterSettings().applySettings(newSettings); @@ -140,15 +112,6 @@ public void testUpdateAfterGetDefaultSettings() { // Check minimum consecutive failures limit updated assertEquals(121, pressureSettings.getMinConsecutiveFailuresLimit()); - - // Check upload bytes moving average window size updated - assertEquals(102, pressureSettings.getUploadBytesMovingAverageWindowSize()); - - // Check upload bytes per sec moving average window size updated - assertEquals(103, pressureSettings.getUploadBytesPerSecMovingAverageWindowSize()); - - // Check upload time moving average window size updated - assertEquals(104, pressureSettings.getUploadTimeMovingAverageWindowSize()); } public void testUpdateAfterGetConfiguredSettings() { @@ -157,9 +120,6 @@ public void testUpdateAfterGetConfiguredSettings() { .put(RemoteStorePressureSettings.BYTES_LAG_VARIANCE_FACTOR.getKey(), 50.0) .put(RemoteStorePressureSettings.UPLOAD_TIME_LAG_VARIANCE_FACTOR.getKey(), 60.0) .put(RemoteStorePressureSettings.MIN_CONSECUTIVE_FAILURES_LIMIT.getKey(), 121) - .put(RemoteStorePressureSettings.UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 102) - .put(RemoteStorePressureSettings.UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 103) - .put(RemoteStorePressureSettings.UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 104) .build(); RemoteStorePressureSettings pressureSettings = new RemoteStorePressureSettings( clusterService, @@ -171,9 +131,6 @@ public void testUpdateAfterGetConfiguredSettings() { .put(RemoteStorePressureSettings.BYTES_LAG_VARIANCE_FACTOR.getKey(), 40.0) .put(RemoteStorePressureSettings.UPLOAD_TIME_LAG_VARIANCE_FACTOR.getKey(), 50.0) .put(RemoteStorePressureSettings.MIN_CONSECUTIVE_FAILURES_LIMIT.getKey(), 111) - .put(RemoteStorePressureSettings.UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 112) - .put(RemoteStorePressureSettings.UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 113) - .put(RemoteStorePressureSettings.UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE.getKey(), 114) .build(); clusterService.getClusterSettings().applySettings(newSettings); @@ -189,59 +146,5 @@ public void testUpdateAfterGetConfiguredSettings() { // Check minimum consecutive failures limit updated assertEquals(111, pressureSettings.getMinConsecutiveFailuresLimit()); - - // Check upload bytes moving average window size updated - assertEquals(112, pressureSettings.getUploadBytesMovingAverageWindowSize()); - - // Check upload bytes per sec moving average window size updated - assertEquals(113, pressureSettings.getUploadBytesPerSecMovingAverageWindowSize()); - - // Check upload time moving average window size updated - assertEquals(114, pressureSettings.getUploadTimeMovingAverageWindowSize()); - } - - public void testUpdateTriggeredInRemotePressureServiceOnUpdateSettings() { - - int toUpdateVal1 = 1121, toUpdateVal2 = 1123, toUpdateVal3 = 1125; - - AtomicInteger updatedUploadBytesWindowSize = new AtomicInteger(); - AtomicInteger updatedUploadBytesPerSecWindowSize = new AtomicInteger(); - AtomicInteger updatedUploadTimeWindowSize = new AtomicInteger(); - - RemoteStorePressureService pressureService = mock(RemoteStorePressureService.class); - - // Upload bytes - doAnswer(invocation -> { - updatedUploadBytesWindowSize.set(invocation.getArgument(0)); - return null; - }).when(pressureService).updateUploadBytesMovingAverageWindowSize(anyInt()); - - // Upload bytes per sec - doAnswer(invocation -> { - updatedUploadBytesPerSecWindowSize.set(invocation.getArgument(0)); - return null; - }).when(pressureService).updateUploadBytesPerSecMovingAverageWindowSize(anyInt()); - - // Upload time - doAnswer(invocation -> { - updatedUploadTimeWindowSize.set(invocation.getArgument(0)); - return null; - }).when(pressureService).updateUploadTimeMsMovingAverageWindowSize(anyInt()); - - RemoteStorePressureSettings pressureSettings = new RemoteStorePressureSettings(clusterService, Settings.EMPTY, pressureService); - Settings newSettings = Settings.builder() - .put(RemoteStorePressureSettings.UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE.getKey(), toUpdateVal1) - .put(RemoteStorePressureSettings.UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE.getKey(), toUpdateVal2) - .put(RemoteStorePressureSettings.UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE.getKey(), toUpdateVal3) - .build(); - clusterService.getClusterSettings().applySettings(newSettings); - - // Assertions - assertEquals(toUpdateVal1, pressureSettings.getUploadBytesMovingAverageWindowSize()); - assertEquals(toUpdateVal1, updatedUploadBytesWindowSize.get()); - assertEquals(toUpdateVal2, pressureSettings.getUploadBytesPerSecMovingAverageWindowSize()); - assertEquals(toUpdateVal2, updatedUploadBytesPerSecWindowSize.get()); - assertEquals(toUpdateVal3, pressureSettings.getUploadTimeMovingAverageWindowSize()); - assertEquals(toUpdateVal3, updatedUploadTimeWindowSize.get()); } } diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteStoreStatsTrackerFactoryTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteStoreStatsTrackerFactoryTests.java new file mode 100644 index 0000000000000..c300f316ac633 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/remote/RemoteStoreStatsTrackerFactoryTests.java @@ -0,0 +1,119 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.remote; + +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; + +import static org.opensearch.index.remote.RemoteStoreTestsHelper.createIndexShard; + +public class RemoteStoreStatsTrackerFactoryTests extends OpenSearchTestCase { + private ThreadPool threadPool; + private ClusterService clusterService; + private Settings settings; + private ShardId shardId; + private IndexShard indexShard; + private RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory; + + @Override + public void setUp() throws Exception { + super.setUp(); + shardId = new ShardId("index", "uuid", 0); + indexShard = createIndexShard(shardId, true); + threadPool = new TestThreadPool(getTestName()); + settings = Settings.builder() + .put( + RemoteStoreStatsTrackerFactory.MOVING_AVERAGE_WINDOW_SIZE.getKey(), + RemoteStoreStatsTrackerFactory.Defaults.MOVING_AVERAGE_WINDOW_SIZE_MIN_VALUE + ) + .build(); + clusterService = new ClusterService(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), threadPool); + remoteStoreStatsTrackerFactory = new RemoteStoreStatsTrackerFactory(clusterService, settings); + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + threadPool.shutdownNow(); + } + + public void testAfterIndexShardCreatedForRemoteBackedIndex() { + remoteStoreStatsTrackerFactory.afterIndexShardCreated(indexShard); + assertNotNull(remoteStoreStatsTrackerFactory.getRemoteSegmentTransferTracker(indexShard.shardId())); + } + + public void testAfterIndexShardCreatedForNonRemoteBackedIndex() { + indexShard = createIndexShard(shardId, false); + remoteStoreStatsTrackerFactory.afterIndexShardCreated(indexShard); + assertNull(remoteStoreStatsTrackerFactory.getRemoteSegmentTransferTracker(indexShard.shardId())); + } + + public void testAfterIndexShardClosed() { + remoteStoreStatsTrackerFactory.afterIndexShardCreated(indexShard); + assertNotNull(remoteStoreStatsTrackerFactory.getRemoteSegmentTransferTracker(shardId)); + remoteStoreStatsTrackerFactory.afterIndexShardClosed(shardId, indexShard, indexShard.indexSettings().getSettings()); + assertNull(remoteStoreStatsTrackerFactory.getRemoteSegmentTransferTracker(shardId)); + } + + public void testGetConfiguredSettings() { + assertEquals( + RemoteStoreStatsTrackerFactory.Defaults.MOVING_AVERAGE_WINDOW_SIZE_MIN_VALUE, + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() + ); + } + + public void testInvalidMovingAverageWindowSize() { + Settings settings = Settings.builder() + .put( + RemoteStoreStatsTrackerFactory.MOVING_AVERAGE_WINDOW_SIZE.getKey(), + RemoteStoreStatsTrackerFactory.Defaults.MOVING_AVERAGE_WINDOW_SIZE_MIN_VALUE - 1 + ) + .build(); + assertThrows( + "Failed to parse value", + IllegalArgumentException.class, + () -> new RemoteStoreStatsTrackerFactory( + new ClusterService(settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), threadPool), + settings + ) + ); + } + + public void testUpdateAfterGetConfiguredSettings() { + assertEquals( + RemoteStoreStatsTrackerFactory.Defaults.MOVING_AVERAGE_WINDOW_SIZE_MIN_VALUE, + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() + ); + + Settings newSettings = Settings.builder().put(RemoteStoreStatsTrackerFactory.MOVING_AVERAGE_WINDOW_SIZE.getKey(), 102).build(); + + clusterService.getClusterSettings().applySettings(newSettings); + + // Check moving average window size updated + assertEquals(102, remoteStoreStatsTrackerFactory.getMovingAverageWindowSize()); + } + + public void testGetDefaultSettings() { + remoteStoreStatsTrackerFactory = new RemoteStoreStatsTrackerFactory( + new ClusterService(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), threadPool), + Settings.EMPTY + ); + // Check moving average window size updated + assertEquals( + RemoteStoreStatsTrackerFactory.Defaults.MOVING_AVERAGE_WINDOW_SIZE, + remoteStoreStatsTrackerFactory.getMovingAverageWindowSize() + ); + } +} diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteStoreTestsHelper.java b/server/src/test/java/org/opensearch/index/remote/RemoteStoreTestsHelper.java new file mode 100644 index 0000000000000..e072d3037caad --- /dev/null +++ b/server/src/test/java/org/opensearch/index/remote/RemoteStoreTestsHelper.java @@ -0,0 +1,40 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.remote; + +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.store.Store; +import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.test.IndexSettingsModule; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** + * Helper functions for Remote Store tests + */ +public class RemoteStoreTestsHelper { + static IndexShard createIndexShard(ShardId shardId, boolean remoteStoreEnabled) { + Settings settings = Settings.builder() + .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) + .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, String.valueOf(remoteStoreEnabled)) + .build(); + IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test_index", settings); + Store store = mock(Store.class); + IndexShard indexShard = mock(IndexShard.class); + when(indexShard.indexSettings()).thenReturn(indexSettings); + when(indexShard.shardId()).thenReturn(shardId); + when(indexShard.store()).thenReturn(store); + return indexShard; + } +} diff --git a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java index df128fbffd81d..2281c086db5d8 100644 --- a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java @@ -1821,8 +1821,8 @@ public void testShardStatsWithRemoteStoreEnabled() throws IOException { .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) .build() ); - RemoteSegmentTransferTracker remoteRefreshSegmentTracker = shard.getRemoteStorePressureService() - .getRemoteRefreshSegmentTracker(shard.shardId); + RemoteSegmentTransferTracker remoteRefreshSegmentTracker = shard.getRemoteStoreStatsTrackerFactory() + .getRemoteSegmentTransferTracker(shard.shardId); populateSampleRemoteStoreStats(remoteRefreshSegmentTracker); ShardStats shardStats = new ShardStats( shard.routingEntry(), diff --git a/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java b/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java index e05f8dc6e4e57..896bbffb10d09 100644 --- a/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java +++ b/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java @@ -27,7 +27,7 @@ import org.opensearch.index.engine.InternalEngineFactory; import org.opensearch.index.engine.NRTReplicationEngineFactory; import org.opensearch.index.remote.RemoteSegmentTransferTracker; -import org.opensearch.index.remote.RemoteStorePressureService; +import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory; import org.opensearch.index.store.RemoteDirectory; import org.opensearch.index.store.RemoteSegmentStoreDirectory; import org.opensearch.index.store.RemoteSegmentStoreDirectory.MetadataFilenameUtils; @@ -60,7 +60,7 @@ public class RemoteStoreRefreshListenerTests extends IndexShardTestCase { private IndexShard indexShard; private ClusterService clusterService; private RemoteStoreRefreshListener remoteStoreRefreshListener; - private RemoteStorePressureService remoteStorePressureService; + private RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory; public void setup(boolean primary, int numberOfDocs) throws IOException { indexShard = newStartedShard( @@ -84,9 +84,9 @@ public void setup(boolean primary, int numberOfDocs) throws IOException { new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), threadPool ); - remoteStorePressureService = new RemoteStorePressureService(clusterService, Settings.EMPTY); - remoteStorePressureService.afterIndexShardCreated(indexShard); - RemoteSegmentTransferTracker tracker = remoteStorePressureService.getRemoteRefreshSegmentTracker(indexShard.shardId()); + remoteStoreStatsTrackerFactory = new RemoteStoreStatsTrackerFactory(clusterService, Settings.EMPTY); + remoteStoreStatsTrackerFactory.afterIndexShardCreated(indexShard); + RemoteSegmentTransferTracker tracker = remoteStoreStatsTrackerFactory.getRemoteSegmentTransferTracker(indexShard.shardId()); remoteStoreRefreshListener = new RemoteStoreRefreshListener(indexShard, SegmentReplicationCheckpointPublisher.EMPTY, tracker); } @@ -317,15 +317,15 @@ public void testRefreshSuccessOnFirstAttempt() throws Exception { // We spy on IndexShard.getEngine() to validate that we have successfully hit the terminal code for ascertaining successful upload. // Value has been set as 3 as during a successful upload IndexShard.getEngine() is hit thrice and with mockito we are counting down CountDownLatch successLatch = new CountDownLatch(3); - Tuple tuple = mockIndexShardWithRetryAndScheduleRefresh( + Tuple tuple = mockIndexShardWithRetryAndScheduleRefresh( succeedOnAttempt, refreshCountLatch, successLatch ); assertBusy(() -> assertEquals(0, refreshCountLatch.getCount())); assertBusy(() -> assertEquals(0, successLatch.getCount())); - RemoteStorePressureService pressureService = tuple.v2(); - RemoteSegmentTransferTracker segmentTracker = pressureService.getRemoteRefreshSegmentTracker(indexShard.shardId()); + RemoteStoreStatsTrackerFactory trackerFactory = tuple.v2(); + RemoteSegmentTransferTracker segmentTracker = trackerFactory.getRemoteSegmentTransferTracker(indexShard.shardId()); assertNoLagAndTotalUploadsFailed(segmentTracker, 0); } @@ -338,15 +338,15 @@ public void testRefreshSuccessOnSecondAttempt() throws Exception { // We spy on IndexShard.getEngine() to validate that we have successfully hit the terminal code for ascertaining successful upload. // Value has been set as 3 as during a successful upload IndexShard.getEngine() is hit thrice and with mockito we are counting down CountDownLatch successLatch = new CountDownLatch(3); - Tuple tuple = mockIndexShardWithRetryAndScheduleRefresh( + Tuple tuple = mockIndexShardWithRetryAndScheduleRefresh( succeedOnAttempt, refreshCountLatch, successLatch ); assertBusy(() -> assertEquals(0, refreshCountLatch.getCount())); assertBusy(() -> assertEquals(0, successLatch.getCount())); - RemoteStorePressureService pressureService = tuple.v2(); - RemoteSegmentTransferTracker segmentTracker = pressureService.getRemoteRefreshSegmentTracker(indexShard.shardId()); + RemoteStoreStatsTrackerFactory trackerFactory = tuple.v2(); + RemoteSegmentTransferTracker segmentTracker = trackerFactory.getRemoteSegmentTransferTracker(indexShard.shardId()); assertNoLagAndTotalUploadsFailed(segmentTracker, 1); } @@ -384,15 +384,15 @@ public void testRefreshSuccessOnThirdAttempt() throws Exception { // We spy on IndexShard.getEngine() to validate that we have successfully hit the terminal code for ascertaining successful upload. // Value has been set as 3 as during a successful upload IndexShard.getEngine() is hit thrice and with mockito we are counting down CountDownLatch successLatch = new CountDownLatch(3); - Tuple tuple = mockIndexShardWithRetryAndScheduleRefresh( + Tuple tuple = mockIndexShardWithRetryAndScheduleRefresh( succeedOnAttempt, refreshCountLatch, successLatch ); assertBusy(() -> assertEquals(0, refreshCountLatch.getCount())); assertBusy(() -> assertEquals(0, successLatch.getCount())); - RemoteStorePressureService pressureService = tuple.v2(); - RemoteSegmentTransferTracker segmentTracker = pressureService.getRemoteRefreshSegmentTracker(indexShard.shardId()); + RemoteStoreStatsTrackerFactory trackerFactory = tuple.v2(); + RemoteSegmentTransferTracker segmentTracker = trackerFactory.getRemoteSegmentTransferTracker(indexShard.shardId()); assertNoLagAndTotalUploadsFailed(segmentTracker, 2); } @@ -406,10 +406,10 @@ private void assertNoLagAndTotalUploadsFailed(RemoteSegmentTransferTracker segme } public void testTrackerData() throws Exception { - Tuple tuple = mockIndexShardWithRetryAndScheduleRefresh(1); + Tuple tuple = mockIndexShardWithRetryAndScheduleRefresh(1); RemoteStoreRefreshListener listener = tuple.v1(); - RemoteStorePressureService pressureService = tuple.v2(); - RemoteSegmentTransferTracker tracker = pressureService.getRemoteRefreshSegmentTracker(indexShard.shardId()); + RemoteStoreStatsTrackerFactory trackerFactory = tuple.v2(); + RemoteSegmentTransferTracker tracker = trackerFactory.getRemoteSegmentTransferTracker(indexShard.shardId()); assertNoLag(tracker); indexDocs(100, randomIntBetween(100, 200)); indexShard.refresh("test"); @@ -431,12 +431,13 @@ private void assertNoLag(RemoteSegmentTransferTracker tracker) { assertEquals(0, tracker.getTotalUploadsFailed()); } - private Tuple mockIndexShardWithRetryAndScheduleRefresh(int succeedOnAttempt) - throws IOException { + private Tuple mockIndexShardWithRetryAndScheduleRefresh( + int succeedOnAttempt + ) throws IOException { return mockIndexShardWithRetryAndScheduleRefresh(succeedOnAttempt, null, null); } - private Tuple mockIndexShardWithRetryAndScheduleRefresh( + private Tuple mockIndexShardWithRetryAndScheduleRefresh( int succeedOnAttempt, CountDownLatch refreshCountLatch, CountDownLatch successLatch @@ -445,7 +446,7 @@ private Tuple mockIndexS return mockIndexShardWithRetryAndScheduleRefresh(succeedOnAttempt, refreshCountLatch, successLatch, 1, noOpLatch); } - private Tuple mockIndexShardWithRetryAndScheduleRefresh( + private Tuple mockIndexShardWithRetryAndScheduleRefresh( int succeedOnAttempt, CountDownLatch refreshCountLatch, CountDownLatch successLatch, @@ -539,13 +540,13 @@ private Tuple mockIndexS new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), threadPool ); - RemoteStorePressureService remoteStorePressureService = indexShard.getRemoteStorePressureService(); + RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory = indexShard.getRemoteStoreStatsTrackerFactory(); when(shard.indexSettings()).thenReturn(indexShard.indexSettings()); when(shard.shardId()).thenReturn(indexShard.shardId()); - RemoteSegmentTransferTracker tracker = remoteStorePressureService.getRemoteRefreshSegmentTracker(indexShard.shardId()); + RemoteSegmentTransferTracker tracker = remoteStoreStatsTrackerFactory.getRemoteSegmentTransferTracker(indexShard.shardId()); RemoteStoreRefreshListener refreshListener = new RemoteStoreRefreshListener(shard, emptyCheckpointPublisher, tracker); refreshListener.afterRefresh(true); - return Tuple.tuple(refreshListener, remoteStorePressureService); + return Tuple.tuple(refreshListener, remoteStoreStatsTrackerFactory); } public static class TestFilterDirectory extends FilterDirectory { diff --git a/server/src/test/java/org/opensearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java b/server/src/test/java/org/opensearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java index 405903c005a84..c455101ff4549 100644 --- a/server/src/test/java/org/opensearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java +++ b/server/src/test/java/org/opensearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java @@ -46,7 +46,7 @@ import org.opensearch.core.index.shard.ShardId; import org.opensearch.index.IndexService; import org.opensearch.index.IndexSettings; -import org.opensearch.index.remote.RemoteStorePressureService; +import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory; import org.opensearch.index.seqno.RetentionLeaseSyncer; import org.opensearch.index.shard.IndexEventListener; import org.opensearch.index.shard.IndexShard; @@ -264,7 +264,7 @@ public MockIndexShard createShard( final RetentionLeaseSyncer retentionLeaseSyncer, final DiscoveryNode targetNode, final DiscoveryNode sourceNode, - final RemoteStorePressureService remoteStorePressureService + final RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory ) throws IOException { failRandomly(); RecoveryState recoveryState = new RecoveryState(shardRouting, targetNode, sourceNode); diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java index 4f7697660096e..a1cedbb6e24c0 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java @@ -178,6 +178,7 @@ import org.opensearch.index.SegmentReplicationPressureService; import org.opensearch.index.analysis.AnalysisRegistry; import org.opensearch.index.remote.RemoteStorePressureService; +import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory; import org.opensearch.index.seqno.GlobalCheckpointSyncAction; import org.opensearch.index.seqno.RetentionLeaseSyncer; import org.opensearch.index.shard.PrimaryReplicaSyncer; @@ -2126,7 +2127,7 @@ public void onFailure(final Exception e) { ), RetentionLeaseSyncer.EMPTY, SegmentReplicationCheckpointPublisher.EMPTY, - mock(RemoteStorePressureService.class) + mock(RemoteStoreStatsTrackerFactory.class) ); final SystemIndices systemIndices = new SystemIndices(emptyMap()); diff --git a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java index 8348584379f9c..b14a42e1e78ae 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java @@ -97,7 +97,7 @@ import org.opensearch.index.engine.NRTReplicationEngineFactory; import org.opensearch.index.mapper.MapperService; import org.opensearch.index.mapper.SourceToParse; -import org.opensearch.index.remote.RemoteStorePressureService; +import org.opensearch.index.remote.RemoteStoreStatsTrackerFactory; import org.opensearch.index.replication.TestReplicationSource; import org.opensearch.index.seqno.ReplicationTracker; import org.opensearch.index.seqno.RetentionLeaseSyncer; @@ -640,7 +640,7 @@ protected IndexShard newShard( clusterSettings ); Store remoteStore = null; - RemoteStorePressureService remoteStorePressureService = null; + RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory = null; RepositoriesService mockRepoSvc = mock(RepositoriesService.class); if (indexSettings.isRemoteStoreEnabled()) { @@ -655,7 +655,7 @@ protected IndexShard newShard( remoteStore = createRemoteStore(remotePath, routing, indexMetadata); - remoteStorePressureService = new RemoteStorePressureService(clusterService, indexSettings.getSettings()); + remoteStoreStatsTrackerFactory = new RemoteStoreStatsTrackerFactory(clusterService, indexSettings.getSettings()); BlobStoreRepository repo = createRepository(remotePath); when(mockRepoSvc.repository(any())).thenAnswer(invocationOnMock -> repo); } @@ -695,11 +695,11 @@ protected IndexShard newShard( translogFactorySupplier, checkpointPublisher, remoteStore, - remoteStorePressureService + remoteStoreStatsTrackerFactory ); indexShard.addShardFailureCallback(DEFAULT_SHARD_FAILURE_HANDLER); - if (remoteStorePressureService != null) { - remoteStorePressureService.afterIndexShardCreated(indexShard); + if (remoteStoreStatsTrackerFactory != null) { + remoteStoreStatsTrackerFactory.afterIndexShardCreated(indexShard); } success = true; } finally { From d66df10b248457d3d9778131d6939dd1a2185e39 Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Thu, 31 Aug 2023 11:02:06 +0530 Subject: [PATCH 20/37] Decouple replication lag from logic to fail stale replicas (#9507) * Decouple replication lag from replication timer logic used to fail stale replicas Signed-off-by: Ankit Kala * Added changelog entry Signed-off-by: Ankit Kala * Addressed comments Signed-off-by: Ankit Kala * Addressed comments 2 Signed-off-by: Ankit Kala * Addressed comments Signed-off-by: Ankit Kala * Retry gradle Signed-off-by: Ankit Kala * fix UT Signed-off-by: Ankit Kala * Addressed comments Signed-off-by: Ankit Kala * Retry Gradle Signed-off-by: Ankit Kala --------- Signed-off-by: Ankit Kala --- CHANGELOG.md | 1 + .../index/SegmentReplicationPressureIT.java | 14 ++++-- .../common/settings/ClusterSettings.java | 3 +- .../SegmentReplicationPressureService.java | 37 ++++++++++---- .../index/SegmentReplicationShardStats.java | 24 ++++++++++ .../index/seqno/ReplicationTracker.java | 48 +++++++++++++------ .../opensearch/index/shard/IndexShard.java | 7 +++ ...SegmentReplicationCheckpointPublisher.java | 1 + .../common/SegmentReplicationLagTimer.java | 48 +++++++++++++++++++ .../cat/RestCatSegmentReplicationAction.java | 2 +- ...egmentReplicationPressureServiceTests.java | 41 ++++++++++++++-- .../index/seqno/ReplicationTrackerTests.java | 17 +++++++ .../RestCatSegmentReplicationActionTests.java | 3 +- 13 files changed, 213 insertions(+), 33 deletions(-) create mode 100644 server/src/main/java/org/opensearch/indices/replication/common/SegmentReplicationLagTimer.java diff --git a/CHANGELOG.md b/CHANGELOG.md index e886ee5ffc97c..4ff22076f5530 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -92,6 +92,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add support for extensions to search responses using SearchExtBuilder ([#9379](https://github.com/opensearch-project/OpenSearch/pull/9379)) - [BWC and API enforcement] Decorate the existing APIs with proper annotations (part 1) ([#9520](https://github.com/opensearch-project/OpenSearch/pull/9520)) - Add concurrent segment search related metrics to node and index stats ([#9622](https://github.com/opensearch-project/OpenSearch/issues/9622)) +- Decouple replication lag from logic to fail stale replicas ([#9507](https://github.com/opensearch-project/OpenSearch/pull/9507)) ### Dependencies - Bump `org.apache.logging.log4j:log4j-core` from 2.17.1 to 2.20.0 ([#8307](https://github.com/opensearch-project/OpenSearch/pull/8307)) diff --git a/server/src/internalClusterTest/java/org/opensearch/index/SegmentReplicationPressureIT.java b/server/src/internalClusterTest/java/org/opensearch/index/SegmentReplicationPressureIT.java index 883e539b74b68..2fbad9099c056 100644 --- a/server/src/internalClusterTest/java/org/opensearch/index/SegmentReplicationPressureIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/index/SegmentReplicationPressureIT.java @@ -39,7 +39,8 @@ import static java.util.Arrays.asList; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; import static org.opensearch.index.SegmentReplicationPressureService.MAX_INDEXING_CHECKPOINTS; -import static org.opensearch.index.SegmentReplicationPressureService.MAX_REPLICATION_TIME_SETTING; +import static org.opensearch.index.SegmentReplicationPressureService.MAX_REPLICATION_LIMIT_STALE_REPLICA_SETTING; +import static org.opensearch.index.SegmentReplicationPressureService.MAX_REPLICATION_TIME_BACKPRESSURE_SETTING; import static org.opensearch.index.SegmentReplicationPressureService.SEGMENT_REPLICATION_INDEXING_PRESSURE_ENABLED; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount; @@ -54,7 +55,7 @@ protected Settings nodeSettings(int nodeOrdinal) { return Settings.builder() .put(super.nodeSettings(nodeOrdinal)) .put(SEGMENT_REPLICATION_INDEXING_PRESSURE_ENABLED.getKey(), true) - .put(MAX_REPLICATION_TIME_SETTING.getKey(), TimeValue.timeValueSeconds(1)) + .put(MAX_REPLICATION_TIME_BACKPRESSURE_SETTING.getKey(), TimeValue.timeValueSeconds(1)) .put(MAX_INDEXING_CHECKPOINTS.getKey(), MAX_CHECKPOINTS_BEHIND) .build(); } @@ -225,7 +226,10 @@ public void testBelowReplicaLimit() throws Exception { public void testFailStaleReplica() throws Exception { - Settings settings = Settings.builder().put(MAX_REPLICATION_TIME_SETTING.getKey(), TimeValue.timeValueMillis(500)).build(); + Settings settings = Settings.builder() + .put(MAX_REPLICATION_TIME_BACKPRESSURE_SETTING.getKey(), TimeValue.timeValueMillis(500)) + .put(MAX_REPLICATION_LIMIT_STALE_REPLICA_SETTING.getKey(), TimeValue.timeValueMillis(1000)) + .build(); // Starts a primary and replica node. final String primaryNode = internalCluster().startNode(settings); createIndex(INDEX_NAME); @@ -264,7 +268,9 @@ public void testWithDocumentReplicationEnabledIndex() throws Exception { "Can't create DocRep index with remote store enabled. Skipping.", Objects.equals(featureFlagSettings().get(FeatureFlags.REMOTE_STORE, "false"), "false") ); - Settings settings = Settings.builder().put(MAX_REPLICATION_TIME_SETTING.getKey(), TimeValue.timeValueMillis(500)).build(); + Settings settings = Settings.builder() + .put(MAX_REPLICATION_TIME_BACKPRESSURE_SETTING.getKey(), TimeValue.timeValueMillis(500)) + .build(); // Starts a primary and replica node. final String primaryNode = internalCluster().startNode(settings); createIndex( diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index de3f13f2e1a1d..6cb4992932b8e 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -645,7 +645,8 @@ public void apply(Settings value, Settings current, Settings previous) { SearchBackpressureSettings.SETTING_CANCELLATION_BURST, // deprecated SegmentReplicationPressureService.SEGMENT_REPLICATION_INDEXING_PRESSURE_ENABLED, SegmentReplicationPressureService.MAX_INDEXING_CHECKPOINTS, - SegmentReplicationPressureService.MAX_REPLICATION_TIME_SETTING, + SegmentReplicationPressureService.MAX_REPLICATION_TIME_BACKPRESSURE_SETTING, + SegmentReplicationPressureService.MAX_REPLICATION_LIMIT_STALE_REPLICA_SETTING, SegmentReplicationPressureService.MAX_ALLOWED_STALE_SHARDS, // Settings related to Searchable Snapshots diff --git a/server/src/main/java/org/opensearch/index/SegmentReplicationPressureService.java b/server/src/main/java/org/opensearch/index/SegmentReplicationPressureService.java index 44597e10f2fc2..0c4fe26eeead5 100644 --- a/server/src/main/java/org/opensearch/index/SegmentReplicationPressureService.java +++ b/server/src/main/java/org/opensearch/index/SegmentReplicationPressureService.java @@ -42,7 +42,8 @@ public class SegmentReplicationPressureService implements Closeable { private volatile boolean isSegmentReplicationBackpressureEnabled; private volatile int maxCheckpointsBehind; private volatile double maxAllowedStaleReplicas; - private volatile TimeValue maxReplicationTime; + private volatile TimeValue replicationTimeLimitBackpressure; + private volatile TimeValue replicationTimeLimitFailReplica; private static final Logger logger = LogManager.getLogger(SegmentReplicationPressureService.class); @@ -65,13 +66,23 @@ public class SegmentReplicationPressureService implements Closeable { Setting.Property.NodeScope ); - public static final Setting MAX_REPLICATION_TIME_SETTING = Setting.positiveTimeSetting( + // Time limit on max allowed replica staleness after which backpressure kicks in on primary. + public static final Setting MAX_REPLICATION_TIME_BACKPRESSURE_SETTING = Setting.positiveTimeSetting( "segrep.pressure.time.limit", TimeValue.timeValueMinutes(5), Setting.Property.Dynamic, Setting.Property.NodeScope ); + // Time limit on max allowed replica staleness after which we start failing the replica shard. + // Defaults to 0(disabled) + public static final Setting MAX_REPLICATION_LIMIT_STALE_REPLICA_SETTING = Setting.positiveTimeSetting( + "segrep.replication.time.limit", + TimeValue.timeValueMinutes(0), + Setting.Property.Dynamic, + Setting.Property.NodeScope + ); + public static final Setting MAX_ALLOWED_STALE_SHARDS = Setting.doubleSetting( "segrep.pressure.replica.stale.limit", .5, @@ -112,8 +123,11 @@ public SegmentReplicationPressureService( this.maxCheckpointsBehind = MAX_INDEXING_CHECKPOINTS.get(settings); clusterSettings.addSettingsUpdateConsumer(MAX_INDEXING_CHECKPOINTS, this::setMaxCheckpointsBehind); - this.maxReplicationTime = MAX_REPLICATION_TIME_SETTING.get(settings); - clusterSettings.addSettingsUpdateConsumer(MAX_REPLICATION_TIME_SETTING, this::setMaxReplicationTime); + this.replicationTimeLimitBackpressure = MAX_REPLICATION_TIME_BACKPRESSURE_SETTING.get(settings); + clusterSettings.addSettingsUpdateConsumer(MAX_REPLICATION_TIME_BACKPRESSURE_SETTING, this::setReplicationTimeLimitBackpressure); + + this.replicationTimeLimitFailReplica = MAX_REPLICATION_LIMIT_STALE_REPLICA_SETTING.get(settings); + clusterSettings.addSettingsUpdateConsumer(MAX_REPLICATION_LIMIT_STALE_REPLICA_SETTING, this::setReplicationTimeLimitFailReplica); this.maxAllowedStaleReplicas = MAX_ALLOWED_STALE_SHARDS.get(settings); clusterSettings.addSettingsUpdateConsumer(MAX_ALLOWED_STALE_SHARDS, this::setMaxAllowedStaleReplicas); @@ -157,7 +171,7 @@ private void validateReplicationGroup(IndexShard shard) { private Set getStaleReplicas(final Set replicas) { return replicas.stream() .filter(entry -> entry.getCheckpointsBehindCount() > maxCheckpointsBehind) - .filter(entry -> entry.getCurrentReplicationTimeMillis() > maxReplicationTime.millis()) + .filter(entry -> entry.getCurrentReplicationTimeMillis() > replicationTimeLimitBackpressure.millis()) .collect(Collectors.toSet()); } @@ -185,8 +199,12 @@ public void setMaxAllowedStaleReplicas(double maxAllowedStaleReplicas) { this.maxAllowedStaleReplicas = maxAllowedStaleReplicas; } - public void setMaxReplicationTime(TimeValue maxReplicationTime) { - this.maxReplicationTime = maxReplicationTime; + public void setReplicationTimeLimitFailReplica(TimeValue replicationTimeLimitFailReplica) { + this.replicationTimeLimitFailReplica = replicationTimeLimitFailReplica; + } + + public void setReplicationTimeLimitBackpressure(TimeValue replicationTimeLimitBackpressure) { + this.replicationTimeLimitBackpressure = replicationTimeLimitBackpressure; } @Override @@ -214,7 +232,8 @@ protected boolean mustReschedule() { @Override protected void runInternal() { - if (pressureService.isSegmentReplicationBackpressureEnabled) { + // Do not fail the replicas if time limit is set to 0 (i.e. disabled). + if (TimeValue.ZERO.equals(pressureService.replicationTimeLimitFailReplica) == false) { final SegmentReplicationStats stats = pressureService.tracker.getStats(); // Find the shardId in node which is having stale replicas with highest current replication time. @@ -240,7 +259,7 @@ protected void runInternal() { } final IndexShard primaryShard = indexService.getShard(shardId.getId()); for (SegmentReplicationShardStats staleReplica : staleReplicas) { - if (staleReplica.getCurrentReplicationTimeMillis() > 2 * pressureService.maxReplicationTime.millis()) { + if (staleReplica.getCurrentReplicationTimeMillis() > pressureService.replicationTimeLimitFailReplica.millis()) { pressureService.shardStateAction.remoteShardFailed( shardId, staleReplica.getAllocationId(), diff --git a/server/src/main/java/org/opensearch/index/SegmentReplicationShardStats.java b/server/src/main/java/org/opensearch/index/SegmentReplicationShardStats.java index 66bc2934f5391..2be0c712f64ef 100644 --- a/server/src/main/java/org/opensearch/index/SegmentReplicationShardStats.java +++ b/server/src/main/java/org/opensearch/index/SegmentReplicationShardStats.java @@ -29,6 +29,10 @@ public class SegmentReplicationShardStats implements Writeable, ToXContentFragme private final String allocationId; private final long checkpointsBehindCount; private final long bytesBehindCount; + // Total Replication lag observed. + private final long currentReplicationLagMillis; + // Total time taken for replicas to catch up. Similar to replication lag except this + // doesn't include time taken by primary to upload data to remote store. private final long currentReplicationTimeMillis; private final long lastCompletedReplicationTimeMillis; @@ -40,12 +44,14 @@ public SegmentReplicationShardStats( long checkpointsBehindCount, long bytesBehindCount, long currentReplicationTimeMillis, + long currentReplicationLagMillis, long lastCompletedReplicationTime ) { this.allocationId = allocationId; this.checkpointsBehindCount = checkpointsBehindCount; this.bytesBehindCount = bytesBehindCount; this.currentReplicationTimeMillis = currentReplicationTimeMillis; + this.currentReplicationLagMillis = currentReplicationLagMillis; this.lastCompletedReplicationTimeMillis = lastCompletedReplicationTime; } @@ -55,6 +61,7 @@ public SegmentReplicationShardStats(StreamInput in) throws IOException { this.bytesBehindCount = in.readVLong(); this.currentReplicationTimeMillis = in.readVLong(); this.lastCompletedReplicationTimeMillis = in.readVLong(); + this.currentReplicationLagMillis = in.readVLong(); } public String getAllocationId() { @@ -73,6 +80,19 @@ public long getCurrentReplicationTimeMillis() { return currentReplicationTimeMillis; } + /** + * Total Replication lag observed. + * @return currentReplicationLagMillis + */ + public long getCurrentReplicationLagMillis() { + return currentReplicationLagMillis; + } + + /** + * Total time taken for replicas to catch up. Similar to replication lag except this doesn't include time taken by + * primary to upload data to remote store. + * @return lastCompletedReplicationTimeMillis + */ public long getLastCompletedReplicationTimeMillis() { return lastCompletedReplicationTimeMillis; } @@ -93,6 +113,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field("checkpoints_behind", checkpointsBehindCount); builder.field("bytes_behind", new ByteSizeValue(bytesBehindCount).toString()); builder.field("current_replication_time", new TimeValue(currentReplicationTimeMillis)); + builder.field("current_replication_lag", new TimeValue(currentReplicationLagMillis)); builder.field("last_completed_replication_time", new TimeValue(lastCompletedReplicationTimeMillis)); if (currentReplicationState != null) { builder.startObject(); @@ -110,6 +131,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVLong(bytesBehindCount); out.writeVLong(currentReplicationTimeMillis); out.writeVLong(lastCompletedReplicationTimeMillis); + out.writeVLong(currentReplicationLagMillis); } @Override @@ -121,6 +143,8 @@ public String toString() { + checkpointsBehindCount + ", bytesBehindCount=" + bytesBehindCount + + ", currentReplicationLagMillis=" + + currentReplicationLagMillis + ", currentReplicationTimeMillis=" + currentReplicationTimeMillis + ", lastCompletedReplicationTimeMillis=" diff --git a/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java b/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java index 37112ee549884..48bfce1013f17 100644 --- a/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java +++ b/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java @@ -58,7 +58,7 @@ import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.ReplicationGroup; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; -import org.opensearch.indices.replication.common.ReplicationTimer; +import org.opensearch.indices.replication.common.SegmentReplicationLagTimer; import java.io.IOException; import java.nio.file.Path; @@ -714,7 +714,7 @@ public static class CheckpointState implements Writeable { * Map of ReplicationCheckpoints to ReplicationTimers. Timers are added as new checkpoints are published, and removed when * the replica is caught up. */ - Map checkpointTimers; + Map checkpointTimers; /** * The time it took to complete the most recent replication event. @@ -1186,9 +1186,9 @@ public synchronized void updateVisibleCheckpointForShard(final String allocation cps.checkpointTimers.entrySet().removeIf((entry) -> { boolean result = entry.getKey().isAheadOf(visibleCheckpoint) == false; if (result) { - final ReplicationTimer timer = entry.getValue(); + final SegmentReplicationLagTimer timer = entry.getValue(); timer.stop(); - lastFinished.set(Math.max(lastFinished.get(), timer.time())); + lastFinished.set(Math.max(lastFinished.get(), timer.totalElapsedTime())); } return result; }); @@ -1208,7 +1208,7 @@ public synchronized void updateVisibleCheckpointForShard(final String allocation } /** - * After a new checkpoint is published, start a timer for each replica to the checkpoint. + * After a new checkpoint is published, create a timer for each replica to the checkpoint. * @param checkpoint {@link ReplicationCheckpoint} */ public synchronized void setLatestReplicationCheckpoint(ReplicationCheckpoint checkpoint) { @@ -1217,7 +1217,7 @@ public synchronized void setLatestReplicationCheckpoint(ReplicationCheckpoint ch this.latestReplicationCheckpoint = checkpoint; } if (primaryMode) { - startReplicationLagTimers(); + createReplicationLagTimers(); } } @@ -1225,7 +1225,7 @@ public ReplicationCheckpoint getLatestReplicationCheckpoint() { return this.latestReplicationCheckpoint; } - private void startReplicationLagTimers() { + private void createReplicationLagTimers() { for (Map.Entry entry : checkpoints.entrySet()) { final String allocationId = entry.getKey(); if (allocationId.equals(this.shardAllocationId) == false) { @@ -1235,11 +1235,7 @@ private void startReplicationLagTimers() { if (cps.inSync && replicationGroup.getUnavailableInSyncShards().contains(allocationId) == false && latestReplicationCheckpoint.isAheadOf(cps.visibleReplicationCheckpoint)) { - cps.checkpointTimers.computeIfAbsent(latestReplicationCheckpoint, ignored -> { - final ReplicationTimer replicationTimer = new ReplicationTimer(); - replicationTimer.start(); - return replicationTimer; - }); + cps.checkpointTimers.computeIfAbsent(latestReplicationCheckpoint, ignored -> new SegmentReplicationLagTimer()); logger.trace( () -> new ParameterizedMessage( "updated last published checkpoint for {} at visible cp {} to {} - timers [{}]", @@ -1254,6 +1250,29 @@ private void startReplicationLagTimers() { } } + /** + * After a new checkpoint is published, start a timer per replica for the checkpoint. + * @param checkpoint {@link ReplicationCheckpoint} + */ + public synchronized void startReplicationLagTimers(ReplicationCheckpoint checkpoint) { + assert indexSettings.isSegRepEnabled(); + if (checkpoint.equals(latestReplicationCheckpoint) == false) { + this.latestReplicationCheckpoint = checkpoint; + } + if (primaryMode) { + checkpoints.entrySet().stream().filter(e -> !e.getKey().equals(this.shardAllocationId)).forEach(e -> { + String allocationId = e.getKey(); + final CheckpointState cps = e.getValue(); + if (cps.inSync + && replicationGroup.getUnavailableInSyncShards().contains(allocationId) == false + && latestReplicationCheckpoint.isAheadOf(cps.visibleReplicationCheckpoint) + && cps.checkpointTimers.containsKey(latestReplicationCheckpoint)) { + cps.checkpointTimers.get(latestReplicationCheckpoint).start(); + } + }); + } + } + /** * Fetch stats on segment replication. * @return {@link Tuple} V1 - TimeValue in ms - mean replication lag for this primary to its entire group, @@ -1282,14 +1301,15 @@ private SegmentReplicationShardStats buildShardStats( final String allocationId, final CheckpointState checkpointState ) { - final Map checkpointTimers = checkpointState.checkpointTimers; + final Map checkpointTimers = checkpointState.checkpointTimers; return new SegmentReplicationShardStats( allocationId, checkpointTimers.size(), checkpointState.visibleReplicationCheckpoint == null ? latestCheckpointLength : Math.max(latestCheckpointLength - checkpointState.visibleReplicationCheckpoint.getLength(), 0), - checkpointTimers.values().stream().mapToLong(ReplicationTimer::time).max().orElse(0), + checkpointTimers.values().stream().mapToLong(SegmentReplicationLagTimer::time).max().orElse(0), + checkpointTimers.values().stream().mapToLong(SegmentReplicationLagTimer::totalElapsedTime).max().orElse(0), checkpointState.lastCompletedReplicationLag ); } diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index ba584b06cff71..0d51126ace8c7 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -1858,6 +1858,10 @@ static Engine.Searcher wrapSearcher( } } + public void onCheckpointPublished(ReplicationCheckpoint checkpoint) { + replicationTracker.startReplicationLagTimers(checkpoint); + } + /** * Used with segment replication during relocation handoff, this method updates current read only engine to global * checkpoint followed by changing to writeable engine @@ -4500,6 +4504,9 @@ public void beforeRefresh() throws IOException {} @Override public void afterRefresh(boolean didRefresh) throws IOException { if (didRefresh) { + // We're only starting to track the replication checkpoint. The timers for replication are started when + // the checkpoint is published. This is done so that the timers do not include the time spent by primary + // in uploading the segments to remote store. updateReplicationCheckpoint(); } } diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/SegmentReplicationCheckpointPublisher.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/SegmentReplicationCheckpointPublisher.java index f5cb32b741862..b4bcdc92e539a 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/SegmentReplicationCheckpointPublisher.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/SegmentReplicationCheckpointPublisher.java @@ -34,6 +34,7 @@ public SegmentReplicationCheckpointPublisher(PublishAction publishAction) { public void publish(IndexShard indexShard, ReplicationCheckpoint checkpoint) { publishAction.publish(indexShard, checkpoint); + indexShard.onCheckpointPublished(checkpoint); } /** diff --git a/server/src/main/java/org/opensearch/indices/replication/common/SegmentReplicationLagTimer.java b/server/src/main/java/org/opensearch/indices/replication/common/SegmentReplicationLagTimer.java new file mode 100644 index 0000000000000..c97edba72da0d --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/common/SegmentReplicationLagTimer.java @@ -0,0 +1,48 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.indices.replication.common; + +import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; + +import java.io.IOException; + +/** + * Wrapper class for Replication Timer which also tracks time elapsed since the timer was created. + * Currently, this is being used to calculate + * 1. Replication Lag: Total time taken by replica to sync after primary refreshed. + * 2. Replication event time: Total time taken by replica to sync after primary published the checkpoint + * (excludes the time spent by primary for uploading the segments to remote store). + * + * @opensearch.internal + */ +public class SegmentReplicationLagTimer extends ReplicationTimer { + private long creationTime; + + public SegmentReplicationLagTimer() { + super(); + creationTime = System.nanoTime(); + } + + public SegmentReplicationLagTimer(StreamInput in) throws IOException { + super(in); + creationTime = in.readVLong(); + } + + @Override + public synchronized void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeVLong(creationTime); + } + + public long totalElapsedTime() { + return TimeValue.nsecToMSec(Math.max(System.nanoTime() - creationTime, 0)); + } +} diff --git a/server/src/main/java/org/opensearch/rest/action/cat/RestCatSegmentReplicationAction.java b/server/src/main/java/org/opensearch/rest/action/cat/RestCatSegmentReplicationAction.java index 1716289ca4c20..aa325443ba6c9 100644 --- a/server/src/main/java/org/opensearch/rest/action/cat/RestCatSegmentReplicationAction.java +++ b/server/src/main/java/org/opensearch/rest/action/cat/RestCatSegmentReplicationAction.java @@ -171,7 +171,7 @@ public Table buildSegmentReplicationTable(RestRequest request, SegmentReplicatio t.addCell(state.getTargetNode().getHostName()); t.addCell(shardStats.getCheckpointsBehindCount()); t.addCell(new ByteSizeValue(shardStats.getBytesBehindCount())); - t.addCell(new TimeValue(shardStats.getCurrentReplicationTimeMillis())); + t.addCell(new TimeValue(shardStats.getCurrentReplicationLagMillis())); t.addCell(new TimeValue(shardStats.getLastCompletedReplicationTimeMillis())); t.addCell(perGroupStats.getRejectedRequestCount()); if (detailed) { diff --git a/server/src/test/java/org/opensearch/index/SegmentReplicationPressureServiceTests.java b/server/src/test/java/org/opensearch/index/SegmentReplicationPressureServiceTests.java index 531c636c9d165..3a08a2c143a89 100644 --- a/server/src/test/java/org/opensearch/index/SegmentReplicationPressureServiceTests.java +++ b/server/src/test/java/org/opensearch/index/SegmentReplicationPressureServiceTests.java @@ -33,13 +33,15 @@ import org.mockito.stubbing.Answer; import static java.util.Arrays.asList; -import static org.opensearch.index.SegmentReplicationPressureService.MAX_REPLICATION_TIME_SETTING; +import static org.opensearch.index.SegmentReplicationPressureService.MAX_REPLICATION_LIMIT_STALE_REPLICA_SETTING; +import static org.opensearch.index.SegmentReplicationPressureService.MAX_REPLICATION_TIME_BACKPRESSURE_SETTING; import static org.opensearch.index.SegmentReplicationPressureService.SEGMENT_REPLICATION_INDEXING_PRESSURE_ENABLED; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -50,7 +52,7 @@ public class SegmentReplicationPressureServiceTests extends OpenSearchIndexLevel private static final Settings settings = Settings.builder() .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) .put(SEGMENT_REPLICATION_INDEXING_PRESSURE_ENABLED.getKey(), true) - .put(MAX_REPLICATION_TIME_SETTING.getKey(), TimeValue.timeValueSeconds(5)) + .put(MAX_REPLICATION_TIME_BACKPRESSURE_SETTING.getKey(), TimeValue.timeValueSeconds(5)) .build(); public void testIsSegrepLimitBreached() throws Exception { @@ -196,7 +198,8 @@ public void testFailStaleReplicaTask() throws Exception { final Settings settings = Settings.builder() .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) .put(SEGMENT_REPLICATION_INDEXING_PRESSURE_ENABLED.getKey(), true) - .put(MAX_REPLICATION_TIME_SETTING.getKey(), TimeValue.timeValueMillis(10)) + .put(MAX_REPLICATION_TIME_BACKPRESSURE_SETTING.getKey(), TimeValue.timeValueMillis(10)) + .put(MAX_REPLICATION_LIMIT_STALE_REPLICA_SETTING.getKey(), TimeValue.timeValueMillis(20)) .build(); try (ReplicationGroup shards = createGroup(1, settings, new NRTReplicationEngineFactory())) { @@ -222,6 +225,38 @@ public void testFailStaleReplicaTask() throws Exception { } } + public void testFailStaleReplicaTaskDisabled() throws Exception { + final Settings settings = Settings.builder() + .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) + .put(SEGMENT_REPLICATION_INDEXING_PRESSURE_ENABLED.getKey(), true) + .put(MAX_REPLICATION_TIME_BACKPRESSURE_SETTING.getKey(), TimeValue.timeValueMillis(10)) + .put(MAX_REPLICATION_LIMIT_STALE_REPLICA_SETTING.getKey(), TimeValue.timeValueMillis(0)) + .build(); + + try (ReplicationGroup shards = createGroup(1, settings, new NRTReplicationEngineFactory())) { + shards.startAll(); + final IndexShard primaryShard = shards.getPrimary(); + SegmentReplicationPressureService service = buildPressureService(settings, primaryShard); + Mockito.reset(shardStateAction); + + // index docs in batches without refreshing + indexInBatches(5, shards, primaryShard); + + // assert that replica shard is few checkpoints behind primary + Set replicationStats = primaryShard.getReplicationStats(); + assertEquals(1, replicationStats.size()); + SegmentReplicationShardStats shardStats = replicationStats.stream().findFirst().get(); + assertEquals(5, shardStats.getCheckpointsBehindCount()); + + // call the background task + service.getFailStaleReplicaTask().runInternal(); + + // verify that remote shard failed method is never called as it is disabled. + verify(shardStateAction, never()).remoteShardFailed(any(), anyString(), anyLong(), anyBoolean(), anyString(), any(), any()); + replicateSegments(primaryShard, shards.getReplicas()); + } + } + private int indexInBatches(int count, ReplicationGroup shards, IndexShard primaryShard) throws Exception { int totalDocs = 0; for (int i = 0; i < count; i++) { diff --git a/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTests.java b/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTests.java index efa901de75c38..ab87d31d15e2f 100644 --- a/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTests.java +++ b/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTests.java @@ -52,6 +52,7 @@ import org.opensearch.index.SegmentReplicationShardStats; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.indices.replication.common.SegmentReplicationLagTimer; import org.opensearch.test.IndexSettingsModule; import java.io.IOException; @@ -1851,8 +1852,11 @@ public void testSegmentReplicationCheckpointTracking() { ); tracker.setLatestReplicationCheckpoint(initialCheckpoint); + tracker.startReplicationLagTimers(initialCheckpoint); tracker.setLatestReplicationCheckpoint(secondCheckpoint); + tracker.startReplicationLagTimers(secondCheckpoint); tracker.setLatestReplicationCheckpoint(thirdCheckpoint); + tracker.startReplicationLagTimers(thirdCheckpoint); final Set expectedIds = ids(initializingIds); @@ -1861,6 +1865,7 @@ public void testSegmentReplicationCheckpointTracking() { for (SegmentReplicationShardStats shardStat : groupStats) { assertEquals(3, shardStat.getCheckpointsBehindCount()); assertEquals(100L, shardStat.getBytesBehindCount()); + assertTrue(shardStat.getCurrentReplicationLagMillis() >= shardStat.getCurrentReplicationTimeMillis()); } // simulate replicas moved up to date. @@ -1936,6 +1941,7 @@ public void testSegmentReplicationCheckpointTrackingInvalidAllocationIDs() { Codec.getDefault().getName() ); tracker.setLatestReplicationCheckpoint(initialCheckpoint); + tracker.startReplicationLagTimers(initialCheckpoint); // we expect that the only returned ids from getSegmentReplicationStats will be the initializing ids we marked with // markAsTrackingAndInSyncQuietly. @@ -2161,4 +2167,15 @@ public void testIllegalStateExceptionIfUnknownAllocationIdWithRemoteTranslogEnab expectThrows(IllegalStateException.class, () -> tracker.markAllocationIdAsInSync(randomAlphaOfLength(10), randomNonNegativeLong())); } + public void testSegRepTimer() throws Throwable { + SegmentReplicationLagTimer timer = new SegmentReplicationLagTimer(); + Thread.sleep(100); + timer.start(); + Thread.sleep(100); + timer.stop(); + assertTrue("Total time since timer started should be greater than 100", timer.time() >= 100); + assertTrue("Total time since timer was created should be greater than 200", timer.totalElapsedTime() >= 200); + assertTrue("Total elapsed time should be greater than time since timer start", timer.totalElapsedTime() - timer.time() >= 100); + } + } diff --git a/server/src/test/java/org/opensearch/rest/action/cat/RestCatSegmentReplicationActionTests.java b/server/src/test/java/org/opensearch/rest/action/cat/RestCatSegmentReplicationActionTests.java index 86cbe2fd991c7..41ad9e8bcbb44 100644 --- a/server/src/test/java/org/opensearch/rest/action/cat/RestCatSegmentReplicationActionTests.java +++ b/server/src/test/java/org/opensearch/rest/action/cat/RestCatSegmentReplicationActionTests.java @@ -84,6 +84,7 @@ public void testSegmentReplicationAction() throws IOException { 0L, 0L, 0L, + 0L, 0L ); segmentReplicationShardStats.setCurrentReplicationState(state); @@ -141,7 +142,7 @@ public void testSegmentReplicationAction() throws IOException { currentReplicationState.getTargetNode().getHostName(), shardStats.getCheckpointsBehindCount(), new ByteSizeValue(shardStats.getBytesBehindCount()), - new TimeValue(shardStats.getCurrentReplicationTimeMillis()), + new TimeValue(shardStats.getCurrentReplicationLagMillis()), new TimeValue(shardStats.getLastCompletedReplicationTimeMillis()), rejectedRequestCount ); From 79e5aeed7adc5a02d3d974378d1b7f29df3ded3e Mon Sep 17 00:00:00 2001 From: Sooraj Sinha <81695996+soosinha@users.noreply.github.com> Date: Thu, 31 Aug 2023 11:49:44 +0530 Subject: [PATCH 21/37] [Remote State] Create service to publish cluster state to remote store (#9160) * Upload all index metadata to remote store using BlobStoreRepository interface Signed-off-by: Sooraj Sinha --- CHANGELOG.md | 1 + .../common/settings/ClusterSettings.java | 7 +- .../opensearch/gateway/GatewayMetaState.java | 138 +++++- .../remote/ClusterMetadataManifest.java | 446 ++++++++++++++++++ .../remote/RemoteClusterStateService.java | 370 +++++++++++++++ .../gateway/remote/package-info.java | 12 + .../blobstore/BlobStoreRepository.java | 4 + .../GatewayMetaStatePersistedStateTests.java | 72 +++ .../remote/ClusterMetadataManifestTests.java | 149 ++++++ .../RemoteClusterStateServiceTests.java | 254 ++++++++++ 10 files changed, 1443 insertions(+), 10 deletions(-) create mode 100644 server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataManifest.java create mode 100644 server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java create mode 100644 server/src/main/java/org/opensearch/gateway/remote/package-info.java create mode 100644 server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataManifestTests.java create mode 100644 server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 4ff22076f5530..e65cc30aecd7e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -90,6 +90,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [Feature] Expose term frequency in Painless script score context ([#9081](https://github.com/opensearch-project/OpenSearch/pull/9081)) - Add support for reading partial files to HDFS repository ([#9513](https://github.com/opensearch-project/OpenSearch/issues/9513)) - Add support for extensions to search responses using SearchExtBuilder ([#9379](https://github.com/opensearch-project/OpenSearch/pull/9379)) +- [Remote State] Create service to publish cluster state to remote store ([#9160](https://github.com/opensearch-project/OpenSearch/pull/9160)) - [BWC and API enforcement] Decorate the existing APIs with proper annotations (part 1) ([#9520](https://github.com/opensearch-project/OpenSearch/pull/9520)) - Add concurrent segment search related metrics to node and index stats ([#9622](https://github.com/opensearch-project/OpenSearch/issues/9622)) - Decouple replication lag from logic to fail stale replicas ([#9507](https://github.com/opensearch-project/OpenSearch/pull/9507)) diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index 6cb4992932b8e..e00e7e3bf4ea7 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -97,6 +97,7 @@ import org.opensearch.gateway.DanglingIndicesState; import org.opensearch.gateway.GatewayService; import org.opensearch.gateway.PersistedClusterStateService; +import org.opensearch.gateway.remote.RemoteClusterStateService; import org.opensearch.http.HttpTransportSettings; import org.opensearch.index.IndexModule; import org.opensearch.index.IndexSettings; @@ -664,7 +665,11 @@ public void apply(Settings value, Settings current, Settings previous) { // Related to monitoring of task cancellation TaskCancellationMonitoringSettings.IS_ENABLED_SETTING, - TaskCancellationMonitoringSettings.DURATION_MILLIS_SETTING + TaskCancellationMonitoringSettings.DURATION_MILLIS_SETTING, + + // Remote cluster state settings + RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING, + RemoteClusterStateService.REMOTE_CLUSTER_STATE_REPOSITORY_SETTING ) ) ); diff --git a/server/src/main/java/org/opensearch/gateway/GatewayMetaState.java b/server/src/main/java/org/opensearch/gateway/GatewayMetaState.java index af894bdbc117e..02f1e5049b95c 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayMetaState.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayMetaState.java @@ -60,8 +60,11 @@ import org.opensearch.common.util.concurrent.OpenSearchThreadPoolExecutor; import org.opensearch.common.util.io.IOUtils; import org.opensearch.env.NodeMetadata; +import org.opensearch.gateway.remote.ClusterMetadataManifest; +import org.opensearch.gateway.remote.RemoteClusterStateService; import org.opensearch.node.Node; import org.opensearch.plugins.MetadataUpgrader; +import org.opensearch.repositories.RepositoryMissingException; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; @@ -84,19 +87,19 @@ /** * Loads (and maybe upgrades) cluster metadata at startup, and persistently stores cluster metadata for future restarts. * - * When started, ensures that this version is compatible with the state stored on disk, and performs a state upgrade if necessary. Note that - * the state being loaded when constructing the instance of this class is not necessarily the state that will be used as {@link - * ClusterState#metadata()} because it might be stale or incomplete. Cluster-manager-eligible nodes must perform an election to find a complete and - * non-stale state, and cluster-manager-ineligible nodes receive the real cluster state from the elected cluster-manager after joining the cluster. + * When started, ensures that this version is compatible with the state stored on disk, and performs a state upgrade if necessary. Note that the state being + * loaded when constructing the instance of this class is not necessarily the state that will be used as {@link ClusterState#metadata()} because it might be + * stale or incomplete. Cluster-manager-eligible nodes must perform an election to find a complete and non-stale state, and cluster-manager-ineligible nodes + * receive the real cluster state from the elected cluster-manager after joining the cluster. * * @opensearch.internal */ public class GatewayMetaState implements Closeable { /** - * Fake node ID for a voting configuration written by a cluster-manager-ineligible data node to indicate that its on-disk state is potentially - * stale (since it is written asynchronously after application, rather than before acceptance). This node ID means that if the node is - * restarted as a cluster-manager-eligible node then it does not win any elections until it has received a fresh cluster state. + * Fake node ID for a voting configuration written by a cluster-manager-ineligible data node to indicate that its on-disk state is potentially stale (since + * it is written asynchronously after application, rather than before acceptance). This node ID means that if the node is restarted as a + * cluster-manager-eligible node then it does not win any elections until it has received a fresh cluster state. */ public static final String STALE_STATE_CONFIG_NODE_ID = "STALE_STATE_CONFIG"; @@ -234,8 +237,8 @@ Metadata upgradeMetadataForNode( } /** - * This method calls {@link MetadataIndexUpgradeService} to makes sure that indices are compatible with the current - * version. The MetadataIndexUpgradeService might also update obsolete settings if needed. + * This method calls {@link MetadataIndexUpgradeService} to makes sure that indices are compatible with the current version. The MetadataIndexUpgradeService + * might also update obsolete settings if needed. * * @return input metadata if no upgrade is needed or an upgraded metadata */ @@ -599,4 +602,121 @@ public void close() throws IOException { IOUtils.close(persistenceWriter.getAndSet(null)); } } + + /** + * Encapsulates the writing of metadata to a remote store using {@link RemoteClusterStateService}. + */ + public static class RemotePersistedState implements PersistedState { + + private static final Logger logger = LogManager.getLogger(RemotePersistedState.class); + + private ClusterState lastAcceptedState; + private ClusterMetadataManifest lastAcceptedManifest; + private final RemoteClusterStateService remoteClusterStateService; + + public RemotePersistedState(final RemoteClusterStateService remoteClusterStateService) { + this.remoteClusterStateService = remoteClusterStateService; + } + + @Override + public long getCurrentTerm() { + return lastAcceptedState != null ? lastAcceptedState.term() : 0L; + } + + @Override + public ClusterState getLastAcceptedState() { + return lastAcceptedState; + } + + @Override + public void setCurrentTerm(long currentTerm) { + // no-op + // For LucenePersistedState, setCurrentTerm is used only while handling StartJoinRequest by all follower nodes. + // But for RemotePersistedState, the state is only pushed by the active cluster. So this method is not required. + } + + @Override + public void setLastAcceptedState(ClusterState clusterState) { + try { + if (lastAcceptedState == null || lastAcceptedState.blocks().hasGlobalBlock(GatewayService.STATE_NOT_RECOVERED_BLOCK)) { + // On the initial bootstrap, repository will not be available. So we do not persist the cluster state and bail out. + logger.info("Cluster is not yet ready to publish state to remote store"); + lastAcceptedState = clusterState; + return; + } + final ClusterMetadataManifest manifest; + if (shouldWriteFullClusterState(clusterState)) { + manifest = remoteClusterStateService.writeFullMetadata(clusterState); + } else { + assert verifyManifestAndClusterState(lastAcceptedManifest, lastAcceptedState) == true + : "Previous manifest and previous ClusterState are not in sync"; + manifest = remoteClusterStateService.writeIncrementalMetadata(lastAcceptedState, clusterState, lastAcceptedManifest); + } + assert verifyManifestAndClusterState(manifest, clusterState) == true : "Manifest and ClusterState are not in sync"; + lastAcceptedManifest = manifest; + lastAcceptedState = clusterState; + } catch (RepositoryMissingException e) { + // TODO This logic needs to be modified once PR for repo registration during bootstrap is pushed + // https://github.com/opensearch-project/OpenSearch/pull/9105/ + // After the above PR is pushed, we can remove this silent failure and throw the exception instead. + logger.error("Remote repository is not yet registered"); + lastAcceptedState = clusterState; + } catch (Exception e) { + handleExceptionOnWrite(e); + } + } + + private boolean verifyManifestAndClusterState(ClusterMetadataManifest manifest, ClusterState clusterState) { + assert manifest != null : "ClusterMetadataManifest is null"; + assert clusterState != null : "ClusterState is null"; + assert clusterState.metadata().indices().size() == manifest.getIndices().size() + : "Number of indices in last accepted state and manifest are different"; + manifest.getIndices().stream().forEach(md -> { + assert clusterState.metadata().indices().containsKey(md.getIndexName()) + : "Last accepted state does not contain the index : " + md.getIndexName(); + assert clusterState.metadata().indices().get(md.getIndexName()).getIndexUUID().equals(md.getIndexUUID()) + : "Last accepted state and manifest do not have same UUID for index : " + md.getIndexName(); + }); + return true; + } + + private boolean shouldWriteFullClusterState(ClusterState clusterState) { + if (lastAcceptedState == null + || lastAcceptedManifest == null + || lastAcceptedState.term() != clusterState.term() + || lastAcceptedManifest.getOpensearchVersion() != Version.CURRENT) { + return true; + } + return false; + } + + @Override + public void markLastAcceptedStateAsCommitted() { + try { + if (lastAcceptedState == null + || lastAcceptedManifest == null + || lastAcceptedState.blocks().hasGlobalBlock(GatewayService.STATE_NOT_RECOVERED_BLOCK)) { + // On the initial bootstrap, repository will not be available. So we do not persist the cluster state and bail out. + logger.trace("Cluster is not yet ready to publish state to remote store"); + return; + } + final ClusterMetadataManifest committedManifest = remoteClusterStateService.markLastStateAsCommitted( + lastAcceptedState, + lastAcceptedManifest + ); + lastAcceptedManifest = committedManifest; + } catch (Exception e) { + handleExceptionOnWrite(e); + } + } + + @Override + public void close() throws IOException { + remoteClusterStateService.close(); + } + + private void handleExceptionOnWrite(Exception e) { + throw ExceptionsHelper.convertToRuntime(e); + } + } } diff --git a/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataManifest.java b/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataManifest.java new file mode 100644 index 0000000000000..cac77f9996438 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataManifest.java @@ -0,0 +1,446 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway.remote; + +import org.opensearch.Version; +import org.opensearch.core.ParseField; +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.common.io.stream.Writeable; +import org.opensearch.core.xcontent.ConstructingObjectParser; +import org.opensearch.core.xcontent.MediaTypeRegistry; +import org.opensearch.core.xcontent.ToXContentFragment; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +/** + * Manifest file which contains the details of the uploaded entity metadata + * + * @opensearch.internal + */ +public class ClusterMetadataManifest implements Writeable, ToXContentFragment { + + private static final ParseField CLUSTER_TERM_FIELD = new ParseField("cluster_term"); + private static final ParseField STATE_VERSION_FIELD = new ParseField("state_version"); + private static final ParseField CLUSTER_UUID_FIELD = new ParseField("cluster_uuid"); + private static final ParseField STATE_UUID_FIELD = new ParseField("state_uuid"); + private static final ParseField OPENSEARCH_VERSION_FIELD = new ParseField("opensearch_version"); + private static final ParseField NODE_ID_FIELD = new ParseField("node_id"); + private static final ParseField COMMITTED_FIELD = new ParseField("committed"); + private static final ParseField INDICES_FIELD = new ParseField("indices"); + + private static long term(Object[] fields) { + return (long) fields[0]; + } + + private static long version(Object[] fields) { + return (long) fields[1]; + } + + private static String clusterUUID(Object[] fields) { + return (String) fields[2]; + } + + private static String stateUUID(Object[] fields) { + return (String) fields[3]; + } + + private static Version opensearchVersion(Object[] fields) { + return Version.fromId((int) fields[4]); + } + + private static String nodeId(Object[] fields) { + return (String) fields[5]; + } + + private static boolean committed(Object[] fields) { + return (boolean) fields[6]; + } + + private static List indices(Object[] fields) { + return (List) fields[7]; + } + + private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + "cluster_metadata_manifest", + fields -> new ClusterMetadataManifest( + term(fields), + version(fields), + clusterUUID(fields), + stateUUID(fields), + opensearchVersion(fields), + nodeId(fields), + committed(fields), + indices(fields) + ) + ); + + static { + PARSER.declareLong(ConstructingObjectParser.constructorArg(), CLUSTER_TERM_FIELD); + PARSER.declareLong(ConstructingObjectParser.constructorArg(), STATE_VERSION_FIELD); + PARSER.declareString(ConstructingObjectParser.constructorArg(), CLUSTER_UUID_FIELD); + PARSER.declareString(ConstructingObjectParser.constructorArg(), STATE_UUID_FIELD); + PARSER.declareInt(ConstructingObjectParser.constructorArg(), OPENSEARCH_VERSION_FIELD); + PARSER.declareString(ConstructingObjectParser.constructorArg(), NODE_ID_FIELD); + PARSER.declareBoolean(ConstructingObjectParser.constructorArg(), COMMITTED_FIELD); + PARSER.declareObjectArray( + ConstructingObjectParser.constructorArg(), + (p, c) -> UploadedIndexMetadata.fromXContent(p), + INDICES_FIELD + ); + } + + private final List indices; + private final long clusterTerm; + private final long stateVersion; + private final String clusterUUID; + private final String stateUUID; + private final Version opensearchVersion; + private final String nodeId; + private final boolean committed; + + public List getIndices() { + return indices; + } + + public long getClusterTerm() { + return clusterTerm; + } + + public long getStateVersion() { + return stateVersion; + } + + public String getClusterUUID() { + return clusterUUID; + } + + public String getStateUUID() { + return stateUUID; + } + + public Version getOpensearchVersion() { + return opensearchVersion; + } + + public String getNodeId() { + return nodeId; + } + + public boolean isCommitted() { + return committed; + } + + public ClusterMetadataManifest( + long clusterTerm, + long version, + String clusterUUID, + String stateUUID, + Version opensearchVersion, + String nodeId, + boolean committed, + List indices + ) { + this.clusterTerm = clusterTerm; + this.stateVersion = version; + this.clusterUUID = clusterUUID; + this.stateUUID = stateUUID; + this.opensearchVersion = opensearchVersion; + this.nodeId = nodeId; + this.committed = committed; + this.indices = Collections.unmodifiableList(indices); + } + + public ClusterMetadataManifest(StreamInput in) throws IOException { + this.clusterTerm = in.readVLong(); + this.stateVersion = in.readVLong(); + this.clusterUUID = in.readString(); + this.stateUUID = in.readString(); + this.opensearchVersion = Version.fromId(in.readInt()); + this.nodeId = in.readString(); + this.committed = in.readBoolean(); + this.indices = Collections.unmodifiableList(in.readList(UploadedIndexMetadata::new)); + } + + public static Builder builder() { + return new Builder(); + } + + public static Builder builder(ClusterMetadataManifest manifest) { + return new Builder(manifest); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.field(CLUSTER_TERM_FIELD.getPreferredName(), getClusterTerm()) + .field(STATE_VERSION_FIELD.getPreferredName(), getStateVersion()) + .field(CLUSTER_UUID_FIELD.getPreferredName(), getClusterUUID()) + .field(STATE_UUID_FIELD.getPreferredName(), getStateUUID()) + .field(OPENSEARCH_VERSION_FIELD.getPreferredName(), getOpensearchVersion().id) + .field(NODE_ID_FIELD.getPreferredName(), getNodeId()) + .field(COMMITTED_FIELD.getPreferredName(), isCommitted()); + builder.startArray(INDICES_FIELD.getPreferredName()); + { + for (UploadedIndexMetadata uploadedIndexMetadata : indices) { + uploadedIndexMetadata.toXContent(builder, params); + } + } + builder.endArray(); + return builder; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVLong(clusterTerm); + out.writeVLong(stateVersion); + out.writeString(clusterUUID); + out.writeString(stateUUID); + out.writeInt(opensearchVersion.id); + out.writeString(nodeId); + out.writeBoolean(committed); + out.writeCollection(indices); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final ClusterMetadataManifest that = (ClusterMetadataManifest) o; + return Objects.equals(indices, that.indices) + && clusterTerm == that.clusterTerm + && stateVersion == that.stateVersion + && Objects.equals(clusterUUID, that.clusterUUID) + && Objects.equals(stateUUID, that.stateUUID) + && Objects.equals(opensearchVersion, that.opensearchVersion) + && Objects.equals(nodeId, that.nodeId) + && Objects.equals(committed, that.committed); + } + + @Override + public int hashCode() { + return Objects.hash(indices, clusterTerm, stateVersion, clusterUUID, stateUUID, opensearchVersion, nodeId, committed); + } + + @Override + public String toString() { + return Strings.toString(MediaTypeRegistry.JSON, this); + } + + public static ClusterMetadataManifest fromXContent(XContentParser parser) throws IOException { + return PARSER.parse(parser, null); + } + + /** + * Builder for ClusterMetadataManifest + * + * @opensearch.internal + */ + public static class Builder { + + private List indices; + private long clusterTerm; + private long stateVersion; + private String clusterUUID; + private String stateUUID; + private Version opensearchVersion; + private String nodeId; + private boolean committed; + + public Builder indices(List indices) { + this.indices = indices; + return this; + } + + public Builder clusterTerm(long clusterTerm) { + this.clusterTerm = clusterTerm; + return this; + } + + public Builder stateVersion(long stateVersion) { + this.stateVersion = stateVersion; + return this; + } + + public Builder clusterUUID(String clusterUUID) { + this.clusterUUID = clusterUUID; + return this; + } + + public Builder stateUUID(String stateUUID) { + this.stateUUID = stateUUID; + return this; + } + + public Builder opensearchVersion(Version opensearchVersion) { + this.opensearchVersion = opensearchVersion; + return this; + } + + public Builder nodeId(String nodeId) { + this.nodeId = nodeId; + return this; + } + + public Builder committed(boolean committed) { + this.committed = committed; + return this; + } + + public List getIndices() { + return indices; + } + + public Builder() { + indices = new ArrayList<>(); + } + + public Builder(ClusterMetadataManifest manifest) { + this.clusterTerm = manifest.clusterTerm; + this.stateVersion = manifest.stateVersion; + this.clusterUUID = manifest.clusterUUID; + this.stateUUID = manifest.stateUUID; + this.opensearchVersion = manifest.opensearchVersion; + this.nodeId = manifest.nodeId; + this.committed = manifest.committed; + this.indices = new ArrayList<>(manifest.indices); + } + + public ClusterMetadataManifest build() { + return new ClusterMetadataManifest( + clusterTerm, + stateVersion, + clusterUUID, + stateUUID, + opensearchVersion, + nodeId, + committed, + indices + ); + } + + } + + /** + * Metadata for uploaded index metadata + * + * @opensearch.internal + */ + public static class UploadedIndexMetadata implements Writeable, ToXContentFragment { + + private static final ParseField INDEX_NAME_FIELD = new ParseField("index_name"); + private static final ParseField INDEX_UUID_FIELD = new ParseField("index_uuid"); + private static final ParseField UPLOADED_FILENAME_FIELD = new ParseField("uploaded_filename"); + + private static String indexName(Object[] fields) { + return (String) fields[0]; + } + + private static String indexUUID(Object[] fields) { + return (String) fields[1]; + } + + private static String uploadedFilename(Object[] fields) { + return (String) fields[2]; + } + + private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + "uploaded_index_metadata", + fields -> new UploadedIndexMetadata(indexName(fields), indexUUID(fields), uploadedFilename(fields)) + ); + + static { + PARSER.declareString(ConstructingObjectParser.constructorArg(), INDEX_NAME_FIELD); + PARSER.declareString(ConstructingObjectParser.constructorArg(), INDEX_UUID_FIELD); + PARSER.declareString(ConstructingObjectParser.constructorArg(), UPLOADED_FILENAME_FIELD); + } + + private final String indexName; + private final String indexUUID; + private final String uploadedFilename; + + public UploadedIndexMetadata(String indexName, String indexUUID, String uploadedFileName) { + this.indexName = indexName; + this.indexUUID = indexUUID; + this.uploadedFilename = uploadedFileName; + } + + public UploadedIndexMetadata(StreamInput in) throws IOException { + this.indexName = in.readString(); + this.indexUUID = in.readString(); + this.uploadedFilename = in.readString(); + } + + public String getUploadedFilename() { + return uploadedFilename; + } + + public String getIndexName() { + return indexName; + } + + public String getIndexUUID() { + return indexUUID; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return builder.startObject() + .field(INDEX_NAME_FIELD.getPreferredName(), getIndexName()) + .field(INDEX_UUID_FIELD.getPreferredName(), getIndexUUID()) + .field(UPLOADED_FILENAME_FIELD.getPreferredName(), getUploadedFilename()) + .endObject(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(indexName); + out.writeString(indexUUID); + out.writeString(uploadedFilename); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final UploadedIndexMetadata that = (UploadedIndexMetadata) o; + return Objects.equals(indexName, that.indexName) + && Objects.equals(indexUUID, that.indexUUID) + && Objects.equals(uploadedFilename, that.uploadedFilename); + } + + @Override + public int hashCode() { + return Objects.hash(indexName, indexUUID, uploadedFilename); + } + + @Override + public String toString() { + return Strings.toString(MediaTypeRegistry.JSON, this); + } + + public static UploadedIndexMetadata fromXContent(XContentParser parser) throws IOException { + return PARSER.parse(parser, null); + } + } +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java new file mode 100644 index 0000000000000..491c04bab3adb --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java @@ -0,0 +1,370 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway.remote; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.Version; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.Nullable; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Setting; +import org.opensearch.common.settings.Setting.Property; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.io.IOUtils; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedIndexMetadata; +import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.repositories.RepositoriesService; +import org.opensearch.repositories.Repository; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.repositories.blobstore.ChecksumBlobStoreFormat; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Base64; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.function.LongSupplier; +import java.util.function.Supplier; +import java.util.stream.Collectors; + +import static org.opensearch.gateway.PersistedClusterStateService.SLOW_WRITE_LOGGING_THRESHOLD; + +/** + * A Service which provides APIs to upload and download cluster metadata from remote store. + * + * @opensearch.internal + */ +public class RemoteClusterStateService implements Closeable { + + public static final String METADATA_NAME_FORMAT = "%s.dat"; + + public static final String METADATA_MANIFEST_NAME_FORMAT = "%s"; + + public static final ChecksumBlobStoreFormat INDEX_METADATA_FORMAT = new ChecksumBlobStoreFormat<>( + "index-metadata", + METADATA_NAME_FORMAT, + IndexMetadata::fromXContent + ); + + public static final ChecksumBlobStoreFormat CLUSTER_METADATA_MANIFEST_FORMAT = new ChecksumBlobStoreFormat<>( + "cluster-metadata-manifest", + METADATA_MANIFEST_NAME_FORMAT, + ClusterMetadataManifest::fromXContent + ); + /** + * Used to specify if cluster state metadata should be published to remote store + */ + // TODO The remote state enabled and repository settings should be read from node attributes. + // Dependent on https://github.com/opensearch-project/OpenSearch/pull/9105/ + public static final Setting REMOTE_CLUSTER_STATE_ENABLED_SETTING = Setting.boolSetting( + "cluster.remote_store.state.enabled", + false, + Property.NodeScope, + Property.Final + ); + /** + * Used to specify default repo to use for cluster state metadata upload + */ + public static final Setting REMOTE_CLUSTER_STATE_REPOSITORY_SETTING = Setting.simpleString( + "cluster.remote_store.state.repository", + "", + Property.NodeScope, + Property.Final + ); + private static final Logger logger = LogManager.getLogger(RemoteClusterStateService.class); + + private static final String DELIMITER = "__"; + + private final String nodeId; + private final Supplier repositoriesService; + private final Settings settings; + private final LongSupplier relativeTimeMillisSupplier; + private BlobStoreRepository blobStoreRepository; + private volatile TimeValue slowWriteLoggingThreshold; + + public RemoteClusterStateService( + String nodeId, + Supplier repositoriesService, + Settings settings, + ClusterSettings clusterSettings, + LongSupplier relativeTimeMillisSupplier + ) { + assert REMOTE_CLUSTER_STATE_ENABLED_SETTING.get(settings) == true : "Remote cluster state is not enabled"; + this.nodeId = nodeId; + this.repositoriesService = repositoriesService; + this.settings = settings; + this.relativeTimeMillisSupplier = relativeTimeMillisSupplier; + this.slowWriteLoggingThreshold = clusterSettings.get(SLOW_WRITE_LOGGING_THRESHOLD); + clusterSettings.addSettingsUpdateConsumer(SLOW_WRITE_LOGGING_THRESHOLD, this::setSlowWriteLoggingThreshold); + } + + /** + * This method uploads entire cluster state metadata to the configured blob store. For now only index metadata upload is supported. This method should be + * invoked by the elected cluster manager when the remote cluster state is enabled. + * + * @return A manifest object which contains the details of uploaded entity metadata. + */ + @Nullable + public ClusterMetadataManifest writeFullMetadata(ClusterState clusterState) throws IOException { + final long startTimeMillis = relativeTimeMillisSupplier.getAsLong(); + if (clusterState.nodes().isLocalNodeElectedClusterManager() == false) { + logger.error("Local node is not elected cluster manager. Exiting"); + return null; + } + ensureRepositorySet(); + + final List allUploadedIndexMetadata = new ArrayList<>(); + // todo parallel upload + // any validations before/after upload ? + for (IndexMetadata indexMetadata : clusterState.metadata().indices().values()) { + // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/index/ftqsCnn9TgOX/metadata_4_1690947200 + final String indexMetadataKey = writeIndexMetadata( + clusterState.getClusterName().value(), + clusterState.getMetadata().clusterUUID(), + indexMetadata, + indexMetadataFileName(indexMetadata) + ); + final UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata( + indexMetadata.getIndex().getName(), + indexMetadata.getIndexUUID(), + indexMetadataKey + ); + allUploadedIndexMetadata.add(uploadedIndexMetadata); + } + final ClusterMetadataManifest manifest = uploadManifest(clusterState, allUploadedIndexMetadata, false); + final long durationMillis = relativeTimeMillisSupplier.getAsLong() - startTimeMillis; + if (durationMillis >= slowWriteLoggingThreshold.getMillis()) { + logger.warn( + "writing cluster state took [{}ms] which is above the warn threshold of [{}]; " + "wrote full state with [{}] indices", + durationMillis, + slowWriteLoggingThreshold, + allUploadedIndexMetadata.size() + ); + } else { + // todo change to debug + logger.info( + "writing cluster state took [{}ms]; " + "wrote full state with [{}] indices", + durationMillis, + allUploadedIndexMetadata.size() + ); + } + return manifest; + } + + /** + * This method uploads the diff between the previous cluster state and the current cluster state. The previous manifest file is needed to create the new + * manifest. The new manifest file is created by using the unchanged metadata from the previous manifest and the new metadata changes from the current cluster + * state. + * + * @return The uploaded ClusterMetadataManifest file + */ + @Nullable + public ClusterMetadataManifest writeIncrementalMetadata( + ClusterState previousClusterState, + ClusterState clusterState, + ClusterMetadataManifest previousManifest + ) throws IOException { + final long startTimeMillis = relativeTimeMillisSupplier.getAsLong(); + if (clusterState.nodes().isLocalNodeElectedClusterManager() == false) { + logger.error("Local node is not elected cluster manager. Exiting"); + return null; + } + assert previousClusterState.metadata().coordinationMetadata().term() == clusterState.metadata().coordinationMetadata().term(); + final Map previousStateIndexMetadataVersionByName = new HashMap<>(); + for (final IndexMetadata indexMetadata : previousClusterState.metadata().indices().values()) { + previousStateIndexMetadataVersionByName.put(indexMetadata.getIndex().getName(), indexMetadata.getVersion()); + } + + int numIndicesUpdated = 0; + int numIndicesUnchanged = 0; + final Map allUploadedIndexMetadata = previousManifest.getIndices() + .stream() + .collect(Collectors.toMap(UploadedIndexMetadata::getIndexName, Function.identity())); + for (final IndexMetadata indexMetadata : clusterState.metadata().indices().values()) { + final Long previousVersion = previousStateIndexMetadataVersionByName.get(indexMetadata.getIndex().getName()); + if (previousVersion == null || indexMetadata.getVersion() != previousVersion) { + logger.trace( + "updating metadata for [{}], changing version from [{}] to [{}]", + indexMetadata.getIndex(), + previousVersion, + indexMetadata.getVersion() + ); + numIndicesUpdated++; + final String indexMetadataKey = writeIndexMetadata( + clusterState.getClusterName().value(), + clusterState.getMetadata().clusterUUID(), + indexMetadata, + indexMetadataFileName(indexMetadata) + ); + final UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata( + indexMetadata.getIndex().getName(), + indexMetadata.getIndexUUID(), + indexMetadataKey + ); + allUploadedIndexMetadata.put(indexMetadata.getIndex().getName(), uploadedIndexMetadata); + } else { + numIndicesUnchanged++; + } + previousStateIndexMetadataVersionByName.remove(indexMetadata.getIndex().getName()); + } + + for (String removedIndexName : previousStateIndexMetadataVersionByName.keySet()) { + allUploadedIndexMetadata.remove(removedIndexName); + } + final ClusterMetadataManifest manifest = uploadManifest( + clusterState, + allUploadedIndexMetadata.values().stream().collect(Collectors.toList()), + false + ); + final long durationMillis = relativeTimeMillisSupplier.getAsLong() - startTimeMillis; + if (durationMillis >= slowWriteLoggingThreshold.getMillis()) { + logger.warn( + "writing cluster state took [{}ms] which is above the warn threshold of [{}]; " + + "wrote metadata for [{}] indices and skipped [{}] unchanged indices", + durationMillis, + slowWriteLoggingThreshold, + numIndicesUpdated, + numIndicesUnchanged + ); + } else { + logger.trace( + "writing cluster state took [{}ms]; " + "wrote metadata for [{}] indices and skipped [{}] unchanged indices", + durationMillis, + numIndicesUpdated, + numIndicesUnchanged + ); + } + return manifest; + } + + @Nullable + public ClusterMetadataManifest markLastStateAsCommitted(ClusterState clusterState, ClusterMetadataManifest previousManifest) + throws IOException { + if (clusterState.nodes().isLocalNodeElectedClusterManager() == false) { + logger.error("Local node is not elected cluster manager. Exiting"); + return null; + } + assert clusterState != null : "Last accepted cluster state is not set"; + assert previousManifest != null : "Last cluster metadata manifest is not set"; + return uploadManifest(clusterState, previousManifest.getIndices(), true); + } + + public ClusterState getLatestClusterState(String clusterUUID) { + // todo + return null; + } + + @Override + public void close() throws IOException { + if (blobStoreRepository != null) { + IOUtils.close(blobStoreRepository); + } + } + + // Visible for testing + void ensureRepositorySet() { + if (blobStoreRepository != null) { + return; + } + final String remoteStoreRepo = REMOTE_CLUSTER_STATE_REPOSITORY_SETTING.get(settings); + assert remoteStoreRepo != null : "Remote Cluster State repository is not configured"; + final Repository repository = repositoriesService.get().repository(remoteStoreRepo); + assert repository instanceof BlobStoreRepository : "Repository should be instance of BlobStoreRepository"; + blobStoreRepository = (BlobStoreRepository) repository; + } + + private ClusterMetadataManifest uploadManifest( + ClusterState clusterState, + List uploadedIndexMetadata, + boolean committed + ) throws IOException { + synchronized (this) { + final String manifestFileName = getManifestFileName(clusterState.term(), clusterState.version()); + final ClusterMetadataManifest manifest = new ClusterMetadataManifest( + clusterState.term(), + clusterState.getVersion(), + clusterState.metadata().clusterUUID(), + clusterState.stateUUID(), + Version.CURRENT, + nodeId, + committed, + uploadedIndexMetadata + ); + writeMetadataManifest(clusterState.getClusterName().value(), clusterState.metadata().clusterUUID(), manifest, manifestFileName); + return manifest; + } + } + + private String writeIndexMetadata(String clusterName, String clusterUUID, IndexMetadata uploadIndexMetadata, String fileName) + throws IOException { + final BlobContainer indexMetadataContainer = indexMetadataContainer(clusterName, clusterUUID, uploadIndexMetadata.getIndexUUID()); + INDEX_METADATA_FORMAT.write(uploadIndexMetadata, indexMetadataContainer, fileName, blobStoreRepository.getCompressor()); + // returning full path + return indexMetadataContainer.path().buildAsString() + fileName; + } + + private void writeMetadataManifest(String clusterName, String clusterUUID, ClusterMetadataManifest uploadManifest, String fileName) + throws IOException { + final BlobContainer metadataManifestContainer = manifestContainer(clusterName, clusterUUID); + CLUSTER_METADATA_MANIFEST_FORMAT.write(uploadManifest, metadataManifestContainer, fileName, blobStoreRepository.getCompressor()); + } + + private BlobContainer indexMetadataContainer(String clusterName, String clusterUUID, String indexUUID) { + // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/index/ftqsCnn9TgOX + return blobStoreRepository.blobStore() + .blobContainer( + blobStoreRepository.basePath() + .add(Base64.getUrlEncoder().withoutPadding().encodeToString(clusterName.getBytes(StandardCharsets.UTF_8))) + .add("cluster-state") + .add(clusterUUID) + .add("index") + .add(indexUUID) + ); + } + + private BlobContainer manifestContainer(String clusterName, String clusterUUID) { + // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/manifest + return blobStoreRepository.blobStore() + .blobContainer( + blobStoreRepository.basePath() + .add(Base64.getUrlEncoder().withoutPadding().encodeToString(clusterName.getBytes(StandardCharsets.UTF_8))) + .add("cluster-state") + .add(clusterUUID) + .add("manifest") + ); + } + + private void setSlowWriteLoggingThreshold(TimeValue slowWriteLoggingThreshold) { + this.slowWriteLoggingThreshold = slowWriteLoggingThreshold; + } + + private static String getManifestFileName(long term, long version) { + // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/manifest/manifest_2147483642_2147483637_456536447 + return String.join( + DELIMITER, + "manifest", + RemoteStoreUtils.invertLong(term), + RemoteStoreUtils.invertLong(version), + RemoteStoreUtils.invertLong(System.currentTimeMillis()) + ); + } + + private static String indexMetadataFileName(IndexMetadata indexMetadata) { + return String.join(DELIMITER, "metadata", String.valueOf(indexMetadata.getVersion()), String.valueOf(System.currentTimeMillis())); + } + +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/package-info.java b/server/src/main/java/org/opensearch/gateway/remote/package-info.java new file mode 100644 index 0000000000000..286e739f66289 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/package-info.java @@ -0,0 +1,12 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +/** + * Package containing class to perform operations on remote cluster state + */ +package org.opensearch.gateway.remote; diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java index 108a022a2612b..ad8168f48558f 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -801,6 +801,10 @@ public RepositoryMetadata getMetadata() { return metadata; } + public Compressor getCompressor() { + return compressor; + } + @Override public RepositoryStats stats() { final BlobStore store = blobStore.get(); diff --git a/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java b/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java index c6b44eaa9d364..47fea55242240 100644 --- a/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java +++ b/server/src/test/java/org/opensearch/gateway/GatewayMetaStatePersistedStateTests.java @@ -35,6 +35,7 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.tests.store.MockDirectoryWrapper; import org.opensearch.ExceptionsHelper; +import org.opensearch.OpenSearchException; import org.opensearch.Version; import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.ClusterState; @@ -59,6 +60,9 @@ import org.opensearch.env.Environment; import org.opensearch.env.NodeEnvironment; import org.opensearch.env.TestEnvironment; +import org.opensearch.gateway.GatewayMetaState.RemotePersistedState; +import org.opensearch.gateway.remote.ClusterMetadataManifest; +import org.opensearch.gateway.remote.RemoteClusterStateService; import org.opensearch.node.Node; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.threadpool.TestThreadPool; @@ -75,11 +79,15 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; +import org.mockito.Mockito; + import static org.opensearch.test.NodeRoles.nonClusterManagerNode; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.nullValue; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.when; public class GatewayMetaStatePersistedStateTests extends OpenSearchTestCase { @@ -647,6 +655,70 @@ Directory createDirectory(Path path) { } } + public void testRemotePersistedState() throws IOException { + final RemoteClusterStateService remoteClusterStateService = Mockito.mock(RemoteClusterStateService.class); + final ClusterMetadataManifest manifest = ClusterMetadataManifest.builder().clusterTerm(1L).stateVersion(5L).build(); + Mockito.when(remoteClusterStateService.writeFullMetadata(Mockito.any())).thenReturn(manifest); + + Mockito.when(remoteClusterStateService.writeIncrementalMetadata(Mockito.any(), Mockito.any(), Mockito.any())).thenReturn(manifest); + CoordinationState.PersistedState remotePersistedState = new RemotePersistedState(remoteClusterStateService); + + assertThat(remotePersistedState.getLastAcceptedState(), nullValue()); + assertThat(remotePersistedState.getCurrentTerm(), equalTo(0L)); + + final long clusterTerm = randomNonNegativeLong(); + final ClusterState clusterState = createClusterState( + randomNonNegativeLong(), + Metadata.builder().coordinationMetadata(CoordinationMetadata.builder().term(clusterTerm).build()).build() + ); + + remotePersistedState.setLastAcceptedState(clusterState); + Mockito.verify(remoteClusterStateService, times(0)).writeFullMetadata(Mockito.any()); + + assertThat(remotePersistedState.getLastAcceptedState(), equalTo(clusterState)); + assertThat(remotePersistedState.getCurrentTerm(), equalTo(clusterTerm)); + + final ClusterState secondClusterState = createClusterState( + randomNonNegativeLong(), + Metadata.builder().coordinationMetadata(CoordinationMetadata.builder().term(clusterTerm).build()).build() + ); + + remotePersistedState.setLastAcceptedState(secondClusterState); + Mockito.verify(remoteClusterStateService, times(1)).writeFullMetadata(Mockito.any()); + + assertThat(remotePersistedState.getLastAcceptedState(), equalTo(secondClusterState)); + assertThat(remotePersistedState.getCurrentTerm(), equalTo(clusterTerm)); + + remotePersistedState.markLastAcceptedStateAsCommitted(); + Mockito.verify(remoteClusterStateService, times(1)).markLastStateAsCommitted(Mockito.any(), Mockito.any()); + + assertThat(remotePersistedState.getLastAcceptedState(), equalTo(secondClusterState)); + assertThat(remotePersistedState.getCurrentTerm(), equalTo(clusterTerm)); + + } + + public void testRemotePersistedStateExceptionOnFullStateUpload() throws IOException { + final RemoteClusterStateService remoteClusterStateService = Mockito.mock(RemoteClusterStateService.class); + Mockito.doThrow(IOException.class).when(remoteClusterStateService).writeFullMetadata(Mockito.any()); + + CoordinationState.PersistedState remotePersistedState = new RemotePersistedState(remoteClusterStateService); + + final long clusterTerm = randomNonNegativeLong(); + final ClusterState clusterState = createClusterState( + randomNonNegativeLong(), + Metadata.builder().coordinationMetadata(CoordinationMetadata.builder().term(clusterTerm).build()).build() + ); + + remotePersistedState.setLastAcceptedState(clusterState); + + final ClusterState secondClusterState = createClusterState( + randomNonNegativeLong(), + Metadata.builder().coordinationMetadata(CoordinationMetadata.builder().term(clusterTerm).build()).build() + ); + + assertThrows(OpenSearchException.class, () -> remotePersistedState.setLastAcceptedState(secondClusterState)); + } + private static BigArrays getBigArrays() { return usually() ? BigArrays.NON_RECYCLING_INSTANCE diff --git a/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataManifestTests.java b/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataManifestTests.java new file mode 100644 index 0000000000000..eafa191581d65 --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataManifestTests.java @@ -0,0 +1,149 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway.remote; + +import org.opensearch.Version; +import org.opensearch.common.xcontent.json.JsonXContent; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.common.io.stream.NamedWriteableRegistry; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedIndexMetadata; +import org.opensearch.test.EqualsHashCodeTestUtils; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.test.VersionUtils; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +public class ClusterMetadataManifestTests extends OpenSearchTestCase { + + public void testClusterMetadataManifestXContent() throws IOException { + UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "test-uuid", "/test/upload/path"); + ClusterMetadataManifest originalManifest = new ClusterMetadataManifest( + 1L, + 1L, + "test-cluster-uuid", + "test-state-uuid", + Version.CURRENT, + "test-node-id", + false, + Collections.singletonList(uploadedIndexMetadata) + ); + final XContentBuilder builder = JsonXContent.contentBuilder(); + builder.startObject(); + originalManifest.toXContent(builder, ToXContent.EMPTY_PARAMS); + builder.endObject(); + + try (XContentParser parser = createParser(JsonXContent.jsonXContent, BytesReference.bytes(builder))) { + final ClusterMetadataManifest fromXContentManifest = ClusterMetadataManifest.fromXContent(parser); + assertEquals(originalManifest, fromXContentManifest); + } + } + + public void testClusterMetadataManifestSerializationEqualsHashCode() { + ClusterMetadataManifest initialManifest = new ClusterMetadataManifest( + randomNonNegativeLong(), + randomNonNegativeLong(), + randomAlphaOfLength(10), + randomAlphaOfLength(10), + VersionUtils.randomOpenSearchVersion(random()), + randomAlphaOfLength(10), + randomBoolean(), + randomUploadedIndexMetadataList() + ); + EqualsHashCodeTestUtils.checkEqualsAndHashCode( + initialManifest, + orig -> OpenSearchTestCase.copyWriteable( + orig, + new NamedWriteableRegistry(Collections.emptyList()), + ClusterMetadataManifest::new + ), + manifest -> { + ClusterMetadataManifest.Builder builder = ClusterMetadataManifest.builder(manifest); + switch (randomInt(7)) { + case 0: + builder.clusterTerm(randomNonNegativeLong()); + break; + case 1: + builder.stateVersion(randomNonNegativeLong()); + break; + case 2: + builder.clusterUUID(randomAlphaOfLength(10)); + break; + case 3: + builder.stateUUID(randomAlphaOfLength(10)); + break; + case 4: + builder.opensearchVersion(VersionUtils.randomOpenSearchVersion(random())); + break; + case 5: + builder.nodeId(randomAlphaOfLength(10)); + break; + case 6: + builder.committed(randomBoolean()); + break; + case 7: + builder.indices(randomUploadedIndexMetadataList()); + break; + } + return builder.build(); + } + ); + } + + private List randomUploadedIndexMetadataList() { + final int size = randomIntBetween(1, 10); + final List uploadedIndexMetadataList = new ArrayList<>(size); + while (uploadedIndexMetadataList.size() < size) { + assertTrue(uploadedIndexMetadataList.add(randomUploadedIndexMetadata())); + } + return uploadedIndexMetadataList; + } + + private UploadedIndexMetadata randomUploadedIndexMetadata() { + return new UploadedIndexMetadata(randomAlphaOfLength(10), randomAlphaOfLength(10), randomAlphaOfLength(10)); + } + + public void testUploadedIndexMetadataSerializationEqualsHashCode() { + UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "test-uuid", "/test/upload/path"); + EqualsHashCodeTestUtils.checkEqualsAndHashCode( + uploadedIndexMetadata, + orig -> OpenSearchTestCase.copyWriteable(orig, new NamedWriteableRegistry(Collections.emptyList()), UploadedIndexMetadata::new), + metadata -> randomlyChangingUploadedIndexMetadata(uploadedIndexMetadata) + ); + } + + private UploadedIndexMetadata randomlyChangingUploadedIndexMetadata(UploadedIndexMetadata uploadedIndexMetadata) { + switch (randomInt(2)) { + case 0: + return new UploadedIndexMetadata( + randomAlphaOfLength(10), + uploadedIndexMetadata.getIndexUUID(), + uploadedIndexMetadata.getUploadedFilename() + ); + case 1: + return new UploadedIndexMetadata( + uploadedIndexMetadata.getIndexName(), + randomAlphaOfLength(10), + uploadedIndexMetadata.getUploadedFilename() + ); + case 2: + return new UploadedIndexMetadata( + uploadedIndexMetadata.getIndexName(), + uploadedIndexMetadata.getIndexUUID(), + randomAlphaOfLength(10) + ); + } + return uploadedIndexMetadata; + } +} diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java new file mode 100644 index 0000000000000..215673642cce5 --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java @@ -0,0 +1,254 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway.remote; + +import org.opensearch.Version; +import org.opensearch.cluster.ClusterName; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.coordination.CoordinationMetadata; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.blobstore.BlobStore; +import org.opensearch.common.compress.DeflateCompressor; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.index.Index; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedIndexMetadata; +import org.opensearch.repositories.FilterRepository; +import org.opensearch.repositories.RepositoriesService; +import org.opensearch.repositories.RepositoryMissingException; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Assert; +import org.junit.Before; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.function.Supplier; + +import org.mockito.ArgumentMatchers; + +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class RemoteClusterStateServiceTests extends OpenSearchTestCase { + + private RemoteClusterStateService remoteClusterStateService; + private Supplier repositoriesServiceSupplier; + private RepositoriesService repositoriesService; + private BlobStoreRepository blobStoreRepository; + + @Before + public void setup() { + repositoriesServiceSupplier = mock(Supplier.class); + repositoriesService = mock(RepositoriesService.class); + when(repositoriesServiceSupplier.get()).thenReturn(repositoriesService); + final Settings settings = Settings.builder() + .put(RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true) + .put(RemoteClusterStateService.REMOTE_CLUSTER_STATE_REPOSITORY_SETTING.getKey(), "remote_store_repository") + .build(); + blobStoreRepository = mock(BlobStoreRepository.class); + when(repositoriesService.repository("remote_store_repository")).thenReturn(blobStoreRepository); + remoteClusterStateService = new RemoteClusterStateService( + "test-node-id", + repositoriesServiceSupplier, + settings, + new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), + () -> 0L + ); + } + + public void testFailWriteFullMetadataNonClusterManagerNode() throws IOException { + final ClusterState clusterState = generateClusterStateWithOneIndex().build(); + final ClusterMetadataManifest manifest = remoteClusterStateService.writeFullMetadata(clusterState); + Assert.assertThat(manifest, nullValue()); + } + + public void testFailInitializationWhenRemoteStateDisabled() throws IOException { + final Settings settings = Settings.builder().build(); + assertThrows( + AssertionError.class, + () -> new RemoteClusterStateService( + "test-node-id", + repositoriesServiceSupplier, + settings, + new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), + () -> 0L + ) + ); + } + + public void testFailWriteFullMetadataWhenRepositoryNotSet() { + final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); + doThrow(new RepositoryMissingException("repository missing")).when(repositoriesService).repository("remote_store_repository"); + assertThrows(RepositoryMissingException.class, () -> remoteClusterStateService.writeFullMetadata(clusterState)); + } + + public void testFailWriteFullMetadataWhenNotBlobRepository() { + final FilterRepository filterRepository = mock(FilterRepository.class); + when(repositoriesService.repository("remote_store_repository")).thenReturn(filterRepository); + final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); + assertThrows(AssertionError.class, () -> remoteClusterStateService.writeFullMetadata(clusterState)); + } + + public void testWriteFullMetadataSuccess() throws IOException { + final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); + mockBlobStoreObjects(); + final ClusterMetadataManifest manifest = remoteClusterStateService.writeFullMetadata(clusterState); + final UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "index-uuid", "metadata-filename"); + List indices = List.of(uploadedIndexMetadata); + + final ClusterMetadataManifest expectedManifest = ClusterMetadataManifest.builder() + .indices(indices) + .clusterTerm(1L) + .stateVersion(1L) + .stateUUID("state-uuid") + .clusterUUID("cluster-uuid") + .build(); + + assertThat(manifest.getIndices().size(), is(1)); + assertThat(manifest.getIndices().get(0).getIndexName(), is(uploadedIndexMetadata.getIndexName())); + assertThat(manifest.getIndices().get(0).getIndexUUID(), is(uploadedIndexMetadata.getIndexUUID())); + assertThat(manifest.getIndices().get(0).getUploadedFilename(), notNullValue()); + assertThat(manifest.getClusterTerm(), is(expectedManifest.getClusterTerm())); + assertThat(manifest.getStateVersion(), is(expectedManifest.getStateVersion())); + assertThat(manifest.getClusterUUID(), is(expectedManifest.getClusterUUID())); + assertThat(manifest.getStateUUID(), is(expectedManifest.getStateUUID())); + } + + public void testFailWriteIncrementalMetadataNonClusterManagerNode() throws IOException { + final ClusterState clusterState = generateClusterStateWithOneIndex().build(); + final ClusterMetadataManifest manifest = remoteClusterStateService.writeIncrementalMetadata(clusterState, clusterState, null); + Assert.assertThat(manifest, nullValue()); + } + + public void testFailWriteIncrementalMetadataWhenTermChanged() { + final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); + final CoordinationMetadata coordinationMetadata = CoordinationMetadata.builder().term(2L).build(); + final ClusterState previousClusterState = ClusterState.builder(ClusterName.DEFAULT) + .metadata(Metadata.builder().coordinationMetadata(coordinationMetadata)) + .build(); + assertThrows( + AssertionError.class, + () -> remoteClusterStateService.writeIncrementalMetadata(previousClusterState, clusterState, null) + ); + } + + public void testWriteIncrementalMetadataSuccess() throws IOException { + final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); + mockBlobStoreObjects(); + final CoordinationMetadata coordinationMetadata = CoordinationMetadata.builder().term(1L).build(); + final ClusterState previousClusterState = ClusterState.builder(ClusterName.DEFAULT) + .metadata(Metadata.builder().coordinationMetadata(coordinationMetadata)) + .build(); + + final ClusterMetadataManifest previousManifest = ClusterMetadataManifest.builder().indices(Collections.emptyList()).build(); + + remoteClusterStateService.ensureRepositorySet(); + final ClusterMetadataManifest manifest = remoteClusterStateService.writeIncrementalMetadata( + previousClusterState, + clusterState, + previousManifest + ); + final UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "index-uuid", "metadata-filename"); + final List indices = List.of(uploadedIndexMetadata); + + final ClusterMetadataManifest expectedManifest = ClusterMetadataManifest.builder() + .indices(indices) + .clusterTerm(1L) + .stateVersion(1L) + .stateUUID("state-uuid") + .clusterUUID("cluster-uuid") + .build(); + + assertThat(manifest.getIndices().size(), is(1)); + assertThat(manifest.getIndices().get(0).getIndexName(), is(uploadedIndexMetadata.getIndexName())); + assertThat(manifest.getIndices().get(0).getIndexUUID(), is(uploadedIndexMetadata.getIndexUUID())); + assertThat(manifest.getIndices().get(0).getUploadedFilename(), notNullValue()); + assertThat(manifest.getClusterTerm(), is(expectedManifest.getClusterTerm())); + assertThat(manifest.getStateVersion(), is(expectedManifest.getStateVersion())); + assertThat(manifest.getClusterUUID(), is(expectedManifest.getClusterUUID())); + assertThat(manifest.getStateUUID(), is(expectedManifest.getStateUUID())); + } + + public void testMarkLastStateAsCommittedSuccess() throws IOException { + final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); + mockBlobStoreObjects(); + remoteClusterStateService.ensureRepositorySet(); + final UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "index-uuid", "metadata-filename"); + List indices = List.of(uploadedIndexMetadata); + final ClusterMetadataManifest previousManifest = ClusterMetadataManifest.builder().indices(indices).build(); + + final ClusterMetadataManifest manifest = remoteClusterStateService.markLastStateAsCommitted(clusterState, previousManifest); + + final ClusterMetadataManifest expectedManifest = ClusterMetadataManifest.builder() + .indices(indices) + .clusterTerm(1L) + .stateVersion(1L) + .stateUUID("state-uuid") + .clusterUUID("cluster-uuid") + .build(); + + assertThat(manifest.getIndices().size(), is(1)); + assertThat(manifest.getIndices().get(0).getIndexName(), is(uploadedIndexMetadata.getIndexName())); + assertThat(manifest.getIndices().get(0).getIndexUUID(), is(uploadedIndexMetadata.getIndexUUID())); + assertThat(manifest.getIndices().get(0).getUploadedFilename(), notNullValue()); + assertThat(manifest.getClusterTerm(), is(expectedManifest.getClusterTerm())); + assertThat(manifest.getStateVersion(), is(expectedManifest.getStateVersion())); + assertThat(manifest.getClusterUUID(), is(expectedManifest.getClusterUUID())); + assertThat(manifest.getStateUUID(), is(expectedManifest.getStateUUID())); + } + + private void mockBlobStoreObjects() { + final BlobStore blobStore = mock(BlobStore.class); + when(blobStoreRepository.blobStore()).thenReturn(blobStore); + final BlobPath blobPath = mock(BlobPath.class); + when((blobStoreRepository.basePath())).thenReturn(blobPath); + when(blobPath.add(anyString())).thenReturn(blobPath); + when(blobPath.buildAsString()).thenReturn("/blob/path/"); + final BlobContainer blobContainer = mock(BlobContainer.class); + when(blobContainer.path()).thenReturn(blobPath); + when(blobStore.blobContainer(ArgumentMatchers.any())).thenReturn(blobContainer); + when(blobStoreRepository.getCompressor()).thenReturn(new DeflateCompressor()); + } + + private static ClusterState.Builder generateClusterStateWithOneIndex() { + final Index index = new Index("test-index", "index-uuid"); + final Settings idxSettings = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_INDEX_UUID, index.getUUID()) + .build(); + final IndexMetadata indexMetadata = new IndexMetadata.Builder(index.getName()).settings(idxSettings) + .numberOfShards(1) + .numberOfReplicas(0) + .build(); + final CoordinationMetadata coordinationMetadata = CoordinationMetadata.builder().term(1L).build(); + + return ClusterState.builder(ClusterName.DEFAULT) + .version(1L) + .stateUUID("state-uuid") + .metadata( + Metadata.builder().put(indexMetadata, true).clusterUUID("cluster-uuid").coordinationMetadata(coordinationMetadata).build() + ); + } + + private static DiscoveryNodes nodesWithLocalNodeClusterManager() { + return DiscoveryNodes.builder().clusterManagerNodeId("cluster-manager-id").localNodeId("cluster-manager-id").build(); + } + +} From f9a661cd983255253307d5d07947dc47b129ab16 Mon Sep 17 00:00:00 2001 From: Vikas Bansal <43470111+vikasvb90@users.noreply.github.com> Date: Thu, 31 Aug 2023 14:36:36 +0530 Subject: [PATCH 22/37] Add Crypto Handler abstractions for encryption/decryption and Crypto Key Provider Plugin to provide keys (#8466) Signed-off-by: Vikas Bansal <43470111+vikasvb90@users.noreply.github.com> Signed-off-by: Gaurav Bafna --- CHANGELOG.md | 2 + buildSrc/version.properties | 2 +- .../common/crypto/CryptoHandler.java | 114 ++++++++++ .../opensearch/common/crypto/DataKeyPair.java | 42 ++++ .../crypto/DecryptedRangedStreamProvider.java | 49 +++++ .../EncryptedHeaderContentSupplier.java | 25 +++ .../common/crypto/MasterKeyProvider.java | 41 ++++ .../common/crypto/package-info.java | 10 + libs/encryption-sdk/build.gradle | 52 +++++ .../aws-encryption-sdk-java-2.4.0.jar.sha1 | 1 + .../aws-encryption-sdk-java-LICENSE.txt | 201 +++++++++++++++++ .../aws-encryption-sdk-java-NOTICE.txt | 11 + .../licenses/bcprov-jdk15to18-1.75.jar.sha1 | 1 + .../licenses/bcprov-jdk15to18-LICENSE.txt | 22 ++ .../licenses/bcprov-jdk15to18-NOTICE.txt | 0 .../licenses/commons-lang3-3.13.0.jar.sha1 | 1 + .../licenses/commons-lang3-LICENSE.txt | 202 ++++++++++++++++++ .../licenses/commons-lang3-NOTICE.txt | 5 + .../licenses/commons-logging-1.2.jar.sha1 | 1 + .../licenses/commons-logging-LICENSE.txt | 202 ++++++++++++++++++ .../licenses/commons-logging-NOTICE.txt | 5 + .../licenses/slf4j-api-1.7.36.jar.sha1 | 1 + .../licenses/slf4j-api-LICENSE.txt | 21 ++ .../licenses/slf4j-api-NOTICE.txt | 0 .../src/forbidden/crypto-signatures.txt | 13 ++ .../src/forbidden/crypto-test-signatures.txt | 13 ++ .../opensearch/encryption/CryptoManager.java | 33 +++ .../encryption/CryptoManagerFactory.java | 127 +++++++++++ .../encryption/NoOpCryptoHandler.java | 128 +++++++++++ .../opensearch/encryption/TrimmingStream.java | 119 +++++++++++ .../keyprovider/CryptoMasterKey.java | 84 ++++++++ .../encryption/keyprovider/package-info.java | 12 ++ .../opensearch/encryption/package-info.java | 12 ++ .../encryption/CryptoManagerFactoryTests.java | 92 ++++++++ .../encryption/MockKeyProvider.java | 109 ++++++++++ .../encryption/NoOpCryptoHandlerTests.java | 95 ++++++++ .../encryption/TrimmingStreamTests.java | 125 +++++++++++ .../src/test/resources/encrypted_key | 1 + .../resources/raw_content_for_crypto_test | 25 +++ .../encryption-sdk/src/test/resources/raw_key | 1 + plugins/ingest-attachment/build.gradle | 2 +- plugins/repository-azure/build.gradle | 2 +- 42 files changed, 2001 insertions(+), 3 deletions(-) create mode 100644 libs/common/src/main/java/org/opensearch/common/crypto/CryptoHandler.java create mode 100644 libs/common/src/main/java/org/opensearch/common/crypto/DataKeyPair.java create mode 100644 libs/common/src/main/java/org/opensearch/common/crypto/DecryptedRangedStreamProvider.java create mode 100644 libs/common/src/main/java/org/opensearch/common/crypto/EncryptedHeaderContentSupplier.java create mode 100644 libs/common/src/main/java/org/opensearch/common/crypto/MasterKeyProvider.java create mode 100644 libs/common/src/main/java/org/opensearch/common/crypto/package-info.java create mode 100644 libs/encryption-sdk/build.gradle create mode 100644 libs/encryption-sdk/licenses/aws-encryption-sdk-java-2.4.0.jar.sha1 create mode 100644 libs/encryption-sdk/licenses/aws-encryption-sdk-java-LICENSE.txt create mode 100644 libs/encryption-sdk/licenses/aws-encryption-sdk-java-NOTICE.txt create mode 100644 libs/encryption-sdk/licenses/bcprov-jdk15to18-1.75.jar.sha1 create mode 100644 libs/encryption-sdk/licenses/bcprov-jdk15to18-LICENSE.txt create mode 100644 libs/encryption-sdk/licenses/bcprov-jdk15to18-NOTICE.txt create mode 100644 libs/encryption-sdk/licenses/commons-lang3-3.13.0.jar.sha1 create mode 100644 libs/encryption-sdk/licenses/commons-lang3-LICENSE.txt create mode 100644 libs/encryption-sdk/licenses/commons-lang3-NOTICE.txt create mode 100644 libs/encryption-sdk/licenses/commons-logging-1.2.jar.sha1 create mode 100644 libs/encryption-sdk/licenses/commons-logging-LICENSE.txt create mode 100644 libs/encryption-sdk/licenses/commons-logging-NOTICE.txt create mode 100644 libs/encryption-sdk/licenses/slf4j-api-1.7.36.jar.sha1 create mode 100644 libs/encryption-sdk/licenses/slf4j-api-LICENSE.txt create mode 100644 libs/encryption-sdk/licenses/slf4j-api-NOTICE.txt create mode 100644 libs/encryption-sdk/src/forbidden/crypto-signatures.txt create mode 100644 libs/encryption-sdk/src/forbidden/crypto-test-signatures.txt create mode 100644 libs/encryption-sdk/src/main/java/org/opensearch/encryption/CryptoManager.java create mode 100644 libs/encryption-sdk/src/main/java/org/opensearch/encryption/CryptoManagerFactory.java create mode 100644 libs/encryption-sdk/src/main/java/org/opensearch/encryption/NoOpCryptoHandler.java create mode 100644 libs/encryption-sdk/src/main/java/org/opensearch/encryption/TrimmingStream.java create mode 100644 libs/encryption-sdk/src/main/java/org/opensearch/encryption/keyprovider/CryptoMasterKey.java create mode 100644 libs/encryption-sdk/src/main/java/org/opensearch/encryption/keyprovider/package-info.java create mode 100644 libs/encryption-sdk/src/main/java/org/opensearch/encryption/package-info.java create mode 100644 libs/encryption-sdk/src/test/java/org/opensearch/encryption/CryptoManagerFactoryTests.java create mode 100644 libs/encryption-sdk/src/test/java/org/opensearch/encryption/MockKeyProvider.java create mode 100644 libs/encryption-sdk/src/test/java/org/opensearch/encryption/NoOpCryptoHandlerTests.java create mode 100644 libs/encryption-sdk/src/test/java/org/opensearch/encryption/TrimmingStreamTests.java create mode 100644 libs/encryption-sdk/src/test/resources/encrypted_key create mode 100644 libs/encryption-sdk/src/test/resources/raw_content_for_crypto_test create mode 100644 libs/encryption-sdk/src/test/resources/raw_key diff --git a/CHANGELOG.md b/CHANGELOG.md index e65cc30aecd7e..5df33dc6e3ace 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -11,6 +11,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add events correlation engine plugin ([#6854](https://github.com/opensearch-project/OpenSearch/issues/6854)) - Introduce new dynamic cluster setting to control slice computation for concurrent segment search ([#9107](https://github.com/opensearch-project/OpenSearch/pull/9107)) - Implement on behalf of token passing for extensions ([#8679](https://github.com/opensearch-project/OpenSearch/pull/8679)) +- Added encryption-sdk lib to provide encryption and decryption capabilities ([#8466](https://github.com/opensearch-project/OpenSearch/pull/8466)) ### Dependencies - Bump `log4j-core` from 2.18.0 to 2.19.0 @@ -40,6 +41,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `org.bouncycastle:bcprov-jdk15on` to `org.bouncycastle:bcprov-jdk15to18` version 1.75 ([#8247](https://github.com/opensearch-project/OpenSearch/pull/8247)) - Bump `org.bouncycastle:bcmail-jdk15on` to `org.bouncycastle:bcmail-jdk15to18` version 1.75 ([#8247](https://github.com/opensearch-project/OpenSearch/pull/8247)) - Bump `org.bouncycastle:bcpkix-jdk15on` to `org.bouncycastle:bcpkix-jdk15to18` version 1.75 ([#8247](https://github.com/opensearch-project/OpenSearch/pull/8247)) +- Add Encryption SDK dependencies ([#8466](https://github.com/opensearch-project/OpenSearch/pull/8466)) ### Changed - [CCR] Add getHistoryOperationsFromTranslog method to fetch the history snapshot from translogs ([#3948](https://github.com/opensearch-project/OpenSearch/pull/3948)) diff --git a/buildSrc/version.properties b/buildSrc/version.properties index 0d84ddebb905c..500727e909494 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -39,7 +39,7 @@ httpcore = 4.4.16 httpasyncclient = 4.1.5 commonslogging = 1.2 commonscodec = 1.15 - +commonslang = 3.13.0 # plugin dependencies aws = 2.20.55 reactivestreams = 1.0.4 diff --git a/libs/common/src/main/java/org/opensearch/common/crypto/CryptoHandler.java b/libs/common/src/main/java/org/opensearch/common/crypto/CryptoHandler.java new file mode 100644 index 0000000000000..bdcaa295f1cc4 --- /dev/null +++ b/libs/common/src/main/java/org/opensearch/common/crypto/CryptoHandler.java @@ -0,0 +1,114 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.common.crypto; + +import org.opensearch.common.io.InputStreamContainer; + +import java.io.IOException; +import java.io.InputStream; + +/** + * Crypto provider abstractions for encryption and decryption of data. Allows registering multiple providers + * for defining different ways of encrypting or decrypting data. + * + * T - Encryption Metadata / CryptoContext + * U - Parsed Encryption Metadata / CryptoContext + */ +public interface CryptoHandler { + + /** + * To initialise or create a new crypto metadata to be used in encryption. This is needed to set the context before + * beginning encryption. + * + * @return crypto metadata instance + */ + T initEncryptionMetadata(); + + /** + * To load crypto metadata to be used in encryption from content header. + * Note that underlying information in the loaded metadata object is same as present in the object created during + * encryption but object type may differ. + * + * @return crypto metadata instance used in decryption. + */ + U loadEncryptionMetadata(EncryptedHeaderContentSupplier encryptedHeaderContentSupplier) throws IOException; + + /** + * Few encryption algorithms have certain conditions on the unit of content to be encrypted. This requires the + * content size to be re adjusted in order to fulfil these conditions for partial writes. If write requests for + * encryption of a part of content do not fulfil these conditions then encryption fails or can result in corrupted + * content depending on the algorithm used. This method exposes a means to re-adjust sizes of such writes. + * + * @param cryptoContext crypto metadata instance + * @param contentSize Size of the raw content + * @return Adjusted size of the content. + */ + long adjustContentSizeForPartialEncryption(T cryptoContext, long contentSize); + + /** + * Estimate length of the encrypted content. It should only be used to determine length of entire content after + * encryption. + * + * @param cryptoContext crypto metadata instance consisting of encryption metadata used in encryption. + * @param contentLength Size of the raw content + * @return Calculated size of the encrypted content. + */ + long estimateEncryptedLengthOfEntireContent(T cryptoContext, long contentLength); + + /** + * For given encrypted content length, estimate the length of the decrypted content. + * @param cryptoContext crypto metadata instance consisting of encryption metadata used in encryption. + * @param contentLength Size of the encrypted content + * @return Calculated size of the decrypted content. + */ + long estimateDecryptedLength(U cryptoContext, long contentLength); + + /** + * Wraps a raw InputStream with encrypting stream + * + * @param encryptionMetadata created earlier to set the crypto metadata. + * @param stream Raw InputStream to encrypt + * @return encrypting stream wrapped around raw InputStream. + */ + InputStreamContainer createEncryptingStream(T encryptionMetadata, InputStreamContainer stream); + + /** + * Provides encrypted stream for a raw stream emitted for a part of content. + * + * @param cryptoContext crypto metadata instance. + * @param stream raw stream for which encrypted stream has to be created. + * @param totalStreams Number of streams being used for the entire content. + * @param streamIdx Index of the current stream. + * @return Encrypted stream for the provided raw stream. + */ + InputStreamContainer createEncryptingStreamOfPart(T cryptoContext, InputStreamContainer stream, int totalStreams, int streamIdx); + + /** + * This method accepts an encrypted stream and provides a decrypting wrapper. + * @param encryptingStream to be decrypted. + * @return Decrypting wrapper stream + */ + InputStream createDecryptingStream(InputStream encryptingStream); + + /** + * This method creates a {@link DecryptedRangedStreamProvider} which provides a wrapped stream to decrypt the + * underlying stream. This also provides adjusted range against the actual range which should be used for fetching + * and supplying the encrypted content for decryption. Extra content outside the range is trimmed down and returned + * by the decrypted stream. + * For partial reads of encrypted content, few algorithms require the range of content to be adjusted for + * successful decryption. Adjusted range may or may not be same as the provided range. If range is adjusted then + * starting offset of resultant range can be lesser than the starting offset of provided range and end + * offset can be greater than the ending offset of the provided range. + * + * @param cryptoContext crypto metadata instance. + * @param startPosOfRawContent starting position in the raw/decrypted content + * @param endPosOfRawContent ending position in the raw/decrypted content + */ + DecryptedRangedStreamProvider createDecryptingStreamOfRange(U cryptoContext, long startPosOfRawContent, long endPosOfRawContent); +} diff --git a/libs/common/src/main/java/org/opensearch/common/crypto/DataKeyPair.java b/libs/common/src/main/java/org/opensearch/common/crypto/DataKeyPair.java new file mode 100644 index 0000000000000..3d487f9028d71 --- /dev/null +++ b/libs/common/src/main/java/org/opensearch/common/crypto/DataKeyPair.java @@ -0,0 +1,42 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ +package org.opensearch.common.crypto; + +/** + * Key pair generated by {@link MasterKeyProvider} + */ +public class DataKeyPair { + private final byte[] rawKey; + private final byte[] encryptedKey; + + /** + * Constructor to initialize key-pair values + * @param rawKey Unencrypted data key used for encryption and decryption + * @param encryptedKey Encrypted version of rawKey + */ + public DataKeyPair(byte[] rawKey, byte[] encryptedKey) { + this.rawKey = rawKey; + this.encryptedKey = encryptedKey; + } + + /** + * Returns raw key + * @return raw/decrypted key + */ + public byte[] getRawKey() { + return rawKey; + } + + /** + * Returns encrypted key + * @return encrypted key + */ + public byte[] getEncryptedKey() { + return encryptedKey; + } +} diff --git a/libs/common/src/main/java/org/opensearch/common/crypto/DecryptedRangedStreamProvider.java b/libs/common/src/main/java/org/opensearch/common/crypto/DecryptedRangedStreamProvider.java new file mode 100644 index 0000000000000..06b18027a0726 --- /dev/null +++ b/libs/common/src/main/java/org/opensearch/common/crypto/DecryptedRangedStreamProvider.java @@ -0,0 +1,49 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.common.crypto; + +import java.io.InputStream; +import java.util.function.UnaryOperator; + +/** + * Contains adjusted range of partial encrypted content which needs to be used for decryption. + */ +public class DecryptedRangedStreamProvider { + + private final long[] adjustedRange; + private final UnaryOperator decryptedStreamProvider; + + /** + * To construct adjusted encrypted range. + * @param adjustedRange range of partial encrypted content which needs to be used for decryption. + * @param decryptedStreamProvider stream provider for decryption and range re-adjustment. + */ + public DecryptedRangedStreamProvider(long[] adjustedRange, UnaryOperator decryptedStreamProvider) { + this.adjustedRange = adjustedRange; + this.decryptedStreamProvider = decryptedStreamProvider; + } + + /** + * Adjusted range of partial encrypted content which needs to be used for decryption. + * @return adjusted range + */ + public long[] getAdjustedRange() { + return adjustedRange; + } + + /** + * A utility stream provider which supplies the stream responsible for decrypting the content and reading the + * desired range of decrypted content by skipping extra content which got decrypted as a result of range adjustment. + * @return stream provider for decryption and supplying the desired range of content. + */ + public UnaryOperator getDecryptedStreamProvider() { + return decryptedStreamProvider; + } + +} diff --git a/libs/common/src/main/java/org/opensearch/common/crypto/EncryptedHeaderContentSupplier.java b/libs/common/src/main/java/org/opensearch/common/crypto/EncryptedHeaderContentSupplier.java new file mode 100644 index 0000000000000..49a037f05f185 --- /dev/null +++ b/libs/common/src/main/java/org/opensearch/common/crypto/EncryptedHeaderContentSupplier.java @@ -0,0 +1,25 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ +package org.opensearch.common.crypto; + +import java.io.IOException; + +/** + * This is used in partial decryption. Header information is required for decryption of actual encrypted content. + * Implementation of this supplier only requires first few bytes of encrypted content to be supplied. + */ +public interface EncryptedHeaderContentSupplier { + + /** + * @param start Start position of the encrypted content (Generally supplied as 0 during usage) + * @param end End position of the header. + * @return Encrypted header content (May contain additional content which is later discarded) + * @throws IOException In case content fetch fails. + */ + byte[] supply(long start, long end) throws IOException; +} diff --git a/libs/common/src/main/java/org/opensearch/common/crypto/MasterKeyProvider.java b/libs/common/src/main/java/org/opensearch/common/crypto/MasterKeyProvider.java new file mode 100644 index 0000000000000..b5feeb6d37ec6 --- /dev/null +++ b/libs/common/src/main/java/org/opensearch/common/crypto/MasterKeyProvider.java @@ -0,0 +1,41 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ +package org.opensearch.common.crypto; + +import java.io.Closeable; +import java.util.Map; + +/** + * Master key provider responsible for management of master keys. + */ +public interface MasterKeyProvider extends Closeable { + + /** + * Returns data key pair + * @return data key pair generated by master key. + */ + DataKeyPair generateDataPair(); + + /** + * Returns decrpted key against the encrypted key. + * @param encryptedKey Key to decrypt + * @return Decrypted version of key. + */ + byte[] decryptKey(byte[] encryptedKey); + + /** + * Returns key id. + * @return key id + */ + String getKeyId(); + + /** + * @return encryption context associated with this master key. + */ + Map getEncryptionContext(); +} diff --git a/libs/common/src/main/java/org/opensearch/common/crypto/package-info.java b/libs/common/src/main/java/org/opensearch/common/crypto/package-info.java new file mode 100644 index 0000000000000..c744689ebf532 --- /dev/null +++ b/libs/common/src/main/java/org/opensearch/common/crypto/package-info.java @@ -0,0 +1,10 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +/** Common crypto utilities used across opensearch. */ +package org.opensearch.common.crypto; diff --git a/libs/encryption-sdk/build.gradle b/libs/encryption-sdk/build.gradle new file mode 100644 index 0000000000000..d229d4edf0a83 --- /dev/null +++ b/libs/encryption-sdk/build.gradle @@ -0,0 +1,52 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +apply plugin: 'opensearch.build' +apply plugin: 'opensearch.publish' + +forbiddenApis.ignoreFailures = false + +thirdPartyAudit.enabled = false +forbiddenApisTest.ignoreFailures = true +testingConventions.enabled = false + +dependencies { + // Common crypto classes + api project(':libs:opensearch-common') + + // Logger + implementation "org.slf4j:slf4j-api:${versions.slf4j}" + implementation 'commons-logging:commons-logging:1.2' + + // Encryption + implementation "com.amazonaws:aws-encryption-sdk-java:2.4.0" + implementation "org.bouncycastle:bcprov-jdk15to18:${versions.bouncycastle}" + implementation "org.apache.commons:commons-lang3:${versions.commonslang}" + + //Tests + testImplementation "junit:junit:${versions.junit}" + testImplementation "org.hamcrest:hamcrest:${versions.hamcrest}" + testImplementation(project(":test:framework")) { + exclude group: 'org.opensearch', module: 'opensearch-encryption-sdk' + } + + compileOnly 'com.google.code.findbugs:annotations:3.0.1' +} + +tasks.named('forbiddenApisMain').configure { + // Only enable limited check because AD code has too many violations. + replaceSignatureFiles 'jdk-signatures' + signaturesFiles += files('src/forbidden/crypto-signatures.txt') +} + +// Encryption SDK files have missing java docs so disabling for the lib. +tasks.named('missingJavadoc').configure { + enabled = false +} + +forbiddenApisTest.setSignaturesFiles(files('src/forbidden/crypto-test-signatures.txt')) diff --git a/libs/encryption-sdk/licenses/aws-encryption-sdk-java-2.4.0.jar.sha1 b/libs/encryption-sdk/licenses/aws-encryption-sdk-java-2.4.0.jar.sha1 new file mode 100644 index 0000000000000..504b4a423a975 --- /dev/null +++ b/libs/encryption-sdk/licenses/aws-encryption-sdk-java-2.4.0.jar.sha1 @@ -0,0 +1 @@ +98943eda1dc05bb01f4f5405e115b08dc541afbf \ No newline at end of file diff --git a/libs/encryption-sdk/licenses/aws-encryption-sdk-java-LICENSE.txt b/libs/encryption-sdk/licenses/aws-encryption-sdk-java-LICENSE.txt new file mode 100644 index 0000000000000..8dada3edaf50d --- /dev/null +++ b/libs/encryption-sdk/licenses/aws-encryption-sdk-java-LICENSE.txt @@ -0,0 +1,201 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "{}" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright {yyyy} {name of copyright owner} + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/libs/encryption-sdk/licenses/aws-encryption-sdk-java-NOTICE.txt b/libs/encryption-sdk/licenses/aws-encryption-sdk-java-NOTICE.txt new file mode 100644 index 0000000000000..e32695955374a --- /dev/null +++ b/libs/encryption-sdk/licenses/aws-encryption-sdk-java-NOTICE.txt @@ -0,0 +1,11 @@ +AWS Encryption SDK +Copyright 2015 Amazon.com, Inc. or its affiliates. All Rights Reserved. + +THIRD PARTY COMPONENTS +********************** +This software includes third party software subject to the following copyrights: + +-Cryptographic functions from Bouncy Castle Crypto APIs for Java - Copyright +2000-2013 The Legion of the Bouncy Castle + +The licenses for these third party components are included in LICENSE.txt diff --git a/libs/encryption-sdk/licenses/bcprov-jdk15to18-1.75.jar.sha1 b/libs/encryption-sdk/licenses/bcprov-jdk15to18-1.75.jar.sha1 new file mode 100644 index 0000000000000..9911bb75f9209 --- /dev/null +++ b/libs/encryption-sdk/licenses/bcprov-jdk15to18-1.75.jar.sha1 @@ -0,0 +1 @@ +df22e1b6a9f6b218913f5b68dd16641344397fe0 \ No newline at end of file diff --git a/libs/encryption-sdk/licenses/bcprov-jdk15to18-LICENSE.txt b/libs/encryption-sdk/licenses/bcprov-jdk15to18-LICENSE.txt new file mode 100644 index 0000000000000..9f27bafe96885 --- /dev/null +++ b/libs/encryption-sdk/licenses/bcprov-jdk15to18-LICENSE.txt @@ -0,0 +1,22 @@ +The MIT License (MIT) + +Copyright (c) 2000 - 2013 The Legion of the Bouncy Castle Inc. + (http://www.bouncycastle.org) + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. diff --git a/libs/encryption-sdk/licenses/bcprov-jdk15to18-NOTICE.txt b/libs/encryption-sdk/licenses/bcprov-jdk15to18-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/libs/encryption-sdk/licenses/commons-lang3-3.13.0.jar.sha1 b/libs/encryption-sdk/licenses/commons-lang3-3.13.0.jar.sha1 new file mode 100644 index 0000000000000..d0c2f2486ee1f --- /dev/null +++ b/libs/encryption-sdk/licenses/commons-lang3-3.13.0.jar.sha1 @@ -0,0 +1 @@ +b7263237aa89c1f99b327197c41d0669707a462e \ No newline at end of file diff --git a/libs/encryption-sdk/licenses/commons-lang3-LICENSE.txt b/libs/encryption-sdk/licenses/commons-lang3-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/libs/encryption-sdk/licenses/commons-lang3-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/libs/encryption-sdk/licenses/commons-lang3-NOTICE.txt b/libs/encryption-sdk/licenses/commons-lang3-NOTICE.txt new file mode 100644 index 0000000000000..13a3140897472 --- /dev/null +++ b/libs/encryption-sdk/licenses/commons-lang3-NOTICE.txt @@ -0,0 +1,5 @@ +Apache Commons Lang +Copyright 2001-2019 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). diff --git a/libs/encryption-sdk/licenses/commons-logging-1.2.jar.sha1 b/libs/encryption-sdk/licenses/commons-logging-1.2.jar.sha1 new file mode 100644 index 0000000000000..f40f0242448e8 --- /dev/null +++ b/libs/encryption-sdk/licenses/commons-logging-1.2.jar.sha1 @@ -0,0 +1 @@ +4bfc12adfe4842bf07b657f0369c4cb522955686 \ No newline at end of file diff --git a/libs/encryption-sdk/licenses/commons-logging-LICENSE.txt b/libs/encryption-sdk/licenses/commons-logging-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/libs/encryption-sdk/licenses/commons-logging-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/libs/encryption-sdk/licenses/commons-logging-NOTICE.txt b/libs/encryption-sdk/licenses/commons-logging-NOTICE.txt new file mode 100644 index 0000000000000..1a45218353e87 --- /dev/null +++ b/libs/encryption-sdk/licenses/commons-logging-NOTICE.txt @@ -0,0 +1,5 @@ +Apache Commons Logging +Copyright 2003-2016 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). diff --git a/libs/encryption-sdk/licenses/slf4j-api-1.7.36.jar.sha1 b/libs/encryption-sdk/licenses/slf4j-api-1.7.36.jar.sha1 new file mode 100644 index 0000000000000..77b9917528382 --- /dev/null +++ b/libs/encryption-sdk/licenses/slf4j-api-1.7.36.jar.sha1 @@ -0,0 +1 @@ +6c62681a2f655b49963a5983b8b0950a6120ae14 \ No newline at end of file diff --git a/libs/encryption-sdk/licenses/slf4j-api-LICENSE.txt b/libs/encryption-sdk/licenses/slf4j-api-LICENSE.txt new file mode 100644 index 0000000000000..8fda22f4d72f6 --- /dev/null +++ b/libs/encryption-sdk/licenses/slf4j-api-LICENSE.txt @@ -0,0 +1,21 @@ +Copyright (c) 2004-2014 QOS.ch +All rights reserved. + +Permission is hereby granted, free of charge, to any person obtaining +a copy of this software and associated documentation files (the +"Software"), to deal in the Software without restriction, including +without limitation the rights to use, copy, modify, merge, publish, +distribute, sublicense, and/or sell copies of the Software, and to +permit persons to whom the Software is furnished to do so, subject to +the following conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE +LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION +OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION +WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. diff --git a/libs/encryption-sdk/licenses/slf4j-api-NOTICE.txt b/libs/encryption-sdk/licenses/slf4j-api-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/libs/encryption-sdk/src/forbidden/crypto-signatures.txt b/libs/encryption-sdk/src/forbidden/crypto-signatures.txt new file mode 100644 index 0000000000000..3699186679924 --- /dev/null +++ b/libs/encryption-sdk/src/forbidden/crypto-signatures.txt @@ -0,0 +1,13 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +@defaultMessage use format with Locale +java.lang.String#format(java.lang.String,java.lang.Object[]) \ No newline at end of file diff --git a/libs/encryption-sdk/src/forbidden/crypto-test-signatures.txt b/libs/encryption-sdk/src/forbidden/crypto-test-signatures.txt new file mode 100644 index 0000000000000..3699186679924 --- /dev/null +++ b/libs/encryption-sdk/src/forbidden/crypto-test-signatures.txt @@ -0,0 +1,13 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +@defaultMessage use format with Locale +java.lang.String#format(java.lang.String,java.lang.Object[]) \ No newline at end of file diff --git a/libs/encryption-sdk/src/main/java/org/opensearch/encryption/CryptoManager.java b/libs/encryption-sdk/src/main/java/org/opensearch/encryption/CryptoManager.java new file mode 100644 index 0000000000000..8e1fc8570d552 --- /dev/null +++ b/libs/encryption-sdk/src/main/java/org/opensearch/encryption/CryptoManager.java @@ -0,0 +1,33 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.encryption; + +import org.opensearch.common.crypto.CryptoHandler; +import org.opensearch.common.util.concurrent.RefCounted; + +/** + * Crypto plugin interface used for encryption and decryption. + */ +public interface CryptoManager extends RefCounted { + + /** + * @return key provider type + */ + String type(); + + /** + * @return key provider name + */ + String name(); + + /** + * @return Crypto provider for encrypting or decrypting raw content. + */ + CryptoHandler getCryptoProvider(); +} diff --git a/libs/encryption-sdk/src/main/java/org/opensearch/encryption/CryptoManagerFactory.java b/libs/encryption-sdk/src/main/java/org/opensearch/encryption/CryptoManagerFactory.java new file mode 100644 index 0000000000000..e1dc9291ed1a6 --- /dev/null +++ b/libs/encryption-sdk/src/main/java/org/opensearch/encryption/CryptoManagerFactory.java @@ -0,0 +1,127 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.encryption; + +import org.opensearch.common.crypto.CryptoHandler; +import org.opensearch.common.crypto.MasterKeyProvider; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.concurrent.AbstractRefCounted; +import org.opensearch.encryption.keyprovider.CryptoMasterKey; + +import java.security.SecureRandom; +import java.util.concurrent.TimeUnit; + +import com.amazonaws.encryptionsdk.CryptoAlgorithm; +import com.amazonaws.encryptionsdk.caching.CachingCryptoMaterialsManager; +import com.amazonaws.encryptionsdk.caching.LocalCryptoMaterialsCache; + +public class CryptoManagerFactory { + + private final int dataKeyCacheSize; + private final String algorithm; + + // - Cache TTL and Jitter is used to decide the Crypto Cache TTL. + // - Random number between: (TTL Jitter, TTL - Jitter) + private final long dataKeyCacheTTL; + private static final long dataKeyCacheJitter = TimeUnit.MINUTES.toMillis(30); // - 30 minutes + + public CryptoManagerFactory(String algorithm, TimeValue keyRefreshInterval, int keyCacheSize) { + this.dataKeyCacheSize = keyCacheSize; + validateAndGetAlgorithmId(algorithm); + this.algorithm = algorithm; + dataKeyCacheTTL = keyRefreshInterval.getMillis(); + } + + private String validateAndGetAlgorithmId(String algorithm) { + // Supporting only 256 bit algorithm + switch (algorithm) { + case "ALG_AES_256_GCM_HKDF_SHA512_COMMIT_KEY": + return CryptoAlgorithm.ALG_AES_256_GCM_HKDF_SHA512_COMMIT_KEY.getDataKeyAlgo(); + case "ALG_AES_256_GCM_HKDF_SHA512_COMMIT_KEY_ECDSA_P384": + return CryptoAlgorithm.ALG_AES_256_GCM_HKDF_SHA512_COMMIT_KEY_ECDSA_P384.getDataKeyAlgo(); + default: + throw new IllegalArgumentException("Unsupported algorithm: " + algorithm); + } + } + + public CryptoManager getOrCreateCryptoManager( + MasterKeyProvider keyProvider, + String keyProviderName, + String keyProviderType, + Runnable onClose + ) { + CachingCryptoMaterialsManager materialsManager = createMaterialsManager( + keyProvider, + keyProviderName, + validateAndGetAlgorithmId(algorithm) + ); + CryptoHandler cryptoHandler = createCryptoProvider(algorithm, materialsManager, keyProvider); + return createCryptoManager(cryptoHandler, keyProviderType, keyProviderName, onClose); + } + + // package private for tests + CryptoHandler createCryptoProvider( + String algorithm, + CachingCryptoMaterialsManager materialsManager, + MasterKeyProvider masterKeyProvider + ) { + return new NoOpCryptoHandler(); + } + + // Package private for tests + CachingCryptoMaterialsManager createMaterialsManager(MasterKeyProvider masterKeyProvider, String keyProviderName, String algorithm) { + SecureRandom r = new SecureRandom(); + long low = dataKeyCacheTTL - dataKeyCacheJitter; + long high = dataKeyCacheTTL + dataKeyCacheJitter; + long masterKeyCacheTTL = r.nextInt((int) (high - low)) + low; + + CryptoMasterKey cryptoMasterKey = new CryptoMasterKey(masterKeyProvider, keyProviderName, algorithm); + return CachingCryptoMaterialsManager.newBuilder() + .withMasterKeyProvider(cryptoMasterKey) + .withCache(new LocalCryptoMaterialsCache(dataKeyCacheSize)) + .withMaxAge(masterKeyCacheTTL, TimeUnit.MILLISECONDS) + .build(); + } + + // package private for tests + CryptoManager createCryptoManager( + CryptoHandler cryptoHandler, + String keyProviderType, + String keyProviderName, + Runnable onClose + ) { + return new CryptoManagerImpl(keyProviderName, keyProviderType) { + @Override + protected void closeInternal() { + onClose.run(); + } + + @Override + public String type() { + return keyProviderType; + } + + @Override + public String name() { + return keyProviderName; + } + + @Override + public CryptoHandler getCryptoProvider() { + return cryptoHandler; + } + }; + } + + private static abstract class CryptoManagerImpl extends AbstractRefCounted implements CryptoManager { + public CryptoManagerImpl(String keyProviderName, String keyProviderType) { + super(keyProviderName + "-" + keyProviderType); + } + } +} diff --git a/libs/encryption-sdk/src/main/java/org/opensearch/encryption/NoOpCryptoHandler.java b/libs/encryption-sdk/src/main/java/org/opensearch/encryption/NoOpCryptoHandler.java new file mode 100644 index 0000000000000..d6b23ed08c6b0 --- /dev/null +++ b/libs/encryption-sdk/src/main/java/org/opensearch/encryption/NoOpCryptoHandler.java @@ -0,0 +1,128 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.encryption; + +import org.opensearch.common.crypto.CryptoHandler; +import org.opensearch.common.crypto.DecryptedRangedStreamProvider; +import org.opensearch.common.crypto.EncryptedHeaderContentSupplier; +import org.opensearch.common.io.InputStreamContainer; + +import java.io.IOException; +import java.io.InputStream; + +public class NoOpCryptoHandler implements CryptoHandler { + + /** + * No op - Initialises metadata store used in encryption. + * @return crypto metadata object constructed with encryption metadata like data key pair, encryption algorithm, etc. + */ + public Object initEncryptionMetadata() { + return new Object(); + } + + /** + * No op content size adjustment of length of a partial content used in partial encryption. + * + * @param cryptoContextObj stateful object for a request consisting of materials required in encryption. + * @param streamSize Size of the stream to be adjusted. + * @return Adjusted size of the stream. + */ + public long adjustContentSizeForPartialEncryption(Object cryptoContextObj, long streamSize) { + return streamSize; + } + + /** + * No op - Estimate length of the encrypted stream. + * + * @param cryptoMetadataObj crypto metadata instance + * @param contentLength Size of the raw content + * @return Calculated size of the encrypted stream for the provided raw stream. + */ + public long estimateEncryptedLengthOfEntireContent(Object cryptoMetadataObj, long contentLength) { + return contentLength; + } + + /** + * No op length estimation for a given content length. + * + * @param cryptoMetadataObj crypto metadata instance + * @param contentLength Size of the encrypted content + * @return Calculated size of the encrypted stream for the provided raw stream. + */ + public long estimateDecryptedLength(Object cryptoMetadataObj, long contentLength) { + return contentLength; + } + + /** + * No op encrypting stream wrapper. + * + * @param cryptoContextObj consists encryption metadata. + * @param stream Raw InputStream to encrypt + * @return encrypting stream wrapped around raw InputStream. + */ + public InputStreamContainer createEncryptingStream(Object cryptoContextObj, InputStreamContainer stream) { + return stream; + } + + /** + * No op encrypting stream provider for a part of content. + * + * @param cryptoContextObj stateful object for a request consisting of materials required in encryption. + * @param stream raw stream for which encrypted stream has to be created. + * @param totalStreams Number of streams being used for the entire content. + * @param streamIdx Index of the current stream. + * @return Encrypted stream for the provided raw stream. + */ + public InputStreamContainer createEncryptingStreamOfPart( + Object cryptoContextObj, + InputStreamContainer stream, + int totalStreams, + int streamIdx + ) { + return stream; + } + + /** + * + * @param encryptedHeaderContentSupplier Supplier used to fetch bytes from source for header creation + * @return parsed encryption metadata object + * @throws IOException if content fetch for header creation fails + */ + public Object loadEncryptionMetadata(EncryptedHeaderContentSupplier encryptedHeaderContentSupplier) throws IOException { + return new Object(); + } + + /** + * No op decrypting stream provider. + * + * @param encryptedStream to be decrypted. + * @return Decrypting wrapper stream + */ + public InputStream createDecryptingStream(InputStream encryptedStream) { + return encryptedStream; + } + + /** + * No Op decrypted stream range provider + * + * @param cryptoContext crypto metadata instance consisting of encryption metadata used in encryption. + * @param startPosOfRawContent starting position in the raw/decrypted content + * @param endPosOfRawContent ending position in the raw/decrypted content + * @return stream provider for decrypted stream for the specified range of content including adjusted range + */ + public DecryptedRangedStreamProvider createDecryptingStreamOfRange( + Object cryptoContext, + long startPosOfRawContent, + long endPosOfRawContent + ) { + long[] range = { startPosOfRawContent, endPosOfRawContent }; + return new DecryptedRangedStreamProvider(range, (encryptedStream) -> encryptedStream); + } + +} diff --git a/libs/encryption-sdk/src/main/java/org/opensearch/encryption/TrimmingStream.java b/libs/encryption-sdk/src/main/java/org/opensearch/encryption/TrimmingStream.java new file mode 100644 index 0000000000000..d6640bbe5e79e --- /dev/null +++ b/libs/encryption-sdk/src/main/java/org/opensearch/encryption/TrimmingStream.java @@ -0,0 +1,119 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.encryption; + +import java.io.IOException; +import java.io.InputStream; + +/** + * Trims content from a given source range to a target range. + */ +public class TrimmingStream extends InputStream { + + private final long sourceStart; + private final long sourceEnd; + private final long targetStart; + private final long targetEnd; + private final InputStream in; + + private long offsetFromStart = 0; + + public TrimmingStream(long sourceStart, long sourceEnd, long targetStart, long targetEnd, InputStream in) { + if (sourceStart < 0 + || targetStart < 0 + || targetEnd < 0 + || targetStart > targetEnd + || sourceStart > targetStart + || sourceEnd < targetEnd) { + throw new IllegalArgumentException("Invalid arguments to the bounded stream"); + } + + this.sourceStart = sourceStart; + this.sourceEnd = sourceEnd; + this.targetStart = targetStart; + this.targetEnd = targetEnd; + this.in = in; + } + + private void skipBytesOutsideBounds() throws IOException { + long relativeOffset = offsetFromStart + sourceStart; + + if (relativeOffset < targetStart) { + skipBytes(relativeOffset, targetStart); + } + + if (relativeOffset > targetEnd) { + skipBytes(relativeOffset, sourceEnd + 1); + } + } + + private void skipBytes(long offset, long end) throws IOException { + long bytesToSkip = end - offset; + while (bytesToSkip > 0) { + long skipped = skip(bytesToSkip); + if (skipped <= 0) { + // End of stream or unable to skip further + break; + } + bytesToSkip -= skipped; + } + } + + @Override + public int read() throws IOException { + skipBytesOutsideBounds(); + if (offsetFromStart + sourceStart > targetEnd) { + return -1; + } + int b = in.read(); + if (b != -1) { + offsetFromStart++; + } + // This call is made again to ensure that source stream is fully consumed when it reaches end of target range. + skipBytesOutsideBounds(); + return b; + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + skipBytesOutsideBounds(); + if (offsetFromStart + sourceStart > targetEnd) { + return -1; + } + len = (int) Math.min(len, targetEnd - offsetFromStart - sourceStart + 1); + int bytesRead = in.read(b, off, len); + if (bytesRead != -1) { + offsetFromStart += bytesRead; + } + // This call is made again to ensure that source stream is fully consumed when it reaches end of target range. + skipBytesOutsideBounds(); + return bytesRead; + } + + /** + * Skips specified number of bytes of input. + * @param n the number of bytes to skip + * @return the actual number of bytes skipped + * @throws IOException if an I/O error has occurred + */ + public long skip(long n) throws IOException { + byte[] buf = new byte[512]; + long total = 0; + while (total < n) { + long len = n - total; + len = in.read(buf, 0, len < buf.length ? (int) len : buf.length); + if (len == -1) { + return total; + } + offsetFromStart += len; + total += len; + } + return total; + } +} diff --git a/libs/encryption-sdk/src/main/java/org/opensearch/encryption/keyprovider/CryptoMasterKey.java b/libs/encryption-sdk/src/main/java/org/opensearch/encryption/keyprovider/CryptoMasterKey.java new file mode 100644 index 0000000000000..6f014c9b4d99b --- /dev/null +++ b/libs/encryption-sdk/src/main/java/org/opensearch/encryption/keyprovider/CryptoMasterKey.java @@ -0,0 +1,84 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ +package org.opensearch.encryption.keyprovider; + +import org.opensearch.common.crypto.DataKeyPair; +import org.opensearch.common.crypto.MasterKeyProvider; + +import javax.crypto.spec.SecretKeySpec; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Collection; +import java.util.Map; + +import com.amazonaws.encryptionsdk.CryptoAlgorithm; +import com.amazonaws.encryptionsdk.DataKey; +import com.amazonaws.encryptionsdk.EncryptedDataKey; +import com.amazonaws.encryptionsdk.MasterKey; +import com.amazonaws.encryptionsdk.exception.AwsCryptoException; + +public class CryptoMasterKey extends MasterKey implements Closeable { + private final MasterKeyProvider keyProvider; + private final String keyProviderName; + private final String cryptoAlgorithm; + + public CryptoMasterKey(MasterKeyProvider keyProvider, String keyProviderName, String cryptoAlgorithm) { + this.keyProvider = keyProvider; + this.keyProviderName = keyProviderName; + this.cryptoAlgorithm = cryptoAlgorithm; + } + + @Override + public String getProviderId() { + return keyProviderName; + } + + @Override + public String getKeyId() { + return keyProvider.getKeyId(); + } + + @Override + public DataKey generateDataKey(CryptoAlgorithm algorithm, Map encryptionContext) { + DataKeyPair dataKeyPairResponse = keyProvider.generateDataPair(); + final SecretKeySpec key = new SecretKeySpec(dataKeyPairResponse.getRawKey(), cryptoAlgorithm); + return new DataKey<>(key, dataKeyPairResponse.getEncryptedKey(), getKeyId().getBytes(StandardCharsets.UTF_8), this); + } + + @Override + public DataKey encryptDataKey(CryptoAlgorithm algorithm, Map encryptionContext, DataKey dataKey) { + throw new UnsupportedOperationException("Multiple data-key encryption is not supported."); + } + + @Override + public DataKey decryptDataKey( + CryptoAlgorithm algorithm, + Collection encryptedDataKeys, + Map encryptionContext + ) throws AwsCryptoException { + if (encryptedDataKeys == null || encryptedDataKeys.isEmpty()) { + throw new IllegalArgumentException("No encrypted data key passed for decryption."); + } + EncryptedDataKey encryptedDataKey = encryptedDataKeys.iterator().next(); + final String keyId = new String(encryptedDataKey.getProviderInformation(), StandardCharsets.UTF_8); + if (!this.getKeyId().equals(keyId)) { + throw new IllegalArgumentException("Invalid provider info present in encrypted key."); + } + + byte[] encryptedKey = encryptedDataKey.getEncryptedDataKey(); + byte[] rawKey = keyProvider.decryptKey(encryptedKey); + return new DataKey<>(new SecretKeySpec(rawKey, cryptoAlgorithm), encryptedKey, keyId.getBytes(StandardCharsets.UTF_8), this); + } + + @Override + public void close() throws IOException { + keyProvider.close(); + } +} diff --git a/libs/encryption-sdk/src/main/java/org/opensearch/encryption/keyprovider/package-info.java b/libs/encryption-sdk/src/main/java/org/opensearch/encryption/keyprovider/package-info.java new file mode 100644 index 0000000000000..611b095a54250 --- /dev/null +++ b/libs/encryption-sdk/src/main/java/org/opensearch/encryption/keyprovider/package-info.java @@ -0,0 +1,12 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +/** + * Key provider package specific to encryption sdk + */ +package org.opensearch.encryption.keyprovider; diff --git a/libs/encryption-sdk/src/main/java/org/opensearch/encryption/package-info.java b/libs/encryption-sdk/src/main/java/org/opensearch/encryption/package-info.java new file mode 100644 index 0000000000000..1fa008797ce87 --- /dev/null +++ b/libs/encryption-sdk/src/main/java/org/opensearch/encryption/package-info.java @@ -0,0 +1,12 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +/** + * Crypto plugin to for encryption and decryption use cases. + */ +package org.opensearch.encryption; diff --git a/libs/encryption-sdk/src/test/java/org/opensearch/encryption/CryptoManagerFactoryTests.java b/libs/encryption-sdk/src/test/java/org/opensearch/encryption/CryptoManagerFactoryTests.java new file mode 100644 index 0000000000000..fb5c477232bc4 --- /dev/null +++ b/libs/encryption-sdk/src/test/java/org/opensearch/encryption/CryptoManagerFactoryTests.java @@ -0,0 +1,92 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.encryption; + +import org.opensearch.common.crypto.CryptoHandler; +import org.opensearch.common.crypto.MasterKeyProvider; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; + +import java.util.Collections; + +import com.amazonaws.encryptionsdk.caching.CachingCryptoMaterialsManager; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class CryptoManagerFactoryTests extends OpenSearchTestCase { + + private CryptoManagerFactory cryptoManagerFactory; + + @Before + public void setup() { + cryptoManagerFactory = new CryptoManagerFactory( + "ALG_AES_256_GCM_HKDF_SHA512_COMMIT_KEY_ECDSA_P384", + TimeValue.timeValueDays(2), + 10 + ); + } + + public void testGetOrCreateCryptoManager() { + MasterKeyProvider mockKeyProvider = mock(MasterKeyProvider.class); + when(mockKeyProvider.getEncryptionContext()).thenReturn(Collections.emptyMap()); + + CryptoManager cryptoManager = cryptoManagerFactory.getOrCreateCryptoManager( + mockKeyProvider, + "keyProviderName", + "keyProviderType", + () -> {} + ); + + assertNotNull(cryptoManager); + } + + public void testCreateCryptoProvider() { + CachingCryptoMaterialsManager mockMaterialsManager = mock(CachingCryptoMaterialsManager.class); + MasterKeyProvider mockKeyProvider = mock(MasterKeyProvider.class); + when(mockKeyProvider.getEncryptionContext()).thenReturn(Collections.emptyMap()); + + CryptoHandler cryptoHandler = cryptoManagerFactory.createCryptoProvider( + "ALG_AES_256_GCM_HKDF_SHA512_COMMIT_KEY_ECDSA_P384", + mockMaterialsManager, + mockKeyProvider + ); + + assertNotNull(cryptoHandler); + } + + public void testCreateMaterialsManager() { + MasterKeyProvider mockKeyProvider = mock(MasterKeyProvider.class); + when(mockKeyProvider.getEncryptionContext()).thenReturn(Collections.emptyMap()); + + CachingCryptoMaterialsManager materialsManager = cryptoManagerFactory.createMaterialsManager( + mockKeyProvider, + "keyProviderName", + "ALG_AES_256_GCM_HKDF_SHA512_COMMIT_KEY_ECDSA_P384" + ); + + assertNotNull(materialsManager); + } + + public void testCreateCryptoManager() { + CryptoHandler mockCryptoHandler = mock(CryptoHandler.class); + CryptoManager cryptoManager = cryptoManagerFactory.createCryptoManager( + mockCryptoHandler, + "keyProviderName", + "keyProviderType", + null + ); + assertNotNull(cryptoManager); + } + + public void testUnsupportedAlgorithm() { + expectThrows(IllegalArgumentException.class, () -> new CryptoManagerFactory("Unsupported_algo", TimeValue.timeValueDays(2), 10)); + } +} diff --git a/libs/encryption-sdk/src/test/java/org/opensearch/encryption/MockKeyProvider.java b/libs/encryption-sdk/src/test/java/org/opensearch/encryption/MockKeyProvider.java new file mode 100644 index 0000000000000..a5e74534ef32b --- /dev/null +++ b/libs/encryption-sdk/src/test/java/org/opensearch/encryption/MockKeyProvider.java @@ -0,0 +1,109 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ +package org.opensearch.encryption; + +import javax.crypto.spec.SecretKeySpec; + +import java.io.BufferedReader; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; +import java.util.Collection; +import java.util.Map; + +import com.amazonaws.encryptionsdk.CryptoAlgorithm; +import com.amazonaws.encryptionsdk.DataKey; +import com.amazonaws.encryptionsdk.MasterKey; +import com.amazonaws.encryptionsdk.exception.AwsCryptoException; +import com.amazonaws.encryptionsdk.exception.UnsupportedProviderException; + +@SuppressWarnings({ "rawtypes", "unchecked" }) +public class MockKeyProvider extends MasterKey { + + private static final String keyId = "test-key-id"; + + public static byte[] loadFile(String file) { + byte[] content; + try { + InputStream in = MockKeyProvider.class.getResourceAsStream(file); + StringBuilder stringBuilder = new StringBuilder(); + BufferedReader bufferedReader = new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8)); + for (String line; (line = bufferedReader.readLine()) != null;) { + stringBuilder.append(line); + } + content = stringBuilder.toString().getBytes(StandardCharsets.UTF_8); + } catch (Exception e) { + throw new IllegalArgumentException("File " + file + " cannot be read correctly."); + } + String text = new String(content, StandardCharsets.UTF_8); + + String[] byteValues = text.substring(1, text.length() - 1).split(","); + byte[] bytes = new byte[byteValues.length]; + + for (int i = 0, len = bytes.length; i < len; i++) { + bytes[i] = Byte.parseByte(byteValues[i].trim()); + } + + return bytes; + } + + private static final byte[] rawKey = loadFile("/raw_key"); + private static final byte[] encryptedKey = loadFile("/encrypted_key"); + + @Override + public String getProviderId() { + return "sample-provider-id"; + } + + @Override + public String getKeyId() { + return "Sample-key-id"; + } + + @Override + public DataKey encryptDataKey(CryptoAlgorithm algorithm, Map encryptionContext, DataKey dataKey) { + throw new UnsupportedOperationException("Multiple data-key encryption is not supported."); + } + + @Override + public DataKey generateDataKey(CryptoAlgorithm algorithm, Map encryptionContext) { + final SecretKeySpec key = new SecretKeySpec(rawKey, algorithm.getDataKeyAlgo()); + return new DataKey(key, encryptedKey, getKeyId().getBytes(StandardCharsets.UTF_8), this); + } + + @Override + public DataKey decryptDataKey(CryptoAlgorithm algorithm, Collection collection, Map encryptionContext) + throws UnsupportedProviderException, AwsCryptoException { + return new DataKey<>( + new SecretKeySpec(rawKey, algorithm.getDataKeyAlgo()), + encryptedKey, + keyId.getBytes(StandardCharsets.UTF_8), + this + ); + } + + static class DataKeyPair { + private final byte[] rawKey; + private final byte[] encryptedKey; + + public DataKeyPair(byte[] rawKey, byte[] encryptedKey) { + this.rawKey = rawKey; + this.encryptedKey = encryptedKey; + } + + public byte[] getRawKey() { + return this.rawKey; + } + + public byte[] getEncryptedKey() { + return this.encryptedKey; + } + } + +} diff --git a/libs/encryption-sdk/src/test/java/org/opensearch/encryption/NoOpCryptoHandlerTests.java b/libs/encryption-sdk/src/test/java/org/opensearch/encryption/NoOpCryptoHandlerTests.java new file mode 100644 index 0000000000000..5e3836fd10988 --- /dev/null +++ b/libs/encryption-sdk/src/test/java/org/opensearch/encryption/NoOpCryptoHandlerTests.java @@ -0,0 +1,95 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.encryption; + +import org.opensearch.common.crypto.DecryptedRangedStreamProvider; +import org.opensearch.common.crypto.EncryptedHeaderContentSupplier; +import org.opensearch.common.io.InputStreamContainer; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; + +public class NoOpCryptoHandlerTests extends OpenSearchTestCase { + + public void testInitEncryptionMetadata() { + NoOpCryptoHandler cryptoProvider = new NoOpCryptoHandler(); + Object encryptionMetadata = cryptoProvider.initEncryptionMetadata(); + assertNotNull(encryptionMetadata); + } + + public void testAdjustContentSizeForPartialEncryption() { + NoOpCryptoHandler cryptoProvider = new NoOpCryptoHandler(); + long originalSize = 1000L; + long adjustedSize = cryptoProvider.adjustContentSizeForPartialEncryption(new Object(), originalSize); + assertEquals(originalSize, adjustedSize); + } + + public void testEstimateEncryptedLengthOfEntireContent() { + NoOpCryptoHandler cryptoProvider = new NoOpCryptoHandler(); + long originalSize = 2000L; + long estimatedSize = cryptoProvider.estimateEncryptedLengthOfEntireContent(new Object(), originalSize); + assertEquals(originalSize, estimatedSize); + } + + public void testEstimateDecryptedLength() { + NoOpCryptoHandler cryptoProvider = new NoOpCryptoHandler(); + long originalSize = 1500L; + long estimatedSize = cryptoProvider.estimateDecryptedLength(new Object(), originalSize); + assertEquals(originalSize, estimatedSize); + } + + public void testCreateEncryptingStream() { + NoOpCryptoHandler cryptoProvider = new NoOpCryptoHandler(); + InputStreamContainer inputStream = randomStream(); + InputStreamContainer encryptedStream = cryptoProvider.createEncryptingStream(new Object(), inputStream); + assertEquals(inputStream, encryptedStream); + } + + public void testCreateEncryptingStreamOfPart() { + NoOpCryptoHandler cryptoProvider = new NoOpCryptoHandler(); + InputStreamContainer inputStream = randomStream(); + InputStreamContainer encryptedStream = cryptoProvider.createEncryptingStreamOfPart(new Object(), inputStream, 2, 1); + assertEquals(inputStream, encryptedStream); + } + + private InputStreamContainer randomStream() { + byte[] bytes = randomAlphaOfLength(10).getBytes(); + ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(bytes); + int offset = randomIntBetween(0, bytes.length - 1); + return new InputStreamContainer(byteArrayInputStream, bytes.length, offset); + } + + public void testLoadEncryptionMetadata() throws IOException { + NoOpCryptoHandler cryptoProvider = new NoOpCryptoHandler(); + EncryptedHeaderContentSupplier supplier = (start, length) -> { throw new UnsupportedOperationException("Not implemented"); }; + Object encryptionMetadata = cryptoProvider.loadEncryptionMetadata(supplier); + assertNotNull(encryptionMetadata); + } + + public void testCreateDecryptingStream() { + NoOpCryptoHandler cryptoProvider = new NoOpCryptoHandler(); + InputStream encryptedStream = randomStream().getInputStream(); + InputStream decryptedStream = cryptoProvider.createDecryptingStream(encryptedStream); + assertEquals(encryptedStream, decryptedStream); + } + + public void testCreateDecryptingStreamOfRange() { + NoOpCryptoHandler cryptoProvider = new NoOpCryptoHandler(); + Object cryptoContext = new Object(); + long startPos = 0L; + long endPos = 100L; + DecryptedRangedStreamProvider streamProvider = cryptoProvider.createDecryptingStreamOfRange(cryptoContext, startPos, endPos); + assertNotNull(streamProvider); + InputStream stream = randomStream().getInputStream(); + InputStream decryptedStream = streamProvider.getDecryptedStreamProvider().apply(stream); // Replace with your encrypted input stream + assertEquals(stream, decryptedStream); + } +} diff --git a/libs/encryption-sdk/src/test/java/org/opensearch/encryption/TrimmingStreamTests.java b/libs/encryption-sdk/src/test/java/org/opensearch/encryption/TrimmingStreamTests.java new file mode 100644 index 0000000000000..f0d957d81e1e1 --- /dev/null +++ b/libs/encryption-sdk/src/test/java/org/opensearch/encryption/TrimmingStreamTests.java @@ -0,0 +1,125 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.encryption; + +import org.opensearch.test.OpenSearchTestCase; + +import java.io.ByteArrayInputStream; +import java.io.IOException; + +public class TrimmingStreamTests extends OpenSearchTestCase { + + static class ReadCountInputStreamTest extends ByteArrayInputStream { + + public ReadCountInputStreamTest(byte[] buf) { + super(buf); + } + + public int getPos() { + return pos; + } + } + + public void testReadInRange() throws IOException { + byte[] data = generateRandomData(100); + ReadCountInputStreamTest input = new ReadCountInputStreamTest(data); + + long sourceStart = generateRandomValue(0, 80); + long sourceEnd = generateRandomValue(sourceStart, 99); + long targetStart = generateRandomValue(sourceStart, sourceEnd); + long targetEnd = generateRandomValue(targetStart, sourceEnd); + + TrimmingStream trimmingStream = new TrimmingStream(sourceStart, sourceEnd, targetStart, targetEnd, input); + + byte[] result = new byte[(int) (sourceEnd - sourceStart + 1)]; + int bytesRead = trimmingStream.read(result, 0, result.length); + + long expectedBytesRead = targetEnd - targetStart + 1; + assertEquals(expectedBytesRead, bytesRead); + assertEquals(sourceEnd - sourceStart + 1, input.getPos()); + } + + public void testReadOutsideRange() throws IOException { + byte[] data = generateRandomData(100); + ReadCountInputStreamTest input = new ReadCountInputStreamTest(data); + + long sourceStart = generateRandomValue(0, 80); + long sourceEnd = generateRandomValue(sourceStart, 99); + long targetStart = generateRandomValue(sourceStart, sourceEnd); + long targetEnd = generateRandomValue(targetStart, sourceEnd); + + TrimmingStream trimmingStream = new TrimmingStream(sourceStart, sourceEnd, targetStart, targetEnd, input); + + byte[] result = new byte[(int) (targetEnd - targetStart + 1)]; + int bytesRead = trimmingStream.read(result, 0, result.length); + + long expectedBytesRead = targetEnd - targetStart + 1; + assertEquals(expectedBytesRead, bytesRead); + assertEquals(sourceEnd - sourceStart + 1, input.getPos()); + + // Try to read more bytes, should return -1 (end of stream) + int additionalBytesRead = trimmingStream.read(result, 0, 50); + assertEquals(-1, additionalBytesRead); + assertEquals(sourceEnd - sourceStart + 1, input.getPos()); + } + + public void testSingleByteReadInRange() throws IOException { + byte[] data = generateRandomData(100); + ReadCountInputStreamTest input = new ReadCountInputStreamTest(data); + + long sourceStart = generateRandomValue(0, 80); + long sourceEnd = generateRandomValue(sourceStart, 99); + long targetStart = generateRandomValue(sourceStart, sourceEnd); + long targetEnd = generateRandomValue(targetStart, sourceEnd); + + TrimmingStream trimmingStream = new TrimmingStream(sourceStart, sourceEnd, targetStart, targetEnd, input); + + int bytesRead = 0; + int value; + while ((value = trimmingStream.read()) != -1) { + bytesRead++; + } + + long expectedBytesRead = targetEnd - targetStart + 1; + assertEquals(expectedBytesRead, bytesRead); + assertEquals(sourceEnd - sourceStart + 1, input.getPos()); + } + + public void testInvalidInputs() { + assertThrows(IllegalArgumentException.class, () -> new TrimmingStream(-10, 60, 20, 40, new ByteArrayInputStream(new byte[100]))); + assertThrows(IllegalArgumentException.class, () -> new TrimmingStream(10, 60, 40, 20, new ByteArrayInputStream(new byte[100]))); + } + + public void testSourceSameAsTarget() throws IOException { + byte[] data = generateRandomData(100); + ReadCountInputStreamTest input = new ReadCountInputStreamTest(data); + + long sourceStart = generateRandomValue(0, 80); + long sourceEnd = generateRandomValue(sourceStart, 99); + TrimmingStream trimmingStream = new TrimmingStream(sourceStart, sourceEnd, sourceStart, sourceEnd, input); + + byte[] result = new byte[(int) (sourceEnd - sourceStart + 1)]; + int bytesRead = trimmingStream.read(result, 0, result.length); + + assertEquals(sourceEnd - sourceStart + 1, bytesRead); + assertEquals(sourceEnd - sourceStart + 1, input.getPos()); + } + + private byte[] generateRandomData(int length) { + byte[] data = new byte[length]; + for (int i = 0; i < length; i++) { + data[i] = (byte) (Math.random() * 256 - 128); + } + return data; + } + + private long generateRandomValue(long min, long max) { + return min + (long) (Math.random() * (max - min + 1)); + } +} diff --git a/libs/encryption-sdk/src/test/resources/encrypted_key b/libs/encryption-sdk/src/test/resources/encrypted_key new file mode 100644 index 0000000000000..da4e503581585 --- /dev/null +++ b/libs/encryption-sdk/src/test/resources/encrypted_key @@ -0,0 +1 @@ +[1, 2, 1, 0, 120, -96, 18, 71, -6, 90, -126, -39, -16, 94, -113, -46, 71, 85, 35, -66, -117, -108, -59, 88, -81, 64, -118, -74, -102, 50, 103, 16, -76, 23, 19, 20, 67, 1, -11, 55, -3, 32, -89, -16, 1, -40, 59, 76, -2, -61, -49, -97, 34, 14, 0, 0, 0, 126, 48, 124, 6, 9, 42, -122, 72, -122, -9, 13, 1, 7, 6, -96, 111, 48, 109, 2, 1, 0, 48, 104, 6, 9, 42, -122, 72, -122, -9, 13, 1, 7, 1, 48, 30, 6, 9, 96, -122, 72, 1, 101, 3, 4, 1, 46, 48, 17, 4, 12, -63, 67, 37, -51, 85, 75, 7, -64, -78, 52, 102, 26, 2, 1, 16, -128, 59, -98, -123, 100, 125, -37, 102, -87, -71, 74, 68, 54, 56, -32, 77, 127, -86, -125, -17, 45, 75, -98, 54, -52, -15, -56, -47, -88, -12, -128, 113, -5, -18, -14, 127, 114, -9, 47, -112, -38, 39, 2, -89, 117, 64, -2, 47, -81, 52, 27, -118, 37, 79, -64, 58, -3, 10, -115, 122, 124] \ No newline at end of file diff --git a/libs/encryption-sdk/src/test/resources/raw_content_for_crypto_test b/libs/encryption-sdk/src/test/resources/raw_content_for_crypto_test new file mode 100644 index 0000000000000..c93b6161ac8d6 --- /dev/null +++ b/libs/encryption-sdk/src/test/resources/raw_content_for_crypto_test @@ -0,0 +1,25 @@ +ewogICJmaWxlSW5mb3MiOiBbCiAgICB7CiAgICAgICJuYW1lIjogIl80LmZubSIsCiAgICAgICJyZW1vdGVfc +GF0aCI6ICIyYzYwMzNmNmZlZTY0NTY1YTU3YzQzZWVmZThmY2QzMS9kdW1teS1jb2xsZWN0aW9uMi9kMDRmYz +AyZi0wMDQ0LTRhYmYtYjgzMy0xMGE0YTA5M2VkNTcvMC8wL2luZGljZXMvMSIsCiAgICAgICJzaXplIjogOTQz +CiAgICB9LAogICAgewogICAgICAibmFtZSI6ICJfMl9MdWNlbmU4MF8wLmR2ZCIsCiAgICAgICJyZW1vdGVfcGF +0aCI6ICIyYzYwMzNmNmZlZTY0NTY1YTU3YzQzZWVmZThmY2QzMS9kdW1teS1jb2xsZWN0aW9uMi9kMDRmYzAyZi0wMDQ0LTRhYmYtYjg +zMy0xMGE0YTA5M2VkNTcvMC8wL2luZGljZXMvMSIsCiAgICAgICJzaXplIjogMzU1CiAgICB9CiAgXQp9 +ewogICJja3BfZmlsZSI6IHsKICAgICJuYW1lIjogInRyYW5zbG9nLTguY2twIiwKICAgICJyZW1vdGVfcGF0aCI6ICIyYz +YwMzNmNmZlZTY0NTY1YTU3YzQzZWVmZThmY2QzMS9kdW1teS1jb2xsZWN0aW9uMi9kMDRmYzAyZi0wMDQ0LTRhYmYtYjgzMy0 +xMGE0YTA5M2VkNTcvMC8wL3RyYW5zbG9nLzEiLAogICAgInNpemUiOiAwCiAgfSwKICAidGxvZ192ZXJzaW9uIjogewogICAgIjg +iOiAiMmM2MDMzZjZmZWU2NDU2NWE1N2M0M2VlZmU4ZmNkMzEvZHVtbXktY29sbGVjdGlvbjIvZDA0ZmMwMmYtMDA0NC00YWJmLWI4MzMtMT +BhNGEwOTNlZDU3LzAvMC90cmFuc2xvZy8xIgogIH0KfQ== +ewogICJmaWxlSW5mb3MiOiBbCiAgICB7CiAgICAgICJuYW1lIjogIl80LmZubSIsCiAgICAgICJyZW1vdGVfcGF0aCI6ICIyYzYwMzNmNmZl +ZTY0NTY1YTU3YzQzZWVmZThmY2QzMS9kdW1teS1jb2xsZWN0aW9uMi9kMDRmYzAyZi0wMDQ0LTRhYmYtYjgzMy0xMGE0YTA5M2VkNTcvMC8wL2luZG +ljZXMvMSIsCiAgICAgICJzaXplIjogOTQzCiAgICB9LAogICAgewogICAgICAibmFtZSI6ICJfNC5mZHQiLAogICAgICAicmVtb3RlX3BhdGgiOiAi +MmM2MDMzZjZmZWU2NDU2NWE1N2M0M2VlZmU4ZmNkMzEvZHVtbXktY29sbGVjdGlvbjIvZDA0ZmMwMmYtMDA0NC00YWJmLWI4MzMtMTBhNGEwOTNlZDU3 +LzAvMC9pbmRpY2VzLzEiLAogICAgICAic2l6ZSI6IDQ1MTMKICAgIH0sCiAgICB7CiAgICAgICJuYW1lIjogInNlZ21lbnRzX2MiLAogICAgICAicmVtb3R +lX3BhdGgiOiAiMmM2MDMzZjZmZWU2NDU2NWE1N2M0M2VlZmU4ZmNkMzEvZHVtbXktY29sbGVjdGlvbjIvZDA0ZmMwMmYtMDA0NC00YWJmLWI4MzM +tMTBhNGEwOTNlZDU3LzAvMC9pbmRpY2VzLzEiLAogICAgICAic2l6ZSI6IDM1NQogICAgfQogIF0KfQ== +ewogICJja3BfZmlsZSI6IHsKICAgICJuYW1lIjogInRyYW5zbG9nLTcuY2twIiwKICAgICJyZW1vdGVfcGF0aCI6ICIyYzYwMzNmNmZlZ +TY0NTY1YTU3YzQzZWVmZThmY2QzMS9kdW1teS1jb2xsZWN0aW9uMi9kMDRmYzAyZi0wMDQ0LTRhYmYtYjgzMy0xMGE0YTA5M2VkNTcvMC8wL3RyY +W5zbG9nLzEiLAogICAgInNpemUiOiAwCiAgfSwKICAidGxvZ192ZXJzaW9uIjogewogICAgIjYiOiAiMmM2MDMzZjZmZWU2NDU2NWE1N2M0M2VlZ +mU4ZmNkMzEvZHVtbXktY29sbGVjdGlvbjIvZDA0ZmMwMmYtMDA0NC00YWJmLWI4MzMtMTBhNGEwOTNlZDU3LzAvMC90cmFuc2xvZy8xIiwKICAgICI3Ijo +gIjJjNjAzM2Y2ZmVlNjQ1NjVhNTdjNDNlZWZlOGZjZDMxL2R1bW15LWNvbGxlY3Rpb24yL2QwNGZjMDJmLTAwNDQtNGFiZi1iODMzLTEwYTRhMDkzZW +Q1Ny8wLzAvdHJhbnNsb2cvMSIKICB9Cn0= + diff --git a/libs/encryption-sdk/src/test/resources/raw_key b/libs/encryption-sdk/src/test/resources/raw_key new file mode 100644 index 0000000000000..3c4f8b54cbb6a --- /dev/null +++ b/libs/encryption-sdk/src/test/resources/raw_key @@ -0,0 +1 @@ +[57, 59, -48, -8, -44, 9, -78, 16, 106, -80, 66, -41, 66, 43, -88, 7, 47, -23, -16, -43, 99, 104, -8, -74, 46, -117, -111, -41, -39, -69, 5, 117] \ No newline at end of file diff --git a/plugins/ingest-attachment/build.gradle b/plugins/ingest-attachment/build.gradle index 17213b96a25f3..2fe5b93704585 100644 --- a/plugins/ingest-attachment/build.gradle +++ b/plugins/ingest-attachment/build.gradle @@ -89,7 +89,7 @@ dependencies { api "org.apache.james:apache-mime4j-core:${versions.mime4j}" api "org.apache.james:apache-mime4j-dom:${versions.mime4j}" // EPUB books - api 'org.apache.commons:commons-lang3:3.13.0' + api "org.apache.commons:commons-lang3:${versions.commonslang}" // Microsoft Word files with visio diagrams api 'org.apache.commons:commons-math3:3.6.1' // POIs dependency diff --git a/plugins/repository-azure/build.gradle b/plugins/repository-azure/build.gradle index 9da2e58c55271..26e2b4813b8a5 100644 --- a/plugins/repository-azure/build.gradle +++ b/plugins/repository-azure/build.gradle @@ -70,7 +70,7 @@ dependencies { api 'org.codehaus.woodstox:stax2-api:4.2.1' implementation "com.fasterxml.woodstox:woodstox-core:${versions.woodstox}" runtimeOnly "com.google.guava:guava:${versions.guava}" - api 'org.apache.commons:commons-lang3:3.13.0' + api "org.apache.commons:commons-lang3:${versions.commonslang}" testImplementation project(':test:fixtures:azure-fixture') } From 082d425c21a77185f1ce303998ab343e3ce475c4 Mon Sep 17 00:00:00 2001 From: Dev Agarwal Date: Thu, 31 Aug 2023 22:29:22 +0530 Subject: [PATCH 23/37] Added sampler based on Blanket Probabilistic Sampling rate and Override for on demand (#9522) Signed-off-by: Dev Agarwal --- CHANGELOG.md | 1 + .../IntegrationTestOTelTelemetryPlugin.java | 6 +- .../telemetry/OTelTelemetryPlugin.java | 8 +- .../tracing/OTelResourceProvider.java | 8 +- .../tracing/sampler/ProbabilisticSampler.java | 82 +++++++++++++++++ .../tracing/sampler/RequestSampler.java | 67 ++++++++++++++ .../tracing/sampler/package-info.java | 12 +++ .../telemetry/OTelTelemetryPluginTests.java | 6 +- .../sampler/ProbabilisticSamplerTests.java | 64 +++++++++++++ .../tracing/sampler/RequestSamplerTests.java | 92 +++++++++++++++++++ .../common/settings/ClusterSettings.java | 2 +- .../opensearch/plugins/TelemetryPlugin.java | 2 +- .../telemetry/TelemetrySettings.java | 29 ++++++ 13 files changed, 367 insertions(+), 12 deletions(-) create mode 100644 plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/sampler/ProbabilisticSampler.java create mode 100644 plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/sampler/RequestSampler.java create mode 100644 plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/sampler/package-info.java create mode 100644 plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/sampler/ProbabilisticSamplerTests.java create mode 100644 plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/sampler/RequestSamplerTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 5df33dc6e3ace..0f3f4be1d295b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -169,6 +169,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [Remote Store] Rate limiter integration for remote store uploads and downloads([#9448](https://github.com/opensearch-project/OpenSearch/pull/9448/)) - [Remote Store] Implicitly use replication type SEGMENT for remote store clusters ([#9264](https://github.com/opensearch-project/OpenSearch/pull/9264)) - Use non-concurrent path for sort request on timeseries index and field([#9562](https://github.com/opensearch-project/OpenSearch/pull/9562)) +- Added sampler based on `Blanket Probabilistic Sampling rate` and `Override for on demand` ([#9621](https://github.com/opensearch-project/OpenSearch/issues/9621)) ### Deprecated diff --git a/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/tracing/IntegrationTestOTelTelemetryPlugin.java b/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/tracing/IntegrationTestOTelTelemetryPlugin.java index 57dbf4e001be4..ed4d13f3abb7d 100644 --- a/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/tracing/IntegrationTestOTelTelemetryPlugin.java +++ b/plugins/telemetry-otel/src/internalClusterTest/java/org/opensearch/telemetry/tracing/IntegrationTestOTelTelemetryPlugin.java @@ -32,10 +32,10 @@ public IntegrationTestOTelTelemetryPlugin(Settings settings) { /** * This method overrides getTelemetry() method in OTel plugin class, so we create only one instance of global OpenTelemetry * resetForTest() will set OpenTelemetry to null again. - * @param settings cluster settings + * @param telemetrySettings telemetry settings */ - public Optional getTelemetry(TelemetrySettings settings) { + public Optional getTelemetry(TelemetrySettings telemetrySettings) { GlobalOpenTelemetry.resetForTest(); - return super.getTelemetry(settings); + return super.getTelemetry(telemetrySettings); } } diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/OTelTelemetryPlugin.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/OTelTelemetryPlugin.java index a1ca3adf4d2a2..1af88196e3727 100644 --- a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/OTelTelemetryPlugin.java +++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/OTelTelemetryPlugin.java @@ -49,8 +49,8 @@ public List> getSettings() { } @Override - public Optional getTelemetry(TelemetrySettings settings) { - return Optional.of(telemetry()); + public Optional getTelemetry(TelemetrySettings telemetrySettings) { + return Optional.of(telemetry(telemetrySettings)); } @Override @@ -58,8 +58,8 @@ public String getName() { return OTEL_TRACER_NAME; } - private Telemetry telemetry() { - return new OTelTelemetry(new OTelTracingTelemetry(OTelResourceProvider.get(settings)), new MetricsTelemetry() { + private Telemetry telemetry(TelemetrySettings telemetrySettings) { + return new OTelTelemetry(new OTelTracingTelemetry(OTelResourceProvider.get(telemetrySettings, settings)), new MetricsTelemetry() { }); } diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelResourceProvider.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelResourceProvider.java index 1ec4818b8b73e..b395a335a4d83 100644 --- a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelResourceProvider.java +++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelResourceProvider.java @@ -9,7 +9,10 @@ package org.opensearch.telemetry.tracing; import org.opensearch.common.settings.Settings; +import org.opensearch.telemetry.TelemetrySettings; import org.opensearch.telemetry.tracing.exporter.OTelSpanExporterFactory; +import org.opensearch.telemetry.tracing.sampler.ProbabilisticSampler; +import org.opensearch.telemetry.tracing.sampler.RequestSampler; import java.util.concurrent.TimeUnit; @@ -37,15 +40,16 @@ private OTelResourceProvider() {} /** * Creates OpenTelemetry instance with default configuration + * @param telemetrySettings telemetry settings * @param settings cluster settings * @return OpenTelemetry instance */ - public static OpenTelemetry get(Settings settings) { + public static OpenTelemetry get(TelemetrySettings telemetrySettings, Settings settings) { return get( settings, OTelSpanExporterFactory.create(settings), ContextPropagators.create(W3CTraceContextPropagator.getInstance()), - Sampler.alwaysOn() + Sampler.parentBased(new RequestSampler(new ProbabilisticSampler(telemetrySettings))) ); } diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/sampler/ProbabilisticSampler.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/sampler/ProbabilisticSampler.java new file mode 100644 index 0000000000000..cab7b1a4af2e6 --- /dev/null +++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/sampler/ProbabilisticSampler.java @@ -0,0 +1,82 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.telemetry.tracing.sampler; + +import org.opensearch.telemetry.TelemetrySettings; + +import java.util.List; +import java.util.Objects; + +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.trace.SpanKind; +import io.opentelemetry.context.Context; +import io.opentelemetry.sdk.trace.data.LinkData; +import io.opentelemetry.sdk.trace.samplers.Sampler; +import io.opentelemetry.sdk.trace.samplers.SamplingResult; + +/** + * ProbabilisticSampler implements a head-based sampling strategy based on provided settings. + */ +public class ProbabilisticSampler implements Sampler { + private Sampler defaultSampler; + private final TelemetrySettings telemetrySettings; + private double samplingRatio; + + /** + * Constructor + * + * @param telemetrySettings Telemetry settings. + */ + public ProbabilisticSampler(TelemetrySettings telemetrySettings) { + this.telemetrySettings = Objects.requireNonNull(telemetrySettings); + this.samplingRatio = telemetrySettings.getTracerHeadSamplerSamplingRatio(); + this.defaultSampler = Sampler.traceIdRatioBased(samplingRatio); + } + + Sampler getSampler() { + double newSamplingRatio = telemetrySettings.getTracerHeadSamplerSamplingRatio(); + if (isSamplingRatioChanged(newSamplingRatio)) { + synchronized (this) { + this.samplingRatio = newSamplingRatio; + defaultSampler = Sampler.traceIdRatioBased(samplingRatio); + } + } + return defaultSampler; + } + + private boolean isSamplingRatioChanged(double newSamplingRatio) { + return Double.compare(this.samplingRatio, newSamplingRatio) != 0; + } + + double getSamplingRatio() { + return samplingRatio; + } + + @Override + public SamplingResult shouldSample( + Context parentContext, + String traceId, + String name, + SpanKind spanKind, + Attributes attributes, + List parentLinks + ) { + return getSampler().shouldSample(parentContext, traceId, name, spanKind, attributes, parentLinks); + } + + @Override + public String getDescription() { + return "Probabilistic Sampler"; + } + + @Override + public String toString() { + return getDescription(); + } +} diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/sampler/RequestSampler.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/sampler/RequestSampler.java new file mode 100644 index 0000000000000..9ea681370a3ec --- /dev/null +++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/sampler/RequestSampler.java @@ -0,0 +1,67 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.telemetry.tracing.sampler; + +import java.util.List; + +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.trace.SpanKind; +import io.opentelemetry.context.Context; +import io.opentelemetry.sdk.trace.data.LinkData; +import io.opentelemetry.sdk.trace.samplers.Sampler; +import io.opentelemetry.sdk.trace.samplers.SamplingResult; + +/** + * HeadBased sampler + */ +public class RequestSampler implements Sampler { + private final Sampler defaultSampler; + + // TODO: Pick value of TRACE from PR #9415. + private static final String TRACE = "trace"; + + /** + * Creates Head based sampler + * @param defaultSampler defaultSampler + */ + public RequestSampler(Sampler defaultSampler) { + this.defaultSampler = defaultSampler; + } + + @Override + public SamplingResult shouldSample( + Context parentContext, + String traceId, + String name, + SpanKind spanKind, + Attributes attributes, + List parentLinks + ) { + + final String trace = attributes.get(AttributeKey.stringKey(TRACE)); + + if (trace != null) { + return (Boolean.parseBoolean(trace) == true) ? SamplingResult.recordAndSample() : SamplingResult.drop(); + } else { + return defaultSampler.shouldSample(parentContext, traceId, name, spanKind, attributes, parentLinks); + } + + } + + @Override + public String getDescription() { + return "Request Sampler"; + } + + @Override + public String toString() { + return getDescription(); + } +} diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/sampler/package-info.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/sampler/package-info.java new file mode 100644 index 0000000000000..6534b33f6177c --- /dev/null +++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/sampler/package-info.java @@ -0,0 +1,12 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +/** + * This package contains classes needed for sampler. + */ +package org.opensearch.telemetry.tracing.sampler; diff --git a/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/OTelTelemetryPluginTests.java b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/OTelTelemetryPluginTests.java index 611656942860f..8c2b5d14733e2 100644 --- a/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/OTelTelemetryPluginTests.java +++ b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/OTelTelemetryPluginTests.java @@ -29,6 +29,8 @@ import static org.opensearch.telemetry.OTelTelemetrySettings.TRACER_EXPORTER_BATCH_SIZE_SETTING; import static org.opensearch.telemetry.OTelTelemetrySettings.TRACER_EXPORTER_DELAY_SETTING; import static org.opensearch.telemetry.OTelTelemetrySettings.TRACER_EXPORTER_MAX_QUEUE_SIZE_SETTING; +import static org.opensearch.telemetry.TelemetrySettings.TRACER_ENABLED_SETTING; +import static org.opensearch.telemetry.TelemetrySettings.TRACER_SAMPLER_PROBABILITY; public class OTelTelemetryPluginTests extends OpenSearchTestCase { @@ -42,7 +44,9 @@ public void setup() { // io.opentelemetry.sdk.OpenTelemetrySdk.close waits only for 10 seconds for shutdown to complete. Settings settings = Settings.builder().put(TRACER_EXPORTER_DELAY_SETTING.getKey(), "1s").build(); oTelTracerModulePlugin = new OTelTelemetryPlugin(settings); - telemetry = oTelTracerModulePlugin.getTelemetry(null); + telemetry = oTelTracerModulePlugin.getTelemetry( + new TelemetrySettings(Settings.EMPTY, new ClusterSettings(settings, Set.of(TRACER_ENABLED_SETTING, TRACER_SAMPLER_PROBABILITY))) + ); tracingTelemetry = telemetry.get().getTracingTelemetry(); } diff --git a/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/sampler/ProbabilisticSamplerTests.java b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/sampler/ProbabilisticSamplerTests.java new file mode 100644 index 0000000000000..639dc341ef0db --- /dev/null +++ b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/sampler/ProbabilisticSamplerTests.java @@ -0,0 +1,64 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.telemetry.tracing.sampler; + +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.telemetry.TelemetrySettings; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.Set; + +import io.opentelemetry.sdk.trace.samplers.Sampler; + +import static org.opensearch.telemetry.OTelTelemetrySettings.TRACER_EXPORTER_DELAY_SETTING; +import static org.opensearch.telemetry.TelemetrySettings.TRACER_ENABLED_SETTING; +import static org.opensearch.telemetry.TelemetrySettings.TRACER_SAMPLER_PROBABILITY; + +public class ProbabilisticSamplerTests extends OpenSearchTestCase { + + // When ProbabilisticSampler is created with OTelTelemetrySettings as null + public void testProbabilisticSamplerWithNullSettings() { + // Verify that the constructor throws IllegalArgumentException when given null settings + assertThrows(NullPointerException.class, () -> { new ProbabilisticSampler(null); }); + } + + public void testDefaultGetSampler() { + Settings settings = Settings.builder().put(TRACER_EXPORTER_DELAY_SETTING.getKey(), "1s").build(); + TelemetrySettings telemetrySettings = new TelemetrySettings( + Settings.EMPTY, + new ClusterSettings(settings, Set.of(TRACER_SAMPLER_PROBABILITY, TRACER_ENABLED_SETTING)) + ); + + // Probabilistic Sampler + ProbabilisticSampler probabilisticSampler = new ProbabilisticSampler(telemetrySettings); + + assertNotNull(probabilisticSampler.getSampler()); + assertEquals(0.01, probabilisticSampler.getSamplingRatio(), 0.0d); + } + + public void testGetSamplerWithUpdatedSamplingRatio() { + Settings settings = Settings.builder().put(TRACER_EXPORTER_DELAY_SETTING.getKey(), "1s").build(); + TelemetrySettings telemetrySettings = new TelemetrySettings( + Settings.EMPTY, + new ClusterSettings(settings, Set.of(TRACER_SAMPLER_PROBABILITY, TRACER_ENABLED_SETTING)) + ); + + // Probabilistic Sampler + ProbabilisticSampler probabilisticSampler = new ProbabilisticSampler(telemetrySettings); + assertEquals(0.01d, probabilisticSampler.getSamplingRatio(), 0.0d); + + telemetrySettings.setSamplingProbability(0.02); + + // Need to call getSampler() to update the value of tracerHeadSamplerSamplingRatio + Sampler updatedProbabilisticSampler = probabilisticSampler.getSampler(); + assertEquals(0.02, probabilisticSampler.getSamplingRatio(), 0.0d); + } + +} diff --git a/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/sampler/RequestSamplerTests.java b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/sampler/RequestSamplerTests.java new file mode 100644 index 0000000000000..facf04623ec46 --- /dev/null +++ b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/sampler/RequestSamplerTests.java @@ -0,0 +1,92 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.telemetry.tracing.sampler; + +import org.opensearch.test.OpenSearchTestCase; + +import java.util.Collections; + +import io.opentelemetry.api.common.AttributeKey; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.trace.SpanKind; +import io.opentelemetry.context.Context; +import io.opentelemetry.sdk.trace.samplers.Sampler; +import io.opentelemetry.sdk.trace.samplers.SamplingResult; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class RequestSamplerTests extends OpenSearchTestCase { + + public void testShouldSampleWithTraceAttributeAsTrue() { + + // Create a mock default sampler + Sampler defaultSampler = mock(Sampler.class); + when(defaultSampler.shouldSample(any(), anyString(), anyString(), any(), any(), any())).thenReturn(SamplingResult.drop()); + + // Create an instance of HeadSampler with the mock default sampler + RequestSampler requestSampler = new RequestSampler(defaultSampler); + + // Create a mock Context and Attributes + Context parentContext = mock(Context.class); + Attributes attributes = Attributes.of(AttributeKey.stringKey("trace"), "true"); + + // Call shouldSample on HeadSampler + SamplingResult result = requestSampler.shouldSample( + parentContext, + "traceId", + "spanName", + SpanKind.INTERNAL, + attributes, + Collections.emptyList() + ); + + assertEquals(SamplingResult.recordAndSample(), result); + + // Verify that the default sampler's shouldSample method was not called + verify(defaultSampler, never()).shouldSample(any(), anyString(), anyString(), any(), any(), any()); + } + + public void testShouldSampleWithoutTraceAttribute() { + + // Create a mock default sampler + Sampler defaultSampler = mock(Sampler.class); + when(defaultSampler.shouldSample(any(), anyString(), anyString(), any(), any(), any())).thenReturn( + SamplingResult.recordAndSample() + ); + + // Create an instance of HeadSampler with the mock default sampler + RequestSampler requestSampler = new RequestSampler(defaultSampler); + + // Create a mock Context and Attributes + Context parentContext = mock(Context.class); + Attributes attributes = Attributes.empty(); + + // Call shouldSample on HeadSampler + SamplingResult result = requestSampler.shouldSample( + parentContext, + "traceId", + "spanName", + SpanKind.INTERNAL, + attributes, + Collections.emptyList() + ); + + // Verify that HeadSampler returned SamplingResult.recordAndSample() + assertEquals(SamplingResult.recordAndSample(), result); + + // Verify that the default sampler's shouldSample method was called + verify(defaultSampler).shouldSample(any(), anyString(), anyString(), any(), any(), any()); + } + +} diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index e00e7e3bf4ea7..8093eab696779 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -694,6 +694,6 @@ public void apply(Settings value, Settings current, Settings previous) { SearchService.CONCURRENT_SEGMENT_SEARCH_TARGET_MAX_SLICE_COUNT_SETTING ), List.of(FeatureFlags.TELEMETRY), - List.of(TelemetrySettings.TRACER_ENABLED_SETTING) + List.of(TelemetrySettings.TRACER_ENABLED_SETTING, TelemetrySettings.TRACER_SAMPLER_PROBABILITY) ); } diff --git a/server/src/main/java/org/opensearch/plugins/TelemetryPlugin.java b/server/src/main/java/org/opensearch/plugins/TelemetryPlugin.java index 33dc9b7a0c843..66033df394d9f 100644 --- a/server/src/main/java/org/opensearch/plugins/TelemetryPlugin.java +++ b/server/src/main/java/org/opensearch/plugins/TelemetryPlugin.java @@ -18,7 +18,7 @@ */ public interface TelemetryPlugin { - Optional getTelemetry(TelemetrySettings settings); + Optional getTelemetry(TelemetrySettings telemetrySettings); String getName(); diff --git a/server/src/main/java/org/opensearch/telemetry/TelemetrySettings.java b/server/src/main/java/org/opensearch/telemetry/TelemetrySettings.java index 7c9e0d5ac8097..aa11a2879e4d7 100644 --- a/server/src/main/java/org/opensearch/telemetry/TelemetrySettings.java +++ b/server/src/main/java/org/opensearch/telemetry/TelemetrySettings.java @@ -23,12 +23,27 @@ public class TelemetrySettings { Setting.Property.Dynamic ); + /** + * Probability of sampler + */ + public static final Setting TRACER_SAMPLER_PROBABILITY = Setting.doubleSetting( + "telemetry.tracer.sampler.probability", + 0.01d, + 0.00d, + 1.00d, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + private volatile boolean tracingEnabled; + private volatile double samplingProbability; public TelemetrySettings(Settings settings, ClusterSettings clusterSettings) { this.tracingEnabled = TRACER_ENABLED_SETTING.get(settings); + this.samplingProbability = TRACER_SAMPLER_PROBABILITY.get(settings); clusterSettings.addSettingsUpdateConsumer(TRACER_ENABLED_SETTING, this::setTracingEnabled); + clusterSettings.addSettingsUpdateConsumer(TRACER_SAMPLER_PROBABILITY, this::setSamplingProbability); } public void setTracingEnabled(boolean tracingEnabled) { @@ -39,4 +54,18 @@ public boolean isTracingEnabled() { return tracingEnabled; } + /** + * Set sampling ratio + * @param samplingProbability double + */ + public void setSamplingProbability(double samplingProbability) { + this.samplingProbability = samplingProbability; + } + + /** + * Get sampling ratio + */ + public double getTracerHeadSamplerSamplingRatio() { + return samplingProbability; + } } From 1126d2f7caac7ec660975e63527404b9d9f6e087 Mon Sep 17 00:00:00 2001 From: Russ Cam Date: Fri, 1 Sep 2023 04:44:07 +1000 Subject: [PATCH 24/37] Expose DelimitedTermFrequencyTokenFilter (#9479) * Expose DelimitedTermFrequencyTokenFilter Relates: #9413 This commit exposes Lucene's delimited term frequency token filter to be able to provide term frequencies along with terms. Signed-off-by: Russ Cam * fix format violations Signed-off-by: Russ Cam * fix test and add to changelog Signed-off-by: Russ Cam * Address PR feedback - Add unit tests for DelimitedTermFrequencyTokenFilterFactory - Remove IllegalArgumentException as caught exception - Add skip to yaml rest tests to skip for version < 2.10 Signed-off-by: Russ Cam * formatting Signed-off-by: Russ Cam * Rename filter Signed-off-by: Russ Cam * update naming in REST tests Signed-off-by: Russ Cam --------- Signed-off-by: Russ Cam --- CHANGELOG.md | 1 + .../common/CommonAnalysisModulePlugin.java | 9 ++ ...imitedTermFrequencyTokenFilterFactory.java | 45 ++++++++++ .../common/CommonAnalysisFactoryTests.java | 2 + ...dTermFrequencyTokenFilterFactoryTests.java | 89 +++++++++++++++++++ .../test/analysis-common/40_token_filters.yml | 40 +++++++++ .../analysis/AnalysisFactoryTestCase.java | 4 +- 7 files changed, 187 insertions(+), 3 deletions(-) create mode 100644 modules/analysis-common/src/main/java/org/opensearch/analysis/common/DelimitedTermFrequencyTokenFilterFactory.java create mode 100644 modules/analysis-common/src/test/java/org/opensearch/analysis/common/DelimitedTermFrequencyTokenFilterFactoryTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 0f3f4be1d295b..d4a060cc16504 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -96,6 +96,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [BWC and API enforcement] Decorate the existing APIs with proper annotations (part 1) ([#9520](https://github.com/opensearch-project/OpenSearch/pull/9520)) - Add concurrent segment search related metrics to node and index stats ([#9622](https://github.com/opensearch-project/OpenSearch/issues/9622)) - Decouple replication lag from logic to fail stale replicas ([#9507](https://github.com/opensearch-project/OpenSearch/pull/9507)) +- Expose DelimitedTermFrequencyTokenFilter to allow providing term frequencies along with terms ([#9479](https://github.com/opensearch-project/OpenSearch/pull/9479)) ### Dependencies - Bump `org.apache.logging.log4j:log4j-core` from 2.17.1 to 2.20.0 ([#8307](https://github.com/opensearch-project/OpenSearch/pull/8307)) diff --git a/modules/analysis-common/src/main/java/org/opensearch/analysis/common/CommonAnalysisModulePlugin.java b/modules/analysis-common/src/main/java/org/opensearch/analysis/common/CommonAnalysisModulePlugin.java index 46220f5369d16..b0d9c1765190a 100644 --- a/modules/analysis-common/src/main/java/org/opensearch/analysis/common/CommonAnalysisModulePlugin.java +++ b/modules/analysis-common/src/main/java/org/opensearch/analysis/common/CommonAnalysisModulePlugin.java @@ -89,6 +89,7 @@ import org.apache.lucene.analysis.lt.LithuanianAnalyzer; import org.apache.lucene.analysis.lv.LatvianAnalyzer; import org.apache.lucene.analysis.miscellaneous.ASCIIFoldingFilter; +import org.apache.lucene.analysis.miscellaneous.DelimitedTermFrequencyTokenFilter; import org.apache.lucene.analysis.miscellaneous.DisableGraphAttribute; import org.apache.lucene.analysis.miscellaneous.KeywordRepeatFilter; import org.apache.lucene.analysis.miscellaneous.LengthFilter; @@ -265,6 +266,7 @@ public Map> getTokenFilters() { ); filters.put("decimal_digit", DecimalDigitFilterFactory::new); filters.put("delimited_payload", DelimitedPayloadTokenFilterFactory::new); + filters.put("delimited_term_freq", DelimitedTermFrequencyTokenFilterFactory::new); filters.put("dictionary_decompounder", requiresAnalysisSettings(DictionaryCompoundWordTokenFilterFactory::new)); filters.put("dutch_stem", DutchStemTokenFilterFactory::new); filters.put("edge_ngram", EdgeNGramTokenFilterFactory::new); @@ -500,6 +502,13 @@ public List getPreConfiguredTokenFilters() { ) ) ); + filters.add( + PreConfiguredTokenFilter.singleton( + "delimited_term_freq", + false, + input -> new DelimitedTermFrequencyTokenFilter(input, DelimitedTermFrequencyTokenFilterFactory.DEFAULT_DELIMITER) + ) + ); filters.add(PreConfiguredTokenFilter.singleton("dutch_stem", false, input -> new SnowballFilter(input, new DutchStemmer()))); filters.add(PreConfiguredTokenFilter.singleton("edge_ngram", false, false, input -> new EdgeNGramTokenFilter(input, 1))); filters.add(PreConfiguredTokenFilter.openSearchVersion("edgeNGram", false, false, (reader, version) -> { diff --git a/modules/analysis-common/src/main/java/org/opensearch/analysis/common/DelimitedTermFrequencyTokenFilterFactory.java b/modules/analysis-common/src/main/java/org/opensearch/analysis/common/DelimitedTermFrequencyTokenFilterFactory.java new file mode 100644 index 0000000000000..8929a7c54ef4c --- /dev/null +++ b/modules/analysis-common/src/main/java/org/opensearch/analysis/common/DelimitedTermFrequencyTokenFilterFactory.java @@ -0,0 +1,45 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.analysis.common; + +import org.apache.lucene.analysis.TokenStream; +import org.apache.lucene.analysis.miscellaneous.DelimitedTermFrequencyTokenFilter; +import org.opensearch.common.settings.Settings; +import org.opensearch.env.Environment; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.analysis.AbstractTokenFilterFactory; + +public class DelimitedTermFrequencyTokenFilterFactory extends AbstractTokenFilterFactory { + public static final char DEFAULT_DELIMITER = '|'; + private static final String DELIMITER = "delimiter"; + private final char delimiter; + + DelimitedTermFrequencyTokenFilterFactory(IndexSettings indexSettings, Environment environment, String name, Settings settings) { + super(indexSettings, name, settings); + delimiter = parseDelimiter(settings); + } + + @Override + public TokenStream create(TokenStream tokenStream) { + return new DelimitedTermFrequencyTokenFilter(tokenStream, delimiter); + } + + private static char parseDelimiter(Settings settings) { + String delimiter = settings.get(DELIMITER); + if (delimiter == null) { + return DEFAULT_DELIMITER; + } else if (delimiter.length() == 1) { + return delimiter.charAt(0); + } + + throw new IllegalArgumentException( + "Setting [" + DELIMITER + "] must be a single, non-null character. [" + delimiter + "] was provided." + ); + } +} diff --git a/modules/analysis-common/src/test/java/org/opensearch/analysis/common/CommonAnalysisFactoryTests.java b/modules/analysis-common/src/test/java/org/opensearch/analysis/common/CommonAnalysisFactoryTests.java index 1c4db089565ff..11713f52f5b18 100644 --- a/modules/analysis-common/src/test/java/org/opensearch/analysis/common/CommonAnalysisFactoryTests.java +++ b/modules/analysis-common/src/test/java/org/opensearch/analysis/common/CommonAnalysisFactoryTests.java @@ -145,6 +145,7 @@ protected Map> getTokenFilters() { filters.put("cjkwidth", CJKWidthFilterFactory.class); filters.put("cjkbigram", CJKBigramFilterFactory.class); filters.put("delimitedpayload", DelimitedPayloadTokenFilterFactory.class); + filters.put("delimitedtermfrequency", DelimitedTermFrequencyTokenFilterFactory.class); filters.put("keepword", KeepWordFilterFactory.class); filters.put("type", KeepTypesFilterFactory.class); filters.put("classic", ClassicFilterFactory.class); @@ -202,6 +203,7 @@ protected Map> getPreConfiguredTokenFilters() { filters.put("decimal_digit", null); filters.put("delimited_payload_filter", org.apache.lucene.analysis.payloads.DelimitedPayloadTokenFilterFactory.class); filters.put("delimited_payload", org.apache.lucene.analysis.payloads.DelimitedPayloadTokenFilterFactory.class); + filters.put("delimited_term_freq", org.apache.lucene.analysis.miscellaneous.DelimitedTermFrequencyTokenFilterFactory.class); filters.put("dutch_stem", SnowballPorterFilterFactory.class); filters.put("edge_ngram", null); filters.put("edgeNGram", null); diff --git a/modules/analysis-common/src/test/java/org/opensearch/analysis/common/DelimitedTermFrequencyTokenFilterFactoryTests.java b/modules/analysis-common/src/test/java/org/opensearch/analysis/common/DelimitedTermFrequencyTokenFilterFactoryTests.java new file mode 100644 index 0000000000000..fab83a75387de --- /dev/null +++ b/modules/analysis-common/src/test/java/org/opensearch/analysis/common/DelimitedTermFrequencyTokenFilterFactoryTests.java @@ -0,0 +1,89 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.analysis.common; + +import org.apache.lucene.analysis.TokenStream; +import org.apache.lucene.analysis.Tokenizer; +import org.apache.lucene.analysis.core.WhitespaceTokenizer; +import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; +import org.apache.lucene.analysis.tokenattributes.TermFrequencyAttribute; +import org.opensearch.common.settings.Settings; +import org.opensearch.env.Environment; +import org.opensearch.index.analysis.AnalysisTestsHelper; +import org.opensearch.index.analysis.TokenFilterFactory; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.test.OpenSearchTokenStreamTestCase; + +import java.io.StringReader; + +public class DelimitedTermFrequencyTokenFilterFactoryTests extends OpenSearchTokenStreamTestCase { + + public void testDefault() throws Exception { + OpenSearchTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings( + Settings.builder() + .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) + .put("index.analysis.filter.my_delimited_term_freq.type", "delimited_term_freq") + .build(), + new CommonAnalysisModulePlugin() + ); + doTest(analysis, "cat|4 dog|5"); + } + + public void testDelimiter() throws Exception { + OpenSearchTestCase.TestAnalysis analysis = AnalysisTestsHelper.createTestAnalysisFromSettings( + Settings.builder() + .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) + .put("index.analysis.filter.my_delimited_term_freq.type", "delimited_term_freq") + .put("index.analysis.filter.my_delimited_term_freq.delimiter", ":") + .build(), + new CommonAnalysisModulePlugin() + ); + doTest(analysis, "cat:4 dog:5"); + } + + public void testDelimiterLongerThanOneCharThrows() { + IllegalArgumentException ex = expectThrows( + IllegalArgumentException.class, + () -> AnalysisTestsHelper.createTestAnalysisFromSettings( + Settings.builder() + .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) + .put("index.analysis.filter.my_delimited_term_freq.type", "delimited_term_freq") + .put("index.analysis.filter.my_delimited_term_freq.delimiter", "^^") + .build(), + new CommonAnalysisModulePlugin() + ) + ); + + assertEquals("Setting [delimiter] must be a single, non-null character. [^^] was provided.", ex.getMessage()); + } + + private void doTest(OpenSearchTestCase.TestAnalysis analysis, String source) throws Exception { + TokenFilterFactory tokenFilter = analysis.tokenFilter.get("my_delimited_term_freq"); + Tokenizer tokenizer = new WhitespaceTokenizer(); + tokenizer.setReader(new StringReader(source)); + + TokenStream stream = tokenFilter.create(tokenizer); + + CharTermAttribute termAtt = stream.getAttribute(CharTermAttribute.class); + TermFrequencyAttribute tfAtt = stream.getAttribute(TermFrequencyAttribute.class); + stream.reset(); + assertTermEquals("cat", stream, termAtt, tfAtt, 4); + assertTermEquals("dog", stream, termAtt, tfAtt, 5); + assertFalse(stream.incrementToken()); + stream.end(); + stream.close(); + } + + void assertTermEquals(String expected, TokenStream stream, CharTermAttribute termAtt, TermFrequencyAttribute tfAtt, int expectedTf) + throws Exception { + assertTrue(stream.incrementToken()); + assertEquals(expected, termAtt.toString()); + assertEquals(expectedTf, tfAtt.getTermFrequency()); + } +} diff --git a/modules/analysis-common/src/yamlRestTest/resources/rest-api-spec/test/analysis-common/40_token_filters.yml b/modules/analysis-common/src/yamlRestTest/resources/rest-api-spec/test/analysis-common/40_token_filters.yml index 40c82ff185661..e92cc0c4838c7 100644 --- a/modules/analysis-common/src/yamlRestTest/resources/rest-api-spec/test/analysis-common/40_token_filters.yml +++ b/modules/analysis-common/src/yamlRestTest/resources/rest-api-spec/test/analysis-common/40_token_filters.yml @@ -1198,6 +1198,46 @@ - match: { tokens.0.token: foo } --- +"delimited_term_freq": + - skip: + version: " - 2.9.99" + reason: "delimited_term_freq token filter was added in v2.10.0" + - do: + indices.create: + index: test + body: + settings: + analysis: + filter: + my_delimited_term_freq: + type: delimited_term_freq + delimiter: ^ + - do: + indices.analyze: + index: test + body: + text: foo^3 + tokenizer: keyword + filter: [my_delimited_term_freq] + attributes: termFrequency + explain: true + - length: { detail.tokenfilters: 1 } + - match: { detail.tokenfilters.0.tokens.0.token: foo } + - match: { detail.tokenfilters.0.tokens.0.termFrequency: 3 } + + # Test pre-configured token filter too: + - do: + indices.analyze: + body: + text: foo|100 + tokenizer: keyword + filter: [delimited_term_freq] + attributes: termFrequency + explain: true + - length: { detail.tokenfilters: 1 } + - match: { detail.tokenfilters.0.tokens.0.token: foo } + - match: { detail.tokenfilters.0.tokens.0.termFrequency: 100 } +--- "keep_filter": - do: indices.create: diff --git a/test/framework/src/main/java/org/opensearch/indices/analysis/AnalysisFactoryTestCase.java b/test/framework/src/main/java/org/opensearch/indices/analysis/AnalysisFactoryTestCase.java index b93cb64e32cfe..c412ae8317f24 100644 --- a/test/framework/src/main/java/org/opensearch/indices/analysis/AnalysisFactoryTestCase.java +++ b/test/framework/src/main/java/org/opensearch/indices/analysis/AnalysisFactoryTestCase.java @@ -98,6 +98,7 @@ public abstract class AnalysisFactoryTestCase extends OpenSearchTestCase { .put("czechstem", MovedToAnalysisCommon.class) .put("decimaldigit", MovedToAnalysisCommon.class) .put("delimitedpayload", MovedToAnalysisCommon.class) + .put("delimitedtermfrequency", MovedToAnalysisCommon.class) .put("dictionarycompoundword", MovedToAnalysisCommon.class) .put("edgengram", MovedToAnalysisCommon.class) .put("elision", MovedToAnalysisCommon.class) @@ -201,9 +202,6 @@ public abstract class AnalysisFactoryTestCase extends OpenSearchTestCase { .put("daterecognizer", Void.class) // for token filters that generate bad offsets, which are now rejected since Lucene 7 .put("fixbrokenoffsets", Void.class) - // should we expose it, or maybe think about higher level integration of the - // fake term frequency feature (LUCENE-7854) - .put("delimitedtermfrequency", Void.class) // LUCENE-8273: ProtectedTermFilterFactory allows analysis chains to skip // particular token filters based on the attributes of the current token. .put("protectedterm", Void.class) From 6765b1654658f3d73911ef3d26b179959d873a55 Mon Sep 17 00:00:00 2001 From: Kunal Kotwani Date: Thu, 31 Aug 2023 17:39:07 -0700 Subject: [PATCH 25/37] Add async blob read and download support using multiple streams (#9592) Signed-off-by: Kunal Kotwani --- CHANGELOG.md | 1 + .../repositories/s3/S3BlobContainer.java | 6 + .../s3/S3BlobStoreContainerTests.java | 12 ++ .../mocks/MockFsVerifyingBlobContainer.java | 24 +++ .../VerifyingMultiStreamBlobContainer.java | 26 +++ .../blobstore/stream/read/ReadContext.java | 46 +++++ .../read/listener/FileCompletionListener.java | 47 +++++ .../stream/read/listener/FilePartWriter.java | 90 ++++++++++ .../read/listener/ReadContextListener.java | 65 +++++++ .../stream/read/listener/package-info.java | 14 ++ .../blobstore/stream/read/package-info.java | 13 ++ .../listener/FileCompletionListenerTests.java | 58 +++++++ .../read/listener/FilePartWriterTests.java | 163 ++++++++++++++++++ .../read/listener/ListenerTestUtils.java | 56 ++++++ .../listener/ReadContextListenerTests.java | 124 +++++++++++++ 15 files changed, 745 insertions(+) create mode 100644 server/src/main/java/org/opensearch/common/blobstore/stream/read/ReadContext.java create mode 100644 server/src/main/java/org/opensearch/common/blobstore/stream/read/listener/FileCompletionListener.java create mode 100644 server/src/main/java/org/opensearch/common/blobstore/stream/read/listener/FilePartWriter.java create mode 100644 server/src/main/java/org/opensearch/common/blobstore/stream/read/listener/ReadContextListener.java create mode 100644 server/src/main/java/org/opensearch/common/blobstore/stream/read/listener/package-info.java create mode 100644 server/src/main/java/org/opensearch/common/blobstore/stream/read/package-info.java create mode 100644 server/src/test/java/org/opensearch/common/blobstore/stream/read/listener/FileCompletionListenerTests.java create mode 100644 server/src/test/java/org/opensearch/common/blobstore/stream/read/listener/FilePartWriterTests.java create mode 100644 server/src/test/java/org/opensearch/common/blobstore/stream/read/listener/ListenerTestUtils.java create mode 100644 server/src/test/java/org/opensearch/common/blobstore/stream/read/listener/ReadContextListenerTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index d4a060cc16504..80ab0fb87e609 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -97,6 +97,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add concurrent segment search related metrics to node and index stats ([#9622](https://github.com/opensearch-project/OpenSearch/issues/9622)) - Decouple replication lag from logic to fail stale replicas ([#9507](https://github.com/opensearch-project/OpenSearch/pull/9507)) - Expose DelimitedTermFrequencyTokenFilter to allow providing term frequencies along with terms ([#9479](https://github.com/opensearch-project/OpenSearch/pull/9479)) +- APIs for performing async blob reads and async downloads from the repository using multiple streams ([#9592](https://github.com/opensearch-project/OpenSearch/issues/9592)) ### Dependencies - Bump `org.apache.logging.log4j:log4j-core` from 2.17.1 to 2.20.0 ([#8307](https://github.com/opensearch-project/OpenSearch/pull/8307)) diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3BlobContainer.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3BlobContainer.java index a97a509adce47..183b5f8fe7ac1 100644 --- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3BlobContainer.java +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3BlobContainer.java @@ -69,6 +69,7 @@ import org.opensearch.common.blobstore.BlobStoreException; import org.opensearch.common.blobstore.DeleteResult; import org.opensearch.common.blobstore.VerifyingMultiStreamBlobContainer; +import org.opensearch.common.blobstore.stream.read.ReadContext; import org.opensearch.common.blobstore.stream.write.WriteContext; import org.opensearch.common.blobstore.stream.write.WritePriority; import org.opensearch.common.blobstore.support.AbstractBlobContainer; @@ -211,6 +212,11 @@ public void asyncBlobUpload(WriteContext writeContext, ActionListener comp } } + @Override + public void readBlobAsync(String blobName, ActionListener listener) { + throw new UnsupportedOperationException(); + } + // package private for testing long getLargeBlobThresholdInBytes() { return blobStore.bufferSizeInBytes(); diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobStoreContainerTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobStoreContainerTests.java index 2438acaf7c1f2..1c4936cae7eba 100644 --- a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobStoreContainerTests.java +++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobStoreContainerTests.java @@ -61,6 +61,7 @@ import software.amazon.awssdk.services.s3.model.UploadPartResponse; import software.amazon.awssdk.services.s3.paginators.ListObjectsV2Iterable; +import org.opensearch.action.support.PlainActionFuture; import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; @@ -881,6 +882,17 @@ public void onFailure(Exception e) {} } } + public void testAsyncBlobDownload() { + final S3BlobStore blobStore = mock(S3BlobStore.class); + final BlobPath blobPath = mock(BlobPath.class); + final String blobName = "test-blob"; + + final UnsupportedOperationException e = expectThrows(UnsupportedOperationException.class, () -> { + final S3BlobContainer blobContainer = new S3BlobContainer(blobPath, blobStore); + blobContainer.readBlobAsync(blobName, new PlainActionFuture<>()); + }); + } + public void testListBlobsByPrefixInLexicographicOrderWithNegativeLimit() throws IOException { testListBlobsByPrefixInLexicographicOrder(-5, 0, BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC); } diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsVerifyingBlobContainer.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsVerifyingBlobContainer.java index d882220c9f4d7..887a4cc6ba9a8 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsVerifyingBlobContainer.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/multipart/mocks/MockFsVerifyingBlobContainer.java @@ -14,6 +14,7 @@ import org.opensearch.common.blobstore.VerifyingMultiStreamBlobContainer; import org.opensearch.common.blobstore.fs.FsBlobContainer; import org.opensearch.common.blobstore.fs.FsBlobStore; +import org.opensearch.common.blobstore.stream.read.ReadContext; import org.opensearch.common.blobstore.stream.write.WriteContext; import org.opensearch.common.io.InputStreamContainer; import org.opensearch.core.action.ActionListener; @@ -24,6 +25,8 @@ import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.StandardOpenOption; +import java.util.ArrayList; +import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; @@ -114,6 +117,27 @@ public void asyncBlobUpload(WriteContext writeContext, ActionListener comp } + @Override + public void readBlobAsync(String blobName, ActionListener listener) { + new Thread(() -> { + try { + long contentLength = listBlobs().get(blobName).length(); + long partSize = contentLength / 10; + int numberOfParts = (int) ((contentLength % partSize) == 0 ? contentLength / partSize : (contentLength / partSize) + 1); + List blobPartStreams = new ArrayList<>(); + for (int partNumber = 0; partNumber < numberOfParts; partNumber++) { + long offset = partNumber * partSize; + InputStreamContainer blobPartStream = new InputStreamContainer(readBlob(blobName, offset, partSize), partSize, offset); + blobPartStreams.add(blobPartStream); + } + ReadContext blobReadContext = new ReadContext(contentLength, blobPartStreams, null); + listener.onResponse(blobReadContext); + } catch (Exception e) { + listener.onFailure(e); + } + }).start(); + } + private boolean isSegmentFile(String filename) { return !filename.endsWith(".tlog") && !filename.endsWith(".ckp"); } diff --git a/server/src/main/java/org/opensearch/common/blobstore/VerifyingMultiStreamBlobContainer.java b/server/src/main/java/org/opensearch/common/blobstore/VerifyingMultiStreamBlobContainer.java index d10445ba14d76..1764c9e634781 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/VerifyingMultiStreamBlobContainer.java +++ b/server/src/main/java/org/opensearch/common/blobstore/VerifyingMultiStreamBlobContainer.java @@ -8,10 +8,15 @@ package org.opensearch.common.blobstore; +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.common.blobstore.stream.read.ReadContext; +import org.opensearch.common.blobstore.stream.read.listener.ReadContextListener; import org.opensearch.common.blobstore.stream.write.WriteContext; import org.opensearch.core.action.ActionListener; +import org.opensearch.threadpool.ThreadPool; import java.io.IOException; +import java.nio.file.Path; /** * An extension of {@link BlobContainer} that adds {@link VerifyingMultiStreamBlobContainer#asyncBlobUpload} to allow @@ -31,4 +36,25 @@ public interface VerifyingMultiStreamBlobContainer extends BlobContainer { * @throws IOException if any of the input streams could not be read, or the target blob could not be written to */ void asyncBlobUpload(WriteContext writeContext, ActionListener completionListener) throws IOException; + + /** + * Creates an async callback of a {@link ReadContext} containing the multipart streams for a specified blob within the container. + * @param blobName The name of the blob for which the {@link ReadContext} needs to be fetched. + * @param listener Async listener for {@link ReadContext} object which serves the input streams and other metadata for the blob + */ + @ExperimentalApi + void readBlobAsync(String blobName, ActionListener listener); + + /** + * Asynchronously downloads the blob to the specified location using an executor from the thread pool. + * @param blobName The name of the blob for which needs to be downloaded. + * @param fileLocation The path on local disk where the blob needs to be downloaded. + * @param threadPool The threadpool instance which will provide the executor for performing a multipart download. + * @param completionListener Listener which will be notified when the download is complete. + */ + @ExperimentalApi + default void asyncBlobDownload(String blobName, Path fileLocation, ThreadPool threadPool, ActionListener completionListener) { + ReadContextListener readContextListener = new ReadContextListener(blobName, fileLocation, threadPool, completionListener); + readBlobAsync(blobName, readContextListener); + } } diff --git a/server/src/main/java/org/opensearch/common/blobstore/stream/read/ReadContext.java b/server/src/main/java/org/opensearch/common/blobstore/stream/read/ReadContext.java new file mode 100644 index 0000000000000..4ba17959f8040 --- /dev/null +++ b/server/src/main/java/org/opensearch/common/blobstore/stream/read/ReadContext.java @@ -0,0 +1,46 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.common.blobstore.stream.read; + +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.common.io.InputStreamContainer; + +import java.util.List; + +/** + * ReadContext is used to encapsulate all data needed by BlobContainer#readBlobAsync + */ +@ExperimentalApi +public class ReadContext { + private final long blobSize; + private final List partStreams; + private final String blobChecksum; + + public ReadContext(long blobSize, List partStreams, String blobChecksum) { + this.blobSize = blobSize; + this.partStreams = partStreams; + this.blobChecksum = blobChecksum; + } + + public String getBlobChecksum() { + return blobChecksum; + } + + public int getNumberOfParts() { + return partStreams.size(); + } + + public long getBlobSize() { + return blobSize; + } + + public List getPartStreams() { + return partStreams; + } +} diff --git a/server/src/main/java/org/opensearch/common/blobstore/stream/read/listener/FileCompletionListener.java b/server/src/main/java/org/opensearch/common/blobstore/stream/read/listener/FileCompletionListener.java new file mode 100644 index 0000000000000..aadd6e2ab304e --- /dev/null +++ b/server/src/main/java/org/opensearch/common/blobstore/stream/read/listener/FileCompletionListener.java @@ -0,0 +1,47 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.common.blobstore.stream.read.listener; + +import org.opensearch.common.annotation.InternalApi; +import org.opensearch.core.action.ActionListener; + +import java.util.concurrent.atomic.AtomicInteger; + +/** + * FileCompletionListener listens for completion of fetch on all the streams for a file, where + * individual streams are handled using {@link FilePartWriter}. The {@link FilePartWriter}(s) + * hold a reference to the file completion listener to be notified. + */ +@InternalApi +class FileCompletionListener implements ActionListener { + + private final int numberOfParts; + private final String fileName; + private final AtomicInteger completedPartsCount; + private final ActionListener completionListener; + + public FileCompletionListener(int numberOfParts, String fileName, ActionListener completionListener) { + this.completedPartsCount = new AtomicInteger(); + this.numberOfParts = numberOfParts; + this.fileName = fileName; + this.completionListener = completionListener; + } + + @Override + public void onResponse(Integer unused) { + if (completedPartsCount.incrementAndGet() == numberOfParts) { + completionListener.onResponse(fileName); + } + } + + @Override + public void onFailure(Exception e) { + completionListener.onFailure(e); + } +} diff --git a/server/src/main/java/org/opensearch/common/blobstore/stream/read/listener/FilePartWriter.java b/server/src/main/java/org/opensearch/common/blobstore/stream/read/listener/FilePartWriter.java new file mode 100644 index 0000000000000..84fd7ed9ffebf --- /dev/null +++ b/server/src/main/java/org/opensearch/common/blobstore/stream/read/listener/FilePartWriter.java @@ -0,0 +1,90 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.common.blobstore.stream.read.listener; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.common.annotation.InternalApi; +import org.opensearch.common.io.Channels; +import org.opensearch.common.io.InputStreamContainer; +import org.opensearch.core.action.ActionListener; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.channels.FileChannel; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * FilePartWriter transfers the provided stream into the specified file path using a {@link FileChannel} + * instance. It performs offset based writes to the file and notifies the {@link FileCompletionListener} on completion. + */ +@InternalApi +class FilePartWriter implements Runnable { + + private final int partNumber; + private final InputStreamContainer blobPartStreamContainer; + private final Path fileLocation; + private final AtomicBoolean anyPartStreamFailed; + private final ActionListener fileCompletionListener; + private static final Logger logger = LogManager.getLogger(FilePartWriter.class); + + // 8 MB buffer for transfer + private static final int BUFFER_SIZE = 8 * 1024 * 2024; + + public FilePartWriter( + int partNumber, + InputStreamContainer blobPartStreamContainer, + Path fileLocation, + AtomicBoolean anyPartStreamFailed, + ActionListener fileCompletionListener + ) { + this.partNumber = partNumber; + this.blobPartStreamContainer = blobPartStreamContainer; + this.fileLocation = fileLocation; + this.anyPartStreamFailed = anyPartStreamFailed; + this.fileCompletionListener = fileCompletionListener; + } + + @Override + public void run() { + // Ensures no writes to the file if any stream fails. + if (anyPartStreamFailed.get() == false) { + try (FileChannel outputFileChannel = FileChannel.open(fileLocation, StandardOpenOption.WRITE, StandardOpenOption.CREATE)) { + try (InputStream inputStream = blobPartStreamContainer.getInputStream()) { + long streamOffset = blobPartStreamContainer.getOffset(); + final byte[] buffer = new byte[BUFFER_SIZE]; + int bytesRead; + while ((bytesRead = inputStream.read(buffer)) != -1) { + Channels.writeToChannel(buffer, 0, bytesRead, outputFileChannel, streamOffset); + streamOffset += bytesRead; + } + } + } catch (IOException e) { + processFailure(e); + return; + } + fileCompletionListener.onResponse(partNumber); + } + } + + void processFailure(Exception e) { + try { + Files.deleteIfExists(fileLocation); + } catch (IOException ex) { + // Die silently + logger.info("Failed to delete file {} on stream failure: {}", fileLocation, ex); + } + if (anyPartStreamFailed.getAndSet(true) == false) { + fileCompletionListener.onFailure(e); + } + } +} diff --git a/server/src/main/java/org/opensearch/common/blobstore/stream/read/listener/ReadContextListener.java b/server/src/main/java/org/opensearch/common/blobstore/stream/read/listener/ReadContextListener.java new file mode 100644 index 0000000000000..4338bddb3fbe7 --- /dev/null +++ b/server/src/main/java/org/opensearch/common/blobstore/stream/read/listener/ReadContextListener.java @@ -0,0 +1,65 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.common.blobstore.stream.read.listener; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.common.annotation.InternalApi; +import org.opensearch.common.blobstore.stream.read.ReadContext; +import org.opensearch.core.action.ActionListener; +import org.opensearch.threadpool.ThreadPool; + +import java.nio.file.Path; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * ReadContextListener orchestrates the async file fetch from the {@link org.opensearch.common.blobstore.BlobContainer} + * using a {@link ReadContext} callback. On response, it spawns off the download using multiple streams which are + * spread across a {@link ThreadPool} executor. + */ +@InternalApi +public class ReadContextListener implements ActionListener { + + private final String fileName; + private final Path fileLocation; + private final ThreadPool threadPool; + private final ActionListener completionListener; + private static final Logger logger = LogManager.getLogger(ReadContextListener.class); + + public ReadContextListener(String fileName, Path fileLocation, ThreadPool threadPool, ActionListener completionListener) { + this.fileName = fileName; + this.fileLocation = fileLocation; + this.threadPool = threadPool; + this.completionListener = completionListener; + } + + @Override + public void onResponse(ReadContext readContext) { + logger.trace("Streams received for blob {}", fileName); + final int numParts = readContext.getNumberOfParts(); + final AtomicBoolean anyPartStreamFailed = new AtomicBoolean(); + FileCompletionListener fileCompletionListener = new FileCompletionListener(numParts, fileName, completionListener); + + for (int partNumber = 0; partNumber < numParts; partNumber++) { + FilePartWriter filePartWriter = new FilePartWriter( + partNumber, + readContext.getPartStreams().get(partNumber), + fileLocation, + anyPartStreamFailed, + fileCompletionListener + ); + threadPool.executor(ThreadPool.Names.GENERIC).submit(filePartWriter); + } + } + + @Override + public void onFailure(Exception e) { + completionListener.onFailure(e); + } +} diff --git a/server/src/main/java/org/opensearch/common/blobstore/stream/read/listener/package-info.java b/server/src/main/java/org/opensearch/common/blobstore/stream/read/listener/package-info.java new file mode 100644 index 0000000000000..fe670fe3eb25c --- /dev/null +++ b/server/src/main/java/org/opensearch/common/blobstore/stream/read/listener/package-info.java @@ -0,0 +1,14 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +/** + * Abstractions for stream based file reads from the blob store. + * Provides listeners for performing the necessary async read operations to perform + * multi stream reads for blobs from the container. + * */ +package org.opensearch.common.blobstore.stream.read.listener; diff --git a/server/src/main/java/org/opensearch/common/blobstore/stream/read/package-info.java b/server/src/main/java/org/opensearch/common/blobstore/stream/read/package-info.java new file mode 100644 index 0000000000000..a9e2ca35c1fa6 --- /dev/null +++ b/server/src/main/java/org/opensearch/common/blobstore/stream/read/package-info.java @@ -0,0 +1,13 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +/** + * Abstractions for stream based file reads from the blob store. + * Provides support for async reads from the blob container. + * */ +package org.opensearch.common.blobstore.stream.read; diff --git a/server/src/test/java/org/opensearch/common/blobstore/stream/read/listener/FileCompletionListenerTests.java b/server/src/test/java/org/opensearch/common/blobstore/stream/read/listener/FileCompletionListenerTests.java new file mode 100644 index 0000000000000..fa13d90f42fa6 --- /dev/null +++ b/server/src/test/java/org/opensearch/common/blobstore/stream/read/listener/FileCompletionListenerTests.java @@ -0,0 +1,58 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.common.blobstore.stream.read.listener; + +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; + +import static org.opensearch.common.blobstore.stream.read.listener.ListenerTestUtils.CountingCompletionListener; + +public class FileCompletionListenerTests extends OpenSearchTestCase { + + public void testFileCompletionListener() { + int numStreams = 10; + String fileName = "test_segment_file"; + CountingCompletionListener completionListener = new CountingCompletionListener(); + FileCompletionListener fileCompletionListener = new FileCompletionListener(numStreams, fileName, completionListener); + + for (int stream = 0; stream < numStreams; stream++) { + // Ensure completion listener called only when all streams are completed + assertEquals(0, completionListener.getResponseCount()); + fileCompletionListener.onResponse(null); + } + + assertEquals(1, completionListener.getResponseCount()); + assertEquals(fileName, completionListener.getResponse()); + } + + public void testFileCompletionListenerFailure() { + int numStreams = 10; + String fileName = "test_segment_file"; + CountingCompletionListener completionListener = new CountingCompletionListener(); + FileCompletionListener fileCompletionListener = new FileCompletionListener(numStreams, fileName, completionListener); + + // Fail the listener initially + IOException exception = new IOException(); + fileCompletionListener.onFailure(exception); + + for (int stream = 0; stream < numStreams - 1; stream++) { + assertEquals(0, completionListener.getResponseCount()); + fileCompletionListener.onResponse(null); + } + + assertEquals(1, completionListener.getFailureCount()); + assertEquals(exception, completionListener.getException()); + assertEquals(0, completionListener.getResponseCount()); + + fileCompletionListener.onFailure(exception); + assertEquals(2, completionListener.getFailureCount()); + assertEquals(exception, completionListener.getException()); + } +} diff --git a/server/src/test/java/org/opensearch/common/blobstore/stream/read/listener/FilePartWriterTests.java b/server/src/test/java/org/opensearch/common/blobstore/stream/read/listener/FilePartWriterTests.java new file mode 100644 index 0000000000000..811566eb5767b --- /dev/null +++ b/server/src/test/java/org/opensearch/common/blobstore/stream/read/listener/FilePartWriterTests.java @@ -0,0 +1,163 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.common.blobstore.stream.read.listener; + +import org.opensearch.common.io.InputStreamContainer; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; + +import static org.opensearch.common.blobstore.stream.read.listener.ListenerTestUtils.CountingCompletionListener; + +public class FilePartWriterTests extends OpenSearchTestCase { + + private Path path; + + @Before + public void init() throws Exception { + path = createTempDir("FilePartWriterTests"); + } + + public void testFilePartWriter() throws Exception { + Path segmentFilePath = path.resolve(UUID.randomUUID().toString()); + int contentLength = 100; + int partNumber = 1; + InputStream inputStream = new ByteArrayInputStream(randomByteArrayOfLength(contentLength)); + InputStreamContainer inputStreamContainer = new InputStreamContainer(inputStream, inputStream.available(), 0); + AtomicBoolean anyStreamFailed = new AtomicBoolean(); + CountingCompletionListener fileCompletionListener = new CountingCompletionListener<>(); + + FilePartWriter filePartWriter = new FilePartWriter( + partNumber, + inputStreamContainer, + segmentFilePath, + anyStreamFailed, + fileCompletionListener + ); + filePartWriter.run(); + + assertTrue(Files.exists(segmentFilePath)); + assertEquals(contentLength, Files.size(segmentFilePath)); + assertEquals(1, fileCompletionListener.getResponseCount()); + assertEquals(Integer.valueOf(partNumber), fileCompletionListener.getResponse()); + } + + public void testFilePartWriterWithOffset() throws Exception { + Path segmentFilePath = path.resolve(UUID.randomUUID().toString()); + int contentLength = 100; + int offset = 10; + int partNumber = 1; + InputStream inputStream = new ByteArrayInputStream(randomByteArrayOfLength(contentLength)); + InputStreamContainer inputStreamContainer = new InputStreamContainer(inputStream, inputStream.available(), offset); + AtomicBoolean anyStreamFailed = new AtomicBoolean(); + CountingCompletionListener fileCompletionListener = new CountingCompletionListener<>(); + + FilePartWriter filePartWriter = new FilePartWriter( + partNumber, + inputStreamContainer, + segmentFilePath, + anyStreamFailed, + fileCompletionListener + ); + filePartWriter.run(); + + assertTrue(Files.exists(segmentFilePath)); + assertEquals(contentLength + offset, Files.size(segmentFilePath)); + assertEquals(1, fileCompletionListener.getResponseCount()); + assertEquals(Integer.valueOf(partNumber), fileCompletionListener.getResponse()); + } + + public void testFilePartWriterLargeInput() throws Exception { + Path segmentFilePath = path.resolve(UUID.randomUUID().toString()); + int contentLength = 20 * 1024 * 1024; + int partNumber = 1; + InputStream inputStream = new ByteArrayInputStream(randomByteArrayOfLength(contentLength)); + InputStreamContainer inputStreamContainer = new InputStreamContainer(inputStream, contentLength, 0); + AtomicBoolean anyStreamFailed = new AtomicBoolean(); + CountingCompletionListener fileCompletionListener = new CountingCompletionListener<>(); + + FilePartWriter filePartWriter = new FilePartWriter( + partNumber, + inputStreamContainer, + segmentFilePath, + anyStreamFailed, + fileCompletionListener + ); + filePartWriter.run(); + + assertTrue(Files.exists(segmentFilePath)); + assertEquals(contentLength, Files.size(segmentFilePath)); + + assertEquals(1, fileCompletionListener.getResponseCount()); + assertEquals(Integer.valueOf(partNumber), fileCompletionListener.getResponse()); + } + + public void testFilePartWriterException() throws Exception { + Path segmentFilePath = path.resolve(UUID.randomUUID().toString()); + int contentLength = 100; + int partNumber = 1; + InputStream inputStream = new ByteArrayInputStream(randomByteArrayOfLength(contentLength)); + InputStreamContainer inputStreamContainer = new InputStreamContainer(inputStream, contentLength, 0); + AtomicBoolean anyStreamFailed = new AtomicBoolean(); + CountingCompletionListener fileCompletionListener = new CountingCompletionListener<>(); + + IOException ioException = new IOException(); + FilePartWriter filePartWriter = new FilePartWriter( + partNumber, + inputStreamContainer, + segmentFilePath, + anyStreamFailed, + fileCompletionListener + ); + assertFalse(anyStreamFailed.get()); + filePartWriter.processFailure(ioException); + + assertTrue(anyStreamFailed.get()); + assertFalse(Files.exists(segmentFilePath)); + + // Fail stream again to simulate another stream failure for same file + filePartWriter.processFailure(ioException); + + assertTrue(anyStreamFailed.get()); + assertFalse(Files.exists(segmentFilePath)); + + assertEquals(0, fileCompletionListener.getResponseCount()); + assertEquals(1, fileCompletionListener.getFailureCount()); + assertEquals(ioException, fileCompletionListener.getException()); + } + + public void testFilePartWriterStreamFailed() throws Exception { + Path segmentFilePath = path.resolve(UUID.randomUUID().toString()); + int contentLength = 100; + int partNumber = 1; + InputStream inputStream = new ByteArrayInputStream(randomByteArrayOfLength(contentLength)); + InputStreamContainer inputStreamContainer = new InputStreamContainer(inputStream, inputStream.available(), 0); + AtomicBoolean anyStreamFailed = new AtomicBoolean(true); + CountingCompletionListener fileCompletionListener = new CountingCompletionListener<>(); + + FilePartWriter filePartWriter = new FilePartWriter( + partNumber, + inputStreamContainer, + segmentFilePath, + anyStreamFailed, + fileCompletionListener + ); + filePartWriter.run(); + + assertFalse(Files.exists(segmentFilePath)); + assertEquals(0, fileCompletionListener.getResponseCount()); + } +} diff --git a/server/src/test/java/org/opensearch/common/blobstore/stream/read/listener/ListenerTestUtils.java b/server/src/test/java/org/opensearch/common/blobstore/stream/read/listener/ListenerTestUtils.java new file mode 100644 index 0000000000000..1e9450c83e3ab --- /dev/null +++ b/server/src/test/java/org/opensearch/common/blobstore/stream/read/listener/ListenerTestUtils.java @@ -0,0 +1,56 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.common.blobstore.stream.read.listener; + +import org.opensearch.core.action.ActionListener; + +/** + * Utility class containing common functionality for read listener based tests + */ +public class ListenerTestUtils { + + /** + * CountingCompletionListener acts as a verification instance for wrapping listener based calls. + * Keeps track of the last response, failure and count of response and failure invocations. + */ + static class CountingCompletionListener implements ActionListener { + private int responseCount; + private int failureCount; + private T response; + private Exception exception; + + @Override + public void onResponse(T response) { + this.response = response; + responseCount++; + } + + @Override + public void onFailure(Exception e) { + exception = e; + failureCount++; + } + + public int getResponseCount() { + return responseCount; + } + + public int getFailureCount() { + return failureCount; + } + + public T getResponse() { + return response; + } + + public Exception getException() { + return exception; + } + } +} diff --git a/server/src/test/java/org/opensearch/common/blobstore/stream/read/listener/ReadContextListenerTests.java b/server/src/test/java/org/opensearch/common/blobstore/stream/read/listener/ReadContextListenerTests.java new file mode 100644 index 0000000000000..f785b5f1191b4 --- /dev/null +++ b/server/src/test/java/org/opensearch/common/blobstore/stream/read/listener/ReadContextListenerTests.java @@ -0,0 +1,124 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.common.blobstore.stream.read.listener; + +import org.opensearch.action.LatchedActionListener; +import org.opensearch.action.support.PlainActionFuture; +import org.opensearch.common.blobstore.stream.read.ReadContext; +import org.opensearch.common.io.InputStreamContainer; +import org.opensearch.core.action.ActionListener; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.CountDownLatch; + +import static org.opensearch.common.blobstore.stream.read.listener.ListenerTestUtils.CountingCompletionListener; + +public class ReadContextListenerTests extends OpenSearchTestCase { + + private Path path; + private static ThreadPool threadPool; + private static final int NUMBER_OF_PARTS = 5; + private static final int PART_SIZE = 10; + private static final String TEST_SEGMENT_FILE = "test_segment_file"; + + @BeforeClass + public static void setup() { + threadPool = new TestThreadPool(ReadContextListenerTests.class.getName()); + } + + @AfterClass + public static void cleanup() { + threadPool.shutdown(); + } + + @Before + public void init() throws Exception { + path = createTempDir("ReadContextListenerTests"); + } + + public void testReadContextListener() throws InterruptedException, IOException { + Path fileLocation = path.resolve(UUID.randomUUID().toString()); + List blobPartStreams = initializeBlobPartStreams(); + CountDownLatch countDownLatch = new CountDownLatch(1); + ActionListener completionListener = new LatchedActionListener<>(new PlainActionFuture<>(), countDownLatch); + ReadContextListener readContextListener = new ReadContextListener(TEST_SEGMENT_FILE, fileLocation, threadPool, completionListener); + ReadContext readContext = new ReadContext((long) PART_SIZE * NUMBER_OF_PARTS, blobPartStreams, null); + readContextListener.onResponse(readContext); + + countDownLatch.await(); + + assertTrue(Files.exists(fileLocation)); + assertEquals(NUMBER_OF_PARTS * PART_SIZE, Files.size(fileLocation)); + } + + public void testReadContextListenerFailure() throws InterruptedException { + Path fileLocation = path.resolve(UUID.randomUUID().toString()); + List blobPartStreams = initializeBlobPartStreams(); + CountDownLatch countDownLatch = new CountDownLatch(1); + ActionListener completionListener = new LatchedActionListener<>(new PlainActionFuture<>(), countDownLatch); + ReadContextListener readContextListener = new ReadContextListener(TEST_SEGMENT_FILE, fileLocation, threadPool, completionListener); + InputStream badInputStream = new InputStream() { + + @Override + public int read(byte[] b, int off, int len) throws IOException { + return read(); + } + + @Override + public int read() throws IOException { + throw new IOException(); + } + + @Override + public int available() { + return PART_SIZE; + } + }; + + blobPartStreams.add(NUMBER_OF_PARTS, new InputStreamContainer(badInputStream, PART_SIZE, PART_SIZE * NUMBER_OF_PARTS)); + ReadContext readContext = new ReadContext((long) (PART_SIZE + 1) * NUMBER_OF_PARTS, blobPartStreams, null); + readContextListener.onResponse(readContext); + + countDownLatch.await(); + + assertFalse(Files.exists(fileLocation)); + } + + public void testReadContextListenerException() { + Path fileLocation = path.resolve(UUID.randomUUID().toString()); + CountingCompletionListener listener = new CountingCompletionListener(); + ReadContextListener readContextListener = new ReadContextListener(TEST_SEGMENT_FILE, fileLocation, threadPool, listener); + IOException exception = new IOException(); + readContextListener.onFailure(exception); + assertEquals(1, listener.getFailureCount()); + assertEquals(exception, listener.getException()); + } + + private List initializeBlobPartStreams() { + List blobPartStreams = new ArrayList<>(); + for (int partNumber = 0; partNumber < NUMBER_OF_PARTS; partNumber++) { + InputStream testStream = new ByteArrayInputStream(randomByteArrayOfLength(PART_SIZE)); + blobPartStreams.add(new InputStreamContainer(testStream, PART_SIZE, (long) partNumber * PART_SIZE)); + } + return blobPartStreams; + } +} From 04c90c7fed76e8f107bc5944eb973fa4c2034b67 Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Thu, 31 Aug 2023 19:56:42 -0700 Subject: [PATCH 26/37] Mute RemoteIndexShardTests primary promotion flaky tests (#9679) --- .../org/opensearch/index/shard/RemoteIndexShardTests.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java index 9dcecbe1059b6..33159b85ec640 100644 --- a/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java @@ -64,18 +64,22 @@ protected ReplicationGroup getReplicationGroup(int numberOfReplicas) throws IOEx return createGroup(numberOfReplicas, settings, indexMapping, new NRTReplicationEngineFactory(), createTempDir()); } + @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/9624") public void testNRTReplicaWithRemoteStorePromotedAsPrimaryRefreshRefresh() throws Exception { testNRTReplicaWithRemoteStorePromotedAsPrimary(false, false); } + @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/9624") public void testNRTReplicaWithRemoteStorePromotedAsPrimaryRefreshCommit() throws Exception { testNRTReplicaWithRemoteStorePromotedAsPrimary(false, true); } + @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/9624") public void testNRTReplicaWithRemoteStorePromotedAsPrimaryCommitRefresh() throws Exception { testNRTReplicaWithRemoteStorePromotedAsPrimary(true, false); } + @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/9624") public void testNRTReplicaWithRemoteStorePromotedAsPrimaryCommitCommit() throws Exception { testNRTReplicaWithRemoteStorePromotedAsPrimary(true, true); } From f5d3fd2483f58cdbb0d77cf53a1e5439b3cc860a Mon Sep 17 00:00:00 2001 From: Ashish Date: Fri, 1 Sep 2023 17:25:39 +0530 Subject: [PATCH 27/37] Introduce cluster default remote translog buffer interval setting (#9584) Signed-off-by: Ashish Singh --- CHANGELOG.md | 1 + .../opensearch/index/shard/IndexShardIT.java | 3 +- .../opensearch/remotestore/RemoteStoreIT.java | 169 ++++++++++++++++++ .../common/settings/ClusterSettings.java | 3 +- .../concurrent/BufferedAsyncIOProcessor.java | 4 + .../org/opensearch/index/IndexModule.java | 6 +- .../org/opensearch/index/IndexService.java | 8 +- .../org/opensearch/index/IndexSettings.java | 7 + .../opensearch/index/shard/IndexShard.java | 20 ++- .../opensearch/indices/IndicesService.java | 31 +++- .../opensearch/index/IndexModuleTests.java | 3 +- .../indices/IndicesServiceTests.java | 5 + .../index/shard/IndexShardTestCase.java | 3 +- 13 files changed, 252 insertions(+), 11 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 80ab0fb87e609..ffefcd416da1c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -98,6 +98,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Decouple replication lag from logic to fail stale replicas ([#9507](https://github.com/opensearch-project/OpenSearch/pull/9507)) - Expose DelimitedTermFrequencyTokenFilter to allow providing term frequencies along with terms ([#9479](https://github.com/opensearch-project/OpenSearch/pull/9479)) - APIs for performing async blob reads and async downloads from the repository using multiple streams ([#9592](https://github.com/opensearch-project/OpenSearch/issues/9592)) +- Introduce cluster default remote translog buffer interval setting ([#9584](https://github.com/opensearch-project/OpenSearch/pull/9584)) ### Dependencies - Bump `org.apache.logging.log4j:log4j-core` from 2.17.1 to 2.20.0 ([#8307](https://github.com/opensearch-project/OpenSearch/pull/8307)) diff --git a/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java b/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java index 98a1c71d8e2b3..bb08b19df765b 100644 --- a/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java @@ -701,7 +701,8 @@ public static final IndexShard newIndexShard( (indexSettings, shardRouting) -> new InternalTranslogFactory(), SegmentReplicationCheckpointPublisher.EMPTY, null, - null + null, + () -> IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL ); } diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java index 9a2948861e967..b87ccdb22d014 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java @@ -9,11 +9,21 @@ package org.opensearch.remotestore; import org.opensearch.action.admin.indices.delete.DeleteIndexRequest; +import org.opensearch.action.admin.indices.get.GetIndexRequest; +import org.opensearch.action.admin.indices.get.GetIndexResponse; import org.opensearch.action.admin.indices.recovery.RecoveryResponse; +import org.opensearch.action.admin.indices.settings.put.UpdateSettingsRequest; import org.opensearch.action.index.IndexResponse; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.routing.RecoverySource; import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.concurrent.BufferedAsyncIOProcessor; +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.indices.IndicesService; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.plugins.Plugin; import org.opensearch.test.OpenSearchIntegTestCase; @@ -26,9 +36,11 @@ import java.util.Collection; import java.util.Map; import java.util.Optional; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import static org.opensearch.index.shard.RemoteStoreRefreshListener.LAST_N_METADATA_FILES_TO_KEEP; +import static org.opensearch.indices.IndicesService.CLUSTER_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount; import static org.hamcrest.Matchers.comparesEqualTo; @@ -184,4 +196,161 @@ public void testStaleCommitDeletionWithoutInvokeFlush() throws Exception { // We also allow (numberOfIterations + 1) as index creation also triggers refresh. MatcherAssert.assertThat(actualFileCount, is(oneOf(numberOfIterations - 1, numberOfIterations, numberOfIterations + 1))); } + + /** + * Tests that when the index setting is not passed during index creation, the buffer interval picked up is the cluster + * default. + */ + public void testDefaultBufferInterval() throws ExecutionException, InterruptedException { + setupRepo(); + String clusterManagerName = internalCluster().getClusterManagerName(); + String dataNode = internalCluster().startDataOnlyNodes(1).get(0); + createIndex(INDEX_NAME); + ensureYellowAndNoInitializingShards(INDEX_NAME); + ensureGreen(INDEX_NAME); + assertClusterRemoteBufferInterval(IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL, dataNode); + + IndexShard indexShard = getIndexShard(dataNode); + assertTrue(indexShard.getTranslogSyncProcessor() instanceof BufferedAsyncIOProcessor); + assertBufferInterval(IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL, indexShard); + + // Next, we change the default buffer interval and the same should reflect in the buffer interval of the index created + TimeValue clusterBufferInterval = TimeValue.timeValueSeconds(randomIntBetween(100, 200)); + client(clusterManagerName).admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings(Settings.builder().put(CLUSTER_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey(), clusterBufferInterval)) + .get(); + assertBufferInterval(clusterBufferInterval, indexShard); + clearClusterBufferIntervalSetting(clusterManagerName); + } + + /** + * This tests multiple cases where the index setting is passed during the index creation with multiple combinations + * with and without cluster default. + */ + public void testOverriddenBufferInterval() throws ExecutionException, InterruptedException { + setupRepo(); + String clusterManagerName = internalCluster().getClusterManagerName(); + String dataNode = internalCluster().startDataOnlyNodes(1).get(0); + + TimeValue bufferInterval = TimeValue.timeValueSeconds(randomIntBetween(0, 100)); + Settings indexSettings = Settings.builder() + .put(indexSettings()) + .put(IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey(), bufferInterval) + .build(); + createIndex(INDEX_NAME, indexSettings); + ensureYellowAndNoInitializingShards(INDEX_NAME); + ensureGreen(INDEX_NAME); + + IndexShard indexShard = getIndexShard(dataNode); + assertTrue(indexShard.getTranslogSyncProcessor() instanceof BufferedAsyncIOProcessor); + assertBufferInterval(bufferInterval, indexShard); + + // Set the cluster default with a different value, validate that the buffer interval is still the overridden value + TimeValue clusterBufferInterval = TimeValue.timeValueSeconds(randomIntBetween(100, 200)); + client(clusterManagerName).admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings(Settings.builder().put(CLUSTER_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey(), clusterBufferInterval)) + .get(); + assertBufferInterval(bufferInterval, indexShard); + + // Set the index setting (index.remote_store.translog.buffer_interval) with a different value and validate that + // the buffer interval is updated + bufferInterval = TimeValue.timeValueSeconds(bufferInterval.seconds() + randomIntBetween(1, 100)); + client(clusterManagerName).admin() + .indices() + .updateSettings( + new UpdateSettingsRequest(INDEX_NAME).settings( + Settings.builder().put(IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey(), bufferInterval) + ) + ) + .get(); + assertBufferInterval(bufferInterval, indexShard); + + // Set the index setting (index.remote_store.translog.buffer_interval) with null and validate the buffer interval + // which will be the cluster default now. + client(clusterManagerName).admin() + .indices() + .updateSettings( + new UpdateSettingsRequest(INDEX_NAME).settings( + Settings.builder().putNull(IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey()) + ) + ) + .get(); + assertBufferInterval(clusterBufferInterval, indexShard); + clearClusterBufferIntervalSetting(clusterManagerName); + } + + /** + * This tests validation which kicks in during index creation failing creation if the value is less than minimum allowed value. + */ + public void testOverriddenBufferIntervalValidation() { + setupRepo(); + TimeValue bufferInterval = TimeValue.timeValueSeconds(-1); + Settings indexSettings = Settings.builder() + .put(indexSettings()) + .put(IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey(), bufferInterval) + .build(); + IllegalArgumentException exceptionDuringCreateIndex = assertThrows( + IllegalArgumentException.class, + () -> createIndex(INDEX_NAME, indexSettings) + ); + assertEquals( + "failed to parse value [-1] for setting [index.remote_store.translog.buffer_interval], must be >= [0ms]", + exceptionDuringCreateIndex.getMessage() + ); + } + + /** + * This tests validation of the cluster setting when being set. + */ + public void testClusterBufferIntervalValidation() { + String clusterManagerName = internalCluster().startClusterManagerOnlyNode(); + setupRepo(false); + IllegalArgumentException exception = assertThrows( + IllegalArgumentException.class, + () -> client(clusterManagerName).admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings( + Settings.builder().put(CLUSTER_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey(), TimeValue.timeValueSeconds(-1)) + ) + .get() + ); + assertEquals( + "failed to parse value [-1] for setting [cluster.remote_store.translog.buffer_interval], must be >= [0ms]", + exception.getMessage() + ); + } + + private IndexShard getIndexShard(String dataNode) throws ExecutionException, InterruptedException { + String clusterManagerName = internalCluster().getClusterManagerName(); + IndicesService indicesService = internalCluster().getInstance(IndicesService.class, dataNode); + GetIndexResponse getIndexResponse = client(clusterManagerName).admin().indices().getIndex(new GetIndexRequest()).get(); + String uuid = getIndexResponse.getSettings().get(INDEX_NAME).get(IndexMetadata.SETTING_INDEX_UUID); + IndexService indexService = indicesService.indexService(new Index(INDEX_NAME, uuid)); + return indexService.getShard(0); + } + + private void assertClusterRemoteBufferInterval(TimeValue expectedBufferInterval, String dataNode) { + IndicesService indicesService = internalCluster().getInstance(IndicesService.class, dataNode); + assertEquals(expectedBufferInterval, indicesService.getClusterRemoteTranslogBufferInterval()); + } + + private void assertBufferInterval(TimeValue expectedBufferInterval, IndexShard indexShard) { + assertEquals( + expectedBufferInterval, + ((BufferedAsyncIOProcessor) indexShard.getTranslogSyncProcessor()).getBufferIntervalSupplier().get() + ); + } + + private void clearClusterBufferIntervalSetting(String clusterManagerName) { + client(clusterManagerName).admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings(Settings.builder().putNull(CLUSTER_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey())) + .get(); + } } diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index 8093eab696779..05938914b019f 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -686,7 +686,8 @@ public void apply(Settings value, Settings current, Settings previous) { List.of( IndicesService.CLUSTER_REMOTE_STORE_ENABLED_SETTING, IndicesService.CLUSTER_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING, - IndicesService.CLUSTER_REMOTE_TRANSLOG_REPOSITORY_SETTING + IndicesService.CLUSTER_REMOTE_TRANSLOG_REPOSITORY_SETTING, + IndicesService.CLUSTER_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING ), List.of(FeatureFlags.CONCURRENT_SEGMENT_SEARCH), List.of( diff --git a/server/src/main/java/org/opensearch/common/util/concurrent/BufferedAsyncIOProcessor.java b/server/src/main/java/org/opensearch/common/util/concurrent/BufferedAsyncIOProcessor.java index 7079aa705d126..be2029b2e7c62 100644 --- a/server/src/main/java/org/opensearch/common/util/concurrent/BufferedAsyncIOProcessor.java +++ b/server/src/main/java/org/opensearch/common/util/concurrent/BufferedAsyncIOProcessor.java @@ -92,4 +92,8 @@ private TimeValue getBufferInterval() { protected abstract String getBufferProcessThreadPoolName(); + // Exclusively for testing, please do not use it elsewhere. + public Supplier getBufferIntervalSupplier() { + return bufferIntervalSupplier; + } } diff --git a/server/src/main/java/org/opensearch/index/IndexModule.java b/server/src/main/java/org/opensearch/index/IndexModule.java index d1e071eedb39e..8692876412ea9 100644 --- a/server/src/main/java/org/opensearch/index/IndexModule.java +++ b/server/src/main/java/org/opensearch/index/IndexModule.java @@ -601,7 +601,8 @@ public IndexService newIndexService( ValuesSourceRegistry valuesSourceRegistry, IndexStorePlugin.DirectoryFactory remoteDirectoryFactory, BiFunction translogFactorySupplier, - Supplier clusterDefaultRefreshIntervalSupplier + Supplier clusterDefaultRefreshIntervalSupplier, + Supplier clusterRemoteTranslogBufferIntervalSupplier ) throws IOException { final IndexEventListener eventListener = freeze(); Function> readerWrapperFactory = indexReaderWrapper @@ -658,7 +659,8 @@ public IndexService newIndexService( valuesSourceRegistry, recoveryStateFactory, translogFactorySupplier, - clusterDefaultRefreshIntervalSupplier + clusterDefaultRefreshIntervalSupplier, + clusterRemoteTranslogBufferIntervalSupplier ); success = true; return indexService; diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index e5028ff2ecff9..80ead0a333ba3 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -177,6 +177,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust private final ValuesSourceRegistry valuesSourceRegistry; private final BiFunction translogFactorySupplier; private final Supplier clusterDefaultRefreshIntervalSupplier; + private final Supplier clusterRemoteTranslogBufferIntervalSupplier; public IndexService( IndexSettings indexSettings, @@ -210,7 +211,8 @@ public IndexService( ValuesSourceRegistry valuesSourceRegistry, IndexStorePlugin.RecoveryStateFactory recoveryStateFactory, BiFunction translogFactorySupplier, - Supplier clusterDefaultRefreshIntervalSupplier + Supplier clusterDefaultRefreshIntervalSupplier, + Supplier clusterRemoteTranslogBufferIntervalSupplier ) { super(indexSettings); this.allowExpensiveQueries = allowExpensiveQueries; @@ -284,6 +286,7 @@ public IndexService( this.globalCheckpointTask = new AsyncGlobalCheckpointTask(this); this.retentionLeaseSyncTask = new AsyncRetentionLeaseSyncTask(this); this.translogFactorySupplier = translogFactorySupplier; + this.clusterRemoteTranslogBufferIntervalSupplier = clusterRemoteTranslogBufferIntervalSupplier; updateFsyncTaskIfNecessary(); } @@ -512,7 +515,8 @@ public synchronized IndexShard createShard( translogFactorySupplier, this.indexSettings.isSegRepEnabled() ? checkpointPublisher : null, remoteStore, - remoteStoreStatsTrackerFactory + remoteStoreStatsTrackerFactory, + clusterRemoteTranslogBufferIntervalSupplier ); eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created"); eventListener.afterIndexShardCreated(indexShard); diff --git a/server/src/main/java/org/opensearch/index/IndexSettings.java b/server/src/main/java/org/opensearch/index/IndexSettings.java index 4609b1f994737..9a9e0d8b1f913 100644 --- a/server/src/main/java/org/opensearch/index/IndexSettings.java +++ b/server/src/main/java/org/opensearch/index/IndexSettings.java @@ -1197,6 +1197,13 @@ public TimeValue getRemoteTranslogUploadBufferInterval() { return remoteTranslogUploadBufferInterval; } + /** + * Returns true iff the remote translog buffer interval setting exists or in other words is explicitly set. + */ + public boolean isRemoteTranslogBufferIntervalExplicit() { + return INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.exists(settings); + } + public void setRemoteTranslogUploadBufferInterval(TimeValue remoteTranslogUploadBufferInterval) { this.remoteTranslogUploadBufferInterval = remoteTranslogUploadBufferInterval; } diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index 0d51126ace8c7..e398cab23a085 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -361,7 +361,8 @@ public IndexShard( final BiFunction translogFactorySupplier, @Nullable final SegmentReplicationCheckpointPublisher checkpointPublisher, @Nullable final Store remoteStore, - final RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory + final RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory, + final Supplier clusterRemoteTranslogBufferIntervalSupplier ) throws IOException { super(shardRouting.shardId(), indexSettings); assert shardRouting.initializing(); @@ -382,7 +383,7 @@ public IndexShard( threadPool, this::getEngine, indexSettings.isRemoteTranslogStoreEnabled(), - indexSettings::getRemoteTranslogUploadBufferInterval + () -> getRemoteTranslogUploadBufferInterval(clusterRemoteTranslogBufferIntervalSupplier) ); this.mapperService = mapperService; this.indexCache = indexCache; @@ -4117,6 +4118,8 @@ private static AsyncIOProcessor createTranslogSyncProcessor( boolean bufferAsyncIoProcessor, Supplier bufferIntervalSupplier ) { + assert bufferAsyncIoProcessor == false || Objects.nonNull(bufferIntervalSupplier) + : "If bufferAsyncIoProcessor is true, then the bufferIntervalSupplier needs to be non null"; ThreadContext threadContext = threadPool.getThreadContext(); CheckedConsumer>>, IOException> writeConsumer = candidates -> { try { @@ -4911,4 +4914,17 @@ RetentionLeaseSyncer getRetentionLeaseSyncer() { public GatedCloseable getSegmentInfosSnapshot() { return getEngine().getSegmentInfosSnapshot(); } + + private TimeValue getRemoteTranslogUploadBufferInterval(Supplier clusterRemoteTranslogBufferIntervalSupplier) { + assert Objects.nonNull(clusterRemoteTranslogBufferIntervalSupplier) : "remote translog buffer interval supplier is null"; + if (indexSettings().isRemoteTranslogBufferIntervalExplicit()) { + return indexSettings().getRemoteTranslogUploadBufferInterval(); + } + return clusterRemoteTranslogBufferIntervalSupplier.get(); + } + + // Exclusively for testing, please do not use it elsewhere. + public AsyncIOProcessor getTranslogSyncProcessor() { + return translogSyncProcessor; + } } diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index 13a82ce5bdf0d..c31acd5a0f966 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -69,6 +69,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.BigArrays; +import org.opensearch.common.util.FeatureFlags; import org.opensearch.common.util.concurrent.AbstractRefCounted; import org.opensearch.common.util.concurrent.AbstractRunnable; import org.opensearch.common.util.concurrent.OpenSearchExecutors; @@ -272,6 +273,17 @@ public class IndicesService extends AbstractLifecycleComponent Property.Final ); + /** + * Used to specify the default translog buffer interval for remote store backed indexes. + */ + public static final Setting CLUSTER_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING = Setting.timeSetting( + "cluster.remote_store.translog.buffer_interval", + IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL, + IndexSettings.MINIMUM_REMOTE_TRANSLOG_BUFFER_INTERVAL, + Property.NodeScope, + Property.Dynamic + ); + /** * This setting is used to set the refresh interval when the {@code index.refresh_interval} index setting is not * provided during index creation or when the existing {@code index.refresh_interval} index setting is set as null. @@ -349,6 +361,7 @@ public class IndicesService extends AbstractLifecycleComponent private final IndexStorePlugin.DirectoryFactory remoteDirectoryFactory; private final BiFunction translogFactorySupplier; private volatile TimeValue clusterDefaultRefreshInterval; + private volatile TimeValue clusterRemoteTranslogBufferInterval; private final FileCacheCleaner fileCacheCleaner; @Override @@ -473,6 +486,12 @@ protected void closeInternal() { this.clusterDefaultRefreshInterval = CLUSTER_DEFAULT_INDEX_REFRESH_INTERVAL_SETTING.get(clusterService.getSettings()); clusterService.getClusterSettings() .addSettingsUpdateConsumer(CLUSTER_DEFAULT_INDEX_REFRESH_INTERVAL_SETTING, this::onRefreshIntervalUpdate); + + if (FeatureFlags.isEnabled(FeatureFlags.REMOTE_STORE)) { + this.clusterRemoteTranslogBufferInterval = CLUSTER_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.get(clusterService.getSettings()); + clusterService.getClusterSettings() + .addSettingsUpdateConsumer(CLUSTER_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING, this::setClusterRemoteTranslogBufferInterval); + } } /** @@ -867,7 +886,8 @@ private synchronized IndexService createIndexService( valuesSourceRegistry, remoteDirectoryFactory, translogFactorySupplier, - this::getClusterDefaultRefreshInterval + this::getClusterDefaultRefreshInterval, + this::getClusterRemoteTranslogBufferInterval ); } @@ -1981,4 +2001,13 @@ private static void validateRefreshIntervalSettings(TimeValue minimumRefreshInte private TimeValue getClusterDefaultRefreshInterval() { return this.clusterDefaultRefreshInterval; } + + // Exclusively for testing, please do not use it elsewhere. + public TimeValue getClusterRemoteTranslogBufferInterval() { + return clusterRemoteTranslogBufferInterval; + } + + private void setClusterRemoteTranslogBufferInterval(TimeValue clusterRemoteTranslogBufferInterval) { + this.clusterRemoteTranslogBufferInterval = clusterRemoteTranslogBufferInterval; + } } diff --git a/server/src/test/java/org/opensearch/index/IndexModuleTests.java b/server/src/test/java/org/opensearch/index/IndexModuleTests.java index 3b43ede3c0a6d..fadce99416cbc 100644 --- a/server/src/test/java/org/opensearch/index/IndexModuleTests.java +++ b/server/src/test/java/org/opensearch/index/IndexModuleTests.java @@ -255,7 +255,8 @@ private IndexService newIndexService(IndexModule module) throws IOException { null, new RemoteSegmentStoreDirectoryFactory(() -> repositoriesService, threadPool), translogFactorySupplier, - () -> IndexSettings.DEFAULT_REFRESH_INTERVAL + () -> IndexSettings.DEFAULT_REFRESH_INTERVAL, + () -> IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL ); } diff --git a/server/src/test/java/org/opensearch/indices/IndicesServiceTests.java b/server/src/test/java/org/opensearch/indices/IndicesServiceTests.java index 1b547ff702d98..d22ad06245687 100644 --- a/server/src/test/java/org/opensearch/indices/IndicesServiceTests.java +++ b/server/src/test/java/org/opensearch/indices/IndicesServiceTests.java @@ -619,4 +619,9 @@ public void testConflictingEngineFactories() { ".*multiple engine factories provided for \\[foobar/.*\\]: \\[.*FooEngineFactory\\],\\[.*BarEngineFactory\\].*"; assertThat(e, hasToString(new RegexMatcher(pattern))); } + + public void testClusterRemoteTranslogBufferIntervalNull() { + IndicesService indicesService = getIndicesService(); + assertNull(indicesService.getClusterRemoteTranslogBufferInterval()); + } } diff --git a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java index b14a42e1e78ae..474acc764620d 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java @@ -695,7 +695,8 @@ protected IndexShard newShard( translogFactorySupplier, checkpointPublisher, remoteStore, - remoteStoreStatsTrackerFactory + remoteStoreStatsTrackerFactory, + () -> IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL ); indexShard.addShardFailureCallback(DEFAULT_SHARD_FAILURE_HANDLER); if (remoteStoreStatsTrackerFactory != null) { From 2fb46940ddeaadf186ba60a4f6a9fc7fe77259f0 Mon Sep 17 00:00:00 2001 From: panguixin Date: Sat, 2 Sep 2023 01:21:41 +0800 Subject: [PATCH 28/37] remove redundent column headers in pit segments response (#9615) Signed-off-by: panguixin --- .../rest/action/cat/RestPitSegmentsAction.java | 18 +----------------- 1 file changed, 1 insertion(+), 17 deletions(-) diff --git a/server/src/main/java/org/opensearch/rest/action/cat/RestPitSegmentsAction.java b/server/src/main/java/org/opensearch/rest/action/cat/RestPitSegmentsAction.java index ba9606e8eb444..5fc6c961b4637 100644 --- a/server/src/main/java/org/opensearch/rest/action/cat/RestPitSegmentsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/cat/RestPitSegmentsAction.java @@ -120,23 +120,7 @@ private Table buildTable(final RestRequest request, Map i Table table = getTableWithHeader(request); DiscoveryNodes nodes = this.nodesInCluster.get(); - table.startRow(); - table.addCell("index", "default:true;alias:i,idx;desc:index name"); - table.addCell("shard", "default:true;alias:s,sh;desc:shard name"); - table.addCell("prirep", "alias:p,pr,primaryOrReplica;default:true;desc:primary or replica"); - table.addCell("ip", "default:true;desc:ip of node where it lives"); - table.addCell("id", "default:false;desc:unique id of node where it lives"); - table.addCell("segment", "default:true;alias:seg;desc:segment name"); - table.addCell("generation", "default:true;alias:g,gen;text-align:right;desc:segment generation"); - table.addCell("docs.count", "default:true;alias:dc,docsCount;text-align:right;desc:number of docs in segment"); - table.addCell("docs.deleted", "default:true;alias:dd,docsDeleted;text-align:right;desc:number of deleted docs in segment"); - table.addCell("size", "default:true;alias:si;text-align:right;desc:segment size in bytes"); - table.addCell("size.memory", "default:true;alias:sm,sizeMemory;text-align:right;desc:segment memory in bytes"); - table.addCell("committed", "default:true;alias:ic,isCommitted;desc:is segment committed"); - table.addCell("searchable", "default:true;alias:is,isSearchable;desc:is segment searched"); - table.addCell("version", "default:true;alias:v,ver;desc:version"); - table.addCell("compound", "default:true;alias:ico,isCompound;desc:is segment compound"); - table.endRow(); + for (IndexSegments indexSegments : indicesSegments.values()) { Map shards = indexSegments.getShards(); for (IndexShardSegments indexShardSegments : shards.values()) { From 96e851b44622477d15fc8e37b02f9a5dc7a07190 Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Fri, 1 Sep 2023 11:08:18 -0700 Subject: [PATCH 29/37] [Segment Replication] Allow segment replication with on disk files not referenced by reader with matching checksum (#9630) * [Segment Replication] Allow segment replication with on disk files not referenced by reader with matching checksum Signed-off-by: Suraj Singh * Address review comments Signed-off-by: Suraj Singh * Address review comments Signed-off-by: Suraj Singh * Spotless fix Signed-off-by: Suraj Singh --------- Signed-off-by: Suraj Singh --- .../replication/SegmentReplicationTarget.java | 47 ++++++- .../index/shard/RemoteIndexShardTests.java | 132 ++++++++++++++++++ 2 files changed, 176 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java index 7a5f9608dace0..5ae480b7d63a4 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java @@ -9,11 +9,14 @@ package org.opensearch.indices.replication; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.IndexFormatTooNewException; import org.apache.lucene.index.IndexFormatTooOldException; import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.store.AlreadyClosedException; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; import org.opensearch.OpenSearchCorruptionException; import org.opensearch.OpenSearchException; import org.opensearch.action.StepListener; @@ -33,8 +36,11 @@ import org.opensearch.indices.replication.common.ReplicationTarget; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.List; import java.util.Locale; +import java.util.Set; +import java.util.stream.Collectors; /** * Represents the target of a replication event. @@ -178,7 +184,27 @@ private List getFiles(CheckpointInfoResponse checkpointInfo) cancellableThreads.checkForCancel(); state.setStage(SegmentReplicationState.Stage.FILE_DIFF); final Store.RecoveryDiff diff = Store.segmentReplicationDiff(checkpointInfo.getMetadataMap(), indexShard.getSegmentMetadataMap()); - logger.trace(() -> new ParameterizedMessage("Replication diff for checkpoint {} {}", checkpointInfo.getCheckpoint(), diff)); + // local files + final Set localFiles = Set.of(indexShard.store().directory().listAll()); + // set of local files that can be reused + final Set reuseFiles = diff.missing.stream() + .filter(storeFileMetadata -> localFiles.contains(storeFileMetadata.name())) + .filter(this::validateLocalChecksum) + .map(StoreFileMetadata::name) + .collect(Collectors.toSet()); + + final List missingFiles = diff.missing.stream() + .filter(md -> reuseFiles.contains(md.name()) == false) + .collect(Collectors.toList()); + + logger.trace( + () -> new ParameterizedMessage( + "Replication diff for checkpoint {} {} {}", + checkpointInfo.getCheckpoint(), + missingFiles, + diff.different + ) + ); /* * Segments are immutable. So if the replica has any segments with the same name that differ from the one in the incoming * snapshot from source that means the local copy of the segment has been corrupted/changed in some way and we throw an @@ -194,10 +220,25 @@ private List getFiles(CheckpointInfoResponse checkpointInfo) ); } - for (StoreFileMetadata file : diff.missing) { + for (StoreFileMetadata file : missingFiles) { state.getIndex().addFileDetail(file.name(), file.length(), false); } - return diff.missing; + return missingFiles; + } + + private boolean validateLocalChecksum(StoreFileMetadata file) { + try (IndexInput indexInput = indexShard.store().directory().openInput(file.name(), IOContext.DEFAULT)) { + String checksum = Store.digestToString(CodecUtil.retrieveChecksum(indexInput)); + if (file.checksum().equals(checksum)) { + return true; + } else { + // clear local copy with mismatch. Safe because file is not referenced by active reader. + store.deleteQuiet(file.name()); + return false; + } + } catch (IOException e) { + throw new UncheckedIOException("Error reading " + file, e); + } } private void finalizeReplication(CheckpointInfoResponse checkpointInfoResponse) throws OpenSearchCorruptionException { diff --git a/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java index 33159b85ec640..0739174fc3afd 100644 --- a/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/RemoteIndexShardTests.java @@ -15,28 +15,45 @@ import org.opensearch.common.concurrent.GatedCloseable; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.FeatureFlags; +import org.opensearch.core.action.ActionListener; import org.opensearch.index.engine.DocIdSeqNoAndSource; import org.opensearch.index.engine.Engine; import org.opensearch.index.engine.InternalEngine; import org.opensearch.index.engine.NRTReplicationEngineFactory; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; +import org.opensearch.indices.replication.CheckpointInfoResponse; +import org.opensearch.indices.replication.GetSegmentFilesResponse; +import org.opensearch.indices.replication.RemoteStoreReplicationSource; +import org.opensearch.indices.replication.SegmentReplicationSourceFactory; +import org.opensearch.indices.replication.SegmentReplicationState; +import org.opensearch.indices.replication.SegmentReplicationTarget; +import org.opensearch.indices.replication.SegmentReplicationTargetService; +import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; +import org.opensearch.indices.replication.common.ReplicationFailedException; import org.opensearch.indices.replication.common.ReplicationType; import org.hamcrest.MatcherAssert; +import org.junit.Assert; import org.junit.Before; import java.io.IOException; import java.nio.file.Path; import java.util.Arrays; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import static org.opensearch.index.engine.EngineTestCase.assertAtMostOneLuceneDocumentPerSequenceNumber; +import static org.opensearch.index.shard.RemoteStoreRefreshListener.EXCLUDE_FILES; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class RemoteIndexShardTests extends SegmentReplicationIndexShardTests { @@ -355,6 +372,121 @@ public void testPrimaryRestart() throws Exception { } } + /** + * This test validates that unreferenced on disk file are ignored while requesting files from replication source to + * prevent FileAlreadyExistsException. It does so by only copying files in first round of segment replication without + * committing locally so that in next round of segment replication those files are not considered for download again + */ + public void testSegRepSucceedsOnPreviousCopiedFiles() throws Exception { + try (ReplicationGroup shards = createGroup(1, getIndexSettings(), new NRTReplicationEngineFactory())) { + shards.startAll(); + IndexShard primary = shards.getPrimary(); + final IndexShard replica = shards.getReplicas().get(0); + + shards.indexDocs(10); + primary.refresh("Test"); + + final SegmentReplicationSourceFactory sourceFactory = mock(SegmentReplicationSourceFactory.class); + final SegmentReplicationTargetService targetService = newTargetService(sourceFactory); + Runnable[] runAfterGetFiles = { () -> { throw new RuntimeException("Simulated"); }, () -> {} }; + AtomicInteger index = new AtomicInteger(0); + RemoteStoreReplicationSource testRSReplicationSource = new RemoteStoreReplicationSource(replica) { + @Override + public void getCheckpointMetadata( + long replicationId, + ReplicationCheckpoint checkpoint, + ActionListener listener + ) { + super.getCheckpointMetadata(replicationId, checkpoint, listener); + } + + @Override + public void getSegmentFiles( + long replicationId, + ReplicationCheckpoint checkpoint, + List filesToFetch, + IndexShard indexShard, + ActionListener listener + ) { + super.getSegmentFiles(replicationId, checkpoint, filesToFetch, indexShard, listener); + runAfterGetFiles[index.getAndIncrement()].run(); + } + + @Override + public String getDescription() { + return "TestRemoteStoreReplicationSource"; + } + }; + when(sourceFactory.get(any())).thenReturn(testRSReplicationSource); + CountDownLatch latch = new CountDownLatch(1); + + // Start first round of segment replication. This should fail with simulated error but with replica having + // files in its local store but not in active reader. + final SegmentReplicationTarget target = targetService.startReplication( + replica, + primary.getLatestReplicationCheckpoint(), + new SegmentReplicationTargetService.SegmentReplicationListener() { + @Override + public void onReplicationDone(SegmentReplicationState state) { + Assert.fail("Replication should fail with simulated error"); + } + + @Override + public void onReplicationFailure( + SegmentReplicationState state, + ReplicationFailedException e, + boolean sendShardFailure + ) { + assertFalse(sendShardFailure); + logger.error("Replication error", e); + latch.countDown(); + } + } + ); + latch.await(); + Set onDiskFiles = new HashSet<>(Arrays.asList(replica.store().directory().listAll())); + onDiskFiles.removeIf(name -> EXCLUDE_FILES.contains(name) || name.startsWith(IndexFileNames.SEGMENTS)); + List activeFiles = replica.getSegmentMetadataMap() + .values() + .stream() + .map(metadata -> metadata.name()) + .collect(Collectors.toList()); + assertTrue("Files should not be committed", activeFiles.isEmpty()); + assertEquals("Files should be copied to disk", false, onDiskFiles.isEmpty()); + assertEquals(target.state().getStage(), SegmentReplicationState.Stage.GET_FILES); + + // Start next round of segment replication + CountDownLatch waitForSecondRound = new CountDownLatch(1); + final SegmentReplicationTarget newTarget = targetService.startReplication( + replica, + primary.getLatestReplicationCheckpoint(), + new SegmentReplicationTargetService.SegmentReplicationListener() { + @Override + public void onReplicationDone(SegmentReplicationState state) { + waitForSecondRound.countDown(); + } + + @Override + public void onReplicationFailure( + SegmentReplicationState state, + ReplicationFailedException e, + boolean sendShardFailure + ) { + logger.error("Replication error", e); + Assert.fail("Replication should not fail"); + waitForSecondRound.countDown(); + } + } + ); + waitForSecondRound.await(); + assertEquals(newTarget.state().getStage(), SegmentReplicationState.Stage.DONE); + activeFiles = replica.getSegmentMetadataMap().values().stream().map(metadata -> metadata.name()).collect(Collectors.toList()); + assertTrue("Replica should have consistent disk & reader", activeFiles.containsAll(onDiskFiles)); + shards.removeReplica(replica); + closeShards(replica); + } + } + private void assertSingleSegmentFile(IndexShard shard, String fileName) throws IOException { final Set segmentsFileNames = Arrays.stream(shard.store().directory().listAll()) .filter(file -> file.startsWith(IndexFileNames.SEGMENTS)) From dbb868a8f55878540c6b5a41f9c7a47f7644dcc0 Mon Sep 17 00:00:00 2001 From: Gagan Juneja Date: Sat, 2 Sep 2023 00:00:46 +0530 Subject: [PATCH 30/37] [Tracing Framework] Redefine telemetry context restoration and propagation (#9617) * Add SpanBuilder support Signed-off-by: Gagan Juneja * Refactor code Signed-off-by: Gagan Juneja * Redefine telemetry context restoration Signed-off-by: Gagan Juneja * Update changelog Signed-off-by: Gagan Juneja * Stores the SpanScope in ThreadLocal Signed-off-by: Gagan Juneja * Revert the context name changes Signed-off-by: Gagan Juneja * Change the span::endSpan and SpanScope::close behaviour Signed-off-by: Gagan Juneja * Supressed warnings Signed-off-by: Gagan Juneja * Add more test cases Signed-off-by: Gagan Juneja * Address review comment Signed-off-by: Gagan Juneja * Address review comment Signed-off-by: Gagan Juneja * Fix java doc Signed-off-by: Gagan Juneja * Address review comment Signed-off-by: Gagan Juneja * Fix failing test Signed-off-by: Gagan Juneja * Empty-Commit Signed-off-by: Gagan Juneja * Empty-Commit Signed-off-by: Gagan Juneja --------- Signed-off-by: Gagan Juneja Signed-off-by: Gagan Juneja Co-authored-by: Gagan Juneja --- CHANGELOG.md | 1 + .../telemetry/tracing/DefaultScopedSpan.java | 89 +++++ .../telemetry/tracing/DefaultSpanScope.java | 77 ++-- .../telemetry/tracing/DefaultTracer.java | 35 +- .../telemetry/tracing/ScopedSpan.java | 74 ++++ .../tracing/SpanCreationContext.java | 45 +++ .../telemetry/tracing/SpanScope.java | 55 +-- .../opensearch/telemetry/tracing/Tracer.java | 51 ++- .../telemetry/tracing/TracingTelemetry.java | 3 +- .../telemetry/tracing/http/HttpTracer.java | 5 +- .../tracing/noop/NoopScopedSpan.java | 59 ++++ .../telemetry/tracing/noop/NoopSpan.java | 81 +++++ .../telemetry/tracing/noop/NoopSpanScope.java | 40 +-- .../telemetry/tracing/noop/NoopTracer.java | 41 ++- .../tracing/runnable/TraceableRunnable.java | 5 +- ...Tests.java => DefaultScopedSpanTests.java} | 39 +- .../telemetry/tracing/DefaultTracerTests.java | 332 +++++++++++++++--- .../tracing/TraceableRunnableTests.java | 26 +- .../telemetry/tracing/OTelSpan.java | 6 + .../tracing/OTelTracingTelemetry.java | 3 +- .../telemetry/tracing/AttributeNames.java | 62 ++++ .../telemetry/tracing/SpanBuilder.java | 124 +++++++ ...hreadContextBasedTracerContextStorage.java | 3 - .../telemetry/tracing/WrappedTracer.java | 28 +- .../telemetry/tracing/SpanBuilderTests.java | 158 +++++++++ .../telemetry/tracing/TracerFactoryTests.java | 20 +- .../test/telemetry/tracing/MockSpan.java | 6 +- 27 files changed, 1236 insertions(+), 232 deletions(-) create mode 100644 libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/DefaultScopedSpan.java create mode 100644 libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/ScopedSpan.java create mode 100644 libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/SpanCreationContext.java create mode 100644 libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/NoopScopedSpan.java create mode 100644 libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/NoopSpan.java rename libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/{DefaultSpanScopeTests.java => DefaultScopedSpanTests.java} (50%) create mode 100644 server/src/main/java/org/opensearch/telemetry/tracing/AttributeNames.java create mode 100644 server/src/main/java/org/opensearch/telemetry/tracing/SpanBuilder.java create mode 100644 server/src/test/java/org/opensearch/telemetry/tracing/SpanBuilderTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index ffefcd416da1c..d8d68c51ceed5 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -171,6 +171,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Separate request-based and settings-based concurrent segment search controls and introduce AggregatorFactory method to determine concurrent search support ([#9469](https://github.com/opensearch-project/OpenSearch/pull/9469)) - [Remote Store] Rate limiter integration for remote store uploads and downloads([#9448](https://github.com/opensearch-project/OpenSearch/pull/9448/)) - [Remote Store] Implicitly use replication type SEGMENT for remote store clusters ([#9264](https://github.com/opensearch-project/OpenSearch/pull/9264)) +- Redefine telemetry context restoration and propagation ([#9617](https://github.com/opensearch-project/OpenSearch/pull/9617)) - Use non-concurrent path for sort request on timeseries index and field([#9562](https://github.com/opensearch-project/OpenSearch/pull/9562)) - Added sampler based on `Blanket Probabilistic Sampling rate` and `Override for on demand` ([#9621](https://github.com/opensearch-project/OpenSearch/issues/9621)) diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/DefaultScopedSpan.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/DefaultScopedSpan.java new file mode 100644 index 0000000000000..87802b1f1931d --- /dev/null +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/DefaultScopedSpan.java @@ -0,0 +1,89 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.telemetry.tracing; + +import java.util.Objects; + +/** + * Default implementation of Scope + * + * @opensearch.internal + */ +final class DefaultScopedSpan implements ScopedSpan { + + private final Span span; + + private final SpanScope spanScope; + + /** + * Creates Scope instance for the given span + * + * @param span underlying span + * @param spanScope span scope. + */ + public DefaultScopedSpan(Span span, SpanScope spanScope) { + this.span = Objects.requireNonNull(span); + this.spanScope = Objects.requireNonNull(spanScope); + } + + @Override + public void addAttribute(String key, String value) { + span.addAttribute(key, value); + } + + @Override + public void addAttribute(String key, long value) { + span.addAttribute(key, value); + } + + @Override + public void addAttribute(String key, double value) { + span.addAttribute(key, value); + } + + @Override + public void addAttribute(String key, boolean value) { + span.addAttribute(key, value); + } + + @Override + public void addEvent(String event) { + span.addEvent(event); + } + + @Override + public void setError(Exception exception) { + span.setError(exception); + } + + /** + * Executes the runnable to end the scope + */ + @Override + public void close() { + span.endSpan(); + spanScope.close(); + } + + /** + * Returns span. + * @return + */ + Span getSpan() { + return span; + } + + /** + * Returns {@link SpanScope} + * @return spanScope + */ + SpanScope getSpanScope() { + return spanScope; + } +} diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/DefaultSpanScope.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/DefaultSpanScope.java index 356b72187de74..037309da0bc30 100644 --- a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/DefaultSpanScope.java +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/DefaultSpanScope.java @@ -8,65 +8,68 @@ package org.opensearch.telemetry.tracing; -import java.util.function.Consumer; +import java.util.Objects; /** - * Default implementation of Scope - * - * @opensearch.internal + * Default implementation for {@link SpanScope} */ -final class DefaultSpanScope implements SpanScope { - +public class DefaultSpanScope implements SpanScope { private final Span span; - - private final Consumer onCloseConsumer; + private final SpanScope previousSpanScope; + private static final ThreadLocal spanScopeThreadLocal = new ThreadLocal<>(); + private final TracerContextStorage tracerContextStorage; /** - * Creates Scope instance for the given span - * - * @param span underlying span - * @param onCloseConsumer consumer to execute on scope close + * Constructor + * @param span span + * @param previousSpanScope before attached span scope. */ - public DefaultSpanScope(Span span, Consumer onCloseConsumer) { - this.span = span; - this.onCloseConsumer = onCloseConsumer; + private DefaultSpanScope(Span span, SpanScope previousSpanScope, TracerContextStorage tracerContextStorage) { + this.span = Objects.requireNonNull(span); + this.previousSpanScope = previousSpanScope; + this.tracerContextStorage = tracerContextStorage; } - @Override - public void addSpanAttribute(String key, String value) { - span.addAttribute(key, value); + /** + * Creates the SpanScope object. + * @param span span. + * @param tracerContextStorage tracer context storage. + * @return SpanScope spanScope + */ + public static SpanScope create(Span span, TracerContextStorage tracerContextStorage) { + final SpanScope beforeSpanScope = spanScopeThreadLocal.get(); + SpanScope newSpanScope = new DefaultSpanScope(span, beforeSpanScope, tracerContextStorage); + spanScopeThreadLocal.set(newSpanScope); + return newSpanScope; } @Override - public void addSpanAttribute(String key, long value) { - span.addAttribute(key, value); + public void close() { + detach(); + spanScopeThreadLocal.set(previousSpanScope); } @Override - public void addSpanAttribute(String key, double value) { - span.addAttribute(key, value); + public SpanScope attach() { + tracerContextStorage.put(TracerContextStorage.CURRENT_SPAN, this.span); + return this; } - @Override - public void addSpanAttribute(String key, boolean value) { - span.addAttribute(key, value); + private void detach() { + if (previousSpanScope != null) { + tracerContextStorage.put(TracerContextStorage.CURRENT_SPAN, previousSpanScope.getSpan()); + } else { + tracerContextStorage.put(TracerContextStorage.CURRENT_SPAN, null); + } } @Override - public void addSpanEvent(String event) { - span.addEvent(event); + public Span getSpan() { + return span; } - @Override - public void setError(Exception exception) { - span.setError(exception); + static SpanScope getCurrentSpanScope() { + return spanScopeThreadLocal.get(); } - /** - * Executes the runnable to end the scope - */ - @Override - public void close() { - onCloseConsumer.accept(span); - } } diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/DefaultTracer.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/DefaultTracer.java index bc1a08e2d3c72..b75d761fc240c 100644 --- a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/DefaultTracer.java +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/DefaultTracer.java @@ -41,17 +41,22 @@ public DefaultTracer(TracingTelemetry tracingTelemetry, TracerContextStorage endSpan(scopeSpan)); + return span; } @Override @@ -77,11 +82,21 @@ public SpanContext getCurrentSpan() { return (currentSpan == null) ? null : new SpanContext(currentSpan); } - private void endSpan(Span span) { - if (span != null) { - span.endSpan(); - setCurrentSpanInContext(span.getParentSpan()); - } + @Override + public ScopedSpan startScopedSpan(SpanCreationContext spanCreationContext) { + return startScopedSpan(spanCreationContext, null); + } + + @Override + public ScopedSpan startScopedSpan(SpanCreationContext spanCreationContext, SpanContext parentSpan) { + Span span = startSpan(spanCreationContext.getSpanName(), parentSpan, spanCreationContext.getAttributes()); + SpanScope spanScope = withSpanInScope(span); + return new DefaultScopedSpan(span, spanScope); + } + + @Override + public SpanScope withSpanInScope(Span span) { + return DefaultSpanScope.create(span, tracerContextStorage).attach(); } private Span createSpan(String spanName, Span parentSpan, Attributes attributes) { @@ -101,7 +116,7 @@ protected void addDefaultAttributes(Span span) { } @Override - public SpanScope startSpan(String spanName, Map> headers, Attributes attributes) { + public Span startSpan(String spanName, Map> headers, Attributes attributes) { Optional propagatedSpan = tracingTelemetry.getContextPropagator().extractFromHeaders(headers); return startSpan(spanName, propagatedSpan.map(SpanContext::new).orElse(null), attributes); } diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/ScopedSpan.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/ScopedSpan.java new file mode 100644 index 0000000000000..833a85ef27baf --- /dev/null +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/ScopedSpan.java @@ -0,0 +1,74 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.telemetry.tracing; + +import org.opensearch.telemetry.tracing.noop.NoopScopedSpan; + +/** + * An auto-closeable that represents scoped span. + * It provides interface for all the span operations. + */ +public interface ScopedSpan extends AutoCloseable { + /** + * No-op Scope implementation + */ + ScopedSpan NO_OP = new NoopScopedSpan(); + + /** + * Adds string attribute to the {@link Span}. + * + * @param key attribute key + * @param value attribute value + */ + void addAttribute(String key, String value); + + /** + * Adds long attribute to the {@link Span}. + * + * @param key attribute key + * @param value attribute value + */ + void addAttribute(String key, long value); + + /** + * Adds double attribute to the {@link Span}. + * + * @param key attribute key + * @param value attribute value + */ + void addAttribute(String key, double value); + + /** + * Adds boolean attribute to the {@link Span}. + * + * @param key attribute key + * @param value attribute value + */ + void addAttribute(String key, boolean value); + + /** + * Adds an event to the {@link Span}. + * + * @param event event name + */ + void addEvent(String event); + + /** + * Records error in the span + * + * @param exception exception to be recorded + */ + void setError(Exception exception); + + /** + * closes the scope + */ + @Override + void close(); +} diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/SpanCreationContext.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/SpanCreationContext.java new file mode 100644 index 0000000000000..0f91eb448deb5 --- /dev/null +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/SpanCreationContext.java @@ -0,0 +1,45 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.telemetry.tracing; + +import org.opensearch.telemetry.tracing.attributes.Attributes; + +/** + * Context for span details. + */ +public final class SpanCreationContext { + private final String spanName; + private final Attributes attributes; + + /** + * Constructor. + * @param spanName span name. + * @param attributes attributes. + */ + public SpanCreationContext(String spanName, Attributes attributes) { + this.spanName = spanName; + this.attributes = attributes; + } + + /** + * Returns the span name. + * @return span name + */ + public String getSpanName() { + return spanName; + } + + /** + * Returns the span attributes. + * @return attributes. + */ + public Attributes getAttributes() { + return attributes; + } +} diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/SpanScope.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/SpanScope.java index cf67165d889bc..99c27b72fe1c7 100644 --- a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/SpanScope.java +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/SpanScope.java @@ -12,63 +12,26 @@ /** * An auto-closeable that represents scope of the span. - * It provides interface for all the span operations. */ public interface SpanScope extends AutoCloseable { + /** * No-op Scope implementation */ SpanScope NO_OP = new NoopSpanScope(); - /** - * Adds string attribute to the {@link Span}. - * - * @param key attribute key - * @param value attribute value - */ - void addSpanAttribute(String key, String value); - - /** - * Adds long attribute to the {@link Span}. - * - * @param key attribute key - * @param value attribute value - */ - void addSpanAttribute(String key, long value); - - /** - * Adds double attribute to the {@link Span}. - * - * @param key attribute key - * @param value attribute value - */ - void addSpanAttribute(String key, double value); - - /** - * Adds boolean attribute to the {@link Span}. - * - * @param key attribute key - * @param value attribute value - */ - void addSpanAttribute(String key, boolean value); - - /** - * Adds an event to the {@link Span}. - * - * @param event event name - */ - void addSpanEvent(String event); + @Override + void close(); /** - * Records error in the span - * - * @param exception exception to be recorded + * Attaches span to the {@link SpanScope} + * @return spanScope */ - void setError(Exception exception); + SpanScope attach(); /** - * closes the scope + * Returns span attached with the {@link SpanScope} + * @return span. */ - @Override - void close(); + Span getSpan(); } diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/Tracer.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/Tracer.java index 40cc5dfd2d743..dc247e45d77f6 100644 --- a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/Tracer.java +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/Tracer.java @@ -20,36 +20,71 @@ * All methods on the Tracer object are multi-thread safe. */ public interface Tracer extends HttpTracer, Closeable { + /** + * Starts the {@link Span} with given {@link SpanCreationContext} + * + * @param context span context + * @return span, must be closed. + */ + Span startSpan(SpanCreationContext context); /** * Starts the {@link Span} with given name * * @param spanName span name - * @return scope of the span, must be closed with explicit close or with try-with-resource + * @return span, must be closed. */ - SpanScope startSpan(String spanName); + Span startSpan(String spanName); /** * Starts the {@link Span} with given name and attributes. This is required in cases when some attribute based * decision needs to be made before starting the span. Very useful in the case of Sampling. - * @param spanName span name. + * + * @param spanName span name. * @param attributes attributes to be added. - * @return scope of the span, must be closed with explicit close or with try-with-resource + * @return span, must be closed. */ - SpanScope startSpan(String spanName, Attributes attributes); + Span startSpan(String spanName, Attributes attributes); /** * Starts the {@link Span} with the given name, parent and attributes. - * @param spanName span name. + * + * @param spanName span name. * @param parentSpan parent span. * @param attributes attributes to be added. - * @return scope of the span, must be closed with explicit close or with try-with-resource + * @return span, must be closed. */ - SpanScope startSpan(String spanName, SpanContext parentSpan, Attributes attributes); + Span startSpan(String spanName, SpanContext parentSpan, Attributes attributes); /** * Returns the current span. * @return current wrapped span. */ SpanContext getCurrentSpan(); + + /** + * Start the span and scoped it. This must be used for scenarios where {@link SpanScope} and {@link Span} lifecycles + * are same and ends within the same thread where created. + * @param spanCreationContext span creation context + * @return scope of the span, must be closed with explicit close or with try-with-resource + */ + ScopedSpan startScopedSpan(SpanCreationContext spanCreationContext); + + /** + * Start the span and scoped it. This must be used for scenarios where {@link SpanScope} and {@link Span} lifecycles + * are same and ends within the same thread where created. + * @param spanCreationContext span creation context + * @param parentSpan parent span. + * @return scope of the span, must be closed with explicit close or with try-with-resource + */ + ScopedSpan startScopedSpan(SpanCreationContext spanCreationContext, SpanContext parentSpan); + + /** + * Creates the Span Scope for a current thread. It's mandatory to scope the span just after creation so that it will + * automatically manage the attach /detach to the current thread. + * @param span span to be scoped + * @return ScopedSpan + */ + SpanScope withSpanInScope(Span span); + } diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/TracingTelemetry.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/TracingTelemetry.java index 2e91cadbf395f..895e14da69f7f 100644 --- a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/TracingTelemetry.java +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/TracingTelemetry.java @@ -21,7 +21,8 @@ public interface TracingTelemetry extends Closeable { /** * Creates span with provided arguments - * @param spanName name of the span + * + * @param spanName name of the span * @param parentSpan span's parent span * @param attributes attributes to be added. * @return span instance diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/http/HttpTracer.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/http/HttpTracer.java index 64ef84335a95b..8027291d5480b 100644 --- a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/http/HttpTracer.java +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/http/HttpTracer.java @@ -9,7 +9,6 @@ package org.opensearch.telemetry.tracing.http; import org.opensearch.telemetry.tracing.Span; -import org.opensearch.telemetry.tracing.SpanScope; import org.opensearch.telemetry.tracing.attributes.Attributes; import java.util.List; @@ -28,7 +27,7 @@ public interface HttpTracer { * @param spanName span name. * @param header http request header. * @param attributes span attributes. - * @return scope of the span, must be closed with explicit close or with try-with-resource + * @return span. */ - SpanScope startSpan(String spanName, Map> header, Attributes attributes); + Span startSpan(String spanName, Map> header, Attributes attributes); } diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/NoopScopedSpan.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/NoopScopedSpan.java new file mode 100644 index 0000000000000..22ee7e7ccb6fb --- /dev/null +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/NoopScopedSpan.java @@ -0,0 +1,59 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.telemetry.tracing.noop; + +import org.opensearch.telemetry.tracing.ScopedSpan; + +/** + * No-op implementation of SpanScope + * + * @opensearch.internal + */ +public final class NoopScopedSpan implements ScopedSpan { + + /** + * No-args constructor + */ + public NoopScopedSpan() {} + + @Override + public void addAttribute(String key, String value) { + + } + + @Override + public void addAttribute(String key, long value) { + + } + + @Override + public void addAttribute(String key, double value) { + + } + + @Override + public void addAttribute(String key, boolean value) { + + } + + @Override + public void addEvent(String event) { + + } + + @Override + public void setError(Exception exception) { + + } + + @Override + public void close() { + + } +} diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/NoopSpan.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/NoopSpan.java new file mode 100644 index 0000000000000..e053e6576fcea --- /dev/null +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/NoopSpan.java @@ -0,0 +1,81 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.telemetry.tracing.noop; + +import org.opensearch.telemetry.tracing.Span; + +/** + * No-op implementation of {@link org.opensearch.telemetry.tracing.Span} + */ +public class NoopSpan implements Span { + + /** + * No-op Span instance + */ + public final static NoopSpan INSTANCE = new NoopSpan(); + + private NoopSpan() { + + } + + @Override + public void endSpan() { + + } + + @Override + public Span getParentSpan() { + return null; + } + + @Override + public String getSpanName() { + return "noop-span"; + } + + @Override + public void addAttribute(String key, String value) { + + } + + @Override + public void addAttribute(String key, Long value) { + + } + + @Override + public void addAttribute(String key, Double value) { + + } + + @Override + public void addAttribute(String key, Boolean value) { + + } + + @Override + public void setError(Exception exception) { + + } + + @Override + public void addEvent(String event) { + + } + + @Override + public String getTraceId() { + return "noop-trace-id"; + } + + @Override + public String getSpanId() { + return "noop-span-id"; + } +} diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/NoopSpanScope.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/NoopSpanScope.java index a1d16d1d80d00..a9b72adeeda0e 100644 --- a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/NoopSpanScope.java +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/NoopSpanScope.java @@ -8,52 +8,32 @@ package org.opensearch.telemetry.tracing.noop; +import org.opensearch.telemetry.tracing.Span; import org.opensearch.telemetry.tracing.SpanScope; /** - * No-op implementation of SpanScope - * - * @opensearch.internal + * No-op implementation of {@link SpanScope} */ -public final class NoopSpanScope implements SpanScope { - +public class NoopSpanScope implements SpanScope { /** - * No-args constructor + * Constructor. */ - public NoopSpanScope() {} - - @Override - public void addSpanAttribute(String key, String value) { - - } - - @Override - public void addSpanAttribute(String key, long value) { - - } - - @Override - public void addSpanAttribute(String key, double value) { - - } - - @Override - public void addSpanAttribute(String key, boolean value) { + public NoopSpanScope() { } @Override - public void addSpanEvent(String event) { + public void close() { } @Override - public void setError(Exception exception) { - + public SpanScope attach() { + return this; } @Override - public void close() { - + public Span getSpan() { + return NoopSpan.INSTANCE; } } diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/NoopTracer.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/NoopTracer.java index 2ff50bf3bcb18..a9eee725b4f1c 100644 --- a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/NoopTracer.java +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/NoopTracer.java @@ -8,7 +8,10 @@ package org.opensearch.telemetry.tracing.noop; +import org.opensearch.telemetry.tracing.ScopedSpan; +import org.opensearch.telemetry.tracing.Span; import org.opensearch.telemetry.tracing.SpanContext; +import org.opensearch.telemetry.tracing.SpanCreationContext; import org.opensearch.telemetry.tracing.SpanScope; import org.opensearch.telemetry.tracing.Tracer; import org.opensearch.telemetry.tracing.attributes.Attributes; @@ -31,32 +34,52 @@ public class NoopTracer implements Tracer { private NoopTracer() {} @Override - public SpanScope startSpan(String spanName) { - return SpanScope.NO_OP; + public Span startSpan(SpanCreationContext context) { + return NoopSpan.INSTANCE; } @Override - public SpanScope startSpan(String spanName, Attributes attributes) { - return SpanScope.NO_OP; + public Span startSpan(String spanName) { + return NoopSpan.INSTANCE; } @Override - public SpanScope startSpan(String spanName, SpanContext parentSpan, Attributes attributes) { - return SpanScope.NO_OP; + public Span startSpan(String spanName, Attributes attributes) { + return NoopSpan.INSTANCE; + } + + @Override + public Span startSpan(String spanName, SpanContext parentSpan, Attributes attributes) { + return NoopSpan.INSTANCE; } @Override public SpanContext getCurrentSpan() { - return null; + return new SpanContext(NoopSpan.INSTANCE); } @Override - public void close() { + public ScopedSpan startScopedSpan(SpanCreationContext spanCreationContext) { + return ScopedSpan.NO_OP; + } + @Override + public ScopedSpan startScopedSpan(SpanCreationContext spanCreationContext, SpanContext parentSpan) { + return ScopedSpan.NO_OP; } @Override - public SpanScope startSpan(String spanName, Map> header, Attributes attributes) { + public SpanScope withSpanInScope(Span span) { return SpanScope.NO_OP; } + + @Override + public void close() { + + } + + @Override + public Span startSpan(String spanName, Map> header, Attributes attributes) { + return NoopSpan.INSTANCE; + } } diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/runnable/TraceableRunnable.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/runnable/TraceableRunnable.java index 54a5a7f1678e6..4672574e9f4ca 100644 --- a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/runnable/TraceableRunnable.java +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/runnable/TraceableRunnable.java @@ -8,8 +8,9 @@ package org.opensearch.telemetry.tracing.runnable; +import org.opensearch.telemetry.tracing.ScopedSpan; import org.opensearch.telemetry.tracing.SpanContext; -import org.opensearch.telemetry.tracing.SpanScope; +import org.opensearch.telemetry.tracing.SpanCreationContext; import org.opensearch.telemetry.tracing.Tracer; import org.opensearch.telemetry.tracing.attributes.Attributes; @@ -41,7 +42,7 @@ public TraceableRunnable(Tracer tracer, String spanName, SpanContext parent, Att @Override public void run() { - try (SpanScope spanScope = tracer.startSpan(spanName, parent, attributes)) { + try (ScopedSpan spanScope = tracer.startScopedSpan(new SpanCreationContext(spanName, attributes), parent)) { runnable.run(); } } diff --git a/libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/DefaultSpanScopeTests.java b/libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/DefaultScopedSpanTests.java similarity index 50% rename from libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/DefaultSpanScopeTests.java rename to libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/DefaultScopedSpanTests.java index eea6b77ce6e1e..1d4871fe1419e 100644 --- a/libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/DefaultSpanScopeTests.java +++ b/libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/DefaultScopedSpanTests.java @@ -10,66 +10,71 @@ import org.opensearch.test.OpenSearchTestCase; -import java.util.function.Consumer; - import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; -public class DefaultSpanScopeTests extends OpenSearchTestCase { +public class DefaultScopedSpanTests extends OpenSearchTestCase { @SuppressWarnings("unchecked") public void testClose() { Span mockSpan = mock(Span.class); - Consumer mockConsumer = mock(Consumer.class); - DefaultSpanScope defaultSpanScope = new DefaultSpanScope(mockSpan, mockConsumer); + SpanScope mockSpanScope = mock(SpanScope.class); + DefaultScopedSpan defaultSpanScope = new DefaultScopedSpan(mockSpan, mockSpanScope); defaultSpanScope.close(); - verify(mockConsumer).accept(mockSpan); + verify(mockSpan).endSpan(); + verify(mockSpanScope).close(); } public void testAddSpanAttributeString() { Span mockSpan = mock(Span.class); - DefaultSpanScope defaultSpanScope = new DefaultSpanScope(mockSpan, null); - defaultSpanScope.addSpanAttribute("key", "value"); + SpanScope mockSpanScope = mock(SpanScope.class); + DefaultScopedSpan defaultSpanScope = new DefaultScopedSpan(mockSpan, mockSpanScope); + defaultSpanScope.addAttribute("key", "value"); verify(mockSpan).addAttribute("key", "value"); } public void testAddSpanAttributeLong() { Span mockSpan = mock(Span.class); - DefaultSpanScope defaultSpanScope = new DefaultSpanScope(mockSpan, null); - defaultSpanScope.addSpanAttribute("key", 1L); + SpanScope mockSpanScope = mock(SpanScope.class); + DefaultScopedSpan defaultSpanScope = new DefaultScopedSpan(mockSpan, mockSpanScope); + defaultSpanScope.addAttribute("key", 1L); verify(mockSpan).addAttribute("key", 1L); } public void testAddSpanAttributeDouble() { Span mockSpan = mock(Span.class); - DefaultSpanScope defaultSpanScope = new DefaultSpanScope(mockSpan, null); - defaultSpanScope.addSpanAttribute("key", 1.0); + SpanScope mockSpanScope = mock(SpanScope.class); + DefaultScopedSpan defaultSpanScope = new DefaultScopedSpan(mockSpan, mockSpanScope); + defaultSpanScope.addAttribute("key", 1.0); verify(mockSpan).addAttribute("key", 1.0); } public void testAddSpanAttributeBoolean() { Span mockSpan = mock(Span.class); - DefaultSpanScope defaultSpanScope = new DefaultSpanScope(mockSpan, null); - defaultSpanScope.addSpanAttribute("key", true); + SpanScope mockSpanScope = mock(SpanScope.class); + DefaultScopedSpan defaultSpanScope = new DefaultScopedSpan(mockSpan, mockSpanScope); + defaultSpanScope.addAttribute("key", true); verify(mockSpan).addAttribute("key", true); } public void testAddEvent() { Span mockSpan = mock(Span.class); - DefaultSpanScope defaultSpanScope = new DefaultSpanScope(mockSpan, null); - defaultSpanScope.addSpanEvent("eventName"); + SpanScope mockSpanScope = mock(SpanScope.class); + DefaultScopedSpan defaultSpanScope = new DefaultScopedSpan(mockSpan, mockSpanScope); + defaultSpanScope.addEvent("eventName"); verify(mockSpan).addEvent("eventName"); } public void testSetError() { Span mockSpan = mock(Span.class); - DefaultSpanScope defaultSpanScope = new DefaultSpanScope(mockSpan, null); + SpanScope mockSpanScope = mock(SpanScope.class); + DefaultScopedSpan defaultSpanScope = new DefaultScopedSpan(mockSpan, mockSpanScope); Exception ex = new Exception("error"); defaultSpanScope.setError(ex); diff --git a/libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/DefaultTracerTests.java b/libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/DefaultTracerTests.java index 150992da06f89..5205bdfc8a031 100644 --- a/libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/DefaultTracerTests.java +++ b/libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/DefaultTracerTests.java @@ -10,17 +10,17 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.core.action.ActionListener; +import org.opensearch.node.Node; import org.opensearch.telemetry.tracing.attributes.Attributes; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.test.telemetry.tracing.MockSpan; import org.opensearch.test.telemetry.tracing.MockTracingTelemetry; -import org.junit.Assert; +import org.opensearch.threadpool.ThreadPool; import java.io.IOException; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicReference; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; @@ -36,15 +36,23 @@ public class DefaultTracerTests extends OpenSearchTestCase { private Span mockSpan; private Span mockParentSpan; + private SpanScope mockSpanScope; + private ThreadPool threadPool; + private ExecutorService executorService; + @Override public void setUp() throws Exception { super.setUp(); + threadPool = new ThreadPool(Settings.builder().put(Node.NODE_NAME_SETTING.getKey(), "default tracer tests").build()); + executorService = threadPool.executor(ThreadPool.Names.GENERIC); setupMocks(); } @Override public void tearDown() throws Exception { super.tearDown(); + executorService.shutdown(); + threadPool.shutdownNow(); } public void testCreateSpan() { @@ -52,43 +60,47 @@ public void testCreateSpan() { defaultTracer.startSpan("span_name"); - Assert.assertEquals("span_name", defaultTracer.getCurrentSpan().getSpan().getSpanName()); + assertEquals("span_name", defaultTracer.getCurrentSpan().getSpan().getSpanName()); } + @SuppressWarnings("unchecked") public void testCreateSpanWithAttributesWithMock() { DefaultTracer defaultTracer = new DefaultTracer(mockTracingTelemetry, mockTracerContextStorage); Attributes attributes = Attributes.create().addAttribute("name", "value"); - when(mockTracingTelemetry.createSpan("span_name", mockParentSpan, attributes)).thenReturn(mockSpan); + when(mockTracingTelemetry.createSpan(eq("span_name"), eq(mockParentSpan), eq(attributes))).thenReturn(mockSpan); defaultTracer.startSpan("span_name", attributes); - verify(mockTracingTelemetry).createSpan("span_name", mockParentSpan, attributes); + verify(mockTracingTelemetry).createSpan(eq("span_name"), eq(mockParentSpan), eq(attributes)); } + @SuppressWarnings("unchecked") public void testCreateSpanWithAttributesWithParentMock() { DefaultTracer defaultTracer = new DefaultTracer(mockTracingTelemetry, mockTracerContextStorage); Attributes attributes = Attributes.create().addAttribute("name", "value"); - when(mockTracingTelemetry.createSpan("span_name", mockParentSpan, attributes)).thenReturn(mockSpan); + when(mockTracingTelemetry.createSpan(eq("span_name"), eq(mockParentSpan), eq(attributes))).thenReturn(mockSpan); defaultTracer.startSpan("span_name", new SpanContext(mockParentSpan), attributes); - verify(mockTracingTelemetry).createSpan("span_name", mockParentSpan, attributes); + verify(mockTracingTelemetry).createSpan(eq("span_name"), eq(mockParentSpan), eq(attributes)); verify(mockTracerContextStorage, never()).get(TracerContextStorage.CURRENT_SPAN); } public void testCreateSpanWithAttributes() { TracingTelemetry tracingTelemetry = new MockTracingTelemetry(); + ThreadContext threadContext = new ThreadContext(Settings.EMPTY); DefaultTracer defaultTracer = new DefaultTracer( tracingTelemetry, - new ThreadContextBasedTracerContextStorage(new ThreadContext(Settings.EMPTY), tracingTelemetry) + new ThreadContextBasedTracerContextStorage(threadContext, tracingTelemetry) ); - defaultTracer.startSpan( + Span span = defaultTracer.startSpan( "span_name", Attributes.create().addAttribute("key1", 1.0).addAttribute("key2", 2l).addAttribute("key3", true).addAttribute("key4", "key4") ); - Assert.assertEquals("span_name", defaultTracer.getCurrentSpan().getSpan().getSpanName()); - Assert.assertEquals(1.0, ((MockSpan) defaultTracer.getCurrentSpan().getSpan()).getAttribute("key1")); - Assert.assertEquals(2l, ((MockSpan) defaultTracer.getCurrentSpan().getSpan()).getAttribute("key2")); - Assert.assertEquals(true, ((MockSpan) defaultTracer.getCurrentSpan().getSpan()).getAttribute("key3")); - Assert.assertEquals("key4", ((MockSpan) defaultTracer.getCurrentSpan().getSpan()).getAttribute("key4")); + assertEquals("span_name", defaultTracer.getCurrentSpan().getSpan().getSpanName()); + assertEquals(1.0, ((MockSpan) defaultTracer.getCurrentSpan().getSpan()).getAttribute("key1")); + assertEquals(2l, ((MockSpan) defaultTracer.getCurrentSpan().getSpan()).getAttribute("key2")); + assertEquals(true, ((MockSpan) defaultTracer.getCurrentSpan().getSpan()).getAttribute("key3")); + assertEquals("key4", ((MockSpan) defaultTracer.getCurrentSpan().getSpan()).getAttribute("key4")); + span.endSpan(); } public void testCreateSpanWithParent() { @@ -98,57 +110,284 @@ public void testCreateSpanWithParent() { new ThreadContextBasedTracerContextStorage(new ThreadContext(Settings.EMPTY), tracingTelemetry) ); - defaultTracer.startSpan("span_name", null); + Span span = defaultTracer.startSpan("span_name", null); SpanContext parentSpan = defaultTracer.getCurrentSpan(); - defaultTracer.startSpan("span_name_1", parentSpan, Attributes.EMPTY); + Span span1 = defaultTracer.startSpan("span_name_1", parentSpan, Attributes.EMPTY); - Assert.assertEquals("span_name_1", defaultTracer.getCurrentSpan().getSpan().getSpanName()); - Assert.assertEquals(parentSpan.getSpan(), defaultTracer.getCurrentSpan().getSpan().getParentSpan()); + assertEquals("span_name_1", defaultTracer.getCurrentSpan().getSpan().getSpanName()); + assertEquals(parentSpan.getSpan(), defaultTracer.getCurrentSpan().getSpan().getParentSpan()); + span1.endSpan(); + span.endSpan(); } - public void testHttpTracer() { - String traceId = "trace_id"; - String spanId = "span_id"; - TracingTelemetry tracingTelemetry = new MockTracingTelemetry(); + @SuppressWarnings("unchecked") + public void testCreateSpanWithContext() { + DefaultTracer defaultTracer = new DefaultTracer(mockTracingTelemetry, mockTracerContextStorage); + Attributes attributes = Attributes.create().addAttribute("name", "value"); + when(mockTracingTelemetry.createSpan(eq("span_name"), eq(mockParentSpan), eq(attributes))).thenReturn(mockSpan); + defaultTracer.startSpan(new SpanCreationContext("span_name", attributes)); + verify(mockTracingTelemetry).createSpan(eq("span_name"), eq(mockParentSpan), eq(attributes)); + } + public void testCreateSpanWithNullParent() { + TracingTelemetry tracingTelemetry = new MockTracingTelemetry(); + ThreadContext threadContext = new ThreadContext(Settings.EMPTY); DefaultTracer defaultTracer = new DefaultTracer( tracingTelemetry, - new ThreadContextBasedTracerContextStorage(new ThreadContext(Settings.EMPTY), tracingTelemetry) + new ThreadContextBasedTracerContextStorage(threadContext, tracingTelemetry) ); - Map> requestHeaders = new HashMap<>(); - requestHeaders.put("traceparent", Arrays.asList(traceId + "~" + spanId)); + Span span = defaultTracer.startSpan("span_name", (SpanContext) null, Attributes.EMPTY); + + assertEquals("span_name", defaultTracer.getCurrentSpan().getSpan().getSpanName()); + assertEquals(null, defaultTracer.getCurrentSpan().getSpan().getParentSpan()); + span.endSpan(); + } + + public void testEndSpanByClosingScopedSpan() { + TracingTelemetry tracingTelemetry = new MockTracingTelemetry(); + ThreadContext threadContext = new ThreadContext(Settings.EMPTY); + ThreadContextBasedTracerContextStorage spanTracerStorage = new ThreadContextBasedTracerContextStorage( + threadContext, + tracingTelemetry + ); + DefaultTracer defaultTracer = new DefaultTracer(tracingTelemetry, spanTracerStorage); + ScopedSpan scopedSpan = defaultTracer.startScopedSpan(new SpanCreationContext("span_name", Attributes.EMPTY)); + assertEquals("span_name", defaultTracer.getCurrentSpan().getSpan().getSpanName()); + assertEquals(((DefaultScopedSpan) scopedSpan).getSpanScope(), DefaultSpanScope.getCurrentSpanScope()); + scopedSpan.close(); + assertTrue(((MockSpan) ((DefaultScopedSpan) scopedSpan).getSpan()).hasEnded()); + assertEquals(null, defaultTracer.getCurrentSpan()); + assertEquals(null, DefaultSpanScope.getCurrentSpanScope()); + + } + + public void testEndSpanByClosingScopedSpanMultiple() { + TracingTelemetry tracingTelemetry = new MockTracingTelemetry(); + ThreadContext threadContext = new ThreadContext(Settings.EMPTY); + ThreadContextBasedTracerContextStorage spanTracerStorage = new ThreadContextBasedTracerContextStorage( + threadContext, + tracingTelemetry + ); + DefaultTracer defaultTracer = new DefaultTracer(tracingTelemetry, spanTracerStorage); + ScopedSpan scopedSpan = defaultTracer.startScopedSpan(new SpanCreationContext("span_name", Attributes.EMPTY)); + ScopedSpan scopedSpan1 = defaultTracer.startScopedSpan(new SpanCreationContext("span_name_1", Attributes.EMPTY)); + + assertEquals("span_name_1", defaultTracer.getCurrentSpan().getSpan().getSpanName()); + assertEquals(((DefaultScopedSpan) scopedSpan1).getSpanScope(), DefaultSpanScope.getCurrentSpanScope()); + + scopedSpan1.close(); + assertTrue(((MockSpan) ((DefaultScopedSpan) scopedSpan1).getSpan()).hasEnded()); + assertEquals("span_name", defaultTracer.getCurrentSpan().getSpan().getSpanName()); + assertEquals(((DefaultScopedSpan) scopedSpan).getSpanScope(), DefaultSpanScope.getCurrentSpanScope()); + + scopedSpan.close(); + assertTrue(((MockSpan) ((DefaultScopedSpan) scopedSpan).getSpan()).hasEnded()); + assertEquals(null, defaultTracer.getCurrentSpan()); + assertEquals(null, DefaultSpanScope.getCurrentSpanScope()); + + } + + public void testEndSpanByClosingSpanScope() { + TracingTelemetry tracingTelemetry = new MockTracingTelemetry(); + ThreadContext threadContext = new ThreadContext(Settings.EMPTY); + ThreadContextBasedTracerContextStorage spanTracerStorage = new ThreadContextBasedTracerContextStorage( + threadContext, + tracingTelemetry + ); + DefaultTracer defaultTracer = new DefaultTracer(tracingTelemetry, spanTracerStorage); + Span span = defaultTracer.startSpan(new SpanCreationContext("span_name", Attributes.EMPTY)); + SpanScope spanScope = defaultTracer.withSpanInScope(span); + assertEquals("span_name", defaultTracer.getCurrentSpan().getSpan().getSpanName()); + assertEquals(spanScope, DefaultSpanScope.getCurrentSpanScope()); - SpanScope spanScope = defaultTracer.startSpan("test_span", requestHeaders, Attributes.EMPTY); - SpanContext currentSpan = defaultTracer.getCurrentSpan(); - assertNotNull(currentSpan); - assertEquals(traceId, currentSpan.getSpan().getTraceId()); - assertEquals(traceId, currentSpan.getSpan().getParentSpan().getTraceId()); - assertEquals(spanId, currentSpan.getSpan().getParentSpan().getSpanId()); + span.endSpan(); spanScope.close(); + assertEquals(null, defaultTracer.getCurrentSpan()); + assertTrue(((MockSpan) span).hasEnded()); + } - public void testCreateSpanWithNullParent() { + public void testEndSpanByClosingSpanScopeMultiple() { TracingTelemetry tracingTelemetry = new MockTracingTelemetry(); - DefaultTracer defaultTracer = new DefaultTracer( - tracingTelemetry, - new ThreadContextBasedTracerContextStorage(new ThreadContext(Settings.EMPTY), tracingTelemetry) + ThreadContext threadContext = new ThreadContext(Settings.EMPTY); + ThreadContextBasedTracerContextStorage spanTracerStorage = new ThreadContextBasedTracerContextStorage( + threadContext, + tracingTelemetry ); + DefaultTracer defaultTracer = new DefaultTracer(tracingTelemetry, spanTracerStorage); + Span span = defaultTracer.startSpan(new SpanCreationContext("span_name", Attributes.EMPTY)); + Span span1 = defaultTracer.startSpan(new SpanCreationContext("span_name_1", Attributes.EMPTY)); + SpanScope spanScope = defaultTracer.withSpanInScope(span); + SpanScope spanScope1 = defaultTracer.withSpanInScope(span1); + assertEquals("span_name_1", defaultTracer.getCurrentSpan().getSpan().getSpanName()); + assertEquals(spanScope1, DefaultSpanScope.getCurrentSpanScope()); + + span1.endSpan(); + spanScope1.close(); + + assertEquals("span_name", defaultTracer.getCurrentSpan().getSpan().getSpanName()); + assertEquals(spanScope, DefaultSpanScope.getCurrentSpanScope()); + assertTrue(((MockSpan) span1).hasEnded()); + assertFalse(((MockSpan) span).hasEnded()); + span.endSpan(); + spanScope.close(); - defaultTracer.startSpan("span_name"); + assertEquals(null, defaultTracer.getCurrentSpan()); + assertEquals(null, DefaultSpanScope.getCurrentSpanScope()); + assertTrue(((MockSpan) span).hasEnded()); + assertTrue(((MockSpan) span1).hasEnded()); - Assert.assertEquals("span_name", defaultTracer.getCurrentSpan().getSpan().getSpanName()); - Assert.assertEquals(null, defaultTracer.getCurrentSpan().getSpan().getParentSpan()); } - public void testEndSpanByClosingScope() { - DefaultTracer defaultTracer = new DefaultTracer(mockTracingTelemetry, mockTracerContextStorage); - try (SpanScope spanScope = defaultTracer.startSpan("span_name")) { - verify(mockTracerContextStorage).put(TracerContextStorage.CURRENT_SPAN, mockSpan); + /** + * 1. CreateSpan in ThreadA (NotScopedSpan) + * 2. create Async task and pass the span + * 3. Scope.close + * 4. verify the current_span is still the same on async thread as the 2 + * 5. verify the main thread has current span as null. + */ + public void testSpanAcrossThreads() { + TracingTelemetry tracingTelemetry = new MockTracingTelemetry(); + ThreadContext threadContext = new ThreadContext(Settings.EMPTY); + AtomicReference currentSpanRefThread1 = new AtomicReference<>(); + AtomicReference currentSpanRefThread2 = new AtomicReference<>(); + AtomicReference currentSpanRefAfterEndThread2 = new AtomicReference<>(); + + AtomicReference spanRef = new AtomicReference<>(); + AtomicReference spanT2Ref = new AtomicReference<>(); + + ThreadContextBasedTracerContextStorage spanTracerStorage = new ThreadContextBasedTracerContextStorage( + threadContext, + tracingTelemetry + ); + DefaultTracer defaultTracer = new DefaultTracer(tracingTelemetry, spanTracerStorage); + + executorService.execute(() -> { + // create a span + Span span = defaultTracer.startSpan(new SpanCreationContext("span_name_t_1", Attributes.EMPTY)); + SpanScope spanScope = defaultTracer.withSpanInScope(span); + spanRef.set(span); + + executorService.execute(() -> { + Span spanT2 = defaultTracer.startSpan(new SpanCreationContext("span_name_t_2", Attributes.EMPTY)); + SpanScope spanScopeT2 = defaultTracer.withSpanInScope(spanT2); + spanT2Ref.set(spanT2); + + currentSpanRefThread2.set(defaultTracer.getCurrentSpan().getSpan()); + + spanT2.endSpan(); + spanScopeT2.close(); + currentSpanRefAfterEndThread2.set(getCurrentSpanFromContext(defaultTracer)); + }); + spanScope.close(); + currentSpanRefThread1.set(getCurrentSpanFromContext(defaultTracer)); + }); + assertEquals(spanT2Ref.get(), currentSpanRefThread2.get()); + assertEquals(spanRef.get(), currentSpanRefAfterEndThread2.get()); + assertEquals(null, currentSpanRefThread1.get()); + } + + public void testSpanCloseOnThread2() { + TracingTelemetry tracingTelemetry = new MockTracingTelemetry(); + ThreadContext threadContext = new ThreadContext(Settings.EMPTY); + ThreadContextBasedTracerContextStorage spanTracerStorage = new ThreadContextBasedTracerContextStorage( + threadContext, + tracingTelemetry + ); + AtomicReference currentSpanRefThread1 = new AtomicReference<>(); + AtomicReference currentSpanRefThread2 = new AtomicReference<>(); + DefaultTracer defaultTracer = new DefaultTracer(tracingTelemetry, spanTracerStorage); + final Span span = defaultTracer.startSpan(new SpanCreationContext("span_name_t1", Attributes.EMPTY)); + try (SpanScope spanScope = defaultTracer.withSpanInScope(span)) { + executorService.execute(() -> async(new ActionListener() { + @Override + public void onResponse(Boolean response) { + span.endSpan(); + currentSpanRefThread2.set(defaultTracer.getCurrentSpan()); + } + + @Override + public void onFailure(Exception e) { + + } + })); + currentSpanRefThread1.set(defaultTracer.getCurrentSpan()); } - verify(mockTracerContextStorage).put(TracerContextStorage.CURRENT_SPAN, mockParentSpan); + assertEquals(null, currentSpanRefThread2.get()); + assertEquals(span, currentSpanRefThread1.get().getSpan()); + assertEquals(null, defaultTracer.getCurrentSpan()); + } + + private void async(ActionListener actionListener) { + actionListener.onResponse(true); + } + + /** + * 1. CreateSpan in ThreadA (NotScopedSpan) + * 2. create Async task and pass the span + * 3. Inside Async task start a new span. + * 4. Scope.close + * 5. Parent Scope.close + * 6. verify the current_span is still the same on async thread as the 2 + * 7. verify the main thread has current span as null. + */ + public void testSpanAcrossThreadsMultipleSpans() { + TracingTelemetry tracingTelemetry = new MockTracingTelemetry(); + ThreadContext threadContext = new ThreadContext(Settings.EMPTY); + AtomicReference currentSpanRefThread1 = new AtomicReference<>(); + AtomicReference currentSpanRefThread2 = new AtomicReference<>(); + AtomicReference currentSpanRefAfterEndThread2 = new AtomicReference<>(); + + AtomicReference parentSpanRef = new AtomicReference<>(); + AtomicReference spanRef = new AtomicReference<>(); + AtomicReference spanT2Ref = new AtomicReference<>(); + + ThreadContextBasedTracerContextStorage spanTracerStorage = new ThreadContextBasedTracerContextStorage( + threadContext, + tracingTelemetry + ); + DefaultTracer defaultTracer = new DefaultTracer(tracingTelemetry, spanTracerStorage); + + executorService.execute(() -> { + // create a parent span + Span parentSpan = defaultTracer.startSpan(new SpanCreationContext("p_span_name", Attributes.EMPTY)); + SpanScope parentSpanScope = defaultTracer.withSpanInScope(parentSpan); + parentSpanRef.set(parentSpan); + // create a span + Span span = defaultTracer.startSpan(new SpanCreationContext("span_name_t_1", Attributes.EMPTY)); + SpanScope spanScope = defaultTracer.withSpanInScope(span); + spanRef.set(span); + + executorService.execute(() -> { + Span spanT2 = defaultTracer.startSpan(new SpanCreationContext("span_name_t_2", Attributes.EMPTY)); + SpanScope spanScopeT2 = defaultTracer.withSpanInScope(spanT2); + + Span spanT21 = defaultTracer.startSpan(new SpanCreationContext("span_name_t_2", Attributes.EMPTY)); + SpanScope spanScopeT21 = defaultTracer.withSpanInScope(spanT2); + spanT2Ref.set(spanT21); + currentSpanRefThread2.set(defaultTracer.getCurrentSpan().getSpan()); + + spanT21.endSpan(); + spanScopeT21.close(); + + spanT2.endSpan(); + spanScopeT2.close(); + currentSpanRefAfterEndThread2.set(getCurrentSpanFromContext(defaultTracer)); + }); + spanScope.close(); + parentSpanScope.close(); + currentSpanRefThread1.set(getCurrentSpanFromContext(defaultTracer)); + }); + assertEquals(spanT2Ref.get(), currentSpanRefThread2.get()); + assertEquals(spanRef.get(), currentSpanRefAfterEndThread2.get()); + assertEquals(null, currentSpanRefThread1.get()); + } + + private static Span getCurrentSpanFromContext(DefaultTracer defaultTracer) { + return defaultTracer.getCurrentSpan() != null ? defaultTracer.getCurrentSpan().getSpan() : null; } public void testClose() throws IOException { @@ -164,6 +403,7 @@ private void setupMocks() { mockTracingTelemetry = mock(TracingTelemetry.class); mockSpan = mock(Span.class); mockParentSpan = mock(Span.class); + mockSpanScope = mock(SpanScope.class); mockTracerContextStorage = mock(TracerContextStorage.class); when(mockSpan.getSpanName()).thenReturn("span_name"); when(mockSpan.getSpanId()).thenReturn("span_id"); diff --git a/libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/TraceableRunnableTests.java b/libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/TraceableRunnableTests.java index bcd8ffe41a17b..a67d9b22ca738 100644 --- a/libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/TraceableRunnableTests.java +++ b/libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/TraceableRunnableTests.java @@ -28,42 +28,46 @@ public class TraceableRunnableTests extends OpenSearchTestCase { public void testRunnableWithNullParent() throws Exception { String spanName = "testRunnable"; - DefaultTracer defaultTracer = new DefaultTracer(new MockTracingTelemetry(), contextStorage); + final DefaultTracer defaultTracer = new DefaultTracer(new MockTracingTelemetry(), contextStorage); final AtomicBoolean isRunnableCompleted = new AtomicBoolean(false); - + final AtomicReference spanNameCaptured = new AtomicReference<>(); + final AtomicReference attributeValue = new AtomicReference<>(); TraceableRunnable traceableRunnable = new TraceableRunnable( defaultTracer, spanName, null, Attributes.create().addAttribute("name", "value"), () -> { + spanNameCaptured.set(defaultTracer.getCurrentSpan().getSpan().getSpanName()); + attributeValue.set((String) ((MockSpan) defaultTracer.getCurrentSpan().getSpan()).getAttribute("name")); isRunnableCompleted.set(true); } ); traceableRunnable.run(); assertTrue(isRunnableCompleted.get()); - assertEquals(spanName, defaultTracer.getCurrentSpan().getSpan().getSpanName()); - assertEquals(null, defaultTracer.getCurrentSpan().getSpan().getParentSpan()); - assertEquals("value", ((MockSpan) defaultTracer.getCurrentSpan().getSpan()).getAttribute("name")); - + assertEquals(spanName, spanNameCaptured.get()); + assertEquals(null, defaultTracer.getCurrentSpan()); + assertEquals(null, defaultTracer.getCurrentSpan()); + assertEquals("value", attributeValue.get()); } public void testRunnableWithParent() throws Exception { String spanName = "testRunnable"; String parentSpanName = "parentSpan"; DefaultTracer defaultTracer = new DefaultTracer(new MockTracingTelemetry(), contextStorage); - defaultTracer.startSpan(parentSpanName); - SpanContext parentSpan = defaultTracer.getCurrentSpan(); + ScopedSpan scopedSpan = defaultTracer.startScopedSpan(new SpanCreationContext(parentSpanName, Attributes.EMPTY)); + SpanContext parentSpanContext = defaultTracer.getCurrentSpan(); AtomicReference currentSpan = new AtomicReference<>(); final AtomicBoolean isRunnableCompleted = new AtomicBoolean(false); - TraceableRunnable traceableRunnable = new TraceableRunnable(defaultTracer, spanName, parentSpan, Attributes.EMPTY, () -> { + TraceableRunnable traceableRunnable = new TraceableRunnable(defaultTracer, spanName, parentSpanContext, Attributes.EMPTY, () -> { isRunnableCompleted.set(true); currentSpan.set(defaultTracer.getCurrentSpan()); }); traceableRunnable.run(); assertTrue(isRunnableCompleted.get()); assertEquals(spanName, currentSpan.get().getSpan().getSpanName()); - assertEquals(parentSpan.getSpan(), currentSpan.get().getSpan().getParentSpan()); - assertEquals(parentSpan.getSpan(), defaultTracer.getCurrentSpan().getSpan()); + assertEquals(((DefaultScopedSpan) scopedSpan).getSpan(), currentSpan.get().getSpan().getParentSpan()); + assertEquals(((DefaultScopedSpan) scopedSpan).getSpan(), defaultTracer.getCurrentSpan().getSpan()); + scopedSpan.close(); } } diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelSpan.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelSpan.java index ba63df4ae47a1..7653a56cbb740 100644 --- a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelSpan.java +++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelSpan.java @@ -19,6 +19,12 @@ class OTelSpan extends AbstractSpan { private final Span delegateSpan; + /** + * Constructor + * @param spanName + * @param span + * @param parentSpan + */ public OTelSpan(String spanName, Span span, org.opensearch.telemetry.tracing.Span parentSpan) { super(spanName, parentSpan); this.delegateSpan = span; diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTracingTelemetry.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTracingTelemetry.java index 9a3a10e63503e..ee2f17aabb7f9 100644 --- a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTracingTelemetry.java +++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTracingTelemetry.java @@ -58,7 +58,8 @@ public TracingContextPropagator getContextPropagator() { private Span createOtelSpan(String spanName, Span parentSpan, Attributes attributes) { io.opentelemetry.api.trace.Span otelSpan = otelSpan(spanName, parentSpan, OTelAttributesConverter.convert(attributes)); - return new OTelSpan(spanName, otelSpan, parentSpan); + Span newSpan = new OTelSpan(spanName, otelSpan, parentSpan); + return newSpan; } io.opentelemetry.api.trace.Span otelSpan(String spanName, Span parentOTelSpan, io.opentelemetry.api.common.Attributes attributes) { diff --git a/server/src/main/java/org/opensearch/telemetry/tracing/AttributeNames.java b/server/src/main/java/org/opensearch/telemetry/tracing/AttributeNames.java new file mode 100644 index 0000000000000..073009ce12128 --- /dev/null +++ b/server/src/main/java/org/opensearch/telemetry/tracing/AttributeNames.java @@ -0,0 +1,62 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.telemetry.tracing; + +/** + * Hold the Attribute names to avoid the duplication and consistensy. + */ +public final class AttributeNames { + + /** + * Constructor + */ + private AttributeNames() { + + } + + /** + * HTTP Protocol Version + */ + public static final String HTTP_PROTOCOL_VERSION = "http.version"; + + /** + * HTTP method + */ + public static final String HTTP_METHOD = "http.method"; + + /** + * HTTP Request URI. + */ + public static final String HTTP_URI = "http.uri"; + + /** + * Rest Request ID. + */ + public static final String REST_REQ_ID = "rest.request_id"; + + /** + * Rest Request Raw Path. + */ + public static final String REST_REQ_RAW_PATH = "rest.raw_path"; + + /** + * Trace key. To be used for on demand sampling. + */ + public static final String TRACE = "trace"; + + /** + * Transport Service send request target host. + */ + public static final String TRANSPORT_TARGET_HOST = "target_host"; + + /** + * Action Name. + */ + public static final String TRANSPORT_ACTION = "action"; +} diff --git a/server/src/main/java/org/opensearch/telemetry/tracing/SpanBuilder.java b/server/src/main/java/org/opensearch/telemetry/tracing/SpanBuilder.java new file mode 100644 index 0000000000000..78924febd3af5 --- /dev/null +++ b/server/src/main/java/org/opensearch/telemetry/tracing/SpanBuilder.java @@ -0,0 +1,124 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.telemetry.tracing; + +import org.opensearch.core.common.Strings; +import org.opensearch.http.HttpRequest; +import org.opensearch.rest.RestRequest; +import org.opensearch.telemetry.tracing.attributes.Attributes; +import org.opensearch.transport.Transport; + +import java.util.Arrays; +import java.util.List; + +/** + * Utility class, helps in creating the {@link SpanCreationContext} for span. + */ +public final class SpanBuilder { + + private static final List HEADERS_TO_BE_ADDED_AS_ATTRIBUTES = Arrays.asList(AttributeNames.TRACE); + /** + * Attribute name Separator + */ + private static final String SEPARATOR = " "; + + /** + * Constructor + */ + private SpanBuilder() { + + } + + /** + * Creates {@link SpanCreationContext} from the {@link HttpRequest} + * @param request Http request. + * @return context. + */ + public static SpanCreationContext from(HttpRequest request) { + return new SpanCreationContext(createSpanName(request), buildSpanAttributes(request)); + } + + /** + * Creates {@link SpanCreationContext} from the {@link RestRequest} + * @param request Rest request + * @return context + */ + public static SpanCreationContext from(RestRequest request) { + return new SpanCreationContext(createSpanName(request), buildSpanAttributes(request)); + } + + /** + * Creates {@link SpanCreationContext} from Transport action and connection details. + * @param action action. + * @param connection transport connection. + * @return context + */ + public static SpanCreationContext from(String action, Transport.Connection connection) { + return new SpanCreationContext(createSpanName(action, connection), buildSpanAttributes(action, connection)); + } + + private static String createSpanName(HttpRequest httpRequest) { + return httpRequest.method().name() + SEPARATOR + httpRequest.uri(); + } + + private static Attributes buildSpanAttributes(HttpRequest httpRequest) { + Attributes attributes = Attributes.create() + .addAttribute(AttributeNames.HTTP_URI, httpRequest.uri()) + .addAttribute(AttributeNames.HTTP_METHOD, httpRequest.method().name()) + .addAttribute(AttributeNames.HTTP_PROTOCOL_VERSION, httpRequest.protocolVersion().name()); + populateHeader(httpRequest, attributes); + return attributes; + } + + private static void populateHeader(HttpRequest httpRequest, Attributes attributes) { + HEADERS_TO_BE_ADDED_AS_ATTRIBUTES.forEach(x -> { + if (httpRequest.getHeaders() != null && httpRequest.getHeaders().get(x) != null) { + attributes.addAttribute(x, Strings.collectionToCommaDelimitedString(httpRequest.getHeaders().get(x))); + } + }); + } + + private static String createSpanName(RestRequest restRequest) { + String spanName = "rest_request"; + if (restRequest != null) { + try { + String methodName = restRequest.method().name(); + // path() does the decoding, which may give error + String path = restRequest.path(); + spanName = methodName + SEPARATOR + path; + } catch (Exception e) { + // swallow the exception and keep the default name. + } + } + return spanName; + } + + private static Attributes buildSpanAttributes(RestRequest restRequest) { + if (restRequest != null) { + return Attributes.create() + .addAttribute(AttributeNames.REST_REQ_ID, restRequest.getRequestId()) + .addAttribute(AttributeNames.REST_REQ_RAW_PATH, restRequest.rawPath()); + } else { + return Attributes.EMPTY; + } + } + + private static String createSpanName(String action, Transport.Connection connection) { + return action + SEPARATOR + (connection.getNode() != null ? connection.getNode().getHostAddress() : null); + } + + private static Attributes buildSpanAttributes(String action, Transport.Connection connection) { + Attributes attributes = Attributes.create().addAttribute(AttributeNames.TRANSPORT_ACTION, action); + if (connection != null && connection.getNode() != null) { + attributes.addAttribute(AttributeNames.TRANSPORT_TARGET_HOST, connection.getNode().getHostAddress()); + } + return attributes; + } + +} diff --git a/server/src/main/java/org/opensearch/telemetry/tracing/ThreadContextBasedTracerContextStorage.java b/server/src/main/java/org/opensearch/telemetry/tracing/ThreadContextBasedTracerContextStorage.java index a32facdc71146..8e50dac169efd 100644 --- a/server/src/main/java/org/opensearch/telemetry/tracing/ThreadContextBasedTracerContextStorage.java +++ b/server/src/main/java/org/opensearch/telemetry/tracing/ThreadContextBasedTracerContextStorage.java @@ -40,9 +40,6 @@ public Span get(String key) { @Override public void put(String key, Span span) { - if (span == null) { - return; - } SpanReference currentSpanRef = threadContext.getTransient(key); if (currentSpanRef == null) { threadContext.putTransient(key, new SpanReference(span)); diff --git a/server/src/main/java/org/opensearch/telemetry/tracing/WrappedTracer.java b/server/src/main/java/org/opensearch/telemetry/tracing/WrappedTracer.java index b699471be7f4c..1fb1eed98f5bb 100644 --- a/server/src/main/java/org/opensearch/telemetry/tracing/WrappedTracer.java +++ b/server/src/main/java/org/opensearch/telemetry/tracing/WrappedTracer.java @@ -38,12 +38,17 @@ public WrappedTracer(TelemetrySettings telemetrySettings, Tracer defaultTracer) } @Override - public SpanScope startSpan(String spanName) { + public Span startSpan(SpanCreationContext context) { + return startSpan(context.getSpanName(), context.getAttributes()); + } + + @Override + public Span startSpan(String spanName) { return startSpan(spanName, Attributes.EMPTY); } @Override - public SpanScope startSpan(String spanName, Attributes attributes) { + public Span startSpan(String spanName, Attributes attributes) { return startSpan(spanName, (SpanContext) null, attributes); } @@ -54,7 +59,22 @@ public SpanContext getCurrentSpan() { } @Override - public SpanScope startSpan(String spanName, SpanContext parentSpan, Attributes attributes) { + public ScopedSpan startScopedSpan(SpanCreationContext spanCreationContext) { + return startScopedSpan(spanCreationContext, null); + } + + @Override + public ScopedSpan startScopedSpan(SpanCreationContext spanCreationContext, SpanContext parentSpan) { + return getDelegateTracer().startScopedSpan(spanCreationContext, parentSpan); + } + + @Override + public SpanScope withSpanInScope(Span span) { + return getDelegateTracer().withSpanInScope(span); + } + + @Override + public Span startSpan(String spanName, SpanContext parentSpan, Attributes attributes) { Tracer delegateTracer = getDelegateTracer(); return delegateTracer.startSpan(spanName, parentSpan, attributes); } @@ -70,7 +90,7 @@ Tracer getDelegateTracer() { } @Override - public SpanScope startSpan(String spanName, Map> headers, Attributes attributes) { + public Span startSpan(String spanName, Map> headers, Attributes attributes) { return defaultTracer.startSpan(spanName, headers, attributes); } } diff --git a/server/src/test/java/org/opensearch/telemetry/tracing/SpanBuilderTests.java b/server/src/test/java/org/opensearch/telemetry/tracing/SpanBuilderTests.java new file mode 100644 index 0000000000000..b4183412cdf02 --- /dev/null +++ b/server/src/test/java/org/opensearch/telemetry/tracing/SpanBuilderTests.java @@ -0,0 +1,158 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.telemetry.tracing; + +import org.opensearch.Version; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.common.network.NetworkAddress; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.common.transport.TransportAddress; +import org.opensearch.core.rest.RestStatus; +import org.opensearch.http.HttpRequest; +import org.opensearch.http.HttpResponse; +import org.opensearch.rest.RestRequest; +import org.opensearch.telemetry.tracing.attributes.Attributes; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.transport.Transport; +import org.opensearch.transport.TransportException; +import org.opensearch.transport.TransportRequest; +import org.opensearch.transport.TransportRequestOptions; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +public class SpanBuilderTests extends OpenSearchTestCase { + + public void testHttpRequestContext() { + HttpRequest httpRequest = createHttpRequest(); + SpanCreationContext context = SpanBuilder.from(httpRequest); + Attributes attributes = context.getAttributes(); + assertEquals("GET /_test", context.getSpanName()); + assertEquals("true", attributes.getAttributesMap().get(AttributeNames.TRACE)); + assertEquals("GET", attributes.getAttributesMap().get(AttributeNames.HTTP_METHOD)); + assertEquals("HTTP_1_0", attributes.getAttributesMap().get(AttributeNames.HTTP_PROTOCOL_VERSION)); + assertEquals("/_test", attributes.getAttributesMap().get(AttributeNames.HTTP_URI)); + } + + public void testRestRequestContext() { + RestRequest restRequest = RestRequest.request(null, createHttpRequest(), null); + SpanCreationContext context = SpanBuilder.from(restRequest); + Attributes attributes = context.getAttributes(); + assertEquals("GET /_test", context.getSpanName()); + assertEquals("/_test", attributes.getAttributesMap().get(AttributeNames.REST_REQ_RAW_PATH)); + assertNotNull(attributes.getAttributesMap().get(AttributeNames.REST_REQ_ID)); + } + + public void testRestRequestContextForNull() { + SpanCreationContext context = SpanBuilder.from((RestRequest) null); + assertEquals("rest_request", context.getSpanName()); + assertEquals(Attributes.EMPTY, context.getAttributes()); + } + + public void testTransportContext() { + String action = "test-action"; + Transport.Connection connection = createTransportConnection(); + SpanCreationContext context = SpanBuilder.from(action, connection); + Attributes attributes = context.getAttributes(); + assertEquals(action + " " + NetworkAddress.format(TransportAddress.META_ADDRESS), context.getSpanName()); + assertEquals(connection.getNode().getHostAddress(), attributes.getAttributesMap().get(AttributeNames.TRANSPORT_TARGET_HOST)); + } + + private static Transport.Connection createTransportConnection() { + return new Transport.Connection() { + @Override + public DiscoveryNode getNode() { + return new DiscoveryNode("local", new TransportAddress(TransportAddress.META_ADDRESS, 9200), Version.V_2_0_0); + } + + @Override + public void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options) + throws IOException, TransportException { + + } + + @Override + public void addCloseListener(ActionListener listener) { + + } + + @Override + public boolean isClosed() { + return false; + } + + @Override + public void close() { + + } + }; + } + + private static HttpRequest createHttpRequest() { + return new HttpRequest() { + @Override + public RestRequest.Method method() { + return RestRequest.Method.GET; + } + + @Override + public String uri() { + return "/_test"; + } + + @Override + public BytesReference content() { + return null; + } + + @Override + public Map> getHeaders() { + return Map.of("trace", Arrays.asList("true")); + } + + @Override + public List strictCookies() { + return null; + } + + @Override + public HttpVersion protocolVersion() { + return HttpVersion.HTTP_1_0; + } + + @Override + public HttpRequest removeHeader(String header) { + return null; + } + + @Override + public HttpResponse createResponse(RestStatus status, BytesReference content) { + return null; + } + + @Override + public Exception getInboundException() { + return null; + } + + @Override + public void release() { + + } + + @Override + public HttpRequest releaseAndCopy() { + return null; + } + }; + } +} diff --git a/server/src/test/java/org/opensearch/telemetry/tracing/TracerFactoryTests.java b/server/src/test/java/org/opensearch/telemetry/tracing/TracerFactoryTests.java index 451a1b9e3eb9c..f202be70c9425 100644 --- a/server/src/test/java/org/opensearch/telemetry/tracing/TracerFactoryTests.java +++ b/server/src/test/java/org/opensearch/telemetry/tracing/TracerFactoryTests.java @@ -15,6 +15,8 @@ import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.telemetry.Telemetry; import org.opensearch.telemetry.TelemetrySettings; +import org.opensearch.telemetry.tracing.attributes.Attributes; +import org.opensearch.telemetry.tracing.noop.NoopSpan; import org.opensearch.telemetry.tracing.noop.NoopTracer; import org.opensearch.test.OpenSearchTestCase; import org.junit.After; @@ -46,7 +48,23 @@ public void testGetTracerWithUnavailableTracingTelemetryReturnsNoopTracer() { Tracer tracer = tracerFactory.getTracer(); assertTrue(tracer instanceof NoopTracer); - assertTrue(tracer.startSpan("foo") == SpanScope.NO_OP); + assertTrue(tracer.startSpan("foo") == NoopSpan.INSTANCE); + assertTrue(tracer.startScopedSpan(new SpanCreationContext("foo", Attributes.EMPTY)) == ScopedSpan.NO_OP); + assertTrue(tracer.startScopedSpan(new SpanCreationContext("foo", Attributes.EMPTY)) == ScopedSpan.NO_OP); + assertTrue(tracer.withSpanInScope(tracer.startSpan("foo")) == SpanScope.NO_OP); + } + + public void testGetTracerWithUnavailableTracingTelemetry() { + Settings settings = Settings.builder().put(TelemetrySettings.TRACER_ENABLED_SETTING.getKey(), false).build(); + TelemetrySettings telemetrySettings = new TelemetrySettings(settings, new ClusterSettings(settings, getClusterSettings())); + Telemetry mockTelemetry = mock(Telemetry.class); + when(mockTelemetry.getTracingTelemetry()).thenReturn(mock(TracingTelemetry.class)); + tracerFactory = new TracerFactory(telemetrySettings, Optional.empty(), new ThreadContext(Settings.EMPTY)); + + Tracer tracer = tracerFactory.getTracer(); + + assertTrue(tracer instanceof NoopTracer); + assertTrue(tracer.startScopedSpan(new SpanCreationContext("foo", Attributes.EMPTY)) == ScopedSpan.NO_OP); } public void testGetTracerWithAvailableTracingTelemetryReturnsWrappedTracer() { diff --git a/test/telemetry/src/main/java/org/opensearch/test/telemetry/tracing/MockSpan.java b/test/telemetry/src/main/java/org/opensearch/test/telemetry/tracing/MockSpan.java index c22a395a6e17c..892c1a8b3eeae 100644 --- a/test/telemetry/src/main/java/org/opensearch/test/telemetry/tracing/MockSpan.java +++ b/test/telemetry/src/main/java/org/opensearch/test/telemetry/tracing/MockSpan.java @@ -36,9 +36,9 @@ public class MockSpan extends AbstractSpan { /** * Base Constructor. - * @param spanName span name - * @param parentSpan parent span - * @param spanProcessor span processor + * @param spanName Span Name + * @param parentSpan Parent Span + * @param spanProcessor Span Processor * @param attributes attributes */ public MockSpan(String spanName, Span parentSpan, SpanProcessor spanProcessor, Attributes attributes) { From f9b6694c0eb4a4a7be326ba143edc41a6eba2f0c Mon Sep 17 00:00:00 2001 From: Marc Handalian Date: Fri, 1 Sep 2023 11:31:06 -0700 Subject: [PATCH 31/37] Fix Segment Replication stats bytes behind metric (#9686) * Fix Segment Replication stats bytes behind metric. This metric currently gives an estimate of the bytes behind based on the difference in size of the segments referenced by the active readers between shards. This does not give a good indication of the amount of bytes that need to be fetched and is inaccurate after deletes and merges. Fixed by sending file metadata with each checkpoint and computing a diff between checkpoints when SegmentReplicationShardStats is built. Signed-off-by: Marc Handalian * Skip SegRep bwc test until this is backported to 2.x. Signed-off-by: Marc Handalian * Add changelog entry. Signed-off-by: Marc Handalian --------- Signed-off-by: Marc Handalian --- CHANGELOG.md | 1 + .../org/opensearch/backwards/IndexingIT.java | 1 + .../index/seqno/ReplicationTracker.java | 28 ++++----- .../opensearch/index/shard/IndexShard.java | 6 +- .../metadata/RemoteSegmentMetadata.java | 29 +++++++-- .../checkpoint/ReplicationCheckpoint.java | 34 ++++++++-- .../index/seqno/ReplicationTrackerTests.java | 25 +++++--- .../SegmentReplicationIndexShardTests.java | 62 +++++++++++++++++++ 8 files changed, 154 insertions(+), 32 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index d8d68c51ceed5..63e4486012a67 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -188,6 +188,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Fix condition to remove index create block ([#9437](https://github.com/opensearch-project/OpenSearch/pull/9437)) - Add support to clear archived index setting ([#9019](https://github.com/opensearch-project/OpenSearch/pull/9019)) - [Segment Replication] Fixed bug where replica shard temporarily serves stale data during an engine reset ([#9495](https://github.com/opensearch-project/OpenSearch/pull/9495)) +- [Segment Replication] Fixed bug where bytes behind metric is not accurate ([#9686](https://github.com/opensearch-project/OpenSearch/pull/9686)) ### Security diff --git a/qa/mixed-cluster/src/test/java/org/opensearch/backwards/IndexingIT.java b/qa/mixed-cluster/src/test/java/org/opensearch/backwards/IndexingIT.java index 686fc78dcec8a..75083a929b491 100644 --- a/qa/mixed-cluster/src/test/java/org/opensearch/backwards/IndexingIT.java +++ b/qa/mixed-cluster/src/test/java/org/opensearch/backwards/IndexingIT.java @@ -114,6 +114,7 @@ private void printClusterRouting() throws IOException, ParseException { * This test verifies that segment replication does not break when primary shards are on lower OS version. It does this * by verifying replica shards contains same number of documents as primary's. */ + @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/9685") public void testIndexingWithPrimaryOnBwcNodes() throws Exception { if (UPGRADE_FROM_VERSION.before(Version.V_2_4_0)) { logger.info("--> Skip test for version {} where segment replication feature is not available", UPGRADE_FROM_VERSION); diff --git a/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java b/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java index 48bfce1013f17..4b6d72b86ff62 100644 --- a/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java +++ b/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java @@ -57,6 +57,8 @@ import org.opensearch.index.shard.AbstractIndexShardComponent; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.ReplicationGroup; +import org.opensearch.index.store.Store; +import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.replication.common.SegmentReplicationLagTimer; @@ -1290,27 +1292,25 @@ public synchronized Set getSegmentReplicationStats && entry.getValue().inSync && replicationGroup.getUnavailableInSyncShards().contains(entry.getKey()) == false ) - .map(entry -> buildShardStats(latestReplicationCheckpoint.getLength(), entry.getKey(), entry.getValue())) + .map(entry -> buildShardStats(entry.getKey(), entry.getValue())) .collect(Collectors.toUnmodifiableSet()); } return Collections.emptySet(); } - private SegmentReplicationShardStats buildShardStats( - final long latestCheckpointLength, - final String allocationId, - final CheckpointState checkpointState - ) { - final Map checkpointTimers = checkpointState.checkpointTimers; + private SegmentReplicationShardStats buildShardStats(final String allocationId, final CheckpointState cps) { + final Store.RecoveryDiff diff = Store.segmentReplicationDiff( + latestReplicationCheckpoint.getMetadataMap(), + cps.visibleReplicationCheckpoint != null ? cps.visibleReplicationCheckpoint.getMetadataMap() : Collections.emptyMap() + ); + final long bytesBehind = diff.missing.stream().mapToLong(StoreFileMetadata::length).sum(); return new SegmentReplicationShardStats( allocationId, - checkpointTimers.size(), - checkpointState.visibleReplicationCheckpoint == null - ? latestCheckpointLength - : Math.max(latestCheckpointLength - checkpointState.visibleReplicationCheckpoint.getLength(), 0), - checkpointTimers.values().stream().mapToLong(SegmentReplicationLagTimer::time).max().orElse(0), - checkpointTimers.values().stream().mapToLong(SegmentReplicationLagTimer::totalElapsedTime).max().orElse(0), - checkpointState.lastCompletedReplicationLag + cps.checkpointTimers.size(), + bytesBehind, + cps.checkpointTimers.values().stream().mapToLong(SegmentReplicationLagTimer::time).max().orElse(0), + cps.checkpointTimers.values().stream().mapToLong(SegmentReplicationLagTimer::totalElapsedTime).max().orElse(0), + cps.lastCompletedReplicationLag ); } diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index e398cab23a085..352876e54547e 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -1610,6 +1610,7 @@ public Tuple, ReplicationCheckpoint> getLatestSegme snapshot = getSegmentInfosSnapshot(); if (snapshot.get() != null) { SegmentInfos segmentInfos = snapshot.get(); + final Map metadataMap = store.getSegmentMetadataMap(segmentInfos); return new Tuple<>( snapshot, new ReplicationCheckpoint( @@ -1617,8 +1618,9 @@ public Tuple, ReplicationCheckpoint> getLatestSegme getOperationPrimaryTerm(), segmentInfos.getGeneration(), segmentInfos.getVersion(), - store.getSegmentMetadataMap(segmentInfos).values().stream().mapToLong(StoreFileMetadata::length).sum(), - getEngine().config().getCodec().getName() + metadataMap.values().stream().mapToLong(StoreFileMetadata::length).sum(), + getEngine().config().getCodec().getName(), + metadataMap ) ); } diff --git a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java index 207620573886d..1cec20ec3f6cc 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java +++ b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java @@ -10,13 +10,16 @@ import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.Version; import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; import org.opensearch.index.store.RemoteSegmentStoreDirectory; +import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import java.io.IOException; import java.util.Map; +import java.util.function.Function; import java.util.stream.Collectors; /** @@ -110,11 +113,13 @@ public void write(IndexOutput out) throws IOException { public static RemoteSegmentMetadata read(IndexInput indexInput) throws IOException { Map metadata = indexInput.readMapOfStrings(); - ReplicationCheckpoint replicationCheckpoint = readCheckpointFromIndexInput(indexInput); + final Map uploadedSegmentMetadataMap = RemoteSegmentMetadata + .fromMapOfStrings(metadata); + ReplicationCheckpoint replicationCheckpoint = readCheckpointFromIndexInput(indexInput, uploadedSegmentMetadataMap); int byteArraySize = (int) indexInput.readLong(); byte[] segmentInfosBytes = new byte[byteArraySize]; indexInput.readBytes(segmentInfosBytes, 0, byteArraySize); - return new RemoteSegmentMetadata(RemoteSegmentMetadata.fromMapOfStrings(metadata), segmentInfosBytes, replicationCheckpoint); + return new RemoteSegmentMetadata(uploadedSegmentMetadataMap, segmentInfosBytes, replicationCheckpoint); } public static void writeCheckpointToIndexOutput(ReplicationCheckpoint replicationCheckpoint, IndexOutput out) throws IOException { @@ -131,14 +136,30 @@ public static void writeCheckpointToIndexOutput(ReplicationCheckpoint replicatio out.writeString(replicationCheckpoint.getCodec()); } - private static ReplicationCheckpoint readCheckpointFromIndexInput(IndexInput in) throws IOException { + private static ReplicationCheckpoint readCheckpointFromIndexInput( + IndexInput in, + Map uploadedSegmentMetadataMap + ) throws IOException { return new ReplicationCheckpoint( new ShardId(new Index(in.readString(), in.readString()), in.readVInt()), in.readLong(), in.readLong(), in.readLong(), in.readLong(), - in.readString() + in.readString(), + toStoreFileMetadata(uploadedSegmentMetadataMap) ); } + + private static Map toStoreFileMetadata( + Map metadata + ) { + return metadata.entrySet() + .stream() + // TODO: Version here should be read from UploadedSegmentMetadata. + .map( + entry -> new StoreFileMetadata(entry.getKey(), entry.getValue().getLength(), entry.getValue().getChecksum(), Version.LATEST) + ) + .collect(Collectors.toMap(StoreFileMetadata::name, Function.identity())); + } } diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java index 70c3e71ba18b9..521522803c726 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java @@ -15,8 +15,11 @@ import org.opensearch.core.common.io.stream.Writeable; import org.opensearch.core.index.shard.ShardId; import org.opensearch.index.seqno.SequenceNumbers; +import org.opensearch.index.store.StoreFileMetadata; import java.io.IOException; +import java.util.Collections; +import java.util.Map; import java.util.Objects; /** @@ -32,6 +35,7 @@ public class ReplicationCheckpoint implements Writeable, Comparable metadataMap; public static ReplicationCheckpoint empty(ShardId shardId) { return empty(shardId, ""); @@ -48,19 +52,29 @@ private ReplicationCheckpoint(ShardId shardId, String codec) { segmentInfosVersion = SequenceNumbers.NO_OPS_PERFORMED; length = 0L; this.codec = codec; + this.metadataMap = Collections.emptyMap(); } public ReplicationCheckpoint(ShardId shardId, long primaryTerm, long segmentsGen, long segmentInfosVersion, String codec) { - this(shardId, primaryTerm, segmentsGen, segmentInfosVersion, 0L, codec); - } - - public ReplicationCheckpoint(ShardId shardId, long primaryTerm, long segmentsGen, long segmentInfosVersion, long length, String codec) { + this(shardId, primaryTerm, segmentsGen, segmentInfosVersion, 0L, codec, Collections.emptyMap()); + } + + public ReplicationCheckpoint( + ShardId shardId, + long primaryTerm, + long segmentsGen, + long segmentInfosVersion, + long length, + String codec, + Map metadataMap + ) { this.shardId = shardId; this.primaryTerm = primaryTerm; this.segmentsGen = segmentsGen; this.segmentInfosVersion = segmentInfosVersion; this.length = length; this.codec = codec; + this.metadataMap = metadataMap; } public ReplicationCheckpoint(StreamInput in) throws IOException { @@ -75,6 +89,11 @@ public ReplicationCheckpoint(StreamInput in) throws IOException { length = 0L; codec = null; } + if (in.getVersion().onOrAfter(Version.V_2_10_0)) { + this.metadataMap = in.readMap(StreamInput::readString, StoreFileMetadata::new); + } else { + this.metadataMap = Collections.emptyMap(); + } } /** @@ -135,6 +154,9 @@ public void writeTo(StreamOutput out) throws IOException { out.writeLong(length); out.writeString(codec); } + if (out.getVersion().onOrAfter(Version.V_2_10_0)) { + out.writeMap(metadataMap, StreamOutput::writeString, (valueOut, fc) -> fc.writeTo(valueOut)); + } } @Override @@ -169,6 +191,10 @@ public boolean isAheadOf(@Nullable ReplicationCheckpoint other) { || (primaryTerm == other.getPrimaryTerm() && segmentInfosVersion > other.getSegmentInfosVersion()); } + public Map getMetadataMap() { + return metadataMap; + } + @Override public String toString() { return "ReplicationCheckpoint{" diff --git a/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTests.java b/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTests.java index ab87d31d15e2f..28c95ddf13fc4 100644 --- a/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTests.java +++ b/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTests.java @@ -33,6 +33,7 @@ package org.opensearch.index.seqno; import org.apache.lucene.codecs.Codec; +import org.apache.lucene.util.Version; import org.opensearch.action.support.replication.ReplicationResponse; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.routing.AllocationId; @@ -50,6 +51,7 @@ import org.opensearch.core.index.shard.ShardId; import org.opensearch.index.IndexSettings; import org.opensearch.index.SegmentReplicationShardStats; +import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.indices.replication.common.SegmentReplicationLagTimer; @@ -1826,29 +1828,35 @@ public void testSegmentReplicationCheckpointTracking() { initializingIds.forEach(aId -> markAsTrackingAndInSyncQuietly(tracker, aId.getId(), NO_OPS_PERFORMED)); + final StoreFileMetadata segment_1 = new StoreFileMetadata("segment_1", 1L, "abcd", Version.LATEST); + final StoreFileMetadata segment_2 = new StoreFileMetadata("segment_2", 50L, "abcd", Version.LATEST); + final StoreFileMetadata segment_3 = new StoreFileMetadata("segment_3", 100L, "abcd", Version.LATEST); final ReplicationCheckpoint initialCheckpoint = new ReplicationCheckpoint( tracker.shardId(), 0L, 1, 1, 1L, - Codec.getDefault().getName() + Codec.getDefault().getName(), + Map.of("segment_1", segment_1) ); final ReplicationCheckpoint secondCheckpoint = new ReplicationCheckpoint( tracker.shardId(), 0L, 2, 2, - 50L, - Codec.getDefault().getName() + 51L, + Codec.getDefault().getName(), + Map.of("segment_1", segment_1, "segment_2", segment_2) ); final ReplicationCheckpoint thirdCheckpoint = new ReplicationCheckpoint( tracker.shardId(), 0L, 2, 3, - 100L, - Codec.getDefault().getName() + 151L, + Codec.getDefault().getName(), + Map.of("segment_1", segment_1, "segment_2", segment_2, "segment_3", segment_3) ); tracker.setLatestReplicationCheckpoint(initialCheckpoint); @@ -1864,7 +1872,7 @@ public void testSegmentReplicationCheckpointTracking() { assertEquals(expectedIds.size(), groupStats.size()); for (SegmentReplicationShardStats shardStat : groupStats) { assertEquals(3, shardStat.getCheckpointsBehindCount()); - assertEquals(100L, shardStat.getBytesBehindCount()); + assertEquals(151L, shardStat.getBytesBehindCount()); assertTrue(shardStat.getCurrentReplicationLagMillis() >= shardStat.getCurrentReplicationTimeMillis()); } @@ -1881,7 +1889,7 @@ public void testSegmentReplicationCheckpointTracking() { assertEquals(expectedIds.size(), groupStats.size()); for (SegmentReplicationShardStats shardStat : groupStats) { assertEquals(2, shardStat.getCheckpointsBehindCount()); - assertEquals(99L, shardStat.getBytesBehindCount()); + assertEquals(150L, shardStat.getBytesBehindCount()); } for (String id : expectedIds) { @@ -1938,7 +1946,8 @@ public void testSegmentReplicationCheckpointTrackingInvalidAllocationIDs() { 1, 1, 1L, - Codec.getDefault().getName() + Codec.getDefault().getName(), + Collections.emptyMap() ); tracker.setLatestReplicationCheckpoint(initialCheckpoint); tracker.startReplicationLagTimers(initialCheckpoint); diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java index 29daa3936e8bb..b7972810dddb9 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java @@ -30,6 +30,8 @@ import org.opensearch.core.action.ActionListener; import org.opensearch.core.xcontent.MediaTypeRegistry; import org.opensearch.index.IndexSettings; +import org.opensearch.index.SegmentReplicationShardStats; +import org.opensearch.index.engine.DocIdSeqNoAndSource; import org.opensearch.index.engine.Engine; import org.opensearch.index.engine.InternalEngineFactory; import org.opensearch.index.engine.NRTReplicationEngine; @@ -69,6 +71,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; @@ -805,6 +808,65 @@ public void testQueryDuringEngineResetShowsDocs() throws Exception { } } + public void testSegmentReplicationStats() throws Exception { + final NRTReplicationEngineFactory engineFactory = new NRTReplicationEngineFactory(); + final NRTReplicationEngineFactory spy = spy(engineFactory); + try (ReplicationGroup shards = createGroup(1, settings, indexMapping, spy, createTempDir())) { + final IndexShard primaryShard = shards.getPrimary(); + final IndexShard replicaShard = shards.getReplicas().get(0); + shards.startAll(); + + assertReplicaCaughtUp(primaryShard); + + shards.indexDocs(10); + shards.refresh("test"); + + final ReplicationCheckpoint primaryCheckpoint = primaryShard.getLatestReplicationCheckpoint(); + final long initialCheckpointSize = primaryCheckpoint.getMetadataMap() + .values() + .stream() + .mapToLong(StoreFileMetadata::length) + .sum(); + + Set postRefreshStats = primaryShard.getReplicationStats(); + SegmentReplicationShardStats shardStats = postRefreshStats.stream().findFirst().get(); + assertEquals(1, shardStats.getCheckpointsBehindCount()); + assertEquals(initialCheckpointSize, shardStats.getBytesBehindCount()); + replicateSegments(primaryShard, shards.getReplicas()); + assertReplicaCaughtUp(primaryShard); + shards.assertAllEqual(10); + + final List docIdAndSeqNos = getDocIdAndSeqNos(primaryShard); + for (DocIdSeqNoAndSource docIdAndSeqNo : docIdAndSeqNos.subList(0, 5)) { + deleteDoc(primaryShard, docIdAndSeqNo.getId()); + // delete on replica for xlog. + deleteDoc(replicaShard, docIdAndSeqNo.getId()); + } + primaryShard.forceMerge(new ForceMergeRequest().maxNumSegments(1).flush(true)); + + final Map segmentMetadataMap = primaryShard.getSegmentMetadataMap(); + final Store.RecoveryDiff diff = Store.segmentReplicationDiff(segmentMetadataMap, replicaShard.getSegmentMetadataMap()); + final long sizeAfterDeleteAndCommit = diff.missing.stream().mapToLong(StoreFileMetadata::length).sum(); + + final Set statsAfterFlush = primaryShard.getReplicationStats(); + shardStats = statsAfterFlush.stream().findFirst().get(); + assertEquals(sizeAfterDeleteAndCommit, shardStats.getBytesBehindCount()); + assertEquals(1, shardStats.getCheckpointsBehindCount()); + + replicateSegments(primaryShard, shards.getReplicas()); + assertReplicaCaughtUp(primaryShard); + shards.assertAllEqual(5); + } + } + + private void assertReplicaCaughtUp(IndexShard primaryShard) { + Set initialStats = primaryShard.getReplicationStats(); + assertEquals(initialStats.size(), 1); + SegmentReplicationShardStats shardStats = initialStats.stream().findFirst().get(); + assertEquals(0, shardStats.getCheckpointsBehindCount()); + assertEquals(0, shardStats.getBytesBehindCount()); + } + /** * Assert persisted and searchable doc counts. This method should not be used while docs are concurrently indexed because * it asserts point in time seqNos are relative to the doc counts. From 070663574e07b53a882c9d0dea176b415c310605 Mon Sep 17 00:00:00 2001 From: Sooraj Sinha Date: Fri, 25 Aug 2023 12:49:03 +0530 Subject: [PATCH 32/37] Upload all index metadata to remote Signed-off-by: Sooraj Sinha --- .../gateway/remote/ClusterMetadataMarker.java | 381 ++++++++++++++++++ .../remote/ClusterMetadataMarkerTests.java | 46 +++ 2 files changed, 427 insertions(+) create mode 100644 server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataMarker.java create mode 100644 server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataMarkerTests.java diff --git a/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataMarker.java b/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataMarker.java new file mode 100644 index 0000000000000..14be75ef166b6 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataMarker.java @@ -0,0 +1,381 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway.remote; + +import org.opensearch.Version; +import org.opensearch.core.ParseField; +import org.opensearch.core.common.Strings; +import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.common.io.stream.Writeable; +import org.opensearch.core.xcontent.ConstructingObjectParser; +import org.opensearch.core.xcontent.MediaTypeRegistry; +import org.opensearch.core.xcontent.ToXContentFragment; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +/** + * Marker file which contains the details of the uploaded entity metadata + * + * @opensearch.internal + */ +public class ClusterMetadataMarker implements Writeable, ToXContentFragment { + + private static final ParseField CLUSTER_TERM_FIELD = new ParseField("cluster_term"); + private static final ParseField STATE_VERSION_FIELD = new ParseField("state_version"); + private static final ParseField CLUSTER_UUID_FIELD = new ParseField("cluster_uuid"); + private static final ParseField STATE_UUID_FIELD = new ParseField("state_uuid"); + private static final ParseField OPENSEARCH_VERSION_FIELD = new ParseField("opensearch_version"); + private static final ParseField COMMITTED_FIELD = new ParseField("committed"); + private static final ParseField INDICES_FIELD = new ParseField("indices"); + + private static long term(Object[] fields) { + return (long) fields[0]; + } + + private static long version(Object[] fields) { + return (long) fields[1]; + } + + private static String clusterUUID(Object[] fields) { + return (String) fields[2]; + } + + private static String stateUUID(Object[] fields) { + return (String) fields[3]; + } + + private static Version opensearchVersion(Object[] fields) { + return Version.fromId((int) fields[4]); + } + + private static boolean committed(Object[] fields) { + return (boolean) fields[5]; + } + + private static List indices(Object[] fields) { + return (List) fields[6]; + } + + private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + "cluster_metadata_marker", + fields -> new ClusterMetadataMarker( + term(fields), + version(fields), + clusterUUID(fields), + stateUUID(fields), + opensearchVersion(fields), + committed(fields), + indices(fields) + ) + ); + + static { + PARSER.declareLong(ConstructingObjectParser.constructorArg(), CLUSTER_TERM_FIELD); + PARSER.declareLong(ConstructingObjectParser.constructorArg(), STATE_VERSION_FIELD); + PARSER.declareString(ConstructingObjectParser.constructorArg(), CLUSTER_UUID_FIELD); + PARSER.declareString(ConstructingObjectParser.constructorArg(), STATE_UUID_FIELD); + PARSER.declareInt(ConstructingObjectParser.constructorArg(), OPENSEARCH_VERSION_FIELD); + PARSER.declareBoolean(ConstructingObjectParser.constructorArg(), COMMITTED_FIELD); + PARSER.declareObjectArray( + ConstructingObjectParser.constructorArg(), + (p, c) -> UploadedIndexMetadata.fromXContent(p), + INDICES_FIELD + ); + } + + private final List indices; + private final long clusterTerm; + private final long stateVersion; + private final String clusterUUID; + private final String stateUUID; + private final Version opensearchVersion; + private final boolean committed; + + public List getIndices() { + return indices; + } + + public long getClusterTerm() { + return clusterTerm; + } + + public long getStateVersion() { + return stateVersion; + } + + public String getClusterUUID() { + return clusterUUID; + } + + public String getStateUUID() { + return stateUUID; + } + + public Version getOpensearchVersion() { + return opensearchVersion; + } + + public boolean isCommitted() { + return committed; + } + + public ClusterMetadataMarker( + long clusterTerm, + long version, + String clusterUUID, + String stateUUID, + Version opensearchVersion, + boolean committed, + List indices + ) { + this.clusterTerm = clusterTerm; + this.stateVersion = version; + this.clusterUUID = clusterUUID; + this.stateUUID = stateUUID; + this.opensearchVersion = opensearchVersion; + this.committed = committed; + this.indices = Collections.unmodifiableList(indices); + } + + public static Builder builder() { + return new Builder(); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.field(CLUSTER_TERM_FIELD.getPreferredName(), getClusterTerm()) + .field(STATE_VERSION_FIELD.getPreferredName(), getStateVersion()) + .field(CLUSTER_UUID_FIELD.getPreferredName(), getClusterUUID()) + .field(STATE_UUID_FIELD.getPreferredName(), getStateUUID()) + .field(OPENSEARCH_VERSION_FIELD.getPreferredName(), getOpensearchVersion().id) + .field(COMMITTED_FIELD.getPreferredName(), isCommitted()); + builder.startArray(INDICES_FIELD.getPreferredName()); + { + for (UploadedIndexMetadata uploadedIndexMetadata : indices) { + uploadedIndexMetadata.toXContent(builder, params); + } + } + builder.endArray(); + return builder; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVLong(clusterTerm); + out.writeVLong(stateVersion); + out.writeString(clusterUUID); + out.writeString(stateUUID); + out.writeInt(opensearchVersion.id); + out.writeBoolean(committed); + out.writeCollection(indices); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final ClusterMetadataMarker that = (ClusterMetadataMarker) o; + return Objects.equals(indices, that.indices) + && clusterTerm == that.clusterTerm + && stateVersion == that.stateVersion + && Objects.equals(clusterUUID, that.clusterUUID) + && Objects.equals(stateUUID, that.stateUUID) + && Objects.equals(opensearchVersion, that.opensearchVersion) + && Objects.equals(committed, that.committed); + } + + @Override + public int hashCode() { + return Objects.hash(indices, clusterTerm, stateVersion, clusterUUID, stateUUID, opensearchVersion, committed); + } + + @Override + public String toString() { + return Strings.toString(MediaTypeRegistry.JSON, this); + } + + public static ClusterMetadataMarker fromXContent(XContentParser parser) throws IOException { + return PARSER.parse(parser, null); + } + + /** + * Builder for ClusterMetadataMarker + * + * @opensearch.internal + */ + public static class Builder { + + private List indices; + private long clusterTerm; + private long stateVersion; + private String clusterUUID; + private String stateUUID; + private Version opensearchVersion; + private boolean committed; + + public Builder indices(List indices) { + this.indices = indices; + return this; + } + + public Builder clusterTerm(long clusterTerm) { + this.clusterTerm = clusterTerm; + return this; + } + + public Builder stateVersion(long stateVersion) { + this.stateVersion = stateVersion; + return this; + } + + public Builder clusterUUID(String clusterUUID) { + this.clusterUUID = clusterUUID; + return this; + } + + public Builder stateUUID(String stateUUID) { + this.stateUUID = stateUUID; + return this; + } + + public Builder opensearchVersion(Version opensearchVersion) { + this.opensearchVersion = opensearchVersion; + return this; + } + + public Builder committed(boolean committed) { + this.committed = committed; + return this; + } + + public List getIndices() { + return indices; + } + + public Builder() { + indices = new ArrayList<>(); + } + + public ClusterMetadataMarker build() { + return new ClusterMetadataMarker(clusterTerm, stateVersion, clusterUUID, stateUUID, opensearchVersion, committed, indices); + } + + } + + /** + * Metadata for uploaded index metadata + * + * @opensearch.internal + */ + public static class UploadedIndexMetadata implements Writeable, ToXContentFragment { + + private static final ParseField INDEX_NAME_FIELD = new ParseField("index_name"); + private static final ParseField INDEX_UUID_FIELD = new ParseField("index_uuid"); + private static final ParseField UPLOADED_FILENAME_FIELD = new ParseField("uploaded_filename"); + + private static String indexName(Object[] fields) { + return (String) fields[0]; + } + + private static String indexUUID(Object[] fields) { + return (String) fields[1]; + } + + private static String uploadedFilename(Object[] fields) { + return (String) fields[2]; + } + + private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + "uploaded_index_metadata", + fields -> new UploadedIndexMetadata(indexName(fields), indexUUID(fields), uploadedFilename(fields)) + ); + + static { + PARSER.declareString(ConstructingObjectParser.constructorArg(), INDEX_NAME_FIELD); + PARSER.declareString(ConstructingObjectParser.constructorArg(), INDEX_UUID_FIELD); + PARSER.declareString(ConstructingObjectParser.constructorArg(), UPLOADED_FILENAME_FIELD); + } + + private final String indexName; + private final String indexUUID; + private final String uploadedFilename; + + public UploadedIndexMetadata(String indexName, String indexUUID, String uploadedFileName) { + this.indexName = indexName; + this.indexUUID = indexUUID; + this.uploadedFilename = uploadedFileName; + } + + public String getUploadedFilename() { + return uploadedFilename; + } + + public String getIndexName() { + return indexName; + } + + public String getIndexUUID() { + return indexUUID; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return builder.startObject() + .field(INDEX_NAME_FIELD.getPreferredName(), getIndexName()) + .field(INDEX_UUID_FIELD.getPreferredName(), getIndexUUID()) + .field(UPLOADED_FILENAME_FIELD.getPreferredName(), getUploadedFilename()) + .endObject(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(indexName); + out.writeString(indexUUID); + out.writeString(uploadedFilename); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final UploadedIndexMetadata that = (UploadedIndexMetadata) o; + return Objects.equals(indexName, that.indexName) + && Objects.equals(indexUUID, that.indexUUID) + && Objects.equals(uploadedFilename, that.uploadedFilename); + } + + @Override + public int hashCode() { + return Objects.hash(indexName, indexUUID, uploadedFilename); + } + + @Override + public String toString() { + return Strings.toString(MediaTypeRegistry.JSON, this); + } + + public static UploadedIndexMetadata fromXContent(XContentParser parser) throws IOException { + return PARSER.parse(parser, null); + } + } +} diff --git a/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataMarkerTests.java b/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataMarkerTests.java new file mode 100644 index 0000000000000..d2d7b85579a89 --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataMarkerTests.java @@ -0,0 +1,46 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.gateway.remote; + +import org.opensearch.Version; +import org.opensearch.common.xcontent.json.JsonXContent; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; +import org.opensearch.gateway.remote.ClusterMetadataMarker.UploadedIndexMetadata; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; +import java.util.Collections; + +public class ClusterMetadataMarkerTests extends OpenSearchTestCase { + + public void testXContent() throws IOException { + UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "test-uuid", "/test/upload/path"); + ClusterMetadataMarker originalMarker = new ClusterMetadataMarker( + 1L, + 1L, + "test-cluster-uuid", + "test-state-uuid", + Version.CURRENT, + false, + Collections.singletonList(uploadedIndexMetadata) + ); + final XContentBuilder builder = JsonXContent.contentBuilder(); + builder.startObject(); + originalMarker.toXContent(builder, ToXContent.EMPTY_PARAMS); + builder.endObject(); + + try (XContentParser parser = createParser(JsonXContent.jsonXContent, BytesReference.bytes(builder))) { + final ClusterMetadataMarker fromXContentMarker = ClusterMetadataMarker.fromXContent(parser); + assertEquals(originalMarker, fromXContentMarker); + } + } +} From 686290b110a9263e9a158cb090ef957670d8742d Mon Sep 17 00:00:00 2001 From: Sooraj Sinha Date: Sun, 27 Aug 2023 21:28:41 +0530 Subject: [PATCH 33/37] Add nodeId field in marker Signed-off-by: Sooraj Sinha --- .../gateway/remote/ClusterMetadataMarker.java | 73 ++++++++++++- .../remote/ClusterMetadataMarkerTests.java | 101 +++++++++++++++++- 2 files changed, 169 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataMarker.java b/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataMarker.java index 14be75ef166b6..b7c89965c6ac1 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataMarker.java +++ b/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataMarker.java @@ -11,6 +11,7 @@ import org.opensearch.Version; import org.opensearch.core.ParseField; 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.common.io.stream.Writeable; import org.opensearch.core.xcontent.ConstructingObjectParser; @@ -37,6 +38,7 @@ public class ClusterMetadataMarker implements Writeable, ToXContentFragment { private static final ParseField CLUSTER_UUID_FIELD = new ParseField("cluster_uuid"); private static final ParseField STATE_UUID_FIELD = new ParseField("state_uuid"); private static final ParseField OPENSEARCH_VERSION_FIELD = new ParseField("opensearch_version"); + private static final ParseField NODE_ID_FIELD = new ParseField("node_id"); private static final ParseField COMMITTED_FIELD = new ParseField("committed"); private static final ParseField INDICES_FIELD = new ParseField("indices"); @@ -60,12 +62,16 @@ private static Version opensearchVersion(Object[] fields) { return Version.fromId((int) fields[4]); } + private static String nodeId(Object[] fields) { + return (String) fields[5]; + } + private static boolean committed(Object[] fields) { - return (boolean) fields[5]; + return (boolean) fields[6]; } private static List indices(Object[] fields) { - return (List) fields[6]; + return (List) fields[7]; } private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( @@ -76,6 +82,7 @@ private static List indices(Object[] fields) { clusterUUID(fields), stateUUID(fields), opensearchVersion(fields), + nodeId(fields), committed(fields), indices(fields) ) @@ -87,6 +94,7 @@ private static List indices(Object[] fields) { PARSER.declareString(ConstructingObjectParser.constructorArg(), CLUSTER_UUID_FIELD); PARSER.declareString(ConstructingObjectParser.constructorArg(), STATE_UUID_FIELD); PARSER.declareInt(ConstructingObjectParser.constructorArg(), OPENSEARCH_VERSION_FIELD); + PARSER.declareString(ConstructingObjectParser.constructorArg(), NODE_ID_FIELD); PARSER.declareBoolean(ConstructingObjectParser.constructorArg(), COMMITTED_FIELD); PARSER.declareObjectArray( ConstructingObjectParser.constructorArg(), @@ -101,6 +109,7 @@ private static List indices(Object[] fields) { private final String clusterUUID; private final String stateUUID; private final Version opensearchVersion; + private final String nodeId; private final boolean committed; public List getIndices() { @@ -127,6 +136,10 @@ public Version getOpensearchVersion() { return opensearchVersion; } + public String getNodeId() { + return nodeId; + } + public boolean isCommitted() { return committed; } @@ -137,6 +150,7 @@ public ClusterMetadataMarker( String clusterUUID, String stateUUID, Version opensearchVersion, + String nodeId, boolean committed, List indices ) { @@ -145,14 +159,30 @@ public ClusterMetadataMarker( this.clusterUUID = clusterUUID; this.stateUUID = stateUUID; this.opensearchVersion = opensearchVersion; + this.nodeId = nodeId; this.committed = committed; this.indices = Collections.unmodifiableList(indices); } + public ClusterMetadataMarker(StreamInput in) throws IOException { + this.clusterTerm = in.readVLong(); + this.stateVersion = in.readVLong(); + this.clusterUUID = in.readString(); + this.stateUUID = in.readString(); + this.opensearchVersion = Version.fromId(in.readInt()); + this.nodeId = in.readString(); + this.committed = in.readBoolean(); + this.indices = Collections.unmodifiableList(in.readList(UploadedIndexMetadata::new)); + } + public static Builder builder() { return new Builder(); } + public static Builder builder(ClusterMetadataMarker marker) { + return new Builder(marker); + } + @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.field(CLUSTER_TERM_FIELD.getPreferredName(), getClusterTerm()) @@ -160,6 +190,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws .field(CLUSTER_UUID_FIELD.getPreferredName(), getClusterUUID()) .field(STATE_UUID_FIELD.getPreferredName(), getStateUUID()) .field(OPENSEARCH_VERSION_FIELD.getPreferredName(), getOpensearchVersion().id) + .field(NODE_ID_FIELD.getPreferredName(), getNodeId()) .field(COMMITTED_FIELD.getPreferredName(), isCommitted()); builder.startArray(INDICES_FIELD.getPreferredName()); { @@ -178,6 +209,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(clusterUUID); out.writeString(stateUUID); out.writeInt(opensearchVersion.id); + out.writeString(nodeId); out.writeBoolean(committed); out.writeCollection(indices); } @@ -197,12 +229,13 @@ public boolean equals(Object o) { && Objects.equals(clusterUUID, that.clusterUUID) && Objects.equals(stateUUID, that.stateUUID) && Objects.equals(opensearchVersion, that.opensearchVersion) + && Objects.equals(nodeId, that.nodeId) && Objects.equals(committed, that.committed); } @Override public int hashCode() { - return Objects.hash(indices, clusterTerm, stateVersion, clusterUUID, stateUUID, opensearchVersion, committed); + return Objects.hash(indices, clusterTerm, stateVersion, clusterUUID, stateUUID, opensearchVersion, nodeId, committed); } @Override @@ -227,6 +260,7 @@ public static class Builder { private String clusterUUID; private String stateUUID; private Version opensearchVersion; + private String nodeId; private boolean committed; public Builder indices(List indices) { @@ -259,6 +293,11 @@ public Builder opensearchVersion(Version opensearchVersion) { return this; } + public Builder nodeId(String nodeId) { + this.nodeId = nodeId; + return this; + } + public Builder committed(boolean committed) { this.committed = committed; return this; @@ -272,8 +311,28 @@ public Builder() { indices = new ArrayList<>(); } + public Builder(ClusterMetadataMarker marker) { + this.clusterTerm = marker.clusterTerm; + this.stateVersion = marker.stateVersion; + this.clusterUUID = marker.clusterUUID; + this.stateUUID = marker.stateUUID; + this.opensearchVersion = marker.opensearchVersion; + this.nodeId = marker.nodeId; + this.committed = marker.committed; + this.indices = new ArrayList<>(marker.indices); + } + public ClusterMetadataMarker build() { - return new ClusterMetadataMarker(clusterTerm, stateVersion, clusterUUID, stateUUID, opensearchVersion, committed, indices); + return new ClusterMetadataMarker( + clusterTerm, + stateVersion, + clusterUUID, + stateUUID, + opensearchVersion, + nodeId, + committed, + indices + ); } } @@ -322,6 +381,12 @@ public UploadedIndexMetadata(String indexName, String indexUUID, String uploaded this.uploadedFilename = uploadedFileName; } + public UploadedIndexMetadata(StreamInput in) throws IOException { + this.indexName = in.readString(); + this.indexUUID = in.readString(); + this.uploadedFilename = in.readString(); + } + public String getUploadedFilename() { return uploadedFilename; } diff --git a/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataMarkerTests.java b/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataMarkerTests.java index d2d7b85579a89..22a3383ac035e 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataMarkerTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataMarkerTests.java @@ -11,18 +11,23 @@ import org.opensearch.Version; import org.opensearch.common.xcontent.json.JsonXContent; import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.common.io.stream.NamedWriteableRegistry; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.gateway.remote.ClusterMetadataMarker.UploadedIndexMetadata; +import org.opensearch.test.EqualsHashCodeTestUtils; import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.test.VersionUtils; import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; +import java.util.List; public class ClusterMetadataMarkerTests extends OpenSearchTestCase { - public void testXContent() throws IOException { + public void testClusterMetadataMarkerXContent() throws IOException { UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "test-uuid", "/test/upload/path"); ClusterMetadataMarker originalMarker = new ClusterMetadataMarker( 1L, @@ -30,6 +35,7 @@ public void testXContent() throws IOException { "test-cluster-uuid", "test-state-uuid", Version.CURRENT, + "test-node-id", false, Collections.singletonList(uploadedIndexMetadata) ); @@ -43,4 +49,97 @@ public void testXContent() throws IOException { assertEquals(originalMarker, fromXContentMarker); } } + + public void testClusterMetadataMarkerSerializationEqualsHashCode() { + ClusterMetadataMarker initialMarker = new ClusterMetadataMarker( + randomNonNegativeLong(), + randomNonNegativeLong(), + randomAlphaOfLength(10), + randomAlphaOfLength(10), + VersionUtils.randomOpenSearchVersion(random()), + randomAlphaOfLength(10), + randomBoolean(), + randomUploadedIndexMetadataList() + ); + EqualsHashCodeTestUtils.checkEqualsAndHashCode( + initialMarker, + orig -> OpenSearchTestCase.copyWriteable(orig, new NamedWriteableRegistry(Collections.emptyList()), ClusterMetadataMarker::new), + marker -> { + ClusterMetadataMarker.Builder builder = ClusterMetadataMarker.builder(marker); + switch (randomInt(7)) { + case 0: + builder.clusterTerm(randomNonNegativeLong()); + break; + case 1: + builder.stateVersion(randomNonNegativeLong()); + break; + case 2: + builder.clusterUUID(randomAlphaOfLength(10)); + break; + case 3: + builder.stateUUID(randomAlphaOfLength(10)); + break; + case 4: + builder.opensearchVersion(VersionUtils.randomOpenSearchVersion(random())); + break; + case 5: + builder.nodeId(randomAlphaOfLength(10)); + break; + case 6: + builder.committed(randomBoolean()); + break; + case 7: + builder.indices(randomUploadedIndexMetadataList()); + break; + } + return builder.build(); + } + ); + } + + private List randomUploadedIndexMetadataList() { + final int size = randomIntBetween(1, 10); + final List uploadedIndexMetadataList = new ArrayList<>(size); + while (uploadedIndexMetadataList.size() < size) { + assertTrue(uploadedIndexMetadataList.add(randomUploadedIndexMetadata())); + } + return uploadedIndexMetadataList; + } + + private UploadedIndexMetadata randomUploadedIndexMetadata() { + return new UploadedIndexMetadata(randomAlphaOfLength(10), randomAlphaOfLength(10), randomAlphaOfLength(10)); + } + + public void testUploadedIndexMetadataSerializationEqualsHashCode() { + UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "test-uuid", "/test/upload/path"); + EqualsHashCodeTestUtils.checkEqualsAndHashCode( + uploadedIndexMetadata, + orig -> OpenSearchTestCase.copyWriteable(orig, new NamedWriteableRegistry(Collections.emptyList()), UploadedIndexMetadata::new), + metadata -> randomlyChangingUploadedIndexMetadata(uploadedIndexMetadata) + ); + } + + private UploadedIndexMetadata randomlyChangingUploadedIndexMetadata(UploadedIndexMetadata uploadedIndexMetadata) { + switch (randomInt(2)) { + case 0: + return new UploadedIndexMetadata( + randomAlphaOfLength(10), + uploadedIndexMetadata.getIndexUUID(), + uploadedIndexMetadata.getUploadedFilename() + ); + case 1: + return new UploadedIndexMetadata( + uploadedIndexMetadata.getIndexName(), + randomAlphaOfLength(10), + uploadedIndexMetadata.getUploadedFilename() + ); + case 2: + return new UploadedIndexMetadata( + uploadedIndexMetadata.getIndexName(), + uploadedIndexMetadata.getIndexUUID(), + randomAlphaOfLength(10) + ); + } + return uploadedIndexMetadata; + } } From 73914c2a303c78723d4ba3591b15f4335f3383c0 Mon Sep 17 00:00:00 2001 From: Sooraj Sinha Date: Fri, 25 Aug 2023 19:13:25 +0530 Subject: [PATCH 34/37] Integrate remote cluster state in publish/commit flow Signed-off-by: Sooraj Sinha --- .../coordination/CoordinationState.java | 32 ++++++++++++- .../cluster/coordination/Coordinator.java | 12 ++++- .../coordination/PersistentStateRegistry.java | 46 +++++++++++++++++++ .../opensearch/gateway/GatewayMetaState.java | 21 ++++++--- .../main/java/org/opensearch/node/Node.java | 17 ++++++- .../coordination/CoordinationStateTests.java | 27 ++++++++--- .../coordination/PreVoteCollectorTests.java | 4 +- .../coordination/PublicationTests.java | 4 +- .../CoordinationStateTestCluster.java | 18 +++++++- .../gateway/MockGatewayMetaState.java | 17 +++++-- 10 files changed, 171 insertions(+), 27 deletions(-) create mode 100644 server/src/main/java/org/opensearch/cluster/coordination/PersistentStateRegistry.java diff --git a/server/src/main/java/org/opensearch/cluster/coordination/CoordinationState.java b/server/src/main/java/org/opensearch/cluster/coordination/CoordinationState.java index 08cd7d0ab02db..312239ae3b5c1 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/CoordinationState.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/CoordinationState.java @@ -37,6 +37,7 @@ import org.opensearch.cluster.coordination.CoordinationMetadata.VotingConfiguration; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.common.settings.Settings; import java.io.Closeable; import java.io.IOException; @@ -49,6 +50,7 @@ import java.util.Set; import static org.opensearch.cluster.coordination.Coordinator.ZEN1_BWC_TERM; +import static org.opensearch.gateway.remote.RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING; /** * The core class of the cluster state coordination algorithm, directly implementing the @@ -66,6 +68,8 @@ public class CoordinationState { // persisted state private final PersistedState persistedState; + // remote persisted state + private final PersistedState remotePersistedState; // transient state private VoteCollection joinVotes; @@ -74,8 +78,15 @@ public class CoordinationState { private long lastPublishedVersion; private VotingConfiguration lastPublishedConfiguration; private VoteCollection publishVotes; - - public CoordinationState(DiscoveryNode localNode, PersistedState persistedState, ElectionStrategy electionStrategy) { + private boolean isRemoteStateEnabled; + + public CoordinationState( + DiscoveryNode localNode, + PersistedState persistedState, + ElectionStrategy electionStrategy, + PersistedState remotePersistedState, + Settings settings + ) { this.localNode = localNode; // persisted state @@ -89,6 +100,8 @@ public CoordinationState(DiscoveryNode localNode, PersistedState persistedState, this.lastPublishedVersion = 0L; this.lastPublishedConfiguration = persistedState.getLastAcceptedState().getLastAcceptedConfiguration(); this.publishVotes = new VoteCollection(); + this.remotePersistedState = remotePersistedState; + this.isRemoteStateEnabled = REMOTE_CLUSTER_STATE_ENABLED_SETTING.get(settings); } public long getCurrentTerm() { @@ -490,6 +503,7 @@ public Optional handlePublishResponse(DiscoveryNode sourceNo publishResponse.getVersion(), publishResponse.getTerm() ); + handleRemoteCommit(); return Optional.of(new ApplyCommitRequest(localNode, publishResponse.getTerm(), publishResponse.getVersion())); } @@ -551,6 +565,20 @@ public void handleCommit(ApplyCommitRequest applyCommit) { assert getLastCommittedConfiguration().equals(getLastAcceptedConfiguration()); } + public void handleRemotePublish(ClusterState clusterState) { + if (isRemoteStateEnabled == true) { + assert remotePersistedState != null : "Remote state has not been initialized"; + remotePersistedState.setLastAcceptedState(clusterState); + } + } + + public void handleRemoteCommit() { + if (isRemoteStateEnabled) { + assert remotePersistedState != null : "Remote state has not been initialized"; + remotePersistedState.markLastAcceptedStateAsCommitted(); + } + } + public void invariant() { assert getLastAcceptedTerm() <= getCurrentTerm(); assert electionWon() == isElectionQuorum(joinVotes); diff --git a/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java b/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java index 0274073ddfdc7..c1ef0b4dbf2c0 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java @@ -48,6 +48,7 @@ import org.opensearch.cluster.coordination.CoordinationState.VoteCollection; import org.opensearch.cluster.coordination.FollowersChecker.FollowerCheckRequest; import org.opensearch.cluster.coordination.JoinHelper.InitialJoinAccumulator; +import org.opensearch.cluster.coordination.PersistentStateRegistry.PersistedStateType; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; @@ -821,7 +822,15 @@ boolean publicationInProgress() { protected void doStart() { synchronized (mutex) { CoordinationState.PersistedState persistedState = persistedStateSupplier.get(); - coordinationState.set(new CoordinationState(getLocalNode(), persistedState, electionStrategy)); + coordinationState.set( + new CoordinationState( + getLocalNode(), + persistedState, + electionStrategy, + PersistentStateRegistry.getPersistedState(PersistedStateType.REMOTE), + settings + ) + ); peerFinder.setCurrentTerm(getCurrentTerm()); configuredHostsResolver.start(); final ClusterState lastAcceptedState = coordinationState.get().getLastAcceptedState(); @@ -1308,6 +1317,7 @@ assert getLocalNode().equals(clusterState.getNodes().get(getLocalNode().getId()) leaderChecker.setCurrentNodes(publishNodes); followersChecker.setCurrentNodes(publishNodes); lagDetector.setTrackedNodes(publishNodes); + coordinationState.get().handleRemotePublish(clusterState); publication.start(followersChecker.getFaultyNodes()); } } catch (Exception e) { diff --git a/server/src/main/java/org/opensearch/cluster/coordination/PersistentStateRegistry.java b/server/src/main/java/org/opensearch/cluster/coordination/PersistentStateRegistry.java new file mode 100644 index 0000000000000..0c16ea4723064 --- /dev/null +++ b/server/src/main/java/org/opensearch/cluster/coordination/PersistentStateRegistry.java @@ -0,0 +1,46 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.cluster.coordination; + +import org.opensearch.cluster.coordination.CoordinationState.PersistedState; + +import java.util.HashMap; +import java.util.Map; + +/** + * A class which encapsulates the PersistedStates + * + * @opensearch.internal + */ +public class PersistentStateRegistry { + + private static final PersistentStateRegistry INSTANCE = new PersistentStateRegistry(); + + private PersistentStateRegistry() {} + + /** + * Distinct Types PersistedState which can be present on a node + */ + public enum PersistedStateType { + LOCAL, + REMOTE + } + + private final Map persistentStates = new HashMap<>(); + + public static void addPersistedState(PersistedStateType persistedStateType, PersistedState persistedState) { + PersistedState existingState = INSTANCE.persistentStates.putIfAbsent(persistedStateType, persistedState); + assert existingState == null : "should only be set once, but already have " + existingState; + } + + public static PersistedState getPersistedState(PersistedStateType persistedStateType) { + return INSTANCE.persistentStates.get(persistedStateType); + } + +} diff --git a/server/src/main/java/org/opensearch/gateway/GatewayMetaState.java b/server/src/main/java/org/opensearch/gateway/GatewayMetaState.java index 02f1e5049b95c..630b34a774520 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayMetaState.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayMetaState.java @@ -45,6 +45,8 @@ import org.opensearch.cluster.coordination.CoordinationMetadata; import org.opensearch.cluster.coordination.CoordinationState.PersistedState; import org.opensearch.cluster.coordination.InMemoryPersistedState; +import org.opensearch.cluster.coordination.PersistentStateRegistry; +import org.opensearch.cluster.coordination.PersistentStateRegistry.PersistedStateType; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.IndexTemplateMetadata; import org.opensearch.cluster.metadata.Manifest; @@ -107,13 +109,13 @@ public class GatewayMetaState implements Closeable { private final SetOnce persistedState = new SetOnce<>(); public PersistedState getPersistedState() { - final PersistedState persistedState = this.persistedState.get(); + final PersistedState persistedState = PersistentStateRegistry.getPersistedState(PersistedStateType.LOCAL); assert persistedState != null : "not started"; return persistedState; } public Metadata getMetadata() { - return getPersistedState().getLastAcceptedState().metadata(); + return PersistentStateRegistry.getPersistedState(PersistedStateType.LOCAL).getLastAcceptedState().metadata(); } public void start( @@ -123,9 +125,11 @@ public void start( MetaStateService metaStateService, MetadataIndexUpgradeService metadataIndexUpgradeService, MetadataUpgrader metadataUpgrader, - PersistedClusterStateService persistedClusterStateService + PersistedClusterStateService persistedClusterStateService, + RemoteClusterStateService remoteClusterStateService ) { - assert persistedState.get() == null : "should only start once, but already have " + persistedState.get(); + assert PersistentStateRegistry.getPersistedState(PersistedStateType.LOCAL) == null : "should only start once, but already have " + + persistedState.get(); if (DiscoveryNode.isClusterManagerNode(settings) || DiscoveryNode.isDataNode(settings)) { try { @@ -147,6 +151,7 @@ public void start( } PersistedState persistedState = null; + PersistedState remotePersistedState = null; boolean success = false; try { final ClusterState clusterState = prepareInitialClusterState( @@ -160,6 +165,7 @@ public void start( if (DiscoveryNode.isClusterManagerNode(settings)) { persistedState = new LucenePersistedState(persistedClusterStateService, currentTerm, clusterState); + remotePersistedState = new RemotePersistedState(remoteClusterStateService); } else { persistedState = new AsyncLucenePersistedState( settings, @@ -184,7 +190,8 @@ public void start( } } - this.persistedState.set(persistedState); + PersistentStateRegistry.addPersistedState(PersistedStateType.LOCAL, persistedState); + PersistentStateRegistry.addPersistedState(PersistedStateType.REMOTE, remotePersistedState); } catch (IOException e) { throw new OpenSearchException("failed to load metadata", e); } @@ -330,12 +337,12 @@ public void applyClusterState(ClusterChangedEvent event) { @Override public void close() throws IOException { - IOUtils.close(persistedState.get()); + IOUtils.close(PersistentStateRegistry.getPersistedState(PersistedStateType.LOCAL)); } // visible for testing public boolean allPendingAsyncStatesWritten() { - final PersistedState ps = persistedState.get(); + final PersistedState ps = PersistentStateRegistry.getPersistedState(PersistedStateType.LOCAL); if (ps instanceof AsyncLucenePersistedState) { return ((AsyncLucenePersistedState) ps).allPendingAsyncStatesWritten(); } else { diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 51cc7c9007159..c54f27d66bc70 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -61,6 +61,8 @@ import org.opensearch.cluster.InternalClusterInfoService; import org.opensearch.cluster.NodeConnectionsService; import org.opensearch.cluster.action.index.MappingUpdatedAction; +import org.opensearch.cluster.coordination.PersistentStateRegistry; +import org.opensearch.cluster.coordination.PersistentStateRegistry.PersistedStateType; import org.opensearch.cluster.metadata.AliasValidator; import org.opensearch.cluster.metadata.IndexTemplateMetadata; import org.opensearch.cluster.metadata.Metadata; @@ -127,6 +129,7 @@ import org.opensearch.gateway.GatewayService; import org.opensearch.gateway.MetaStateService; import org.opensearch.gateway.PersistedClusterStateService; +import org.opensearch.gateway.remote.RemoteClusterStateService; import org.opensearch.http.HttpServerTransport; import org.opensearch.identity.IdentityService; import org.opensearch.index.IndexModule; @@ -669,6 +672,12 @@ protected Node( clusterService.getClusterSettings(), threadPool::relativeTimeInMillis ); + final RemoteClusterStateService remoteClusterStateService = new RemoteClusterStateService( + repositoriesServiceReference::get, + settings, + clusterService.getClusterSettings(), + threadPool::relativeTimeInMillis + ); // collect engine factory providers from plugins final Collection enginePlugins = pluginsService.filterPlugins(EnginePlugin.class); @@ -1155,6 +1164,7 @@ protected Node( b.bind(SystemIndices.class).toInstance(systemIndices); b.bind(IdentityService.class).toInstance(identityService); b.bind(Tracer.class).toInstance(tracer); + b.bind(RemoteClusterStateService.class).toInstance(remoteClusterStateService); }); injector = modules.createInjector(); @@ -1302,7 +1312,8 @@ public Node start() throws NodeValidationException { injector.getInstance(MetaStateService.class), injector.getInstance(MetadataIndexUpgradeService.class), injector.getInstance(MetadataUpgrader.class), - injector.getInstance(PersistedClusterStateService.class) + injector.getInstance(PersistedClusterStateService.class), + injector.getInstance(RemoteClusterStateService.class) ); if (Assertions.ENABLED) { try { @@ -1321,7 +1332,9 @@ public Node start() throws NodeValidationException { } // we load the global state here (the persistent part of the cluster state stored on disk) to // pass it to the bootstrap checks to allow plugins to enforce certain preconditions based on the recovered state. - final Metadata onDiskMetadata = gatewayMetaState.getPersistedState().getLastAcceptedState().metadata(); + final Metadata onDiskMetadata = PersistentStateRegistry.getPersistedState(PersistedStateType.LOCAL) + .getLastAcceptedState() + .metadata(); assert onDiskMetadata != null : "metadata is null but shouldn't"; // this is never null validateNodeBeforeAcceptingRequests( new BootstrapContext(environment, onDiskMetadata), diff --git a/server/src/test/java/org/opensearch/cluster/coordination/CoordinationStateTests.java b/server/src/test/java/org/opensearch/cluster/coordination/CoordinationStateTests.java index 45e71138abf99..3161cb843a8b9 100644 --- a/server/src/test/java/org/opensearch/cluster/coordination/CoordinationStateTests.java +++ b/server/src/test/java/org/opensearch/cluster/coordination/CoordinationStateTests.java @@ -98,9 +98,19 @@ public void setupNodes() { ps1 = new InMemoryPersistedState(0L, initialStateNode1); - cs1 = createCoordinationState(ps1, node1); - cs2 = createCoordinationState(new InMemoryPersistedState(0L, initialStateNode2), node2); - cs3 = createCoordinationState(new InMemoryPersistedState(0L, initialStateNode3), node3); + cs1 = createCoordinationState(ps1, node1, ps1, Settings.EMPTY); + cs2 = createCoordinationState( + new InMemoryPersistedState(0L, initialStateNode2), + node2, + new InMemoryPersistedState(0L, initialStateNode2), + Settings.EMPTY + ); + cs3 = createCoordinationState( + new InMemoryPersistedState(0L, initialStateNode3), + node3, + new InMemoryPersistedState(0L, initialStateNode3), + Settings.EMPTY + ); } public static DiscoveryNode createNode(String id) { @@ -200,7 +210,7 @@ public void testJoinBeforeBootstrap() { public void testJoinWithNoStartJoinAfterReboot() { StartJoinRequest startJoinRequest1 = new StartJoinRequest(node1, randomLongBetween(1, 5)); Join v1 = cs1.handleStartJoin(startJoinRequest1); - cs1 = createCoordinationState(ps1, node1); + cs1 = createCoordinationState(ps1, node1, ps1, Settings.EMPTY); assertThat( expectThrows(CoordinationStateRejectedException.class, () -> cs1.handleJoin(v1)).getMessage(), containsString("ignored join as term has not been incremented yet after reboot") @@ -886,8 +896,13 @@ public void testSafety() { ).runRandomly(); } - public static CoordinationState createCoordinationState(PersistedState storage, DiscoveryNode localNode) { - return new CoordinationState(localNode, storage, ElectionStrategy.DEFAULT_INSTANCE); + public static CoordinationState createCoordinationState( + PersistedState storage, + DiscoveryNode localNode, + PersistedState remoteState, + Settings settings + ) { + return new CoordinationState(localNode, storage, ElectionStrategy.DEFAULT_INSTANCE, remoteState, settings); } public static ClusterState clusterState( diff --git a/server/src/test/java/org/opensearch/cluster/coordination/PreVoteCollectorTests.java b/server/src/test/java/org/opensearch/cluster/coordination/PreVoteCollectorTests.java index 8b35856f0fb4c..dc717a9c0b251 100644 --- a/server/src/test/java/org/opensearch/cluster/coordination/PreVoteCollectorTests.java +++ b/server/src/test/java/org/opensearch/cluster/coordination/PreVoteCollectorTests.java @@ -293,7 +293,9 @@ public void testPrevotingIndicatesElectionSuccess() { final CoordinationState coordinationState = new CoordinationState( localNode, new InMemoryPersistedState(currentTerm, makeClusterState(votingNodes)), - ElectionStrategy.DEFAULT_INSTANCE + ElectionStrategy.DEFAULT_INSTANCE, + new InMemoryPersistedState(currentTerm, makeClusterState(votingNodes)), + Settings.EMPTY ); final long newTerm = randomLongBetween(currentTerm + 1, Long.MAX_VALUE); diff --git a/server/src/test/java/org/opensearch/cluster/coordination/PublicationTests.java b/server/src/test/java/org/opensearch/cluster/coordination/PublicationTests.java index 79c141aa69b9f..52b99140871c2 100644 --- a/server/src/test/java/org/opensearch/cluster/coordination/PublicationTests.java +++ b/server/src/test/java/org/opensearch/cluster/coordination/PublicationTests.java @@ -94,7 +94,9 @@ class MockNode { coordinationState = new CoordinationState( localNode, new InMemoryPersistedState(0L, initialState), - ElectionStrategy.DEFAULT_INSTANCE + ElectionStrategy.DEFAULT_INSTANCE, + new InMemoryPersistedState(0L, initialState), + Settings.EMPTY ); } diff --git a/test/framework/src/main/java/org/opensearch/cluster/coordination/CoordinationStateTestCluster.java b/test/framework/src/main/java/org/opensearch/cluster/coordination/CoordinationStateTestCluster.java index 9f7802a401391..f5b3cc9d303ea 100644 --- a/test/framework/src/main/java/org/opensearch/cluster/coordination/CoordinationStateTestCluster.java +++ b/test/framework/src/main/java/org/opensearch/cluster/coordination/CoordinationStateTestCluster.java @@ -128,6 +128,8 @@ static class ClusterNode { DiscoveryNode localNode; CoordinationState.PersistedState persistedState; + CoordinationState.PersistedState remotePersistedState; + CoordinationState state; ClusterNode(DiscoveryNode localNode, ElectionStrategy electionStrategy) { @@ -143,8 +145,20 @@ static class ClusterNode { 0L ) ); + remotePersistedState = new InMemoryPersistedState( + 0L, + clusterState( + 0L, + 0L, + localNode, + CoordinationMetadata.VotingConfiguration.EMPTY_CONFIG, + CoordinationMetadata.VotingConfiguration.EMPTY_CONFIG, + 0L + ) + ); + this.electionStrategy = electionStrategy; - state = new CoordinationState(localNode, persistedState, electionStrategy); + state = new CoordinationState(localNode, persistedState, electionStrategy, remotePersistedState, Settings.EMPTY); } void reboot() { @@ -183,7 +197,7 @@ void reboot() { localNode.getVersion() ); - state = new CoordinationState(localNode, persistedState, electionStrategy); + state = new CoordinationState(localNode, persistedState, electionStrategy, remotePersistedState, Settings.EMPTY); } void setInitialState(CoordinationMetadata.VotingConfiguration initialConfig, long initialValue) { diff --git a/test/framework/src/main/java/org/opensearch/gateway/MockGatewayMetaState.java b/test/framework/src/main/java/org/opensearch/gateway/MockGatewayMetaState.java index 6a3748e55394e..3396193e69b55 100644 --- a/test/framework/src/main/java/org/opensearch/gateway/MockGatewayMetaState.java +++ b/test/framework/src/main/java/org/opensearch/gateway/MockGatewayMetaState.java @@ -32,6 +32,7 @@ package org.opensearch.gateway; +import java.util.Collections; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.Manifest; import org.opensearch.cluster.metadata.Metadata; @@ -44,7 +45,9 @@ import org.opensearch.common.util.BigArrays; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.NodeEnvironment; +import org.opensearch.gateway.remote.RemoteClusterStateService; import org.opensearch.plugins.MetadataUpgrader; +import org.opensearch.repositories.RepositoriesService; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; @@ -54,12 +57,12 @@ import static org.mockito.Mockito.when; /** - * {@link GatewayMetaState} constructor accepts a lot of arguments. - * It's not always easy / convenient to construct these dependencies. - * This class constructor takes far fewer dependencies and constructs usable {@link GatewayMetaState} with 2 restrictions: - * no metadata upgrade will be performed and no cluster state updaters will be run. This is sufficient for most of the tests. + * {@link GatewayMetaState} constructor accepts a lot of arguments. It's not always easy / convenient to construct these dependencies. This class constructor + * takes far fewer dependencies and constructs usable {@link GatewayMetaState} with 2 restrictions: no metadata upgrade will be performed and no cluster state + * updaters will be run. This is sufficient for most of the tests. */ public class MockGatewayMetaState extends GatewayMetaState { + private final DiscoveryNode localNode; private final BigArrays bigArrays; @@ -110,7 +113,11 @@ public void start(Settings settings, NodeEnvironment nodeEnvironment, NamedXCont bigArrays, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), () -> 0L - ) + ), + new RemoteClusterStateService( + () -> new RepositoriesService(settings, clusterService, transportService, Collections.emptyMap(), Collections.emptyMap(), + transportService.getThreadPool()), settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), + () -> 0L) ); } } From d36b1ba55dbbc4da507aeabb0582a15e6993461d Mon Sep 17 00:00:00 2001 From: Sooraj Sinha Date: Mon, 28 Aug 2023 09:49:01 +0530 Subject: [PATCH 35/37] Remove persistedState from CoordinationState Signed-off-by: Sooraj Sinha --- .../coordination/CoordinationState.java | 8 +++-- .../cluster/coordination/Coordinator.java | 10 ++++-- ...istry.java => PersistedStateRegistry.java} | 14 ++++---- .../opensearch/gateway/GatewayMetaState.java | 34 ++++++++++--------- .../main/java/org/opensearch/node/Node.java | 7 ++-- .../gateway/MockGatewayMetaState.java | 9 ++--- 6 files changed, 46 insertions(+), 36 deletions(-) rename server/src/main/java/org/opensearch/cluster/coordination/{PersistentStateRegistry.java => PersistedStateRegistry.java} (69%) diff --git a/server/src/main/java/org/opensearch/cluster/coordination/CoordinationState.java b/server/src/main/java/org/opensearch/cluster/coordination/CoordinationState.java index 312239ae3b5c1..7ee6ce7a27f7b 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/CoordinationState.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/CoordinationState.java @@ -503,7 +503,7 @@ public Optional handlePublishResponse(DiscoveryNode sourceNo publishResponse.getVersion(), publishResponse.getTerm() ); - handleRemoteCommit(); + handlePreCommit(); return Optional.of(new ApplyCommitRequest(localNode, publishResponse.getTerm(), publishResponse.getVersion())); } @@ -565,14 +565,16 @@ public void handleCommit(ApplyCommitRequest applyCommit) { assert getLastCommittedConfiguration().equals(getLastAcceptedConfiguration()); } - public void handleRemotePublish(ClusterState clusterState) { + public void handlePrePublish(ClusterState clusterState) { + // Publishing the current state to remote store if (isRemoteStateEnabled == true) { assert remotePersistedState != null : "Remote state has not been initialized"; remotePersistedState.setLastAcceptedState(clusterState); } } - public void handleRemoteCommit() { + public void handlePreCommit() { + // Publishing the committed state to remote store before sending apply commit to other nodes. if (isRemoteStateEnabled) { assert remotePersistedState != null : "Remote state has not been initialized"; remotePersistedState.markLastAcceptedStateAsCommitted(); diff --git a/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java b/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java index c1ef0b4dbf2c0..abb55cbf9e936 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java @@ -48,7 +48,7 @@ import org.opensearch.cluster.coordination.CoordinationState.VoteCollection; import org.opensearch.cluster.coordination.FollowersChecker.FollowerCheckRequest; import org.opensearch.cluster.coordination.JoinHelper.InitialJoinAccumulator; -import org.opensearch.cluster.coordination.PersistentStateRegistry.PersistedStateType; +import org.opensearch.cluster.coordination.PersistedStateRegistry.PersistedStateType; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; @@ -827,7 +827,7 @@ protected void doStart() { getLocalNode(), persistedState, electionStrategy, - PersistentStateRegistry.getPersistedState(PersistedStateType.REMOTE), + PersistedStateRegistry.getPersistedState(PersistedStateType.REMOTE), settings ) ); @@ -1317,7 +1317,11 @@ assert getLocalNode().equals(clusterState.getNodes().get(getLocalNode().getId()) leaderChecker.setCurrentNodes(publishNodes); followersChecker.setCurrentNodes(publishNodes); lagDetector.setTrackedNodes(publishNodes); - coordinationState.get().handleRemotePublish(clusterState); + // Publishing the current state to remote store before sending the cluster state to other nodes. + // This is to ensure the remote store is the single source of truth for current state. Even if the current node + // goes down after sending the cluster state to other nodes, we should be able to read the remote state and + // recover the cluster. + coordinationState.get().handlePrePublish(clusterState); publication.start(followersChecker.getFaultyNodes()); } } catch (Exception e) { diff --git a/server/src/main/java/org/opensearch/cluster/coordination/PersistentStateRegistry.java b/server/src/main/java/org/opensearch/cluster/coordination/PersistedStateRegistry.java similarity index 69% rename from server/src/main/java/org/opensearch/cluster/coordination/PersistentStateRegistry.java rename to server/src/main/java/org/opensearch/cluster/coordination/PersistedStateRegistry.java index 0c16ea4723064..5b1a23ac7b56b 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/PersistentStateRegistry.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/PersistedStateRegistry.java @@ -10,19 +10,19 @@ import org.opensearch.cluster.coordination.CoordinationState.PersistedState; -import java.util.HashMap; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; /** * A class which encapsulates the PersistedStates * * @opensearch.internal */ -public class PersistentStateRegistry { +public class PersistedStateRegistry { - private static final PersistentStateRegistry INSTANCE = new PersistentStateRegistry(); + private static final PersistedStateRegistry INSTANCE = new PersistedStateRegistry(); - private PersistentStateRegistry() {} + private PersistedStateRegistry() {} /** * Distinct Types PersistedState which can be present on a node @@ -32,15 +32,15 @@ public enum PersistedStateType { REMOTE } - private final Map persistentStates = new HashMap<>(); + private final Map persistedStates = new ConcurrentHashMap<>(); public static void addPersistedState(PersistedStateType persistedStateType, PersistedState persistedState) { - PersistedState existingState = INSTANCE.persistentStates.putIfAbsent(persistedStateType, persistedState); + PersistedState existingState = INSTANCE.persistedStates.putIfAbsent(persistedStateType, persistedState); assert existingState == null : "should only be set once, but already have " + existingState; } public static PersistedState getPersistedState(PersistedStateType persistedStateType) { - return INSTANCE.persistentStates.get(persistedStateType); + return INSTANCE.persistedStates.get(persistedStateType); } } diff --git a/server/src/main/java/org/opensearch/gateway/GatewayMetaState.java b/server/src/main/java/org/opensearch/gateway/GatewayMetaState.java index 630b34a774520..473078c6e8e85 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayMetaState.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayMetaState.java @@ -45,8 +45,8 @@ import org.opensearch.cluster.coordination.CoordinationMetadata; import org.opensearch.cluster.coordination.CoordinationState.PersistedState; import org.opensearch.cluster.coordination.InMemoryPersistedState; -import org.opensearch.cluster.coordination.PersistentStateRegistry; -import org.opensearch.cluster.coordination.PersistentStateRegistry.PersistedStateType; +import org.opensearch.cluster.coordination.PersistedStateRegistry; +import org.opensearch.cluster.coordination.PersistedStateRegistry.PersistedStateType; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.IndexTemplateMetadata; import org.opensearch.cluster.metadata.Manifest; @@ -54,7 +54,6 @@ import org.opensearch.cluster.metadata.MetadataIndexUpgradeService; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.SetOnce; import org.opensearch.common.collect.Tuple; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.concurrent.AbstractRunnable; @@ -85,6 +84,7 @@ import java.util.function.UnaryOperator; import static org.opensearch.common.util.concurrent.OpenSearchExecutors.daemonThreadFactory; +import static org.opensearch.gateway.remote.RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING; /** * Loads (and maybe upgrades) cluster metadata at startup, and persistently stores cluster metadata for future restarts. @@ -105,17 +105,14 @@ public class GatewayMetaState implements Closeable { */ public static final String STALE_STATE_CONFIG_NODE_ID = "STALE_STATE_CONFIG"; - // Set by calling start() - private final SetOnce persistedState = new SetOnce<>(); - public PersistedState getPersistedState() { - final PersistedState persistedState = PersistentStateRegistry.getPersistedState(PersistedStateType.LOCAL); + final PersistedState persistedState = PersistedStateRegistry.getPersistedState(PersistedStateType.LOCAL); assert persistedState != null : "not started"; return persistedState; } public Metadata getMetadata() { - return PersistentStateRegistry.getPersistedState(PersistedStateType.LOCAL).getLastAcceptedState().metadata(); + return PersistedStateRegistry.getPersistedState(PersistedStateType.LOCAL).getLastAcceptedState().metadata(); } public void start( @@ -128,8 +125,8 @@ public void start( PersistedClusterStateService persistedClusterStateService, RemoteClusterStateService remoteClusterStateService ) { - assert PersistentStateRegistry.getPersistedState(PersistedStateType.LOCAL) == null : "should only start once, but already have " - + persistedState.get(); + assert PersistedStateRegistry.getPersistedState(PersistedStateType.LOCAL) == null : "should only start once, but already have " + + PersistedStateRegistry.getPersistedState(PersistedStateType.LOCAL); if (DiscoveryNode.isClusterManagerNode(settings) || DiscoveryNode.isDataNode(settings)) { try { @@ -165,7 +162,9 @@ public void start( if (DiscoveryNode.isClusterManagerNode(settings)) { persistedState = new LucenePersistedState(persistedClusterStateService, currentTerm, clusterState); - remotePersistedState = new RemotePersistedState(remoteClusterStateService); + if (REMOTE_CLUSTER_STATE_ENABLED_SETTING.get(settings) == true) { + remotePersistedState = new RemotePersistedState(remoteClusterStateService); + } } else { persistedState = new AsyncLucenePersistedState( settings, @@ -187,11 +186,14 @@ public void start( } finally { if (success == false) { IOUtils.closeWhileHandlingException(persistedState); + IOUtils.closeWhileHandlingException(remotePersistedState); } } - PersistentStateRegistry.addPersistedState(PersistedStateType.LOCAL, persistedState); - PersistentStateRegistry.addPersistedState(PersistedStateType.REMOTE, remotePersistedState); + PersistedStateRegistry.addPersistedState(PersistedStateType.LOCAL, persistedState); + if (remotePersistedState != null) { + PersistedStateRegistry.addPersistedState(PersistedStateType.REMOTE, remotePersistedState); + } } catch (IOException e) { throw new OpenSearchException("failed to load metadata", e); } @@ -218,7 +220,7 @@ public void start( throw new UncheckedIOException(e); } } - persistedState.set(new InMemoryPersistedState(currentTerm, clusterState)); + PersistedStateRegistry.addPersistedState(PersistedStateType.LOCAL, new InMemoryPersistedState(currentTerm, clusterState)); } } @@ -337,12 +339,12 @@ public void applyClusterState(ClusterChangedEvent event) { @Override public void close() throws IOException { - IOUtils.close(PersistentStateRegistry.getPersistedState(PersistedStateType.LOCAL)); + IOUtils.close(PersistedStateRegistry.getPersistedState(PersistedStateType.LOCAL)); } // visible for testing public boolean allPendingAsyncStatesWritten() { - final PersistedState ps = PersistentStateRegistry.getPersistedState(PersistedStateType.LOCAL); + final PersistedState ps = PersistedStateRegistry.getPersistedState(PersistedStateType.LOCAL); if (ps instanceof AsyncLucenePersistedState) { return ((AsyncLucenePersistedState) ps).allPendingAsyncStatesWritten(); } else { diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index c54f27d66bc70..8df36ad2b5a2e 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -61,8 +61,8 @@ import org.opensearch.cluster.InternalClusterInfoService; import org.opensearch.cluster.NodeConnectionsService; import org.opensearch.cluster.action.index.MappingUpdatedAction; -import org.opensearch.cluster.coordination.PersistentStateRegistry; -import org.opensearch.cluster.coordination.PersistentStateRegistry.PersistedStateType; +import org.opensearch.cluster.coordination.PersistedStateRegistry; +import org.opensearch.cluster.coordination.PersistedStateRegistry.PersistedStateType; import org.opensearch.cluster.metadata.AliasValidator; import org.opensearch.cluster.metadata.IndexTemplateMetadata; import org.opensearch.cluster.metadata.Metadata; @@ -673,6 +673,7 @@ protected Node( threadPool::relativeTimeInMillis ); final RemoteClusterStateService remoteClusterStateService = new RemoteClusterStateService( + nodeEnvironment.nodeId(), repositoriesServiceReference::get, settings, clusterService.getClusterSettings(), @@ -1332,7 +1333,7 @@ public Node start() throws NodeValidationException { } // we load the global state here (the persistent part of the cluster state stored on disk) to // pass it to the bootstrap checks to allow plugins to enforce certain preconditions based on the recovered state. - final Metadata onDiskMetadata = PersistentStateRegistry.getPersistedState(PersistedStateType.LOCAL) + final Metadata onDiskMetadata = PersistedStateRegistry.getPersistedState(PersistedStateType.LOCAL) .getLastAcceptedState() .metadata(); assert onDiskMetadata != null : "metadata is null but shouldn't"; // this is never null diff --git a/test/framework/src/main/java/org/opensearch/gateway/MockGatewayMetaState.java b/test/framework/src/main/java/org/opensearch/gateway/MockGatewayMetaState.java index 3396193e69b55..f7ca353e5b5df 100644 --- a/test/framework/src/main/java/org/opensearch/gateway/MockGatewayMetaState.java +++ b/test/framework/src/main/java/org/opensearch/gateway/MockGatewayMetaState.java @@ -57,12 +57,12 @@ import static org.mockito.Mockito.when; /** - * {@link GatewayMetaState} constructor accepts a lot of arguments. It's not always easy / convenient to construct these dependencies. This class constructor - * takes far fewer dependencies and constructs usable {@link GatewayMetaState} with 2 restrictions: no metadata upgrade will be performed and no cluster state - * updaters will be run. This is sufficient for most of the tests. + * {@link GatewayMetaState} constructor accepts a lot of arguments. + * It's not always easy / convenient to construct these dependencies. + * This class constructor takes far fewer dependencies and constructs usable {@link GatewayMetaState} with 2 restrictions: + * no metadata upgrade will be performed and no cluster state updaters will be run. This is sufficient for most of the tests. */ public class MockGatewayMetaState extends GatewayMetaState { - private final DiscoveryNode localNode; private final BigArrays bigArrays; @@ -115,6 +115,7 @@ public void start(Settings settings, NodeEnvironment nodeEnvironment, NamedXCont () -> 0L ), new RemoteClusterStateService( + nodeEnvironment.nodeId(), () -> new RepositoriesService(settings, clusterService, transportService, Collections.emptyMap(), Collections.emptyMap(), transportService.getThreadPool()), settings, new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), () -> 0L) From 823fab4b64050e2f58f4ac81d9f5117939d856d2 Mon Sep 17 00:00:00 2001 From: bansvaru Date: Tue, 29 Aug 2023 01:12:20 +0530 Subject: [PATCH 36/37] integrate index metadata upload with multipart upload Signed-off-by: bansvaru --- .../remote/RemoteClusterStateService.java | 139 ++++++++++++++---- .../blobstore/ChecksumBlobStoreFormat.java | 45 ++++++ 2 files changed, 154 insertions(+), 30 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java index 491c04bab3adb..b0354af2e609c 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java @@ -10,7 +10,9 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.Version; +import org.opensearch.action.LatchedActionListener; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.Nullable; @@ -21,10 +23,12 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.action.ActionListener; import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedIndexMetadata; import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.Repository; +import org.opensearch.repositories.RepositoryMissingException; import org.opensearch.repositories.blobstore.BlobStoreRepository; import org.opensearch.repositories.blobstore.ChecksumBlobStoreFormat; @@ -35,7 +39,11 @@ import java.util.Base64; import java.util.HashMap; import java.util.List; +import java.util.Locale; import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.function.Function; import java.util.function.LongSupplier; import java.util.function.Supplier; @@ -127,23 +135,12 @@ public ClusterMetadataManifest writeFullMetadata(ClusterState clusterState) thro } ensureRepositorySet(); - final List allUploadedIndexMetadata = new ArrayList<>(); - // todo parallel upload // any validations before/after upload ? - for (IndexMetadata indexMetadata : clusterState.metadata().indices().values()) { - // 123456789012_test-cluster/cluster-state/dsgYj10Nkso7/index/ftqsCnn9TgOX/metadata_4_1690947200 - final String indexMetadataKey = writeIndexMetadata( - clusterState.getClusterName().value(), - clusterState.getMetadata().clusterUUID(), - indexMetadata, - indexMetadataFileName(indexMetadata) - ); - final UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata( - indexMetadata.getIndex().getName(), - indexMetadata.getIndexUUID(), - indexMetadataKey - ); - allUploadedIndexMetadata.add(uploadedIndexMetadata); + final List allUploadedIndexMetadata; + try { + allUploadedIndexMetadata = writeIndexMetadata(clusterState, new ArrayList<>(clusterState.metadata().indices().values())); + } catch (InterruptedException | TimeoutException e) { + throw new RuntimeException(e); } final ClusterMetadataManifest manifest = uploadManifest(clusterState, allUploadedIndexMetadata, false); final long durationMillis = relativeTimeMillisSupplier.getAsLong() - startTimeMillis; @@ -194,6 +191,9 @@ public ClusterMetadataManifest writeIncrementalMetadata( final Map allUploadedIndexMetadata = previousManifest.getIndices() .stream() .collect(Collectors.toMap(UploadedIndexMetadata::getIndexName, Function.identity())); + + List toUpload = new ArrayList<>(); + for (final IndexMetadata indexMetadata : clusterState.metadata().indices().values()) { final Long previousVersion = previousStateIndexMetadataVersionByName.get(indexMetadata.getIndex().getName()); if (previousVersion == null || indexMetadata.getVersion() != previousVersion) { @@ -204,30 +204,25 @@ public ClusterMetadataManifest writeIncrementalMetadata( indexMetadata.getVersion() ); numIndicesUpdated++; - final String indexMetadataKey = writeIndexMetadata( - clusterState.getClusterName().value(), - clusterState.getMetadata().clusterUUID(), - indexMetadata, - indexMetadataFileName(indexMetadata) - ); - final UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata( - indexMetadata.getIndex().getName(), - indexMetadata.getIndexUUID(), - indexMetadataKey - ); - allUploadedIndexMetadata.put(indexMetadata.getIndex().getName(), uploadedIndexMetadata); + toUpload.add(indexMetadata); } else { numIndicesUnchanged++; } previousStateIndexMetadataVersionByName.remove(indexMetadata.getIndex().getName()); } + try { + writeIndexMetadata(clusterState, toUpload); + } catch (InterruptedException | TimeoutException e) { + throw new RuntimeException(e); + } + for (String removedIndexName : previousStateIndexMetadataVersionByName.keySet()) { allUploadedIndexMetadata.remove(removedIndexName); } final ClusterMetadataManifest manifest = uploadManifest( clusterState, - allUploadedIndexMetadata.values().stream().collect(Collectors.toList()), + new ArrayList<>(allUploadedIndexMetadata.values()), false ); final long durationMillis = relativeTimeMillisSupplier.getAsLong() - startTimeMillis; @@ -251,6 +246,72 @@ public ClusterMetadataManifest writeIncrementalMetadata( return manifest; } + private List writeIndexMetadata(ClusterState clusterState, List toUpload) throws IOException, + InterruptedException, TimeoutException { + List exceptionList = new ArrayList<>(toUpload.size()); + final CountDownLatch latch = new CountDownLatch(toUpload.size()); + List result = new ArrayList<>(toUpload.size()); + + LatchedActionListener latchedActionListener = new LatchedActionListener<>(ActionListener.wrap((IndexMetadata t) -> { + logger.trace(String.format(Locale.ROOT, "IndexMetadata uploaded successfully for %s", t.getIndex().toString())); + }, ex -> { + assert ex instanceof IndexMetadataTransferException; + logger.error( + () -> new ParameterizedMessage( + "Exception during transfer of IndexMetadata to Remote {}", + ((IndexMetadataTransferException) ex).getIndexMetadata().getIndex().toString() + ), + ex + ); + exceptionList.add(ex); + }), latch); + + for (IndexMetadata indexMetadata : toUpload) { + final UploadedIndexMetadata uploadedIndexMetadata = writeIndexMetadata(clusterState, latchedActionListener, indexMetadata); + result.add(uploadedIndexMetadata); + } + + try { + if (latch.await(20000, TimeUnit.MILLISECONDS) == false) { + TimeoutException ex = new TimeoutException("Timed out waiting for transfer of index metadata to complete"); + exceptionList.forEach(ex::addSuppressed); + throw ex; + } + } catch (InterruptedException ex) { + exceptionList.forEach(ex::addSuppressed); + Thread.currentThread().interrupt(); + throw ex; + } + + return result; + } + + private UploadedIndexMetadata writeIndexMetadata( + ClusterState clusterState, + LatchedActionListener latchedActionListener, + IndexMetadata indexMetadata + ) throws IOException { + final BlobContainer indexMetadataContainer = indexMetadataContainer( + clusterState.getClusterName().value(), + clusterState.metadata().clusterUUID(), + indexMetadata.getIndexUUID() + ); + + ActionListener completionListener = ActionListener.wrap( + resp -> latchedActionListener.onResponse(indexMetadata), + ex -> latchedActionListener.onFailure(new IndexMetadataTransferException(indexMetadata, ex)) + ); + INDEX_METADATA_FORMAT.writeAsync( + indexMetadata, + indexMetadataContainer, + indexMetadataFileName(indexMetadata), + blobStoreRepository.getCompressor(), + completionListener + ); + final String indexMetadataKey = indexMetadataContainer.path().buildAsString() + indexMetadataFileName(indexMetadata); + return new UploadedIndexMetadata(indexMetadata.getIndex().getName(), indexMetadata.getIndexUUID(), indexMetadataKey); + } + @Nullable public ClusterMetadataManifest markLastStateAsCommitted(ClusterState clusterState, ClusterMetadataManifest previousManifest) throws IOException { @@ -282,7 +343,12 @@ void ensureRepositorySet() { } final String remoteStoreRepo = REMOTE_CLUSTER_STATE_REPOSITORY_SETTING.get(settings); assert remoteStoreRepo != null : "Remote Cluster State repository is not configured"; - final Repository repository = repositoriesService.get().repository(remoteStoreRepo); + final Repository repository; + try { + repository = repositoriesService.get().repository(remoteStoreRepo); + } catch (RepositoryMissingException e) { + return; + } assert repository instanceof BlobStoreRepository : "Repository should be instance of BlobStoreRepository"; blobStoreRepository = (BlobStoreRepository) repository; } @@ -367,4 +433,17 @@ private static String indexMetadataFileName(IndexMetadata indexMetadata) { return String.join(DELIMITER, "metadata", String.valueOf(indexMetadata.getVersion()), String.valueOf(System.currentTimeMillis())); } + class IndexMetadataTransferException extends RuntimeException { + + private final IndexMetadata indexMetadata; + + public IndexMetadataTransferException(IndexMetadata indexMetadata, Throwable cause) { + super(cause); + this.indexMetadata = indexMetadata; + } + + public IndexMetadata getIndexMetadata() { + return indexMetadata; + } + } } diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumBlobStoreFormat.java b/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumBlobStoreFormat.java index 9048757405108..28d01256ec3e3 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumBlobStoreFormat.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumBlobStoreFormat.java @@ -43,6 +43,10 @@ import org.opensearch.cluster.metadata.Metadata; import org.opensearch.common.CheckedFunction; import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.VerifyingMultiStreamBlobContainer; +import org.opensearch.common.blobstore.stream.write.WritePriority; +import org.opensearch.common.blobstore.transfer.RemoteTransferContainer; +import org.opensearch.common.blobstore.transfer.stream.OffsetRangeIndexInputStream; import org.opensearch.common.io.Streams; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.lucene.store.ByteArrayIndexInput; @@ -50,6 +54,7 @@ import org.opensearch.common.xcontent.LoggingDeprecationHandler; import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.compress.Compressor; import org.opensearch.core.xcontent.MediaTypeRegistry; @@ -167,6 +172,46 @@ public void write(final T obj, final BlobContainer blobContainer, final String n blobContainer.writeBlob(blobName, bytes.streamInput(), bytes.length(), false); } + /** + * Writes blob with resolving the blob name using {@link #blobName} method. + * Leverages the multipart upload if supported by the blobContainer. + * + * @param obj object to be serialized + * @param blobContainer blob container + * @param name blob name + * @param compressor whether to use compression + * @param listener listener to listen to write result + */ + public void writeAsync( + final T obj, + final BlobContainer blobContainer, + final String name, + final Compressor compressor, + ActionListener listener + ) throws IOException { + if (blobContainer instanceof VerifyingMultiStreamBlobContainer == false) { + write(obj, blobContainer, name, compressor); + return; + } + final String blobName = blobName(name); + final BytesReference bytes = serialize(obj, blobName, compressor); + + IndexInput input = new ByteArrayIndexInput("", BytesReference.toBytes(bytes)); + + RemoteTransferContainer remoteTransferContainer = new RemoteTransferContainer( + blobName, + blobName, + bytes.length(), + true, + WritePriority.HIGH, + (size, position) -> new OffsetRangeIndexInputStream(input, size, position), + CodecUtil.checksumEntireFile(input), + true + ); + + ((VerifyingMultiStreamBlobContainer) blobContainer).asyncBlobUpload(remoteTransferContainer.createWriteContext(), listener); + } + public BytesReference serialize(final T obj, final String blobName, final Compressor compressor) throws IOException { try (BytesStreamOutput outputStream = new BytesStreamOutput()) { try ( From 2080dc1fd41e14f3c1a4dd49f6efd0b592a3a366 Mon Sep 17 00:00:00 2001 From: bansvaru Date: Sat, 2 Sep 2023 15:13:44 +0530 Subject: [PATCH 37/37] fix checksum calculation for s3 upload Signed-off-by: bansvaru --- .../transfer/RemoteTransferContainer.java | 18 ++++++++++++++++++ .../remote/RemoteClusterStateService.java | 6 +----- .../index/store/RemoteDirectory.java | 10 ++-------- .../blobstore/ChecksumBlobStoreFormat.java | 19 +++++++++++++++++-- 4 files changed, 38 insertions(+), 15 deletions(-) diff --git a/server/src/main/java/org/opensearch/common/blobstore/transfer/RemoteTransferContainer.java b/server/src/main/java/org/opensearch/common/blobstore/transfer/RemoteTransferContainer.java index c7cfef5c5ce3d..736feb7c732d9 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/transfer/RemoteTransferContainer.java +++ b/server/src/main/java/org/opensearch/common/blobstore/transfer/RemoteTransferContainer.java @@ -10,7 +10,9 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.store.IndexInput; import org.opensearch.common.CheckedTriFunction; import org.opensearch.common.SetOnce; import org.opensearch.common.StreamContext; @@ -19,11 +21,13 @@ import org.opensearch.common.blobstore.transfer.stream.OffsetRangeInputStream; import org.opensearch.common.blobstore.transfer.stream.ResettableCheckedInputStream; import org.opensearch.common.io.InputStreamContainer; +import org.opensearch.common.util.ByteUtils; import java.io.Closeable; import java.io.IOException; import java.io.InputStream; import java.util.Objects; +import java.util.zip.CRC32; import com.jcraft.jzlib.JZlib; @@ -244,4 +248,18 @@ public void close() throws IOException { throw new IOException("Closure of some of the multi-part streams failed."); } } + + /** + * Compute final checksum for IndexInput + * @param indexInput IndexInput with checksum in footer + * @param checksumBytesLength length of checksum bytes + * @return final computed checksum of entire indexInput + * @throws IOException + */ + public static long checksumOfChecksum(IndexInput indexInput, int checksumBytesLength) throws IOException { + long storedChecksum = CodecUtil.retrieveChecksum(indexInput); + CRC32 checksumOfChecksum = new CRC32(); + checksumOfChecksum.update(ByteUtils.toByteArrayBE(storedChecksum)); + return JZlib.crc32_combine(storedChecksum, checksumOfChecksum.getValue(), checksumBytesLength); + } } diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java index b0354af2e609c..af5838098caf8 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java @@ -220,11 +220,7 @@ public ClusterMetadataManifest writeIncrementalMetadata( for (String removedIndexName : previousStateIndexMetadataVersionByName.keySet()) { allUploadedIndexMetadata.remove(removedIndexName); } - final ClusterMetadataManifest manifest = uploadManifest( - clusterState, - new ArrayList<>(allUploadedIndexMetadata.values()), - false - ); + final ClusterMetadataManifest manifest = uploadManifest(clusterState, new ArrayList<>(allUploadedIndexMetadata.values()), false); final long durationMillis = relativeTimeMillisSupplier.getAsLong() - startTimeMillis; if (durationMillis >= slowWriteLoggingThreshold.getMillis()) { logger.warn( diff --git a/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java b/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java index 04b5d7eb7c6bd..67b8ce55be353 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java @@ -11,7 +11,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; @@ -29,7 +28,6 @@ import org.opensearch.common.blobstore.transfer.RemoteTransferContainer; import org.opensearch.common.blobstore.transfer.stream.OffsetRangeIndexInputStream; import org.opensearch.common.blobstore.transfer.stream.OffsetRangeInputStream; -import org.opensearch.common.util.ByteUtils; import org.opensearch.core.action.ActionListener; import org.opensearch.index.store.exception.ChecksumCombinationException; @@ -47,9 +45,8 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.UnaryOperator; import java.util.stream.Collectors; -import java.util.zip.CRC32; -import com.jcraft.jzlib.JZlib; +import static org.opensearch.common.blobstore.transfer.RemoteTransferContainer.checksumOfChecksum; /** * A {@code RemoteDirectory} provides an abstraction layer for storing a list of files to a remote store. @@ -380,11 +377,8 @@ private void uploadBlob( private long calculateChecksumOfChecksum(Directory directory, String file) throws IOException { try (IndexInput indexInput = directory.openInput(file, IOContext.DEFAULT)) { - long storedChecksum = CodecUtil.retrieveChecksum(indexInput); - CRC32 checksumOfChecksum = new CRC32(); - checksumOfChecksum.update(ByteUtils.toByteArrayBE(storedChecksum)); try { - return JZlib.crc32_combine(storedChecksum, checksumOfChecksum.getValue(), SEGMENT_CHECKSUM_BYTES); + return checksumOfChecksum(indexInput, SEGMENT_CHECKSUM_BYTES); } catch (Exception e) { throw new ChecksumCombinationException( "Potentially corrupted file: Checksum combination failed while combining stored checksum " diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumBlobStoreFormat.java b/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumBlobStoreFormat.java index 28d01256ec3e3..1d7b507a40628 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumBlobStoreFormat.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumBlobStoreFormat.java @@ -63,6 +63,7 @@ import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.gateway.CorruptStateException; +import org.opensearch.index.store.exception.ChecksumCombinationException; import org.opensearch.snapshots.SnapshotInfo; import java.io.IOException; @@ -72,6 +73,8 @@ import java.util.Locale; import java.util.Map; +import static org.opensearch.common.blobstore.transfer.RemoteTransferContainer.checksumOfChecksum; + /** * Snapshot metadata file format used in v2.0 and above * @@ -196,7 +199,19 @@ public void writeAsync( final String blobName = blobName(name); final BytesReference bytes = serialize(obj, blobName, compressor); - IndexInput input = new ByteArrayIndexInput("", BytesReference.toBytes(bytes)); + IndexInput input = new ByteArrayIndexInput("ChecksumBlobStoreFormat", BytesReference.toBytes(bytes)); + + long expectedChecksum; + try { + expectedChecksum = checksumOfChecksum(input.clone(), 8); + } catch (Exception e) { + throw new ChecksumCombinationException( + "Potentially corrupted file: Checksum combination failed while combining stored checksum " + + "and calculated checksum of stored checksum in stream", + "ChecksumBlobStoreFormat", + e + ); + } RemoteTransferContainer remoteTransferContainer = new RemoteTransferContainer( blobName, @@ -205,7 +220,7 @@ public void writeAsync( true, WritePriority.HIGH, (size, position) -> new OffsetRangeIndexInputStream(input, size, position), - CodecUtil.checksumEntireFile(input), + expectedChecksum, true );