From 18da095b5afee38a8e9ee4b6dc9a646130b6668d Mon Sep 17 00:00:00 2001 From: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Date: Fri, 19 Jul 2024 11:56:07 +0530 Subject: [PATCH 01/37] [Remote Store Migration] Reconcile remote store based index settings during STRICT mode switch (#14792) Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> --- .../MigrationBaseTestCase.java | 27 ++++ .../RemoteMigrationIndexMetadataUpdateIT.java | 67 +++++++++ .../TransportClusterUpdateSettingsAction.java | 51 +++---- .../index/remote/RemoteStoreUtils.java | 123 ++++++++++++++++ ...ransportClusterManagerNodeActionTests.java | 84 ----------- .../index/remote/RemoteStoreUtilsTests.java | 139 ++++++++++++++++++ 6 files changed, 375 insertions(+), 116 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java index 2bea36ed80c9f..e4e681a5433b5 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/MigrationBaseTestCase.java @@ -46,6 +46,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; import static org.opensearch.cluster.routing.allocation.decider.EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING; import static org.opensearch.gateway.remote.RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING; import static org.opensearch.node.remotestore.RemoteStoreNodeService.MIGRATION_DIRECTION_SETTING; @@ -277,4 +278,30 @@ protected IndexShard getIndexShard(String dataNode, String indexName) throws Exe IndexService indexService = indicesService.indexService(new Index(indexName, uuid)); return indexService.getShard(0); } + + public void changeReplicaCountAndEnsureGreen(int replicaCount, String indexName) { + assertAcked( + client().admin() + .indices() + .prepareUpdateSettings(indexName) + .setSettings(Settings.builder().put(SETTING_NUMBER_OF_REPLICAS, replicaCount)) + ); + ensureGreen(indexName); + } + + public void completeDocRepToRemoteMigration() { + assertTrue( + internalCluster().client() + .admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings( + Settings.builder() + .putNull(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey()) + .putNull(MIGRATION_DIRECTION_SETTING.getKey()) + ) + .get() + .isAcknowledged() + ); + } } diff --git a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataUpdateIT.java b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataUpdateIT.java index 216c104dfecc1..b55219e1cb37f 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataUpdateIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotemigration/RemoteMigrationIndexMetadataUpdateIT.java @@ -546,6 +546,73 @@ public void testRemoteIndexPathFileExistsAfterMigration() throws Exception { assertTrue(Arrays.stream(files).anyMatch(file -> file.toString().contains(fileNamePrefix))); } + /** + * Scenario: + * Creates an index with 1 pri 1 rep setup with 3 docrep nodes (1 cluster manager + 2 data nodes), + * initiate migration and create 3 remote nodes (1 cluster manager + 2 data nodes) and moves over + * only primary shard copy of the index + * After the primary shard copy is relocated, decrease replica count to 0, stop all docrep nodes + * and conclude migration. Remote store index settings should be applied to the index at this point. + */ + public void testIndexSettingsUpdateDuringReplicaCountDecrement() throws Exception { + String indexName = "migration-index-replica-decrement"; + String docrepClusterManager = internalCluster().startClusterManagerOnlyNode(); + + logger.info("---> Starting 2 docrep nodes"); + List docrepNodeNames = internalCluster().startDataOnlyNodes(2); + internalCluster().validateClusterFormed(); + + logger.info("---> Creating index with 1 primary and 1 replica"); + Settings oneReplica = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .build(); + createIndexAndAssertDocrepProperties(indexName, oneReplica); + + int docsToIndex = randomIntBetween(10, 100); + logger.info("---> Indexing {} on both indices", docsToIndex); + indexBulk(indexName, docsToIndex); + + logger.info( + "---> Stopping shard rebalancing to ensure shards do not automatically move over to newer nodes after they are launched" + ); + stopShardRebalancing(); + + logger.info("---> Starting 3 remote store enabled nodes"); + initDocRepToRemoteMigration(); + setAddRemote(true); + internalCluster().startClusterManagerOnlyNode(); + List remoteNodeNames = internalCluster().startDataOnlyNodes(2); + internalCluster().validateClusterFormed(); + + String primaryNode = primaryNodeName(indexName); + + logger.info("---> Moving over primary to remote store enabled nodes"); + assertAcked( + client().admin() + .cluster() + .prepareReroute() + .add(new MoveAllocationCommand(indexName, 0, primaryNode, remoteNodeNames.get(0))) + .execute() + .actionGet() + ); + waitForRelocation(); + waitNoPendingTasksOnAll(); + + logger.info("---> Reducing replica count to 0 for the index"); + changeReplicaCountAndEnsureGreen(0, indexName); + + logger.info("---> Stopping all docrep nodes"); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(docrepClusterManager)); + for (String node : docrepNodeNames) { + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(node)); + } + internalCluster().validateClusterFormed(); + completeDocRepToRemoteMigration(); + waitNoPendingTasksOnAll(); + assertRemoteProperties(indexName); + } + private void createIndexAndAssertDocrepProperties(String index, Settings settings) { createIndexAssertHealthAndDocrepProperties(index, settings, this::ensureGreen); } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java index 216e1fb2ed1cc..3988d50b2ce1e 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/settings/TransportClusterUpdateSettingsAction.java @@ -42,7 +42,6 @@ import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.block.ClusterBlockException; import org.opensearch.cluster.block.ClusterBlockLevel; -import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.IndexNameExpressionResolver; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.node.DiscoveryNode; @@ -59,17 +58,13 @@ import org.opensearch.common.settings.SettingsException; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.io.stream.StreamInput; -import org.opensearch.index.remote.RemoteMigrationIndexMetadataUpdater; import org.opensearch.node.remotestore.RemoteStoreNodeService; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; import java.io.IOException; -import java.util.Collection; -import java.util.Set; -import java.util.stream.Collectors; -import static org.opensearch.index.remote.RemoteMigrationIndexMetadataUpdater.indexHasAllRemoteStoreRelatedMetadata; +import static org.opensearch.index.remote.RemoteStoreUtils.checkAndFinalizeRemoteStoreMigration; /** * Transport action for updating cluster settings @@ -262,13 +257,14 @@ public void onFailure(String source, Exception e) { @Override public ClusterState execute(final ClusterState currentState) { - validateCompatibilityModeSettingRequest(request, state); - final ClusterState clusterState = updater.updateSettings( + boolean isCompatibilityModeChanging = validateCompatibilityModeSettingRequest(request, state); + ClusterState clusterState = updater.updateSettings( currentState, clusterSettings.upgradeSettings(request.transientSettings()), clusterSettings.upgradeSettings(request.persistentSettings()), logger ); + clusterState = checkAndFinalizeRemoteStoreMigration(isCompatibilityModeChanging, request, clusterState, logger); changed = clusterState != currentState; return clusterState; } @@ -278,19 +274,23 @@ public ClusterState execute(final ClusterState currentState) { /** * Runs various checks associated with changing cluster compatibility mode + * * @param request cluster settings update request, for settings to be updated and new values * @param clusterState current state of cluster, for information on nodes + * @return true if the incoming cluster settings update request is switching compatibility modes */ - public void validateCompatibilityModeSettingRequest(ClusterUpdateSettingsRequest request, ClusterState clusterState) { + public boolean validateCompatibilityModeSettingRequest(ClusterUpdateSettingsRequest request, ClusterState clusterState) { Settings settings = Settings.builder().put(request.persistentSettings()).put(request.transientSettings()).build(); if (RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING.exists(settings)) { - String value = RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING.get(settings).mode; validateAllNodesOfSameVersion(clusterState.nodes()); - if (RemoteStoreNodeService.CompatibilityMode.STRICT.mode.equals(value)) { + if (RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING.get( + settings + ) == RemoteStoreNodeService.CompatibilityMode.STRICT) { validateAllNodesOfSameType(clusterState.nodes()); - validateIndexSettings(clusterState); } + return true; } + return false; } /** @@ -310,31 +310,18 @@ private void validateAllNodesOfSameVersion(DiscoveryNodes discoveryNodes) { * @param discoveryNodes current discovery nodes in the cluster */ private void validateAllNodesOfSameType(DiscoveryNodes discoveryNodes) { - Set nodeTypes = discoveryNodes.getNodes() + boolean allNodesDocrepEnabled = discoveryNodes.getNodes() .values() .stream() - .map(DiscoveryNode::isRemoteStoreNode) - .collect(Collectors.toSet()); - if (nodeTypes.size() != 1) { + .allMatch(discoveryNode -> discoveryNode.isRemoteStoreNode() == false); + boolean allNodesRemoteStoreEnabled = discoveryNodes.getNodes() + .values() + .stream() + .allMatch(discoveryNode -> discoveryNode.isRemoteStoreNode()); + if (allNodesDocrepEnabled == false && allNodesRemoteStoreEnabled == false) { throw new SettingsException( "can not switch to STRICT compatibility mode when the cluster contains both remote and non-remote nodes" ); } } - - /** - * Verifies that while trying to switch to STRICT compatibility mode, - * all indices in the cluster have {@link RemoteMigrationIndexMetadataUpdater#indexHasAllRemoteStoreRelatedMetadata(IndexMetadata)} as true. - * If not, throws {@link SettingsException} - * @param clusterState current cluster state - */ - private void validateIndexSettings(ClusterState clusterState) { - Collection allIndicesMetadata = clusterState.metadata().indices().values(); - if (allIndicesMetadata.isEmpty() == false - && allIndicesMetadata.stream().anyMatch(indexMetadata -> indexHasAllRemoteStoreRelatedMetadata(indexMetadata) == false)) { - throw new SettingsException( - "can not switch to STRICT compatibility mode since all indices in the cluster does not have remote store based index settings" - ); - } - } } diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java index 654e554c96bf0..a5e0c10f72301 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java @@ -11,17 +11,23 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.opensearch.Version; +import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.RoutingTable; import org.opensearch.common.collect.Tuple; import org.opensearch.common.settings.Settings; import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; +import org.opensearch.node.remotestore.RemoteStoreNodeService; import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Base64; +import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Locale; @@ -29,7 +35,9 @@ import java.util.Objects; import java.util.Optional; import java.util.function.Function; +import java.util.stream.Collectors; +import static org.opensearch.index.remote.RemoteMigrationIndexMetadataUpdater.indexHasRemoteStoreSettings; import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_HASH_ALGORITHM_SETTING; import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_PATH_TYPE_SETTING; import static org.opensearch.indices.RemoteStoreSettings.CLUSTER_REMOTE_STORE_TRANSLOG_METADATA; @@ -250,4 +258,119 @@ public static Map getRemoteStoreRepoName(DiscoveryNodes discover .findFirst(); return remoteNode.map(RemoteStoreNodeAttribute::getDataRepoNames).orElseGet(HashMap::new); } + + /** + * Invoked after a cluster settings update. + * Checks if the applied cluster settings has switched the cluster to STRICT mode. + * If so, checks and applies appropriate index settings depending on the current set + * of node types in the cluster + * This has been intentionally done after the cluster settings update + * flow. That way we are not interfering with the usual settings update + * and the cluster state mutation that comes along with it + * + * @param isCompatibilityModeChanging flag passed from cluster settings update call to denote if a compatibility mode change has been done + * @param request request payload passed from cluster settings update + * @param currentState cluster state generated after changing cluster settings were applied + * @param logger Logger reference + * @return Mutated cluster state with remote store index settings applied, no-op if the cluster is not switching to `STRICT` compatibility mode + */ + public static ClusterState checkAndFinalizeRemoteStoreMigration( + boolean isCompatibilityModeChanging, + ClusterUpdateSettingsRequest request, + ClusterState currentState, + Logger logger + ) { + if (isCompatibilityModeChanging && isSwitchToStrictCompatibilityMode(request)) { + return finalizeMigration(currentState, logger); + } + return currentState; + } + + /** + * Finalizes the docrep to remote-store migration process by applying remote store based index settings + * on indices that are missing them. No-Op if all indices already have the settings applied through + * IndexMetadataUpdater + * + * @param incomingState mutated cluster state after cluster settings were applied + * @return new cluster state with index settings updated + */ + public static ClusterState finalizeMigration(ClusterState incomingState, Logger logger) { + Map discoveryNodeMap = incomingState.nodes().getNodes(); + if (discoveryNodeMap.isEmpty() == false) { + // At this point, we have already validated that all nodes in the cluster are of uniform type. + // Either all of them are remote store enabled, or all of them are docrep enabled + boolean remoteStoreEnabledNodePresent = discoveryNodeMap.values().stream().findFirst().get().isRemoteStoreNode(); + if (remoteStoreEnabledNodePresent == true) { + List indicesWithoutRemoteStoreSettings = getIndicesWithoutRemoteStoreSettings(incomingState, logger); + if (indicesWithoutRemoteStoreSettings.isEmpty() == true) { + logger.info("All indices in the cluster has remote store based index settings"); + } else { + Metadata mutatedMetadata = applyRemoteStoreSettings(incomingState, indicesWithoutRemoteStoreSettings, logger); + return ClusterState.builder(incomingState).metadata(mutatedMetadata).build(); + } + } else { + logger.debug("All nodes in the cluster are not remote nodes. Skipping."); + } + } + return incomingState; + } + + /** + * Filters out indices which does not have remote store based + * index settings applied even after all shard copies have + * migrated to remote store enabled nodes + */ + private static List getIndicesWithoutRemoteStoreSettings(ClusterState clusterState, Logger logger) { + Collection allIndicesMetadata = clusterState.metadata().indices().values(); + if (allIndicesMetadata.isEmpty() == false) { + List indicesWithoutRemoteSettings = allIndicesMetadata.stream() + .filter(idxMd -> indexHasRemoteStoreSettings(idxMd.getSettings()) == false) + .collect(Collectors.toList()); + logger.debug( + "Attempting to switch to strict mode. Count of indices without remote store settings {}", + indicesWithoutRemoteSettings.size() + ); + return indicesWithoutRemoteSettings; + } + return Collections.emptyList(); + } + + /** + * Applies remote store index settings through {@link RemoteMigrationIndexMetadataUpdater} + */ + private static Metadata applyRemoteStoreSettings( + ClusterState clusterState, + List indicesWithoutRemoteStoreSettings, + Logger logger + ) { + Metadata.Builder metadataBuilder = Metadata.builder(clusterState.getMetadata()); + RoutingTable currentRoutingTable = clusterState.getRoutingTable(); + DiscoveryNodes currentDiscoveryNodes = clusterState.getNodes(); + Settings currentClusterSettings = clusterState.metadata().settings(); + for (IndexMetadata indexMetadata : indicesWithoutRemoteStoreSettings) { + IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(indexMetadata); + RemoteMigrationIndexMetadataUpdater indexMetadataUpdater = new RemoteMigrationIndexMetadataUpdater( + currentDiscoveryNodes, + currentRoutingTable, + indexMetadata, + currentClusterSettings, + logger + ); + indexMetadataUpdater.maybeAddRemoteIndexSettings(indexMetadataBuilder, indexMetadata.getIndex().getName()); + metadataBuilder.put(indexMetadataBuilder); + } + return metadataBuilder.build(); + } + + /** + * Checks if the incoming cluster settings payload is attempting to switch + * the cluster to `STRICT` compatibility mode + * Visible only for tests + */ + public static boolean isSwitchToStrictCompatibilityMode(ClusterUpdateSettingsRequest request) { + Settings incomingSettings = Settings.builder().put(request.persistentSettings()).put(request.transientSettings()).build(); + return RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING.get( + incomingSettings + ) == RemoteStoreNodeService.CompatibilityMode.STRICT; + } } diff --git a/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeActionTests.java b/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeActionTests.java index 35c5c5e605b4d..37e884502b613 100644 --- a/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeActionTests.java +++ b/server/src/test/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeActionTests.java @@ -47,7 +47,6 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.settings.SettingsException; import org.opensearch.common.unit.TimeValue; -import org.opensearch.common.util.FeatureFlags; import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.core.action.ActionListener; import org.opensearch.core.action.ActionResponse; @@ -85,8 +84,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import static org.opensearch.common.util.FeatureFlags.REMOTE_STORE_MIGRATION_EXPERIMENTAL; -import static org.opensearch.index.remote.RemoteMigrationIndexMetadataUpdaterTests.createIndexMetadataWithDocrepSettings; import static org.opensearch.index.remote.RemoteMigrationIndexMetadataUpdaterTests.createIndexMetadataWithRemoteStoreSettings; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY; @@ -718,9 +715,6 @@ protected void masterOperation(Task task, Request request, ClusterState state, A } public void testDontAllowSwitchingToStrictCompatibilityModeForMixedCluster() { - Settings nodeSettings = Settings.builder().put(REMOTE_STORE_MIGRATION_EXPERIMENTAL, "true").build(); - FeatureFlags.initializeFeatureFlags(nodeSettings); - // request to change cluster compatibility mode to STRICT Settings currentCompatibilityModeSettings = Settings.builder() .put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), RemoteStoreNodeService.CompatibilityMode.MIXED) @@ -809,84 +803,7 @@ public void testDontAllowSwitchingToStrictCompatibilityModeForMixedCluster() { transportClusterUpdateSettingsAction.validateCompatibilityModeSettingRequest(request, sameTypeClusterState); } - public void testDontAllowSwitchingToStrictCompatibilityModeWithoutRemoteIndexSettings() { - Settings nodeSettings = Settings.builder().put(REMOTE_STORE_MIGRATION_EXPERIMENTAL, "true").build(); - FeatureFlags.initializeFeatureFlags(nodeSettings); - Settings currentCompatibilityModeSettings = Settings.builder() - .put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), RemoteStoreNodeService.CompatibilityMode.MIXED) - .build(); - Settings intendedCompatibilityModeSettings = Settings.builder() - .put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), RemoteStoreNodeService.CompatibilityMode.STRICT) - .build(); - ClusterUpdateSettingsRequest request = new ClusterUpdateSettingsRequest(); - request.persistentSettings(intendedCompatibilityModeSettings); - DiscoveryNode remoteNode1 = new DiscoveryNode( - UUIDs.base64UUID(), - buildNewFakeTransportAddress(), - getRemoteStoreNodeAttributes(), - DiscoveryNodeRole.BUILT_IN_ROLES, - Version.CURRENT - ); - DiscoveryNode remoteNode2 = new DiscoveryNode( - UUIDs.base64UUID(), - buildNewFakeTransportAddress(), - getRemoteStoreNodeAttributes(), - DiscoveryNodeRole.BUILT_IN_ROLES, - Version.CURRENT - ); - DiscoveryNodes discoveryNodes = DiscoveryNodes.builder() - .add(remoteNode1) - .localNodeId(remoteNode1.getId()) - .add(remoteNode2) - .localNodeId(remoteNode2.getId()) - .build(); - AllocationService allocationService = new AllocationService( - new AllocationDeciders(Collections.singleton(new MaxRetryAllocationDecider())), - new TestGatewayAllocator(), - new BalancedShardsAllocator(Settings.EMPTY), - EmptyClusterInfoService.INSTANCE, - EmptySnapshotsInfoService.INSTANCE - ); - TransportClusterUpdateSettingsAction transportClusterUpdateSettingsAction = new TransportClusterUpdateSettingsAction( - transportService, - clusterService, - threadPool, - allocationService, - new ActionFilters(Collections.emptySet()), - new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), - clusterService.getClusterSettings() - ); - - Metadata nonRemoteIndexMd = Metadata.builder(createIndexMetadataWithDocrepSettings("test")) - .persistentSettings(currentCompatibilityModeSettings) - .build(); - final ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT) - .metadata(nonRemoteIndexMd) - .nodes(discoveryNodes) - .build(); - final SettingsException exception = expectThrows( - SettingsException.class, - () -> transportClusterUpdateSettingsAction.validateCompatibilityModeSettingRequest(request, clusterState) - ); - assertEquals( - "can not switch to STRICT compatibility mode since all indices in the cluster does not have remote store based index settings", - exception.getMessage() - ); - - Metadata remoteIndexMd = Metadata.builder(createIndexMetadataWithRemoteStoreSettings("test")) - .persistentSettings(currentCompatibilityModeSettings) - .build(); - ClusterState clusterStateWithRemoteIndices = ClusterState.builder(ClusterName.DEFAULT) - .metadata(remoteIndexMd) - .nodes(discoveryNodes) - .build(); - transportClusterUpdateSettingsAction.validateCompatibilityModeSettingRequest(request, clusterStateWithRemoteIndices); - } - public void testDontAllowSwitchingCompatibilityModeForClusterWithMultipleVersions() { - Settings nodeSettings = Settings.builder().put(REMOTE_STORE_MIGRATION_EXPERIMENTAL, "true").build(); - FeatureFlags.initializeFeatureFlags(nodeSettings); - // request to change cluster compatibility mode boolean toStrictMode = randomBoolean(); Settings currentCompatibilityModeSettings = Settings.builder() @@ -988,5 +905,4 @@ private Map getRemoteStoreNodeAttributes() { remoteStoreNodeAttributes.put(REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY, "my-translog-repo-1"); return remoteStoreNodeAttributes; } - } diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteStoreUtilsTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteStoreUtilsTests.java index 15915ee431972..ec48032df4a15 100644 --- a/server/src/test/java/org/opensearch/index/remote/RemoteStoreUtilsTests.java +++ b/server/src/test/java/org/opensearch/index/remote/RemoteStoreUtilsTests.java @@ -9,15 +9,29 @@ package org.opensearch.index.remote; import org.opensearch.Version; +import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; +import org.opensearch.cluster.ClusterName; +import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodeRole; import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.IndexShardRoutingTable; +import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.routing.ShardRoutingState; +import org.opensearch.cluster.routing.TestShardRouting; +import org.opensearch.common.UUIDs; import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.support.PlainBlobMetadata; import org.opensearch.common.settings.Settings; +import org.opensearch.core.index.Index; +import org.opensearch.core.index.shard.ShardId; import org.opensearch.index.shard.IndexShardTestUtils; import org.opensearch.index.store.RemoteSegmentStoreDirectory; import org.opensearch.index.translog.transfer.TranslogTransferMetadata; +import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; import org.opensearch.test.OpenSearchTestCase; @@ -28,11 +42,15 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.UUID; import java.util.stream.Collectors; import static org.opensearch.cluster.metadata.IndexMetadata.REMOTE_STORE_CUSTOM_KEY; +import static org.opensearch.index.remote.RemoteMigrationIndexMetadataUpdaterTests.createIndexMetadataWithDocrepSettings; import static org.opensearch.index.remote.RemoteStoreUtils.URL_BASE64_CHARSET; import static org.opensearch.index.remote.RemoteStoreUtils.determineTranslogMetadataEnabled; +import static org.opensearch.index.remote.RemoteStoreUtils.finalizeMigration; +import static org.opensearch.index.remote.RemoteStoreUtils.isSwitchToStrictCompatibilityMode; import static org.opensearch.index.remote.RemoteStoreUtils.longToCompositeBase64AndBinaryEncoding; import static org.opensearch.index.remote.RemoteStoreUtils.longToUrlBase64; import static org.opensearch.index.remote.RemoteStoreUtils.urlBase64ToLong; @@ -42,6 +60,9 @@ import static org.opensearch.index.store.RemoteSegmentStoreDirectory.MetadataFilenameUtils.METADATA_PREFIX; import static org.opensearch.index.store.RemoteSegmentStoreDirectory.MetadataFilenameUtils.SEPARATOR; import static org.opensearch.index.translog.transfer.TranslogTransferMetadata.METADATA_SEPARATOR; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY; +import static org.opensearch.node.remotestore.RemoteStoreNodeService.REMOTE_STORE_COMPATIBILITY_MODE_SETTING; public class RemoteStoreUtilsTests extends OpenSearchTestCase { @@ -398,4 +419,122 @@ private static Map getCustomDataMap(int option) { ); } + public void testFinalizeMigrationWithAllRemoteNodes() { + String migratedIndex = "migrated-index"; + Settings mockSettings = Settings.builder().put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), "strict").build(); + DiscoveryNode remoteNode1 = new DiscoveryNode( + UUIDs.base64UUID(), + buildNewFakeTransportAddress(), + getRemoteStoreNodeAttributes(), + DiscoveryNodeRole.BUILT_IN_ROLES, + Version.CURRENT + ); + DiscoveryNode remoteNode2 = new DiscoveryNode( + UUIDs.base64UUID(), + buildNewFakeTransportAddress(), + getRemoteStoreNodeAttributes(), + DiscoveryNodeRole.BUILT_IN_ROLES, + Version.CURRENT + ); + DiscoveryNodes discoveryNodes = DiscoveryNodes.builder() + .add(remoteNode1) + .localNodeId(remoteNode1.getId()) + .add(remoteNode2) + .localNodeId(remoteNode2.getId()) + .build(); + Metadata docrepIdxMetadata = createIndexMetadataWithDocrepSettings(migratedIndex); + assertDocrepSettingsApplied(docrepIdxMetadata.index(migratedIndex)); + Metadata remoteIndexMd = Metadata.builder(docrepIdxMetadata).persistentSettings(mockSettings).build(); + ClusterState clusterStateWithDocrepIndexSettings = ClusterState.builder(ClusterName.DEFAULT) + .metadata(remoteIndexMd) + .nodes(discoveryNodes) + .routingTable(createRoutingTableAllShardsStarted(migratedIndex, 1, 1, remoteNode1, remoteNode2)) + .build(); + Metadata mutatedMetadata = finalizeMigration(clusterStateWithDocrepIndexSettings, logger).metadata(); + assertTrue(mutatedMetadata.index(migratedIndex).getVersion() > docrepIdxMetadata.index(migratedIndex).getVersion()); + assertRemoteSettingsApplied(mutatedMetadata.index(migratedIndex)); + } + + public void testFinalizeMigrationWithAllDocrepNodes() { + String docrepIndex = "docrep-index"; + Settings mockSettings = Settings.builder().put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), "strict").build(); + DiscoveryNode docrepNode1 = new DiscoveryNode(UUIDs.base64UUID(), buildNewFakeTransportAddress(), Version.CURRENT); + DiscoveryNode docrepNode2 = new DiscoveryNode(UUIDs.base64UUID(), buildNewFakeTransportAddress(), Version.CURRENT); + DiscoveryNodes discoveryNodes = DiscoveryNodes.builder() + .add(docrepNode1) + .localNodeId(docrepNode1.getId()) + .add(docrepNode2) + .localNodeId(docrepNode2.getId()) + .build(); + Metadata docrepIdxMetadata = createIndexMetadataWithDocrepSettings(docrepIndex); + assertDocrepSettingsApplied(docrepIdxMetadata.index(docrepIndex)); + Metadata remoteIndexMd = Metadata.builder(docrepIdxMetadata).persistentSettings(mockSettings).build(); + ClusterState clusterStateWithDocrepIndexSettings = ClusterState.builder(ClusterName.DEFAULT) + .metadata(remoteIndexMd) + .nodes(discoveryNodes) + .routingTable(createRoutingTableAllShardsStarted(docrepIndex, 1, 1, docrepNode1, docrepNode2)) + .build(); + Metadata mutatedMetadata = finalizeMigration(clusterStateWithDocrepIndexSettings, logger).metadata(); + assertEquals(docrepIdxMetadata.index(docrepIndex).getVersion(), mutatedMetadata.index(docrepIndex).getVersion()); + assertDocrepSettingsApplied(mutatedMetadata.index(docrepIndex)); + } + + public void testIsSwitchToStrictCompatibilityMode() { + Settings mockSettings = Settings.builder().put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), "strict").build(); + ClusterUpdateSettingsRequest request = new ClusterUpdateSettingsRequest(); + request.persistentSettings(mockSettings); + assertTrue(isSwitchToStrictCompatibilityMode(request)); + + mockSettings = Settings.builder().put(REMOTE_STORE_COMPATIBILITY_MODE_SETTING.getKey(), "mixed").build(); + request.persistentSettings(mockSettings); + assertFalse(isSwitchToStrictCompatibilityMode(request)); + } + + private void assertRemoteSettingsApplied(IndexMetadata indexMetadata) { + assertTrue(IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING.get(indexMetadata.getSettings())); + assertTrue(IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING.exists(indexMetadata.getSettings())); + assertTrue(IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING.exists(indexMetadata.getSettings())); + assertEquals(ReplicationType.SEGMENT, IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.get(indexMetadata.getSettings())); + } + + private void assertDocrepSettingsApplied(IndexMetadata indexMetadata) { + assertFalse(IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING.get(indexMetadata.getSettings())); + assertFalse(IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING.exists(indexMetadata.getSettings())); + assertFalse(IndexMetadata.INDEX_REMOTE_SEGMENT_STORE_REPOSITORY_SETTING.exists(indexMetadata.getSettings())); + assertEquals(ReplicationType.DOCUMENT, IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.get(indexMetadata.getSettings())); + } + + private RoutingTable createRoutingTableAllShardsStarted( + String indexName, + int numberOfShards, + int numberOfReplicas, + DiscoveryNode primaryHostingNode, + DiscoveryNode replicaHostingNode + ) { + RoutingTable.Builder builder = RoutingTable.builder(); + Index index = new Index(indexName, UUID.randomUUID().toString()); + + IndexRoutingTable.Builder indexRoutingTableBuilder = IndexRoutingTable.builder(index); + for (int i = 0; i < numberOfShards; i++) { + ShardId shardId = new ShardId(index, i); + IndexShardRoutingTable.Builder indexShardRoutingTable = new IndexShardRoutingTable.Builder(shardId); + indexShardRoutingTable.addShard( + TestShardRouting.newShardRouting(shardId, primaryHostingNode.getId(), true, ShardRoutingState.STARTED) + ); + for (int j = 0; j < numberOfReplicas; j++) { + indexShardRoutingTable.addShard( + TestShardRouting.newShardRouting(shardId, replicaHostingNode.getId(), false, ShardRoutingState.STARTED) + ); + } + indexRoutingTableBuilder.addIndexShard(indexShardRoutingTable.build()); + } + return builder.add(indexRoutingTableBuilder.build()).build(); + } + + private Map getRemoteStoreNodeAttributes() { + Map remoteStoreNodeAttributes = new HashMap<>(); + remoteStoreNodeAttributes.put(REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY, "my-segment-repo-1"); + remoteStoreNodeAttributes.put(REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY, "my-translog-repo-1"); + return remoteStoreNodeAttributes; + } } From e288962d2870efa28b3dc67f5bcb7ad1a2f9d57f Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Fri, 19 Jul 2024 20:35:08 +0530 Subject: [PATCH 02/37] Add prefix mode verification setting for repository verification (#14790) * Add prefix mode verification setting for repository verification Signed-off-by: Ashish Singh * Add UTs and randomise prefix mode repository verification Signed-off-by: Ashish Singh * Incorporate PR review feedback Signed-off-by: Ashish Singh --------- Signed-off-by: Ashish Singh --- CHANGELOG.md | 1 + .../blobstore/BlobStoreRepository.java | 43 +++++++++++++++++-- .../blobstore/BlobStoreRepositoryTests.java | 22 ++++++++++ .../test/OpenSearchIntegTestCase.java | 11 ++++- 4 files changed, 72 insertions(+), 5 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 98b4f520a5bfb..a173a8a2d5ed9 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -20,6 +20,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add matchesPluginSystemIndexPattern to SystemIndexRegistry ([#14750](https://github.com/opensearch-project/OpenSearch/pull/14750)) - Add Plugin interface for loading application based configuration templates (([#14659](https://github.com/opensearch-project/OpenSearch/issues/14659))) - Refactor remote-routing-table service inline with remote state interfaces([#14668](https://github.com/opensearch-project/OpenSearch/pull/14668)) +- Add prefix mode verification setting for repository verification (([#14790](https://github.com/opensearch-project/OpenSearch/pull/14790))) ### Dependencies - Bump `org.gradle.test-retry` from 1.5.8 to 1.5.9 ([#13442](https://github.com/opensearch-project/OpenSearch/pull/13442)) 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 02290b6a5e566..c4908f8c5fc4b 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -109,6 +109,7 @@ import org.opensearch.core.xcontent.XContentParser; import org.opensearch.index.mapper.MapperService; import org.opensearch.index.remote.RemoteStorePathStrategy; +import org.opensearch.index.remote.RemoteStorePathStrategy.PathInput; import org.opensearch.index.snapshots.IndexShardRestoreFailedException; import org.opensearch.index.snapshots.IndexShardSnapshotStatus; import org.opensearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; @@ -157,6 +158,7 @@ import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.concurrent.BlockingQueue; @@ -174,6 +176,8 @@ import java.util.stream.LongStream; import java.util.stream.Stream; +import static org.opensearch.index.remote.RemoteStoreEnums.PathHashAlgorithm.FNV_1A_COMPOSITE_1; +import static org.opensearch.index.remote.RemoteStoreEnums.PathType.HASHED_PREFIX; import static org.opensearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo.canonicalName; import static org.opensearch.repositories.blobstore.ChecksumBlobStoreFormat.SNAPSHOT_ONLY_FORMAT_PARAMS; @@ -302,6 +306,16 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp Setting.Property.NodeScope ); + /** + * Setting to enable prefix mode verification. In this mode, a hashed string is prepended at the prefix of the base + * path during repository verification. + */ + public static final Setting PREFIX_MODE_VERIFICATION_SETTING = Setting.boolSetting( + "prefix_mode_verification", + false, + Setting.Property.NodeScope + ); + protected volatile boolean supportURLRepo; private volatile int maxShardBlobDeleteBatch; @@ -369,6 +383,8 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp private final boolean isSystemRepository; + private final boolean prefixModeVerification; + private final Object lock = new Object(); private final SetOnce blobContainer = new SetOnce<>(); @@ -426,6 +442,7 @@ protected BlobStoreRepository( readRepositoryMetadata(repositoryMetadata); isSystemRepository = SYSTEM_REPOSITORY_SETTING.get(metadata.settings()); + prefixModeVerification = PREFIX_MODE_VERIFICATION_SETTING.get(metadata.settings()); this.namedXContentRegistry = namedXContentRegistry; this.threadPool = clusterService.getClusterApplierService().threadPool(); this.clusterService = clusterService; @@ -767,6 +784,10 @@ protected BlobStore getBlobStore() { return blobStore.get(); } + boolean getPrefixModeVerification() { + return prefixModeVerification; + } + /** * maintains single lazy instance of {@link BlobContainer} */ @@ -1918,7 +1939,7 @@ public String startVerification() { } else { String seed = UUIDs.randomBase64UUID(); byte[] testBytes = Strings.toUTF8Bytes(seed); - BlobContainer testContainer = blobStore().blobContainer(basePath().add(testBlobPrefix(seed))); + BlobContainer testContainer = testContainer(seed); BytesArray bytes = new BytesArray(testBytes); if (isSystemRepository == false) { try (InputStream stream = bytes.streamInput()) { @@ -1936,12 +1957,26 @@ public String startVerification() { } } + /** + * Returns the blobContainer depending on the seed and {@code prefixModeVerification}. + */ + private BlobContainer testContainer(String seed) { + BlobPath testBlobPath; + if (prefixModeVerification == true) { + PathInput pathInput = PathInput.builder().basePath(basePath()).indexUUID(seed).build(); + testBlobPath = HASHED_PREFIX.path(pathInput, FNV_1A_COMPOSITE_1); + } else { + testBlobPath = basePath(); + } + assert Objects.nonNull(testBlobPath); + return blobStore().blobContainer(testBlobPath.add(testBlobPrefix(seed))); + } + @Override public void endVerification(String seed) { if (isReadOnly() == false) { try { - final String testPrefix = testBlobPrefix(seed); - blobStore().blobContainer(basePath().add(testPrefix)).delete(); + testContainer(seed).delete(); } catch (Exception exp) { throw new RepositoryVerificationException(metadata.name(), "cannot delete test data at " + basePath(), exp); } @@ -3266,7 +3301,7 @@ public void verify(String seed, DiscoveryNode localNode) { ); } } else { - BlobContainer testBlobContainer = blobStore().blobContainer(basePath().add(testBlobPrefix(seed))); + BlobContainer testBlobContainer = testContainer(seed); try { BytesArray bytes = new BytesArray(seed); try (InputStream stream = bytes.streamInput()) { diff --git a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryTests.java b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryTests.java index 2445cad01574c..bd47507da4863 100644 --- a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryTests.java +++ b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryTests.java @@ -255,6 +255,28 @@ public void testBadChunksize() throws Exception { ); } + public void testPrefixModeVerification() throws Exception { + final Client client = client(); + final Path location = OpenSearchIntegTestCase.randomRepoPath(node().settings()); + final String repositoryName = "test-repo"; + AcknowledgedResponse putRepositoryResponse = client.admin() + .cluster() + .preparePutRepository(repositoryName) + .setType(REPO_TYPE) + .setSettings( + Settings.builder() + .put(node().settings()) + .put("location", location) + .put(BlobStoreRepository.PREFIX_MODE_VERIFICATION_SETTING.getKey(), true) + ) + .get(); + assertTrue(putRepositoryResponse.isAcknowledged()); + + final RepositoriesService repositoriesService = getInstanceFromNode(RepositoriesService.class); + final BlobStoreRepository repository = (BlobStoreRepository) repositoriesService.repository(repositoryName); + assertTrue(repository.getPrefixModeVerification()); + } + public void testFsRepositoryCompressDeprecatedIgnored() { final Path location = OpenSearchIntegTestCase.randomRepoPath(node().settings()); final Settings settings = Settings.builder().put(node().settings()).put("location", location).build(); diff --git a/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java b/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java index 7a50502e418e2..9853cef482254 100644 --- a/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java +++ b/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java @@ -152,6 +152,7 @@ import org.opensearch.node.remotestore.RemoteStoreNodeService; import org.opensearch.plugins.NetworkPlugin; import org.opensearch.plugins.Plugin; +import org.opensearch.repositories.blobstore.BlobStoreRepository; import org.opensearch.repositories.fs.ReloadableFsRepository; import org.opensearch.script.MockScriptService; import org.opensearch.search.MockSearchService; @@ -386,6 +387,8 @@ public abstract class OpenSearchIntegTestCase extends OpenSearchTestCase { protected static final String REMOTE_BACKED_STORAGE_REPOSITORY_NAME = "test-remote-store-repo"; + private static Boolean prefixModeVerificationEnable; + private Path remoteStoreRepositoryPath; private ReplicationType randomReplicationType; @@ -394,6 +397,7 @@ public abstract class OpenSearchIntegTestCase extends OpenSearchTestCase { @BeforeClass public static void beforeClass() throws Exception { + prefixModeVerificationEnable = randomBoolean(); testClusterRule.beforeClass(); } @@ -2645,16 +2649,21 @@ private static Settings buildRemoteStoreNodeAttributes( segmentRepoName ); + String prefixModeVerificationSuffix = BlobStoreRepository.PREFIX_MODE_VERIFICATION_SETTING.getKey(); + Settings.Builder settings = Settings.builder() .put("node.attr." + REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY, segmentRepoName) .put(segmentRepoTypeAttributeKey, segmentRepoType) .put(segmentRepoSettingsAttributeKeyPrefix + "location", segmentRepoPath) + .put(segmentRepoSettingsAttributeKeyPrefix + prefixModeVerificationSuffix, prefixModeVerificationEnable) .put("node.attr." + REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY, translogRepoName) .put(translogRepoTypeAttributeKey, translogRepoType) .put(translogRepoSettingsAttributeKeyPrefix + "location", translogRepoPath) + .put(translogRepoSettingsAttributeKeyPrefix + prefixModeVerificationSuffix, prefixModeVerificationEnable) .put("node.attr." + REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY, segmentRepoName) .put(stateRepoTypeAttributeKey, segmentRepoType) - .put(stateRepoSettingsAttributeKeyPrefix + "location", segmentRepoPath); + .put(stateRepoSettingsAttributeKeyPrefix + "location", segmentRepoPath) + .put(stateRepoSettingsAttributeKeyPrefix + prefixModeVerificationSuffix, prefixModeVerificationEnable); if (withRateLimiterAttributes) { settings.put(segmentRepoSettingsAttributeKeyPrefix + "compress", randomBoolean()) From 77a74e2fcf2c6b56b0959a563fbc6a0d7aff220a Mon Sep 17 00:00:00 2001 From: Rishabh Singh Date: Fri, 19 Jul 2024 10:08:35 -0700 Subject: [PATCH 03/37] add length check on comment body for benchmark workflow (#14834) Signed-off-by: Rishabh Singh --- .github/workflows/add-performance-comment.yml | 2 +- .github/workflows/benchmark-pull-request.yml | 49 +++++++++++++------ 2 files changed, 36 insertions(+), 15 deletions(-) diff --git a/.github/workflows/add-performance-comment.yml b/.github/workflows/add-performance-comment.yml index 3939de25e4cbe..b522d348c84b2 100644 --- a/.github/workflows/add-performance-comment.yml +++ b/.github/workflows/add-performance-comment.yml @@ -1,7 +1,7 @@ name: Performance Label Action on: - pull_request: + pull_request_target: types: [labeled] jobs: diff --git a/.github/workflows/benchmark-pull-request.yml b/.github/workflows/benchmark-pull-request.yml index 0de50981fa3d7..1aa2b6271719b 100644 --- a/.github/workflows/benchmark-pull-request.yml +++ b/.github/workflows/benchmark-pull-request.yml @@ -25,20 +25,41 @@ jobs: echo "USER_TAGS=pull_request_number:${{ github.event.issue.number }},repository:OpenSearch" >> $GITHUB_ENV - name: Check comment format id: check_comment - run: | - comment='${{ github.event.comment.body }}' - if echo "$comment" | jq -e 'has("run-benchmark-test")'; then - echo "Valid comment format detected, check if valid config id is provided" - config_id=$(echo $comment | jq -r '.["run-benchmark-test"]') - benchmark_configs=$(cat .github/benchmark-configs.json) - if echo $benchmark_configs | jq -e --arg id "$config_id" 'has($id)' && echo "$benchmark_configs" | jq -e --arg version "$OPENSEARCH_MAJOR_VERSION" --arg id "$config_id" '.[$id].supported_major_versions | index($version) != null' > /dev/null; then - echo $benchmark_configs | jq -r --arg id "$config_id" '.[$id]."cluster-benchmark-configs" | to_entries[] | "\(.key)=\(.value)"' >> $GITHUB_ENV - else - echo "invalid=true" >> $GITHUB_OUTPUT - fi - else - echo "invalid=true" >> $GITHUB_OUTPUT - fi + uses: actions/github-script@v6 + with: + script: | + const fs = require('fs'); + const comment = context.payload.comment.body; + let commentJson; + try { + commentJson = JSON.parse(comment); + } catch (error) { + core.setOutput('invalid', 'true'); + return; + } + if (!commentJson.hasOwnProperty('run-benchmark-test')) { + core.setOutput('invalid', 'true'); + return; + } + const configId = commentJson['run-benchmark-test']; + let benchmarkConfigs; + try { + benchmarkConfigs = JSON.parse(fs.readFileSync('.github/benchmark-configs.json', 'utf8')); + } catch (error) { + core.setFailed('Failed to read benchmark-configs.json'); + return; + } + const openSearchMajorVersion = process.env.OPENSEARCH_MAJOR_VERSION; + console.log('MAJOR_VERSION', openSearchMajorVersion) + if (!benchmarkConfigs.hasOwnProperty(configId) || + !benchmarkConfigs[configId].supported_major_versions.includes(openSearchMajorVersion)) { + core.setOutput('invalid', 'true'); + return; + } + const clusterBenchmarkConfigs = benchmarkConfigs[configId]['cluster-benchmark-configs']; + for (const [key, value] of Object.entries(clusterBenchmarkConfigs)) { + core.exportVariable(key, value); + } - name: Post invalid format comment if: steps.check_comment.outputs.invalid == 'true' uses: actions/github-script@v6 From 9c6e6187864c976656889cdec45064920cf856ee Mon Sep 17 00:00:00 2001 From: Rishabh Singh Date: Fri, 19 Jul 2024 12:28:02 -0700 Subject: [PATCH 04/37] Add restore-from-snapshot test procedure for snapshot run benchmark config (#14842) Signed-off-by: Rishabh Singh --- .github/benchmark-configs.json | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/.github/benchmark-configs.json b/.github/benchmark-configs.json index a5b1951d2240c..5b44198cd3b8e 100644 --- a/.github/benchmark-configs.json +++ b/.github/benchmark-configs.json @@ -40,7 +40,8 @@ "MIN_DISTRIBUTION": "true", "TEST_WORKLOAD": "nyc_taxis", "WORKLOAD_PARAMS": "{\"snapshot_repo_name\":\"benchmark-workloads-repo-300\",\"snapshot_bucket_name\":\"benchmark-workload-snapshots\",\"snapshot_region\":\"us-east-1\",\"snapshot_base_path\":\"workload-snapshots-300\",\"snapshot_name\":\"nyc_taxis_1_shard\"}", - "CAPTURE_NODE_STAT": "true" + "CAPTURE_NODE_STAT": "true", + "TEST_PROCEDURE": "restore-from-snapshot" }, "cluster_configuration": { "size": "Single-Node", @@ -55,7 +56,8 @@ "MIN_DISTRIBUTION": "true", "TEST_WORKLOAD": "http_logs", "WORKLOAD_PARAMS": "{\"snapshot_repo_name\":\"benchmark-workloads-repo-300\",\"snapshot_bucket_name\":\"benchmark-workload-snapshots\",\"snapshot_region\":\"us-east-1\",\"snapshot_base_path\":\"workload-snapshots-300\",\"snapshot_name\":\"http_logs_1_shard\"}", - "CAPTURE_NODE_STAT": "true" + "CAPTURE_NODE_STAT": "true", + "TEST_PROCEDURE": "restore-from-snapshot" }, "cluster_configuration": { "size": "Single-Node", @@ -70,7 +72,8 @@ "MIN_DISTRIBUTION": "true", "TEST_WORKLOAD": "big5", "WORKLOAD_PARAMS": "{\"snapshot_repo_name\":\"benchmark-workloads-repo-300\",\"snapshot_bucket_name\":\"benchmark-workload-snapshots\",\"snapshot_region\":\"us-east-1\",\"snapshot_base_path\":\"workload-snapshots-300\",\"snapshot_name\":\"big5_1_shard\"}", - "CAPTURE_NODE_STAT": "true" + "CAPTURE_NODE_STAT": "true", + "TEST_PROCEDURE": "restore-from-snapshot" }, "cluster_configuration": { "size": "Single-Node", @@ -85,7 +88,8 @@ "MIN_DISTRIBUTION": "true", "TEST_WORKLOAD": "nyc_taxis", "WORKLOAD_PARAMS": "{\"snapshot_repo_name\":\"benchmark-workloads-repo\",\"snapshot_bucket_name\":\"benchmark-workload-snapshots\",\"snapshot_region\":\"us-east-1\",\"snapshot_base_path\":\"workload-snapshots\",\"snapshot_name\":\"nyc_taxis_1_shard\"}", - "CAPTURE_NODE_STAT": "true" + "CAPTURE_NODE_STAT": "true", + "TEST_PROCEDURE": "restore-from-snapshot" }, "cluster_configuration": { "size": "Single-Node", @@ -100,7 +104,8 @@ "MIN_DISTRIBUTION": "true", "TEST_WORKLOAD": "http_logs", "WORKLOAD_PARAMS": "{\"snapshot_repo_name\":\"benchmark-workloads-repo\",\"snapshot_bucket_name\":\"benchmark-workload-snapshots\",\"snapshot_region\":\"us-east-1\",\"snapshot_base_path\":\"workload-snapshots\",\"snapshot_name\":\"http_logs_1_shard\"}", - "CAPTURE_NODE_STAT": "true" + "CAPTURE_NODE_STAT": "true", + "TEST_PROCEDURE": "restore-from-snapshot" }, "cluster_configuration": { "size": "Single-Node", @@ -115,7 +120,8 @@ "MIN_DISTRIBUTION": "true", "TEST_WORKLOAD": "big5", "WORKLOAD_PARAMS": "{\"snapshot_repo_name\":\"benchmark-workloads-repo\",\"snapshot_bucket_name\":\"benchmark-workload-snapshots\",\"snapshot_region\":\"us-east-1\",\"snapshot_base_path\":\"workload-snapshots\",\"snapshot_name\":\"big5_1_shard\"}", - "CAPTURE_NODE_STAT": "true" + "CAPTURE_NODE_STAT": "true", + "TEST_PROCEDURE": "restore-from-snapshot" }, "cluster_configuration": { "size": "Single-Node", From 0bcbafdbbd72e636ce77213e82729256c58c8d46 Mon Sep 17 00:00:00 2001 From: Rishabh Singh Date: Fri, 19 Jul 2024 16:14:27 -0700 Subject: [PATCH 05/37] Fix env variable name typo (#14843) Signed-off-by: Rishabh Singh --- .github/workflows/benchmark-pull-request.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/benchmark-pull-request.yml b/.github/workflows/benchmark-pull-request.yml index 1aa2b6271719b..2e2e83eb132de 100644 --- a/.github/workflows/benchmark-pull-request.yml +++ b/.github/workflows/benchmark-pull-request.yml @@ -115,7 +115,7 @@ jobs: headRepo=$(echo '${{ steps.get_pr.outputs.result }}' | jq -r '.headRepoFullName') headRef=$(echo '${{ steps.get_pr.outputs.result }}' | jq -r '.headRef') echo "prHeadRepo=$headRepo" >> $GITHUB_ENV - echo "prheadRef=$headRef" >> $GITHUB_ENV + echo "prHeadRef=$headRef" >> $GITHUB_ENV - name: Checkout PR Repo uses: actions/checkout@v2 with: From b980b12e7fa14620aab38363f316ce72af70df1e Mon Sep 17 00:00:00 2001 From: bowenlan-amzn Date: Fri, 19 Jul 2024 16:36:29 -0700 Subject: [PATCH 06/37] Use circuit breaker in InternalHistogram when adding empty buckets (#14754) * introduce circuit breaker in InternalHistogram Signed-off-by: bowenlan-amzn * use circuit breaker from reduce context Signed-off-by: bowenlan-amzn * add test Signed-off-by: bowenlan-amzn * revert use_real_memory change in OpenSearchNode Signed-off-by: bowenlan-amzn * add change log Signed-off-by: bowenlan-amzn --------- Signed-off-by: bowenlan-amzn --- CHANGELOG.md | 1 + .../bucket/histogram/InternalHistogram.java | 6 ++- .../histogram/InternalHistogramTests.java | 43 +++++++++++++++++++ 3 files changed, 49 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index a173a8a2d5ed9..29e70c5026bb8 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -78,6 +78,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Fix bulk upsert ignores the default_pipeline and final_pipeline when auto-created index matches the index template ([#12891](https://github.com/opensearch-project/OpenSearch/pull/12891)) - Fix NPE in ReplicaShardAllocator ([#14385](https://github.com/opensearch-project/OpenSearch/pull/14385)) - Fix constant_keyword field type used when creating index ([#14807](https://github.com/opensearch-project/OpenSearch/pull/14807)) +- Use circuit breaker in InternalHistogram when adding empty buckets ([#14754](https://github.com/opensearch-project/OpenSearch/pull/14754)) ### Security diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/InternalHistogram.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/InternalHistogram.java index a27c689127ac9..a988b911de5a3 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/InternalHistogram.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/InternalHistogram.java @@ -395,6 +395,7 @@ private void addEmptyBuckets(List list, ReduceContext reduceContext) { // fill with empty buckets for (double key = round(emptyBucketInfo.minBound); key <= emptyBucketInfo.maxBound; key = nextKey(key)) { iter.add(new Bucket(key, 0, keyed, format, reducedEmptySubAggs)); + reduceContext.consumeBucketsAndMaybeBreak(0); } } else { Bucket first = list.get(iter.nextIndex()); @@ -402,11 +403,12 @@ private void addEmptyBuckets(List list, ReduceContext reduceContext) { // fill with empty buckets until the first key for (double key = round(emptyBucketInfo.minBound); key < first.key; key = nextKey(key)) { iter.add(new Bucket(key, 0, keyed, format, reducedEmptySubAggs)); + reduceContext.consumeBucketsAndMaybeBreak(0); } } // now adding the empty buckets within the actual data, - // e.g. if the data series is [1,2,3,7] there're 3 empty buckets that will be created for 4,5,6 + // e.g. if the data series is [1,2,3,7] there are 3 empty buckets that will be created for 4,5,6 Bucket lastBucket = null; do { Bucket nextBucket = list.get(iter.nextIndex()); @@ -414,6 +416,7 @@ private void addEmptyBuckets(List list, ReduceContext reduceContext) { double key = nextKey(lastBucket.key); while (key < nextBucket.key) { iter.add(new Bucket(key, 0, keyed, format, reducedEmptySubAggs)); + reduceContext.consumeBucketsAndMaybeBreak(0); key = nextKey(key); } assert key == nextBucket.key || Double.isNaN(nextBucket.key) : "key: " + key + ", nextBucket.key: " + nextBucket.key; @@ -424,6 +427,7 @@ private void addEmptyBuckets(List list, ReduceContext reduceContext) { // finally, adding the empty buckets *after* the actual data (based on the extended_bounds.max requested by the user) for (double key = nextKey(lastBucket.key); key <= emptyBucketInfo.maxBound; key = nextKey(key)) { iter.add(new Bucket(key, 0, keyed, format, reducedEmptySubAggs)); + reduceContext.consumeBucketsAndMaybeBreak(0); } } } diff --git a/server/src/test/java/org/opensearch/search/aggregations/bucket/histogram/InternalHistogramTests.java b/server/src/test/java/org/opensearch/search/aggregations/bucket/histogram/InternalHistogramTests.java index 288b22ccfcc92..98c6ac2b3de45 100644 --- a/server/src/test/java/org/opensearch/search/aggregations/bucket/histogram/InternalHistogramTests.java +++ b/server/src/test/java/org/opensearch/search/aggregations/bucket/histogram/InternalHistogramTests.java @@ -33,10 +33,15 @@ package org.opensearch.search.aggregations.bucket.histogram; import org.apache.lucene.tests.util.TestUtil; +import org.opensearch.core.common.breaker.CircuitBreaker; +import org.opensearch.core.common.breaker.CircuitBreakingException; import org.opensearch.search.DocValueFormat; import org.opensearch.search.aggregations.BucketOrder; +import org.opensearch.search.aggregations.InternalAggregation; import org.opensearch.search.aggregations.InternalAggregations; +import org.opensearch.search.aggregations.MultiBucketConsumerService; import org.opensearch.search.aggregations.ParsedMultiBucketAggregation; +import org.opensearch.search.aggregations.pipeline.PipelineAggregator; import org.opensearch.test.InternalAggregationTestCase; import org.opensearch.test.InternalMultiBucketAggregationTestCase; @@ -47,6 +52,8 @@ import java.util.Map; import java.util.TreeMap; +import org.mockito.Mockito; + public class InternalHistogramTests extends InternalMultiBucketAggregationTestCase { private boolean keyed; @@ -123,6 +130,42 @@ public void testHandlesNaN() { ); } + public void testCircuitBreakerWhenAddEmptyBuckets() { + String name = randomAlphaOfLength(5); + double interval = 1; + double lowerBound = 1; + double upperBound = 1026; + List bucket1 = List.of( + new InternalHistogram.Bucket(lowerBound, 1, false, format, InternalAggregations.EMPTY) + ); + List bucket2 = List.of( + new InternalHistogram.Bucket(upperBound, 1, false, format, InternalAggregations.EMPTY) + ); + BucketOrder order = BucketOrder.key(true); + InternalHistogram.EmptyBucketInfo emptyBucketInfo = new InternalHistogram.EmptyBucketInfo( + interval, + 0, + lowerBound, + upperBound, + InternalAggregations.EMPTY + ); + InternalHistogram histogram1 = new InternalHistogram(name, bucket1, order, 0, emptyBucketInfo, format, false, null); + InternalHistogram histogram2 = new InternalHistogram(name, bucket2, order, 0, emptyBucketInfo, format, false, null); + + CircuitBreaker breaker = Mockito.mock(CircuitBreaker.class); + Mockito.when(breaker.addEstimateBytesAndMaybeBreak(0, "allocated_buckets")).thenThrow(CircuitBreakingException.class); + + MultiBucketConsumerService.MultiBucketConsumer bucketConsumer = new MultiBucketConsumerService.MultiBucketConsumer(0, breaker); + InternalAggregation.ReduceContext reduceContext = InternalAggregation.ReduceContext.forFinalReduction( + null, + null, + bucketConsumer, + PipelineAggregator.PipelineTree.EMPTY + ); + expectThrows(CircuitBreakingException.class, () -> histogram1.reduce(List.of(histogram1, histogram2), reduceContext)); + Mockito.verify(breaker, Mockito.times(1)).addEstimateBytesAndMaybeBreak(0, "allocated_buckets"); + } + @Override protected void assertReduced(InternalHistogram reduced, List inputs) { TreeMap expectedCounts = new TreeMap<>(); From b58546914c1191d6364d9afa006a48bba00ef596 Mon Sep 17 00:00:00 2001 From: Shivansh Arora Date: Mon, 22 Jul 2024 15:03:13 +0530 Subject: [PATCH 07/37] [Remote State] Create interface RemoteEntitiesManager (#14671) * Create interface RemoteEntitiesManager Signed-off-by: Shivansh Arora --- .../InternalRemoteRoutingTableService.java | 13 +- .../remote/NoopRemoteRoutingTableService.java | 7 +- .../remote/RemoteRoutingTableService.java | 5 +- .../AbstractRemoteWritableEntityManager.java | 84 ++++ .../remote/RemoteWritableEntityManager.java | 47 ++ .../RemoteClusterStateAttributesManager.java | 55 +-- .../remote/RemoteClusterStateService.java | 459 ++++++++---------- .../remote/RemoteGlobalMetadataManager.java | 57 +-- .../remote/RemoteIndexMetadataManager.java | 87 ++-- .../RemoteRoutingTableServiceTests.java | 4 +- ...tractRemoteWritableEntityManagerTests.java | 64 +++ ...oteClusterStateAttributesManagerTests.java | 59 +-- .../RemoteClusterStateServiceTests.java | 346 ++++++------- .../RemoteGlobalMetadataManagerTests.java | 114 +++-- .../RemoteIndexMetadataManagerTests.java | 42 +- 15 files changed, 769 insertions(+), 674 deletions(-) create mode 100644 server/src/main/java/org/opensearch/common/remote/AbstractRemoteWritableEntityManager.java create mode 100644 server/src/main/java/org/opensearch/common/remote/RemoteWritableEntityManager.java create mode 100644 server/src/test/java/org/opensearch/common/remote/AbstractRemoteWritableEntityManagerTests.java diff --git a/server/src/main/java/org/opensearch/cluster/routing/remote/InternalRemoteRoutingTableService.java b/server/src/main/java/org/opensearch/cluster/routing/remote/InternalRemoteRoutingTableService.java index f3f245ee9f8f0..d7ebc54598b37 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/remote/InternalRemoteRoutingTableService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/remote/InternalRemoteRoutingTableService.java @@ -15,7 +15,6 @@ import org.opensearch.cluster.DiffableUtils; import org.opensearch.cluster.routing.IndexRoutingTable; import org.opensearch.cluster.routing.RoutingTable; -import org.opensearch.common.CheckedRunnable; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.lifecycle.AbstractLifecycleComponent; import org.opensearch.common.remote.RemoteWritableEntityStore; @@ -102,16 +101,16 @@ public DiffableUtils.MapDiff getAsyncIndexRoutingWriteAction( + public void getAsyncIndexRoutingWriteAction( String clusterUUID, long term, long version, @@ -128,7 +127,7 @@ public CheckedRunnable getAsyncIndexRoutingWriteAction( ) ); - return () -> remoteIndexRoutingTableStore.writeAsync(remoteIndexRoutingTable, completionListener); + remoteIndexRoutingTableStore.writeAsync(remoteIndexRoutingTable, completionListener); } /** @@ -156,7 +155,7 @@ public List getAllUploadedIndices } @Override - public CheckedRunnable getAsyncIndexRoutingReadAction( + public void getAsyncIndexRoutingReadAction( String clusterUUID, String uploadedFilename, LatchedActionListener latchedActionListener @@ -169,7 +168,7 @@ public CheckedRunnable getAsyncIndexRoutingReadAction( RemoteIndexRoutingTable remoteIndexRoutingTable = new RemoteIndexRoutingTable(uploadedFilename, clusterUUID, compressor); - return () -> remoteIndexRoutingTableStore.readAsync(remoteIndexRoutingTable, actionListener); + remoteIndexRoutingTableStore.readAsync(remoteIndexRoutingTable, actionListener); } @Override diff --git a/server/src/main/java/org/opensearch/cluster/routing/remote/NoopRemoteRoutingTableService.java b/server/src/main/java/org/opensearch/cluster/routing/remote/NoopRemoteRoutingTableService.java index 4636e492df28f..e6e68e01e761f 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/remote/NoopRemoteRoutingTableService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/remote/NoopRemoteRoutingTableService.java @@ -12,7 +12,6 @@ import org.opensearch.cluster.DiffableUtils; import org.opensearch.cluster.routing.IndexRoutingTable; import org.opensearch.cluster.routing.RoutingTable; -import org.opensearch.common.CheckedRunnable; import org.opensearch.common.lifecycle.AbstractLifecycleComponent; import org.opensearch.gateway.remote.ClusterMetadataManifest; @@ -39,7 +38,7 @@ public DiffableUtils.MapDiff getAsyncIndexRoutingWriteAction( + public void getAsyncIndexRoutingWriteAction( String clusterUUID, long term, long version, @@ -47,7 +46,6 @@ public CheckedRunnable getAsyncIndexRoutingWriteAction( LatchedActionListener latchedActionListener ) { // noop - return () -> {}; } @Override @@ -61,13 +59,12 @@ public List getAllUploadedIndices } @Override - public CheckedRunnable getAsyncIndexRoutingReadAction( + public void getAsyncIndexRoutingReadAction( String clusterUUID, String uploadedFilename, LatchedActionListener latchedActionListener ) { // noop - return () -> {}; } @Override diff --git a/server/src/main/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableService.java b/server/src/main/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableService.java index d319123bc2cee..0b0b4bb7dbc84 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableService.java @@ -12,7 +12,6 @@ import org.opensearch.cluster.DiffableUtils; import org.opensearch.cluster.routing.IndexRoutingTable; import org.opensearch.cluster.routing.RoutingTable; -import org.opensearch.common.CheckedRunnable; import org.opensearch.common.lifecycle.LifecycleComponent; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -43,7 +42,7 @@ public IndexRoutingTable read(StreamInput in, String key) throws IOException { List getIndicesRouting(RoutingTable routingTable); - CheckedRunnable getAsyncIndexRoutingReadAction( + void getAsyncIndexRoutingReadAction( String clusterUUID, String uploadedFilename, LatchedActionListener latchedActionListener @@ -59,7 +58,7 @@ DiffableUtils.MapDiff> RoutingTable after ); - CheckedRunnable getAsyncIndexRoutingWriteAction( + void getAsyncIndexRoutingWriteAction( String clusterUUID, long term, long version, diff --git a/server/src/main/java/org/opensearch/common/remote/AbstractRemoteWritableEntityManager.java b/server/src/main/java/org/opensearch/common/remote/AbstractRemoteWritableEntityManager.java new file mode 100644 index 0000000000000..dc301635c4a80 --- /dev/null +++ b/server/src/main/java/org/opensearch/common/remote/AbstractRemoteWritableEntityManager.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.common.remote; + +import org.opensearch.core.action.ActionListener; +import org.opensearch.gateway.remote.ClusterMetadataManifest; +import org.opensearch.gateway.remote.model.RemoteReadResult; + +import java.util.HashMap; +import java.util.Map; + +/** + * An abstract class that provides a base implementation for managing remote entities in the remote store. + */ +public abstract class AbstractRemoteWritableEntityManager implements RemoteWritableEntityManager { + /** + * A map that stores the remote writable entity stores, keyed by the entity type. + */ + protected final Map remoteWritableEntityStores = new HashMap<>(); + + /** + * Retrieves the remote writable entity store for the given entity. + * + * @param entity the entity for which the store is requested + * @return the remote writable entity store for the given entity + * @throws IllegalArgumentException if the entity type is unknown + */ + protected RemoteWritableEntityStore getStore(AbstractRemoteWritableBlobEntity entity) { + RemoteWritableEntityStore remoteStore = remoteWritableEntityStores.get(entity.getType()); + if (remoteStore == null) { + throw new IllegalArgumentException("Unknown entity type [" + entity.getType() + "]"); + } + return remoteStore; + } + + /** + * Returns an ActionListener for handling the write operation for the specified component, remote object, and latched action listener. + * + * @param component the component for which the write operation is performed + * @param remoteEntity the remote object to be written + * @param listener the listener to be notified when the write operation completes + * @return an ActionListener for handling the write operation + */ + protected abstract ActionListener getWrappedWriteListener( + String component, + AbstractRemoteWritableBlobEntity remoteEntity, + ActionListener listener + ); + + /** + * Returns an ActionListener for handling the read operation for the specified component, + * remote object, and latched action listener. + * + * @param component the component for which the read operation is performed + * @param remoteEntity the remote object to be read + * @param listener the listener to be notified when the read operation completes + * @return an ActionListener for handling the read operation + */ + protected abstract ActionListener getWrappedReadListener( + String component, + AbstractRemoteWritableBlobEntity remoteEntity, + ActionListener listener + ); + + @Override + public void writeAsync( + String component, + AbstractRemoteWritableBlobEntity entity, + ActionListener listener + ) { + getStore(entity).writeAsync(entity, getWrappedWriteListener(component, entity, listener)); + } + + @Override + public void readAsync(String component, AbstractRemoteWritableBlobEntity entity, ActionListener listener) { + getStore(entity).readAsync(entity, getWrappedReadListener(component, entity, listener)); + } +} diff --git a/server/src/main/java/org/opensearch/common/remote/RemoteWritableEntityManager.java b/server/src/main/java/org/opensearch/common/remote/RemoteWritableEntityManager.java new file mode 100644 index 0000000000000..7693d1b5284bd --- /dev/null +++ b/server/src/main/java/org/opensearch/common/remote/RemoteWritableEntityManager.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.remote; + +import org.opensearch.core.action.ActionListener; +import org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadata; +import org.opensearch.gateway.remote.model.RemoteReadResult; + +/** + * The RemoteWritableEntityManager interface provides async read and write methods for managing remote entities in the remote store + */ +public interface RemoteWritableEntityManager { + + /** + * Performs an asynchronous read operation for the specified component and entity. + * + * @param component the component for which the read operation is performed + * @param entity the entity to be read + * @param listener the listener to be notified when the read operation completes. + * The listener's {@link ActionListener#onResponse(Object)} method + * is called with a {@link RemoteReadResult} object containing the + * read data on successful read. The + * {@link ActionListener#onFailure(Exception)} method is called with + * an exception if the read operation fails. + */ + void readAsync(String component, AbstractRemoteWritableBlobEntity entity, ActionListener listener); + + /** + * Performs an asynchronous write operation for the specified component and entity. + * + * @param component the component for which the write operation is performed + * @param entity the entity to be written + * @param listener the listener to be notified when the write operation completes. + * The listener's {@link ActionListener#onResponse(Object)} method + * is called with a {@link UploadedMetadata} object containing the + * uploaded metadata on successful write. The + * {@link ActionListener#onFailure(Exception)} method is called with + * an exception if the write operation fails. + */ + void writeAsync(String component, AbstractRemoteWritableBlobEntity entity, ActionListener listener); +} diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManager.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManager.java index 8f986423587d7..67ac8d2b9a810 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManager.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManager.java @@ -8,13 +8,11 @@ package org.opensearch.gateway.remote; -import org.opensearch.action.LatchedActionListener; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.DiffableUtils; import org.opensearch.cluster.DiffableUtils.NonDiffableValueSerializer; -import org.opensearch.common.CheckedRunnable; import org.opensearch.common.remote.AbstractRemoteWritableBlobEntity; -import org.opensearch.common.remote.RemoteWritableEntityStore; +import org.opensearch.common.remote.AbstractRemoteWritableEntityManager; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; import org.opensearch.gateway.remote.model.RemoteClusterBlocks; @@ -26,9 +24,7 @@ import org.opensearch.repositories.blobstore.BlobStoreRepository; import org.opensearch.threadpool.ThreadPool; -import java.io.IOException; import java.util.Collections; -import java.util.HashMap; import java.util.Map; /** @@ -36,13 +32,11 @@ * * @opensearch.internal */ -public class RemoteClusterStateAttributesManager { +public class RemoteClusterStateAttributesManager extends AbstractRemoteWritableEntityManager { public static final String CLUSTER_STATE_ATTRIBUTE = "cluster_state_attribute"; public static final String DISCOVERY_NODES = "nodes"; public static final String CLUSTER_BLOCKS = "blocks"; public static final int CLUSTER_STATE_ATTRIBUTES_CURRENT_CODEC_VERSION = 1; - private final Map remoteWritableEntityStores; - private final NamedWriteableRegistry namedWriteableRegistry; RemoteClusterStateAttributesManager( String clusterName, @@ -51,8 +45,6 @@ public class RemoteClusterStateAttributesManager { NamedWriteableRegistry namedWriteableRegistry, ThreadPool threadpool ) { - this.namedWriteableRegistry = namedWriteableRegistry; - this.remoteWritableEntityStores = new HashMap<>(); this.remoteWritableEntityStores.put( RemoteDiscoveryNodes.DISCOVERY_NODES, new RemoteClusterStateBlobStore<>( @@ -85,46 +77,28 @@ public class RemoteClusterStateAttributesManager { ); } - /** - * Allows async upload of Cluster State Attribute components to remote - */ - CheckedRunnable getAsyncMetadataWriteAction( + @Override + protected ActionListener getWrappedWriteListener( String component, - AbstractRemoteWritableBlobEntity blobEntity, - LatchedActionListener latchedActionListener - ) { - return () -> getStore(blobEntity).writeAsync(blobEntity, getActionListener(component, blobEntity, latchedActionListener)); - } - - private ActionListener getActionListener( - String component, - AbstractRemoteWritableBlobEntity remoteObject, - LatchedActionListener latchedActionListener + AbstractRemoteWritableBlobEntity remoteEntity, + ActionListener listener ) { return ActionListener.wrap( - resp -> latchedActionListener.onResponse(remoteObject.getUploadedMetadata()), - ex -> latchedActionListener.onFailure(new RemoteStateTransferException(component, remoteObject, ex)) + resp -> listener.onResponse(remoteEntity.getUploadedMetadata()), + ex -> listener.onFailure(new RemoteStateTransferException("Upload failed for " + component, remoteEntity, ex)) ); } - private RemoteWritableEntityStore getStore(AbstractRemoteWritableBlobEntity entity) { - RemoteWritableEntityStore remoteStore = remoteWritableEntityStores.get(entity.getType()); - if (remoteStore == null) { - throw new IllegalArgumentException("Unknown entity type [" + entity.getType() + "]"); - } - return remoteStore; - } - - public CheckedRunnable getAsyncMetadataReadAction( + @Override + protected ActionListener getWrappedReadListener( String component, - AbstractRemoteWritableBlobEntity blobEntity, - LatchedActionListener listener + AbstractRemoteWritableBlobEntity remoteEntity, + ActionListener listener ) { - final ActionListener actionListener = ActionListener.wrap( + return ActionListener.wrap( response -> listener.onResponse(new RemoteReadResult(response, CLUSTER_STATE_ATTRIBUTE, component)), - listener::onFailure + ex -> listener.onFailure(new RemoteStateTransferException("Download failed for " + component, remoteEntity, ex)) ); - return () -> getStore(blobEntity).readAsync(blobEntity, actionListener); } public DiffableUtils.MapDiff> getUpdatedCustoms( @@ -158,4 +132,5 @@ public DiffableUtils.MapDiff> uploadTasks = new ConcurrentHashMap<>(totalUploadTasks); + List uploadTasks = Collections.synchronizedList(new ArrayList<>(totalUploadTasks)); Map results = new ConcurrentHashMap<>(totalUploadTasks); List exceptionList = Collections.synchronizedList(new ArrayList<>(totalUploadTasks)); @@ -516,167 +515,155 @@ UploadedMetadataResults writeMetadataInParallel( ); if (uploadSettingsMetadata) { - uploadTasks.put( + uploadTasks.add(SETTING_METADATA); + remoteGlobalMetadataManager.writeAsync( SETTING_METADATA, - remoteGlobalMetadataManager.getAsyncMetadataWriteAction( - new RemotePersistentSettingsMetadata( - clusterState.metadata().persistentSettings(), - clusterState.metadata().version(), - clusterState.metadata().clusterUUID(), - blobStoreRepository.getCompressor(), - blobStoreRepository.getNamedXContentRegistry() - ), - listener - ) + new RemotePersistentSettingsMetadata( + clusterState.metadata().persistentSettings(), + clusterState.metadata().version(), + clusterState.metadata().clusterUUID(), + blobStoreRepository.getCompressor(), + blobStoreRepository.getNamedXContentRegistry() + ), + listener ); } if (uploadTransientSettingMetadata) { - uploadTasks.put( + uploadTasks.add(TRANSIENT_SETTING_METADATA); + remoteGlobalMetadataManager.writeAsync( TRANSIENT_SETTING_METADATA, - remoteGlobalMetadataManager.getAsyncMetadataWriteAction( - new RemoteTransientSettingsMetadata( - clusterState.metadata().transientSettings(), - clusterState.metadata().version(), - clusterState.metadata().clusterUUID(), - blobStoreRepository.getCompressor(), - blobStoreRepository.getNamedXContentRegistry() - ), - listener - ) + new RemoteTransientSettingsMetadata( + clusterState.metadata().transientSettings(), + clusterState.metadata().version(), + clusterState.metadata().clusterUUID(), + blobStoreRepository.getCompressor(), + blobStoreRepository.getNamedXContentRegistry() + ), + listener ); } if (uploadCoordinationMetadata) { - uploadTasks.put( + uploadTasks.add(COORDINATION_METADATA); + remoteGlobalMetadataManager.writeAsync( COORDINATION_METADATA, - remoteGlobalMetadataManager.getAsyncMetadataWriteAction( - new RemoteCoordinationMetadata( - clusterState.metadata().coordinationMetadata(), - clusterState.metadata().version(), - clusterState.metadata().clusterUUID(), - blobStoreRepository.getCompressor(), - blobStoreRepository.getNamedXContentRegistry() - ), - listener - ) + new RemoteCoordinationMetadata( + clusterState.metadata().coordinationMetadata(), + clusterState.metadata().version(), + clusterState.metadata().clusterUUID(), + blobStoreRepository.getCompressor(), + blobStoreRepository.getNamedXContentRegistry() + ), + listener ); } if (uploadTemplateMetadata) { - uploadTasks.put( + uploadTasks.add(TEMPLATES_METADATA); + remoteGlobalMetadataManager.writeAsync( TEMPLATES_METADATA, - remoteGlobalMetadataManager.getAsyncMetadataWriteAction( - new RemoteTemplatesMetadata( - clusterState.metadata().templatesMetadata(), - clusterState.metadata().version(), - clusterState.metadata().clusterUUID(), - blobStoreRepository.getCompressor(), - blobStoreRepository.getNamedXContentRegistry() - ), - listener - ) + new RemoteTemplatesMetadata( + clusterState.metadata().templatesMetadata(), + clusterState.metadata().version(), + clusterState.metadata().clusterUUID(), + blobStoreRepository.getCompressor(), + blobStoreRepository.getNamedXContentRegistry() + ), + listener ); } if (uploadDiscoveryNodes) { - uploadTasks.put( - DISCOVERY_NODES, - remoteClusterStateAttributesManager.getAsyncMetadataWriteAction( - RemoteDiscoveryNodes.DISCOVERY_NODES, - new RemoteDiscoveryNodes( - clusterState.nodes(), - clusterState.version(), - clusterState.stateUUID(), - blobStoreRepository.getCompressor() - ), - listener - ) + uploadTasks.add(DISCOVERY_NODES); + remoteClusterStateAttributesManager.writeAsync( + RemoteDiscoveryNodes.DISCOVERY_NODES, + new RemoteDiscoveryNodes( + clusterState.nodes(), + clusterState.version(), + clusterState.stateUUID(), + blobStoreRepository.getCompressor() + ), + listener ); } if (uploadClusterBlock) { - uploadTasks.put( - CLUSTER_BLOCKS, - remoteClusterStateAttributesManager.getAsyncMetadataWriteAction( - RemoteClusterBlocks.CLUSTER_BLOCKS, - new RemoteClusterBlocks( - clusterState.blocks(), - clusterState.version(), - clusterState.metadata().clusterUUID(), - blobStoreRepository.getCompressor() - ), - listener - ) + uploadTasks.add(CLUSTER_BLOCKS); + remoteClusterStateAttributesManager.writeAsync( + RemoteClusterBlocks.CLUSTER_BLOCKS, + new RemoteClusterBlocks( + clusterState.blocks(), + clusterState.version(), + clusterState.metadata().clusterUUID(), + blobStoreRepository.getCompressor() + ), + listener ); } if (uploadHashesOfConsistentSettings) { - uploadTasks.put( + uploadTasks.add(HASHES_OF_CONSISTENT_SETTINGS); + remoteGlobalMetadataManager.writeAsync( HASHES_OF_CONSISTENT_SETTINGS, - remoteGlobalMetadataManager.getAsyncMetadataWriteAction( - new RemoteHashesOfConsistentSettings( - (DiffableStringMap) clusterState.metadata().hashesOfConsistentSettings(), - clusterState.metadata().version(), - clusterState.metadata().clusterUUID(), - blobStoreRepository.getCompressor() - ), - listener - ) + new RemoteHashesOfConsistentSettings( + (DiffableStringMap) clusterState.metadata().hashesOfConsistentSettings(), + clusterState.metadata().version(), + clusterState.metadata().clusterUUID(), + blobStoreRepository.getCompressor() + ), + listener ); } customToUpload.forEach((key, value) -> { String customComponent = String.join(CUSTOM_DELIMITER, CUSTOM_METADATA, key); - uploadTasks.put( + uploadTasks.add(customComponent); + remoteGlobalMetadataManager.writeAsync( customComponent, - remoteGlobalMetadataManager.getAsyncMetadataWriteAction( - new RemoteCustomMetadata( - value, - key, - clusterState.metadata().version(), - clusterState.metadata().clusterUUID(), - blobStoreRepository.getCompressor(), - namedWriteableRegistry - ), - listener - ) + new RemoteCustomMetadata( + value, + key, + clusterState.metadata().version(), + clusterState.metadata().clusterUUID(), + blobStoreRepository.getCompressor(), + namedWriteableRegistry + ), + listener ); }); indexToUpload.forEach(indexMetadata -> { - uploadTasks.put( + uploadTasks.add(indexMetadata.getIndex().getName()); + remoteIndexMetadataManager.writeAsync( indexMetadata.getIndex().getName(), - remoteIndexMetadataManager.getAsyncIndexMetadataWriteAction(indexMetadata, clusterState.metadata().clusterUUID(), listener) + new RemoteIndexMetadata( + indexMetadata, + clusterState.metadata().clusterUUID(), + blobStoreRepository.getCompressor(), + blobStoreRepository.getNamedXContentRegistry() + ), + listener ); }); clusterStateCustomToUpload.forEach((key, value) -> { - uploadTasks.put( - key, - remoteClusterStateAttributesManager.getAsyncMetadataWriteAction( - CLUSTER_STATE_CUSTOM, - new RemoteClusterStateCustoms( - value, - key, - clusterState.version(), - clusterState.metadata().clusterUUID(), - blobStoreRepository.getCompressor(), - namedWriteableRegistry - ), - listener - ) + uploadTasks.add(key); + remoteClusterStateAttributesManager.writeAsync( + CLUSTER_STATE_CUSTOM, + new RemoteClusterStateCustoms( + value, + key, + clusterState.version(), + clusterState.metadata().clusterUUID(), + blobStoreRepository.getCompressor(), + namedWriteableRegistry + ), + listener ); }); indicesRoutingToUpload.forEach(indexRoutingTable -> { - uploadTasks.put( - INDEX_ROUTING_METADATA_PREFIX + indexRoutingTable.getIndex().getName(), - remoteRoutingTableService.getAsyncIndexRoutingWriteAction( - clusterState.metadata().clusterUUID(), - clusterState.term(), - clusterState.version(), - indexRoutingTable, - listener - ) + uploadTasks.add(INDEX_ROUTING_METADATA_PREFIX + indexRoutingTable.getIndex().getName()); + remoteRoutingTableService.getAsyncIndexRoutingWriteAction( + clusterState.metadata().clusterUUID(), + clusterState.term(), + clusterState.version(), + indexRoutingTable, + listener ); }); - - // start async upload of all required metadata files - for (CheckedRunnable uploadTask : uploadTasks.values()) { - uploadTask.run(); - } invokeIndexMetadataUploadListeners(indexToUpload, prevIndexMetadataByName, latch, exceptionList); try { @@ -686,7 +673,7 @@ UploadedMetadataResults writeMetadataInParallel( String.format( Locale.ROOT, "Timed out waiting for transfer of following metadata to complete - %s", - String.join(", ", uploadTasks.keySet()) + String.join(", ", uploadTasks) ) ); exceptionList.forEach(ex::addSuppressed); @@ -695,11 +682,7 @@ UploadedMetadataResults writeMetadataInParallel( } catch (InterruptedException ex) { exceptionList.forEach(ex::addSuppressed); RemoteStateTransferException exception = new RemoteStateTransferException( - String.format( - Locale.ROOT, - "Timed out waiting for transfer of metadata to complete - %s", - String.join(", ", uploadTasks.keySet()) - ), + String.format(Locale.ROOT, "Timed out waiting for transfer of metadata to complete - %s", String.join(", ", uploadTasks)), ex ); Thread.currentThread().interrupt(); @@ -707,14 +690,20 @@ UploadedMetadataResults writeMetadataInParallel( } if (!exceptionList.isEmpty()) { RemoteStateTransferException exception = new RemoteStateTransferException( + String.format(Locale.ROOT, "Exception during transfer of following metadata to Remote - %s", String.join(", ", uploadTasks)) + ); + exceptionList.forEach(exception::addSuppressed); + throw exception; + } + if (results.size() != uploadTasks.size()) { + throw new RemoteStateTransferException( String.format( Locale.ROOT, - "Exception during transfer of following metadata to Remote - %s", - String.join(", ", uploadTasks.keySet()) + "Some metadata components were not uploaded successfully. Objects to be uploaded: %s, uploaded objects: %s", + String.join(", ", uploadTasks), + String.join(", ", results.keySet()) ) ); - exceptionList.forEach(exception::addSuppressed); - throw exception; } UploadedMetadataResults response = new UploadedMetadataResults(); results.forEach((name, uploadedMetadata) -> { @@ -998,7 +987,6 @@ ClusterState readClusterStateInParallel( + (readTransientSettingsMetadata ? 1 : 0) + (readHashesOfConsistentSettings ? 1 : 0) + clusterStateCustomToRead.size() + indicesRoutingToRead.size(); CountDownLatch latch = new CountDownLatch(totalReadTasks); - List> asyncMetadataReadActions = new ArrayList<>(); List readResults = Collections.synchronizedList(new ArrayList<>()); List readIndexRoutingTableResults = Collections.synchronizedList(new ArrayList<>()); List exceptionList = Collections.synchronizedList(new ArrayList<>(totalReadTasks)); @@ -1012,8 +1000,15 @@ ClusterState readClusterStateInParallel( }), latch); for (UploadedIndexMetadata indexMetadata : indicesToRead) { - asyncMetadataReadActions.add( - remoteIndexMetadataManager.getAsyncIndexMetadataReadAction(clusterUUID, indexMetadata.getUploadedFilename(), listener) + remoteIndexMetadataManager.readAsync( + indexMetadata.getIndexName(), + new RemoteIndexMetadata( + RemoteClusterStateUtils.getFormattedIndexFileName(indexMetadata.getUploadedFilename()), + clusterUUID, + blobStoreRepository.getCompressor(), + blobStoreRepository.getNamedXContentRegistry() + ), + listener ); } @@ -1029,154 +1024,130 @@ ClusterState readClusterStateInParallel( ); for (UploadedIndexMetadata indexRouting : indicesRoutingToRead) { - asyncMetadataReadActions.add( - remoteRoutingTableService.getAsyncIndexRoutingReadAction( - clusterUUID, - indexRouting.getUploadedFilename(), - routingTableLatchedActionListener - ) + remoteRoutingTableService.getAsyncIndexRoutingReadAction( + clusterUUID, + indexRouting.getUploadedFilename(), + routingTableLatchedActionListener ); } for (Map.Entry entry : customToRead.entrySet()) { - asyncMetadataReadActions.add( - remoteGlobalMetadataManager.getAsyncMetadataReadAction( - new RemoteCustomMetadata( - entry.getValue().getUploadedFilename(), - entry.getKey(), - clusterUUID, - blobStoreRepository.getCompressor(), - namedWriteableRegistry - ), - entry.getValue().getAttributeName(), - listener - ) + remoteGlobalMetadataManager.readAsync( + entry.getValue().getAttributeName(), + new RemoteCustomMetadata( + entry.getValue().getUploadedFilename(), + entry.getKey(), + clusterUUID, + blobStoreRepository.getCompressor(), + namedWriteableRegistry + ), + listener ); } if (readCoordinationMetadata) { - asyncMetadataReadActions.add( - remoteGlobalMetadataManager.getAsyncMetadataReadAction( - new RemoteCoordinationMetadata( - manifest.getCoordinationMetadata().getUploadedFilename(), - clusterUUID, - blobStoreRepository.getCompressor(), - blobStoreRepository.getNamedXContentRegistry() - ), - COORDINATION_METADATA, - listener - ) + remoteGlobalMetadataManager.readAsync( + COORDINATION_METADATA, + new RemoteCoordinationMetadata( + manifest.getCoordinationMetadata().getUploadedFilename(), + clusterUUID, + blobStoreRepository.getCompressor(), + blobStoreRepository.getNamedXContentRegistry() + ), + listener ); } if (readSettingsMetadata) { - asyncMetadataReadActions.add( - remoteGlobalMetadataManager.getAsyncMetadataReadAction( - new RemotePersistentSettingsMetadata( - manifest.getSettingsMetadata().getUploadedFilename(), - clusterUUID, - blobStoreRepository.getCompressor(), - blobStoreRepository.getNamedXContentRegistry() - ), - SETTING_METADATA, - listener - ) + remoteGlobalMetadataManager.readAsync( + SETTING_METADATA, + new RemotePersistentSettingsMetadata( + manifest.getSettingsMetadata().getUploadedFilename(), + clusterUUID, + blobStoreRepository.getCompressor(), + blobStoreRepository.getNamedXContentRegistry() + ), + listener ); } if (readTransientSettingsMetadata) { - asyncMetadataReadActions.add( - remoteGlobalMetadataManager.getAsyncMetadataReadAction( - new RemoteTransientSettingsMetadata( - manifest.getTransientSettingsMetadata().getUploadedFilename(), - clusterUUID, - blobStoreRepository.getCompressor(), - blobStoreRepository.getNamedXContentRegistry() - ), - TRANSIENT_SETTING_METADATA, - listener - ) + remoteGlobalMetadataManager.readAsync( + TRANSIENT_SETTING_METADATA, + new RemoteTransientSettingsMetadata( + manifest.getTransientSettingsMetadata().getUploadedFilename(), + clusterUUID, + blobStoreRepository.getCompressor(), + blobStoreRepository.getNamedXContentRegistry() + ), + listener ); } if (readTemplatesMetadata) { - asyncMetadataReadActions.add( - remoteGlobalMetadataManager.getAsyncMetadataReadAction( - new RemoteTemplatesMetadata( - manifest.getTemplatesMetadata().getUploadedFilename(), - clusterUUID, - blobStoreRepository.getCompressor(), - blobStoreRepository.getNamedXContentRegistry() - ), - TEMPLATES_METADATA, - listener - ) + remoteGlobalMetadataManager.readAsync( + TEMPLATES_METADATA, + new RemoteTemplatesMetadata( + manifest.getTemplatesMetadata().getUploadedFilename(), + clusterUUID, + blobStoreRepository.getCompressor(), + blobStoreRepository.getNamedXContentRegistry() + ), + listener ); } if (readDiscoveryNodes) { - asyncMetadataReadActions.add( - remoteClusterStateAttributesManager.getAsyncMetadataReadAction( - DISCOVERY_NODES, - new RemoteDiscoveryNodes( - manifest.getDiscoveryNodesMetadata().getUploadedFilename(), - clusterUUID, - blobStoreRepository.getCompressor() - ), - listener - ) + remoteClusterStateAttributesManager.readAsync( + DISCOVERY_NODES, + new RemoteDiscoveryNodes( + manifest.getDiscoveryNodesMetadata().getUploadedFilename(), + clusterUUID, + blobStoreRepository.getCompressor() + ), + listener ); } if (readClusterBlocks) { - asyncMetadataReadActions.add( - remoteClusterStateAttributesManager.getAsyncMetadataReadAction( - CLUSTER_BLOCKS, - new RemoteClusterBlocks( - manifest.getClusterBlocksMetadata().getUploadedFilename(), - clusterUUID, - blobStoreRepository.getCompressor() - ), - listener - ) + remoteClusterStateAttributesManager.readAsync( + CLUSTER_BLOCKS, + new RemoteClusterBlocks( + manifest.getClusterBlocksMetadata().getUploadedFilename(), + clusterUUID, + blobStoreRepository.getCompressor() + ), + listener ); } if (readHashesOfConsistentSettings) { - asyncMetadataReadActions.add( - remoteGlobalMetadataManager.getAsyncMetadataReadAction( - new RemoteHashesOfConsistentSettings( - manifest.getHashesOfConsistentSettings().getUploadedFilename(), - clusterUUID, - blobStoreRepository.getCompressor() - ), - HASHES_OF_CONSISTENT_SETTINGS, - listener - ) + remoteGlobalMetadataManager.readAsync( + HASHES_OF_CONSISTENT_SETTINGS, + new RemoteHashesOfConsistentSettings( + manifest.getHashesOfConsistentSettings().getUploadedFilename(), + clusterUUID, + blobStoreRepository.getCompressor() + ), + listener ); } for (Map.Entry entry : clusterStateCustomToRead.entrySet()) { - asyncMetadataReadActions.add( - remoteClusterStateAttributesManager.getAsyncMetadataReadAction( - // pass component name as cluster-state-custom--, so that we can interpret it later - String.join(CUSTOM_DELIMITER, CLUSTER_STATE_CUSTOM, entry.getKey()), - new RemoteClusterStateCustoms( - entry.getValue().getUploadedFilename(), - entry.getValue().getAttributeName(), - clusterUUID, - blobStoreRepository.getCompressor(), - namedWriteableRegistry - ), - listener - ) + remoteClusterStateAttributesManager.readAsync( + // pass component name as cluster-state-custom--, so that we can interpret it later + String.join(CUSTOM_DELIMITER, CLUSTER_STATE_CUSTOM, entry.getKey()), + new RemoteClusterStateCustoms( + entry.getValue().getUploadedFilename(), + entry.getValue().getAttributeName(), + clusterUUID, + blobStoreRepository.getCompressor(), + namedWriteableRegistry + ), + listener ); } - for (CheckedRunnable asyncMetadataReadAction : asyncMetadataReadActions) { - asyncMetadataReadAction.run(); - } - try { if (latch.await(this.remoteStateReadTimeout.getMillis(), TimeUnit.MILLISECONDS) == false) { RemoteStateTransferException exception = new RemoteStateTransferException( diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManager.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManager.java index 2c5aad99adc0c..5a6f4b7e9f1f1 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManager.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManager.java @@ -8,7 +8,6 @@ package org.opensearch.gateway.remote; -import org.opensearch.action.LatchedActionListener; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.DiffableUtils; import org.opensearch.cluster.DiffableUtils.NonDiffableValueSerializer; @@ -17,9 +16,8 @@ import org.opensearch.cluster.metadata.Metadata.Custom; import org.opensearch.cluster.metadata.Metadata.XContentContext; import org.opensearch.cluster.metadata.TemplatesMetadata; -import org.opensearch.common.CheckedRunnable; import org.opensearch.common.remote.AbstractRemoteWritableBlobEntity; -import org.opensearch.common.remote.RemoteWritableEntityStore; +import org.opensearch.common.remote.AbstractRemoteWritableEntityManager; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; @@ -43,7 +41,6 @@ import java.io.IOException; import java.util.Collections; -import java.util.HashMap; import java.util.Locale; import java.util.Map; import java.util.Map.Entry; @@ -56,7 +53,7 @@ * * @opensearch.internal */ -public class RemoteGlobalMetadataManager { +public class RemoteGlobalMetadataManager extends AbstractRemoteWritableEntityManager { public static final TimeValue GLOBAL_METADATA_UPLOAD_TIMEOUT_DEFAULT = TimeValue.timeValueMillis(20000); @@ -70,7 +67,6 @@ public class RemoteGlobalMetadataManager { public static final int GLOBAL_METADATA_CURRENT_CODEC_VERSION = 1; private volatile TimeValue globalMetadataUploadTimeout; - private Map remoteWritableEntityStores; private final Compressor compressor; private final NamedXContentRegistry namedXContentRegistry; private final NamedWriteableRegistry namedWriteableRegistry; @@ -87,7 +83,6 @@ public class RemoteGlobalMetadataManager { this.compressor = blobStoreRepository.getCompressor(); this.namedXContentRegistry = blobStoreRepository.getNamedXContentRegistry(); this.namedWriteableRegistry = namedWriteableRegistry; - this.remoteWritableEntityStores = new HashMap<>(); this.remoteWritableEntityStores.put( RemoteGlobalMetadata.GLOBAL_METADATA, new RemoteClusterStateBlobStore<>( @@ -161,46 +156,28 @@ public class RemoteGlobalMetadataManager { clusterSettings.addSettingsUpdateConsumer(GLOBAL_METADATA_UPLOAD_TIMEOUT_SETTING, this::setGlobalMetadataUploadTimeout); } - /** - * Allows async upload of Metadata components to remote - */ - CheckedRunnable getAsyncMetadataWriteAction( - AbstractRemoteWritableBlobEntity writeEntity, - LatchedActionListener latchedActionListener - ) { - return (() -> getStore(writeEntity).writeAsync(writeEntity, getActionListener(writeEntity, latchedActionListener))); - } - - private RemoteWritableEntityStore getStore(AbstractRemoteWritableBlobEntity entity) { - RemoteWritableEntityStore remoteStore = remoteWritableEntityStores.get(entity.getType()); - if (remoteStore == null) { - throw new IllegalArgumentException("Unknown entity type [" + entity.getType() + "]"); - } - return remoteStore; - } - - private ActionListener getActionListener( - AbstractRemoteWritableBlobEntity remoteBlobStoreObject, - LatchedActionListener latchedActionListener + @Override + protected ActionListener getWrappedWriteListener( + String component, + AbstractRemoteWritableBlobEntity remoteEntity, + ActionListener listener ) { return ActionListener.wrap( - resp -> latchedActionListener.onResponse(remoteBlobStoreObject.getUploadedMetadata()), - ex -> latchedActionListener.onFailure( - new RemoteStateTransferException("Upload failed for " + remoteBlobStoreObject.getType(), ex) - ) + resp -> listener.onResponse(remoteEntity.getUploadedMetadata()), + ex -> listener.onFailure(new RemoteStateTransferException("Upload failed for " + component, remoteEntity, ex)) ); } - CheckedRunnable getAsyncMetadataReadAction( - AbstractRemoteWritableBlobEntity readEntity, - String componentName, - LatchedActionListener listener + @Override + protected ActionListener getWrappedReadListener( + String component, + AbstractRemoteWritableBlobEntity remoteEntity, + ActionListener listener ) { - ActionListener actionListener = ActionListener.wrap( - response -> listener.onResponse(new RemoteReadResult(response, readEntity.getType(), componentName)), - listener::onFailure + return ActionListener.wrap( + response -> listener.onResponse(new RemoteReadResult(response, remoteEntity.getType(), component)), + ex -> listener.onFailure(new RemoteStateTransferException("Download failed for " + component, remoteEntity, ex)) ); - return () -> getStore(readEntity).readAsync(readEntity, actionListener); } Metadata getGlobalMetadata(String clusterUUID, ClusterMetadataManifest clusterMetadataManifest) { diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteIndexMetadataManager.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteIndexMetadataManager.java index c595f19279354..c30721c8f625c 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteIndexMetadataManager.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteIndexMetadataManager.java @@ -8,10 +8,9 @@ package org.opensearch.gateway.remote; -import org.opensearch.action.LatchedActionListener; import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.common.CheckedRunnable; -import org.opensearch.common.remote.RemoteWritableEntityStore; +import org.opensearch.common.remote.AbstractRemoteWritableBlobEntity; +import org.opensearch.common.remote.AbstractRemoteWritableEntityManager; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; import org.opensearch.common.unit.TimeValue; @@ -33,7 +32,7 @@ * * @opensearch.internal */ -public class RemoteIndexMetadataManager { +public class RemoteIndexMetadataManager extends AbstractRemoteWritableEntityManager { public static final TimeValue INDEX_METADATA_UPLOAD_TIMEOUT_DEFAULT = TimeValue.timeValueMillis(20000); @@ -45,7 +44,6 @@ public class RemoteIndexMetadataManager { Setting.Property.Deprecated ); - private final RemoteWritableEntityStore indexMetadataBlobStore; private final Compressor compressor; private final NamedXContentRegistry namedXContentRegistry; @@ -58,12 +56,15 @@ public RemoteIndexMetadataManager( BlobStoreTransferService blobStoreTransferService, ThreadPool threadpool ) { - this.indexMetadataBlobStore = new RemoteClusterStateBlobStore<>( - blobStoreTransferService, - blobStoreRepository, - clusterName, - threadpool, - ThreadPool.Names.REMOTE_STATE_READ + this.remoteWritableEntityStores.put( + RemoteIndexMetadata.INDEX, + new RemoteClusterStateBlobStore<>( + blobStoreTransferService, + blobStoreRepository, + clusterName, + threadpool, + ThreadPool.Names.REMOTE_STATE_READ + ) ); this.namedXContentRegistry = blobStoreRepository.getNamedXContentRegistry(); this.compressor = blobStoreRepository.getCompressor(); @@ -71,45 +72,6 @@ public RemoteIndexMetadataManager( clusterSettings.addSettingsUpdateConsumer(INDEX_METADATA_UPLOAD_TIMEOUT_SETTING, this::setIndexMetadataUploadTimeout); } - /** - * Allows async Upload of IndexMetadata to remote - * - * @param indexMetadata {@link IndexMetadata} to upload - * @param latchedActionListener listener to respond back on after upload finishes - */ - CheckedRunnable getAsyncIndexMetadataWriteAction( - IndexMetadata indexMetadata, - String clusterUUID, - LatchedActionListener latchedActionListener - ) { - RemoteIndexMetadata remoteIndexMetadata = new RemoteIndexMetadata(indexMetadata, clusterUUID, compressor, namedXContentRegistry); - ActionListener completionListener = ActionListener.wrap( - resp -> latchedActionListener.onResponse(remoteIndexMetadata.getUploadedMetadata()), - ex -> latchedActionListener.onFailure(new RemoteStateTransferException(indexMetadata.getIndex().getName(), ex)) - ); - return () -> indexMetadataBlobStore.writeAsync(remoteIndexMetadata, completionListener); - } - - CheckedRunnable getAsyncIndexMetadataReadAction( - String clusterUUID, - String uploadedFilename, - LatchedActionListener latchedActionListener - ) { - RemoteIndexMetadata remoteIndexMetadata = new RemoteIndexMetadata( - RemoteClusterStateUtils.getFormattedIndexFileName(uploadedFilename), - clusterUUID, - compressor, - namedXContentRegistry - ); - ActionListener actionListener = ActionListener.wrap( - response -> latchedActionListener.onResponse( - new RemoteReadResult(response, RemoteIndexMetadata.INDEX, response.getIndex().getName()) - ), - latchedActionListener::onFailure - ); - return () -> indexMetadataBlobStore.readAsync(remoteIndexMetadata, actionListener); - } - /** * Fetch index metadata from remote cluster state * @@ -124,7 +86,7 @@ IndexMetadata getIndexMetadata(ClusterMetadataManifest.UploadedIndexMetadata upl namedXContentRegistry ); try { - return indexMetadataBlobStore.read(remoteIndexMetadata); + return (IndexMetadata) getStore(remoteIndexMetadata).read(remoteIndexMetadata); } catch (IOException e) { throw new IllegalStateException( String.format(Locale.ROOT, "Error while downloading IndexMetadata - %s", uploadedIndexMetadata.getUploadedFilename()), @@ -141,4 +103,27 @@ private void setIndexMetadataUploadTimeout(TimeValue newIndexMetadataUploadTimeo this.indexMetadataUploadTimeout = newIndexMetadataUploadTimeout; } + @Override + protected ActionListener getWrappedWriteListener( + String component, + AbstractRemoteWritableBlobEntity remoteEntity, + ActionListener listener + ) { + return ActionListener.wrap( + resp -> listener.onResponse(remoteEntity.getUploadedMetadata()), + ex -> listener.onFailure(new RemoteStateTransferException("Upload failed for " + component, remoteEntity, ex)) + ); + } + + @Override + protected ActionListener getWrappedReadListener( + String component, + AbstractRemoteWritableBlobEntity remoteEntity, + ActionListener listener + ) { + return ActionListener.wrap( + response -> listener.onResponse(new RemoteReadResult(response, RemoteIndexMetadata.INDEX, component)), + ex -> listener.onFailure(new RemoteStateTransferException("Download failed for " + component, remoteEntity, ex)) + ); + } } diff --git a/server/src/test/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableServiceTests.java b/server/src/test/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableServiceTests.java index 564c7f7aed304..f66e096e9b548 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableServiceTests.java @@ -543,7 +543,7 @@ public void testGetAsyncIndexRoutingReadAction() throws Exception { "cluster-uuid", uploadedFileName, new LatchedActionListener<>(listener, latch) - ).run(); + ); latch.await(); assertNull(listener.getFailure()); @@ -584,7 +584,7 @@ public void testGetAsyncIndexRoutingWriteAction() throws Exception { clusterState.version(), clusterState.getRoutingTable().indicesRouting().get(indexName), new LatchedActionListener<>(listener, latch) - ).run(); + ); latch.await(); assertNull(listener.getFailure()); assertNotNull(listener.getResult()); diff --git a/server/src/test/java/org/opensearch/common/remote/AbstractRemoteWritableEntityManagerTests.java b/server/src/test/java/org/opensearch/common/remote/AbstractRemoteWritableEntityManagerTests.java new file mode 100644 index 0000000000000..3d10bbf59f6ad --- /dev/null +++ b/server/src/test/java/org/opensearch/common/remote/AbstractRemoteWritableEntityManagerTests.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.common.remote; + +import org.opensearch.core.action.ActionListener; +import org.opensearch.gateway.remote.ClusterMetadataManifest; +import org.opensearch.gateway.remote.model.RemoteReadResult; +import org.opensearch.test.OpenSearchTestCase; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class AbstractRemoteWritableEntityManagerTests extends OpenSearchTestCase { + public void testGetStoreWithKnownEntityType() { + AbstractRemoteWritableEntityManager manager = new ConcreteRemoteWritableEntityManager(); + String knownEntityType = "knownType"; + RemoteWritableEntityStore mockStore = mock(RemoteWritableEntityStore.class); + manager.remoteWritableEntityStores.put(knownEntityType, mockStore); + AbstractRemoteWritableBlobEntity mockEntity = mock(AbstractRemoteWritableBlobEntity.class); + when(mockEntity.getType()).thenReturn(knownEntityType); + + RemoteWritableEntityStore store = manager.getStore(mockEntity); + verify(mockEntity).getType(); + assertEquals(mockStore, store); + } + + public void testGetStoreWithUnknownEntityType() { + AbstractRemoteWritableEntityManager manager = new ConcreteRemoteWritableEntityManager(); + String unknownEntityType = "unknownType"; + AbstractRemoteWritableBlobEntity mockEntity = mock(AbstractRemoteWritableBlobEntity.class); + when(mockEntity.getType()).thenReturn(unknownEntityType); + + assertThrows(IllegalArgumentException.class, () -> manager.getStore(mockEntity)); + verify(mockEntity, times(2)).getType(); + } + + private static class ConcreteRemoteWritableEntityManager extends AbstractRemoteWritableEntityManager { + @Override + protected ActionListener getWrappedWriteListener( + String component, + AbstractRemoteWritableBlobEntity remoteEntity, + ActionListener listener + ) { + return null; + } + + @Override + protected ActionListener getWrappedReadListener( + String component, + AbstractRemoteWritableBlobEntity remoteEntity, + ActionListener listener + ) { + return null; + } + } +} diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManagerTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManagerTests.java index 3f2edd1a6c5a5..4ef459e6657a1 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManagerTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateAttributesManagerTests.java @@ -107,7 +107,7 @@ public void tearDown() throws Exception { threadPool.shutdown(); } - public void testGetAsyncMetadataWriteAction_DiscoveryNodes() throws IOException, InterruptedException { + public void testGetAsyncWriteRunnable_DiscoveryNodes() throws IOException, InterruptedException { DiscoveryNodes discoveryNodes = getDiscoveryNodes(); RemoteDiscoveryNodes remoteDiscoveryNodes = new RemoteDiscoveryNodes(discoveryNodes, VERSION, CLUSTER_UUID, compressor); doAnswer(invocationOnMock -> { @@ -117,11 +117,7 @@ public void testGetAsyncMetadataWriteAction_DiscoveryNodes() throws IOException, .uploadBlob(any(InputStream.class), anyIterable(), anyString(), eq(URGENT), any(ActionListener.class)); final CountDownLatch latch = new CountDownLatch(1); final TestCapturingListener listener = new TestCapturingListener<>(); - remoteClusterStateAttributesManager.getAsyncMetadataWriteAction( - DISCOVERY_NODES, - remoteDiscoveryNodes, - new LatchedActionListener<>(listener, latch) - ).run(); + remoteClusterStateAttributesManager.writeAsync(DISCOVERY_NODES, remoteDiscoveryNodes, new LatchedActionListener<>(listener, latch)); latch.await(); assertNull(listener.getFailure()); assertNotNull(listener.getResult()); @@ -140,7 +136,7 @@ public void testGetAsyncMetadataWriteAction_DiscoveryNodes() throws IOException, assertEquals(CLUSTER_STATE_ATTRIBUTES_CURRENT_CODEC_VERSION, Integer.parseInt(splitFileName[3])); } - public void testGetAsyncMetadataReadAction_DiscoveryNodes() throws IOException, InterruptedException { + public void testGetAsyncReadRunnable_DiscoveryNodes() throws IOException, InterruptedException { DiscoveryNodes discoveryNodes = getDiscoveryNodes(); String fileName = randomAlphaOfLength(10); when(blobStoreTransferService.downloadBlob(anyIterable(), anyString())).thenReturn( @@ -149,11 +145,7 @@ public void testGetAsyncMetadataReadAction_DiscoveryNodes() throws IOException, RemoteDiscoveryNodes remoteObjForDownload = new RemoteDiscoveryNodes(fileName, "cluster-uuid", compressor); CountDownLatch latch = new CountDownLatch(1); TestCapturingListener listener = new TestCapturingListener<>(); - remoteClusterStateAttributesManager.getAsyncMetadataReadAction( - DISCOVERY_NODES, - remoteObjForDownload, - new LatchedActionListener<>(listener, latch) - ).run(); + remoteClusterStateAttributesManager.readAsync(DISCOVERY_NODES, remoteObjForDownload, new LatchedActionListener<>(listener, latch)); latch.await(); assertNull(listener.getFailure()); assertNotNull(listener.getResult()); @@ -165,7 +157,7 @@ public void testGetAsyncMetadataReadAction_DiscoveryNodes() throws IOException, assertEquals(discoveryNodes.getClusterManagerNodeId(), readDiscoveryNodes.getClusterManagerNodeId()); } - public void testGetAsyncMetadataWriteAction_ClusterBlocks() throws IOException, InterruptedException { + public void testGetAsyncWriteRunnable_ClusterBlocks() throws IOException, InterruptedException { ClusterBlocks clusterBlocks = randomClusterBlocks(); RemoteClusterBlocks remoteClusterBlocks = new RemoteClusterBlocks(clusterBlocks, VERSION, CLUSTER_UUID, compressor); doAnswer(invocationOnMock -> { @@ -175,11 +167,7 @@ public void testGetAsyncMetadataWriteAction_ClusterBlocks() throws IOException, .uploadBlob(any(InputStream.class), anyIterable(), anyString(), eq(URGENT), any(ActionListener.class)); final CountDownLatch latch = new CountDownLatch(1); final TestCapturingListener listener = new TestCapturingListener<>(); - remoteClusterStateAttributesManager.getAsyncMetadataWriteAction( - CLUSTER_BLOCKS, - remoteClusterBlocks, - new LatchedActionListener<>(listener, latch) - ).run(); + remoteClusterStateAttributesManager.writeAsync(CLUSTER_BLOCKS, remoteClusterBlocks, new LatchedActionListener<>(listener, latch)); latch.await(); assertNull(listener.getFailure()); assertNotNull(listener.getResult()); @@ -198,7 +186,7 @@ public void testGetAsyncMetadataWriteAction_ClusterBlocks() throws IOException, assertEquals(CLUSTER_STATE_ATTRIBUTES_CURRENT_CODEC_VERSION, Integer.parseInt(splitFileName[3])); } - public void testGetAsyncMetadataReadAction_ClusterBlocks() throws IOException, InterruptedException { + public void testGetAsyncReadRunnable_ClusterBlocks() throws IOException, InterruptedException { ClusterBlocks clusterBlocks = randomClusterBlocks(); String fileName = randomAlphaOfLength(10); when(blobStoreTransferService.downloadBlob(anyIterable(), anyString())).thenReturn( @@ -208,11 +196,7 @@ public void testGetAsyncMetadataReadAction_ClusterBlocks() throws IOException, I CountDownLatch latch = new CountDownLatch(1); TestCapturingListener listener = new TestCapturingListener<>(); - remoteClusterStateAttributesManager.getAsyncMetadataReadAction( - CLUSTER_BLOCKS, - remoteClusterBlocks, - new LatchedActionListener<>(listener, latch) - ).run(); + remoteClusterStateAttributesManager.readAsync(CLUSTER_BLOCKS, remoteClusterBlocks, new LatchedActionListener<>(listener, latch)); latch.await(); assertNull(listener.getFailure()); assertNotNull(listener.getResult()); @@ -226,7 +210,7 @@ public void testGetAsyncMetadataReadAction_ClusterBlocks() throws IOException, I } } - public void testGetAsyncMetadataWriteAction_Custom() throws IOException, InterruptedException { + public void testGetAsyncWriteRunnable_Custom() throws IOException, InterruptedException { Custom custom = getClusterStateCustom(); RemoteClusterStateCustoms remoteClusterStateCustoms = new RemoteClusterStateCustoms( custom, @@ -243,11 +227,11 @@ public void testGetAsyncMetadataWriteAction_Custom() throws IOException, Interru .uploadBlob(any(InputStream.class), anyIterable(), anyString(), eq(URGENT), any(ActionListener.class)); final TestCapturingListener listener = new TestCapturingListener<>(); final CountDownLatch latch = new CountDownLatch(1); - remoteClusterStateAttributesManager.getAsyncMetadataWriteAction( + remoteClusterStateAttributesManager.writeAsync( CLUSTER_STATE_CUSTOM, remoteClusterStateCustoms, new LatchedActionListener<>(listener, latch) - ).run(); + ); latch.await(); assertNull(listener.getFailure()); assertNotNull(listener.getResult()); @@ -266,7 +250,7 @@ public void testGetAsyncMetadataWriteAction_Custom() throws IOException, Interru assertEquals(CLUSTER_STATE_ATTRIBUTES_CURRENT_CODEC_VERSION, Integer.parseInt(splitFileName[3])); } - public void testGetAsyncMetadataReadAction_Custom() throws IOException, InterruptedException { + public void testGetAsyncReadRunnable_Custom() throws IOException, InterruptedException { Custom custom = getClusterStateCustom(); String fileName = randomAlphaOfLength(10); RemoteClusterStateCustoms remoteClusterStateCustoms = new RemoteClusterStateCustoms( @@ -281,11 +265,11 @@ public void testGetAsyncMetadataReadAction_Custom() throws IOException, Interrup ); TestCapturingListener capturingListener = new TestCapturingListener<>(); final CountDownLatch latch = new CountDownLatch(1); - remoteClusterStateAttributesManager.getAsyncMetadataReadAction( + remoteClusterStateAttributesManager.readAsync( CLUSTER_STATE_CUSTOM, remoteClusterStateCustoms, new LatchedActionListener<>(capturingListener, latch) - ).run(); + ); latch.await(); assertNull(capturingListener.getFailure()); assertNotNull(capturingListener.getResult()); @@ -294,7 +278,7 @@ public void testGetAsyncMetadataReadAction_Custom() throws IOException, Interrup assertEquals(CLUSTER_STATE_CUSTOM, capturingListener.getResult().getComponentName()); } - public void testGetAsyncMetadataWriteAction_Exception() throws IOException, InterruptedException { + public void testGetAsyncWriteRunnable_Exception() throws IOException, InterruptedException { DiscoveryNodes discoveryNodes = getDiscoveryNodes(); RemoteDiscoveryNodes remoteDiscoveryNodes = new RemoteDiscoveryNodes(discoveryNodes, VERSION, CLUSTER_UUID, compressor); @@ -307,32 +291,33 @@ public void testGetAsyncMetadataWriteAction_Exception() throws IOException, Inte TestCapturingListener capturingListener = new TestCapturingListener<>(); final CountDownLatch latch = new CountDownLatch(1); - remoteClusterStateAttributesManager.getAsyncMetadataWriteAction( + remoteClusterStateAttributesManager.writeAsync( DISCOVERY_NODES, remoteDiscoveryNodes, new LatchedActionListener<>(capturingListener, latch) - ).run(); + ); latch.await(); assertNull(capturingListener.getResult()); assertTrue(capturingListener.getFailure() instanceof RemoteStateTransferException); assertEquals(ioException, capturingListener.getFailure().getCause()); } - public void testGetAsyncMetadataReadAction_Exception() throws IOException, InterruptedException { + public void testGetAsyncReadRunnable_Exception() throws IOException, InterruptedException { String fileName = randomAlphaOfLength(10); RemoteDiscoveryNodes remoteDiscoveryNodes = new RemoteDiscoveryNodes(fileName, CLUSTER_UUID, compressor); Exception ioException = new IOException("mock test exception"); when(blobStoreTransferService.downloadBlob(anyIterable(), anyString())).thenThrow(ioException); CountDownLatch latch = new CountDownLatch(1); TestCapturingListener capturingListener = new TestCapturingListener<>(); - remoteClusterStateAttributesManager.getAsyncMetadataReadAction( + remoteClusterStateAttributesManager.readAsync( DISCOVERY_NODES, remoteDiscoveryNodes, new LatchedActionListener<>(capturingListener, latch) - ).run(); + ); latch.await(); assertNull(capturingListener.getResult()); - assertEquals(ioException, capturingListener.getFailure()); + assertEquals(ioException, capturingListener.getFailure().getCause()); + assertTrue(capturingListener.getFailure() instanceof RemoteStateTransferException); } public void testGetUpdatedCustoms() { diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java index ebd3488d06007..6c764585c48e7 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java @@ -121,6 +121,7 @@ import static org.opensearch.gateway.remote.RemoteClusterStateUtils.DELIMITER; import static org.opensearch.gateway.remote.RemoteClusterStateUtils.FORMAT_PARAMS; import static org.opensearch.gateway.remote.RemoteClusterStateUtils.getFormattedIndexFileName; +import static org.opensearch.gateway.remote.RemoteGlobalMetadataManager.GLOBAL_METADATA_UPLOAD_TIMEOUT_DEFAULT; import static org.opensearch.gateway.remote.model.RemoteClusterBlocks.CLUSTER_BLOCKS_FORMAT; import static org.opensearch.gateway.remote.model.RemoteClusterBlocksTests.randomClusterBlocks; import static org.opensearch.gateway.remote.model.RemoteClusterMetadataManifest.MANIFEST_CURRENT_CODEC_VERSION; @@ -590,6 +591,55 @@ public void testFailWriteIncrementalMetadataWhenTermChanged() { ); } + public void testWriteMetadataInParallelIncompleteUpload() throws IOException { + final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); + final RemoteClusterStateService rcssSpy = Mockito.spy(remoteClusterStateService); + rcssSpy.start(); + RemoteIndexMetadataManager mockedIndexManager = mock(RemoteIndexMetadataManager.class); + RemoteGlobalMetadataManager mockedGlobalMetadataManager = mock(RemoteGlobalMetadataManager.class); + RemoteClusterStateAttributesManager mockedClusterStateAttributeManager = mock(RemoteClusterStateAttributesManager.class); + ClusterMetadataManifest.UploadedMetadata mockedUploadedMetadata = mock(ClusterMetadataManifest.UploadedMetadata.class); + rcssSpy.setRemoteIndexMetadataManager(mockedIndexManager); + rcssSpy.setRemoteGlobalMetadataManager(mockedGlobalMetadataManager); + rcssSpy.setRemoteClusterStateAttributesManager(mockedClusterStateAttributeManager); + ArgumentCaptor listenerArgumentCaptor = ArgumentCaptor.forClass(LatchedActionListener.class); + + when(mockedGlobalMetadataManager.getGlobalMetadataUploadTimeout()).thenReturn(GLOBAL_METADATA_UPLOAD_TIMEOUT_DEFAULT); + when(mockedUploadedMetadata.getComponent()).thenReturn("test-component"); + doAnswer(invocation -> { + listenerArgumentCaptor.getValue().onResponse(mockedUploadedMetadata); + return null; + }).when(mockedIndexManager).writeAsync(any(), any(), listenerArgumentCaptor.capture()); + doAnswer(invocation -> { + listenerArgumentCaptor.getValue().onResponse(mockedUploadedMetadata); + return null; + }).when(mockedGlobalMetadataManager).writeAsync(anyString(), any(), listenerArgumentCaptor.capture()); + doAnswer(invocation -> { + listenerArgumentCaptor.getValue().onResponse(mockedUploadedMetadata); + return null; + }).when(mockedClusterStateAttributeManager).writeAsync(any(), any(), listenerArgumentCaptor.capture()); + + RemoteStateTransferException exception = expectThrows( + RemoteStateTransferException.class, + () -> rcssSpy.writeMetadataInParallel( + clusterState, + new ArrayList<>(clusterState.getMetadata().indices().values()), + emptyMap(), + clusterState.getMetadata().customs(), + true, + true, + true, + true, + true, + true, + clusterState.getCustoms(), + true, + emptyList() + ) + ); + assertTrue(exception.getMessage().startsWith("Some metadata components were not uploaded successfully")); + } + public void testWriteIncrementalMetadataSuccess() throws IOException { final ClusterState clusterState = generateClusterStateWithOneIndex().nodes(nodesWithLocalNodeClusterManager()).build(); mockBlobStoreObjects(); @@ -781,14 +831,18 @@ public void testGetClusterStateForManifest_IncludeEphemeral() throws IOException ArgumentCaptor> listenerArgumentCaptor = ArgumentCaptor.forClass( LatchedActionListener.class ); - when(mockedIndexManager.getAsyncIndexMetadataReadAction(any(), anyString(), listenerArgumentCaptor.capture())).thenReturn( - () -> listenerArgumentCaptor.getValue().onResponse(mockedResult) - ); - when(mockedGlobalMetadataManager.getAsyncMetadataReadAction(any(), anyString(), listenerArgumentCaptor.capture())).thenReturn( - () -> listenerArgumentCaptor.getValue().onResponse(mockedResult) - ); - when(mockedClusterStateAttributeManager.getAsyncMetadataReadAction(anyString(), any(), listenerArgumentCaptor.capture())) - .thenReturn(() -> listenerArgumentCaptor.getValue().onResponse(mockedResult)); + doAnswer(invocation -> { + listenerArgumentCaptor.getValue().onResponse(mockedResult); + return null; + }).when(mockedIndexManager).readAsync(any(), any(), listenerArgumentCaptor.capture()); + doAnswer(invocation -> { + listenerArgumentCaptor.getValue().onResponse(mockedResult); + return null; + }).when(mockedGlobalMetadataManager).readAsync(any(), any(), listenerArgumentCaptor.capture()); + doAnswer(invocation -> { + listenerArgumentCaptor.getValue().onResponse(mockedResult); + return null; + }).when(mockedClusterStateAttributeManager).readAsync(anyString(), any(), listenerArgumentCaptor.capture()); when(mockedResult.getComponent()).thenReturn(COORDINATION_METADATA); RemoteClusterStateService mockService = spy(remoteClusterStateService); mockService.getClusterStateForManifest(ClusterName.DEFAULT.value(), manifest, NODE_ID, true); @@ -823,14 +877,18 @@ public void testGetClusterStateForManifest_ExcludeEphemeral() throws IOException ArgumentCaptor> listenerArgumentCaptor = ArgumentCaptor.forClass( LatchedActionListener.class ); - when(mockedIndexManager.getAsyncIndexMetadataReadAction(any(), anyString(), listenerArgumentCaptor.capture())).thenReturn( - () -> listenerArgumentCaptor.getValue().onResponse(mockedResult) - ); - when(mockedGlobalMetadataManager.getAsyncMetadataReadAction(any(), anyString(), listenerArgumentCaptor.capture())).thenReturn( - () -> listenerArgumentCaptor.getValue().onResponse(mockedResult) - ); - when(mockedClusterStateAttributeManager.getAsyncMetadataReadAction(anyString(), any(), listenerArgumentCaptor.capture())) - .thenReturn(() -> listenerArgumentCaptor.getValue().onResponse(mockedResult)); + doAnswer(invocation -> { + listenerArgumentCaptor.getValue().onResponse(mockedResult); + return null; + }).when(mockedIndexManager).readAsync(anyString(), any(), listenerArgumentCaptor.capture()); + doAnswer(invocation -> { + listenerArgumentCaptor.getValue().onResponse(mockedResult); + return null; + }).when(mockedGlobalMetadataManager).readAsync(anyString(), any(), listenerArgumentCaptor.capture()); + doAnswer(invocation -> { + listenerArgumentCaptor.getValue().onResponse(mockedResult); + return null; + }).when(mockedClusterStateAttributeManager).readAsync(anyString(), any(), listenerArgumentCaptor.capture()); when(mockedResult.getComponent()).thenReturn(COORDINATION_METADATA); remoteClusterStateService.setRemoteIndexMetadataManager(mockedIndexManager); remoteClusterStateService.setRemoteGlobalMetadataManager(mockedGlobalMetadataManager); @@ -877,9 +935,10 @@ public void testGetClusterStateFromManifest_CodecV1() throws IOException { ArgumentCaptor> listenerArgumentCaptor = ArgumentCaptor.forClass( LatchedActionListener.class ); - when(mockedIndexManager.getAsyncIndexMetadataReadAction(any(), anyString(), listenerArgumentCaptor.capture())).thenReturn( - () -> listenerArgumentCaptor.getValue().onResponse(new RemoteReadResult(indexMetadata, INDEX, INDEX)) - ); + doAnswer(invocation -> { + listenerArgumentCaptor.getValue().onResponse(new RemoteReadResult(indexMetadata, INDEX, INDEX)); + return null; + }).when(mockedIndexManager).readAsync(anyString(), any(), listenerArgumentCaptor.capture()); when(mockedGlobalMetadataManager.getGlobalMetadata(anyString(), eq(manifest))).thenReturn(Metadata.EMPTY_METADATA); RemoteClusterStateService spiedService = spy(remoteClusterStateService); spiedService.getClusterStateForManifest(ClusterName.DEFAULT.value(), manifest, NODE_ID, true); @@ -1258,7 +1317,7 @@ public void testReadClusterStateInParallel_ExceptionDuringRead() throws IOExcept ); assertEquals("Exception during reading cluster state from remote", exception.getMessage()); assertTrue(exception.getSuppressed().length > 0); - assertEquals(mockException, exception.getSuppressed()[0]); + assertEquals(mockException, exception.getSuppressed()[0].getCause()); } public void testReadClusterStateInParallel_UnexpectedResult() throws IOException { @@ -1322,19 +1381,20 @@ public void testReadClusterStateInParallel_UnexpectedResult() throws IOException RemoteIndexMetadataManager mockIndexMetadataManager = mock(RemoteIndexMetadataManager.class); CheckedRunnable mockRunnable = mock(CheckedRunnable.class); ArgumentCaptor> latchCapture = ArgumentCaptor.forClass(LatchedActionListener.class); - when(mockIndexMetadataManager.getAsyncIndexMetadataReadAction(anyString(), anyString(), latchCapture.capture())).thenReturn( - mockRunnable - ); + doAnswer(invocation -> { + latchCapture.getValue().onResponse(mockResult); + return null; + }).when(mockIndexMetadataManager).readAsync(anyString(), any(), latchCapture.capture()); RemoteGlobalMetadataManager mockGlobalMetadataManager = mock(RemoteGlobalMetadataManager.class); - when(mockGlobalMetadataManager.getAsyncMetadataReadAction(any(), anyString(), latchCapture.capture())).thenReturn(mockRunnable); + doAnswer(invocation -> { + latchCapture.getValue().onResponse(mockResult); + return null; + }).when(mockGlobalMetadataManager).readAsync(any(), any(), latchCapture.capture()); RemoteClusterStateAttributesManager mockClusterStateAttributeManager = mock(RemoteClusterStateAttributesManager.class); - when(mockClusterStateAttributeManager.getAsyncMetadataReadAction(anyString(), any(), latchCapture.capture())).thenReturn( - mockRunnable - ); - doAnswer(invocationOnMock -> { + doAnswer(invocation -> { latchCapture.getValue().onResponse(mockResult); return null; - }).when(mockRunnable).run(); + }).when(mockClusterStateAttributeManager).readAsync(anyString(), any(), latchCapture.capture()); when(mockResult.getComponent()).thenReturn("mock-result"); remoteClusterStateService.start(); remoteClusterStateService.setRemoteIndexMetadataManager(mockIndexMetadataManager); @@ -1363,56 +1423,56 @@ public void testReadClusterStateInParallel_UnexpectedResult() throws IOException ); assertEquals("Unknown component: mock-result", exception.getMessage()); newIndicesToRead.forEach( - uploadedIndexMetadata -> verify(mockIndexMetadataManager, times(1)).getAsyncIndexMetadataReadAction( - eq(previousClusterState.getMetadata().clusterUUID()), - eq(uploadedIndexMetadata.getUploadedFilename()), + uploadedIndexMetadata -> verify(mockIndexMetadataManager, times(1)).readAsync( + eq("test-index-1"), + argThat(new BlobNameMatcher(uploadedIndexMetadata.getUploadedFilename())), any() ) ); - verify(mockGlobalMetadataManager, times(1)).getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(COORDINATION_METADATA_FILENAME)), + verify(mockGlobalMetadataManager, times(1)).readAsync( eq(COORDINATION_METADATA), + argThat(new BlobNameMatcher(COORDINATION_METADATA_FILENAME)), any() ); - verify(mockGlobalMetadataManager, times(1)).getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(PERSISTENT_SETTINGS_FILENAME)), + verify(mockGlobalMetadataManager, times(1)).readAsync( eq(SETTING_METADATA), + argThat(new BlobNameMatcher(PERSISTENT_SETTINGS_FILENAME)), any() ); - verify(mockGlobalMetadataManager, times(1)).getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(TRANSIENT_SETTINGS_FILENAME)), + verify(mockGlobalMetadataManager, times(1)).readAsync( eq(TRANSIENT_SETTING_METADATA), + argThat(new BlobNameMatcher(TRANSIENT_SETTINGS_FILENAME)), any() ); - verify(mockGlobalMetadataManager, times(1)).getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(TEMPLATES_METADATA_FILENAME)), + verify(mockGlobalMetadataManager, times(1)).readAsync( eq(TEMPLATES_METADATA), + argThat(new BlobNameMatcher(TEMPLATES_METADATA_FILENAME)), any() ); - verify(mockGlobalMetadataManager, times(1)).getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(HASHES_OF_CONSISTENT_SETTINGS_FILENAME)), + verify(mockGlobalMetadataManager, times(1)).readAsync( eq(HASHES_OF_CONSISTENT_SETTINGS), + argThat(new BlobNameMatcher(HASHES_OF_CONSISTENT_SETTINGS_FILENAME)), any() ); newCustomMetadataMap.keySet().forEach(uploadedCustomMetadataKey -> { - verify(mockGlobalMetadataManager, times(1)).getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(newCustomMetadataMap.get(uploadedCustomMetadataKey).getUploadedFilename())), + verify(mockGlobalMetadataManager, times(1)).readAsync( eq(uploadedCustomMetadataKey), + argThat(new BlobNameMatcher(newCustomMetadataMap.get(uploadedCustomMetadataKey).getUploadedFilename())), any() ); }); - verify(mockClusterStateAttributeManager, times(1)).getAsyncMetadataReadAction( + verify(mockClusterStateAttributeManager, times(1)).readAsync( eq(DISCOVERY_NODES), argThat(new BlobNameMatcher(DISCOVERY_NODES_FILENAME)), any() ); - verify(mockClusterStateAttributeManager, times(1)).getAsyncMetadataReadAction( + verify(mockClusterStateAttributeManager, times(1)).readAsync( eq(CLUSTER_BLOCKS), argThat(new BlobNameMatcher(CLUSTER_BLOCKS_FILENAME)), any() ); newClusterStateCustoms.keySet().forEach(uploadedClusterStateCustomMetadataKey -> { - verify(mockClusterStateAttributeManager, times(1)).getAsyncMetadataReadAction( + verify(mockClusterStateAttributeManager, times(1)).readAsync( eq(String.join(CUSTOM_DELIMITER, CLUSTER_STATE_CUSTOM, uploadedClusterStateCustomMetadataKey)), argThat(new BlobNameMatcher(newClusterStateCustoms.get(uploadedClusterStateCustomMetadataKey).getUploadedFilename())), any() @@ -1495,131 +1555,81 @@ public void testReadClusterStateInParallel_Success() throws IOException { RemoteGlobalMetadataManager mockedGlobalMetadataManager = mock(RemoteGlobalMetadataManager.class); RemoteClusterStateAttributesManager mockedClusterStateAttributeManager = mock(RemoteClusterStateAttributesManager.class); - when( - mockedIndexManager.getAsyncIndexMetadataReadAction( - eq(manifest.getClusterUUID()), - eq(indexFilename), - any(LatchedActionListener.class) - ) - ).thenAnswer(invocationOnMock -> { + doAnswer(invocationOnMock -> { LatchedActionListener latchedActionListener = invocationOnMock.getArgument(2, LatchedActionListener.class); - return (CheckedRunnable) () -> latchedActionListener.onResponse( - new RemoteReadResult(newIndexMetadata, INDEX, "test-index-1") - ); - }); - when( - mockedGlobalMetadataManager.getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(customMetadataFilename)), - eq("custom_md_3"), - any() - ) - ).thenAnswer(invocationOnMock -> { + latchedActionListener.onResponse(new RemoteReadResult(newIndexMetadata, INDEX, "test-index-1")); + return null; + }).when(mockedIndexManager) + .readAsync(eq("test-index-1"), argThat(new BlobNameMatcher(indexFilename)), any(LatchedActionListener.class)); + doAnswer(invocationOnMock -> { LatchedActionListener latchedActionListener = invocationOnMock.getArgument(2, LatchedActionListener.class); - return (CheckedRunnable) () -> latchedActionListener.onResponse( - new RemoteReadResult(customMetadata3, CUSTOM_METADATA, "custom_md_3") - ); - }); - when( - mockedGlobalMetadataManager.getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(COORDINATION_METADATA_FILENAME)), - eq(COORDINATION_METADATA), - any() - ) - ).thenAnswer(invocationOnMock -> { + latchedActionListener.onResponse(new RemoteReadResult(customMetadata3, CUSTOM_METADATA, "custom_md_3")); + return null; + }).when(mockedGlobalMetadataManager).readAsync(eq("custom_md_3"), argThat(new BlobNameMatcher(customMetadataFilename)), any()); + doAnswer(invocationOnMock -> { LatchedActionListener latchedActionListener = invocationOnMock.getArgument(2, LatchedActionListener.class); - return (CheckedRunnable) () -> latchedActionListener.onResponse( + latchedActionListener.onResponse( new RemoteReadResult(updatedCoordinationMetadata, COORDINATION_METADATA, COORDINATION_METADATA) ); - }); - when( - mockedGlobalMetadataManager.getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(PERSISTENT_SETTINGS_FILENAME)), - eq(SETTING_METADATA), - any() - ) - ).thenAnswer(invocationOnMock -> { + return null; + }).when(mockedGlobalMetadataManager) + .readAsync(eq(COORDINATION_METADATA), argThat(new BlobNameMatcher(COORDINATION_METADATA_FILENAME)), any()); + doAnswer(invocationOnMock -> { LatchedActionListener latchedActionListener = invocationOnMock.getArgument(2, LatchedActionListener.class); - - return (CheckedRunnable) () -> latchedActionListener.onResponse( - new RemoteReadResult(updatedPersistentSettings, SETTING_METADATA, SETTING_METADATA) - ); - }); - when( - mockedGlobalMetadataManager.getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(TRANSIENT_SETTINGS_FILENAME)), - eq(TRANSIENT_SETTING_METADATA), - any() - ) - ).thenAnswer(invocationOnMock -> { + latchedActionListener.onResponse(new RemoteReadResult(updatedPersistentSettings, SETTING_METADATA, SETTING_METADATA)); + return null; + }).when(mockedGlobalMetadataManager) + .readAsync(eq(SETTING_METADATA), argThat(new BlobNameMatcher(PERSISTENT_SETTINGS_FILENAME)), any()); + doAnswer(invocationOnMock -> { LatchedActionListener latchedActionListener = invocationOnMock.getArgument(2, LatchedActionListener.class); - return (CheckedRunnable) () -> latchedActionListener.onResponse( + latchedActionListener.onResponse( new RemoteReadResult(updatedTransientSettings, TRANSIENT_SETTING_METADATA, TRANSIENT_SETTING_METADATA) ); - }); - when( - mockedGlobalMetadataManager.getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(TEMPLATES_METADATA_FILENAME)), - eq(TEMPLATES_METADATA), - any() - ) - ).thenAnswer(invocationOnMock -> { + return null; + }).when(mockedGlobalMetadataManager) + .readAsync(eq(TRANSIENT_SETTING_METADATA), argThat(new BlobNameMatcher(TRANSIENT_SETTINGS_FILENAME)), any()); + doAnswer(invocationOnMock -> { LatchedActionListener latchedActionListener = invocationOnMock.getArgument(2, LatchedActionListener.class); - return (CheckedRunnable) () -> latchedActionListener.onResponse( - new RemoteReadResult(updatedTemplateMetadata, TEMPLATES_METADATA, TEMPLATES_METADATA) - ); - }); - when( - mockedGlobalMetadataManager.getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(HASHES_OF_CONSISTENT_SETTINGS_FILENAME)), - eq(HASHES_OF_CONSISTENT_SETTINGS), - any() - ) - ).thenAnswer(invocationOnMock -> { + latchedActionListener.onResponse(new RemoteReadResult(updatedTemplateMetadata, TEMPLATES_METADATA, TEMPLATES_METADATA)); + return null; + }).when(mockedGlobalMetadataManager) + .readAsync(eq(TEMPLATES_METADATA), argThat(new BlobNameMatcher(TEMPLATES_METADATA_FILENAME)), any()); + doAnswer(invocationOnMock -> { LatchedActionListener latchedActionListener = invocationOnMock.getArgument(2, LatchedActionListener.class); - return (CheckedRunnable) () -> latchedActionListener.onResponse( + latchedActionListener.onResponse( new RemoteReadResult(updatedHashesOfConsistentSettings, HASHES_OF_CONSISTENT_SETTINGS, HASHES_OF_CONSISTENT_SETTINGS) ); - }); - when( - mockedClusterStateAttributeManager.getAsyncMetadataReadAction( - eq(DISCOVERY_NODES), - argThat(new BlobNameMatcher(DISCOVERY_NODES_FILENAME)), - any() - ) - ).thenAnswer(invocationOnMock -> { + return null; + }).when(mockedGlobalMetadataManager) + .readAsync(eq(HASHES_OF_CONSISTENT_SETTINGS), argThat(new BlobNameMatcher(HASHES_OF_CONSISTENT_SETTINGS_FILENAME)), any()); + doAnswer(invocationOnMock -> { LatchedActionListener latchedActionListener = invocationOnMock.getArgument(2, LatchedActionListener.class); - return (CheckedRunnable) () -> latchedActionListener.onResponse( - new RemoteReadResult(updatedDiscoveryNodes, CLUSTER_STATE_ATTRIBUTE, DISCOVERY_NODES) - ); - }); - when( - mockedClusterStateAttributeManager.getAsyncMetadataReadAction( - eq(CLUSTER_BLOCKS), - argThat(new BlobNameMatcher(CLUSTER_BLOCKS_FILENAME)), - any() - ) - ).thenAnswer(invocationOnMock -> { + latchedActionListener.onResponse(new RemoteReadResult(updatedDiscoveryNodes, CLUSTER_STATE_ATTRIBUTE, DISCOVERY_NODES)); + return null; + }).when(mockedClusterStateAttributeManager) + .readAsync(eq(DISCOVERY_NODES), argThat(new BlobNameMatcher(DISCOVERY_NODES_FILENAME)), any()); + doAnswer(invocationOnMock -> { LatchedActionListener latchedActionListener = invocationOnMock.getArgument(2, LatchedActionListener.class); - return (CheckedRunnable) () -> latchedActionListener.onResponse( - new RemoteReadResult(updatedClusterBlocks, CLUSTER_STATE_ATTRIBUTE, CLUSTER_BLOCKS) - ); - }); - when( - mockedClusterStateAttributeManager.getAsyncMetadataReadAction( - eq(String.join(CUSTOM_DELIMITER, CLUSTER_STATE_CUSTOM, updatedClusterStateCustom3.getWriteableName())), - argThat(new BlobNameMatcher(clusterStateCustomFilename)), - any() - ) - ).thenAnswer(invocationOnMock -> { + latchedActionListener.onResponse(new RemoteReadResult(updatedClusterBlocks, CLUSTER_STATE_ATTRIBUTE, CLUSTER_BLOCKS)); + return null; + }).when(mockedClusterStateAttributeManager) + .readAsync(eq(CLUSTER_BLOCKS), argThat(new BlobNameMatcher(CLUSTER_BLOCKS_FILENAME)), any()); + doAnswer(invocationOnMock -> { LatchedActionListener latchedActionListener = invocationOnMock.getArgument(2, LatchedActionListener.class); - return (CheckedRunnable) () -> latchedActionListener.onResponse( + latchedActionListener.onResponse( new RemoteReadResult( updatedClusterStateCustom3, CLUSTER_STATE_ATTRIBUTE, String.join(CUSTOM_DELIMITER, CLUSTER_STATE_CUSTOM, updatedClusterStateCustom3.getWriteableName()) ) ); - }); + return null; + }).when(mockedClusterStateAttributeManager) + .readAsync( + eq(String.join(CUSTOM_DELIMITER, CLUSTER_STATE_CUSTOM, updatedClusterStateCustom3.getWriteableName())), + argThat(new BlobNameMatcher(clusterStateCustomFilename)), + any() + ); remoteClusterStateService.start(); remoteClusterStateService.setRemoteIndexMetadataManager(mockedIndexManager); @@ -1665,56 +1675,56 @@ public void testReadClusterStateInParallel_Success() throws IOException { uploadedClusterStateCustomMap.keySet().forEach(key -> assertTrue(updatedClusterState.customs().containsKey(key))); assertEquals(updatedClusterStateCustom3, updatedClusterState.custom("custom_3")); newIndicesToRead.forEach( - uploadedIndexMetadata -> verify(mockedIndexManager, times(1)).getAsyncIndexMetadataReadAction( - eq(previousClusterState.getMetadata().clusterUUID()), - eq(uploadedIndexMetadata.getUploadedFilename()), + uploadedIndexMetadata -> verify(mockedIndexManager, times(1)).readAsync( + eq("test-index-1"), + argThat(new BlobNameMatcher(uploadedIndexMetadata.getUploadedFilename())), any() ) ); - verify(mockedGlobalMetadataManager, times(1)).getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(COORDINATION_METADATA_FILENAME)), + verify(mockedGlobalMetadataManager, times(1)).readAsync( eq(COORDINATION_METADATA), + argThat(new BlobNameMatcher(COORDINATION_METADATA_FILENAME)), any() ); - verify(mockedGlobalMetadataManager, times(1)).getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(PERSISTENT_SETTINGS_FILENAME)), + verify(mockedGlobalMetadataManager, times(1)).readAsync( eq(SETTING_METADATA), + argThat(new BlobNameMatcher(PERSISTENT_SETTINGS_FILENAME)), any() ); - verify(mockedGlobalMetadataManager, times(1)).getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(TRANSIENT_SETTINGS_FILENAME)), + verify(mockedGlobalMetadataManager, times(1)).readAsync( eq(TRANSIENT_SETTING_METADATA), + argThat(new BlobNameMatcher(TRANSIENT_SETTINGS_FILENAME)), any() ); - verify(mockedGlobalMetadataManager, times(1)).getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(TEMPLATES_METADATA_FILENAME)), + verify(mockedGlobalMetadataManager, times(1)).readAsync( eq(TEMPLATES_METADATA), + argThat(new BlobNameMatcher(TEMPLATES_METADATA_FILENAME)), any() ); - verify(mockedGlobalMetadataManager, times(1)).getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(HASHES_OF_CONSISTENT_SETTINGS_FILENAME)), + verify(mockedGlobalMetadataManager, times(1)).readAsync( eq(HASHES_OF_CONSISTENT_SETTINGS), + argThat(new BlobNameMatcher(HASHES_OF_CONSISTENT_SETTINGS_FILENAME)), any() ); newCustomMetadataMap.keySet().forEach(uploadedCustomMetadataKey -> { - verify(mockedGlobalMetadataManager, times(1)).getAsyncMetadataReadAction( - argThat(new BlobNameMatcher(newCustomMetadataMap.get(uploadedCustomMetadataKey).getUploadedFilename())), + verify(mockedGlobalMetadataManager, times(1)).readAsync( eq(uploadedCustomMetadataKey), + argThat(new BlobNameMatcher(newCustomMetadataMap.get(uploadedCustomMetadataKey).getUploadedFilename())), any() ); }); - verify(mockedClusterStateAttributeManager, times(1)).getAsyncMetadataReadAction( + verify(mockedClusterStateAttributeManager, times(1)).readAsync( eq(DISCOVERY_NODES), argThat(new BlobNameMatcher(DISCOVERY_NODES_FILENAME)), any() ); - verify(mockedClusterStateAttributeManager, times(1)).getAsyncMetadataReadAction( + verify(mockedClusterStateAttributeManager, times(1)).readAsync( eq(CLUSTER_BLOCKS), argThat(new BlobNameMatcher(CLUSTER_BLOCKS_FILENAME)), any() ); newClusterStateCustoms.keySet().forEach(uploadedClusterStateCustomMetadataKey -> { - verify(mockedClusterStateAttributeManager, times(1)).getAsyncMetadataReadAction( + verify(mockedClusterStateAttributeManager, times(1)).readAsync( eq(String.join(CUSTOM_DELIMITER, CLUSTER_STATE_CUSTOM, uploadedClusterStateCustomMetadataKey)), argThat(new BlobNameMatcher(newClusterStateCustoms.get(uploadedClusterStateCustomMetadataKey).getUploadedFilename())), any() diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManagerTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManagerTests.java index 917794ec03c3a..a2da1e8b0fdb2 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManagerTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteGlobalMetadataManagerTests.java @@ -158,7 +158,7 @@ public void testGlobalMetadataUploadWaitTimeSetting() { assertEquals(globalMetadataUploadTimeout, remoteGlobalMetadataManager.getGlobalMetadataUploadTimeout().seconds()); } - public void testGetReadMetadataAsyncAction_CoordinationMetadata() throws Exception { + public void testGetAsyncReadRunnable_CoordinationMetadata() throws Exception { CoordinationMetadata coordinationMetadata = getCoordinationMetadata(); String fileName = randomAlphaOfLength(10); RemoteCoordinationMetadata coordinationMetadataForDownload = new RemoteCoordinationMetadata( @@ -173,11 +173,11 @@ public void testGetReadMetadataAsyncAction_CoordinationMetadata() throws Excepti TestCapturingListener listener = new TestCapturingListener<>(); CountDownLatch latch = new CountDownLatch(1); - remoteGlobalMetadataManager.getAsyncMetadataReadAction( - coordinationMetadataForDownload, + remoteGlobalMetadataManager.readAsync( COORDINATION_METADATA, + coordinationMetadataForDownload, new LatchedActionListener<>(listener, latch) - ).run(); + ); latch.await(); assertNull(listener.getFailure()); assertNotNull(listener.getResult()); @@ -186,7 +186,7 @@ public void testGetReadMetadataAsyncAction_CoordinationMetadata() throws Excepti assertEquals(COORDINATION_METADATA, listener.getResult().getComponentName()); } - public void testGetAsyncMetadataWriteAction_CoordinationMetadata() throws Exception { + public void testGetAsyncWriteRunnable_CoordinationMetadata() throws Exception { CoordinationMetadata coordinationMetadata = getCoordinationMetadata(); RemoteCoordinationMetadata remoteCoordinationMetadata = new RemoteCoordinationMetadata( coordinationMetadata, @@ -203,8 +203,11 @@ public void testGetAsyncMetadataWriteAction_CoordinationMetadata() throws Except TestCapturingListener listener = new TestCapturingListener<>(); CountDownLatch latch = new CountDownLatch(1); - remoteGlobalMetadataManager.getAsyncMetadataWriteAction(remoteCoordinationMetadata, new LatchedActionListener<>(listener, latch)) - .run(); + remoteGlobalMetadataManager.writeAsync( + COORDINATION_METADATA, + remoteCoordinationMetadata, + new LatchedActionListener<>(listener, latch) + ); latch.await(); assertNull(listener.getFailure()); assertNotNull(listener.getResult()); @@ -224,7 +227,7 @@ public void testGetAsyncMetadataWriteAction_CoordinationMetadata() throws Except assertEquals(GLOBAL_METADATA_CURRENT_CODEC_VERSION, Integer.parseInt(splitFileName[3])); } - public void testGetReadMetadataAsyncAction_PersistentSettings() throws Exception { + public void testGetAsyncReadRunnable_PersistentSettings() throws Exception { Settings settingsMetadata = getSettings(); String fileName = randomAlphaOfLength(10); RemotePersistentSettingsMetadata persistentSettings = new RemotePersistentSettingsMetadata( @@ -240,11 +243,7 @@ public void testGetReadMetadataAsyncAction_PersistentSettings() throws Exception TestCapturingListener listener = new TestCapturingListener<>(); CountDownLatch latch = new CountDownLatch(1); - remoteGlobalMetadataManager.getAsyncMetadataReadAction( - persistentSettings, - SETTING_METADATA, - new LatchedActionListener<>(listener, latch) - ).run(); + remoteGlobalMetadataManager.readAsync(SETTING_METADATA, persistentSettings, new LatchedActionListener<>(listener, latch)); latch.await(); assertNull(listener.getFailure()); assertNotNull(listener.getResult()); @@ -253,7 +252,7 @@ public void testGetReadMetadataAsyncAction_PersistentSettings() throws Exception assertEquals(SETTING_METADATA, listener.getResult().getComponentName()); } - public void testGetAsyncMetadataWriteAction_PersistentSettings() throws Exception { + public void testGetAsyncWriteRunnable_PersistentSettings() throws Exception { Settings settingsMetadata = getSettings(); RemotePersistentSettingsMetadata persistentSettings = new RemotePersistentSettingsMetadata( settingsMetadata, @@ -269,7 +268,7 @@ public void testGetAsyncMetadataWriteAction_PersistentSettings() throws Exceptio .uploadBlob(any(InputStream.class), anyIterable(), anyString(), eq(URGENT), any(ActionListener.class)); TestCapturingListener listener = new TestCapturingListener<>(); CountDownLatch latch = new CountDownLatch(1); - remoteGlobalMetadataManager.getAsyncMetadataWriteAction(persistentSettings, new LatchedActionListener<>(listener, latch)).run(); + remoteGlobalMetadataManager.writeAsync(SETTING_METADATA, persistentSettings, new LatchedActionListener<>(listener, latch)); latch.await(); assertNull(listener.getFailure()); @@ -290,7 +289,7 @@ public void testGetAsyncMetadataWriteAction_PersistentSettings() throws Exceptio assertEquals(GLOBAL_METADATA_CURRENT_CODEC_VERSION, Integer.parseInt(splitFileName[3])); } - public void testGetReadMetadataAsyncAction_TransientSettings() throws Exception { + public void testGetAsyncReadRunnable_TransientSettings() throws Exception { Settings settingsMetadata = getSettings(); String fileName = randomAlphaOfLength(10); RemoteTransientSettingsMetadata transientSettings = new RemoteTransientSettingsMetadata( @@ -306,11 +305,7 @@ public void testGetReadMetadataAsyncAction_TransientSettings() throws Exception TestCapturingListener listener = new TestCapturingListener<>(); CountDownLatch latch = new CountDownLatch(1); - remoteGlobalMetadataManager.getAsyncMetadataReadAction( - transientSettings, - TRANSIENT_SETTING_METADATA, - new LatchedActionListener<>(listener, latch) - ).run(); + remoteGlobalMetadataManager.readAsync(TRANSIENT_SETTING_METADATA, transientSettings, new LatchedActionListener<>(listener, latch)); latch.await(); assertNull(listener.getFailure()); assertNotNull(listener.getResult()); @@ -319,7 +314,7 @@ public void testGetReadMetadataAsyncAction_TransientSettings() throws Exception assertEquals(TRANSIENT_SETTING_METADATA, listener.getResult().getComponentName()); } - public void testGetAsyncMetadataWriteAction_TransientSettings() throws Exception { + public void testGetAsyncWriteRunnable_TransientSettings() throws Exception { Settings settingsMetadata = getSettings(); RemoteTransientSettingsMetadata transientSettings = new RemoteTransientSettingsMetadata( settingsMetadata, @@ -335,7 +330,7 @@ public void testGetAsyncMetadataWriteAction_TransientSettings() throws Exception .uploadBlob(any(InputStream.class), anyIterable(), anyString(), eq(URGENT), any(ActionListener.class)); TestCapturingListener listener = new TestCapturingListener<>(); CountDownLatch latch = new CountDownLatch(1); - remoteGlobalMetadataManager.getAsyncMetadataWriteAction(transientSettings, new LatchedActionListener<>(listener, latch)).run(); + remoteGlobalMetadataManager.writeAsync(TRANSIENT_SETTING_METADATA, transientSettings, new LatchedActionListener<>(listener, latch)); latch.await(); assertNull(listener.getFailure()); assertNotNull(listener.getResult()); @@ -355,7 +350,7 @@ public void testGetAsyncMetadataWriteAction_TransientSettings() throws Exception assertEquals(GLOBAL_METADATA_CURRENT_CODEC_VERSION, Integer.parseInt(splitFileName[3])); } - public void testGetReadMetadataAsyncAction_HashesOfConsistentSettings() throws Exception { + public void testGetAsyncReadRunnable_HashesOfConsistentSettings() throws Exception { DiffableStringMap hashesOfConsistentSettings = getHashesOfConsistentSettings(); String fileName = randomAlphaOfLength(10); RemoteHashesOfConsistentSettings hashesOfConsistentSettingsForDownload = new RemoteHashesOfConsistentSettings( @@ -369,11 +364,11 @@ public void testGetReadMetadataAsyncAction_HashesOfConsistentSettings() throws E TestCapturingListener listener = new TestCapturingListener<>(); CountDownLatch latch = new CountDownLatch(1); - remoteGlobalMetadataManager.getAsyncMetadataReadAction( - hashesOfConsistentSettingsForDownload, + remoteGlobalMetadataManager.readAsync( HASHES_OF_CONSISTENT_SETTINGS, + hashesOfConsistentSettingsForDownload, new LatchedActionListener<>(listener, latch) - ).run(); + ); latch.await(); assertNull(listener.getFailure()); assertNotNull(listener.getResult()); @@ -382,7 +377,7 @@ public void testGetReadMetadataAsyncAction_HashesOfConsistentSettings() throws E assertEquals(HASHES_OF_CONSISTENT_SETTINGS, listener.getResult().getComponentName()); } - public void testGetAsyncMetadataWriteAction_HashesOfConsistentSettings() throws Exception { + public void testGetAsyncWriteRunnable_HashesOfConsistentSettings() throws Exception { DiffableStringMap hashesOfConsistentSettings = getHashesOfConsistentSettings(); RemoteHashesOfConsistentSettings hashesOfConsistentSettingsForUpload = new RemoteHashesOfConsistentSettings( hashesOfConsistentSettings, @@ -397,10 +392,11 @@ public void testGetAsyncMetadataWriteAction_HashesOfConsistentSettings() throws .uploadBlob(any(InputStream.class), anyIterable(), anyString(), eq(URGENT), any(ActionListener.class)); TestCapturingListener listener = new TestCapturingListener<>(); CountDownLatch latch = new CountDownLatch(1); - remoteGlobalMetadataManager.getAsyncMetadataWriteAction( + remoteGlobalMetadataManager.writeAsync( + HASHES_OF_CONSISTENT_SETTINGS, hashesOfConsistentSettingsForUpload, new LatchedActionListener<>(listener, latch) - ).run(); + ); latch.await(); assertNull(listener.getFailure()); assertNotNull(listener.getResult()); @@ -420,7 +416,7 @@ public void testGetAsyncMetadataWriteAction_HashesOfConsistentSettings() throws assertEquals(GLOBAL_METADATA_CURRENT_CODEC_VERSION, Integer.parseInt(splitFileName[3])); } - public void testGetReadMetadataAsyncAction_TemplatesMetadata() throws Exception { + public void testGetAsyncReadRunnable_TemplatesMetadata() throws Exception { TemplatesMetadata templatesMetadata = getTemplatesMetadata(); String fileName = randomAlphaOfLength(10); RemoteTemplatesMetadata templatesMetadataForDownload = new RemoteTemplatesMetadata( @@ -434,11 +430,11 @@ public void testGetReadMetadataAsyncAction_TemplatesMetadata() throws Exception ); TestCapturingListener listener = new TestCapturingListener<>(); CountDownLatch latch = new CountDownLatch(1); - remoteGlobalMetadataManager.getAsyncMetadataReadAction( - templatesMetadataForDownload, + remoteGlobalMetadataManager.readAsync( TEMPLATES_METADATA, + templatesMetadataForDownload, new LatchedActionListener<>(listener, latch) - ).run(); + ); latch.await(); assertNull(listener.getFailure()); assertNotNull(listener.getResult()); @@ -447,7 +443,7 @@ public void testGetReadMetadataAsyncAction_TemplatesMetadata() throws Exception assertEquals(TEMPLATES_METADATA, listener.getResult().getComponentName()); } - public void testGetAsyncMetadataWriteAction_TemplatesMetadata() throws Exception { + public void testGetAsyncWriteRunnable_TemplatesMetadata() throws Exception { TemplatesMetadata templatesMetadata = getTemplatesMetadata(); RemoteTemplatesMetadata templateMetadataForUpload = new RemoteTemplatesMetadata( templatesMetadata, @@ -463,8 +459,7 @@ public void testGetAsyncMetadataWriteAction_TemplatesMetadata() throws Exception .uploadBlob(any(InputStream.class), anyIterable(), anyString(), eq(URGENT), any(ActionListener.class)); TestCapturingListener listener = new TestCapturingListener<>(); CountDownLatch latch = new CountDownLatch(1); - remoteGlobalMetadataManager.getAsyncMetadataWriteAction(templateMetadataForUpload, new LatchedActionListener<>(listener, latch)) - .run(); + remoteGlobalMetadataManager.writeAsync(TEMPLATES_METADATA, templateMetadataForUpload, new LatchedActionListener<>(listener, latch)); latch.await(); assertNull(listener.getFailure()); assertNotNull(listener.getResult()); @@ -484,7 +479,7 @@ public void testGetAsyncMetadataWriteAction_TemplatesMetadata() throws Exception assertEquals(GLOBAL_METADATA_CURRENT_CODEC_VERSION, Integer.parseInt(splitFileName[3])); } - public void testGetReadMetadataAsyncAction_CustomMetadata() throws Exception { + public void testGetAsyncReadRunnable_CustomMetadata() throws Exception { Metadata.Custom customMetadata = getCustomMetadata(); String fileName = randomAlphaOfLength(10); RemoteCustomMetadata customMetadataForDownload = new RemoteCustomMetadata( @@ -499,11 +494,7 @@ public void testGetReadMetadataAsyncAction_CustomMetadata() throws Exception { ); TestCapturingListener listener = new TestCapturingListener<>(); CountDownLatch latch = new CountDownLatch(1); - remoteGlobalMetadataManager.getAsyncMetadataReadAction( - customMetadataForDownload, - IndexGraveyard.TYPE, - new LatchedActionListener<>(listener, latch) - ).run(); + remoteGlobalMetadataManager.readAsync(IndexGraveyard.TYPE, customMetadataForDownload, new LatchedActionListener<>(listener, latch)); latch.await(); assertNull(listener.getFailure()); assertNotNull(listener.getResult()); @@ -512,7 +503,7 @@ public void testGetReadMetadataAsyncAction_CustomMetadata() throws Exception { assertEquals(IndexGraveyard.TYPE, listener.getResult().getComponentName()); } - public void testGetAsyncMetadataWriteAction_CustomMetadata() throws Exception { + public void testGetAsyncWriteRunnable_CustomMetadata() throws Exception { Metadata.Custom customMetadata = getCustomMetadata(); RemoteCustomMetadata customMetadataForUpload = new RemoteCustomMetadata( customMetadata, @@ -529,8 +520,11 @@ public void testGetAsyncMetadataWriteAction_CustomMetadata() throws Exception { .uploadBlob(any(InputStream.class), anyIterable(), anyString(), eq(URGENT), any(ActionListener.class)); TestCapturingListener listener = new TestCapturingListener<>(); CountDownLatch latch = new CountDownLatch(1); - remoteGlobalMetadataManager.getAsyncMetadataWriteAction(customMetadataForUpload, new LatchedActionListener<>(listener, latch)) - .run(); + remoteGlobalMetadataManager.writeAsync( + customMetadataForUpload.getType(), + customMetadataForUpload, + new LatchedActionListener<>(listener, latch) + ); latch.await(); assertNull(listener.getFailure()); assertNotNull(listener.getResult()); @@ -550,7 +544,7 @@ public void testGetAsyncMetadataWriteAction_CustomMetadata() throws Exception { assertEquals(GLOBAL_METADATA_CURRENT_CODEC_VERSION, Integer.parseInt(splitFileName[3])); } - public void testGetReadMetadataAsyncAction_GlobalMetadata() throws Exception { + public void testGetAsyncReadRunnable_GlobalMetadata() throws Exception { Metadata metadata = getGlobalMetadata(); String fileName = randomAlphaOfLength(10); RemoteGlobalMetadata globalMetadataForDownload = new RemoteGlobalMetadata(fileName, CLUSTER_UUID, compressor, xContentRegistry); @@ -559,11 +553,7 @@ public void testGetReadMetadataAsyncAction_GlobalMetadata() throws Exception { ); TestCapturingListener listener = new TestCapturingListener<>(); CountDownLatch latch = new CountDownLatch(1); - remoteGlobalMetadataManager.getAsyncMetadataReadAction( - globalMetadataForDownload, - GLOBAL_METADATA, - new LatchedActionListener<>(listener, latch) - ).run(); + remoteGlobalMetadataManager.readAsync(GLOBAL_METADATA, globalMetadataForDownload, new LatchedActionListener<>(listener, latch)); latch.await(); assertNull(listener.getFailure()); assertNotNull(listener.getResult()); @@ -572,7 +562,7 @@ public void testGetReadMetadataAsyncAction_GlobalMetadata() throws Exception { assertEquals(GLOBAL_METADATA, listener.getResult().getComponentName()); } - public void testGetReadMetadataAsyncAction_IOException() throws Exception { + public void testGetAsyncReadRunnable_IOException() throws Exception { String fileName = randomAlphaOfLength(10); RemoteCoordinationMetadata coordinationMetadataForDownload = new RemoteCoordinationMetadata( fileName, @@ -584,18 +574,19 @@ public void testGetReadMetadataAsyncAction_IOException() throws Exception { when(blobStoreTransferService.downloadBlob(anyIterable(), anyString())).thenThrow(ioException); TestCapturingListener listener = new TestCapturingListener<>(); CountDownLatch latch = new CountDownLatch(1); - remoteGlobalMetadataManager.getAsyncMetadataReadAction( - coordinationMetadataForDownload, + remoteGlobalMetadataManager.readAsync( COORDINATION_METADATA, + coordinationMetadataForDownload, new LatchedActionListener<>(listener, latch) - ).run(); + ); latch.await(); assertNull(listener.getResult()); assertNotNull(listener.getFailure()); - assertEquals(ioException, listener.getFailure()); + assertEquals(ioException, listener.getFailure().getCause()); + assertTrue(listener.getFailure() instanceof RemoteStateTransferException); } - public void testGetAsyncMetadataWriteAction_IOException() throws Exception { + public void testGetAsyncWriteRunnable_IOException() throws Exception { CoordinationMetadata coordinationMetadata = getCoordinationMetadata(); RemoteCoordinationMetadata remoteCoordinationMetadata = new RemoteCoordinationMetadata( coordinationMetadata, @@ -613,8 +604,11 @@ public void testGetAsyncMetadataWriteAction_IOException() throws Exception { TestCapturingListener listener = new TestCapturingListener<>(); CountDownLatch latch = new CountDownLatch(1); - remoteGlobalMetadataManager.getAsyncMetadataWriteAction(remoteCoordinationMetadata, new LatchedActionListener<>(listener, latch)) - .run(); + remoteGlobalMetadataManager.writeAsync( + COORDINATION_METADATA, + remoteCoordinationMetadata, + new LatchedActionListener<>(listener, latch) + ); assertNull(listener.getResult()); assertNotNull(listener.getFailure()); assertTrue(listener.getFailure() instanceof RemoteStateTransferException); diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteIndexMetadataManagerTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteIndexMetadataManagerTests.java index 817fc7b55d09a..76c5792677ea0 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteIndexMetadataManagerTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteIndexMetadataManagerTests.java @@ -24,6 +24,7 @@ import org.opensearch.core.action.ActionListener; import org.opensearch.core.compress.Compressor; import org.opensearch.core.compress.NoneCompressor; +import org.opensearch.gateway.remote.model.RemoteIndexMetadata; import org.opensearch.gateway.remote.model.RemoteReadResult; import org.opensearch.index.remote.RemoteStoreUtils; import org.opensearch.index.translog.transfer.BlobStoreTransferService; @@ -83,7 +84,7 @@ public void tearDown() throws Exception { threadPool.shutdown(); } - public void testGetAsyncIndexMetadataWriteAction_Success() throws Exception { + public void testGetAsyncWriteRunnable_Success() throws Exception { IndexMetadata indexMetadata = getIndexMetadata(randomAlphaOfLength(10), randomBoolean(), randomAlphaOfLength(10)); BlobContainer blobContainer = mock(AsyncMultiStreamBlobContainer.class); BlobStore blobStore = mock(BlobStore.class); @@ -97,11 +98,11 @@ public void testGetAsyncIndexMetadataWriteAction_Success() throws Exception { return null; })).when(blobStoreTransferService).uploadBlob(any(), any(), any(), eq(WritePriority.URGENT), any(ActionListener.class)); - remoteIndexMetadataManager.getAsyncIndexMetadataWriteAction( - indexMetadata, - "cluster-uuid", + remoteIndexMetadataManager.writeAsync( + INDEX, + new RemoteIndexMetadata(indexMetadata, "cluster-uuid", compressor, null), new LatchedActionListener<>(listener, latch) - ).run(); + ); latch.await(); assertNull(listener.getFailure()); @@ -116,7 +117,7 @@ public void testGetAsyncIndexMetadataWriteAction_Success() throws Exception { assertTrue(pathTokens[6].startsWith(expectedFilePrefix)); } - public void testGetAsyncIndexMetadataWriteAction_IOFailure() throws Exception { + public void testGetAsyncWriteRunnable_IOFailure() throws Exception { IndexMetadata indexMetadata = getIndexMetadata(randomAlphaOfLength(10), randomBoolean(), randomAlphaOfLength(10)); BlobContainer blobContainer = mock(AsyncMultiStreamBlobContainer.class); BlobStore blobStore = mock(BlobStore.class); @@ -129,18 +130,18 @@ public void testGetAsyncIndexMetadataWriteAction_IOFailure() throws Exception { return null; })).when(blobStoreTransferService).uploadBlob(any(), any(), any(), eq(WritePriority.URGENT), any(ActionListener.class)); - remoteIndexMetadataManager.getAsyncIndexMetadataWriteAction( - indexMetadata, - "cluster-uuid", + remoteIndexMetadataManager.writeAsync( + INDEX, + new RemoteIndexMetadata(indexMetadata, "cluster-uuid", compressor, null), new LatchedActionListener<>(listener, latch) - ).run(); + ); latch.await(); assertNull(listener.getResult()); assertNotNull(listener.getFailure()); assertTrue(listener.getFailure() instanceof RemoteStateTransferException); } - public void testGetAsyncIndexMetadataReadAction_Success() throws Exception { + public void testGetAsyncReadRunnable_Success() throws Exception { IndexMetadata indexMetadata = getIndexMetadata(randomAlphaOfLength(10), randomBoolean(), randomAlphaOfLength(10)); String fileName = randomAlphaOfLength(10); fileName = fileName + DELIMITER + '2'; @@ -150,15 +151,18 @@ public void testGetAsyncIndexMetadataReadAction_Success() throws Exception { TestCapturingListener listener = new TestCapturingListener<>(); CountDownLatch latch = new CountDownLatch(1); - remoteIndexMetadataManager.getAsyncIndexMetadataReadAction("cluster-uuid", fileName, new LatchedActionListener<>(listener, latch)) - .run(); + remoteIndexMetadataManager.readAsync( + INDEX, + new RemoteIndexMetadata(fileName, "cluster-uuid", compressor, null), + new LatchedActionListener<>(listener, latch) + ); latch.await(); assertNull(listener.getFailure()); assertNotNull(listener.getResult()); assertEquals(indexMetadata, listener.getResult().getObj()); } - public void testGetAsyncIndexMetadataReadAction_IOFailure() throws Exception { + public void testGetAsyncReadRunnable_IOFailure() throws Exception { String fileName = randomAlphaOfLength(10); fileName = fileName + DELIMITER + '2'; Exception exception = new IOException("testing failure"); @@ -166,12 +170,16 @@ public void testGetAsyncIndexMetadataReadAction_IOFailure() throws Exception { TestCapturingListener listener = new TestCapturingListener<>(); CountDownLatch latch = new CountDownLatch(1); - remoteIndexMetadataManager.getAsyncIndexMetadataReadAction("cluster-uuid", fileName, new LatchedActionListener<>(listener, latch)) - .run(); + remoteIndexMetadataManager.readAsync( + INDEX, + new RemoteIndexMetadata(fileName, "cluster-uuid", compressor, null), + new LatchedActionListener<>(listener, latch) + ); latch.await(); assertNull(listener.getResult()); assertNotNull(listener.getFailure()); - assertEquals(exception, listener.getFailure()); + assertEquals(exception, listener.getFailure().getCause()); + assertTrue(listener.getFailure() instanceof RemoteStateTransferException); } private IndexMetadata getIndexMetadata(String name, @Nullable Boolean writeIndex, String... aliases) { From 0040f4b766459610fae3a0342f26d8f78735778e Mon Sep 17 00:00:00 2001 From: Pranshu Shukla <55992439+Pranshu-S@users.noreply.github.com> Date: Mon, 22 Jul 2024 17:09:56 +0530 Subject: [PATCH 08/37] =?UTF-8?q?Optimise=20TransportNodesAction=20to=20no?= =?UTF-8?q?t=20send=20DiscoveryNodes=20for=20NodeStat=E2=80=A6=20(#14749)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Optimize TransportNodesAction to not send DiscoveryNodes for NodeStats, NodesInfo and ClusterStats call Signed-off-by: Pranshu Shukla --- CHANGELOG.md | 1 + .../node/info/TransportNodesInfoAction.java | 2 +- .../node/stats/TransportNodesStatsAction.java | 2 +- .../stats/TransportClusterStatsAction.java | 2 +- .../support/nodes/BaseNodesRequest.java | 16 ++ .../support/nodes/TransportNodesAction.java | 12 +- .../admin/cluster/RestClusterStatsAction.java | 1 + .../admin/cluster/RestNodesInfoAction.java | 2 +- .../admin/cluster/RestNodesStatsAction.java | 1 + .../rest/action/cat/RestNodesAction.java | 2 + .../action/RestStatsActionTests.java | 59 +++++++ .../TransportClusterStatsActionTests.java | 165 ++++++++++++++++++ .../nodes/TransportNodesActionTests.java | 13 +- .../nodes/TransportNodesInfoActionTests.java | 131 ++++++++++++++ .../nodes/TransportNodesStatsActionTests.java | 130 ++++++++++++++ 15 files changed, 528 insertions(+), 11 deletions(-) create mode 100644 server/src/test/java/org/opensearch/action/RestStatsActionTests.java create mode 100644 server/src/test/java/org/opensearch/action/support/nodes/TransportClusterStatsActionTests.java create mode 100644 server/src/test/java/org/opensearch/action/support/nodes/TransportNodesInfoActionTests.java create mode 100644 server/src/test/java/org/opensearch/action/support/nodes/TransportNodesStatsActionTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 29e70c5026bb8..ab0c80e37e14c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -21,6 +21,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add Plugin interface for loading application based configuration templates (([#14659](https://github.com/opensearch-project/OpenSearch/issues/14659))) - Refactor remote-routing-table service inline with remote state interfaces([#14668](https://github.com/opensearch-project/OpenSearch/pull/14668)) - Add prefix mode verification setting for repository verification (([#14790](https://github.com/opensearch-project/OpenSearch/pull/14790))) +- Optimize TransportNodesAction to not send DiscoveryNodes for NodeStats, NodesInfo and ClusterStats call ([14749](https://github.com/opensearch-project/OpenSearch/pull/14749)) ### Dependencies - Bump `org.gradle.test-retry` from 1.5.8 to 1.5.9 ([#13442](https://github.com/opensearch-project/OpenSearch/pull/13442)) diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/info/TransportNodesInfoAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/info/TransportNodesInfoAction.java index 2c4f8522a5a5c..dda54cce334ec 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/node/info/TransportNodesInfoAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/info/TransportNodesInfoAction.java @@ -129,7 +129,7 @@ protected NodeInfo nodeOperation(NodeInfoRequest nodeRequest) { */ public static class NodeInfoRequest extends TransportRequest { - NodesInfoRequest request; + protected NodesInfoRequest request; public NodeInfoRequest(StreamInput in) throws IOException { super(in); diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java index 2e93e5e7841cb..2c808adc97c7a 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java @@ -140,7 +140,7 @@ protected NodeStats nodeOperation(NodeStatsRequest nodeStatsRequest) { */ public static class NodeStatsRequest extends TransportRequest { - NodesStatsRequest request; + protected NodesStatsRequest request; public NodeStatsRequest(StreamInput in) throws IOException { super(in); diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java index e4f483f796f44..c7d03596a2a36 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java @@ -223,7 +223,7 @@ protected ClusterStatsNodeResponse nodeOperation(ClusterStatsNodeRequest nodeReq */ public static class ClusterStatsNodeRequest extends TransportRequest { - ClusterStatsRequest request; + protected ClusterStatsRequest request; public ClusterStatsNodeRequest(StreamInput in) throws IOException { super(in); diff --git a/server/src/main/java/org/opensearch/action/support/nodes/BaseNodesRequest.java b/server/src/main/java/org/opensearch/action/support/nodes/BaseNodesRequest.java index 4d54ce51c923c..a4f6d8afeaf38 100644 --- a/server/src/main/java/org/opensearch/action/support/nodes/BaseNodesRequest.java +++ b/server/src/main/java/org/opensearch/action/support/nodes/BaseNodesRequest.java @@ -65,6 +65,14 @@ public abstract class BaseNodesRequest * will be ignored and this will be used. * */ private DiscoveryNode[] concreteNodes; + + /** + * Since do not use the discovery nodes coming from the request in all code paths following a request extended off from + * BaseNodeRequest, we do not require it to sent around across all nodes. + * + * Setting default behavior as `true` but can be explicitly changed in requests that do not require. + */ + private boolean includeDiscoveryNodes = true; private final TimeValue DEFAULT_TIMEOUT_SECS = TimeValue.timeValueSeconds(30); private TimeValue timeout; @@ -119,6 +127,14 @@ public void setConcreteNodes(DiscoveryNode[] concreteNodes) { this.concreteNodes = concreteNodes; } + public void setIncludeDiscoveryNodes(boolean value) { + includeDiscoveryNodes = value; + } + + public boolean getIncludeDiscoveryNodes() { + return includeDiscoveryNodes; + } + @Override public ActionRequestValidationException validate() { return null; diff --git a/server/src/main/java/org/opensearch/action/support/nodes/TransportNodesAction.java b/server/src/main/java/org/opensearch/action/support/nodes/TransportNodesAction.java index 9a1a28dd70636..3acd12f632e0f 100644 --- a/server/src/main/java/org/opensearch/action/support/nodes/TransportNodesAction.java +++ b/server/src/main/java/org/opensearch/action/support/nodes/TransportNodesAction.java @@ -226,6 +226,7 @@ class AsyncAction { private final NodesRequest request; private final ActionListener listener; private final AtomicReferenceArray responses; + private final DiscoveryNode[] concreteNodes; private final AtomicInteger counter = new AtomicInteger(); private final Task task; @@ -238,10 +239,18 @@ class AsyncAction { assert request.concreteNodes() != null; } this.responses = new AtomicReferenceArray<>(request.concreteNodes().length); + this.concreteNodes = request.concreteNodes(); + + if (request.getIncludeDiscoveryNodes() == false) { + // As we transfer the ownership of discovery nodes to route the request to into the AsyncAction class, we + // remove the list of DiscoveryNodes from the request. This reduces the payload of the request and improves + // the number of concrete nodes in the memory. + request.setConcreteNodes(null); + } } void start() { - final DiscoveryNode[] nodes = request.concreteNodes(); + final DiscoveryNode[] nodes = this.concreteNodes; if (nodes.length == 0) { // nothing to notify threadPool.generic().execute(() -> listener.onResponse(newResponse(request, responses))); @@ -260,7 +269,6 @@ void start() { if (task != null) { nodeRequest.setParentTask(clusterService.localNode().getId(), task.getId()); } - transportService.sendRequest( node, getTransportNodeAction(node), diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestClusterStatsAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestClusterStatsAction.java index 0766e838210fa..913db3c81e951 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestClusterStatsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestClusterStatsAction.java @@ -66,6 +66,7 @@ public String getName() { public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { ClusterStatsRequest clusterStatsRequest = new ClusterStatsRequest().nodesIds(request.paramAsStringArray("nodeId", null)); clusterStatsRequest.timeout(request.param("timeout")); + clusterStatsRequest.setIncludeDiscoveryNodes(false); return channel -> client.admin().cluster().clusterStats(clusterStatsRequest, new NodesResponseRestListener<>(channel)); } diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesInfoAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesInfoAction.java index 3b83bf9d6f68c..4ac51933ea382 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesInfoAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesInfoAction.java @@ -88,7 +88,7 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC final NodesInfoRequest nodesInfoRequest = prepareRequest(request); nodesInfoRequest.timeout(request.param("timeout")); settingsFilter.addFilterSettingParams(request); - + nodesInfoRequest.setIncludeDiscoveryNodes(false); return channel -> client.admin().cluster().nodesInfo(nodesInfoRequest, new NodesResponseRestListener<>(channel)); } diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesStatsAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesStatsAction.java index 267bfde576dec..ed9c0b171aa56 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesStatsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesStatsAction.java @@ -232,6 +232,7 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC // If no levels are passed in this results in an empty array. String[] levels = Strings.splitStringByCommaToArray(request.param("level")); nodesStatsRequest.indices().setLevels(levels); + nodesStatsRequest.setIncludeDiscoveryNodes(false); return channel -> client.admin().cluster().nodesStats(nodesStatsRequest, new NodesResponseRestListener<>(channel)); } 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 bffb50cc63401..0330fe627ccd0 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 @@ -125,6 +125,7 @@ public RestChannelConsumer doCatRequest(final RestRequest request, final NodeCli public void processResponse(final ClusterStateResponse clusterStateResponse) { NodesInfoRequest nodesInfoRequest = new NodesInfoRequest(); nodesInfoRequest.timeout(request.param("timeout")); + nodesInfoRequest.setIncludeDiscoveryNodes(false); nodesInfoRequest.clear() .addMetrics( NodesInfoRequest.Metric.JVM.metricName(), @@ -137,6 +138,7 @@ public void processResponse(final ClusterStateResponse clusterStateResponse) { public void processResponse(final NodesInfoResponse nodesInfoResponse) { NodesStatsRequest nodesStatsRequest = new NodesStatsRequest(); nodesStatsRequest.timeout(request.param("timeout")); + nodesStatsRequest.setIncludeDiscoveryNodes(false); nodesStatsRequest.clear() .indices(true) .addMetrics( diff --git a/server/src/test/java/org/opensearch/action/RestStatsActionTests.java b/server/src/test/java/org/opensearch/action/RestStatsActionTests.java new file mode 100644 index 0000000000000..9b8a0640ee343 --- /dev/null +++ b/server/src/test/java/org/opensearch/action/RestStatsActionTests.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.action; + +import org.opensearch.client.node.NodeClient; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.settings.SettingsFilter; +import org.opensearch.rest.action.admin.cluster.RestClusterStatsAction; +import org.opensearch.rest.action.admin.cluster.RestNodesInfoAction; +import org.opensearch.rest.action.admin.cluster.RestNodesStatsAction; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.test.rest.FakeRestRequest; +import org.opensearch.threadpool.TestThreadPool; +import org.junit.After; + +import java.util.Collections; + +public class RestStatsActionTests extends OpenSearchTestCase { + private final TestThreadPool threadPool = new TestThreadPool(RestStatsActionTests.class.getName()); + private final NodeClient client = new NodeClient(Settings.EMPTY, threadPool); + + @After + public void terminateThreadPool() { + terminate(threadPool); + } + + public void testClusterStatsActionPrepareRequestNoError() { + RestClusterStatsAction action = new RestClusterStatsAction(); + try { + action.prepareRequest(new FakeRestRequest(), client); + } catch (Throwable t) { + fail(t.getMessage()); + } + } + + public void testNodesStatsActionPrepareRequestNoError() { + RestNodesStatsAction action = new RestNodesStatsAction(); + try { + action.prepareRequest(new FakeRestRequest(), client); + } catch (Throwable t) { + fail(t.getMessage()); + } + } + + public void testNodesInfoActionPrepareRequestNoError() { + RestNodesInfoAction action = new RestNodesInfoAction(new SettingsFilter(Collections.singleton("foo.filtered"))); + try { + action.prepareRequest(new FakeRestRequest(), client); + } catch (Throwable t) { + fail(t.getMessage()); + } + } +} diff --git a/server/src/test/java/org/opensearch/action/support/nodes/TransportClusterStatsActionTests.java b/server/src/test/java/org/opensearch/action/support/nodes/TransportClusterStatsActionTests.java new file mode 100644 index 0000000000000..f8e14b477b8ef --- /dev/null +++ b/server/src/test/java/org/opensearch/action/support/nodes/TransportClusterStatsActionTests.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.action.support.nodes; + +import org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest; +import org.opensearch.action.admin.cluster.stats.ClusterStatsRequest; +import org.opensearch.action.admin.cluster.stats.TransportClusterStatsAction; +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.PlainActionFuture; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.indices.IndicesService; +import org.opensearch.node.NodeService; +import org.opensearch.test.transport.CapturingTransport; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportService; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class TransportClusterStatsActionTests extends TransportNodesActionTests { + + /** + * By default, we send discovery nodes list to each request that is sent across from the coordinator node. This + * behavior is asserted in this test. + */ + public void testClusterStatsActionWithRetentionOfDiscoveryNodesList() { + ClusterStatsRequest request = new ClusterStatsRequest(); + request.setIncludeDiscoveryNodes(true); + Map> combinedSentRequest = performNodesInfoAction(request); + + assertNotNull(combinedSentRequest); + combinedSentRequest.forEach((node, capturedRequestList) -> { + assertNotNull(capturedRequestList); + capturedRequestList.forEach(sentRequest -> { + assertNotNull(sentRequest.getDiscoveryNodes()); + assertEquals(sentRequest.getDiscoveryNodes().length, clusterService.state().nodes().getSize()); + }); + }); + } + + public void testClusterStatsActionWithPreFilledConcreteNodesAndWithRetentionOfDiscoveryNodesList() { + ClusterStatsRequest request = new ClusterStatsRequest(); + Collection discoveryNodes = clusterService.state().getNodes().getNodes().values(); + request.setConcreteNodes(discoveryNodes.toArray(DiscoveryNode[]::new)); + Map> combinedSentRequest = performNodesInfoAction(request); + + assertNotNull(combinedSentRequest); + combinedSentRequest.forEach((node, capturedRequestList) -> { + assertNotNull(capturedRequestList); + capturedRequestList.forEach(sentRequest -> { + assertNotNull(sentRequest.getDiscoveryNodes()); + assertEquals(sentRequest.getDiscoveryNodes().length, clusterService.state().nodes().getSize()); + }); + }); + } + + /** + * In the optimized ClusterStats Request, we do not send the DiscoveryNodes List to each node. This behavior is + * asserted in this test. + */ + public void testClusterStatsActionWithoutRetentionOfDiscoveryNodesList() { + ClusterStatsRequest request = new ClusterStatsRequest(); + request.setIncludeDiscoveryNodes(false); + Map> combinedSentRequest = performNodesInfoAction(request); + + assertNotNull(combinedSentRequest); + combinedSentRequest.forEach((node, capturedRequestList) -> { + assertNotNull(capturedRequestList); + capturedRequestList.forEach(sentRequest -> { assertNull(sentRequest.getDiscoveryNodes()); }); + }); + } + + public void testClusterStatsActionWithPreFilledConcreteNodesAndWithoutRetentionOfDiscoveryNodesList() { + ClusterStatsRequest request = new ClusterStatsRequest(); + Collection discoveryNodes = clusterService.state().getNodes().getNodes().values(); + request.setConcreteNodes(discoveryNodes.toArray(DiscoveryNode[]::new)); + request.setIncludeDiscoveryNodes(false); + Map> combinedSentRequest = performNodesInfoAction(request); + + assertNotNull(combinedSentRequest); + combinedSentRequest.forEach((node, capturedRequestList) -> { + assertNotNull(capturedRequestList); + capturedRequestList.forEach(sentRequest -> { assertNull(sentRequest.getDiscoveryNodes()); }); + }); + } + + private Map> performNodesInfoAction(ClusterStatsRequest request) { + TransportNodesAction action = getTestTransportClusterStatsAction(); + PlainActionFuture listener = new PlainActionFuture<>(); + action.new AsyncAction(null, request, listener).start(); + Map> capturedRequests = transport.getCapturedRequestsByTargetNodeAndClear(); + Map> combinedSentRequest = new HashMap<>(); + + capturedRequests.forEach((node, capturedRequestList) -> { + List sentRequestList = new ArrayList<>(); + + capturedRequestList.forEach(preSentRequest -> { + BytesStreamOutput out = new BytesStreamOutput(); + try { + TransportClusterStatsAction.ClusterStatsNodeRequest clusterStatsNodeRequestFromCoordinator = + (TransportClusterStatsAction.ClusterStatsNodeRequest) preSentRequest.request; + clusterStatsNodeRequestFromCoordinator.writeTo(out); + StreamInput in = out.bytes().streamInput(); + MockClusterStatsNodeRequest mockClusterStatsNodeRequest = new MockClusterStatsNodeRequest(in); + sentRequestList.add(mockClusterStatsNodeRequest); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + + combinedSentRequest.put(node, sentRequestList); + }); + + return combinedSentRequest; + } + + private TestTransportClusterStatsAction getTestTransportClusterStatsAction() { + return new TestTransportClusterStatsAction( + THREAD_POOL, + clusterService, + transportService, + nodeService, + indicesService, + new ActionFilters(Collections.emptySet()) + ); + } + + private static class TestTransportClusterStatsAction extends TransportClusterStatsAction { + public TestTransportClusterStatsAction( + ThreadPool threadPool, + ClusterService clusterService, + TransportService transportService, + NodeService nodeService, + IndicesService indicesService, + ActionFilters actionFilters + ) { + super(threadPool, clusterService, transportService, nodeService, indicesService, actionFilters); + } + } + + private static class MockClusterStatsNodeRequest extends TransportClusterStatsAction.ClusterStatsNodeRequest { + + public MockClusterStatsNodeRequest(StreamInput in) throws IOException { + super(in); + } + + public DiscoveryNode[] getDiscoveryNodes() { + return this.request.concreteNodes(); + } + } +} diff --git a/server/src/test/java/org/opensearch/action/support/nodes/TransportNodesActionTests.java b/server/src/test/java/org/opensearch/action/support/nodes/TransportNodesActionTests.java index 445934b0ccdfd..7e968aa8fb199 100644 --- a/server/src/test/java/org/opensearch/action/support/nodes/TransportNodesActionTests.java +++ b/server/src/test/java/org/opensearch/action/support/nodes/TransportNodesActionTests.java @@ -46,6 +46,8 @@ 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.indices.IndicesService; +import org.opensearch.node.NodeService; import org.opensearch.telemetry.tracing.noop.NoopTracer; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.test.transport.CapturingTransport; @@ -76,11 +78,12 @@ public class TransportNodesActionTests extends OpenSearchTestCase { - private static ThreadPool THREAD_POOL; - - private ClusterService clusterService; - private CapturingTransport transport; - private TransportService transportService; + protected static ThreadPool THREAD_POOL; + protected ClusterService clusterService; + protected CapturingTransport transport; + protected TransportService transportService; + protected NodeService nodeService; + protected IndicesService indicesService; public void testRequestIsSentToEachNode() throws Exception { TransportNodesAction action = getTestTransportNodesAction(); diff --git a/server/src/test/java/org/opensearch/action/support/nodes/TransportNodesInfoActionTests.java b/server/src/test/java/org/opensearch/action/support/nodes/TransportNodesInfoActionTests.java new file mode 100644 index 0000000000000..e9e09d0dbbbf9 --- /dev/null +++ b/server/src/test/java/org/opensearch/action/support/nodes/TransportNodesInfoActionTests.java @@ -0,0 +1,131 @@ +/* + * 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.action.support.nodes; + +import org.opensearch.action.admin.cluster.node.info.NodesInfoRequest; +import org.opensearch.action.admin.cluster.node.info.TransportNodesInfoAction; +import org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest; +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.PlainActionFuture; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.node.NodeService; +import org.opensearch.test.transport.CapturingTransport; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportService; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class TransportNodesInfoActionTests extends TransportNodesActionTests { + + /** + * By default, we send discovery nodes list to each request that is sent across from the coordinator node. This + * behavior is asserted in this test. + */ + public void testNodesInfoActionWithRetentionOfDiscoveryNodesList() { + NodesInfoRequest request = new NodesInfoRequest(); + request.setIncludeDiscoveryNodes(true); + Map> combinedSentRequest = performNodesInfoAction(request); + + assertNotNull(combinedSentRequest); + combinedSentRequest.forEach((node, capturedRequestList) -> { + assertNotNull(capturedRequestList); + capturedRequestList.forEach(sentRequest -> { + assertNotNull(sentRequest.getDiscoveryNodes()); + assertEquals(sentRequest.getDiscoveryNodes().length, clusterService.state().nodes().getSize()); + }); + }); + } + + /** + * In the optimized ClusterStats Request, we do not send the DiscoveryNodes List to each node. This behavior is + * asserted in this test. + */ + public void testNodesInfoActionWithoutRetentionOfDiscoveryNodesList() { + NodesInfoRequest request = new NodesInfoRequest(); + request.setIncludeDiscoveryNodes(false); + Map> combinedSentRequest = performNodesInfoAction(request); + + assertNotNull(combinedSentRequest); + combinedSentRequest.forEach((node, capturedRequestList) -> { + assertNotNull(capturedRequestList); + capturedRequestList.forEach(sentRequest -> { assertNull(sentRequest.getDiscoveryNodes()); }); + }); + } + + private Map> performNodesInfoAction(NodesInfoRequest request) { + TransportNodesAction action = getTestTransportNodesInfoAction(); + PlainActionFuture listener = new PlainActionFuture<>(); + action.new AsyncAction(null, request, listener).start(); + Map> capturedRequests = transport.getCapturedRequestsByTargetNodeAndClear(); + Map> combinedSentRequest = new HashMap<>(); + + capturedRequests.forEach((node, capturedRequestList) -> { + List sentRequestList = new ArrayList<>(); + + capturedRequestList.forEach(preSentRequest -> { + BytesStreamOutput out = new BytesStreamOutput(); + try { + TransportNodesInfoAction.NodeInfoRequest nodesInfoRequestFromCoordinator = + (TransportNodesInfoAction.NodeInfoRequest) preSentRequest.request; + nodesInfoRequestFromCoordinator.writeTo(out); + StreamInput in = out.bytes().streamInput(); + MockNodesInfoRequest nodesStatsRequest = new MockNodesInfoRequest(in); + sentRequestList.add(nodesStatsRequest); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + + combinedSentRequest.put(node, sentRequestList); + }); + + return combinedSentRequest; + } + + private TestTransportNodesInfoAction getTestTransportNodesInfoAction() { + return new TestTransportNodesInfoAction( + THREAD_POOL, + clusterService, + transportService, + nodeService, + new ActionFilters(Collections.emptySet()) + ); + } + + private static class TestTransportNodesInfoAction extends TransportNodesInfoAction { + public TestTransportNodesInfoAction( + ThreadPool threadPool, + ClusterService clusterService, + TransportService transportService, + NodeService nodeService, + ActionFilters actionFilters + ) { + super(threadPool, clusterService, transportService, nodeService, actionFilters); + } + } + + private static class MockNodesInfoRequest extends TransportNodesInfoAction.NodeInfoRequest { + + public MockNodesInfoRequest(StreamInput in) throws IOException { + super(in); + } + + public DiscoveryNode[] getDiscoveryNodes() { + return this.request.concreteNodes(); + } + } +} diff --git a/server/src/test/java/org/opensearch/action/support/nodes/TransportNodesStatsActionTests.java b/server/src/test/java/org/opensearch/action/support/nodes/TransportNodesStatsActionTests.java new file mode 100644 index 0000000000000..c7c420e353e1a --- /dev/null +++ b/server/src/test/java/org/opensearch/action/support/nodes/TransportNodesStatsActionTests.java @@ -0,0 +1,130 @@ +/* + * 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.action.support.nodes; + +import org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest; +import org.opensearch.action.admin.cluster.node.stats.TransportNodesStatsAction; +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.PlainActionFuture; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.node.NodeService; +import org.opensearch.test.transport.CapturingTransport; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportService; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class TransportNodesStatsActionTests extends TransportNodesActionTests { + + /** + * By default, we send discovery nodes list to each request that is sent across from the coordinator node. This + * behavior is asserted in this test. + */ + public void testNodesStatsActionWithRetentionOfDiscoveryNodesList() { + NodesStatsRequest request = new NodesStatsRequest(); + request.setIncludeDiscoveryNodes(true); + Map> combinedSentRequest = performNodesStatsAction(request); + + assertNotNull(combinedSentRequest); + combinedSentRequest.forEach((node, capturedRequestList) -> { + assertNotNull(capturedRequestList); + capturedRequestList.forEach(sentRequest -> { + assertNotNull(sentRequest.getDiscoveryNodes()); + assertEquals(sentRequest.getDiscoveryNodes().length, clusterService.state().nodes().getSize()); + }); + }); + } + + /** + * By default, we send discovery nodes list to each request that is sent across from the coordinator node. This + * behavior is asserted in this test. + */ + public void testNodesStatsActionWithoutRetentionOfDiscoveryNodesList() { + NodesStatsRequest request = new NodesStatsRequest(); + request.setIncludeDiscoveryNodes(false); + Map> combinedSentRequest = performNodesStatsAction(request); + + assertNotNull(combinedSentRequest); + combinedSentRequest.forEach((node, capturedRequestList) -> { + assertNotNull(capturedRequestList); + capturedRequestList.forEach(sentRequest -> { assertNull(sentRequest.getDiscoveryNodes()); }); + }); + } + + private Map> performNodesStatsAction(NodesStatsRequest request) { + TransportNodesAction action = getTestTransportNodesStatsAction(); + PlainActionFuture listener = new PlainActionFuture<>(); + action.new AsyncAction(null, request, listener).start(); + Map> capturedRequests = transport.getCapturedRequestsByTargetNodeAndClear(); + Map> combinedSentRequest = new HashMap<>(); + + capturedRequests.forEach((node, capturedRequestList) -> { + List sentRequestList = new ArrayList<>(); + + capturedRequestList.forEach(preSentRequest -> { + BytesStreamOutput out = new BytesStreamOutput(); + try { + TransportNodesStatsAction.NodeStatsRequest nodesStatsRequestFromCoordinator = + (TransportNodesStatsAction.NodeStatsRequest) preSentRequest.request; + nodesStatsRequestFromCoordinator.writeTo(out); + StreamInput in = out.bytes().streamInput(); + MockNodeStatsRequest nodesStatsRequest = new MockNodeStatsRequest(in); + sentRequestList.add(nodesStatsRequest); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + + combinedSentRequest.put(node, sentRequestList); + }); + + return combinedSentRequest; + } + + private TestTransportNodesStatsAction getTestTransportNodesStatsAction() { + return new TestTransportNodesStatsAction( + THREAD_POOL, + clusterService, + transportService, + nodeService, + new ActionFilters(Collections.emptySet()) + ); + } + + private static class TestTransportNodesStatsAction extends TransportNodesStatsAction { + public TestTransportNodesStatsAction( + ThreadPool threadPool, + ClusterService clusterService, + TransportService transportService, + NodeService nodeService, + ActionFilters actionFilters + ) { + super(threadPool, clusterService, transportService, nodeService, actionFilters); + } + } + + private static class MockNodeStatsRequest extends TransportNodesStatsAction.NodeStatsRequest { + + public MockNodeStatsRequest(StreamInput in) throws IOException { + super(in); + } + + public DiscoveryNode[] getDiscoveryNodes() { + return this.request.concreteNodes(); + } + } +} From ceb60d0720c55433dd3135314618129fc595c57a Mon Sep 17 00:00:00 2001 From: rajiv-kv <157019998+rajiv-kv@users.noreply.github.com> Date: Mon, 22 Jul 2024 20:54:37 +0530 Subject: [PATCH 09/37] Enabling term version check on local state for all ClusterManager Read Transport Actions (#14273) * enabling term version check on local state for all admin read actions Signed-off-by: Rajiv Kumar Vaidyanathan --- CHANGELOG.md | 1 + .../opensearch/action/IndicesRequestIT.java | 29 ++- .../AdmissionForClusterManagerIT.java | 34 ++- .../TransportGetDecommissionStateAction.java | 3 +- .../health/TransportClusterHealthAction.java | 5 + .../get/TransportGetRepositoriesAction.java | 3 +- .../TransportClusterSearchShardsAction.java | 3 +- .../TransportGetWeightedRoutingAction.java | 3 +- .../state/TransportClusterStateAction.java | 6 +- .../TransportGetStoredScriptAction.java | 3 +- .../TransportPendingClusterTasksAction.java | 5 + .../alias/get/TransportGetAliasesAction.java | 3 +- .../indices/TransportIndicesExistsAction.java | 3 +- .../TransportIndicesShardStoresAction.java | 3 +- .../TransportGetComponentTemplateAction.java | 3 +- ...sportGetComposableIndexTemplateAction.java | 3 +- .../get/TransportGetIndexTemplatesAction.java | 3 +- .../ingest/GetPipelineTransportAction.java | 3 +- .../GetSearchPipelineTransportAction.java | 3 +- .../TransportClusterManagerNodeAction.java | 60 +++-- ...TransportClusterManagerNodeReadAction.java | 23 +- .../info/TransportClusterInfoAction.java | 1 + .../mapping/get/GetMappingsActionTests.java | 227 ++++++++++++++++++ 23 files changed, 382 insertions(+), 48 deletions(-) create mode 100644 server/src/test/java/org/opensearch/action/admin/indices/mapping/get/GetMappingsActionTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index ab0c80e37e14c..e77b183601674 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -22,6 +22,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Refactor remote-routing-table service inline with remote state interfaces([#14668](https://github.com/opensearch-project/OpenSearch/pull/14668)) - Add prefix mode verification setting for repository verification (([#14790](https://github.com/opensearch-project/OpenSearch/pull/14790))) - Optimize TransportNodesAction to not send DiscoveryNodes for NodeStats, NodesInfo and ClusterStats call ([14749](https://github.com/opensearch-project/OpenSearch/pull/14749)) +- Enabling term version check on local state for all ClusterManager Read Transport Actions ([#14273](https://github.com/opensearch-project/OpenSearch/pull/14273)) ### Dependencies - Bump `org.gradle.test-retry` from 1.5.8 to 1.5.9 ([#13442](https://github.com/opensearch-project/OpenSearch/pull/13442)) diff --git a/server/src/internalClusterTest/java/org/opensearch/action/IndicesRequestIT.java b/server/src/internalClusterTest/java/org/opensearch/action/IndicesRequestIT.java index 84d833569edcb..927a79d4884ef 100644 --- a/server/src/internalClusterTest/java/org/opensearch/action/IndicesRequestIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/action/IndicesRequestIT.java @@ -84,6 +84,8 @@ import org.opensearch.action.search.SearchResponse; import org.opensearch.action.search.SearchTransportService; import org.opensearch.action.search.SearchType; +import org.opensearch.action.support.clustermanager.term.GetTermVersionAction; +import org.opensearch.action.support.clustermanager.term.GetTermVersionRequest; import org.opensearch.action.support.replication.TransportReplicationActionTests; import org.opensearch.action.termvectors.MultiTermVectorsAction; import org.opensearch.action.termvectors.MultiTermVectorsRequest; @@ -195,6 +197,7 @@ public void cleanUp() { } public void testGetFieldMappings() { + String getFieldMappingsShardAction = GetFieldMappingsAction.NAME + "[index][s]"; interceptTransportActions(getFieldMappingsShardAction); @@ -545,13 +548,14 @@ public void testDeleteIndex() { } public void testGetMappings() { - interceptTransportActions(GetMappingsAction.NAME); - + interceptTransportActions(GetTermVersionAction.NAME, GetMappingsAction.NAME); GetMappingsRequest getMappingsRequest = new GetMappingsRequest().indices(randomIndicesOrAliases()); internalCluster().coordOnlyNodeClient().admin().indices().getMappings(getMappingsRequest).actionGet(); clearInterceptedActions(); - assertSameIndices(getMappingsRequest, GetMappingsAction.NAME); + + assertActionInvocation(GetTermVersionAction.NAME, GetTermVersionRequest.class); + assertNoActionInvocation(GetMappingsAction.NAME); } public void testPutMapping() { @@ -565,8 +569,8 @@ public void testPutMapping() { } public void testGetSettings() { - interceptTransportActions(GetSettingsAction.NAME); + interceptTransportActions(GetSettingsAction.NAME); GetSettingsRequest getSettingsRequest = new GetSettingsRequest().indices(randomIndicesOrAliases()); internalCluster().coordOnlyNodeClient().admin().indices().getSettings(getSettingsRequest).actionGet(); @@ -662,6 +666,21 @@ private static void assertSameIndices(IndicesRequest originalRequest, boolean op } } + private static void assertActionInvocation(String action, Class requestClass) { + List requests = consumeTransportRequests(action); + assertFalse(requests.isEmpty()); + for (TransportRequest internalRequest : requests) { + assertTrue(internalRequest.getClass() == requestClass); + } + } + + private static void assertNoActionInvocation(String... actions) { + for (String action : actions) { + List requests = consumeTransportRequests(action); + assertTrue(requests.isEmpty()); + } + } + private static void assertIndicesSubset(List indices, String... actions) { // indices returned by each bulk shard request need to be a subset of the original indices for (String action : actions) { @@ -781,7 +800,6 @@ public List getTransportInterceptors( } private final Set actions = new HashSet<>(); - private final Map> requests = new HashMap<>(); @Override @@ -831,6 +849,7 @@ public void messageReceived(T request, TransportChannel channel, Task task) thro } } requestHandler.messageReceived(request, channel, task); + } } } diff --git a/server/src/internalClusterTest/java/org/opensearch/ratelimitting/admissioncontrol/AdmissionForClusterManagerIT.java b/server/src/internalClusterTest/java/org/opensearch/ratelimitting/admissioncontrol/AdmissionForClusterManagerIT.java index b9da5ffb86af0..e3a4216e772fb 100644 --- a/server/src/internalClusterTest/java/org/opensearch/ratelimitting/admissioncontrol/AdmissionForClusterManagerIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/ratelimitting/admissioncontrol/AdmissionForClusterManagerIT.java @@ -12,7 +12,11 @@ import org.apache.logging.log4j.Logger; import org.opensearch.action.admin.indices.alias.get.GetAliasesRequest; import org.opensearch.action.admin.indices.alias.get.GetAliasesResponse; +import org.opensearch.action.support.clustermanager.term.GetTermVersionAction; +import org.opensearch.action.support.clustermanager.term.GetTermVersionResponse; import org.opensearch.client.node.NodeClient; +import org.opensearch.cluster.coordination.ClusterStateTermVersion; +import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; @@ -20,6 +24,7 @@ import org.opensearch.node.IoUsageStats; import org.opensearch.node.ResourceUsageCollectorService; import org.opensearch.node.resource.tracker.ResourceTrackerSettings; +import org.opensearch.plugins.Plugin; import org.opensearch.ratelimitting.admissioncontrol.controllers.CpuBasedAdmissionController; import org.opensearch.ratelimitting.admissioncontrol.enums.AdmissionControlActionType; import org.opensearch.ratelimitting.admissioncontrol.enums.AdmissionControlMode; @@ -29,9 +34,13 @@ import org.opensearch.rest.action.admin.indices.RestGetAliasesAction; import org.opensearch.test.OpenSearchIntegTestCase; import org.opensearch.test.rest.FakeRestRequest; +import org.opensearch.test.transport.MockTransportService; +import org.opensearch.transport.TransportService; import org.junit.Before; +import java.util.Collection; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicReference; @@ -62,6 +71,10 @@ public class AdmissionForClusterManagerIT extends OpenSearchIntegTestCase { .put(CLUSTER_ADMIN_CPU_USAGE_LIMIT.getKey(), 50) .build(); + protected Collection> nodePlugins() { + return List.of(MockTransportService.TestPlugin.class); + } + @Before public void init() { String clusterManagerNode = internalCluster().startClusterManagerOnlyNode( @@ -79,6 +92,25 @@ public void init() { // Enable admission control client().admin().cluster().prepareUpdateSettings().setTransientSettings(ENFORCE_ADMISSION_CONTROL).execute().actionGet(); + MockTransportService primaryService = (MockTransportService) internalCluster().getInstance( + TransportService.class, + clusterManagerNode + ); + + // Force always fetch from ClusterManager + ClusterService clusterService = internalCluster().clusterService(); + GetTermVersionResponse oosTerm = new GetTermVersionResponse( + new ClusterStateTermVersion( + clusterService.state().getClusterName(), + clusterService.state().metadata().clusterUUID(), + clusterService.state().term() - 1, + clusterService.state().version() - 1 + ) + ); + primaryService.addRequestHandlingBehavior( + GetTermVersionAction.NAME, + (handler, request, channel, task) -> channel.sendResponse(oosTerm) + ); } public void testAdmissionControlEnforced() throws Exception { @@ -86,8 +118,8 @@ public void testAdmissionControlEnforced() throws Exception { // Write API on ClusterManager assertAcked(prepareCreate("test").setMapping("field", "type=text").setAliases("{\"alias1\" : {}}")); - // Read API on ClusterManager + GetAliasesRequest aliasesRequest = new GetAliasesRequest(); aliasesRequest.aliases("alias1"); try { diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/decommission/awareness/get/TransportGetDecommissionStateAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/decommission/awareness/get/TransportGetDecommissionStateAction.java index 22feb4d99297a..c8a3be78a790e 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/decommission/awareness/get/TransportGetDecommissionStateAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/decommission/awareness/get/TransportGetDecommissionStateAction.java @@ -48,7 +48,8 @@ public TransportGetDecommissionStateAction( threadPool, actionFilters, GetDecommissionStateRequest::new, - indexNameExpressionResolver + indexNameExpressionResolver, + true ); } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/health/TransportClusterHealthAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/health/TransportClusterHealthAction.java index 1cc357a4c20f4..f69f462372888 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/health/TransportClusterHealthAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/health/TransportClusterHealthAction.java @@ -534,4 +534,9 @@ private ClusterHealthResponse clusterHealth( pendingTaskTimeInQueue ); } + + @Override + protected boolean localExecuteSupportedByAction() { + return false; + } } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/repositories/get/TransportGetRepositoriesAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/repositories/get/TransportGetRepositoriesAction.java index c7d784dbc96e7..c99b52dfe34f4 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/repositories/get/TransportGetRepositoriesAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/repositories/get/TransportGetRepositoriesAction.java @@ -79,7 +79,8 @@ public TransportGetRepositoriesAction( threadPool, actionFilters, GetRepositoriesRequest::new, - indexNameExpressionResolver + indexNameExpressionResolver, + true ); } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/shards/TransportClusterSearchShardsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/shards/TransportClusterSearchShardsAction.java index a2a65b6400c97..83e104236f640 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/shards/TransportClusterSearchShardsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/shards/TransportClusterSearchShardsAction.java @@ -85,7 +85,8 @@ public TransportClusterSearchShardsAction( threadPool, actionFilters, ClusterSearchShardsRequest::new, - indexNameExpressionResolver + indexNameExpressionResolver, + true ); this.indicesService = indicesService; } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/shards/routing/weighted/get/TransportGetWeightedRoutingAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/shards/routing/weighted/get/TransportGetWeightedRoutingAction.java index 50368d85e0011..6c110c0ea2a73 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/shards/routing/weighted/get/TransportGetWeightedRoutingAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/shards/routing/weighted/get/TransportGetWeightedRoutingAction.java @@ -55,7 +55,8 @@ public TransportGetWeightedRoutingAction( threadPool, actionFilters, ClusterGetWeightedRoutingRequest::new, - indexNameExpressionResolver + indexNameExpressionResolver, + true ); this.weightedRoutingService = weightedRoutingService; } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/state/TransportClusterStateAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/state/TransportClusterStateAction.java index cae465a90446e..13ea7eaa43bf8 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/state/TransportClusterStateAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/state/TransportClusterStateAction.java @@ -92,6 +92,7 @@ public TransportClusterStateAction( ClusterStateRequest::new, indexNameExpressionResolver ); + this.localExecuteSupported = true; } @Override @@ -233,9 +234,4 @@ private ClusterStateResponse buildResponse(final ClusterStateRequest request, fi return new ClusterStateResponse(currentState.getClusterName(), builder.build(), false); } - - @Override - protected boolean localExecuteSupportedByAction() { - return true; - } } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/storedscripts/TransportGetStoredScriptAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/storedscripts/TransportGetStoredScriptAction.java index db1f1edde2812..c34ec49406802 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/storedscripts/TransportGetStoredScriptAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/storedscripts/TransportGetStoredScriptAction.java @@ -73,7 +73,8 @@ public TransportGetStoredScriptAction( threadPool, actionFilters, GetStoredScriptRequest::new, - indexNameExpressionResolver + indexNameExpressionResolver, + true ); this.scriptService = scriptService; } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/tasks/TransportPendingClusterTasksAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/tasks/TransportPendingClusterTasksAction.java index 5d5053cc80738..01846ef46c1ed 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/tasks/TransportPendingClusterTasksAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/tasks/TransportPendingClusterTasksAction.java @@ -110,4 +110,9 @@ protected void clusterManagerOperation( logger.trace("done fetching pending tasks from cluster service"); listener.onResponse(new PendingClusterTasksResponse(pendingTasks)); } + + @Override + protected boolean localExecuteSupportedByAction() { + return false; + } } diff --git a/server/src/main/java/org/opensearch/action/admin/indices/alias/get/TransportGetAliasesAction.java b/server/src/main/java/org/opensearch/action/admin/indices/alias/get/TransportGetAliasesAction.java index 3aca9c1976f16..4f4e3bd481ee7 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/alias/get/TransportGetAliasesAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/alias/get/TransportGetAliasesAction.java @@ -86,7 +86,8 @@ public TransportGetAliasesAction( threadPool, actionFilters, GetAliasesRequest::new, - indexNameExpressionResolver + indexNameExpressionResolver, + true ); this.systemIndices = systemIndices; } diff --git a/server/src/main/java/org/opensearch/action/admin/indices/exists/indices/TransportIndicesExistsAction.java b/server/src/main/java/org/opensearch/action/admin/indices/exists/indices/TransportIndicesExistsAction.java index 428a0eb35513d..a298eae1aa865 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/exists/indices/TransportIndicesExistsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/exists/indices/TransportIndicesExistsAction.java @@ -71,7 +71,8 @@ public TransportIndicesExistsAction( threadPool, actionFilters, IndicesExistsRequest::new, - indexNameExpressionResolver + indexNameExpressionResolver, + true ); } diff --git a/server/src/main/java/org/opensearch/action/admin/indices/shards/TransportIndicesShardStoresAction.java b/server/src/main/java/org/opensearch/action/admin/indices/shards/TransportIndicesShardStoresAction.java index 3fbf9ac1bb570..a8b97d0f344ae 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/shards/TransportIndicesShardStoresAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/shards/TransportIndicesShardStoresAction.java @@ -105,7 +105,8 @@ public TransportIndicesShardStoresAction( threadPool, actionFilters, IndicesShardStoresRequest::new, - indexNameExpressionResolver + indexNameExpressionResolver, + true ); this.listShardStoresInfo = listShardStoresInfo; } diff --git a/server/src/main/java/org/opensearch/action/admin/indices/template/get/TransportGetComponentTemplateAction.java b/server/src/main/java/org/opensearch/action/admin/indices/template/get/TransportGetComponentTemplateAction.java index e2594cd792cd3..c3217d109044d 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/template/get/TransportGetComponentTemplateAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/template/get/TransportGetComponentTemplateAction.java @@ -76,7 +76,8 @@ public TransportGetComponentTemplateAction( threadPool, actionFilters, GetComponentTemplateAction.Request::new, - indexNameExpressionResolver + indexNameExpressionResolver, + true ); } diff --git a/server/src/main/java/org/opensearch/action/admin/indices/template/get/TransportGetComposableIndexTemplateAction.java b/server/src/main/java/org/opensearch/action/admin/indices/template/get/TransportGetComposableIndexTemplateAction.java index b1ef32db7274f..84fbb59481c10 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/template/get/TransportGetComposableIndexTemplateAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/template/get/TransportGetComposableIndexTemplateAction.java @@ -76,7 +76,8 @@ public TransportGetComposableIndexTemplateAction( threadPool, actionFilters, GetComposableIndexTemplateAction.Request::new, - indexNameExpressionResolver + indexNameExpressionResolver, + true ); } diff --git a/server/src/main/java/org/opensearch/action/admin/indices/template/get/TransportGetIndexTemplatesAction.java b/server/src/main/java/org/opensearch/action/admin/indices/template/get/TransportGetIndexTemplatesAction.java index 10b4975f7b9d0..522234dda509f 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/template/get/TransportGetIndexTemplatesAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/template/get/TransportGetIndexTemplatesAction.java @@ -76,7 +76,8 @@ public TransportGetIndexTemplatesAction( threadPool, actionFilters, GetIndexTemplatesRequest::new, - indexNameExpressionResolver + indexNameExpressionResolver, + true ); } diff --git a/server/src/main/java/org/opensearch/action/ingest/GetPipelineTransportAction.java b/server/src/main/java/org/opensearch/action/ingest/GetPipelineTransportAction.java index 80333c7346f92..7bc0380bccbc0 100644 --- a/server/src/main/java/org/opensearch/action/ingest/GetPipelineTransportAction.java +++ b/server/src/main/java/org/opensearch/action/ingest/GetPipelineTransportAction.java @@ -70,7 +70,8 @@ public GetPipelineTransportAction( threadPool, actionFilters, GetPipelineRequest::new, - indexNameExpressionResolver + indexNameExpressionResolver, + true ); } diff --git a/server/src/main/java/org/opensearch/action/search/GetSearchPipelineTransportAction.java b/server/src/main/java/org/opensearch/action/search/GetSearchPipelineTransportAction.java index a7fcb8f1cfbae..215b7ae1a610c 100644 --- a/server/src/main/java/org/opensearch/action/search/GetSearchPipelineTransportAction.java +++ b/server/src/main/java/org/opensearch/action/search/GetSearchPipelineTransportAction.java @@ -48,7 +48,8 @@ public GetSearchPipelineTransportAction( threadPool, actionFilters, GetSearchPipelineRequest::new, - indexNameExpressionResolver + indexNameExpressionResolver, + true ); } diff --git a/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeAction.java b/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeAction.java index 080b0d607e991..4e869f29878cd 100644 --- a/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeAction.java +++ b/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeAction.java @@ -267,24 +267,7 @@ protected void doStart(ClusterState clusterState) { final DiscoveryNodes nodes = clusterState.nodes(); if (nodes.isLocalNodeElectedClusterManager() || localExecute(request)) { // check for block, if blocked, retry, else, execute locally - final ClusterBlockException blockException = checkBlock(request, clusterState); - if (blockException != null) { - if (!blockException.retryable()) { - listener.onFailure(blockException); - } else { - logger.debug("can't execute due to a cluster block, retrying", blockException); - retry(clusterState, blockException, newState -> { - try { - ClusterBlockException newException = checkBlock(request, newState); - return (newException == null || !newException.retryable()); - } catch (Exception e) { - // accept state as block will be rechecked by doStart() and listener.onFailure() then called - logger.trace("exception occurred during cluster block checking, accepting state", e); - return true; - } - }); - } - } else { + if (!checkForBlock(request, clusterState)) { threadPool.executor(executor) .execute( ActionRunnable.wrap( @@ -422,12 +405,43 @@ public GetTermVersionResponse read(StreamInput in) throws IOException { }; } + private boolean checkForBlock(Request request, ClusterState localClusterState) { + final ClusterBlockException blockException = checkBlock(request, localClusterState); + if (blockException != null) { + if (!blockException.retryable()) { + listener.onFailure(blockException); + } else { + logger.debug("can't execute due to a cluster block, retrying", blockException); + retry(localClusterState, blockException, newState -> { + try { + ClusterBlockException newException = checkBlock(request, newState); + return (newException == null || !newException.retryable()); + } catch (Exception e) { + // accept state as block will be rechecked by doStart() and listener.onFailure() then called + logger.trace("exception occurred during cluster block checking, accepting state", e); + return true; + } + }); + } + return true; + } else { + return false; + } + } + private void executeOnLocalNode(ClusterState localClusterState) { - Runnable runTask = ActionRunnable.wrap( - getDelegateForLocalExecute(localClusterState), - l -> clusterManagerOperation(task, request, localClusterState, l) - ); - threadPool.executor(executor).execute(runTask); + try { + // check for block, if blocked, retry, else, execute locally + if (!checkForBlock(request, localClusterState)) { + Runnable runTask = ActionRunnable.wrap( + getDelegateForLocalExecute(localClusterState), + l -> clusterManagerOperation(task, request, localClusterState, l) + ); + threadPool.executor(executor).execute(runTask); + } + } catch (Exception e) { + listener.onFailure(e); + } } private void executeOnClusterManager(DiscoveryNode clusterManagerNode, ClusterState clusterState) { diff --git a/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeReadAction.java b/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeReadAction.java index d58487a475bcf..88cb2ed6a9bf0 100644 --- a/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeReadAction.java +++ b/server/src/main/java/org/opensearch/action/support/clustermanager/TransportClusterManagerNodeReadAction.java @@ -51,6 +51,8 @@ public abstract class TransportClusterManagerNodeReadAction< Request extends ClusterManagerNodeReadRequest, Response extends ActionResponse> extends TransportClusterManagerNodeAction { + protected boolean localExecuteSupported = false; + protected TransportClusterManagerNodeReadAction( String actionName, TransportService transportService, @@ -58,7 +60,8 @@ protected TransportClusterManagerNodeReadAction( ThreadPool threadPool, ActionFilters actionFilters, Writeable.Reader request, - IndexNameExpressionResolver indexNameExpressionResolver + IndexNameExpressionResolver indexNameExpressionResolver, + boolean localExecuteSupported ) { this( actionName, @@ -71,6 +74,19 @@ protected TransportClusterManagerNodeReadAction( request, indexNameExpressionResolver ); + this.localExecuteSupported = localExecuteSupported; + } + + protected TransportClusterManagerNodeReadAction( + String actionName, + TransportService transportService, + ClusterService clusterService, + ThreadPool threadPool, + ActionFilters actionFilters, + Writeable.Reader request, + IndexNameExpressionResolver indexNameExpressionResolver + ) { + this(actionName, transportService, clusterService, threadPool, actionFilters, request, indexNameExpressionResolver, false); } protected TransportClusterManagerNodeReadAction( @@ -124,4 +140,9 @@ protected TransportClusterManagerNodeReadAction( protected final boolean localExecute(Request request) { return request.local(); } + + protected boolean localExecuteSupportedByAction() { + return localExecuteSupported; + } + } diff --git a/server/src/main/java/org/opensearch/action/support/clustermanager/info/TransportClusterInfoAction.java b/server/src/main/java/org/opensearch/action/support/clustermanager/info/TransportClusterInfoAction.java index 65f00a4731ab5..8a0082ad05f66 100644 --- a/server/src/main/java/org/opensearch/action/support/clustermanager/info/TransportClusterInfoAction.java +++ b/server/src/main/java/org/opensearch/action/support/clustermanager/info/TransportClusterInfoAction.java @@ -62,6 +62,7 @@ public TransportClusterInfoAction( IndexNameExpressionResolver indexNameExpressionResolver ) { super(actionName, transportService, clusterService, threadPool, actionFilters, request, indexNameExpressionResolver); + this.localExecuteSupported = true; } @Override diff --git a/server/src/test/java/org/opensearch/action/admin/indices/mapping/get/GetMappingsActionTests.java b/server/src/test/java/org/opensearch/action/admin/indices/mapping/get/GetMappingsActionTests.java new file mode 100644 index 0000000000000..87f218760038e --- /dev/null +++ b/server/src/test/java/org/opensearch/action/admin/indices/mapping/get/GetMappingsActionTests.java @@ -0,0 +1,227 @@ +/* + * 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. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you 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. + */ +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.action.admin.indices.mapping.get; + +import org.opensearch.Version; +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.clustermanager.term.GetTermVersionResponse; +import org.opensearch.action.support.replication.ClusterStateCreationUtils; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.block.ClusterBlock; +import org.opensearch.cluster.block.ClusterBlockException; +import org.opensearch.cluster.block.ClusterBlockLevel; +import org.opensearch.cluster.block.ClusterBlocks; +import org.opensearch.cluster.coordination.ClusterStateTermVersion; +import org.opensearch.cluster.metadata.IndexNameExpressionResolver; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodeRole; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.settings.SettingsFilter; +import org.opensearch.common.settings.SettingsModule; +import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.rest.RestStatus; +import org.opensearch.indices.IndicesService; +import org.opensearch.telemetry.tracing.noop.NoopTracer; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.test.transport.CapturingTransport; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportService; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; + +import java.util.Collections; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.concurrent.TimeUnit; + +import static java.util.Collections.emptyList; +import static java.util.Collections.emptySet; +import static org.opensearch.test.ClusterServiceUtils.createClusterService; +import static org.opensearch.test.ClusterServiceUtils.setState; +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Mockito.mock; + +public class GetMappingsActionTests extends OpenSearchTestCase { + private TransportService transportService; + private ClusterService clusterService; + private ThreadPool threadPool; + private SettingsFilter settingsFilter; + private final String indexName = "test_index"; + CapturingTransport capturingTransport = new CapturingTransport(); + private DiscoveryNode localNode; + private DiscoveryNode remoteNode; + private DiscoveryNode[] allNodes; + private TransportGetMappingsAction transportAction = null; + + @Before + public void setUp() throws Exception { + super.setUp(); + + settingsFilter = new SettingsModule(Settings.EMPTY, emptyList(), emptyList(), emptySet()).getSettingsFilter(); + threadPool = new TestThreadPool("GetIndexActionTests"); + clusterService = createClusterService(threadPool); + + transportService = capturingTransport.createTransportService( + clusterService.getSettings(), + threadPool, + TransportService.NOOP_TRANSPORT_INTERCEPTOR, + boundAddress -> clusterService.localNode(), + null, + emptySet(), + NoopTracer.INSTANCE + ); + transportService.start(); + transportService.acceptIncomingRequests(); + + localNode = new DiscoveryNode( + "local_node", + buildNewFakeTransportAddress(), + Collections.emptyMap(), + Collections.singleton(DiscoveryNodeRole.DATA_ROLE), + Version.CURRENT + ); + remoteNode = new DiscoveryNode( + "remote_node", + buildNewFakeTransportAddress(), + Collections.emptyMap(), + Collections.singleton(DiscoveryNodeRole.CLUSTER_MANAGER_ROLE), + Version.CURRENT + ); + allNodes = new DiscoveryNode[] { localNode, remoteNode }; + setState(clusterService, ClusterStateCreationUtils.state(localNode, remoteNode, allNodes)); + transportAction = new TransportGetMappingsAction( + GetMappingsActionTests.this.transportService, + GetMappingsActionTests.this.clusterService, + GetMappingsActionTests.this.threadPool, + new ActionFilters(emptySet()), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), + mock(IndicesService.class) + ); + + } + + @After + public void tearDown() throws Exception { + super.tearDown(); + clusterService.close(); + transportService.close(); + ThreadPool.terminate(threadPool, 30, TimeUnit.SECONDS); + } + + public void testGetTransportWithoutMatchingTerm() { + transportAction.execute(null, new GetMappingsRequest(), ActionListener.wrap(Assert::assertNotNull, exception -> { + throw new AssertionError(exception); + })); + assertThat(capturingTransport.capturedRequests().length, equalTo(1)); + CapturingTransport.CapturedRequest capturedRequest = capturingTransport.capturedRequests()[0]; + // mismatch term and version + GetTermVersionResponse termResp = new GetTermVersionResponse( + new ClusterStateTermVersion( + clusterService.state().getClusterName(), + clusterService.state().metadata().clusterUUID(), + clusterService.state().term() - 1, + clusterService.state().version() - 1 + ) + ); + capturingTransport.handleResponse(capturedRequest.requestId, termResp); + + assertThat(capturingTransport.capturedRequests().length, equalTo(2)); + CapturingTransport.CapturedRequest capturedRequest1 = capturingTransport.capturedRequests()[1]; + + capturingTransport.handleResponse(capturedRequest1.requestId, new GetMappingsResponse(new HashMap<>())); + } + + public void testGetTransportWithMatchingTerm() { + transportAction.execute(null, new GetMappingsRequest(), ActionListener.wrap(Assert::assertNotNull, exception -> { + throw new AssertionError(exception); + })); + assertThat(capturingTransport.capturedRequests().length, equalTo(1)); + CapturingTransport.CapturedRequest capturedRequest = capturingTransport.capturedRequests()[0]; + GetTermVersionResponse termResp = new GetTermVersionResponse( + new ClusterStateTermVersion( + clusterService.state().getClusterName(), + clusterService.state().metadata().clusterUUID(), + clusterService.state().term(), + clusterService.state().version() + ) + ); + capturingTransport.handleResponse(capturedRequest.requestId, termResp); + + // no more transport calls + assertThat(capturingTransport.capturedRequests().length, equalTo(1)); + } + + public void testGetTransportClusterBlockWithMatchingTerm() { + ClusterBlock readClusterBlock = new ClusterBlock( + 1, + "uuid", + "", + false, + true, + true, + RestStatus.OK, + EnumSet.of(ClusterBlockLevel.METADATA_READ) + ); + ClusterBlocks.Builder builder = ClusterBlocks.builder(); + builder.addGlobalBlock(readClusterBlock); + ClusterState metadataReadBlockedState = ClusterState.builder(ClusterStateCreationUtils.state(localNode, remoteNode, allNodes)) + .blocks(builder) + .build(); + setState(clusterService, metadataReadBlockedState); + + transportAction.execute( + null, + new GetMappingsRequest(), + ActionListener.wrap(response -> { throw new AssertionError(response); }, exception -> { + Assert.assertTrue(exception instanceof ClusterBlockException); + }) + ); + assertThat(capturingTransport.capturedRequests().length, equalTo(1)); + CapturingTransport.CapturedRequest capturedRequest = capturingTransport.capturedRequests()[0]; + GetTermVersionResponse termResp = new GetTermVersionResponse( + new ClusterStateTermVersion( + clusterService.state().getClusterName(), + clusterService.state().metadata().clusterUUID(), + clusterService.state().term(), + clusterService.state().version() + ) + ); + capturingTransport.handleResponse(capturedRequest.requestId, termResp); + + // no more transport calls + assertThat(capturingTransport.capturedRequests().length, equalTo(1)); + } +} From b35690c886f42d2ca01fa3081e80cb4ba4aa19d9 Mon Sep 17 00:00:00 2001 From: Sumit Bansal Date: Mon, 22 Jul 2024 22:03:37 +0530 Subject: [PATCH 10/37] Reduce logging in DEBUG for MasterService:run (#14795) * Reduce logging in DEBUG for MasteService:run by introducing short and long summary in Taskbatcher Signed-off-by: Sumit Bansal --- CHANGELOG.md | 1 + .../cluster/service/MasterService.java | 55 ++-- .../cluster/service/TaskBatcher.java | 28 +- .../cluster/service/MasterServiceTests.java | 285 ++++++++++++++++-- .../cluster/service/TaskBatcherTests.java | 3 +- 5 files changed, 309 insertions(+), 63 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index e77b183601674..adbb69ff72a0e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -22,6 +22,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Refactor remote-routing-table service inline with remote state interfaces([#14668](https://github.com/opensearch-project/OpenSearch/pull/14668)) - Add prefix mode verification setting for repository verification (([#14790](https://github.com/opensearch-project/OpenSearch/pull/14790))) - Optimize TransportNodesAction to not send DiscoveryNodes for NodeStats, NodesInfo and ClusterStats call ([14749](https://github.com/opensearch-project/OpenSearch/pull/14749)) +- Reduce logging in DEBUG for MasterService:run ([#14795](https://github.com/opensearch-project/OpenSearch/pull/14795)) - Enabling term version check on local state for all ClusterManager Read Transport Actions ([#14273](https://github.com/opensearch-project/OpenSearch/pull/14273)) ### Dependencies diff --git a/server/src/main/java/org/opensearch/cluster/service/MasterService.java b/server/src/main/java/org/opensearch/cluster/service/MasterService.java index 686e9793a8fd3..4ab8255df7658 100644 --- a/server/src/main/java/org/opensearch/cluster/service/MasterService.java +++ b/server/src/main/java/org/opensearch/cluster/service/MasterService.java @@ -84,6 +84,7 @@ import java.util.Objects; import java.util.Optional; import java.util.concurrent.TimeUnit; +import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -221,10 +222,10 @@ protected void onTimeout(List tasks, TimeValue timeout) { } @Override - protected void run(Object batchingKey, List tasks, String tasksSummary) { + protected void run(Object batchingKey, List tasks, Function taskSummaryGenerator) { ClusterStateTaskExecutor taskExecutor = (ClusterStateTaskExecutor) batchingKey; List updateTasks = (List) tasks; - runTasks(new TaskInputs(taskExecutor, updateTasks, tasksSummary)); + runTasks(new TaskInputs(taskExecutor, updateTasks, taskSummaryGenerator)); } class UpdateTask extends BatchedTask { @@ -297,26 +298,33 @@ public static boolean assertNotMasterUpdateThread(String reason) { } private void runTasks(TaskInputs taskInputs) { - final String summary = taskInputs.summary; + final String longSummary = logger.isTraceEnabled() ? taskInputs.taskSummaryGenerator.apply(true) : ""; + final String shortSummary = taskInputs.taskSummaryGenerator.apply(false); + if (!lifecycle.started()) { - logger.debug("processing [{}]: ignoring, cluster-manager service not started", summary); + logger.debug("processing [{}]: ignoring, cluster-manager service not started", shortSummary); return; } - logger.debug("executing cluster state update for [{}]", summary); + if (logger.isTraceEnabled()) { + logger.trace("executing cluster state update for [{}]", longSummary); + } else { + logger.debug("executing cluster state update for [{}]", shortSummary); + } + final ClusterState previousClusterState = state(); if (!previousClusterState.nodes().isLocalNodeElectedClusterManager() && taskInputs.runOnlyWhenClusterManager()) { - logger.debug("failing [{}]: local node is no longer cluster-manager", summary); + logger.debug("failing [{}]: local node is no longer cluster-manager", shortSummary); taskInputs.onNoLongerClusterManager(); return; } final long computationStartTime = threadPool.preciseRelativeTimeInNanos(); - final TaskOutputs taskOutputs = calculateTaskOutputs(taskInputs, previousClusterState); + final TaskOutputs taskOutputs = calculateTaskOutputs(taskInputs, previousClusterState, shortSummary); taskOutputs.notifyFailedTasks(); final TimeValue computationTime = getTimeSince(computationStartTime); - logExecutionTime(computationTime, "compute cluster state update", summary); + logExecutionTime(computationTime, "compute cluster state update", shortSummary); clusterManagerMetrics.recordLatency( clusterManagerMetrics.clusterStateComputeHistogram, @@ -328,17 +336,17 @@ private void runTasks(TaskInputs taskInputs) { final long notificationStartTime = threadPool.preciseRelativeTimeInNanos(); taskOutputs.notifySuccessfulTasksOnUnchangedClusterState(); final TimeValue executionTime = getTimeSince(notificationStartTime); - logExecutionTime(executionTime, "notify listeners on unchanged cluster state", summary); + logExecutionTime(executionTime, "notify listeners on unchanged cluster state", shortSummary); } else { final ClusterState newClusterState = taskOutputs.newClusterState; if (logger.isTraceEnabled()) { - logger.trace("cluster state updated, source [{}]\n{}", summary, newClusterState); + logger.trace("cluster state updated, source [{}]\n{}", longSummary, newClusterState); } else { - logger.debug("cluster state updated, version [{}], source [{}]", newClusterState.version(), summary); + logger.debug("cluster state updated, version [{}], source [{}]", newClusterState.version(), shortSummary); } final long publicationStartTime = threadPool.preciseRelativeTimeInNanos(); try { - ClusterChangedEvent clusterChangedEvent = new ClusterChangedEvent(summary, newClusterState, previousClusterState); + ClusterChangedEvent clusterChangedEvent = new ClusterChangedEvent(shortSummary, newClusterState, previousClusterState); // new cluster state, notify all listeners final DiscoveryNodes.Delta nodesDelta = clusterChangedEvent.nodesDelta(); if (nodesDelta.hasChanges() && logger.isInfoEnabled()) { @@ -346,7 +354,7 @@ private void runTasks(TaskInputs taskInputs) { if (nodesDeltaSummary.length() > 0) { logger.info( "{}, term: {}, version: {}, delta: {}", - summary, + shortSummary, newClusterState.term(), newClusterState.version(), nodesDeltaSummary @@ -357,7 +365,7 @@ private void runTasks(TaskInputs taskInputs) { logger.debug("publishing cluster state version [{}]", newClusterState.version()); publish(clusterChangedEvent, taskOutputs, publicationStartTime); } catch (Exception e) { - handleException(summary, publicationStartTime, newClusterState, e); + handleException(shortSummary, publicationStartTime, newClusterState, e); } } } @@ -452,8 +460,8 @@ private void handleException(String summary, long startTimeMillis, ClusterState // TODO: do we want to call updateTask.onFailure here? } - private TaskOutputs calculateTaskOutputs(TaskInputs taskInputs, ClusterState previousClusterState) { - ClusterTasksResult clusterTasksResult = executeTasks(taskInputs, previousClusterState); + private TaskOutputs calculateTaskOutputs(TaskInputs taskInputs, ClusterState previousClusterState, String taskSummary) { + ClusterTasksResult clusterTasksResult = executeTasks(taskInputs, previousClusterState, taskSummary); ClusterState newClusterState = patchVersions(previousClusterState, clusterTasksResult); return new TaskOutputs( taskInputs, @@ -897,7 +905,7 @@ public void onTimeout() { } } - private ClusterTasksResult executeTasks(TaskInputs taskInputs, ClusterState previousClusterState) { + private ClusterTasksResult executeTasks(TaskInputs taskInputs, ClusterState previousClusterState, String taskSummary) { ClusterTasksResult clusterTasksResult; try { List inputs = taskInputs.updateTasks.stream().map(tUpdateTask -> tUpdateTask.task).collect(Collectors.toList()); @@ -913,7 +921,7 @@ private ClusterTasksResult executeTasks(TaskInputs taskInputs, ClusterSt "failed to execute cluster state update (on version: [{}], uuid: [{}]) for [{}]\n{}{}{}", previousClusterState.version(), previousClusterState.stateUUID(), - taskInputs.summary, + taskSummary, previousClusterState.nodes(), previousClusterState.routingTable(), previousClusterState.getRoutingNodes() @@ -955,14 +963,19 @@ private List getNonFailedTasks(TaskInputs taskInputs, Cluste * Represents a set of tasks to be processed together with their executor */ private class TaskInputs { - final String summary; + final List updateTasks; final ClusterStateTaskExecutor executor; + final Function taskSummaryGenerator; - TaskInputs(ClusterStateTaskExecutor executor, List updateTasks, String summary) { - this.summary = summary; + TaskInputs( + ClusterStateTaskExecutor executor, + List updateTasks, + final Function taskSummaryGenerator + ) { this.executor = executor; this.updateTasks = updateTasks; + this.taskSummaryGenerator = taskSummaryGenerator; } boolean runOnlyWhenClusterManager() { diff --git a/server/src/main/java/org/opensearch/cluster/service/TaskBatcher.java b/server/src/main/java/org/opensearch/cluster/service/TaskBatcher.java index 5e58f495a16fb..3513bfffb7157 100644 --- a/server/src/main/java/org/opensearch/cluster/service/TaskBatcher.java +++ b/server/src/main/java/org/opensearch/cluster/service/TaskBatcher.java @@ -177,7 +177,6 @@ void runIfNotProcessed(BatchedTask updateTask) { // to give other tasks with different batching key a chance to execute. if (updateTask.processed.get() == false) { final List toExecute = new ArrayList<>(); - final Map> processTasksBySource = new HashMap<>(); // While removing task, need to remove task first from taskMap and then remove identity from identityMap. // Changing this order might lead to duplicate task during submission. LinkedHashSet pending = tasksPerBatchingKey.remove(updateTask.batchingKey); @@ -187,7 +186,6 @@ void runIfNotProcessed(BatchedTask updateTask) { if (task.processed.getAndSet(true) == false) { logger.trace("will process {}", task); toExecute.add(task); - processTasksBySource.computeIfAbsent(task.source, s -> new ArrayList<>()).add(task); } else { logger.trace("skipping {}, already processed", task); } @@ -195,22 +193,34 @@ void runIfNotProcessed(BatchedTask updateTask) { } if (toExecute.isEmpty() == false) { - final String tasksSummary = processTasksBySource.entrySet().stream().map(entry -> { - String tasks = updateTask.describeTasks(entry.getValue()); - return tasks.isEmpty() ? entry.getKey() : entry.getKey() + "[" + tasks + "]"; - }).reduce((s1, s2) -> s1 + ", " + s2).orElse(""); - + Function taskSummaryGenerator = (longSummaryRequired) -> { + if (longSummaryRequired == null || !longSummaryRequired) { + return buildShortSummary(updateTask.batchingKey, toExecute.size()); + } + final Map> processTasksBySource = new HashMap<>(); + for (final BatchedTask task : toExecute) { + processTasksBySource.computeIfAbsent(task.source, s -> new ArrayList<>()).add(task); + } + return processTasksBySource.entrySet().stream().map(entry -> { + String tasks = updateTask.describeTasks(entry.getValue()); + return tasks.isEmpty() ? entry.getKey() : entry.getKey() + "[" + tasks + "]"; + }).reduce((s1, s2) -> s1 + ", " + s2).orElse(""); + }; taskBatcherListener.onBeginProcessing(toExecute); - run(updateTask.batchingKey, toExecute, tasksSummary); + run(updateTask.batchingKey, toExecute, taskSummaryGenerator); } } } + private String buildShortSummary(final Object batchingKey, final int taskCount) { + return "Tasks batched with key: " + batchingKey.toString().split("\\$")[0] + " and count: " + taskCount; + } + /** * Action to be implemented by the specific batching implementation * All tasks have the given batching key. */ - protected abstract void run(Object batchingKey, List tasks, String tasksSummary); + protected abstract void run(Object batchingKey, List tasks, Function taskSummaryGenerator); /** * Represents a runnable task that supports batching. diff --git a/server/src/test/java/org/opensearch/cluster/service/MasterServiceTests.java b/server/src/test/java/org/opensearch/cluster/service/MasterServiceTests.java index 8c84ac365dfd1..7562dfc2e9d33 100644 --- a/server/src/test/java/org/opensearch/cluster/service/MasterServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/service/MasterServiceTests.java @@ -376,13 +376,13 @@ public void onFailure(String source, Exception e) {} } @TestLogging(value = "org.opensearch.cluster.service:TRACE", reason = "to ensure that we log cluster state events on TRACE level") - public void testClusterStateUpdateLogging() throws Exception { + public void testClusterStateUpdateLoggingWithTraceEnabled() throws Exception { try (MockLogAppender mockAppender = MockLogAppender.createForLoggers(LogManager.getLogger(MasterService.class))) { mockAppender.addExpectation( new MockLogAppender.SeenEventExpectation( "test1 start", MasterService.class.getCanonicalName(), - Level.DEBUG, + Level.TRACE, "executing cluster state update for [test1]" ) ); @@ -391,7 +391,7 @@ public void testClusterStateUpdateLogging() throws Exception { "test1 computation", MasterService.class.getCanonicalName(), Level.DEBUG, - "took [1s] to compute cluster state update for [test1]" + "took [1s] to compute cluster state update for [Tasks batched with key: org.opensearch.cluster.service.MasterServiceTests and count: 1]" ) ); mockAppender.addExpectation( @@ -399,7 +399,7 @@ public void testClusterStateUpdateLogging() throws Exception { "test1 notification", MasterService.class.getCanonicalName(), Level.DEBUG, - "took [0s] to notify listeners on unchanged cluster state for [test1]" + "took [0s] to notify listeners on unchanged cluster state for [Tasks batched with key: org.opensearch.cluster.service.MasterServiceTests and count: 1]" ) ); @@ -407,7 +407,7 @@ public void testClusterStateUpdateLogging() throws Exception { new MockLogAppender.SeenEventExpectation( "test2 start", MasterService.class.getCanonicalName(), - Level.DEBUG, + Level.TRACE, "executing cluster state update for [test2]" ) ); @@ -416,7 +416,7 @@ public void testClusterStateUpdateLogging() throws Exception { "test2 failure", MasterService.class.getCanonicalName(), Level.TRACE, - "failed to execute cluster state update (on version: [*], uuid: [*]) for [test2]*" + "failed to execute cluster state update (on version: [*], uuid: [*]) for [Tasks batched with key: org.opensearch.cluster.service.MasterServiceTests and count: 1]*" ) ); mockAppender.addExpectation( @@ -424,7 +424,7 @@ public void testClusterStateUpdateLogging() throws Exception { "test2 computation", MasterService.class.getCanonicalName(), Level.DEBUG, - "took [2s] to compute cluster state update for [test2]" + "took [2s] to compute cluster state update for [Tasks batched with key: org.opensearch.cluster.service.MasterServiceTests and count: 1]" ) ); mockAppender.addExpectation( @@ -432,7 +432,7 @@ public void testClusterStateUpdateLogging() throws Exception { "test2 notification", MasterService.class.getCanonicalName(), Level.DEBUG, - "took [0s] to notify listeners on unchanged cluster state for [test2]" + "took [0s] to notify listeners on unchanged cluster state for [Tasks batched with key: org.opensearch.cluster.service.MasterServiceTests and count: 1]" ) ); @@ -440,7 +440,7 @@ public void testClusterStateUpdateLogging() throws Exception { new MockLogAppender.SeenEventExpectation( "test3 start", MasterService.class.getCanonicalName(), - Level.DEBUG, + Level.TRACE, "executing cluster state update for [test3]" ) ); @@ -449,7 +449,7 @@ public void testClusterStateUpdateLogging() throws Exception { "test3 computation", MasterService.class.getCanonicalName(), Level.DEBUG, - "took [3s] to compute cluster state update for [test3]" + "took [3s] to compute cluster state update for [Tasks batched with key: org.opensearch.cluster.service.MasterServiceTests and count: 1]" ) ); mockAppender.addExpectation( @@ -457,7 +457,7 @@ public void testClusterStateUpdateLogging() throws Exception { "test3 notification", MasterService.class.getCanonicalName(), Level.DEBUG, - "took [4s] to notify listeners on successful publication of cluster state (version: *, uuid: *) for [test3]" + "took [4s] to notify listeners on successful publication of cluster state (version: *, uuid: *) for [Tasks batched with key: org.opensearch.cluster.service.MasterServiceTests and count: 1]" ) ); @@ -465,7 +465,7 @@ public void testClusterStateUpdateLogging() throws Exception { new MockLogAppender.SeenEventExpectation( "test4", MasterService.class.getCanonicalName(), - Level.DEBUG, + Level.TRACE, "executing cluster state update for [test4]" ) ); @@ -540,6 +540,171 @@ public void onFailure(String source, Exception e) { } } + @TestLogging(value = "org.opensearch.cluster.service:DEBUG", reason = "to ensure that we log cluster state events on DEBUG level") + public void testClusterStateUpdateLoggingWithDebugEnabled() throws Exception { + try (MockLogAppender mockAppender = MockLogAppender.createForLoggers(LogManager.getLogger(MasterService.class))) { + mockAppender.addExpectation( + new MockLogAppender.SeenEventExpectation( + "test1 start", + MasterService.class.getCanonicalName(), + Level.DEBUG, + "executing cluster state update for [Tasks batched with key: org.opensearch.cluster.service.MasterServiceTests and count: 1]" + ) + ); + mockAppender.addExpectation( + new MockLogAppender.SeenEventExpectation( + "test1 computation", + MasterService.class.getCanonicalName(), + Level.DEBUG, + "took [1s] to compute cluster state update for [Tasks batched with key: org.opensearch.cluster.service.MasterServiceTests and count: 1]" + ) + ); + mockAppender.addExpectation( + new MockLogAppender.SeenEventExpectation( + "test1 notification", + MasterService.class.getCanonicalName(), + Level.DEBUG, + "took [0s] to notify listeners on unchanged cluster state for [Tasks batched with key: org.opensearch.cluster.service.MasterServiceTests and count: 1]" + ) + ); + + mockAppender.addExpectation( + new MockLogAppender.SeenEventExpectation( + "test2 start", + MasterService.class.getCanonicalName(), + Level.DEBUG, + "executing cluster state update for [Tasks batched with key: org.opensearch.cluster.service.MasterServiceTests and count: 1]" + ) + ); + mockAppender.addExpectation( + new MockLogAppender.UnseenEventExpectation( + "test2 failure", + MasterService.class.getCanonicalName(), + Level.DEBUG, + "failed to execute cluster state update (on version: [*], uuid: [*]) for [Tasks batched with key: org.opensearch.cluster.service.MasterServiceTests and count: 1]*" + ) + ); + mockAppender.addExpectation( + new MockLogAppender.SeenEventExpectation( + "test2 computation", + MasterService.class.getCanonicalName(), + Level.DEBUG, + "took [2s] to compute cluster state update for [Tasks batched with key: org.opensearch.cluster.service.MasterServiceTests and count: 1]" + ) + ); + mockAppender.addExpectation( + new MockLogAppender.SeenEventExpectation( + "test2 notification", + MasterService.class.getCanonicalName(), + Level.DEBUG, + "took [0s] to notify listeners on unchanged cluster state for [Tasks batched with key: org.opensearch.cluster.service.MasterServiceTests and count: 1]" + ) + ); + + mockAppender.addExpectation( + new MockLogAppender.SeenEventExpectation( + "test3 start", + MasterService.class.getCanonicalName(), + Level.DEBUG, + "executing cluster state update for [Tasks batched with key: org.opensearch.cluster.service.MasterServiceTests and count: 1]" + ) + ); + mockAppender.addExpectation( + new MockLogAppender.SeenEventExpectation( + "test3 computation", + MasterService.class.getCanonicalName(), + Level.DEBUG, + "took [3s] to compute cluster state update for [Tasks batched with key: org.opensearch.cluster.service.MasterServiceTests and count: 1]" + ) + ); + mockAppender.addExpectation( + new MockLogAppender.SeenEventExpectation( + "test3 notification", + MasterService.class.getCanonicalName(), + Level.DEBUG, + "took [4s] to notify listeners on successful publication of cluster state (version: *, uuid: *) for [Tasks batched with key: org.opensearch.cluster.service.MasterServiceTests and count: 1]" + ) + ); + + mockAppender.addExpectation( + new MockLogAppender.SeenEventExpectation( + "test4", + MasterService.class.getCanonicalName(), + Level.DEBUG, + "executing cluster state update for [Tasks batched with key: org.opensearch.cluster.service.MasterServiceTests and count: 1]" + ) + ); + + try (ClusterManagerService clusterManagerService = createClusterManagerService(true)) { + clusterManagerService.submitStateUpdateTask("test1", new ClusterStateUpdateTask() { + @Override + public ClusterState execute(ClusterState currentState) { + timeDiffInMillis += TimeValue.timeValueSeconds(1).millis(); + return currentState; + } + + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {} + + @Override + public void onFailure(String source, Exception e) { + fail(); + } + }); + clusterManagerService.submitStateUpdateTask("test2", new ClusterStateUpdateTask() { + @Override + public ClusterState execute(ClusterState currentState) { + timeDiffInMillis += TimeValue.timeValueSeconds(2).millis(); + throw new IllegalArgumentException("Testing handling of exceptions in the cluster state task"); + } + + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + fail(); + } + + @Override + public void onFailure(String source, Exception e) {} + }); + clusterManagerService.submitStateUpdateTask("test3", new ClusterStateUpdateTask() { + @Override + public ClusterState execute(ClusterState currentState) { + timeDiffInMillis += TimeValue.timeValueSeconds(3).millis(); + return ClusterState.builder(currentState).incrementVersion().build(); + } + + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + timeDiffInMillis += TimeValue.timeValueSeconds(4).millis(); + } + + @Override + public void onFailure(String source, Exception e) { + fail(); + } + }); + clusterManagerService.submitStateUpdateTask("test4", new ClusterStateUpdateTask() { + @Override + public ClusterState execute(ClusterState currentState) { + return currentState; + } + + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {} + + @Override + public void onFailure(String source, Exception e) { + fail(); + } + }); + assertBusy(mockAppender::assertAllExpectationsMatched); + // verify stats values after state is published + assertEquals(1, clusterManagerService.getClusterStateStats().getUpdateSuccess()); + assertEquals(0, clusterManagerService.getClusterStateStats().getUpdateFailed()); + } + } + } + public void testClusterStateBatchedUpdates() throws BrokenBarrierException, InterruptedException { AtomicInteger counter = new AtomicInteger(); class Task { @@ -1073,7 +1238,7 @@ public void testLongClusterStateUpdateLogging() throws Exception { "test2", MasterService.class.getCanonicalName(), Level.WARN, - "*took [*], which is over [10s], to compute cluster state update for [test2]" + "*took [*], which is over [10s], to compute cluster state update for [Tasks batched with key: org.opensearch.cluster.service.MasterServiceTests and count: 1]" ) ); mockAppender.addExpectation( @@ -1081,7 +1246,7 @@ public void testLongClusterStateUpdateLogging() throws Exception { "test3", MasterService.class.getCanonicalName(), Level.WARN, - "*took [*], which is over [10s], to compute cluster state update for [test3]" + "*took [*], which is over [10s], to compute cluster state update for [Tasks batched with key: org.opensearch.cluster.service.MasterServiceTests and count: 1]" ) ); mockAppender.addExpectation( @@ -1089,7 +1254,7 @@ public void testLongClusterStateUpdateLogging() throws Exception { "test4", MasterService.class.getCanonicalName(), Level.WARN, - "*took [*], which is over [10s], to compute cluster state update for [test4]" + "*took [*], which is over [10s], to compute cluster state update for [Tasks batched with key: org.opensearch.cluster.service.MasterServiceTests and count: 1]" ) ); mockAppender.addExpectation( @@ -1100,14 +1265,6 @@ public void testLongClusterStateUpdateLogging() throws Exception { "*took*test5*" ) ); - mockAppender.addExpectation( - new MockLogAppender.SeenEventExpectation( - "test6 should log due to slow and failing publication", - MasterService.class.getCanonicalName(), - Level.WARN, - "took [*] and then failed to publish updated cluster state (version: *, uuid: *) for [test6]:*" - ) - ); try ( ClusterManagerService clusterManagerService = new ClusterManagerService( @@ -1139,19 +1296,13 @@ public void testLongClusterStateUpdateLogging() throws Exception { Settings.EMPTY ).millis() + randomLongBetween(1, 1000000); } - if (event.source().contains("test6")) { - timeDiffInMillis += ClusterManagerService.CLUSTER_MANAGER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.get( - Settings.EMPTY - ).millis() + randomLongBetween(1, 1000000); - throw new OpenSearchException("simulated error during slow publication which should trigger logging"); - } clusterStateRef.set(event.state()); publishListener.onResponse(null); }); clusterManagerService.setClusterStateSupplier(clusterStateRef::get); clusterManagerService.start(); - final CountDownLatch latch = new CountDownLatch(6); + final CountDownLatch latch = new CountDownLatch(5); final CountDownLatch processedFirstTask = new CountDownLatch(1); clusterManagerService.submitStateUpdateTask("test1", new ClusterStateUpdateTask() { @Override @@ -1249,7 +1400,77 @@ public void onFailure(String source, Exception e) { fail(); } }); + // Additional update task to make sure all previous logging made it to the loggerName + // We don't check logging for this on since there is no guarantee that it will occur before our check clusterManagerService.submitStateUpdateTask("test6", new ClusterStateUpdateTask() { + @Override + public ClusterState execute(ClusterState currentState) { + return currentState; + } + + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + latch.countDown(); + } + + @Override + public void onFailure(String source, Exception e) { + fail(); + } + }); + latch.await(); + } + mockAppender.assertAllExpectationsMatched(); + } + } + + @TestLogging(value = "org.opensearch.cluster.service:WARN", reason = "to ensure that we log failed cluster state events on WARN level") + public void testLongClusterStateUpdateLoggingForFailedPublication() throws Exception { + try (MockLogAppender mockAppender = MockLogAppender.createForLoggers(LogManager.getLogger(MasterService.class))) { + mockAppender.addExpectation( + new MockLogAppender.SeenEventExpectation( + "test1 should log due to slow and failing publication", + MasterService.class.getCanonicalName(), + Level.WARN, + "took [*] and then failed to publish updated cluster state (version: *, uuid: *) for [Tasks batched with key: org.opensearch.cluster.service.MasterServiceTests and count: 1]:*" + ) + ); + + try ( + ClusterManagerService clusterManagerService = new ClusterManagerService( + Settings.builder() + .put(ClusterName.CLUSTER_NAME_SETTING.getKey(), MasterServiceTests.class.getSimpleName()) + .put(Node.NODE_NAME_SETTING.getKey(), "test_node") + .build(), + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), + threadPool, + new ClusterManagerMetrics(NoopMetricsRegistry.INSTANCE) + ) + ) { + + final DiscoveryNode localNode = new DiscoveryNode( + "node1", + buildNewFakeTransportAddress(), + emptyMap(), + emptySet(), + Version.CURRENT + ); + final ClusterState initialClusterState = ClusterState.builder(new ClusterName(MasterServiceTests.class.getSimpleName())) + .nodes(DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId()).masterNodeId(localNode.getId())) + .blocks(ClusterBlocks.EMPTY_CLUSTER_BLOCK) + .build(); + final AtomicReference clusterStateRef = new AtomicReference<>(initialClusterState); + clusterManagerService.setClusterStatePublisher((event, publishListener, ackListener) -> { + timeDiffInMillis += ClusterManagerService.CLUSTER_MANAGER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.get( + Settings.EMPTY + ).millis() + randomLongBetween(1, 1000000); + throw new OpenSearchException("simulated error during slow publication which should trigger logging"); + }); + clusterManagerService.setClusterStateSupplier(clusterStateRef::get); + clusterManagerService.start(); + + final CountDownLatch latch = new CountDownLatch(1); + clusterManagerService.submitStateUpdateTask("test1", new ClusterStateUpdateTask() { @Override public ClusterState execute(ClusterState currentState) { return ClusterState.builder(currentState).incrementVersion().build(); @@ -1262,12 +1483,12 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS @Override public void onFailure(String source, Exception e) { - fail(); // maybe we should notify here? + fail(); } }); // Additional update task to make sure all previous logging made it to the loggerName // We don't check logging for this on since there is no guarantee that it will occur before our check - clusterManagerService.submitStateUpdateTask("test7", new ClusterStateUpdateTask() { + clusterManagerService.submitStateUpdateTask("test2", new ClusterStateUpdateTask() { @Override public ClusterState execute(ClusterState currentState) { return currentState; diff --git a/server/src/test/java/org/opensearch/cluster/service/TaskBatcherTests.java b/server/src/test/java/org/opensearch/cluster/service/TaskBatcherTests.java index b0916ce9236f7..0ebcb51b557ae 100644 --- a/server/src/test/java/org/opensearch/cluster/service/TaskBatcherTests.java +++ b/server/src/test/java/org/opensearch/cluster/service/TaskBatcherTests.java @@ -55,6 +55,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.Semaphore; +import java.util.function.Function; import java.util.stream.Collectors; import static org.hamcrest.Matchers.containsString; @@ -78,7 +79,7 @@ static class TestTaskBatcher extends TaskBatcher { } @Override - protected void run(Object batchingKey, List tasks, String tasksSummary) { + protected void run(Object batchingKey, List tasks, Function taskSummaryGenerator) { List updateTasks = (List) tasks; ((TestExecutor) batchingKey).execute(updateTasks.stream().map(t -> t.task).collect(Collectors.toList())); updateTasks.forEach(updateTask -> updateTask.listener.processed(updateTask.source)); From 45c5f8d3154db84e28a6dc201d9b31ff91288fde Mon Sep 17 00:00:00 2001 From: Daniel Widdis Date: Mon, 22 Jul 2024 09:57:00 -0700 Subject: [PATCH 11/37] Add SplitResponseProcessor to Search Pipelines (#14800) * Add SplitResponseProcessor for search pipelines Signed-off-by: Daniel Widdis * Register the split processor factory Signed-off-by: Daniel Widdis * Address code review comments Signed-off-by: Daniel Widdis * Avoid list copy by casting array Signed-off-by: Daniel Widdis --------- Signed-off-by: Daniel Widdis --- CHANGELOG.md | 1 + .../SearchPipelineCommonModulePlugin.java | 4 +- .../common/SplitResponseProcessor.java | 162 +++++++++++++ ...SearchPipelineCommonModulePluginTests.java | 2 +- .../common/SplitResponseProcessorTests.java | 213 ++++++++++++++++++ 5 files changed, 380 insertions(+), 2 deletions(-) create mode 100644 modules/search-pipeline-common/src/main/java/org/opensearch/search/pipeline/common/SplitResponseProcessor.java create mode 100644 modules/search-pipeline-common/src/test/java/org/opensearch/search/pipeline/common/SplitResponseProcessorTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index adbb69ff72a0e..e32b6de84a195 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -21,6 +21,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add Plugin interface for loading application based configuration templates (([#14659](https://github.com/opensearch-project/OpenSearch/issues/14659))) - Refactor remote-routing-table service inline with remote state interfaces([#14668](https://github.com/opensearch-project/OpenSearch/pull/14668)) - Add prefix mode verification setting for repository verification (([#14790](https://github.com/opensearch-project/OpenSearch/pull/14790))) +- Add SplitResponseProcessor to Search Pipelines (([#14800](https://github.com/opensearch-project/OpenSearch/issues/14800))) - Optimize TransportNodesAction to not send DiscoveryNodes for NodeStats, NodesInfo and ClusterStats call ([14749](https://github.com/opensearch-project/OpenSearch/pull/14749)) - Reduce logging in DEBUG for MasterService:run ([#14795](https://github.com/opensearch-project/OpenSearch/pull/14795)) - Enabling term version check on local state for all ClusterManager Read Transport Actions ([#14273](https://github.com/opensearch-project/OpenSearch/pull/14273)) diff --git a/modules/search-pipeline-common/src/main/java/org/opensearch/search/pipeline/common/SearchPipelineCommonModulePlugin.java b/modules/search-pipeline-common/src/main/java/org/opensearch/search/pipeline/common/SearchPipelineCommonModulePlugin.java index 1574621a8200e..d05101da2817c 100644 --- a/modules/search-pipeline-common/src/main/java/org/opensearch/search/pipeline/common/SearchPipelineCommonModulePlugin.java +++ b/modules/search-pipeline-common/src/main/java/org/opensearch/search/pipeline/common/SearchPipelineCommonModulePlugin.java @@ -96,7 +96,9 @@ public Map> getResponseProces TruncateHitsResponseProcessor.TYPE, new TruncateHitsResponseProcessor.Factory(), CollapseResponseProcessor.TYPE, - new CollapseResponseProcessor.Factory() + new CollapseResponseProcessor.Factory(), + SplitResponseProcessor.TYPE, + new SplitResponseProcessor.Factory() ) ); } diff --git a/modules/search-pipeline-common/src/main/java/org/opensearch/search/pipeline/common/SplitResponseProcessor.java b/modules/search-pipeline-common/src/main/java/org/opensearch/search/pipeline/common/SplitResponseProcessor.java new file mode 100644 index 0000000000000..0762f8f59b76e --- /dev/null +++ b/modules/search-pipeline-common/src/main/java/org/opensearch/search/pipeline/common/SplitResponseProcessor.java @@ -0,0 +1,162 @@ +/* + * 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.pipeline.common; + +import org.opensearch.action.search.SearchRequest; +import org.opensearch.action.search.SearchResponse; +import org.opensearch.common.collect.Tuple; +import org.opensearch.common.document.DocumentField; +import org.opensearch.common.xcontent.XContentHelper; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.xcontent.MediaType; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.ingest.ConfigurationUtils; +import org.opensearch.search.SearchHit; +import org.opensearch.search.pipeline.AbstractProcessor; +import org.opensearch.search.pipeline.Processor; +import org.opensearch.search.pipeline.SearchResponseProcessor; + +import java.util.Arrays; +import java.util.Map; +import java.util.Objects; + +/** + * Processor that sorts an array of items. + * Throws exception is the specified field is not an array. + */ +public class SplitResponseProcessor extends AbstractProcessor implements SearchResponseProcessor { + /** Key to reference this processor type from a search pipeline. */ + public static final String TYPE = "split"; + /** Key defining the string field to be split. */ + public static final String SPLIT_FIELD = "field"; + /** Key defining the delimiter used to split the string. This can be a regular expression pattern. */ + public static final String SEPARATOR = "separator"; + /** Optional key for handling empty trailing fields. */ + public static final String PRESERVE_TRAILING = "preserve_trailing"; + /** Optional key to put the split values in a different field. */ + public static final String TARGET_FIELD = "target_field"; + + private final String splitField; + private final String separator; + private final boolean preserveTrailing; + private final String targetField; + + SplitResponseProcessor( + String tag, + String description, + boolean ignoreFailure, + String splitField, + String separator, + boolean preserveTrailing, + String targetField + ) { + super(tag, description, ignoreFailure); + this.splitField = Objects.requireNonNull(splitField); + this.separator = Objects.requireNonNull(separator); + this.preserveTrailing = preserveTrailing; + this.targetField = targetField == null ? splitField : targetField; + } + + /** + * Getter function for splitField + * @return sortField + */ + public String getSplitField() { + return splitField; + } + + /** + * Getter function for separator + * @return separator + */ + public String getSeparator() { + return separator; + } + + /** + * Getter function for preserveTrailing + * @return preserveTrailing; + */ + public boolean isPreserveTrailing() { + return preserveTrailing; + } + + /** + * Getter function for targetField + * @return targetField + */ + public String getTargetField() { + return targetField; + } + + @Override + public String getType() { + return TYPE; + } + + @Override + public SearchResponse processResponse(SearchRequest request, SearchResponse response) throws Exception { + SearchHit[] hits = response.getHits().getHits(); + for (SearchHit hit : hits) { + Map fields = hit.getFields(); + if (fields.containsKey(splitField)) { + DocumentField docField = hit.getFields().get(splitField); + if (docField == null) { + throw new IllegalArgumentException("field [" + splitField + "] is null, cannot split."); + } + Object val = docField.getValue(); + if (val == null || !String.class.isAssignableFrom(val.getClass())) { + throw new IllegalArgumentException("field [" + splitField + "] is not a string, cannot split"); + } + Object[] strings = ((String) val).split(separator, preserveTrailing ? -1 : 0); + hit.setDocumentField(targetField, new DocumentField(targetField, Arrays.asList(strings))); + } + if (hit.hasSource()) { + BytesReference sourceRef = hit.getSourceRef(); + Tuple> typeAndSourceMap = XContentHelper.convertToMap( + sourceRef, + false, + (MediaType) null + ); + + Map sourceAsMap = typeAndSourceMap.v2(); + if (sourceAsMap.containsKey(splitField)) { + Object val = sourceAsMap.get(splitField); + if (val instanceof String) { + Object[] strings = ((String) val).split(separator, preserveTrailing ? -1 : 0); + sourceAsMap.put(targetField, Arrays.asList(strings)); + } + XContentBuilder builder = XContentBuilder.builder(typeAndSourceMap.v1().xContent()); + builder.map(sourceAsMap); + hit.sourceRef(BytesReference.bytes(builder)); + } + } + } + return response; + } + + static class Factory implements Processor.Factory { + + @Override + public SplitResponseProcessor create( + Map> processorFactories, + String tag, + String description, + boolean ignoreFailure, + Map config, + PipelineContext pipelineContext + ) { + String splitField = ConfigurationUtils.readStringProperty(TYPE, tag, config, SPLIT_FIELD); + String separator = ConfigurationUtils.readStringProperty(TYPE, tag, config, SEPARATOR); + boolean preserveTrailing = ConfigurationUtils.readBooleanProperty(TYPE, tag, config, PRESERVE_TRAILING, false); + String targetField = ConfigurationUtils.readStringProperty(TYPE, tag, config, TARGET_FIELD, splitField); + return new SplitResponseProcessor(tag, description, ignoreFailure, splitField, separator, preserveTrailing, targetField); + } + } +} diff --git a/modules/search-pipeline-common/src/test/java/org/opensearch/search/pipeline/common/SearchPipelineCommonModulePluginTests.java b/modules/search-pipeline-common/src/test/java/org/opensearch/search/pipeline/common/SearchPipelineCommonModulePluginTests.java index 519468ebe17ff..d4f9ae2490a10 100644 --- a/modules/search-pipeline-common/src/test/java/org/opensearch/search/pipeline/common/SearchPipelineCommonModulePluginTests.java +++ b/modules/search-pipeline-common/src/test/java/org/opensearch/search/pipeline/common/SearchPipelineCommonModulePluginTests.java @@ -82,7 +82,7 @@ public void testAllowlistNotSpecified() throws IOException { try (SearchPipelineCommonModulePlugin plugin = new SearchPipelineCommonModulePlugin()) { assertEquals(Set.of("oversample", "filter_query", "script"), plugin.getRequestProcessors(createParameters(settings)).keySet()); assertEquals( - Set.of("rename_field", "truncate_hits", "collapse"), + Set.of("rename_field", "truncate_hits", "collapse", "split"), plugin.getResponseProcessors(createParameters(settings)).keySet() ); assertEquals(Set.of(), plugin.getSearchPhaseResultsProcessors(createParameters(settings)).keySet()); diff --git a/modules/search-pipeline-common/src/test/java/org/opensearch/search/pipeline/common/SplitResponseProcessorTests.java b/modules/search-pipeline-common/src/test/java/org/opensearch/search/pipeline/common/SplitResponseProcessorTests.java new file mode 100644 index 0000000000000..fcbc8ccf43cff --- /dev/null +++ b/modules/search-pipeline-common/src/test/java/org/opensearch/search/pipeline/common/SplitResponseProcessorTests.java @@ -0,0 +1,213 @@ +/* + * 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.java + * compatible open source license. + */ + +package org.opensearch.search.pipeline.common; + +import org.apache.lucene.search.TotalHits; +import org.opensearch.OpenSearchParseException; +import org.opensearch.action.search.SearchRequest; +import org.opensearch.action.search.SearchResponse; +import org.opensearch.action.search.SearchResponseSections; +import org.opensearch.common.document.DocumentField; +import org.opensearch.core.common.bytes.BytesArray; +import org.opensearch.index.query.QueryBuilder; +import org.opensearch.index.query.TermQueryBuilder; +import org.opensearch.ingest.RandomDocumentPicks; +import org.opensearch.search.SearchHit; +import org.opensearch.search.SearchHits; +import org.opensearch.search.builder.SearchSourceBuilder; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class SplitResponseProcessorTests extends OpenSearchTestCase { + + private static final String NO_TRAILING = "one,two,three"; + private static final String TRAILING = "alpha,beta,gamma,"; + private static final String REGEX_DELIM = "one1two2three"; + + private SearchRequest createDummyRequest() { + QueryBuilder query = new TermQueryBuilder("field", "value"); + SearchSourceBuilder source = new SearchSourceBuilder().query(query); + return new SearchRequest().source(source); + } + + private SearchResponse createTestResponse() { + SearchHit[] hits = new SearchHit[2]; + + // one response with source + Map csvMap = new HashMap<>(); + csvMap.put("csv", new DocumentField("csv", List.of(NO_TRAILING))); + hits[0] = new SearchHit(0, "doc 1", csvMap, Collections.emptyMap()); + hits[0].sourceRef(new BytesArray("{ \"csv\" : \"" + NO_TRAILING + "\" }")); + hits[0].score(1f); + + // one without source + csvMap = new HashMap<>(); + csvMap.put("csv", new DocumentField("csv", List.of(TRAILING))); + hits[1] = new SearchHit(1, "doc 2", csvMap, Collections.emptyMap()); + hits[1].score(2f); + + SearchHits searchHits = new SearchHits(hits, new TotalHits(2, TotalHits.Relation.EQUAL_TO), 2); + SearchResponseSections searchResponseSections = new SearchResponseSections(searchHits, null, null, false, false, null, 0); + return new SearchResponse(searchResponseSections, null, 1, 1, 0, 10, null, null); + } + + private SearchResponse createTestResponseRegex() { + SearchHit[] hits = new SearchHit[1]; + + Map dsvMap = new HashMap<>(); + dsvMap.put("dsv", new DocumentField("dsv", List.of(REGEX_DELIM))); + hits[0] = new SearchHit(0, "doc 1", dsvMap, Collections.emptyMap()); + hits[0].sourceRef(new BytesArray("{ \"dsv\" : \"" + REGEX_DELIM + "\" }")); + hits[0].score(1f); + + SearchHits searchHits = new SearchHits(hits, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1); + SearchResponseSections searchResponseSections = new SearchResponseSections(searchHits, null, null, false, false, null, 0); + return new SearchResponse(searchResponseSections, null, 1, 1, 0, 10, null, null); + } + + private SearchResponse createTestResponseNullField() { + SearchHit[] hits = new SearchHit[1]; + + Map map = new HashMap<>(); + map.put("csv", null); + hits[0] = new SearchHit(0, "doc 1", map, Collections.emptyMap()); + hits[0].sourceRef(new BytesArray("{ \"csv\" : null }")); + hits[0].score(1f); + + SearchHits searchHits = new SearchHits(hits, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1); + SearchResponseSections searchResponseSections = new SearchResponseSections(searchHits, null, null, false, false, null, 0); + return new SearchResponse(searchResponseSections, null, 1, 1, 0, 10, null, null); + } + + private SearchResponse createTestResponseEmptyList() { + SearchHit[] hits = new SearchHit[1]; + + Map map = new HashMap<>(); + map.put("empty", new DocumentField("empty", List.of())); + hits[0] = new SearchHit(0, "doc 1", map, Collections.emptyMap()); + hits[0].sourceRef(new BytesArray("{ \"empty\" : [] }")); + hits[0].score(1f); + + SearchHits searchHits = new SearchHits(hits, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1); + SearchResponseSections searchResponseSections = new SearchResponseSections(searchHits, null, null, false, false, null, 0); + return new SearchResponse(searchResponseSections, null, 1, 1, 0, 10, null, null); + } + + private SearchResponse createTestResponseNotString() { + SearchHit[] hits = new SearchHit[1]; + + Map piMap = new HashMap<>(); + piMap.put("maps", new DocumentField("maps", List.of(Map.of("foo", "I'm the Map!")))); + hits[0] = new SearchHit(0, "doc 1", piMap, Collections.emptyMap()); + hits[0].sourceRef(new BytesArray("{ \"maps\" : [{ \"foo\" : \"I'm the Map!\"}]] }")); + hits[0].score(1f); + + SearchHits searchHits = new SearchHits(hits, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1); + SearchResponseSections searchResponseSections = new SearchResponseSections(searchHits, null, null, false, false, null, 0); + return new SearchResponse(searchResponseSections, null, 1, 1, 0, 10, null, null); + } + + public void testSplitResponse() throws Exception { + SearchRequest request = createDummyRequest(); + + SplitResponseProcessor splitResponseProcessor = new SplitResponseProcessor(null, null, false, "csv", ",", false, "split"); + SearchResponse response = createTestResponse(); + SearchResponse splitResponse = splitResponseProcessor.processResponse(request, response); + + assertEquals(response.getHits(), splitResponse.getHits()); + + assertEquals(NO_TRAILING, splitResponse.getHits().getHits()[0].field("csv").getValue()); + assertEquals(List.of("one", "two", "three"), splitResponse.getHits().getHits()[0].field("split").getValues()); + Map map = splitResponse.getHits().getHits()[0].getSourceAsMap(); + assertNotNull(map); + assertEquals(List.of("one", "two", "three"), map.get("split")); + + assertEquals(TRAILING, splitResponse.getHits().getHits()[1].field("csv").getValue()); + assertEquals(List.of("alpha", "beta", "gamma"), splitResponse.getHits().getHits()[1].field("split").getValues()); + assertNull(splitResponse.getHits().getHits()[1].getSourceAsMap()); + } + + public void testSplitResponseRegex() throws Exception { + SearchRequest request = createDummyRequest(); + + SplitResponseProcessor splitResponseProcessor = new SplitResponseProcessor(null, null, false, "dsv", "\\d", false, "split"); + SearchResponse response = createTestResponseRegex(); + SearchResponse splitResponse = splitResponseProcessor.processResponse(request, response); + + assertEquals(response.getHits(), splitResponse.getHits()); + + assertEquals(REGEX_DELIM, splitResponse.getHits().getHits()[0].field("dsv").getValue()); + assertEquals(List.of("one", "two", "three"), splitResponse.getHits().getHits()[0].field("split").getValues()); + Map map = splitResponse.getHits().getHits()[0].getSourceAsMap(); + assertNotNull(map); + assertEquals(List.of("one", "two", "three"), map.get("split")); + } + + public void testSplitResponseSameField() throws Exception { + SearchRequest request = createDummyRequest(); + + SplitResponseProcessor splitResponseProcessor = new SplitResponseProcessor(null, null, false, "csv", ",", true, null); + SearchResponse response = createTestResponse(); + SearchResponse splitResponse = splitResponseProcessor.processResponse(request, response); + + assertEquals(response.getHits(), splitResponse.getHits()); + assertEquals(List.of("one", "two", "three"), splitResponse.getHits().getHits()[0].field("csv").getValues()); + assertEquals(List.of("alpha", "beta", "gamma", ""), splitResponse.getHits().getHits()[1].field("csv").getValues()); + } + + public void testSplitResponseEmptyList() { + SearchRequest request = createDummyRequest(); + + SplitResponseProcessor splitResponseProcessor = new SplitResponseProcessor(null, null, false, "empty", ",", false, null); + assertThrows(IllegalArgumentException.class, () -> splitResponseProcessor.processResponse(request, createTestResponseEmptyList())); + } + + public void testNullField() { + SearchRequest request = createDummyRequest(); + + SplitResponseProcessor splitResponseProcessor = new SplitResponseProcessor(null, null, false, "csv", ",", false, null); + + assertThrows(IllegalArgumentException.class, () -> splitResponseProcessor.processResponse(request, createTestResponseNullField())); + } + + public void testNotStringField() { + SearchRequest request = createDummyRequest(); + + SplitResponseProcessor splitResponseProcessor = new SplitResponseProcessor(null, null, false, "maps", ",", false, null); + + assertThrows(IllegalArgumentException.class, () -> splitResponseProcessor.processResponse(request, createTestResponseNotString())); + } + + public void testFactory() { + String splitField = RandomDocumentPicks.randomFieldName(random()); + String targetField = RandomDocumentPicks.randomFieldName(random()); + Map config = new HashMap<>(); + config.put("field", splitField); + config.put("separator", ","); + config.put("preserve_trailing", true); + config.put("target_field", targetField); + + SplitResponseProcessor.Factory factory = new SplitResponseProcessor.Factory(); + SplitResponseProcessor processor = factory.create(Collections.emptyMap(), null, null, false, config, null); + assertEquals("split", processor.getType()); + assertEquals(splitField, processor.getSplitField()); + assertEquals(",", processor.getSeparator()); + assertTrue(processor.isPreserveTrailing()); + assertEquals(targetField, processor.getTargetField()); + + expectThrows( + OpenSearchParseException.class, + () -> factory.create(Collections.emptyMap(), null, null, false, Collections.emptyMap(), null) + ); + } +} From b9d5804a7868feb1214caf8a7c02bcd7117a3b3a Mon Sep 17 00:00:00 2001 From: shailendra0811 <167273922+shailendra0811@users.noreply.github.com> Date: Mon, 22 Jul 2024 23:56:53 +0530 Subject: [PATCH 12/37] Add integration tests for RemoteRoutingTable Service. (#14631) Signed-off-by: Shailendra Singh --- .../RemoteClusterStateCleanupManagerIT.java | 135 ++++++++ .../remote/RemoteRoutingTableServiceIT.java | 297 ++++++++++++++++++ .../RemoteStoreBaseIntegTestCase.java | 17 + .../coordination/PersistedStateStats.java | 4 + .../PersistedStateStatsTests.java | 62 ++++ .../test/OpenSearchIntegTestCase.java | 121 +++++++ 6 files changed, 636 insertions(+) create mode 100644 server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteRoutingTableServiceIT.java create mode 100644 server/src/test/java/org/opensearch/cluster/coordination/PersistedStateStatsTests.java diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java index 5074971ab1a1f..7d2e24c777da3 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerIT.java @@ -8,9 +8,17 @@ package org.opensearch.gateway.remote; +import org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest; +import org.opensearch.action.admin.cluster.node.stats.NodesStatsResponse; import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; +import org.opensearch.cluster.coordination.PersistedStateStats; +import org.opensearch.cluster.routing.IndexRoutingTable; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.settings.Settings; +import org.opensearch.discovery.DiscoveryStats; +import org.opensearch.gateway.remote.model.RemoteRoutingTableBlobStore; +import org.opensearch.index.remote.RemoteStoreEnums; +import org.opensearch.index.remote.RemoteStorePathStrategy; import org.opensearch.remotestore.RemoteStoreBaseIntegTestCase; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.blobstore.BlobStoreRepository; @@ -18,21 +26,29 @@ import org.junit.Before; import java.nio.charset.StandardCharsets; +import java.nio.file.Path; +import java.util.ArrayList; import java.util.Base64; +import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import static org.opensearch.common.util.FeatureFlags.REMOTE_PUBLICATION_EXPERIMENTAL; import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.CLUSTER_STATE_CLEANUP_INTERVAL_DEFAULT; import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING; import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.RETAINED_MANIFESTS; import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.SKIP_CLEANUP_STATE_CHANGES; import static org.opensearch.gateway.remote.RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING; +import static org.opensearch.gateway.remote.routingtable.RemoteIndexRoutingTable.INDEX_ROUTING_TABLE; import static org.opensearch.indices.IndicesService.CLUSTER_DEFAULT_INDEX_REFRESH_INTERVAL_SETTING; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY; @OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) public class RemoteClusterStateCleanupManagerIT extends RemoteStoreBaseIntegTestCase { private static final String INDEX_NAME = "test-index"; + private final RemoteStoreEnums.PathType pathType = RemoteStoreEnums.PathType.HASHED_PREFIX; @Before public void setup() { @@ -52,6 +68,11 @@ private Map initialTestSetup(int shardCount, int replicaCount, int return indexStats; } + private void initialTestSetup(int shardCount, int replicaCount, int dataNodeCount, int clusterManagerNodeCount, Settings settings) { + prepareCluster(clusterManagerNodeCount, dataNodeCount, INDEX_NAME, replicaCount, shardCount, settings); + ensureGreen(INDEX_NAME); + } + public void testRemoteCleanupTaskUpdated() { int shardCount = randomIntBetween(1, 2); int replicaCount = 1; @@ -144,6 +165,102 @@ public void testRemoteCleanupDeleteStale() throws Exception { assertTrue(response.isAcknowledged()); } + public void testRemoteCleanupDeleteStaleIndexRoutingFiles() throws Exception { + clusterSettingsSuppliedByTest = true; + Path segmentRepoPath = randomRepoPath(); + Path translogRepoPath = randomRepoPath(); + Path remoteRoutingTableRepoPath = randomRepoPath(); + Settings.Builder settingsBuilder = Settings.builder(); + settingsBuilder.put( + buildRemoteStoreNodeAttributes( + REPOSITORY_NAME, + segmentRepoPath, + REPOSITORY_2_NAME, + translogRepoPath, + REMOTE_ROUTING_TABLE_REPO, + remoteRoutingTableRepoPath, + false + ) + ); + settingsBuilder.put( + RemoteRoutingTableBlobStore.REMOTE_ROUTING_TABLE_PATH_TYPE_SETTING.getKey(), + RemoteStoreEnums.PathType.HASHED_PREFIX.toString() + ) + .put("node.attr." + REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY, REMOTE_ROUTING_TABLE_REPO) + .put(REMOTE_PUBLICATION_EXPERIMENTAL, true); + + int shardCount = randomIntBetween(1, 2); + int replicaCount = 1; + int dataNodeCount = shardCount * (replicaCount + 1); + int clusterManagerNodeCount = 1; + initialTestSetup(shardCount, replicaCount, dataNodeCount, clusterManagerNodeCount, settingsBuilder.build()); + + // update cluster state 21 times to ensure that clean up has run after this will upload 42 manifest files + // to repository, if manifest files are less than that it means clean up has run + updateClusterStateNTimes(RETAINED_MANIFESTS + SKIP_CLEANUP_STATE_CHANGES + 1); + + RepositoriesService repositoriesService = internalCluster().getClusterManagerNodeInstance(RepositoriesService.class); + BlobStoreRepository repository = (BlobStoreRepository) repositoriesService.repository(REPOSITORY_NAME); + BlobPath baseMetadataPath = getBaseMetadataPath(repository); + + BlobStoreRepository routingTableRepository = (BlobStoreRepository) repositoriesService.repository(REMOTE_ROUTING_TABLE_REPO); + List indexRoutingTables = new ArrayList<>(getClusterState().routingTable().indicesRouting().values()); + BlobPath indexRoutingPath = getIndexRoutingPath(baseMetadataPath, indexRoutingTables.get(0).getIndex().getUUID()); + assertBusy(() -> { + // There would be >=3 files as shards will transition from UNASSIGNED -> INIT -> STARTED state + assertTrue(routingTableRepository.blobStore().blobContainer(indexRoutingPath).listBlobs().size() >= 3); + }); + + RemoteClusterStateCleanupManager remoteClusterStateCleanupManager = internalCluster().getClusterManagerNodeInstance( + RemoteClusterStateCleanupManager.class + ); + + // set cleanup interval to 100 ms to make the test faster + ClusterUpdateSettingsResponse response = client().admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING.getKey(), "100ms")) + .get(); + + assertTrue(response.isAcknowledged()); + assertBusy(() -> assertEquals(100, remoteClusterStateCleanupManager.getStaleFileDeletionTask().getInterval().getMillis())); + + String clusterManagerNode = internalCluster().getClusterManagerName(); + NodesStatsResponse nodesStatsResponse = client().admin() + .cluster() + .prepareNodesStats(clusterManagerNode) + .addMetric(NodesStatsRequest.Metric.DISCOVERY.metricName()) + .get(); + verifyIndexRoutingFilesDeletion(routingTableRepository, indexRoutingPath, nodesStatsResponse); + + // disable the clean up to avoid race condition during shutdown + response = client().admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(REMOTE_CLUSTER_STATE_CLEANUP_INTERVAL_SETTING.getKey(), "-1")) + .get(); + assertTrue(response.isAcknowledged()); + } + + private void verifyIndexRoutingFilesDeletion( + BlobStoreRepository routingTableRepository, + BlobPath indexRoutingPath, + NodesStatsResponse nodesStatsResponse + ) throws Exception { + assertBusy(() -> { assertEquals(1, routingTableRepository.blobStore().blobContainer(indexRoutingPath).listBlobs().size()); }); + + // Verify index routing files delete stats + DiscoveryStats discoveryStats = nodesStatsResponse.getNodes().get(0).getDiscoveryStats(); + assertNotNull(discoveryStats.getClusterStateStats()); + for (PersistedStateStats persistedStateStats : discoveryStats.getClusterStateStats().getPersistenceStats()) { + Map extendedFields = persistedStateStats.getExtendedFields(); + assertTrue(extendedFields.containsKey(RemotePersistenceStats.INDEX_ROUTING_FILES_CLEANUP_ATTEMPT_FAILED_COUNT)); + long cleanupAttemptFailedCount = extendedFields.get(RemotePersistenceStats.INDEX_ROUTING_FILES_CLEANUP_ATTEMPT_FAILED_COUNT) + .get(); + assertEquals(0, cleanupAttemptFailedCount); + } + } + private void updateClusterStateNTimes(int n) { int newReplicaCount = randomIntBetween(0, 3); for (int i = n; i > 0; i--) { @@ -155,4 +272,22 @@ private void updateClusterStateNTimes(int n) { assertTrue(response.isAcknowledged()); } } + + private BlobPath getBaseMetadataPath(BlobStoreRepository repository) { + return repository.basePath() + .add( + Base64.getUrlEncoder() + .withoutPadding() + .encodeToString(getClusterState().getClusterName().value().getBytes(StandardCharsets.UTF_8)) + ) + .add("cluster-state") + .add(getClusterState().metadata().clusterUUID()); + } + + private BlobPath getIndexRoutingPath(BlobPath baseMetadataPath, String indexUUID) { + return pathType.path( + RemoteStorePathStrategy.PathInput.builder().basePath(baseMetadataPath.add(INDEX_ROUTING_TABLE)).indexUUID(indexUUID).build(), + RemoteStoreEnums.PathHashAlgorithm.FNV_1A_BASE64 + ); + } } diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteRoutingTableServiceIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteRoutingTableServiceIT.java new file mode 100644 index 0000000000000..53764c0b4d0e8 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteRoutingTableServiceIT.java @@ -0,0 +1,297 @@ +/* + * 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.action.admin.cluster.state.ClusterStateRequest; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.settings.Settings; +import org.opensearch.gateway.remote.model.RemoteRoutingTableBlobStore; +import org.opensearch.index.remote.RemoteStoreEnums; +import org.opensearch.index.remote.RemoteStorePathStrategy; +import org.opensearch.remotestore.RemoteStoreBaseIntegTestCase; +import org.opensearch.repositories.RepositoriesService; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.test.OpenSearchIntegTestCase; +import org.junit.Before; + +import java.nio.charset.StandardCharsets; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Base64; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.opensearch.common.util.FeatureFlags.REMOTE_PUBLICATION_EXPERIMENTAL; +import static org.opensearch.gateway.remote.RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING; +import static org.opensearch.gateway.remote.routingtable.RemoteIndexRoutingTable.INDEX_ROUTING_TABLE; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY; + +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) +public class RemoteRoutingTableServiceIT extends RemoteStoreBaseIntegTestCase { + private static final String INDEX_NAME = "test-index"; + BlobPath indexRoutingPath; + AtomicInteger indexRoutingFiles = new AtomicInteger(); + private final RemoteStoreEnums.PathType pathType = RemoteStoreEnums.PathType.HASHED_PREFIX; + + @Before + public void setup() { + asyncUploadMockFsRepo = false; + } + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder() + .put(super.nodeSettings(nodeOrdinal)) + .put(REMOTE_CLUSTER_STATE_ENABLED_SETTING.getKey(), true) + .put( + RemoteRoutingTableBlobStore.REMOTE_ROUTING_TABLE_PATH_TYPE_SETTING.getKey(), + RemoteStoreEnums.PathType.HASHED_PREFIX.toString() + ) + .put("node.attr." + REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY, REMOTE_ROUTING_TABLE_REPO) + .put(REMOTE_PUBLICATION_EXPERIMENTAL, true) + .build(); + } + + public void testRemoteRoutingTableIndexLifecycle() throws Exception { + BlobStoreRepository repository = prepareClusterAndVerifyRepository(); + + RemoteClusterStateService remoteClusterStateService = internalCluster().getClusterManagerNodeInstance( + RemoteClusterStateService.class + ); + RemoteManifestManager remoteManifestManager = remoteClusterStateService.getRemoteManifestManager(); + verifyUpdatesInManifestFile(remoteManifestManager); + + List routingTableVersions = getRoutingTableFromAllNodes(); + assertTrue(areRoutingTablesSame(routingTableVersions)); + + // Update index settings + updateIndexSettings(INDEX_NAME, IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 2); + ensureGreen(INDEX_NAME); + assertBusy(() -> { + int indexRoutingFilesAfterUpdate = repository.blobStore().blobContainer(indexRoutingPath).listBlobs().size(); + // At-least 3 new index routing files will be created as shards will transition from INIT -> UNASSIGNED -> STARTED state + assertTrue(indexRoutingFilesAfterUpdate >= indexRoutingFiles.get() + 3); + }); + + verifyUpdatesInManifestFile(remoteManifestManager); + + routingTableVersions = getRoutingTableFromAllNodes(); + assertTrue(areRoutingTablesSame(routingTableVersions)); + + // Delete the index and assert its deletion + deleteIndexAndVerify(remoteManifestManager); + + routingTableVersions = getRoutingTableFromAllNodes(); + assertTrue(areRoutingTablesSame(routingTableVersions)); + } + + public void testRemoteRoutingTableIndexNodeRestart() throws Exception { + BlobStoreRepository repository = prepareClusterAndVerifyRepository(); + + List routingTableVersions = getRoutingTableFromAllNodes(); + assertTrue(areRoutingTablesSame(routingTableVersions)); + + // Ensure node comes healthy after restart + Set dataNodes = internalCluster().getDataNodeNames(); + internalCluster().restartNode(randomFrom(dataNodes)); + ensureGreen(); + ensureGreen(INDEX_NAME); + + // ensure restarted node joins and the cluster is stable + assertEquals(3, internalCluster().clusterService().state().nodes().getDataNodes().size()); + ensureStableCluster(4); + assertRemoteStoreRepositoryOnAllNodes(REMOTE_ROUTING_TABLE_REPO); + + assertBusy(() -> { + int indexRoutingFilesAfterNodeDrop = repository.blobStore().blobContainer(indexRoutingPath).listBlobs().size(); + assertTrue(indexRoutingFilesAfterNodeDrop > indexRoutingFiles.get()); + }); + + RemoteClusterStateService remoteClusterStateService = internalCluster().getClusterManagerNodeInstance( + RemoteClusterStateService.class + ); + RemoteManifestManager remoteManifestManager = remoteClusterStateService.getRemoteManifestManager(); + verifyUpdatesInManifestFile(remoteManifestManager); + } + + public void testRemoteRoutingTableIndexMasterRestart1() throws Exception { + BlobStoreRepository repository = prepareClusterAndVerifyRepository(); + + List routingTableVersions = getRoutingTableFromAllNodes(); + assertTrue(areRoutingTablesSame(routingTableVersions)); + + // Ensure node comes healthy after restart + String clusterManagerName = internalCluster().getClusterManagerName(); + internalCluster().restartNode(clusterManagerName); + ensureGreen(); + ensureGreen(INDEX_NAME); + + // ensure master is elected and the cluster is stable + assertNotNull(internalCluster().clusterService().state().nodes().getClusterManagerNode()); + ensureStableCluster(4); + assertRemoteStoreRepositoryOnAllNodes(REMOTE_ROUTING_TABLE_REPO); + + assertBusy(() -> { + int indexRoutingFilesAfterNodeDrop = repository.blobStore().blobContainer(indexRoutingPath).listBlobs().size(); + assertTrue(indexRoutingFilesAfterNodeDrop > indexRoutingFiles.get()); + }); + + RemoteClusterStateService remoteClusterStateService = internalCluster().getClusterManagerNodeInstance( + RemoteClusterStateService.class + ); + RemoteManifestManager remoteManifestManager = remoteClusterStateService.getRemoteManifestManager(); + verifyUpdatesInManifestFile(remoteManifestManager); + } + + private BlobStoreRepository prepareClusterAndVerifyRepository() throws Exception { + clusterSettingsSuppliedByTest = true; + Path segmentRepoPath = randomRepoPath(); + Path translogRepoPath = randomRepoPath(); + Path remoteRoutingTableRepoPath = randomRepoPath(); + Settings settings = buildRemoteStoreNodeAttributes( + REPOSITORY_NAME, + segmentRepoPath, + REPOSITORY_2_NAME, + translogRepoPath, + REMOTE_ROUTING_TABLE_REPO, + remoteRoutingTableRepoPath, + false + ); + prepareCluster(1, 3, INDEX_NAME, 1, 5, settings); + ensureGreen(INDEX_NAME); + + RepositoriesService repositoriesService = internalCluster().getClusterManagerNodeInstance(RepositoriesService.class); + BlobStoreRepository repository = (BlobStoreRepository) repositoriesService.repository(REMOTE_ROUTING_TABLE_REPO); + + BlobPath baseMetadataPath = getBaseMetadataPath(repository); + List indexRoutingTables = new ArrayList<>(getClusterState().routingTable().indicesRouting().values()); + indexRoutingPath = getIndexRoutingPath(baseMetadataPath.add(INDEX_ROUTING_TABLE), indexRoutingTables.get(0).getIndex().getUUID()); + + assertBusy(() -> { + indexRoutingFiles.set(repository.blobStore().blobContainer(indexRoutingPath).listBlobs().size()); + // There would be >=3 files as shards will transition from UNASSIGNED -> INIT -> STARTED state + assertTrue(indexRoutingFiles.get() >= 3); + }); + assertRemoteStoreRepositoryOnAllNodes(REMOTE_ROUTING_TABLE_REPO); + return repository; + } + + private BlobPath getBaseMetadataPath(BlobStoreRepository repository) { + return repository.basePath() + .add( + Base64.getUrlEncoder() + .withoutPadding() + .encodeToString(getClusterState().getClusterName().value().getBytes(StandardCharsets.UTF_8)) + ) + .add("cluster-state") + .add(getClusterState().metadata().clusterUUID()); + } + + private BlobPath getIndexRoutingPath(BlobPath indexRoutingPath, String indexUUID) { + RemoteStoreEnums.PathHashAlgorithm pathHashAlgo = RemoteStoreEnums.PathHashAlgorithm.FNV_1A_BASE64; + return pathType.path( + RemoteStorePathStrategy.PathInput.builder().basePath(indexRoutingPath).indexUUID(indexUUID).build(), + pathHashAlgo + ); + } + + private void verifyUpdatesInManifestFile(RemoteManifestManager remoteManifestManager) { + Optional latestManifest = remoteManifestManager.getLatestClusterMetadataManifest( + getClusterState().getClusterName().value(), + getClusterState().getMetadata().clusterUUID() + ); + assertTrue(latestManifest.isPresent()); + ClusterMetadataManifest manifest = latestManifest.get(); + assertTrue(manifest.getDiffManifest().getIndicesRoutingUpdated().contains(INDEX_NAME)); + assertTrue(manifest.getDiffManifest().getIndicesDeleted().isEmpty()); + assertFalse(manifest.getIndicesRouting().isEmpty()); + assertEquals(1, manifest.getIndicesRouting().size()); + assertTrue(manifest.getIndicesRouting().get(0).getUploadedFilename().contains(indexRoutingPath.buildAsString())); + } + + private List getRoutingTableFromAllNodes() throws ExecutionException, InterruptedException { + String[] allNodes = internalCluster().getNodeNames(); + List routingTables = new ArrayList<>(); + for (String node : allNodes) { + RoutingTable routingTable = internalCluster().client(node) + .admin() + .cluster() + .state(new ClusterStateRequest().local(true)) + .get() + .getState() + .routingTable(); + routingTables.add(routingTable); + } + return routingTables; + } + + private boolean areRoutingTablesSame(List routingTables) { + if (routingTables == null || routingTables.isEmpty()) { + return false; + } + + RoutingTable firstRoutingTable = routingTables.get(0); + for (RoutingTable routingTable : routingTables) { + if (!compareRoutingTables(firstRoutingTable, routingTable)) { + logger.info("Responses are not the same: {} {}", firstRoutingTable, routingTable); + return false; + } + } + return true; + } + + private boolean compareRoutingTables(RoutingTable a, RoutingTable b) { + if (a == b) return true; + if (b == null || a.getClass() != b.getClass()) return false; + if (a.version() != b.version()) return false; + if (a.indicesRouting().size() != b.indicesRouting().size()) return false; + + for (Map.Entry entry : a.indicesRouting().entrySet()) { + IndexRoutingTable thisIndexRoutingTable = entry.getValue(); + IndexRoutingTable thatIndexRoutingTable = b.indicesRouting().get(entry.getKey()); + if (!thatIndexRoutingTable.equals(thatIndexRoutingTable)) { + return false; + } + } + return true; + } + + private void updateIndexSettings(String indexName, String settingKey, int settingValue) { + client().admin() + .indices() + .prepareUpdateSettings(indexName) + .setSettings(Settings.builder().put(settingKey, settingValue)) + .execute() + .actionGet(); + } + + private void deleteIndexAndVerify(RemoteManifestManager remoteManifestManager) { + client().admin().indices().prepareDelete(INDEX_NAME).execute().actionGet(); + assertFalse(client().admin().indices().prepareExists(INDEX_NAME).get().isExists()); + + // Verify index is marked deleted in manifest + Optional latestManifest = remoteManifestManager.getLatestClusterMetadataManifest( + getClusterState().getClusterName().value(), + getClusterState().getMetadata().clusterUUID() + ); + assertTrue(latestManifest.isPresent()); + ClusterMetadataManifest manifest = latestManifest.get(); + assertTrue(manifest.getDiffManifest().getIndicesRoutingUpdated().isEmpty()); + assertTrue(manifest.getDiffManifest().getIndicesDeleted().contains(INDEX_NAME)); + assertTrue(manifest.getIndicesRouting().isEmpty()); + } + +} diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java index 64efcee6ef1b5..63a9451a27a12 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java @@ -69,6 +69,7 @@ public class RemoteStoreBaseIntegTestCase extends OpenSearchIntegTestCase { protected static final String REPOSITORY_NAME = "test-remote-store-repo"; protected static final String REPOSITORY_2_NAME = "test-remote-store-repo-2"; + protected static final String REMOTE_ROUTING_TABLE_REPO = "remote-routing-table-repo"; protected static final int SHARD_COUNT = 1; protected static int REPLICA_COUNT = 1; protected static final String TOTAL_OPERATIONS = "total-operations"; @@ -360,4 +361,20 @@ protected void prepareCluster(int numClusterManagerNodes, int numDataOnlyNodes, ensureGreen(index); } } + + protected void prepareCluster( + int numClusterManagerNodes, + int numDataOnlyNodes, + String indices, + int replicaCount, + int shardCount, + Settings settings + ) { + internalCluster().startClusterManagerOnlyNodes(numClusterManagerNodes, settings); + internalCluster().startDataOnlyNodes(numDataOnlyNodes, settings); + for (String index : indices.split(",")) { + createIndex(index, remoteStoreIndexSettings(replicaCount, shardCount)); + ensureGreen(index); + } + } } diff --git a/server/src/main/java/org/opensearch/cluster/coordination/PersistedStateStats.java b/server/src/main/java/org/opensearch/cluster/coordination/PersistedStateStats.java index 0b7ed4fee5775..023c2db1a574a 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/PersistedStateStats.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/PersistedStateStats.java @@ -117,6 +117,10 @@ protected void addToExtendedFields(String extendedField, AtomicLong extendedFiel this.extendedFields.put(extendedField, extendedFieldValue); } + public Map getExtendedFields() { + return extendedFields; + } + public String getStatsName() { return statsName; } diff --git a/server/src/test/java/org/opensearch/cluster/coordination/PersistedStateStatsTests.java b/server/src/test/java/org/opensearch/cluster/coordination/PersistedStateStatsTests.java new file mode 100644 index 0000000000000..15c7d3ea206ef --- /dev/null +++ b/server/src/test/java/org/opensearch/cluster/coordination/PersistedStateStatsTests.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.cluster.coordination; + +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; + +import java.util.concurrent.atomic.AtomicLong; + +public class PersistedStateStatsTests extends OpenSearchTestCase { + private PersistedStateStats persistedStateStats; + + @Before + public void setup() { + persistedStateStats = new PersistedStateStats("testStats"); + } + + public void testAddToExtendedFieldsNewField() { + String fieldName = "testField"; + AtomicLong fieldValue = new AtomicLong(42); + + persistedStateStats.addToExtendedFields(fieldName, fieldValue); + + assertTrue(persistedStateStats.getExtendedFields().containsKey(fieldName)); + assertEquals(42, persistedStateStats.getExtendedFields().get(fieldName).get()); + } + + public void testAddToExtendedFieldsExistingField() { + String fieldName = "testField"; + AtomicLong initialValue = new AtomicLong(42); + persistedStateStats.addToExtendedFields(fieldName, initialValue); + + AtomicLong newValue = new AtomicLong(84); + persistedStateStats.addToExtendedFields(fieldName, newValue); + + assertTrue(persistedStateStats.getExtendedFields().containsKey(fieldName)); + assertEquals(84, persistedStateStats.getExtendedFields().get(fieldName).get()); + } + + public void testAddMultipleFields() { + String fieldName1 = "testField1"; + AtomicLong fieldValue1 = new AtomicLong(42); + + String fieldName2 = "testField2"; + AtomicLong fieldValue2 = new AtomicLong(84); + + persistedStateStats.addToExtendedFields(fieldName1, fieldValue1); + persistedStateStats.addToExtendedFields(fieldName2, fieldValue2); + + assertTrue(persistedStateStats.getExtendedFields().containsKey(fieldName1)); + assertTrue(persistedStateStats.getExtendedFields().containsKey(fieldName2)); + + assertEquals(42, persistedStateStats.getExtendedFields().get(fieldName1).get()); + assertEquals(84, persistedStateStats.getExtendedFields().get(fieldName2).get()); + } +} diff --git a/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java b/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java index 9853cef482254..b86cce682c68e 100644 --- a/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java +++ b/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java @@ -153,6 +153,7 @@ import org.opensearch.plugins.NetworkPlugin; import org.opensearch.plugins.Plugin; import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.repositories.fs.FsRepository; import org.opensearch.repositories.fs.ReloadableFsRepository; import org.opensearch.script.MockScriptService; import org.opensearch.search.MockSearchService; @@ -220,6 +221,7 @@ import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_CLUSTER_STATE_REPOSITORY_NAME_ATTRIBUTE_KEY; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_REPOSITORY_TYPE_ATTRIBUTE_KEY_FORMAT; +import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_SEGMENT_REPOSITORY_NAME_ATTRIBUTE_KEY; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_TRANSLOG_REPOSITORY_NAME_ATTRIBUTE_KEY; import static org.opensearch.node.remotestore.RemoteStoreNodeService.MIGRATION_DIRECTION_SETTING; @@ -2580,6 +2582,35 @@ public static Settings remoteStoreClusterSettings( return settingsBuilder.build(); } + public static Settings remoteStoreClusterSettings( + String segmentRepoName, + Path segmentRepoPath, + String segmentRepoType, + String translogRepoName, + Path translogRepoPath, + String translogRepoType, + String routingTableRepoName, + Path routingTableRepoPath, + String routingTableRepoType + ) { + Settings.Builder settingsBuilder = Settings.builder(); + settingsBuilder.put( + buildRemoteStoreNodeAttributes( + segmentRepoName, + segmentRepoPath, + segmentRepoType, + translogRepoName, + translogRepoPath, + translogRepoType, + routingTableRepoName, + routingTableRepoPath, + routingTableRepoType, + false + ) + ); + return settingsBuilder.build(); + } + public static Settings remoteStoreClusterSettings( String segmentRepoName, Path segmentRepoPath, @@ -2591,6 +2622,29 @@ public static Settings remoteStoreClusterSettings( return settingsBuilder.build(); } + public static Settings remoteStoreClusterSettings( + String segmentRepoName, + Path segmentRepoPath, + String translogRepoName, + Path translogRepoPath, + String remoteRoutingTableRepoName, + Path remoteRoutingTableRepoPath + ) { + Settings.Builder settingsBuilder = Settings.builder(); + settingsBuilder.put( + buildRemoteStoreNodeAttributes( + segmentRepoName, + segmentRepoPath, + translogRepoName, + translogRepoPath, + remoteRoutingTableRepoName, + remoteRoutingTableRepoPath, + false + ) + ); + return settingsBuilder.build(); + } + public static Settings buildRemoteStoreNodeAttributes( String segmentRepoName, Path segmentRepoPath, @@ -2609,6 +2663,29 @@ public static Settings buildRemoteStoreNodeAttributes( ); } + public static Settings buildRemoteStoreNodeAttributes( + String segmentRepoName, + Path segmentRepoPath, + String translogRepoName, + Path translogRepoPath, + String remoteRoutingTableRepoName, + Path remoteRoutingTableRepoPath, + boolean withRateLimiterAttributes + ) { + return buildRemoteStoreNodeAttributes( + segmentRepoName, + segmentRepoPath, + ReloadableFsRepository.TYPE, + translogRepoName, + translogRepoPath, + ReloadableFsRepository.TYPE, + remoteRoutingTableRepoName, + remoteRoutingTableRepoPath, + FsRepository.TYPE, + withRateLimiterAttributes + ); + } + private static Settings buildRemoteStoreNodeAttributes( String segmentRepoName, Path segmentRepoPath, @@ -2617,6 +2694,32 @@ private static Settings buildRemoteStoreNodeAttributes( Path translogRepoPath, String translogRepoType, boolean withRateLimiterAttributes + ) { + return buildRemoteStoreNodeAttributes( + segmentRepoName, + segmentRepoPath, + segmentRepoType, + translogRepoName, + translogRepoPath, + translogRepoType, + null, + null, + null, + withRateLimiterAttributes + ); + } + + private static Settings buildRemoteStoreNodeAttributes( + String segmentRepoName, + Path segmentRepoPath, + String segmentRepoType, + String translogRepoName, + Path translogRepoPath, + String translogRepoType, + String routingTableRepoName, + Path routingTableRepoPath, + String routingTableRepoType, + boolean withRateLimiterAttributes ) { String segmentRepoTypeAttributeKey = String.format( Locale.getDefault(), @@ -2648,6 +2751,19 @@ private static Settings buildRemoteStoreNodeAttributes( "node.attr." + REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX, segmentRepoName ); + String routingTableRepoAttributeKey = null, routingTableRepoSettingsAttributeKeyPrefix = null; + if (routingTableRepoName != null) { + routingTableRepoAttributeKey = String.format( + Locale.getDefault(), + "node.attr." + REMOTE_STORE_REPOSITORY_TYPE_ATTRIBUTE_KEY_FORMAT, + routingTableRepoName + ); + routingTableRepoSettingsAttributeKeyPrefix = String.format( + Locale.getDefault(), + "node.attr." + REMOTE_STORE_REPOSITORY_SETTINGS_ATTRIBUTE_KEY_PREFIX, + routingTableRepoName + ); + } String prefixModeVerificationSuffix = BlobStoreRepository.PREFIX_MODE_VERIFICATION_SETTING.getKey(); @@ -2664,6 +2780,11 @@ private static Settings buildRemoteStoreNodeAttributes( .put(stateRepoTypeAttributeKey, segmentRepoType) .put(stateRepoSettingsAttributeKeyPrefix + "location", segmentRepoPath) .put(stateRepoSettingsAttributeKeyPrefix + prefixModeVerificationSuffix, prefixModeVerificationEnable); + if (routingTableRepoName != null) { + settings.put("node.attr." + REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY, routingTableRepoName) + .put(routingTableRepoAttributeKey, routingTableRepoType) + .put(routingTableRepoSettingsAttributeKeyPrefix + "location", routingTableRepoPath); + } if (withRateLimiterAttributes) { settings.put(segmentRepoSettingsAttributeKeyPrefix + "compress", randomBoolean()) From 5de0c8a7a3a63455758a8bfe24199f4955f29dca Mon Sep 17 00:00:00 2001 From: Daniel Widdis Date: Mon, 22 Jul 2024 11:48:53 -0700 Subject: [PATCH 13/37] Add SortResponseProcessor to Search Pipelines (#14785) * Add SortResponseProcessor for search pipelines Signed-off-by: Daniel Widdis * Add stupid and unnecessary javadocs to satisfy overly strict CI Signed-off-by: Daniel Widdis * Split casting and sorting methods for readability Signed-off-by: Daniel Widdis * Register the sort processor factory Signed-off-by: Daniel Widdis * Address code review comments Signed-off-by: Daniel Widdis * Cast individual list elements to avoid creating two lists Signed-off-by: Daniel Widdis * Add yamlRestTests Signed-off-by: Daniel Widdis * Clarify why there's unusual sorting Signed-off-by: Daniel Widdis * Use instanceof instead of isAssignableFrom Signed-off-by: Daniel Widdis --------- Signed-off-by: Daniel Widdis --- CHANGELOG.md | 1 + .../SearchPipelineCommonModulePlugin.java | 4 +- .../common/SortResponseProcessor.java | 209 ++++++++++++++++ .../common/SplitResponseProcessor.java | 2 +- ...SearchPipelineCommonModulePluginTests.java | 2 +- .../common/SortResponseProcessorTests.java | 230 ++++++++++++++++++ .../test/search_pipeline/80_sort_response.yml | 152 ++++++++++++ 7 files changed, 596 insertions(+), 4 deletions(-) create mode 100644 modules/search-pipeline-common/src/main/java/org/opensearch/search/pipeline/common/SortResponseProcessor.java create mode 100644 modules/search-pipeline-common/src/test/java/org/opensearch/search/pipeline/common/SortResponseProcessorTests.java create mode 100644 modules/search-pipeline-common/src/yamlRestTest/resources/rest-api-spec/test/search_pipeline/80_sort_response.yml diff --git a/CHANGELOG.md b/CHANGELOG.md index e32b6de84a195..80dd5a27ffdaa 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -20,6 +20,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add matchesPluginSystemIndexPattern to SystemIndexRegistry ([#14750](https://github.com/opensearch-project/OpenSearch/pull/14750)) - Add Plugin interface for loading application based configuration templates (([#14659](https://github.com/opensearch-project/OpenSearch/issues/14659))) - Refactor remote-routing-table service inline with remote state interfaces([#14668](https://github.com/opensearch-project/OpenSearch/pull/14668)) +- Add SortResponseProcessor to Search Pipelines (([#14785](https://github.com/opensearch-project/OpenSearch/issues/14785))) - Add prefix mode verification setting for repository verification (([#14790](https://github.com/opensearch-project/OpenSearch/pull/14790))) - Add SplitResponseProcessor to Search Pipelines (([#14800](https://github.com/opensearch-project/OpenSearch/issues/14800))) - Optimize TransportNodesAction to not send DiscoveryNodes for NodeStats, NodesInfo and ClusterStats call ([14749](https://github.com/opensearch-project/OpenSearch/pull/14749)) diff --git a/modules/search-pipeline-common/src/main/java/org/opensearch/search/pipeline/common/SearchPipelineCommonModulePlugin.java b/modules/search-pipeline-common/src/main/java/org/opensearch/search/pipeline/common/SearchPipelineCommonModulePlugin.java index d05101da2817c..2a2de9debb9d9 100644 --- a/modules/search-pipeline-common/src/main/java/org/opensearch/search/pipeline/common/SearchPipelineCommonModulePlugin.java +++ b/modules/search-pipeline-common/src/main/java/org/opensearch/search/pipeline/common/SearchPipelineCommonModulePlugin.java @@ -97,8 +97,8 @@ public Map> getResponseProces new TruncateHitsResponseProcessor.Factory(), CollapseResponseProcessor.TYPE, new CollapseResponseProcessor.Factory(), - SplitResponseProcessor.TYPE, - new SplitResponseProcessor.Factory() + SortResponseProcessor.TYPE, + new SortResponseProcessor.Factory() ) ); } diff --git a/modules/search-pipeline-common/src/main/java/org/opensearch/search/pipeline/common/SortResponseProcessor.java b/modules/search-pipeline-common/src/main/java/org/opensearch/search/pipeline/common/SortResponseProcessor.java new file mode 100644 index 0000000000000..e0bfd38b26376 --- /dev/null +++ b/modules/search-pipeline-common/src/main/java/org/opensearch/search/pipeline/common/SortResponseProcessor.java @@ -0,0 +1,209 @@ +/* + * 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.pipeline.common; + +import org.opensearch.action.search.SearchRequest; +import org.opensearch.action.search.SearchResponse; +import org.opensearch.common.collect.Tuple; +import org.opensearch.common.document.DocumentField; +import org.opensearch.common.xcontent.XContentHelper; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.xcontent.MediaType; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.ingest.ConfigurationUtils; +import org.opensearch.search.SearchHit; +import org.opensearch.search.pipeline.AbstractProcessor; +import org.opensearch.search.pipeline.Processor; +import org.opensearch.search.pipeline.SearchResponseProcessor; + +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; + +/** + * Processor that sorts an array of items. + * Throws exception is the specified field is not an array. + */ +public class SortResponseProcessor extends AbstractProcessor implements SearchResponseProcessor { + /** Key to reference this processor type from a search pipeline. */ + public static final String TYPE = "sort"; + /** Key defining the array field to be sorted. */ + public static final String SORT_FIELD = "field"; + /** Optional key defining the sort order. */ + public static final String SORT_ORDER = "order"; + /** Optional key to put the sorted values in a different field. */ + public static final String TARGET_FIELD = "target_field"; + /** Default sort order if not specified */ + public static final String DEFAULT_ORDER = "asc"; + + /** Enum defining how elements will be sorted */ + public enum SortOrder { + /** Sort in ascending (natural) order */ + ASCENDING("asc"), + /** Sort in descending (reverse) order */ + DESCENDING("desc"); + + private final String direction; + + SortOrder(String direction) { + this.direction = direction; + } + + @Override + public String toString() { + return this.direction; + } + + /** + * Converts the string representation of the enum value to the enum. + * @param value A string ("asc" or "desc") + * @return the corresponding enum value + */ + public static SortOrder fromString(String value) { + if (value == null) { + throw new IllegalArgumentException("Sort direction cannot be null"); + } + + if (value.equals(ASCENDING.toString())) { + return ASCENDING; + } else if (value.equals(DESCENDING.toString())) { + return DESCENDING; + } + throw new IllegalArgumentException("Sort direction [" + value + "] not recognized." + " Valid values are: [asc, desc]"); + } + } + + private final String sortField; + private final SortOrder sortOrder; + private final String targetField; + + SortResponseProcessor( + String tag, + String description, + boolean ignoreFailure, + String sortField, + SortOrder sortOrder, + String targetField + ) { + super(tag, description, ignoreFailure); + this.sortField = Objects.requireNonNull(sortField); + this.sortOrder = Objects.requireNonNull(sortOrder); + this.targetField = targetField == null ? sortField : targetField; + } + + /** + * Getter function for sortField + * @return sortField + */ + public String getSortField() { + return sortField; + } + + /** + * Getter function for targetField + * @return targetField + */ + public String getTargetField() { + return targetField; + } + + /** + * Getter function for sortOrder + * @return sortOrder + */ + public SortOrder getSortOrder() { + return sortOrder; + } + + @Override + public String getType() { + return TYPE; + } + + @Override + public SearchResponse processResponse(SearchRequest request, SearchResponse response) throws Exception { + SearchHit[] hits = response.getHits().getHits(); + for (SearchHit hit : hits) { + Map fields = hit.getFields(); + if (fields.containsKey(sortField)) { + DocumentField docField = hit.getFields().get(sortField); + if (docField == null) { + throw new IllegalArgumentException("field [" + sortField + "] is null, cannot sort."); + } + hit.setDocumentField(targetField, new DocumentField(targetField, getSortedValues(docField.getValues()))); + } + if (hit.hasSource()) { + BytesReference sourceRef = hit.getSourceRef(); + Tuple> typeAndSourceMap = XContentHelper.convertToMap( + sourceRef, + false, + (MediaType) null + ); + + Map sourceAsMap = typeAndSourceMap.v2(); + if (sourceAsMap.containsKey(sortField)) { + Object val = sourceAsMap.get(sortField); + if (val instanceof List) { + @SuppressWarnings("unchecked") + List listVal = (List) val; + sourceAsMap.put(targetField, getSortedValues(listVal)); + } + XContentBuilder builder = XContentBuilder.builder(typeAndSourceMap.v1().xContent()); + builder.map(sourceAsMap); + hit.sourceRef(BytesReference.bytes(builder)); + } + } + } + return response; + } + + private List getSortedValues(List values) { + return values.stream() + .map(this::downcastToComparable) + .sorted(sortOrder.equals(SortOrder.ASCENDING) ? Comparator.naturalOrder() : Comparator.reverseOrder()) + .collect(Collectors.toList()); + } + + @SuppressWarnings("unchecked") + private Comparable downcastToComparable(Object obj) { + if (obj instanceof Comparable) { + return (Comparable) obj; + } else if (obj == null) { + throw new IllegalArgumentException("field [" + sortField + "] contains a null value.]"); + } else { + throw new IllegalArgumentException("field [" + sortField + "] of type [" + obj.getClass().getName() + "] is not comparable.]"); + } + } + + static class Factory implements Processor.Factory { + + @Override + public SortResponseProcessor create( + Map> processorFactories, + String tag, + String description, + boolean ignoreFailure, + Map config, + PipelineContext pipelineContext + ) { + String sortField = ConfigurationUtils.readStringProperty(TYPE, tag, config, SORT_FIELD); + String targetField = ConfigurationUtils.readStringProperty(TYPE, tag, config, TARGET_FIELD, sortField); + try { + SortOrder sortOrder = SortOrder.fromString( + ConfigurationUtils.readStringProperty(TYPE, tag, config, SORT_ORDER, DEFAULT_ORDER) + ); + return new SortResponseProcessor(tag, description, ignoreFailure, sortField, sortOrder, targetField); + } catch (IllegalArgumentException e) { + throw ConfigurationUtils.newConfigurationException(TYPE, tag, SORT_ORDER, e.getMessage()); + } + } + } +} diff --git a/modules/search-pipeline-common/src/main/java/org/opensearch/search/pipeline/common/SplitResponseProcessor.java b/modules/search-pipeline-common/src/main/java/org/opensearch/search/pipeline/common/SplitResponseProcessor.java index 0762f8f59b76e..bb3db4d9bc2c1 100644 --- a/modules/search-pipeline-common/src/main/java/org/opensearch/search/pipeline/common/SplitResponseProcessor.java +++ b/modules/search-pipeline-common/src/main/java/org/opensearch/search/pipeline/common/SplitResponseProcessor.java @@ -111,7 +111,7 @@ public SearchResponse processResponse(SearchRequest request, SearchResponse resp throw new IllegalArgumentException("field [" + splitField + "] is null, cannot split."); } Object val = docField.getValue(); - if (val == null || !String.class.isAssignableFrom(val.getClass())) { + if (!(val instanceof String)) { throw new IllegalArgumentException("field [" + splitField + "] is not a string, cannot split"); } Object[] strings = ((String) val).split(separator, preserveTrailing ? -1 : 0); diff --git a/modules/search-pipeline-common/src/test/java/org/opensearch/search/pipeline/common/SearchPipelineCommonModulePluginTests.java b/modules/search-pipeline-common/src/test/java/org/opensearch/search/pipeline/common/SearchPipelineCommonModulePluginTests.java index d4f9ae2490a10..404842742629c 100644 --- a/modules/search-pipeline-common/src/test/java/org/opensearch/search/pipeline/common/SearchPipelineCommonModulePluginTests.java +++ b/modules/search-pipeline-common/src/test/java/org/opensearch/search/pipeline/common/SearchPipelineCommonModulePluginTests.java @@ -82,7 +82,7 @@ public void testAllowlistNotSpecified() throws IOException { try (SearchPipelineCommonModulePlugin plugin = new SearchPipelineCommonModulePlugin()) { assertEquals(Set.of("oversample", "filter_query", "script"), plugin.getRequestProcessors(createParameters(settings)).keySet()); assertEquals( - Set.of("rename_field", "truncate_hits", "collapse", "split"), + Set.of("rename_field", "truncate_hits", "collapse", "sort"), plugin.getResponseProcessors(createParameters(settings)).keySet() ); assertEquals(Set.of(), plugin.getSearchPhaseResultsProcessors(createParameters(settings)).keySet()); diff --git a/modules/search-pipeline-common/src/test/java/org/opensearch/search/pipeline/common/SortResponseProcessorTests.java b/modules/search-pipeline-common/src/test/java/org/opensearch/search/pipeline/common/SortResponseProcessorTests.java new file mode 100644 index 0000000000000..c18c6b34b05d1 --- /dev/null +++ b/modules/search-pipeline-common/src/test/java/org/opensearch/search/pipeline/common/SortResponseProcessorTests.java @@ -0,0 +1,230 @@ +/* + * 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.java + * compatible open source license. + */ + +package org.opensearch.search.pipeline.common; + +import org.apache.lucene.search.TotalHits; +import org.opensearch.OpenSearchParseException; +import org.opensearch.action.search.SearchRequest; +import org.opensearch.action.search.SearchResponse; +import org.opensearch.action.search.SearchResponseSections; +import org.opensearch.common.document.DocumentField; +import org.opensearch.core.common.bytes.BytesArray; +import org.opensearch.index.query.QueryBuilder; +import org.opensearch.index.query.TermQueryBuilder; +import org.opensearch.ingest.RandomDocumentPicks; +import org.opensearch.search.SearchHit; +import org.opensearch.search.SearchHits; +import org.opensearch.search.builder.SearchSourceBuilder; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class SortResponseProcessorTests extends OpenSearchTestCase { + + private static final List PI = List.of(3, 1, 4, 1, 5, 9, 2, 6); + private static final List E = List.of(2, 7, 1, 8, 2, 8, 1, 8); + private static final List X; + static { + List x = new ArrayList<>(); + x.add(1); + x.add(null); + x.add(3); + X = x; + } + + private SearchRequest createDummyRequest() { + QueryBuilder query = new TermQueryBuilder("field", "value"); + SearchSourceBuilder source = new SearchSourceBuilder().query(query); + return new SearchRequest().source(source); + } + + private SearchResponse createTestResponse() { + SearchHit[] hits = new SearchHit[2]; + + // one response with source + Map piMap = new HashMap<>(); + piMap.put("digits", new DocumentField("digits", PI)); + hits[0] = new SearchHit(0, "doc 1", piMap, Collections.emptyMap()); + hits[0].sourceRef(new BytesArray("{ \"digits\" : " + PI + " }")); + hits[0].score((float) Math.PI); + + // one without source + Map eMap = new HashMap<>(); + eMap.put("digits", new DocumentField("digits", E)); + hits[1] = new SearchHit(1, "doc 2", eMap, Collections.emptyMap()); + hits[1].score((float) Math.E); + + SearchHits searchHits = new SearchHits(hits, new TotalHits(2, TotalHits.Relation.EQUAL_TO), 2); + SearchResponseSections searchResponseSections = new SearchResponseSections(searchHits, null, null, false, false, null, 0); + return new SearchResponse(searchResponseSections, null, 1, 1, 0, 10, null, null); + } + + private SearchResponse createTestResponseNullField() { + SearchHit[] hits = new SearchHit[1]; + + Map map = new HashMap<>(); + map.put("digits", null); + hits[0] = new SearchHit(0, "doc 1", map, Collections.emptyMap()); + hits[0].sourceRef(new BytesArray("{ \"digits\" : null }")); + hits[0].score((float) Math.PI); + + SearchHits searchHits = new SearchHits(hits, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1); + SearchResponseSections searchResponseSections = new SearchResponseSections(searchHits, null, null, false, false, null, 0); + return new SearchResponse(searchResponseSections, null, 1, 1, 0, 10, null, null); + } + + private SearchResponse createTestResponseNullListEntry() { + SearchHit[] hits = new SearchHit[1]; + + Map xMap = new HashMap<>(); + xMap.put("digits", new DocumentField("digits", X)); + hits[0] = new SearchHit(0, "doc 1", xMap, Collections.emptyMap()); + hits[0].sourceRef(new BytesArray("{ \"digits\" : " + X + " }")); + hits[0].score((float) Math.PI); + + SearchHits searchHits = new SearchHits(hits, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1); + SearchResponseSections searchResponseSections = new SearchResponseSections(searchHits, null, null, false, false, null, 0); + return new SearchResponse(searchResponseSections, null, 1, 1, 0, 10, null, null); + } + + private SearchResponse createTestResponseNotComparable() { + SearchHit[] hits = new SearchHit[1]; + + Map piMap = new HashMap<>(); + piMap.put("maps", new DocumentField("maps", List.of(Map.of("foo", "I'm incomparable!")))); + hits[0] = new SearchHit(0, "doc 1", piMap, Collections.emptyMap()); + hits[0].sourceRef(new BytesArray("{ \"maps\" : [{ \"foo\" : \"I'm incomparable!\"}]] }")); + hits[0].score((float) Math.PI); + + SearchHits searchHits = new SearchHits(hits, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1); + SearchResponseSections searchResponseSections = new SearchResponseSections(searchHits, null, null, false, false, null, 0); + return new SearchResponse(searchResponseSections, null, 1, 1, 0, 10, null, null); + } + + public void testSortResponse() throws Exception { + SearchRequest request = createDummyRequest(); + + SortResponseProcessor sortResponseProcessor = new SortResponseProcessor( + null, + null, + false, + "digits", + SortResponseProcessor.SortOrder.ASCENDING, + "sorted" + ); + SearchResponse response = createTestResponse(); + SearchResponse sortResponse = sortResponseProcessor.processResponse(request, response); + + assertEquals(response.getHits(), sortResponse.getHits()); + + assertEquals(PI, sortResponse.getHits().getHits()[0].field("digits").getValues()); + assertEquals(List.of(1, 1, 2, 3, 4, 5, 6, 9), sortResponse.getHits().getHits()[0].field("sorted").getValues()); + Map map = sortResponse.getHits().getHits()[0].getSourceAsMap(); + assertNotNull(map); + assertEquals(List.of(1, 1, 2, 3, 4, 5, 6, 9), map.get("sorted")); + + assertEquals(E, sortResponse.getHits().getHits()[1].field("digits").getValues()); + assertEquals(List.of(1, 1, 2, 2, 7, 8, 8, 8), sortResponse.getHits().getHits()[1].field("sorted").getValues()); + assertNull(sortResponse.getHits().getHits()[1].getSourceAsMap()); + } + + public void testSortResponseSameField() throws Exception { + SearchRequest request = createDummyRequest(); + + SortResponseProcessor sortResponseProcessor = new SortResponseProcessor( + null, + null, + false, + "digits", + SortResponseProcessor.SortOrder.DESCENDING, + null + ); + SearchResponse response = createTestResponse(); + SearchResponse sortResponse = sortResponseProcessor.processResponse(request, response); + + assertEquals(response.getHits(), sortResponse.getHits()); + assertEquals(List.of(9, 6, 5, 4, 3, 2, 1, 1), sortResponse.getHits().getHits()[0].field("digits").getValues()); + assertEquals(List.of(8, 8, 8, 7, 2, 2, 1, 1), sortResponse.getHits().getHits()[1].field("digits").getValues()); + } + + public void testSortResponseNullListEntry() { + SearchRequest request = createDummyRequest(); + + SortResponseProcessor sortResponseProcessor = new SortResponseProcessor( + null, + null, + false, + "digits", + SortResponseProcessor.SortOrder.ASCENDING, + null + ); + assertThrows( + IllegalArgumentException.class, + () -> sortResponseProcessor.processResponse(request, createTestResponseNullListEntry()) + ); + } + + public void testNullField() { + SearchRequest request = createDummyRequest(); + + SortResponseProcessor sortResponseProcessor = new SortResponseProcessor( + null, + null, + false, + "digits", + SortResponseProcessor.SortOrder.DESCENDING, + null + ); + + assertThrows(IllegalArgumentException.class, () -> sortResponseProcessor.processResponse(request, createTestResponseNullField())); + } + + public void testNotComparableField() { + SearchRequest request = createDummyRequest(); + + SortResponseProcessor sortResponseProcessor = new SortResponseProcessor( + null, + null, + false, + "maps", + SortResponseProcessor.SortOrder.ASCENDING, + null + ); + + assertThrows( + IllegalArgumentException.class, + () -> sortResponseProcessor.processResponse(request, createTestResponseNotComparable()) + ); + } + + public void testFactory() { + String sortField = RandomDocumentPicks.randomFieldName(random()); + String targetField = RandomDocumentPicks.randomFieldName(random()); + Map config = new HashMap<>(); + config.put("field", sortField); + config.put("order", "desc"); + config.put("target_field", targetField); + + SortResponseProcessor.Factory factory = new SortResponseProcessor.Factory(); + SortResponseProcessor processor = factory.create(Collections.emptyMap(), null, null, false, config, null); + assertEquals("sort", processor.getType()); + assertEquals(sortField, processor.getSortField()); + assertEquals(targetField, processor.getTargetField()); + assertEquals(SortResponseProcessor.SortOrder.DESCENDING, processor.getSortOrder()); + + expectThrows( + OpenSearchParseException.class, + () -> factory.create(Collections.emptyMap(), null, null, false, Collections.emptyMap(), null) + ); + } +} diff --git a/modules/search-pipeline-common/src/yamlRestTest/resources/rest-api-spec/test/search_pipeline/80_sort_response.yml b/modules/search-pipeline-common/src/yamlRestTest/resources/rest-api-spec/test/search_pipeline/80_sort_response.yml new file mode 100644 index 0000000000000..c160b550b2a6e --- /dev/null +++ b/modules/search-pipeline-common/src/yamlRestTest/resources/rest-api-spec/test/search_pipeline/80_sort_response.yml @@ -0,0 +1,152 @@ +--- +teardown: + - do: + search_pipeline.delete: + id: "my_pipeline" + ignore: 404 + +--- +"Test sort processor": + - do: + search_pipeline.put: + id: "my_pipeline" + body: > + { + "description": "test pipeline", + "response_processors": [ + { + "sort": + { + "field": "a", + "target_field": "b" + } + } + ] + } + - match: { acknowledged: true } + + - do: + search_pipeline.put: + id: "my_pipeline_2" + body: > + { + "description": "test pipeline with ignore failure true", + "response_processors": [ + { + "sort": + { + "field": "aa", + "ignore_failure": true + } + } + ] + } + - match: { acknowledged: true } + + - do: + search_pipeline.put: + id: "my_pipeline_3" + body: > + { + "description": "test pipeline", + "response_processors": [ + { + "sort": + { + "field": "a", + "order": "desc", + "target_field": "b" + } + } + ] + } + - match: { acknowledged: true } + + - do: + indices.create: + index: test + + - do: + indices.put_mapping: + index: test + body: + properties: + a: + type: integer + store: true + doc_values: true + + - do: + index: + index: test + id: 1 + body: { + "a": [ 3, 1, 4 ] + } + + - do: + indices.refresh: + index: test + + - do: + search: + body: { } + - match: { hits.total.value: 1 } + + - do: + search: + index: test + search_pipeline: "my_pipeline" + body: { } + - match: { hits.total.value: 1 } + - match: { hits.hits.0._source: { "a": [3, 1, 4], "b": [1, 3, 4] } } + + # Should also work with no search body specified + - do: + search: + index: test + search_pipeline: "my_pipeline" + - match: { hits.total.value: 1 } + - match: { hits.hits.0._source: { "a": [3, 1, 4], "b": [1, 3, 4] } } + + # Pipeline with ignore_failure set to true + # Should return while catching error + - do: + search: + index: test + search_pipeline: "my_pipeline_2" + - match: { hits.total.value: 1 } + - match: { hits.hits.0._source: { "a": [3, 1, 4] } } + + # Pipeline with desc sort order + - do: + search: + index: test + search_pipeline: "my_pipeline_3" + body: { } + - match: { hits.total.value: 1 } + - match: { hits.hits.0._source: { "a": [3, 1, 4], "b": [4, 3, 1] } } + + # No source, using stored_fields + - do: + search: + index: test + search_pipeline: "my_pipeline" + body: { + "_source": false, + "stored_fields": [ "a" ] + } + - match: { hits.hits.0.fields: { "a": [3, 1, 4], "b": [1, 3, 4] } } + + # No source, using docvalue_fields + - do: + search: + index: test + search_pipeline: "my_pipeline_3" + body: { + "_source": false, + "docvalue_fields": [ "a" ] + } + # a is stored sorted because docvalue_fields is pre-sorted to optimize aggregations + # this is poorly documented which makes it really hard to write "expected" values on tests + - match: { hits.hits.0.fields: { "a": [1, 3, 4], "b": [4, 3, 1] } } From 6227dc6ae70d82b7826f8f08bcc57b277c254056 Mon Sep 17 00:00:00 2001 From: "Park, Yeongwu" Date: Tue, 23 Jul 2024 05:24:51 +0900 Subject: [PATCH 14/37] Fix allowUnmappedFields, mapUnmappedFieldAsString settings to be applied when parsing query string query (#13957) * Modify to invoke QueryShardContext.fieldMapper() method to apply allowUnmappedFields and mapUnmappedFieldAsString settings Signed-off-by: imyp92 * Add test cases to verify returning 400 responses if unmapped fields are included for some types of query Signed-off-by: imyp92 * Add changelog Signed-off-by: imyp92 --------- Signed-off-by: imyp92 Signed-off-by: gaobinlong Co-authored-by: gaobinlong --- CHANGELOG.md | 1 + .../resources/rest-api-spec/test/10_basic.yml | 45 +++++++++++++++++++ .../index/query/ExistsQueryBuilder.java | 10 ++--- .../index/search/QueryParserHelper.java | 2 +- 4 files changed, 50 insertions(+), 8 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 80dd5a27ffdaa..ad655f3849b7e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -63,6 +63,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Remove query categorization changes ([#14759](https://github.com/opensearch-project/OpenSearch/pull/14759)) ### Fixed +- Fix allowUnmappedFields, mapUnmappedFieldAsString settings are not applied when parsing certain types of query string query ([#13957](https://github.com/opensearch-project/OpenSearch/pull/13957)) - Fix bug in SBP cancellation logic ([#13259](https://github.com/opensearch-project/OpenSearch/pull/13474)) - Fix handling of Short and Byte data types in ScriptProcessor ingest pipeline ([#14379](https://github.com/opensearch-project/OpenSearch/issues/14379)) - Switch to iterative version of WKT format parser ([#14086](https://github.com/opensearch-project/OpenSearch/pull/14086)) diff --git a/modules/percolator/src/yamlRestTest/resources/rest-api-spec/test/10_basic.yml b/modules/percolator/src/yamlRestTest/resources/rest-api-spec/test/10_basic.yml index 35ebb2b099139..61f79326dab06 100644 --- a/modules/percolator/src/yamlRestTest/resources/rest-api-spec/test/10_basic.yml +++ b/modules/percolator/src/yamlRestTest/resources/rest-api-spec/test/10_basic.yml @@ -83,3 +83,48 @@ index: documents_index id: some_id - match: { responses.0.hits.total: 1 } + + - do: + catch: bad_request + index: + index: queries_index + body: + query: + query_string: + query: "unmapped: *" + + - do: + catch: bad_request + index: + index: queries_index + body: + query: + query_string: + query: "_exists_: unmappedField" + + - do: + catch: bad_request + index: + index: queries_index + body: + query: + query_string: + query: "unmappedField: <100" + + - do: + catch: bad_request + index: + index: queries_index + body: + query: + query_string: + query: "unmappedField: test~" + + - do: + catch: bad_request + index: + index: queries_index + body: + query: + query_string: + query: "unmappedField: test*" diff --git a/server/src/main/java/org/opensearch/index/query/ExistsQueryBuilder.java b/server/src/main/java/org/opensearch/index/query/ExistsQueryBuilder.java index 3011a48fbb296..6ae40fe1b1e64 100644 --- a/server/src/main/java/org/opensearch/index/query/ExistsQueryBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/ExistsQueryBuilder.java @@ -230,20 +230,16 @@ private static Collection getMappedField(QueryShardContext context, Stri if (context.getObjectMapper(fieldPattern) != null) { // the _field_names field also indexes objects, so we don't have to // do any more work to support exists queries on whole objects - fields = Collections.singleton(fieldPattern); + return Collections.singleton(fieldPattern); } else { fields = context.simpleMatchToIndexNames(fieldPattern); } if (fields.size() == 1) { String field = fields.iterator().next(); - MappedFieldType fieldType = context.getMapperService().fieldType(field); + MappedFieldType fieldType = context.fieldMapper(field); if (fieldType == null) { - // The field does not exist as a leaf but could be an object so - // check for an object mapper - if (context.getObjectMapper(field) == null) { - return Collections.emptySet(); - } + return Collections.emptySet(); } } diff --git a/server/src/main/java/org/opensearch/index/search/QueryParserHelper.java b/server/src/main/java/org/opensearch/index/search/QueryParserHelper.java index 06f450f090e63..603e81f6bf113 100644 --- a/server/src/main/java/org/opensearch/index/search/QueryParserHelper.java +++ b/server/src/main/java/org/opensearch/index/search/QueryParserHelper.java @@ -143,7 +143,7 @@ static Map resolveMappingField( fieldName = fieldName + fieldSuffix; } - MappedFieldType fieldType = context.getMapperService().fieldType(fieldName); + MappedFieldType fieldType = context.fieldMapper(fieldName); if (fieldType == null) { fieldType = context.resolveDerivedFieldType(fieldName); } From 250feb29cdb87e8dec3bde32e27f9202e90c532b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 22 Jul 2024 16:39:33 -0400 Subject: [PATCH 15/37] Bump com.microsoft.azure:msal4j from 1.16.0 to 1.16.1 in /plugins/repository-azure (#14857) * Bump com.microsoft.azure:msal4j in /plugins/repository-azure Bumps [com.microsoft.azure:msal4j](https://github.com/AzureAD/microsoft-authentication-library-for-java) from 1.16.0 to 1.16.1. - [Release notes](https://github.com/AzureAD/microsoft-authentication-library-for-java/releases) - [Changelog](https://github.com/AzureAD/microsoft-authentication-library-for-java/blob/dev/changelog.txt) - [Commits](https://github.com/AzureAD/microsoft-authentication-library-for-java/compare/v1.16.0...v1.16.1) --- updated-dependencies: - dependency-name: com.microsoft.azure:msal4j 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 | 2 +- plugins/repository-azure/build.gradle | 2 +- plugins/repository-azure/licenses/msal4j-1.16.0.jar.sha1 | 1 - plugins/repository-azure/licenses/msal4j-1.16.1.jar.sha1 | 1 + 4 files changed, 3 insertions(+), 3 deletions(-) delete mode 100644 plugins/repository-azure/licenses/msal4j-1.16.0.jar.sha1 create mode 100644 plugins/repository-azure/licenses/msal4j-1.16.1.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index ad655f3849b7e..2f16af3c50c61 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -42,7 +42,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `opentelemetry-semconv` from 1.25.0-alpha to 1.26.0-alpha ([#14674](https://github.com/opensearch-project/OpenSearch/pull/14674)) - Bump `azure-identity` from 1.11.4 to 1.13.0, Bump `msal4j` from 1.14.3 to 1.15.1, Bump `msal4j-persistence-extension` from 1.2.0 to 1.3.0 ([#14506](https://github.com/opensearch-project/OpenSearch/pull/14673)) - Bump `com.azure:azure-storage-common` from 12.21.2 to 12.25.1 ([#14517](https://github.com/opensearch-project/OpenSearch/pull/14517)) -- Bump `com.microsoft.azure:msal4j` from 1.15.1 to 1.16.0 ([#14610](https://github.com/opensearch-project/OpenSearch/pull/14610)) +- Bump `com.microsoft.azure:msal4j` from 1.15.1 to 1.16.1 ([#14610](https://github.com/opensearch-project/OpenSearch/pull/14610), [#14857](https://github.com/opensearch-project/OpenSearch/pull/14857)) - Bump `com.github.spullara.mustache.java:compiler` from 0.9.13 to 0.9.14 ([#14672](https://github.com/opensearch-project/OpenSearch/pull/14672)) - Bump `net.minidev:accessors-smart` from 2.5.0 to 2.5.1 ([#14673](https://github.com/opensearch-project/OpenSearch/pull/14673)) - Bump `jackson` from 2.17.1 to 2.17.2 ([#14687](https://github.com/opensearch-project/OpenSearch/pull/14687)) diff --git a/plugins/repository-azure/build.gradle b/plugins/repository-azure/build.gradle index 980940e35b0b0..7bd7be1481a2f 100644 --- a/plugins/repository-azure/build.gradle +++ b/plugins/repository-azure/build.gradle @@ -61,7 +61,7 @@ dependencies { // Start of transitive dependencies for azure-identity api 'com.microsoft.azure:msal4j-persistence-extension:1.3.0' api "net.java.dev.jna:jna-platform:${versions.jna}" - api 'com.microsoft.azure:msal4j:1.16.0' + api 'com.microsoft.azure:msal4j:1.16.1' api 'com.nimbusds:oauth2-oidc-sdk:11.9.1' api 'com.nimbusds:nimbus-jose-jwt:9.40' api 'com.nimbusds:content-type:2.3' diff --git a/plugins/repository-azure/licenses/msal4j-1.16.0.jar.sha1 b/plugins/repository-azure/licenses/msal4j-1.16.0.jar.sha1 deleted file mode 100644 index 29fe5022a1570..0000000000000 --- a/plugins/repository-azure/licenses/msal4j-1.16.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -708a0a986ed091054f1c08866712e5b41aec6700 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/msal4j-1.16.1.jar.sha1 b/plugins/repository-azure/licenses/msal4j-1.16.1.jar.sha1 new file mode 100644 index 0000000000000..7d24922196be4 --- /dev/null +++ b/plugins/repository-azure/licenses/msal4j-1.16.1.jar.sha1 @@ -0,0 +1 @@ +4ad89b4632ef9abab883114e77c079843a206862 \ No newline at end of file From c7cebc5cc9ccc61b9798b30aa975901de1e343c3 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 22 Jul 2024 16:40:57 -0400 Subject: [PATCH 16/37] Bump com.gradle.develocity from 3.17.5 to 3.17.6 (#14856) * Bump com.gradle.develocity from 3.17.5 to 3.17.6 Bumps com.gradle.develocity from 3.17.5 to 3.17.6. --- updated-dependencies: - dependency-name: com.gradle.develocity dependency-type: direct:production update-type: version-update:semver-patch ... 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 | 2 +- settings.gradle | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 2f16af3c50c61..fdce3a5e24342 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -37,7 +37,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `commons-net:commons-net` from 3.10.0 to 3.11.1 ([#14396](https://github.com/opensearch-project/OpenSearch/pull/14396)) - Bump `com.nimbusds:nimbus-jose-jwt` from 9.37.3 to 9.40 ([#14398](https://github.com/opensearch-project/OpenSearch/pull/14398)) - Bump `org.apache.commons:commons-configuration2` from 2.10.1 to 2.11.0 ([#14399](https://github.com/opensearch-project/OpenSearch/pull/14399)) -- Bump `com.gradle.develocity` from 3.17.4 to 3.17.5 ([#14397](https://github.com/opensearch-project/OpenSearch/pull/14397)) +- Bump `com.gradle.develocity` from 3.17.4 to 3.17.6 ([#14397](https://github.com/opensearch-project/OpenSearch/pull/14397), [#14856](https://github.com/opensearch-project/OpenSearch/pull/14856)) - Bump `opentelemetry` from 1.36.0 to 1.40.0 ([#14457](https://github.com/opensearch-project/OpenSearch/pull/14457), [#14674](https://github.com/opensearch-project/OpenSearch/pull/14674)) - Bump `opentelemetry-semconv` from 1.25.0-alpha to 1.26.0-alpha ([#14674](https://github.com/opensearch-project/OpenSearch/pull/14674)) - Bump `azure-identity` from 1.11.4 to 1.13.0, Bump `msal4j` from 1.14.3 to 1.15.1, Bump `msal4j-persistence-extension` from 1.2.0 to 1.3.0 ([#14506](https://github.com/opensearch-project/OpenSearch/pull/14673)) diff --git a/settings.gradle b/settings.gradle index a96d00a4ab863..ae9f5384be592 100644 --- a/settings.gradle +++ b/settings.gradle @@ -10,7 +10,7 @@ */ plugins { - id "com.gradle.develocity" version "3.17.5" + id "com.gradle.develocity" version "3.17.6" } ext.disableBuildCache = hasProperty('DISABLE_BUILD_CACHE') || System.getenv().containsKey('DISABLE_BUILD_CACHE') From 11a9730196ba9f789c5114033aa1596d86013880 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 22 Jul 2024 16:47:01 -0400 Subject: [PATCH 17/37] Bump org.jline:jline in /test/fixtures/hdfs-fixture (#14859) Bumps [org.jline:jline](https://github.com/jline/jline3) from 3.26.2 to 3.26.3. - [Release notes](https://github.com/jline/jline3/releases) - [Changelog](https://github.com/jline/jline3/blob/master/changelog.md) - [Commits](https://github.com/jline/jline3/compare/jline-parent-3.26.2...jline-parent-3.26.3) --- updated-dependencies: - dependency-name: org.jline:jline dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- test/fixtures/hdfs-fixture/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/fixtures/hdfs-fixture/build.gradle b/test/fixtures/hdfs-fixture/build.gradle index a3c2932be64c4..9b8f62b8c55b8 100644 --- a/test/fixtures/hdfs-fixture/build.gradle +++ b/test/fixtures/hdfs-fixture/build.gradle @@ -76,7 +76,7 @@ dependencies { api "ch.qos.logback:logback-core:1.5.6" api "ch.qos.logback:logback-classic:1.2.13" api "org.jboss.xnio:xnio-nio:3.8.16.Final" - api 'org.jline:jline:3.26.2' + api 'org.jline:jline:3.26.3' api 'org.apache.commons:commons-configuration2:2.11.0' api 'com.nimbusds:nimbus-jose-jwt:9.40' api ('org.apache.kerby:kerb-admin:2.0.3') { From 4e45c9ed68d7a4ba77c8c3406453c05bede170e2 Mon Sep 17 00:00:00 2001 From: ebraminio Date: Tue, 23 Jul 2024 00:55:43 +0330 Subject: [PATCH 18/37] Use Lucene provided Persian stem (#14847) Lucene provided Persian stem apparently isn't hooked yet and this change is doing that based on what is done for Arabic stem support. Signed-off-by: Ebrahim Byagowi Signed-off-by: Daniel (dB.) Doubrovkine Co-authored-by: Daniel (dB.) Doubrovkine --- CHANGELOG.md | 1 + .../common/CommonAnalysisModulePlugin.java | 3 ++ .../common/PersianStemTokenFilterFactory.java | 52 +++++++++++++++++++ .../common/StemmerTokenFilterFactory.java | 3 ++ .../common/CommonAnalysisFactoryTests.java | 2 + .../test/analysis-common/40_token_filters.yml | 31 +++++++++++ .../analysis/AnalysisFactoryTestCase.java | 2 +- 7 files changed, 93 insertions(+), 1 deletion(-) create mode 100644 modules/analysis-common/src/main/java/org/opensearch/analysis/common/PersianStemTokenFilterFactory.java diff --git a/CHANGELOG.md b/CHANGELOG.md index fdce3a5e24342..66322087a73c4 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -26,6 +26,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Optimize TransportNodesAction to not send DiscoveryNodes for NodeStats, NodesInfo and ClusterStats call ([14749](https://github.com/opensearch-project/OpenSearch/pull/14749)) - Reduce logging in DEBUG for MasterService:run ([#14795](https://github.com/opensearch-project/OpenSearch/pull/14795)) - Enabling term version check on local state for all ClusterManager Read Transport Actions ([#14273](https://github.com/opensearch-project/OpenSearch/pull/14273)) +- Add persian_stem filter (([#14847](https://github.com/opensearch-project/OpenSearch/pull/14847))) ### Dependencies - Bump `org.gradle.test-retry` from 1.5.8 to 1.5.9 ([#13442](https://github.com/opensearch-project/OpenSearch/pull/13442)) 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 cf2736a8583d2..f14e499081ce9 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 @@ -75,6 +75,7 @@ import org.apache.lucene.analysis.eu.BasqueAnalyzer; import org.apache.lucene.analysis.fa.PersianAnalyzer; import org.apache.lucene.analysis.fa.PersianNormalizationFilter; +import org.apache.lucene.analysis.fa.PersianStemFilter; import org.apache.lucene.analysis.fi.FinnishAnalyzer; import org.apache.lucene.analysis.fr.FrenchAnalyzer; import org.apache.lucene.analysis.ga.IrishAnalyzer; @@ -315,6 +316,7 @@ public Map> getTokenFilters() { filters.put("pattern_capture", requiresAnalysisSettings(PatternCaptureGroupTokenFilterFactory::new)); filters.put("pattern_replace", requiresAnalysisSettings(PatternReplaceTokenFilterFactory::new)); filters.put("persian_normalization", PersianNormalizationFilterFactory::new); + filters.put("persian_stem", PersianStemTokenFilterFactory::new); filters.put("porter_stem", PorterStemTokenFilterFactory::new); filters.put( "predicate_token_filter", @@ -558,6 +560,7 @@ public List getPreConfiguredTokenFilters() { ); })); filters.add(PreConfiguredTokenFilter.singleton("persian_normalization", true, PersianNormalizationFilter::new)); + filters.add(PreConfiguredTokenFilter.singleton("persian_stem", true, PersianStemFilter::new)); filters.add(PreConfiguredTokenFilter.singleton("porter_stem", false, PorterStemFilter::new)); filters.add(PreConfiguredTokenFilter.singleton("reverse", false, ReverseStringFilter::new)); filters.add(PreConfiguredTokenFilter.singleton("russian_stem", false, input -> new SnowballFilter(input, "Russian"))); diff --git a/modules/analysis-common/src/main/java/org/opensearch/analysis/common/PersianStemTokenFilterFactory.java b/modules/analysis-common/src/main/java/org/opensearch/analysis/common/PersianStemTokenFilterFactory.java new file mode 100644 index 0000000000000..afe8058343e17 --- /dev/null +++ b/modules/analysis-common/src/main/java/org/opensearch/analysis/common/PersianStemTokenFilterFactory.java @@ -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. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you 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. + */ + +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.analysis.common; + +import org.apache.lucene.analysis.TokenStream; +import org.apache.lucene.analysis.fa.PersianStemFilter; +import org.opensearch.common.settings.Settings; +import org.opensearch.env.Environment; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.analysis.AbstractTokenFilterFactory; + +public class PersianStemTokenFilterFactory extends AbstractTokenFilterFactory { + + PersianStemTokenFilterFactory(IndexSettings indexSettings, Environment environment, String name, Settings settings) { + super(indexSettings, name, settings); + } + + @Override + public TokenStream create(TokenStream tokenStream) { + return new PersianStemFilter(tokenStream); + } +} diff --git a/modules/analysis-common/src/main/java/org/opensearch/analysis/common/StemmerTokenFilterFactory.java b/modules/analysis-common/src/main/java/org/opensearch/analysis/common/StemmerTokenFilterFactory.java index 5506626e40da0..e81f3c6cc09cc 100644 --- a/modules/analysis-common/src/main/java/org/opensearch/analysis/common/StemmerTokenFilterFactory.java +++ b/modules/analysis-common/src/main/java/org/opensearch/analysis/common/StemmerTokenFilterFactory.java @@ -47,6 +47,7 @@ import org.apache.lucene.analysis.en.KStemFilter; import org.apache.lucene.analysis.en.PorterStemFilter; import org.apache.lucene.analysis.es.SpanishLightStemFilter; +import org.apache.lucene.analysis.fa.PersianStemFilter; import org.apache.lucene.analysis.fi.FinnishLightStemFilter; import org.apache.lucene.analysis.fr.FrenchLightStemFilter; import org.apache.lucene.analysis.fr.FrenchMinimalStemFilter; @@ -239,6 +240,8 @@ public TokenStream create(TokenStream tokenStream) { return new NorwegianLightStemFilter(tokenStream, NorwegianLightStemmer.NYNORSK); } else if ("minimal_nynorsk".equalsIgnoreCase(language) || "minimalNynorsk".equalsIgnoreCase(language)) { return new NorwegianMinimalStemFilter(tokenStream, NorwegianLightStemmer.NYNORSK); + } else if ("persian".equalsIgnoreCase(language)) { + return new PersianStemFilter(tokenStream); // Portuguese stemmers } else if ("portuguese".equalsIgnoreCase(language)) { 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 11713f52f5b18..7e3140f8bcba3 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 @@ -158,6 +158,7 @@ protected Map> getTokenFilters() { filters.put("brazilianstem", BrazilianStemTokenFilterFactory.class); filters.put("czechstem", CzechStemTokenFilterFactory.class); filters.put("germanstem", GermanStemTokenFilterFactory.class); + filters.put("persianstem", PersianStemTokenFilterFactory.class); filters.put("telugunormalization", TeluguNormalizationFilterFactory.class); filters.put("telugustem", TeluguStemFilterFactory.class); // this filter is not exposed and should only be used internally @@ -220,6 +221,7 @@ protected Map> getPreConfiguredTokenFilters() { filters.put("ngram", null); filters.put("nGram", null); filters.put("persian_normalization", null); + filters.put("persian_stem", null); filters.put("porter_stem", null); filters.put("reverse", ReverseStringFilterFactory.class); filters.put("russian_stem", SnowballPorterFilterFactory.class); 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 802c79c780689..c6b075571f221 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 @@ -1781,6 +1781,37 @@ - length: { tokens: 1 } - match: { tokens.0.token: abschliess } +--- +"persian_stem": + - do: + indices.create: + index: test + body: + settings: + analysis: + filter: + my_persian_stem: + type: persian_stem + - do: + indices.analyze: + index: test + body: + text: جامدات + tokenizer: keyword + filter: [my_persian_stem] + - length: { tokens: 1 } + - match: { tokens.0.token: جامد } + + # Test pre-configured token filter too: + - do: + indices.analyze: + body: + text: جامدات + tokenizer: keyword + filter: [persian_stem] + - length: { tokens: 1 } + - match: { tokens.0.token: جامد } + --- "russian_stem": - do: 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 5231fe095f0f0..23cf4d47a49d9 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 @@ -139,6 +139,7 @@ public abstract class AnalysisFactoryTestCase extends OpenSearchTestCase { .put("patterncapturegroup", MovedToAnalysisCommon.class) .put("patternreplace", MovedToAnalysisCommon.class) .put("persiannormalization", MovedToAnalysisCommon.class) + .put("persianstem", MovedToAnalysisCommon.class) .put("porterstem", MovedToAnalysisCommon.class) .put("portuguesestem", MovedToAnalysisCommon.class) .put("portugueselightstem", MovedToAnalysisCommon.class) @@ -219,7 +220,6 @@ public abstract class AnalysisFactoryTestCase extends OpenSearchTestCase { .put("spanishpluralstem", Void.class) // LUCENE-10352 .put("daitchmokotoffsoundex", Void.class) - .put("persianstem", Void.class) // https://github.com/apache/lucene/pull/12169 .put("word2vecsynonym", Void.class) // https://github.com/apache/lucene/pull/12915 From 58451061e59c0d811a70367b75d7af6671ee9911 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 22 Jul 2024 17:27:43 -0400 Subject: [PATCH 19/37] Bump actions/checkout from 2 to 4 (#14858) * Bump actions/checkout from 2 to 4 Bumps [actions/checkout](https://github.com/actions/checkout) from 2 to 4. - [Release notes](https://github.com/actions/checkout/releases) - [Changelog](https://github.com/actions/checkout/blob/main/CHANGELOG.md) - [Commits](https://github.com/actions/checkout/compare/v2...v4) --- updated-dependencies: - dependency-name: actions/checkout dependency-type: direct:production update-type: version-update:semver-major ... 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] --- .github/workflows/benchmark-pull-request.yml | 4 ++-- CHANGELOG.md | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/.github/workflows/benchmark-pull-request.yml b/.github/workflows/benchmark-pull-request.yml index 2e2e83eb132de..9d83331e81d5a 100644 --- a/.github/workflows/benchmark-pull-request.yml +++ b/.github/workflows/benchmark-pull-request.yml @@ -13,7 +13,7 @@ jobs: pull-requests: write steps: - name: Checkout Repository - uses: actions/checkout@v3 + uses: actions/checkout@v4 - name: Set up required env vars run: | echo "PR_NUMBER=${{ github.event.issue.number }}" >> $GITHUB_ENV @@ -117,7 +117,7 @@ jobs: echo "prHeadRepo=$headRepo" >> $GITHUB_ENV echo "prHeadRef=$headRef" >> $GITHUB_ENV - name: Checkout PR Repo - uses: actions/checkout@v2 + uses: actions/checkout@v4 with: repository: ${{ env.prHeadRepo }} ref: ${{ env.prHeadRef }} diff --git a/CHANGELOG.md b/CHANGELOG.md index 66322087a73c4..f90424ab07870 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -48,6 +48,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `net.minidev:accessors-smart` from 2.5.0 to 2.5.1 ([#14673](https://github.com/opensearch-project/OpenSearch/pull/14673)) - Bump `jackson` from 2.17.1 to 2.17.2 ([#14687](https://github.com/opensearch-project/OpenSearch/pull/14687)) - Bump `net.minidev:json-smart` from 2.5.0 to 2.5.1 ([#14748](https://github.com/opensearch-project/OpenSearch/pull/14748)) +- Bump `actions/checkout` from 2 to 4 ([#14858](https://github.com/opensearch-project/OpenSearch/pull/14858)) ### Changed - [Tiered Caching] Move query recomputation logic outside write lock ([#14187](https://github.com/opensearch-project/OpenSearch/pull/14187)) From 97f26ccfd56bc52f91cad74368662f5cfd5811df Mon Sep 17 00:00:00 2001 From: Liyun Xiu Date: Tue, 23 Jul 2024 05:35:07 +0800 Subject: [PATCH 20/37] Deprecate batch_size parameter on bulk API (#14725) By default the full _bulk payload will be passed to ingest processors as a batch, with any sub batching logic to be implemented by each processor if necessary. Signed-off-by: Liyun Xiu --- CHANGELOG.md | 1 + .../rest-api-spec/test/ingest/70_bulk.yml | 33 +------- .../org/opensearch/ingest/IngestClientIT.java | 81 +++++++++++++++++++ .../opensearch/action/bulk/BulkRequest.java | 2 +- .../org/opensearch/ingest/IngestService.java | 64 +-------------- .../rest/action/document/RestBulkAction.java | 8 +- .../opensearch/ingest/IngestServiceTests.java | 53 ++++++++++-- 7 files changed, 141 insertions(+), 101 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index f90424ab07870..0931ff63c145b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -60,6 +60,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Allow system index warning in OpenSearchRestTestCase.refreshAllIndices ([#14635](https://github.com/opensearch-project/OpenSearch/pull/14635)) ### Deprecated +- Deprecate batch_size parameter on bulk API ([#14725](https://github.com/opensearch-project/OpenSearch/pull/14725)) ### Removed - Remove query categorization changes ([#14759](https://github.com/opensearch-project/OpenSearch/pull/14759)) diff --git a/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/70_bulk.yml b/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/70_bulk.yml index 36b2b5351dcad..47cc80d6df310 100644 --- a/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/70_bulk.yml +++ b/modules/ingest-common/src/yamlRestTest/resources/rest-api-spec/test/ingest/70_bulk.yml @@ -207,7 +207,7 @@ teardown: - match: { _source: {"f1": "v2", "f2": 47, "field1": "value1", "field2": "value2"}} --- -"Test bulk API with batch enabled happy case": +"Test bulk API with default batch size": - skip: version: " - 2.13.99" reason: "Added in 2.14.0" @@ -215,7 +215,6 @@ teardown: - do: bulk: refresh: true - batch_size: 2 pipeline: "pipeline1" body: - '{"index": {"_index": "test_index", "_id": "test_id1"}}' @@ -245,36 +244,6 @@ teardown: id: test_id3 - match: { _source: { "text": "text3", "field1": "value1" } } ---- -"Test bulk API with batch_size missing": - - skip: - version: " - 2.13.99" - reason: "Added in 2.14.0" - - - do: - bulk: - refresh: true - pipeline: "pipeline1" - body: - - '{"index": {"_index": "test_index", "_id": "test_id1"}}' - - '{"text": "text1"}' - - '{"index": {"_index": "test_index", "_id": "test_id2"}}' - - '{"text": "text2"}' - - - match: { errors: false } - - - do: - get: - index: test_index - id: test_id1 - - match: { _source: { "text": "text1", "field1": "value1" } } - - - do: - get: - index: test_index - id: test_id2 - - match: { _source: { "text": "text2", "field1": "value1" } } - --- "Test bulk API with invalid batch_size": - skip: diff --git a/server/src/internalClusterTest/java/org/opensearch/ingest/IngestClientIT.java b/server/src/internalClusterTest/java/org/opensearch/ingest/IngestClientIT.java index 657d0f178e096..0eb37a7b25618 100644 --- a/server/src/internalClusterTest/java/org/opensearch/ingest/IngestClientIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/ingest/IngestClientIT.java @@ -315,6 +315,87 @@ public void testBulkWithUpsert() throws Exception { assertThat(upserted.get("processed"), equalTo(true)); } + public void testSingleDocIngestFailure() throws Exception { + createIndex("test"); + BytesReference source = BytesReference.bytes( + jsonBuilder().startObject() + .field("description", "my_pipeline") + .startArray("processors") + .startObject() + .startObject("test") + .endObject() + .endObject() + .endArray() + .endObject() + ); + PutPipelineRequest putPipelineRequest = new PutPipelineRequest("_id", source, MediaTypeRegistry.JSON); + client().admin().cluster().putPipeline(putPipelineRequest).get(); + + GetPipelineRequest getPipelineRequest = new GetPipelineRequest("_id"); + GetPipelineResponse getResponse = client().admin().cluster().getPipeline(getPipelineRequest).get(); + assertThat(getResponse.isFound(), is(true)); + assertThat(getResponse.pipelines().size(), equalTo(1)); + assertThat(getResponse.pipelines().get(0).getId(), equalTo("_id")); + + assertThrows( + IllegalArgumentException.class, + () -> client().prepareIndex("test") + .setId("1") + .setPipeline("_id") + .setSource(Requests.INDEX_CONTENT_TYPE, "field", "value", "fail", true) + .get() + ); + + DeletePipelineRequest deletePipelineRequest = new DeletePipelineRequest("_id"); + AcknowledgedResponse response = client().admin().cluster().deletePipeline(deletePipelineRequest).get(); + assertThat(response.isAcknowledged(), is(true)); + + getResponse = client().admin().cluster().prepareGetPipeline("_id").get(); + assertThat(getResponse.isFound(), is(false)); + assertThat(getResponse.pipelines().size(), equalTo(0)); + } + + public void testSingleDocIngestDrop() throws Exception { + createIndex("test"); + BytesReference source = BytesReference.bytes( + jsonBuilder().startObject() + .field("description", "my_pipeline") + .startArray("processors") + .startObject() + .startObject("test") + .endObject() + .endObject() + .endArray() + .endObject() + ); + PutPipelineRequest putPipelineRequest = new PutPipelineRequest("_id", source, MediaTypeRegistry.JSON); + client().admin().cluster().putPipeline(putPipelineRequest).get(); + + GetPipelineRequest getPipelineRequest = new GetPipelineRequest("_id"); + GetPipelineResponse getResponse = client().admin().cluster().getPipeline(getPipelineRequest).get(); + assertThat(getResponse.isFound(), is(true)); + assertThat(getResponse.pipelines().size(), equalTo(1)); + assertThat(getResponse.pipelines().get(0).getId(), equalTo("_id")); + + DocWriteResponse indexResponse = client().prepareIndex("test") + .setId("1") + .setPipeline("_id") + .setSource(Requests.INDEX_CONTENT_TYPE, "field", "value", "drop", true) + .get(); + assertEquals(DocWriteResponse.Result.NOOP, indexResponse.getResult()); + + Map doc = client().prepareGet("test", "1").get().getSourceAsMap(); + assertNull(doc); + + DeletePipelineRequest deletePipelineRequest = new DeletePipelineRequest("_id"); + AcknowledgedResponse response = client().admin().cluster().deletePipeline(deletePipelineRequest).get(); + assertThat(response.isAcknowledged(), is(true)); + + getResponse = client().admin().cluster().prepareGetPipeline("_id").get(); + assertThat(getResponse.isFound(), is(false)); + assertThat(getResponse.pipelines().size(), equalTo(0)); + } + public void test() throws Exception { BytesReference source = BytesReference.bytes( jsonBuilder().startObject() diff --git a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java index 7614206cd226f..e686585095962 100644 --- a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java +++ b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java @@ -96,7 +96,7 @@ public class BulkRequest extends ActionRequest implements CompositeIndicesReques private String globalRouting; private String globalIndex; private Boolean globalRequireAlias; - private int batchSize = 1; + private int batchSize = Integer.MAX_VALUE; private long sizeInBytes = 0; diff --git a/server/src/main/java/org/opensearch/ingest/IngestService.java b/server/src/main/java/org/opensearch/ingest/IngestService.java index 2281ccd4c0382..17eb23422e68b 100644 --- a/server/src/main/java/org/opensearch/ingest/IngestService.java +++ b/server/src/main/java/org/opensearch/ingest/IngestService.java @@ -525,61 +525,7 @@ public void onFailure(Exception e) { @Override protected void doRun() { - int batchSize = originalBulkRequest.batchSize(); - if (shouldExecuteBulkRequestInBatch(originalBulkRequest.requests().size(), batchSize)) { - runBulkRequestInBatch(numberOfActionRequests, actionRequests, onFailure, onCompletion, onDropped, originalBulkRequest); - return; - } - - final Thread originalThread = Thread.currentThread(); - final AtomicInteger counter = new AtomicInteger(numberOfActionRequests); - int i = 0; - for (DocWriteRequest actionRequest : actionRequests) { - IndexRequest indexRequest = TransportBulkAction.getIndexWriteRequest(actionRequest); - if (indexRequest == null) { - if (counter.decrementAndGet() == 0) { - onCompletion.accept(originalThread, null); - } - assert counter.get() >= 0; - i++; - continue; - } - final String pipelineId = indexRequest.getPipeline(); - indexRequest.setPipeline(NOOP_PIPELINE_NAME); - final String finalPipelineId = indexRequest.getFinalPipeline(); - indexRequest.setFinalPipeline(NOOP_PIPELINE_NAME); - boolean hasFinalPipeline = true; - final List pipelines; - if (IngestService.NOOP_PIPELINE_NAME.equals(pipelineId) == false - && IngestService.NOOP_PIPELINE_NAME.equals(finalPipelineId) == false) { - pipelines = Arrays.asList(pipelineId, finalPipelineId); - } else if (IngestService.NOOP_PIPELINE_NAME.equals(pipelineId) == false) { - pipelines = Collections.singletonList(pipelineId); - hasFinalPipeline = false; - } else if (IngestService.NOOP_PIPELINE_NAME.equals(finalPipelineId) == false) { - pipelines = Collections.singletonList(finalPipelineId); - } else { - if (counter.decrementAndGet() == 0) { - onCompletion.accept(originalThread, null); - } - assert counter.get() >= 0; - i++; - continue; - } - - executePipelines( - i, - pipelines.iterator(), - hasFinalPipeline, - indexRequest, - onDropped, - onFailure, - counter, - onCompletion, - originalThread - ); - i++; - } + runBulkRequestInBatch(numberOfActionRequests, actionRequests, onFailure, onCompletion, onDropped, originalBulkRequest); } }); } @@ -635,7 +581,7 @@ private void runBulkRequestInBatch( i++; } - int batchSize = originalBulkRequest.batchSize(); + int batchSize = Math.min(numberOfActionRequests, originalBulkRequest.batchSize()); List> batches = prepareBatches(batchSize, indexRequestWrappers); logger.debug("batchSize: {}, batches: {}", batchSize, batches.size()); @@ -654,10 +600,6 @@ private void runBulkRequestInBatch( } } - private boolean shouldExecuteBulkRequestInBatch(int documentSize, int batchSize) { - return documentSize > 1 && batchSize > 1; - } - /** * IndexRequests are grouped by unique (index + pipeline_ids) before batching. * Only IndexRequests in the same group could be batched. It's to ensure batched documents always @@ -685,7 +627,7 @@ static List> prepareBatches(int batchSize, List> batchedIndexRequests = new ArrayList<>(); for (Map.Entry> indexRequestsPerKey : indexRequestsPerIndexAndPipelines.entrySet()) { - for (int i = 0; i < indexRequestsPerKey.getValue().size(); i += batchSize) { + for (int i = 0; i < indexRequestsPerKey.getValue().size(); i += Math.min(indexRequestsPerKey.getValue().size(), batchSize)) { batchedIndexRequests.add( new ArrayList<>( indexRequestsPerKey.getValue().subList(i, i + Math.min(batchSize, indexRequestsPerKey.getValue().size() - i)) diff --git a/server/src/main/java/org/opensearch/rest/action/document/RestBulkAction.java b/server/src/main/java/org/opensearch/rest/action/document/RestBulkAction.java index 0bc4234c9b8b8..ce52c5620b968 100644 --- a/server/src/main/java/org/opensearch/rest/action/document/RestBulkAction.java +++ b/server/src/main/java/org/opensearch/rest/action/document/RestBulkAction.java @@ -38,6 +38,7 @@ import org.opensearch.action.support.ActiveShardCount; import org.opensearch.client.Requests; import org.opensearch.client.node.NodeClient; +import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.settings.Settings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; @@ -66,6 +67,8 @@ public class RestBulkAction extends BaseRestHandler { private final boolean allowExplicitIndex; + private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(RestBulkAction.class); + static final String BATCH_SIZE_DEPRECATED_MESSAGE = "The batch size option in bulk API is deprecated and will be removed in 3.0."; public RestBulkAction(Settings settings) { this.allowExplicitIndex = MULTI_ALLOW_EXPLICIT_INDEX.get(settings); @@ -97,7 +100,10 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC Boolean defaultRequireAlias = request.paramAsBoolean(DocWriteRequest.REQUIRE_ALIAS, null); bulkRequest.timeout(request.paramAsTime("timeout", BulkShardRequest.DEFAULT_TIMEOUT)); bulkRequest.setRefreshPolicy(request.param("refresh")); - bulkRequest.batchSize(request.paramAsInt("batch_size", 1)); + if (request.hasParam("batch_size")) { + deprecationLogger.deprecate("batch_size_deprecation", BATCH_SIZE_DEPRECATED_MESSAGE); + } + bulkRequest.batchSize(request.paramAsInt("batch_size", Integer.MAX_VALUE)); bulkRequest.add( request.requiredContent(), defaultIndex, diff --git a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java index e61fbb6e1dbff..9d03127692975 100644 --- a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java +++ b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java @@ -1134,10 +1134,14 @@ public void testBulkRequestExecutionWithFailures() throws Exception { Exception error = new RuntimeException(); doAnswer(args -> { @SuppressWarnings("unchecked") - BiConsumer handler = (BiConsumer) args.getArguments()[1]; - handler.accept(null, error); + List ingestDocumentWrappers = (List) args.getArguments()[0]; + Consumer> handler = (Consumer) args.getArguments()[1]; + for (IngestDocumentWrapper wrapper : ingestDocumentWrappers) { + wrapper.update(wrapper.getIngestDocument(), error); + } + handler.accept(ingestDocumentWrappers); return null; - }).when(processor).execute(any(), any()); + }).when(processor).batchExecute(any(), any()); IngestService ingestService = createWithProcessors( Collections.singletonMap("mock", (factories, tag, description, config) -> processor) ); @@ -1192,10 +1196,11 @@ public void testBulkRequestExecution() throws Exception { when(processor.getTag()).thenReturn("mockTag"); doAnswer(args -> { @SuppressWarnings("unchecked") - BiConsumer handler = (BiConsumer) args.getArguments()[1]; - handler.accept(RandomDocumentPicks.randomIngestDocument(random()), null); + List ingestDocumentWrappers = (List) args.getArguments()[0]; + Consumer> handler = (Consumer) args.getArguments()[1]; + handler.accept(ingestDocumentWrappers); return null; - }).when(processor).execute(any(), any()); + }).when(processor).batchExecute(any(), any()); Map map = new HashMap<>(2); map.put("mock", (factories, tag, description, config) -> processor); @@ -1957,6 +1962,42 @@ public void testExecuteBulkRequestInBatchWithExceptionAndDropInCallback() { verify(mockCompoundProcessor, never()).execute(any(), any()); } + public void testExecuteBulkRequestInBatchWithDefaultBatchSize() { + CompoundProcessor mockCompoundProcessor = mockCompoundProcessor(); + IngestService ingestService = createWithProcessors( + Collections.singletonMap("mock", (factories, tag, description, config) -> mockCompoundProcessor) + ); + createPipeline("_id", ingestService); + BulkRequest bulkRequest = new BulkRequest(); + IndexRequest indexRequest1 = new IndexRequest("_index").id("_id1").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); + bulkRequest.add(indexRequest1); + IndexRequest indexRequest2 = new IndexRequest("_index").id("_id2").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); + bulkRequest.add(indexRequest2); + IndexRequest indexRequest3 = new IndexRequest("_index").id("_id3").source(emptyMap()).setPipeline("_none").setFinalPipeline("_id"); + bulkRequest.add(indexRequest3); + IndexRequest indexRequest4 = new IndexRequest("_index").id("_id4").source(emptyMap()).setPipeline("_id").setFinalPipeline("_none"); + bulkRequest.add(indexRequest4); + @SuppressWarnings("unchecked") + final Map failureHandler = new HashMap<>(); + final Map completionHandler = new HashMap<>(); + final List dropHandler = new ArrayList<>(); + ingestService.executeBulkRequest( + 4, + bulkRequest.requests(), + failureHandler::put, + completionHandler::put, + dropHandler::add, + Names.WRITE, + bulkRequest + ); + assertTrue(failureHandler.isEmpty()); + assertTrue(dropHandler.isEmpty()); + assertEquals(1, completionHandler.size()); + assertNull(completionHandler.get(Thread.currentThread())); + verify(mockCompoundProcessor, times(1)).batchExecute(any(), any()); + verify(mockCompoundProcessor, never()).execute(any(), any()); + } + public void testPrepareBatches_same_index_pipeline() { IngestService.IndexRequestWrapper wrapper1 = createIndexRequestWrapper("index1", Collections.singletonList("p1")); IngestService.IndexRequestWrapper wrapper2 = createIndexRequestWrapper("index1", Collections.singletonList("p1")); From 90d5500ecbf13b08d2f6a9fa6ad67119acd37a17 Mon Sep 17 00:00:00 2001 From: Finn Date: Mon, 22 Jul 2024 16:59:47 -0700 Subject: [PATCH 21/37] Add perms for remote snapshot cache eviction on scripted query (#14411) Signed-off-by: Finn Carroll --- CHANGELOG.md | 1 + .../store/remote/utils/TransferManager.java | 74 +++++++++---------- 2 files changed, 38 insertions(+), 37 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 0931ff63c145b..ec5b838a542c4 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -88,6 +88,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Fix NPE in ReplicaShardAllocator ([#14385](https://github.com/opensearch-project/OpenSearch/pull/14385)) - Fix constant_keyword field type used when creating index ([#14807](https://github.com/opensearch-project/OpenSearch/pull/14807)) - Use circuit breaker in InternalHistogram when adding empty buckets ([#14754](https://github.com/opensearch-project/OpenSearch/pull/14754)) +- Fix searchable snapshot failure with scripted fields ([#14411](https://github.com/opensearch-project/OpenSearch/pull/14411)) ### Security diff --git a/server/src/main/java/org/opensearch/index/store/remote/utils/TransferManager.java b/server/src/main/java/org/opensearch/index/store/remote/utils/TransferManager.java index df26f2f0925f6..f07c4832d982c 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/utils/TransferManager.java +++ b/server/src/main/java/org/opensearch/index/store/remote/utils/TransferManager.java @@ -64,16 +64,22 @@ public IndexInput fetchBlob(BlobFetchRequest blobFetchRequest) throws IOExceptio final Path key = blobFetchRequest.getFilePath(); logger.trace("fetchBlob called for {}", key.toString()); - final CachedIndexInput cacheEntry = fileCache.compute(key, (path, cachedIndexInput) -> { - if (cachedIndexInput == null || cachedIndexInput.isClosed()) { - logger.trace("Transfer Manager - IndexInput closed or not in cache"); - // Doesn't exist or is closed, either way create a new one - return new DelayedCreationCachedIndexInput(fileCache, streamReader, blobFetchRequest); - } else { - logger.trace("Transfer Manager - Already in cache"); - // already in the cache and ready to be used (open) - return cachedIndexInput; - } + // We need to do a privileged action here in order to fetch from remote + // and write/evict from local file cache in case this is invoked as a side + // effect of a plugin (such as a scripted search) that doesn't have the + // necessary permissions. + final CachedIndexInput cacheEntry = AccessController.doPrivileged((PrivilegedAction) () -> { + return fileCache.compute(key, (path, cachedIndexInput) -> { + if (cachedIndexInput == null || cachedIndexInput.isClosed()) { + logger.trace("Transfer Manager - IndexInput closed or not in cache"); + // Doesn't exist or is closed, either way create a new one + return new DelayedCreationCachedIndexInput(fileCache, streamReader, blobFetchRequest); + } else { + logger.trace("Transfer Manager - Already in cache"); + // already in the cache and ready to be used (open) + return cachedIndexInput; + } + }); }); // Cache entry was either retrieved from the cache or newly added, either @@ -88,37 +94,31 @@ public IndexInput fetchBlob(BlobFetchRequest blobFetchRequest) throws IOExceptio @SuppressWarnings("removal") private static FileCachedIndexInput createIndexInput(FileCache fileCache, StreamReader streamReader, BlobFetchRequest request) { - // We need to do a privileged action here in order to fetch from remote - // and write to the local file cache in case this is invoked as a side - // effect of a plugin (such as a scripted search) that doesn't have the - // necessary permissions. - return AccessController.doPrivileged((PrivilegedAction) () -> { - try { - if (Files.exists(request.getFilePath()) == false) { - logger.trace("Fetching from Remote in createIndexInput of Transfer Manager"); - try ( - OutputStream fileOutputStream = Files.newOutputStream(request.getFilePath()); - OutputStream localFileOutputStream = new BufferedOutputStream(fileOutputStream) - ) { - for (BlobFetchRequest.BlobPart blobPart : request.blobParts()) { - try ( - InputStream snapshotFileInputStream = streamReader.read( - blobPart.getBlobName(), - blobPart.getPosition(), - blobPart.getLength() - ); - ) { - snapshotFileInputStream.transferTo(localFileOutputStream); - } + try { + if (Files.exists(request.getFilePath()) == false) { + logger.trace("Fetching from Remote in createIndexInput of Transfer Manager"); + try ( + OutputStream fileOutputStream = Files.newOutputStream(request.getFilePath()); + OutputStream localFileOutputStream = new BufferedOutputStream(fileOutputStream) + ) { + for (BlobFetchRequest.BlobPart blobPart : request.blobParts()) { + try ( + InputStream snapshotFileInputStream = streamReader.read( + blobPart.getBlobName(), + blobPart.getPosition(), + blobPart.getLength() + ); + ) { + snapshotFileInputStream.transferTo(localFileOutputStream); } } } - final IndexInput luceneIndexInput = request.getDirectory().openInput(request.getFileName(), IOContext.READ); - return new FileCachedIndexInput(fileCache, request.getFilePath(), luceneIndexInput); - } catch (IOException e) { - throw new UncheckedIOException(e); } - }); + final IndexInput luceneIndexInput = request.getDirectory().openInput(request.getFileName(), IOContext.READ); + return new FileCachedIndexInput(fileCache, request.getFilePath(), luceneIndexInput); + } catch (IOException e) { + throw new UncheckedIOException(e); + } } /** From c82a282351b4e913d57b50d2fef94d4f046b155c Mon Sep 17 00:00:00 2001 From: Neetika Singhal Date: Mon, 22 Jul 2024 20:39:14 -0700 Subject: [PATCH 22/37] Add rest, transport layer changes for Hot to warm tiering - dedicated setup (#13980) Signed-off-by: Neetika Singhal --- CHANGELOG.md | 1 + .../org/opensearch/action/ActionModule.java | 9 + .../tiering/HotToWarmTieringAction.java | 28 ++ .../tiering/HotToWarmTieringResponse.java | 157 +++++++++ .../tiering/RestWarmTieringAction.java | 61 ++++ .../indices/tiering/TieringIndexRequest.java | 195 +++++++++++ .../tiering/TieringValidationResult.java | 83 +++++ .../TransportHotToWarmTieringAction.java | 110 ++++++ .../admin/indices/tiering/package-info.java | 36 ++ .../common/settings/IndexScopedSettings.java | 2 +- .../org/opensearch/index/IndexModule.java | 20 ++ .../tiering/TieringRequestValidator.java | 277 +++++++++++++++ .../indices/tiering/package-info.java | 36 ++ .../HotToWarmTieringResponseTests.java | 101 ++++++ .../tiering/TieringIndexRequestTests.java | 79 +++++ .../TransportHotToWarmTieringActionTests.java | 118 +++++++ .../tiering/TieringRequestValidatorTests.java | 318 ++++++++++++++++++ 17 files changed, 1630 insertions(+), 1 deletion(-) create mode 100644 server/src/main/java/org/opensearch/action/admin/indices/tiering/HotToWarmTieringAction.java create mode 100644 server/src/main/java/org/opensearch/action/admin/indices/tiering/HotToWarmTieringResponse.java create mode 100644 server/src/main/java/org/opensearch/action/admin/indices/tiering/RestWarmTieringAction.java create mode 100644 server/src/main/java/org/opensearch/action/admin/indices/tiering/TieringIndexRequest.java create mode 100644 server/src/main/java/org/opensearch/action/admin/indices/tiering/TieringValidationResult.java create mode 100644 server/src/main/java/org/opensearch/action/admin/indices/tiering/TransportHotToWarmTieringAction.java create mode 100644 server/src/main/java/org/opensearch/action/admin/indices/tiering/package-info.java create mode 100644 server/src/main/java/org/opensearch/indices/tiering/TieringRequestValidator.java create mode 100644 server/src/main/java/org/opensearch/indices/tiering/package-info.java create mode 100644 server/src/test/java/org/opensearch/action/admin/indices/tiering/HotToWarmTieringResponseTests.java create mode 100644 server/src/test/java/org/opensearch/action/admin/indices/tiering/TieringIndexRequestTests.java create mode 100644 server/src/test/java/org/opensearch/action/admin/indices/tiering/TransportHotToWarmTieringActionTests.java create mode 100644 server/src/test/java/org/opensearch/indices/tiering/TieringRequestValidatorTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index ec5b838a542c4..e5534577a67a6 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -27,6 +27,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Reduce logging in DEBUG for MasterService:run ([#14795](https://github.com/opensearch-project/OpenSearch/pull/14795)) - Enabling term version check on local state for all ClusterManager Read Transport Actions ([#14273](https://github.com/opensearch-project/OpenSearch/pull/14273)) - Add persian_stem filter (([#14847](https://github.com/opensearch-project/OpenSearch/pull/14847))) +- Add rest, transport layer changes for hot to warm tiering - dedicated setup (([#13980](https://github.com/opensearch-project/OpenSearch/pull/13980)) ### Dependencies - Bump `org.gradle.test-retry` from 1.5.8 to 1.5.9 ([#13442](https://github.com/opensearch-project/OpenSearch/pull/13442)) diff --git a/server/src/main/java/org/opensearch/action/ActionModule.java b/server/src/main/java/org/opensearch/action/ActionModule.java index 16c15f553951c..574b7029a6501 100644 --- a/server/src/main/java/org/opensearch/action/ActionModule.java +++ b/server/src/main/java/org/opensearch/action/ActionModule.java @@ -216,6 +216,9 @@ import org.opensearch.action.admin.indices.template.put.TransportPutComponentTemplateAction; import org.opensearch.action.admin.indices.template.put.TransportPutComposableIndexTemplateAction; import org.opensearch.action.admin.indices.template.put.TransportPutIndexTemplateAction; +import org.opensearch.action.admin.indices.tiering.HotToWarmTieringAction; +import org.opensearch.action.admin.indices.tiering.RestWarmTieringAction; +import org.opensearch.action.admin.indices.tiering.TransportHotToWarmTieringAction; import org.opensearch.action.admin.indices.upgrade.get.TransportUpgradeStatusAction; import org.opensearch.action.admin.indices.upgrade.get.UpgradeStatusAction; import org.opensearch.action.admin.indices.upgrade.post.TransportUpgradeAction; @@ -634,6 +637,9 @@ public void reg actions.register(CreateSnapshotAction.INSTANCE, TransportCreateSnapshotAction.class); actions.register(CloneSnapshotAction.INSTANCE, TransportCloneSnapshotAction.class); actions.register(RestoreSnapshotAction.INSTANCE, TransportRestoreSnapshotAction.class); + if (FeatureFlags.isEnabled(FeatureFlags.TIERED_REMOTE_INDEX)) { + actions.register(HotToWarmTieringAction.INSTANCE, TransportHotToWarmTieringAction.class); + } actions.register(SnapshotsStatusAction.INSTANCE, TransportSnapshotsStatusAction.class); actions.register(ClusterAddWeightedRoutingAction.INSTANCE, TransportAddWeightedRoutingAction.class); @@ -966,6 +972,9 @@ public void initRestHandlers(Supplier nodesInCluster) { registerHandler.accept(new RestNodeAttrsAction()); registerHandler.accept(new RestRepositoriesAction()); registerHandler.accept(new RestSnapshotAction()); + if (FeatureFlags.isEnabled(FeatureFlags.TIERED_REMOTE_INDEX)) { + registerHandler.accept(new RestWarmTieringAction()); + } registerHandler.accept(new RestTemplatesAction()); // Point in time API diff --git a/server/src/main/java/org/opensearch/action/admin/indices/tiering/HotToWarmTieringAction.java b/server/src/main/java/org/opensearch/action/admin/indices/tiering/HotToWarmTieringAction.java new file mode 100644 index 0000000000000..ae34a9a734221 --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/indices/tiering/HotToWarmTieringAction.java @@ -0,0 +1,28 @@ +/* + * 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.action.admin.indices.tiering; + +import org.opensearch.action.ActionType; +import org.opensearch.common.annotation.ExperimentalApi; + +/** + * Tiering action to move indices from hot to warm + * + * @opensearch.experimental + */ +@ExperimentalApi +public class HotToWarmTieringAction extends ActionType { + + public static final HotToWarmTieringAction INSTANCE = new HotToWarmTieringAction(); + public static final String NAME = "indices:admin/tier/hot_to_warm"; + + private HotToWarmTieringAction() { + super(NAME, HotToWarmTieringResponse::new); + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/indices/tiering/HotToWarmTieringResponse.java b/server/src/main/java/org/opensearch/action/admin/indices/tiering/HotToWarmTieringResponse.java new file mode 100644 index 0000000000000..275decf7a8ea5 --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/indices/tiering/HotToWarmTieringResponse.java @@ -0,0 +1,157 @@ +/* + * 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.action.admin.indices.tiering; + +import org.opensearch.action.support.master.AcknowledgedResponse; +import org.opensearch.common.annotation.ExperimentalApi; +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.MediaTypeRegistry; +import org.opensearch.core.xcontent.ToXContentFragment; +import org.opensearch.core.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; + +/** + * Response object for an {@link TieringIndexRequest} which is sent to client after the initial verification of the request + * by the backend service. The format of the response object will be as below: + * + * { + * "acknowledged": true/false, + * "failed_indices": [ + * { + * "index": "index1", + * "error": "Low disk threshold watermark breached" + * }, + * { + * "index": "index2", + * "error": "Index is not a remote store backed index" + * } + * ] + * } + * + * @opensearch.experimental + */ +@ExperimentalApi +public class HotToWarmTieringResponse extends AcknowledgedResponse { + + private final List failedIndices; + + public HotToWarmTieringResponse(boolean acknowledged) { + super(acknowledged); + this.failedIndices = Collections.emptyList(); + } + + public HotToWarmTieringResponse(boolean acknowledged, List indicesResults) { + super(acknowledged); + this.failedIndices = (indicesResults == null) + ? Collections.emptyList() + : indicesResults.stream().sorted(Comparator.comparing(IndexResult::getIndex)).collect(Collectors.toList()); + } + + public HotToWarmTieringResponse(StreamInput in) throws IOException { + super(in); + failedIndices = Collections.unmodifiableList(in.readList(IndexResult::new)); + } + + public List getFailedIndices() { + return this.failedIndices; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeList(this.failedIndices); + } + + @Override + protected void addCustomFields(XContentBuilder builder, Params params) throws IOException { + super.addCustomFields(builder, params); + builder.startArray("failed_indices"); + + for (IndexResult failedIndex : failedIndices) { + failedIndex.toXContent(builder, params); + } + builder.endArray(); + } + + @Override + public String toString() { + return Strings.toString(MediaTypeRegistry.JSON, this); + } + + /** + * Inner class to represent the result of a failed index for tiering. + * @opensearch.experimental + */ + @ExperimentalApi + public static class IndexResult implements Writeable, ToXContentFragment { + private final String index; + private final String failureReason; + + public IndexResult(String index, String failureReason) { + this.index = index; + this.failureReason = failureReason; + } + + IndexResult(StreamInput in) throws IOException { + this.index = in.readString(); + this.failureReason = in.readString(); + } + + public String getIndex() { + return index; + } + + public String getFailureReason() { + return failureReason; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(index); + out.writeString(failureReason); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field("index", index); + builder.field("error", failureReason); + return builder.endObject(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + IndexResult that = (IndexResult) o; + return Objects.equals(index, that.index) && Objects.equals(failureReason, that.failureReason); + } + + @Override + public int hashCode() { + int result = Objects.hashCode(index); + result = 31 * result + Objects.hashCode(failureReason); + return result; + } + + @Override + public String toString() { + return Strings.toString(MediaTypeRegistry.JSON, this); + } + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/indices/tiering/RestWarmTieringAction.java b/server/src/main/java/org/opensearch/action/admin/indices/tiering/RestWarmTieringAction.java new file mode 100644 index 0000000000000..6f2eceafa9e77 --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/indices/tiering/RestWarmTieringAction.java @@ -0,0 +1,61 @@ +/* + * 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.action.admin.indices.tiering; + +import org.opensearch.action.support.IndicesOptions; +import org.opensearch.client.node.NodeClient; +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.rest.BaseRestHandler; +import org.opensearch.rest.RestHandler; +import org.opensearch.rest.RestRequest; +import org.opensearch.rest.action.RestToXContentListener; + +import java.util.List; + +import static java.util.Collections.singletonList; +import static org.opensearch.core.common.Strings.splitStringByCommaToArray; +import static org.opensearch.rest.RestRequest.Method.POST; + +/** + * Rest Tiering API action to move indices to warm tier + * + * @opensearch.experimental + */ +@ExperimentalApi +public class RestWarmTieringAction extends BaseRestHandler { + + private static final String TARGET_TIER = "warm"; + + @Override + public List routes() { + return singletonList(new RestHandler.Route(POST, "/{index}/_tier/" + TARGET_TIER)); + } + + @Override + public String getName() { + return "warm_tiering_action"; + } + + @Override + protected BaseRestHandler.RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) { + final TieringIndexRequest tieringIndexRequest = new TieringIndexRequest( + TARGET_TIER, + splitStringByCommaToArray(request.param("index")) + ); + tieringIndexRequest.timeout(request.paramAsTime("timeout", tieringIndexRequest.timeout())); + tieringIndexRequest.clusterManagerNodeTimeout( + request.paramAsTime("cluster_manager_timeout", tieringIndexRequest.clusterManagerNodeTimeout()) + ); + tieringIndexRequest.indicesOptions(IndicesOptions.fromRequest(request, tieringIndexRequest.indicesOptions())); + tieringIndexRequest.waitForCompletion(request.paramAsBoolean("wait_for_completion", tieringIndexRequest.waitForCompletion())); + return channel -> client.admin() + .cluster() + .execute(HotToWarmTieringAction.INSTANCE, tieringIndexRequest, new RestToXContentListener<>(channel)); + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/indices/tiering/TieringIndexRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/tiering/TieringIndexRequest.java new file mode 100644 index 0000000000000..ed458a47ddb7d --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/indices/tiering/TieringIndexRequest.java @@ -0,0 +1,195 @@ +/* + * 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.action.admin.indices.tiering; + +import org.opensearch.action.ActionRequestValidationException; +import org.opensearch.action.IndicesRequest; +import org.opensearch.action.support.IndicesOptions; +import org.opensearch.action.support.master.AcknowledgedRequest; +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Locale; +import java.util.Objects; + +import static org.opensearch.action.ValidateActions.addValidationError; + +/** + * Represents the tiering request for indices to move to a different tier + * + * @opensearch.experimental + */ +@ExperimentalApi +public class TieringIndexRequest extends AcknowledgedRequest implements IndicesRequest.Replaceable { + + private String[] indices; + private final Tier targetTier; + private IndicesOptions indicesOptions; + private boolean waitForCompletion; + + public TieringIndexRequest(String targetTier, String... indices) { + this.targetTier = Tier.fromString(targetTier); + this.indices = indices; + this.indicesOptions = IndicesOptions.fromOptions(false, false, true, false); + this.waitForCompletion = false; + } + + public TieringIndexRequest(StreamInput in) throws IOException { + super(in); + indices = in.readStringArray(); + targetTier = Tier.fromString(in.readString()); + indicesOptions = IndicesOptions.readIndicesOptions(in); + waitForCompletion = in.readBoolean(); + } + + // pkg private for testing + TieringIndexRequest(Tier targetTier, IndicesOptions indicesOptions, boolean waitForCompletion, String... indices) { + this.indices = indices; + this.targetTier = targetTier; + this.indicesOptions = indicesOptions; + this.waitForCompletion = waitForCompletion; + } + + @Override + public ActionRequestValidationException validate() { + ActionRequestValidationException validationException = null; + if (indices == null) { + validationException = addValidationError("Mandatory parameter - indices is missing from the request", validationException); + } else { + for (String index : indices) { + if (index == null || index.length() == 0) { + validationException = addValidationError( + String.format(Locale.ROOT, "Specified index in the request [%s] is null or empty", index), + validationException + ); + } + } + } + if (!Tier.WARM.equals(targetTier)) { + validationException = addValidationError("The specified tier is not supported", validationException); + } + return validationException; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeStringArray(indices); + out.writeString(targetTier.value()); + indicesOptions.writeIndicesOptions(out); + out.writeBoolean(waitForCompletion); + } + + @Override + public String[] indices() { + return indices; + } + + @Override + public IndicesOptions indicesOptions() { + return indicesOptions; + } + + @Override + public boolean includeDataStreams() { + return true; + } + + @Override + public TieringIndexRequest indices(String... indices) { + this.indices = indices; + return this; + } + + public TieringIndexRequest indicesOptions(IndicesOptions indicesOptions) { + this.indicesOptions = indicesOptions; + return this; + } + + /** + * If this parameter is set to true the operation will wait for completion of tiering process before returning. + * + * @param waitForCompletion if true the operation will wait for completion + * @return this request + */ + public TieringIndexRequest waitForCompletion(boolean waitForCompletion) { + this.waitForCompletion = waitForCompletion; + return this; + } + + /** + * Returns wait for completion setting + * + * @return true if the operation will wait for completion + */ + public boolean waitForCompletion() { + return waitForCompletion; + } + + public Tier tier() { + return targetTier; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TieringIndexRequest that = (TieringIndexRequest) o; + return clusterManagerNodeTimeout.equals(that.clusterManagerNodeTimeout) + && timeout.equals(that.timeout) + && Objects.equals(indicesOptions, that.indicesOptions) + && Arrays.equals(indices, that.indices) + && targetTier.equals(that.targetTier) + && waitForCompletion == that.waitForCompletion; + } + + @Override + public int hashCode() { + return Objects.hash(clusterManagerNodeTimeout, timeout, indicesOptions, waitForCompletion, Arrays.hashCode(indices)); + } + + /** + * Represents the supported tiers for an index + * + * @opensearch.experimental + */ + @ExperimentalApi + public enum Tier { + HOT, + WARM; + + public static Tier fromString(String name) { + if (name == null) { + throw new IllegalArgumentException("Tiering type cannot be null"); + } + String upperCase = name.trim().toUpperCase(Locale.ROOT); + switch (upperCase) { + case "HOT": + return HOT; + case "WARM": + return WARM; + default: + throw new IllegalArgumentException( + "Tiering type [" + name + "] is not supported. Supported types are " + HOT + " and " + WARM + ); + } + } + + public String value() { + return name().toLowerCase(Locale.ROOT); + } + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/indices/tiering/TieringValidationResult.java b/server/src/main/java/org/opensearch/action/admin/indices/tiering/TieringValidationResult.java new file mode 100644 index 0000000000000..ccd60daf027ce --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/indices/tiering/TieringValidationResult.java @@ -0,0 +1,83 @@ +/* + * 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.action.admin.indices.tiering; + +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.core.index.Index; + +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Validation result for tiering + * + * @opensearch.experimental + */ + +@ExperimentalApi +public class TieringValidationResult { + private final Set acceptedIndices; + private final Map rejectedIndices; + + public TieringValidationResult(Set concreteIndices) { + // by default all the indices are added to the accepted set + this.acceptedIndices = ConcurrentHashMap.newKeySet(); + acceptedIndices.addAll(concreteIndices); + this.rejectedIndices = new HashMap<>(); + } + + public Set getAcceptedIndices() { + return acceptedIndices; + } + + public Map getRejectedIndices() { + return rejectedIndices; + } + + public void addToRejected(Index index, String reason) { + acceptedIndices.remove(index); + rejectedIndices.put(index, reason); + } + + public HotToWarmTieringResponse constructResponse() { + final List indicesResult = new LinkedList<>(); + for (Map.Entry rejectedIndex : rejectedIndices.entrySet()) { + indicesResult.add(new HotToWarmTieringResponse.IndexResult(rejectedIndex.getKey().getName(), rejectedIndex.getValue())); + } + return new HotToWarmTieringResponse(acceptedIndices.size() > 0, indicesResult); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + TieringValidationResult that = (TieringValidationResult) o; + + if (!Objects.equals(acceptedIndices, that.acceptedIndices)) return false; + return Objects.equals(rejectedIndices, that.rejectedIndices); + } + + @Override + public int hashCode() { + int result = acceptedIndices != null ? acceptedIndices.hashCode() : 0; + result = 31 * result + (rejectedIndices != null ? rejectedIndices.hashCode() : 0); + return result; + } + + @Override + public String toString() { + return "TieringValidationResult{" + "acceptedIndices=" + acceptedIndices + ", rejectedIndices=" + rejectedIndices + '}'; + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/indices/tiering/TransportHotToWarmTieringAction.java b/server/src/main/java/org/opensearch/action/admin/indices/tiering/TransportHotToWarmTieringAction.java new file mode 100644 index 0000000000000..8d1ab0bb37cdd --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/indices/tiering/TransportHotToWarmTieringAction.java @@ -0,0 +1,110 @@ +/* + * 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.action.admin.indices.tiering; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.clustermanager.TransportClusterManagerNodeAction; +import org.opensearch.cluster.ClusterInfoService; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.block.ClusterBlockException; +import org.opensearch.cluster.block.ClusterBlockLevel; +import org.opensearch.cluster.metadata.IndexNameExpressionResolver; +import org.opensearch.cluster.routing.allocation.DiskThresholdSettings; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.common.inject.Inject; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.index.Index; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportService; + +import java.io.IOException; +import java.util.Set; + +import static org.opensearch.indices.tiering.TieringRequestValidator.validateHotToWarm; + +/** + * Transport Tiering action to move indices from hot to warm + * + * @opensearch.experimental + */ +@ExperimentalApi +public class TransportHotToWarmTieringAction extends TransportClusterManagerNodeAction { + + private static final Logger logger = LogManager.getLogger(TransportHotToWarmTieringAction.class); + private final ClusterInfoService clusterInfoService; + private final DiskThresholdSettings diskThresholdSettings; + + @Inject + public TransportHotToWarmTieringAction( + TransportService transportService, + ClusterService clusterService, + ThreadPool threadPool, + ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver, + ClusterInfoService clusterInfoService, + Settings settings + ) { + super( + HotToWarmTieringAction.NAME, + transportService, + clusterService, + threadPool, + actionFilters, + TieringIndexRequest::new, + indexNameExpressionResolver + ); + this.clusterInfoService = clusterInfoService; + this.diskThresholdSettings = new DiskThresholdSettings(settings, clusterService.getClusterSettings()); + } + + @Override + protected String executor() { + return ThreadPool.Names.SAME; + } + + @Override + protected HotToWarmTieringResponse read(StreamInput in) throws IOException { + return new HotToWarmTieringResponse(in); + } + + @Override + protected ClusterBlockException checkBlock(TieringIndexRequest request, ClusterState state) { + return state.blocks() + .indicesBlockedException(ClusterBlockLevel.METADATA_WRITE, indexNameExpressionResolver.concreteIndexNames(state, request)); + } + + @Override + protected void clusterManagerOperation( + TieringIndexRequest request, + ClusterState state, + ActionListener listener + ) throws Exception { + Index[] concreteIndices = indexNameExpressionResolver.concreteIndices(state, request); + if (concreteIndices == null || concreteIndices.length == 0) { + listener.onResponse(new HotToWarmTieringResponse(true)); + return; + } + final TieringValidationResult tieringValidationResult = validateHotToWarm( + state, + Set.of(concreteIndices), + clusterInfoService.getClusterInfo(), + diskThresholdSettings + ); + + if (tieringValidationResult.getAcceptedIndices().isEmpty()) { + listener.onResponse(tieringValidationResult.constructResponse()); + return; + } + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/indices/tiering/package-info.java b/server/src/main/java/org/opensearch/action/admin/indices/tiering/package-info.java new file mode 100644 index 0000000000000..878e3575a3934 --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/indices/tiering/package-info.java @@ -0,0 +1,36 @@ +/* + * 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. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you 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. + */ + +/** + * Actions that OpenSearch can take to tier the indices + */ +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.action.admin.indices.tiering; diff --git a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java index ca2c4dab6102b..6e7d77d0c00d4 100644 --- a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java @@ -273,7 +273,7 @@ public final class IndexScopedSettings extends AbstractScopedSettings { */ public static final Map> FEATURE_FLAGGED_INDEX_SETTINGS = Map.of( FeatureFlags.TIERED_REMOTE_INDEX, - List.of(IndexModule.INDEX_STORE_LOCALITY_SETTING) + List.of(IndexModule.INDEX_STORE_LOCALITY_SETTING, IndexModule.INDEX_TIERING_STATE) ); public static final IndexScopedSettings DEFAULT_SCOPED_SETTINGS = new IndexScopedSettings(Settings.EMPTY, BUILT_IN_INDEX_SETTINGS); diff --git a/server/src/main/java/org/opensearch/index/IndexModule.java b/server/src/main/java/org/opensearch/index/IndexModule.java index 09b904394ee09..93ff1b78b1ac5 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.ExperimentalApi; import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.settings.Setting; @@ -174,6 +175,14 @@ public final class IndexModule { Property.NodeScope ); + public static final Setting INDEX_TIERING_STATE = new Setting<>( + "index.tiering.state", + TieringState.HOT.name(), + Function.identity(), + Property.IndexScope, + Property.PrivateIndex + ); + /** Which lucene file extensions to load with the mmap directory when using hybridfs store. This settings is ignored if {@link #INDEX_STORE_HYBRID_NIO_EXTENSIONS} is set. * This is an expert setting. * @see Lucene File Extensions. @@ -663,6 +672,17 @@ public static Type defaultStoreType(final boolean allowMmap) { } } + /** + * Represents the tiering state of the index. + */ + @ExperimentalApi + public enum TieringState { + HOT, + HOT_TO_WARM, + WARM, + WARM_TO_HOT; + } + public IndexService newIndexService( IndexService.IndexCreationContext indexCreationContext, NodeEnvironment environment, diff --git a/server/src/main/java/org/opensearch/indices/tiering/TieringRequestValidator.java b/server/src/main/java/org/opensearch/indices/tiering/TieringRequestValidator.java new file mode 100644 index 0000000000000..2de50f4d4295d --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/tiering/TieringRequestValidator.java @@ -0,0 +1,277 @@ +/* + * 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.tiering; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.action.admin.indices.tiering.TieringValidationResult; +import org.opensearch.cluster.ClusterInfo; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.DiskUsage; +import org.opensearch.cluster.health.ClusterHealthStatus; +import org.opensearch.cluster.health.ClusterIndexHealth; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.allocation.DiskThresholdSettings; +import org.opensearch.core.index.Index; +import org.opensearch.index.IndexModule; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.opensearch.index.IndexModule.INDEX_TIERING_STATE; + +/** + * Validator class to validate the tiering requests of the index + * @opensearch.experimental + */ +public class TieringRequestValidator { + + private static final Logger logger = LogManager.getLogger(TieringRequestValidator.class); + + /** + * Validates the tiering request for indices going from hot to warm tier + * + * @param currentState current cluster state + * @param concreteIndices set of indices to be validated + * @param clusterInfo the current nodes usage info for the cluster + * @param diskThresholdSettings the disk threshold settings of the cluster + * @return result of the validation + */ + public static TieringValidationResult validateHotToWarm( + final ClusterState currentState, + final Set concreteIndices, + final ClusterInfo clusterInfo, + final DiskThresholdSettings diskThresholdSettings + ) { + final String indexNames = concreteIndices.stream().map(Index::getName).collect(Collectors.joining(", ")); + validateSearchNodes(currentState, indexNames); + validateDiskThresholdWaterMarkNotBreached(currentState, clusterInfo, diskThresholdSettings, indexNames); + + final TieringValidationResult tieringValidationResult = new TieringValidationResult(concreteIndices); + + for (Index index : concreteIndices) { + if (!validateHotIndex(currentState, index)) { + tieringValidationResult.addToRejected(index, "index is not in the HOT tier"); + continue; + } + if (!validateRemoteStoreIndex(currentState, index)) { + tieringValidationResult.addToRejected(index, "index is not backed up by the remote store"); + continue; + } + if (!validateOpenIndex(currentState, index)) { + tieringValidationResult.addToRejected(index, "index is closed"); + continue; + } + if (!validateIndexHealth(currentState, index)) { + tieringValidationResult.addToRejected(index, "index is red"); + continue; + } + } + + validateEligibleNodesCapacity(clusterInfo, currentState, tieringValidationResult); + logger.info( + "Successfully accepted indices for tiering are [{}], rejected indices are [{}]", + tieringValidationResult.getAcceptedIndices(), + tieringValidationResult.getRejectedIndices() + ); + + return tieringValidationResult; + } + + /** + * Validates that there are eligible nodes with the search role in the current cluster state. + * (only for the dedicated case - to be removed later) + * + * @param currentState the current cluster state + * @param indexNames the names of the indices being validated + * @throws IllegalArgumentException if there are no eligible search nodes in the cluster + */ + static void validateSearchNodes(final ClusterState currentState, final String indexNames) { + if (getEligibleNodes(currentState).isEmpty()) { + final String errorMsg = "Rejecting tiering request for indices [" + + indexNames + + "] because there are no nodes found with the search role"; + logger.warn(errorMsg); + throw new IllegalArgumentException(errorMsg); + } + } + + /** + * Validates that the specified index has the remote store setting enabled. + * + * @param state the current cluster state + * @param index the index to be validated + * @return true if the remote store setting is enabled for the index, false otherwise + */ + static boolean validateRemoteStoreIndex(final ClusterState state, final Index index) { + return IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING.get(state.metadata().getIndexSafe(index).getSettings()); + } + + /** + * Validates that the specified index is in the "hot" tiering state. + * + * @param state the current cluster state + * @param index the index to be validated + * @return true if the index is in the "hot" tiering state, false otherwise + */ + static boolean validateHotIndex(final ClusterState state, final Index index) { + return IndexModule.TieringState.HOT.name().equals(INDEX_TIERING_STATE.get(state.metadata().getIndexSafe(index).getSettings())); + } + + /** + * Validates the health of the specified index in the current cluster state. + * + * @param currentState the current cluster state + * @param index the index to be validated + * @return true if the index health is not in the "red" state, false otherwise + */ + static boolean validateIndexHealth(final ClusterState currentState, final Index index) { + final IndexRoutingTable indexRoutingTable = currentState.routingTable().index(index); + final IndexMetadata indexMetadata = currentState.metadata().index(index); + final ClusterIndexHealth indexHealth = new ClusterIndexHealth(indexMetadata, indexRoutingTable); + return !ClusterHealthStatus.RED.equals(indexHealth.getStatus()); + } + + /** + * Validates that the specified index is in the open state in the current cluster state. + * + * @param currentState the current cluster state + * @param index the index to be validated + * @return true if the index is in the open state, false otherwise + */ + static boolean validateOpenIndex(final ClusterState currentState, final Index index) { + return currentState.metadata().index(index).getState() == IndexMetadata.State.OPEN; + } + + /** + * Validates that the disk threshold low watermark is not breached on all the eligible nodes in the cluster. + * + * @param currentState the current cluster state + * @param clusterInfo the current nodes usage info for the cluster + * @param diskThresholdSettings the disk threshold settings of the cluster + * @param indexNames the names of the indices being validated + * @throws IllegalArgumentException if the disk threshold low watermark is breached on all eligible nodes + */ + static void validateDiskThresholdWaterMarkNotBreached( + final ClusterState currentState, + final ClusterInfo clusterInfo, + final DiskThresholdSettings diskThresholdSettings, + final String indexNames + ) { + final Map usages = clusterInfo.getNodeLeastAvailableDiskUsages(); + if (usages == null) { + logger.trace("skipping monitor as no disk usage information is available"); + return; + } + final Set nodeIds = getEligibleNodes(currentState).stream().map(DiscoveryNode::getId).collect(Collectors.toSet()); + for (String node : nodeIds) { + final DiskUsage nodeUsage = usages.get(node); + if (nodeUsage != null && nodeUsage.getFreeBytes() > diskThresholdSettings.getFreeBytesThresholdLow().getBytes()) { + return; + } + } + throw new IllegalArgumentException( + "Disk threshold low watermark is breached on all the search nodes, rejecting tiering request for indices: " + indexNames + ); + } + + /** + * Validates the capacity of eligible nodes in the cluster to accommodate the specified indices + * and adds the rejected indices to tieringValidationResult + * + * @param clusterInfo the current nodes usage info for the cluster + * @param currentState the current cluster state + * @param tieringValidationResult contains the indices to validate + */ + static void validateEligibleNodesCapacity( + final ClusterInfo clusterInfo, + final ClusterState currentState, + final TieringValidationResult tieringValidationResult + ) { + + final Set eligibleNodeIds = getEligibleNodes(currentState).stream().map(DiscoveryNode::getId).collect(Collectors.toSet()); + long totalAvailableBytesInWarmTier = getTotalAvailableBytesInWarmTier( + clusterInfo.getNodeLeastAvailableDiskUsages(), + eligibleNodeIds + ); + + Map indexSizes = new HashMap<>(); + for (Index index : tieringValidationResult.getAcceptedIndices()) { + indexSizes.put(index, getIndexPrimaryStoreSize(currentState, clusterInfo, index.getName())); + } + + if (indexSizes.values().stream().mapToLong(Long::longValue).sum() < totalAvailableBytesInWarmTier) { + return; + } + HashMap sortedIndexSizes = indexSizes.entrySet() + .stream() + .sorted(Map.Entry.comparingByValue()) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue, (e1, e2) -> e1, HashMap::new)); + + long requestIndexBytes = 0L; + for (Index index : sortedIndexSizes.keySet()) { + requestIndexBytes += sortedIndexSizes.get(index); + if (requestIndexBytes >= totalAvailableBytesInWarmTier) { + tieringValidationResult.addToRejected(index, "insufficient node capacity"); + } + } + } + + /** + * Calculates the total size of the specified index in the cluster. + * Note: This function only accounts for the primary shard size. + * + * @param clusterState the current state of the cluster + * @param clusterInfo the current nodes usage info for the cluster + * @param index the name of the index for which the total size is to be calculated + * @return the total size of the specified index in the cluster + */ + static long getIndexPrimaryStoreSize(ClusterState clusterState, ClusterInfo clusterInfo, String index) { + long totalIndexSize = 0; + List shardRoutings = clusterState.routingTable().allShards(index); + for (ShardRouting shardRouting : shardRoutings) { + if (shardRouting.primary()) { + totalIndexSize += clusterInfo.getShardSize(shardRouting, 0); + } + } + return totalIndexSize; + } + + /** + * Calculates the total available bytes in the warm tier of the cluster. + * + * @param usages the current disk usage of the cluster + * @param nodeIds the set of warm nodes ids in the cluster + * @return the total available bytes in the warm tier + */ + static long getTotalAvailableBytesInWarmTier(final Map usages, final Set nodeIds) { + long totalAvailableBytes = 0; + for (String node : nodeIds) { + totalAvailableBytes += usages.get(node).getFreeBytes(); + } + return totalAvailableBytes; + } + + /** + * Retrieves the set of eligible(search) nodes from the current cluster state. + * + * @param currentState the current cluster state + * @return the set of eligible nodes + */ + static Set getEligibleNodes(final ClusterState currentState) { + final Map nodes = currentState.getNodes().getDataNodes(); + return nodes.values().stream().filter(DiscoveryNode::isSearchNode).collect(Collectors.toSet()); + } +} diff --git a/server/src/main/java/org/opensearch/indices/tiering/package-info.java b/server/src/main/java/org/opensearch/indices/tiering/package-info.java new file mode 100644 index 0000000000000..552f87382ea15 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/tiering/package-info.java @@ -0,0 +1,36 @@ +/* + * 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. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you 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. + */ + +/** + * Validator layer checks that OpenSearch can perform to tier the indices + */ +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +package org.opensearch.indices.tiering; diff --git a/server/src/test/java/org/opensearch/action/admin/indices/tiering/HotToWarmTieringResponseTests.java b/server/src/test/java/org/opensearch/action/admin/indices/tiering/HotToWarmTieringResponseTests.java new file mode 100644 index 0000000000000..85cabe0fa1491 --- /dev/null +++ b/server/src/test/java/org/opensearch/action/admin/indices/tiering/HotToWarmTieringResponseTests.java @@ -0,0 +1,101 @@ +/* + * 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.action.admin.indices.tiering; + +import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.io.stream.Writeable; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.test.AbstractWireSerializingTestCase; + +import java.util.LinkedList; +import java.util.List; + +public class HotToWarmTieringResponseTests extends AbstractWireSerializingTestCase { + + @Override + protected Writeable.Reader instanceReader() { + return HotToWarmTieringResponse::new; + } + + @Override + protected HotToWarmTieringResponse createTestInstance() { + return randomHotToWarmTieringResponse(); + } + + @Override + protected void assertEqualInstances(HotToWarmTieringResponse expected, HotToWarmTieringResponse actual) { + assertNotSame(expected, actual); + assertEquals(actual.isAcknowledged(), expected.isAcknowledged()); + + for (int i = 0; i < expected.getFailedIndices().size(); i++) { + HotToWarmTieringResponse.IndexResult expectedIndexResult = expected.getFailedIndices().get(i); + HotToWarmTieringResponse.IndexResult actualIndexResult = actual.getFailedIndices().get(i); + assertNotSame(expectedIndexResult, actualIndexResult); + assertEquals(actualIndexResult.getIndex(), expectedIndexResult.getIndex()); + assertEquals(actualIndexResult.getFailureReason(), expectedIndexResult.getFailureReason()); + } + } + + /** + * Verifies that ToXContent works with any random {@link HotToWarmTieringResponse} object + * @throws Exception - in case of error + */ + public void testToXContentWorksForRandomResponse() throws Exception { + HotToWarmTieringResponse testResponse = randomHotToWarmTieringResponse(); + XContentType xContentType = randomFrom(XContentType.values()); + try (XContentBuilder builder = XContentBuilder.builder(xContentType.xContent())) { + testResponse.toXContent(builder, ToXContent.EMPTY_PARAMS); + } + } + + /** + * Verify the XContent output of the response object + * @throws Exception - in case of error + */ + public void testToXContentOutput() throws Exception { + String[] indices = new String[] { "index2", "index1" }; + String[] errorReasons = new String[] { "reason2", "reason1" }; + List results = new LinkedList<>(); + for (int i = 0; i < indices.length; ++i) { + results.add(new HotToWarmTieringResponse.IndexResult(indices[i], errorReasons[i])); + } + HotToWarmTieringResponse testResponse = new HotToWarmTieringResponse(true, results); + + // generate a corresponding expected xcontent + XContentBuilder content = XContentFactory.jsonBuilder().startObject().field("acknowledged", true).startArray("failed_indices"); + // expected result should be in the sorted order + content.startObject().field("index", "index1").field("error", "reason1").endObject(); + content.startObject().field("index", "index2").field("error", "reason2").endObject(); + content.endArray().endObject(); + assertEquals(content.toString(), testResponse.toString()); + } + + /** + * @return - randomly generated object of type {@link HotToWarmTieringResponse.IndexResult} + */ + private HotToWarmTieringResponse.IndexResult randomIndexResult() { + String indexName = randomAlphaOfLengthBetween(1, 50); + String failureReason = randomAlphaOfLengthBetween(1, 200); + return new HotToWarmTieringResponse.IndexResult(indexName, failureReason); + } + + /** + * @return - randomly generated object of type {@link HotToWarmTieringResponse} + */ + private HotToWarmTieringResponse randomHotToWarmTieringResponse() { + int numIndexResult = randomIntBetween(0, 10); + List indexResults = new LinkedList<>(); + for (int i = 0; i < numIndexResult; ++i) { + indexResults.add(randomIndexResult()); + } + return new HotToWarmTieringResponse(randomBoolean(), indexResults); + } +} diff --git a/server/src/test/java/org/opensearch/action/admin/indices/tiering/TieringIndexRequestTests.java b/server/src/test/java/org/opensearch/action/admin/indices/tiering/TieringIndexRequestTests.java new file mode 100644 index 0000000000000..e33d10268a617 --- /dev/null +++ b/server/src/test/java/org/opensearch/action/admin/indices/tiering/TieringIndexRequestTests.java @@ -0,0 +1,79 @@ +/* + * 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.action.admin.indices.tiering; + +import org.opensearch.action.ActionRequestValidationException; +import org.opensearch.action.support.IndicesOptions; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; + +import static org.hamcrest.CoreMatchers.equalTo; + +public class TieringIndexRequestTests extends OpenSearchTestCase { + + public void testTieringRequestWithListOfIndices() { + TieringIndexRequest request = new TieringIndexRequest( + TieringIndexRequest.Tier.WARM, + IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), randomBoolean(), randomBoolean()), + false, + "foo", + "bar", + "baz" + ); + ActionRequestValidationException validationException = request.validate(); + assertNull(validationException); + } + + public void testTieringRequestWithIndexPattern() { + TieringIndexRequest request = new TieringIndexRequest(TieringIndexRequest.Tier.WARM.name(), "foo-*"); + ActionRequestValidationException validationException = request.validate(); + assertNull(validationException); + } + + public void testTieringRequestWithNullOrEmptyIndices() { + TieringIndexRequest request = new TieringIndexRequest(TieringIndexRequest.Tier.WARM.name(), null, ""); + ActionRequestValidationException validationException = request.validate(); + assertNotNull(validationException); + } + + public void testTieringRequestWithNotSupportedTier() { + TieringIndexRequest request = new TieringIndexRequest(TieringIndexRequest.Tier.HOT.name(), "test"); + ActionRequestValidationException validationException = request.validate(); + assertNotNull(validationException); + } + + public void testTieringTypeFromString() { + expectThrows(IllegalArgumentException.class, () -> TieringIndexRequest.Tier.fromString("tier")); + expectThrows(IllegalArgumentException.class, () -> TieringIndexRequest.Tier.fromString(null)); + } + + public void testSerDeOfTieringRequest() throws IOException { + TieringIndexRequest request = new TieringIndexRequest(TieringIndexRequest.Tier.WARM.name(), "test"); + try (BytesStreamOutput out = new BytesStreamOutput()) { + request.writeTo(out); + try (StreamInput in = out.bytes().streamInput()) { + final TieringIndexRequest deserializedRequest = new TieringIndexRequest(in); + assertEquals(request, deserializedRequest); + } + } + } + + public void testTieringRequestEquals() { + final TieringIndexRequest original = new TieringIndexRequest(TieringIndexRequest.Tier.WARM.name(), "test"); + original.indicesOptions(IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), randomBoolean(), randomBoolean())); + final TieringIndexRequest expected = new TieringIndexRequest(TieringIndexRequest.Tier.WARM.name(), original.indices()); + expected.indicesOptions(original.indicesOptions()); + assertThat(expected, equalTo(original)); + assertThat(expected.indices(), equalTo(original.indices())); + assertThat(expected.indicesOptions(), equalTo(original.indicesOptions())); + } +} diff --git a/server/src/test/java/org/opensearch/action/admin/indices/tiering/TransportHotToWarmTieringActionTests.java b/server/src/test/java/org/opensearch/action/admin/indices/tiering/TransportHotToWarmTieringActionTests.java new file mode 100644 index 0000000000000..10273366af804 --- /dev/null +++ b/server/src/test/java/org/opensearch/action/admin/indices/tiering/TransportHotToWarmTieringActionTests.java @@ -0,0 +1,118 @@ +/* + * 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.action.admin.indices.tiering; + +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; + +import org.opensearch.action.support.IndicesOptions; +import org.opensearch.cluster.ClusterInfoService; +import org.opensearch.cluster.MockInternalClusterInfoService; +import org.opensearch.cluster.block.ClusterBlockException; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.FeatureFlags; +import org.opensearch.core.common.unit.ByteSizeUnit; +import org.opensearch.core.common.unit.ByteSizeValue; +import org.opensearch.index.IndexNotFoundException; +import org.opensearch.index.store.remote.file.CleanerDaemonThreadLeakFilter; +import org.opensearch.monitor.fs.FsInfo; +import org.opensearch.plugins.Plugin; +import org.opensearch.test.OpenSearchIntegTestCase; +import org.junit.After; +import org.junit.Before; + +import java.util.Collection; +import java.util.Collections; + +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_READ_ONLY_ALLOW_DELETE; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; + +@ThreadLeakFilters(filters = CleanerDaemonThreadLeakFilter.class) +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0, supportsDedicatedMasters = false) +public class TransportHotToWarmTieringActionTests extends OpenSearchIntegTestCase { + protected static final String TEST_IDX_1 = "test-idx-1"; + protected static final String TEST_IDX_2 = "idx-2"; + protected static final String TARGET_TIER = "warm"; + private String[] indices; + + @Override + protected Settings featureFlagSettings() { + Settings.Builder featureSettings = Settings.builder(); + featureSettings.put(FeatureFlags.TIERED_REMOTE_INDEX, true); + return featureSettings.build(); + } + + @Override + protected Collection> nodePlugins() { + return Collections.singletonList(MockInternalClusterInfoService.TestPlugin.class); + } + + @Before + public void setup() { + internalCluster().startClusterManagerOnlyNode(); + internalCluster().ensureAtLeastNumSearchAndDataNodes(1); + long bytes = new ByteSizeValue(1000, ByteSizeUnit.KB).getBytes(); + final MockInternalClusterInfoService clusterInfoService = getMockInternalClusterInfoService(); + clusterInfoService.setDiskUsageFunctionAndRefresh((discoveryNode, fsInfoPath) -> setDiskUsage(fsInfoPath, bytes, bytes - 1)); + + final int numReplicasIndex = 0; + final Settings settings = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, numReplicasIndex) + .build(); + + indices = new String[] { TEST_IDX_1, TEST_IDX_2 }; + for (String index : indices) { + assertAcked(client().admin().indices().prepareCreate(index).setSettings(settings).get()); + ensureGreen(index); + } + } + + @After + public void cleanup() { + client().admin().indices().prepareDelete(indices).get(); + } + + MockInternalClusterInfoService getMockInternalClusterInfoService() { + return (MockInternalClusterInfoService) internalCluster().getCurrentClusterManagerNodeInstance(ClusterInfoService.class); + } + + static FsInfo.Path setDiskUsage(FsInfo.Path original, long totalBytes, long freeBytes) { + return new FsInfo.Path(original.getPath(), original.getMount(), totalBytes, freeBytes, freeBytes); + } + + public void testIndexLevelBlocks() { + enableIndexBlock(TEST_IDX_1, SETTING_READ_ONLY_ALLOW_DELETE); + TieringIndexRequest request = new TieringIndexRequest(TARGET_TIER, TEST_IDX_1); + expectThrows(ClusterBlockException.class, () -> client().execute(HotToWarmTieringAction.INSTANCE, request).actionGet()); + } + + public void testIndexNotFound() { + TieringIndexRequest request = new TieringIndexRequest(TARGET_TIER, "foo"); + expectThrows(IndexNotFoundException.class, () -> client().execute(HotToWarmTieringAction.INSTANCE, request).actionGet()); + } + + public void testNoConcreteIndices() { + TieringIndexRequest request = new TieringIndexRequest(TARGET_TIER, "foo"); + request.indicesOptions(IndicesOptions.fromOptions(true, true, true, false)); + HotToWarmTieringResponse response = client().admin().indices().execute(HotToWarmTieringAction.INSTANCE, request).actionGet(); + assertTrue(response.isAcknowledged()); + assertTrue(response.getFailedIndices().isEmpty()); + } + + public void testNoAcceptedIndices() { + TieringIndexRequest request = new TieringIndexRequest(TARGET_TIER, "test-idx-*", "idx-*"); + HotToWarmTieringResponse response = client().admin().indices().execute(HotToWarmTieringAction.INSTANCE, request).actionGet(); + assertFalse(response.isAcknowledged()); + assertEquals(2, response.getFailedIndices().size()); + for (HotToWarmTieringResponse.IndexResult result : response.getFailedIndices()) { + assertEquals("index is not backed up by the remote store", result.getFailureReason()); + } + } +} diff --git a/server/src/test/java/org/opensearch/indices/tiering/TieringRequestValidatorTests.java b/server/src/test/java/org/opensearch/indices/tiering/TieringRequestValidatorTests.java new file mode 100644 index 0000000000000..6b6f74353812b --- /dev/null +++ b/server/src/test/java/org/opensearch/indices/tiering/TieringRequestValidatorTests.java @@ -0,0 +1,318 @@ +/* + * 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.tiering; + +import org.opensearch.Version; +import org.opensearch.action.admin.indices.tiering.TieringValidationResult; +import org.opensearch.cluster.ClusterInfo; +import org.opensearch.cluster.ClusterName; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.DiskUsage; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodeRole; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.routing.allocation.DiskThresholdSettings; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.index.Index; +import org.opensearch.index.IndexModule; +import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.UUID; + +import static org.opensearch.cluster.routing.allocation.DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_DISK_FLOOD_STAGE_WATERMARK_SETTING; +import static org.opensearch.cluster.routing.allocation.DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING; +import static org.opensearch.cluster.routing.allocation.DiskThresholdSettings.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING; +import static org.opensearch.indices.tiering.TieringRequestValidator.getEligibleNodes; +import static org.opensearch.indices.tiering.TieringRequestValidator.getIndexPrimaryStoreSize; +import static org.opensearch.indices.tiering.TieringRequestValidator.getTotalAvailableBytesInWarmTier; +import static org.opensearch.indices.tiering.TieringRequestValidator.validateDiskThresholdWaterMarkNotBreached; +import static org.opensearch.indices.tiering.TieringRequestValidator.validateEligibleNodesCapacity; +import static org.opensearch.indices.tiering.TieringRequestValidator.validateHotIndex; +import static org.opensearch.indices.tiering.TieringRequestValidator.validateIndexHealth; +import static org.opensearch.indices.tiering.TieringRequestValidator.validateOpenIndex; +import static org.opensearch.indices.tiering.TieringRequestValidator.validateRemoteStoreIndex; +import static org.opensearch.indices.tiering.TieringRequestValidator.validateSearchNodes; + +public class TieringRequestValidatorTests extends OpenSearchTestCase { + + public void testValidateSearchNodes() { + ClusterState clusterStateWithSearchNodes = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .nodes(createNodes(2, 0, 0)) + .build(); + + // throws no errors + validateSearchNodes(clusterStateWithSearchNodes, "test_index"); + } + + public void testWithNoSearchNodesInCluster() { + ClusterState clusterStateWithNoSearchNodes = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .nodes(createNodes(0, 1, 1)) + .build(); + // throws error + IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> validateSearchNodes(clusterStateWithNoSearchNodes, "test") + ); + } + + public void testValidRemoteStoreIndex() { + String indexUuid = UUID.randomUUID().toString(); + String indexName = "test_index"; + + ClusterState clusterState1 = buildClusterState( + indexName, + indexUuid, + Settings.builder() + .put(IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING.getKey(), true) + .put(IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.getKey(), ReplicationType.SEGMENT) + .build() + ); + + assertTrue(validateRemoteStoreIndex(clusterState1, new Index(indexName, indexUuid))); + } + + public void testDocRepIndex() { + String indexUuid = UUID.randomUUID().toString(); + String indexName = "test_index"; + assertFalse(validateRemoteStoreIndex(buildClusterState(indexName, indexUuid, Settings.EMPTY), new Index(indexName, indexUuid))); + } + + public void testValidHotIndex() { + String indexUuid = UUID.randomUUID().toString(); + String indexName = "test_index"; + assertTrue(validateHotIndex(buildClusterState(indexName, indexUuid, Settings.EMPTY), new Index(indexName, indexUuid))); + } + + public void testIndexWithOngoingOrCompletedTiering() { + String indexUuid = UUID.randomUUID().toString(); + String indexName = "test_index"; + + IndexModule.TieringState tieringState = randomBoolean() ? IndexModule.TieringState.HOT_TO_WARM : IndexModule.TieringState.WARM; + + ClusterState clusterState = buildClusterState( + indexName, + indexUuid, + Settings.builder().put(IndexModule.INDEX_TIERING_STATE.getKey(), tieringState).build() + ); + assertFalse(validateHotIndex(clusterState, new Index(indexName, indexUuid))); + } + + public void testValidateIndexHealth() { + String indexUuid = UUID.randomUUID().toString(); + String indexName = "test_index"; + ClusterState clusterState = buildClusterState(indexName, indexUuid, Settings.EMPTY); + assertTrue(validateIndexHealth(clusterState, new Index(indexName, indexUuid))); + } + + public void testValidOpenIndex() { + String indexUuid = UUID.randomUUID().toString(); + String indexName = "test_index"; + assertTrue(validateOpenIndex(buildClusterState(indexName, indexUuid, Settings.EMPTY), new Index(indexName, indexUuid))); + } + + public void testCloseIndex() { + String indexUuid = UUID.randomUUID().toString(); + String indexName = "test_index"; + assertFalse( + validateOpenIndex( + buildClusterState(indexName, indexUuid, Settings.EMPTY, IndexMetadata.State.CLOSE), + new Index(indexName, indexUuid) + ) + ); + } + + public void testValidateDiskThresholdWaterMarkNotBreached() { + int noOfNodes = 2; + ClusterState clusterState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .nodes(createNodes(noOfNodes, 0, 0)) + .build(); + + ClusterInfo clusterInfo = clusterInfo(noOfNodes, 100, 20); + DiskThresholdSettings diskThresholdSettings = diskThresholdSettings("10b", "10b", "5b"); + // throws no error + validateDiskThresholdWaterMarkNotBreached(clusterState, clusterInfo, diskThresholdSettings, "test"); + } + + public void testValidateDiskThresholdWaterMarkNotBreachedThrowsError() { + int noOfNodes = 2; + ClusterState clusterState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .nodes(createNodes(noOfNodes, 0, 0)) + .build(); + ClusterInfo clusterInfo = clusterInfo(noOfNodes, 100, 5); + DiskThresholdSettings diskThresholdSettings = diskThresholdSettings("10b", "10b", "5b"); + // throws error + expectThrows( + IllegalArgumentException.class, + () -> validateDiskThresholdWaterMarkNotBreached(clusterState, clusterInfo, diskThresholdSettings, "test") + ); + } + + public void testGetTotalIndexSize() { + String indexUuid = UUID.randomUUID().toString(); + String indexName = "test_index"; + ClusterState clusterState = ClusterState.builder(buildClusterState(indexName, indexUuid, Settings.EMPTY)) + .nodes(createNodes(1, 0, 0)) + .build(); + Map diskUsages = diskUsages(1, 100, 50); + final Map shardSizes = new HashMap<>(); + shardSizes.put("[test_index][0][p]", 10L); // 10 bytes + ClusterInfo clusterInfo = new ClusterInfo(diskUsages, null, shardSizes, null, Map.of(), Map.of()); + assertEquals(10, getIndexPrimaryStoreSize(clusterState, clusterInfo, indexName)); + } + + public void testValidateEligibleNodesCapacityWithAllAccepted() { + String indexUuid = UUID.randomUUID().toString(); + String indexName = "test_index"; + Set indices = Set.of(new Index(indexName, indexUuid)); + ClusterState clusterState = ClusterState.builder(buildClusterState(indexName, indexUuid, Settings.EMPTY)) + .nodes(createNodes(1, 0, 0)) + .build(); + Map diskUsages = diskUsages(1, 100, 50); + final Map shardSizes = new HashMap<>(); + shardSizes.put("[test_index][0][p]", 10L); // 10 bytes + ClusterInfo clusterInfo = new ClusterInfo(diskUsages, null, shardSizes, null, Map.of(), Map.of()); + TieringValidationResult tieringValidationResult = new TieringValidationResult(indices); + validateEligibleNodesCapacity(clusterInfo, clusterState, tieringValidationResult); + assertEquals(indices, tieringValidationResult.getAcceptedIndices()); + assertTrue(tieringValidationResult.getRejectedIndices().isEmpty()); + } + + public void testValidateEligibleNodesCapacityWithAllRejected() { + String indexUuid = UUID.randomUUID().toString(); + String indexName = "test_index"; + Set indices = Set.of(new Index(indexName, indexUuid)); + ClusterState clusterState = ClusterState.builder(buildClusterState(indexName, indexUuid, Settings.EMPTY)) + .nodes(createNodes(1, 0, 0)) + .build(); + Map diskUsages = diskUsages(1, 100, 10); + final Map shardSizes = new HashMap<>(); + shardSizes.put("[test_index][0][p]", 20L); // 20 bytes + ClusterInfo clusterInfo = new ClusterInfo(diskUsages, null, shardSizes, null, Map.of(), Map.of()); + TieringValidationResult tieringValidationResult = new TieringValidationResult(indices); + validateEligibleNodesCapacity(clusterInfo, clusterState, tieringValidationResult); + assertEquals(indices.size(), tieringValidationResult.getRejectedIndices().size()); + assertEquals(indices, tieringValidationResult.getRejectedIndices().keySet()); + assertTrue(tieringValidationResult.getAcceptedIndices().isEmpty()); + } + + public void testGetTotalAvailableBytesInWarmTier() { + Map diskUsages = diskUsages(2, 500, 100); + assertEquals(200, getTotalAvailableBytesInWarmTier(diskUsages, Set.of("node-s0", "node-s1"))); + } + + public void testEligibleNodes() { + ClusterState clusterState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .nodes(createNodes(2, 0, 0)) + .build(); + + assertEquals(2, getEligibleNodes(clusterState).size()); + + clusterState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .nodes(createNodes(0, 1, 1)) + .build(); + assertEquals(0, getEligibleNodes(clusterState).size()); + } + + private static ClusterState buildClusterState(String indexName, String indexUuid, Settings settings) { + return buildClusterState(indexName, indexUuid, settings, IndexMetadata.State.OPEN); + } + + private static ClusterState buildClusterState(String indexName, String indexUuid, Settings settings, IndexMetadata.State state) { + Settings combinedSettings = Settings.builder().put(settings).put(createDefaultIndexSettings(indexUuid)).build(); + + Metadata metadata = Metadata.builder().put(IndexMetadata.builder(indexName).settings(combinedSettings).state(state)).build(); + + RoutingTable routingTable = RoutingTable.builder().addAsNew(metadata.index(indexName)).build(); + + return ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(routingTable) + .build(); + } + + private static Settings createDefaultIndexSettings(String indexUuid) { + return Settings.builder() + .put("index.version.created", Version.CURRENT) + .put(IndexMetadata.SETTING_INDEX_UUID, indexUuid) + .put(IndexMetadata.INDEX_NUMBER_OF_SHARDS_SETTING.getKey(), 2) + .put(IndexMetadata.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), 1) + .build(); + } + + private DiscoveryNodes createNodes(int numOfSearchNodes, int numOfDataNodes, int numOfIngestNodes) { + DiscoveryNodes.Builder discoveryNodesBuilder = DiscoveryNodes.builder(); + for (int i = 0; i < numOfSearchNodes; i++) { + discoveryNodesBuilder.add( + new DiscoveryNode( + "node-s" + i, + buildNewFakeTransportAddress(), + Collections.emptyMap(), + Collections.singleton(DiscoveryNodeRole.SEARCH_ROLE), + Version.CURRENT + ) + ); + } + for (int i = 0; i < numOfDataNodes; i++) { + discoveryNodesBuilder.add( + new DiscoveryNode( + "node-d" + i, + buildNewFakeTransportAddress(), + Collections.emptyMap(), + Collections.singleton(DiscoveryNodeRole.DATA_ROLE), + Version.CURRENT + ) + ); + } + for (int i = 0; i < numOfIngestNodes; i++) { + discoveryNodesBuilder.add( + new DiscoveryNode( + "node-i" + i, + buildNewFakeTransportAddress(), + Collections.emptyMap(), + Collections.singleton(DiscoveryNodeRole.INGEST_ROLE), + Version.CURRENT + ) + ); + } + return discoveryNodesBuilder.build(); + } + + private static DiskThresholdSettings diskThresholdSettings(String low, String high, String flood) { + return new DiskThresholdSettings( + Settings.builder() + .put(CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK_SETTING.getKey(), low) + .put(CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK_SETTING.getKey(), high) + .put(CLUSTER_ROUTING_ALLOCATION_DISK_FLOOD_STAGE_WATERMARK_SETTING.getKey(), flood) + .build(), + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) + ); + } + + private static ClusterInfo clusterInfo(int noOfNodes, long totalBytes, long freeBytes) { + final Map diskUsages = diskUsages(noOfNodes, totalBytes, freeBytes); + return new ClusterInfo(diskUsages, null, null, null, Map.of(), Map.of()); + } + + private static Map diskUsages(int noOfSearchNodes, long totalBytes, long freeBytes) { + final Map diskUsages = new HashMap<>(); + for (int i = 0; i < noOfSearchNodes; i++) { + diskUsages.put("node-s" + i, new DiskUsage("node-s" + i, "node-s" + i, "/foo/bar", totalBytes, freeBytes)); + } + return diskUsages; + } +} From 8ff3bcc4632287d8a784a1cba662957d6f921851 Mon Sep 17 00:00:00 2001 From: Siddhant Deshmukh Date: Mon, 22 Jul 2024 21:45:40 -0700 Subject: [PATCH 23/37] Create listener to refresh search thread resource usage (#14832) * [bug fix] fix incorrect coordinator node search resource usages Signed-off-by: Chenyang Ji * fix bug on serialization when passing task resource usage to coordinator Signed-off-by: Chenyang Ji * add more unit tests Signed-off-by: Chenyang Ji * remove query insights plugin related code Signed-off-by: Chenyang Ji * create per request listener to refresh task resource usage Signed-off-by: Chenyang Ji * Make new listener API public Signed-off-by: Siddhant Deshmukh * Add changelog Signed-off-by: Siddhant Deshmukh * Remove wrong files added Signed-off-by: Siddhant Deshmukh * Address review comments Signed-off-by: Siddhant Deshmukh * Build fix Signed-off-by: Siddhant Deshmukh * Make singleton Signed-off-by: Siddhant Deshmukh * Address review comments Signed-off-by: Siddhant Deshmukh * Make sure listener runs before plugin listeners Signed-off-by: Siddhant Deshmukh * Spotless Signed-off-by: Siddhant Deshmukh * Minor fix Signed-off-by: Siddhant Deshmukh --------- Signed-off-by: Chenyang Ji Signed-off-by: Siddhant Deshmukh Signed-off-by: Jay Deng Co-authored-by: Chenyang Ji Co-authored-by: Jay Deng --- CHANGELOG.md | 1 + .../SearchTaskRequestOperationsListener.java | 30 +++++++++++++++++++ .../main/java/org/opensearch/node/Node.java | 18 ++++++----- 3 files changed, 42 insertions(+), 7 deletions(-) create mode 100644 server/src/main/java/org/opensearch/action/search/SearchTaskRequestOperationsListener.java diff --git a/CHANGELOG.md b/CHANGELOG.md index e5534577a67a6..29c78ea7e3e4f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -27,6 +27,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Reduce logging in DEBUG for MasterService:run ([#14795](https://github.com/opensearch-project/OpenSearch/pull/14795)) - Enabling term version check on local state for all ClusterManager Read Transport Actions ([#14273](https://github.com/opensearch-project/OpenSearch/pull/14273)) - Add persian_stem filter (([#14847](https://github.com/opensearch-project/OpenSearch/pull/14847))) +- Create listener to refresh search thread resource usage ([#14832](https://github.com/opensearch-project/OpenSearch/pull/14832)) - Add rest, transport layer changes for hot to warm tiering - dedicated setup (([#13980](https://github.com/opensearch-project/OpenSearch/pull/13980)) ### Dependencies diff --git a/server/src/main/java/org/opensearch/action/search/SearchTaskRequestOperationsListener.java b/server/src/main/java/org/opensearch/action/search/SearchTaskRequestOperationsListener.java new file mode 100644 index 0000000000000..4434d71793b23 --- /dev/null +++ b/server/src/main/java/org/opensearch/action/search/SearchTaskRequestOperationsListener.java @@ -0,0 +1,30 @@ +/* + * 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.action.search; + +import org.opensearch.tasks.TaskResourceTrackingService; + +/** + * SearchTaskRequestOperationsListener subscriber for operations on search tasks resource usages. + * Listener ensures to refreshResourceStats on request end capturing the search task resource usage + * upon request completion. + * + */ +public final class SearchTaskRequestOperationsListener extends SearchRequestOperationsListener { + private final TaskResourceTrackingService taskResourceTrackingService; + + public SearchTaskRequestOperationsListener(TaskResourceTrackingService taskResourceTrackingService) { + this.taskResourceTrackingService = taskResourceTrackingService; + } + + @Override + public void onRequestEnd(SearchPhaseContext context, SearchRequestContext searchRequestContext) { + taskResourceTrackingService.refreshResourceStats(context.getTask()); + } +} diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index d91b2a45a48c6..448cb3627651c 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -52,6 +52,7 @@ import org.opensearch.action.search.SearchRequestOperationsListener; import org.opensearch.action.search.SearchRequestSlowLog; import org.opensearch.action.search.SearchRequestStats; +import org.opensearch.action.search.SearchTaskRequestOperationsListener; import org.opensearch.action.search.SearchTransportService; import org.opensearch.action.support.TransportAction; import org.opensearch.action.update.UpdateHelper; @@ -855,8 +856,17 @@ protected Node( threadPool ); + final TaskResourceTrackingService taskResourceTrackingService = new TaskResourceTrackingService( + settings, + clusterService.getClusterSettings(), + threadPool + ); + final SearchRequestStats searchRequestStats = new SearchRequestStats(clusterService.getClusterSettings()); final SearchRequestSlowLog searchRequestSlowLog = new SearchRequestSlowLog(clusterService); + final SearchTaskRequestOperationsListener searchTaskRequestOperationsListener = new SearchTaskRequestOperationsListener( + taskResourceTrackingService + ); remoteStoreStatsTrackerFactory = new RemoteStoreStatsTrackerFactory(clusterService, settings); CacheModule cacheModule = new CacheModule(pluginsService.filterPlugins(CachePlugin.class), settings); @@ -988,7 +998,7 @@ protected Node( final SearchRequestOperationsCompositeListenerFactory searchRequestOperationsCompositeListenerFactory = new SearchRequestOperationsCompositeListenerFactory( Stream.concat( - Stream.of(searchRequestStats, searchRequestSlowLog), + Stream.of(searchRequestStats, searchRequestSlowLog, searchTaskRequestOperationsListener), pluginComponents.stream() .filter(p -> p instanceof SearchRequestOperationsListener) .map(p -> (SearchRequestOperationsListener) p) @@ -1117,12 +1127,6 @@ protected Node( // development. Then we can deprecate Getter and Setter for IndexingPressureService in ClusterService (#478). clusterService.setIndexingPressureService(indexingPressureService); - final TaskResourceTrackingService taskResourceTrackingService = new TaskResourceTrackingService( - settings, - clusterService.getClusterSettings(), - threadPool - ); - final SearchBackpressureSettings searchBackpressureSettings = new SearchBackpressureSettings( settings, clusterService.getClusterSettings() From 130500218a794f15df522c3ba5a31acbc77209e4 Mon Sep 17 00:00:00 2001 From: rishavz_sagar Date: Tue, 23 Jul 2024 11:08:10 +0530 Subject: [PATCH 24/37] Caching avg total bytes and avg free bytes inside ClusterInfo (#14851) Signed-off-by: RS146BIJAY --- .../org/opensearch/cluster/ClusterInfo.java | 37 +++++++++++++++ .../decider/DiskThresholdDecider.java | 45 +++++++++---------- .../decider/DiskThresholdDeciderTests.java | 13 ------ 3 files changed, 57 insertions(+), 38 deletions(-) diff --git a/server/src/main/java/org/opensearch/cluster/ClusterInfo.java b/server/src/main/java/org/opensearch/cluster/ClusterInfo.java index 4c38d6fd99f5d..7216c447acc3e 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterInfo.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterInfo.java @@ -33,6 +33,7 @@ package org.opensearch.cluster; import org.opensearch.Version; +import org.opensearch.cluster.routing.RoutingNode; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; @@ -68,6 +69,8 @@ public class ClusterInfo implements ToXContentFragment, Writeable { final Map routingToDataPath; final Map reservedSpace; final Map nodeFileCacheStats; + private long avgTotalBytes; + private long avgFreeByte; protected ClusterInfo() { this(Map.of(), Map.of(), Map.of(), Map.of(), Map.of(), Map.of()); @@ -97,6 +100,7 @@ public ClusterInfo( this.routingToDataPath = routingToDataPath; this.reservedSpace = reservedSpace; this.nodeFileCacheStats = nodeFileCacheStats; + calculateAvgFreeAndTotalBytes(mostAvailableSpaceUsage); } public ClusterInfo(StreamInput in) throws IOException { @@ -117,6 +121,39 @@ public ClusterInfo(StreamInput in) throws IOException { } else { this.nodeFileCacheStats = Map.of(); } + + calculateAvgFreeAndTotalBytes(mostAvailableSpaceUsage); + } + + /** + * Returns a {@link DiskUsage} for the {@link RoutingNode} using the + * average usage of other nodes in the disk usage map. + * @param usages Map of nodeId to DiskUsage for all known nodes + */ + private void calculateAvgFreeAndTotalBytes(final Map usages) { + if (usages == null || usages.isEmpty()) { + this.avgTotalBytes = 0; + this.avgFreeByte = 0; + return; + } + + long totalBytes = 0; + long freeBytes = 0; + for (DiskUsage du : usages.values()) { + totalBytes += du.getTotalBytes(); + freeBytes += du.getFreeBytes(); + } + + this.avgTotalBytes = totalBytes / usages.size(); + this.avgFreeByte = freeBytes / usages.size(); + } + + public long getAvgFreeByte() { + return avgFreeByte; + } + + public long getAvgTotalBytes() { + return avgTotalBytes; } @Override diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/DiskThresholdDecider.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/DiskThresholdDecider.java index efa5115939d3c..5fc3f282f33f7 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/DiskThresholdDecider.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/DiskThresholdDecider.java @@ -140,9 +140,8 @@ public static long sizeOfRelocatingShards( // Where reserved space is unavailable (e.g. stats are out-of-sync) compute a conservative estimate for initialising shards final List initializingShards = node.shardsWithState(ShardRoutingState.INITIALIZING); - initializingShards.removeIf(shardRouting -> reservedSpace.containsShardId(shardRouting.shardId())); for (ShardRouting routing : initializingShards) { - if (routing.relocatingNodeId() == null) { + if (routing.relocatingNodeId() == null || reservedSpace.containsShardId(routing.shardId())) { // in practice the only initializing-but-not-relocating shards with a nonzero expected shard size will be ones created // by a resize (shrink/split/clone) operation which we expect to happen using hard links, so they shouldn't be taking // any additional space and can be ignored here @@ -230,7 +229,14 @@ public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, Routing // subtractLeavingShards is passed as false here, because they still use disk space, and therefore we should be extra careful // and take the size into account - final DiskUsageWithRelocations usage = getDiskUsage(node, allocation, usages, false); + final DiskUsageWithRelocations usage = getDiskUsage( + node, + allocation, + usages, + clusterInfo.getAvgFreeByte(), + clusterInfo.getAvgTotalBytes(), + false + ); // First, check that the node currently over the low watermark double freeDiskPercentage = usage.getFreeDiskAsPercentage(); // Cache the used disk percentage for displaying disk percentages consistent with documentation @@ -492,7 +498,14 @@ public Decision canRemain(ShardRouting shardRouting, RoutingNode node, RoutingAl // subtractLeavingShards is passed as true here, since this is only for shards remaining, we will *eventually* have enough disk // since shards are moving away. No new shards will be incoming since in canAllocate we pass false for this check. - final DiskUsageWithRelocations usage = getDiskUsage(node, allocation, usages, true); + final DiskUsageWithRelocations usage = getDiskUsage( + node, + allocation, + usages, + clusterInfo.getAvgFreeByte(), + clusterInfo.getAvgTotalBytes(), + true + ); final String dataPath = clusterInfo.getDataPath(shardRouting); // If this node is already above the high threshold, the shard cannot remain (get it off!) final double freeDiskPercentage = usage.getFreeDiskAsPercentage(); @@ -581,13 +594,15 @@ private DiskUsageWithRelocations getDiskUsage( RoutingNode node, RoutingAllocation allocation, final Map usages, + final long avgFreeBytes, + final long avgTotalBytes, boolean subtractLeavingShards ) { DiskUsage usage = usages.get(node.nodeId()); if (usage == null) { // If there is no usage, and we have other nodes in the cluster, // use the average usage for all nodes as the usage for this node - usage = averageUsage(node, usages); + usage = new DiskUsage(node.nodeId(), node.node().getName(), "_na_", avgTotalBytes, avgFreeBytes); if (logger.isDebugEnabled()) { logger.debug( "unable to determine disk usage for {}, defaulting to average across nodes [{} total] [{} free] [{}% free]", @@ -619,26 +634,6 @@ private DiskUsageWithRelocations getDiskUsage( return diskUsageWithRelocations; } - /** - * Returns a {@link DiskUsage} for the {@link RoutingNode} using the - * average usage of other nodes in the disk usage map. - * @param node Node to return an averaged DiskUsage object for - * @param usages Map of nodeId to DiskUsage for all known nodes - * @return DiskUsage representing given node using the average disk usage - */ - DiskUsage averageUsage(RoutingNode node, final Map usages) { - if (usages.size() == 0) { - return new DiskUsage(node.nodeId(), node.node().getName(), "_na_", 0, 0); - } - long totalBytes = 0; - long freeBytes = 0; - for (DiskUsage du : usages.values()) { - totalBytes += du.getTotalBytes(); - freeBytes += du.getFreeBytes(); - } - return new DiskUsage(node.nodeId(), node.node().getName(), "_na_", totalBytes / usages.size(), freeBytes / usages.size()); - } - /** * Given the DiskUsage for a node and the size of the shard, return the * percentage of free disk if the shard were to be allocated to the node. diff --git a/server/src/test/java/org/opensearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java b/server/src/test/java/org/opensearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java index 652633e689b93..2e24640fe858d 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/allocation/decider/DiskThresholdDeciderTests.java @@ -863,19 +863,6 @@ public void testUnknownDiskUsage() { assertThat(clusterState.getRoutingNodes().node("node1").size(), equalTo(1)); } - public void testAverageUsage() { - RoutingNode rn = new RoutingNode("node1", newNode("node1")); - DiskThresholdDecider decider = makeDecider(Settings.EMPTY); - - final Map usages = new HashMap<>(); - usages.put("node2", new DiskUsage("node2", "n2", "/dev/null", 100, 50)); // 50% used - usages.put("node3", new DiskUsage("node3", "n3", "/dev/null", 100, 0)); // 100% used - - DiskUsage node1Usage = decider.averageUsage(rn, usages); - assertThat(node1Usage.getTotalBytes(), equalTo(100L)); - assertThat(node1Usage.getFreeBytes(), equalTo(25L)); - } - public void testFreeDiskPercentageAfterShardAssigned() { DiskThresholdDecider decider = makeDecider(Settings.EMPTY); From e485856e2794de2b019be34a50df389dac136b89 Mon Sep 17 00:00:00 2001 From: Liyun Xiu Date: Tue, 23 Jul 2024 20:14:26 +0800 Subject: [PATCH 25/37] Use default value when index.number_of_replicas is null (#14812) * Use default value when index.number_of_replicas is null Signed-off-by: Liyun Xiu * Add integration test Signed-off-by: Liyun Xiu * Add changelog Signed-off-by: Liyun Xiu --------- Signed-off-by: Liyun Xiu --- CHANGELOG.md | 1 + .../admin/indices/create/CreateIndexIT.java | 24 +++++++++++++++++ .../metadata/MetadataCreateIndexService.java | 3 ++- .../MetadataCreateIndexServiceTests.java | 27 +++++++++++++++++++ 4 files changed, 54 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 29c78ea7e3e4f..5a54c5150da76 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -85,6 +85,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Fix create or update alias API doesn't throw exception for unsupported parameters ([#14719](https://github.com/opensearch-project/OpenSearch/pull/14719)) - Refactoring FilterPath.parse by using an iterative approach ([#14200](https://github.com/opensearch-project/OpenSearch/pull/14200)) - Refactoring Grok.validatePatternBank by using an iterative approach ([#14206](https://github.com/opensearch-project/OpenSearch/pull/14206)) +- Fix NPE when creating index with index.number_of_replicas set to null ([#14812](https://github.com/opensearch-project/OpenSearch/pull/14812)) - Update help output for _cat ([#14722](https://github.com/opensearch-project/OpenSearch/pull/14722)) - Fix bulk upsert ignores the default_pipeline and final_pipeline when auto-created index matches the index template ([#12891](https://github.com/opensearch-project/OpenSearch/pull/12891)) - Fix NPE in ReplicaShardAllocator ([#14385](https://github.com/opensearch-project/OpenSearch/pull/14385)) diff --git a/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/CreateIndexIT.java b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/CreateIndexIT.java index 1c182b05fa4a8..fbe713d9e22c4 100644 --- a/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/CreateIndexIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/create/CreateIndexIT.java @@ -406,4 +406,28 @@ public void testIndexNameInResponse() { assertEquals("Should have index name in response", "foo", response.index()); } + public void testCreateIndexWithNullReplicaCountPickUpClusterReplica() { + int numReplicas = 3; + String indexName = "test-idx-1"; + assertAcked( + client().admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put("cluster.default_number_of_replicas", numReplicas).build()) + .get() + ); + Settings settings = Settings.builder() + .put(IndexMetadata.INDEX_NUMBER_OF_SHARDS_SETTING.getKey(), 1) + .put(IndexMetadata.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), (String) null) + .build(); + assertAcked(client().admin().indices().prepareCreate(indexName).setSettings(settings).get()); + IndicesService indicesService = internalCluster().getInstance(IndicesService.class, internalCluster().getClusterManagerName()); + for (IndexService indexService : indicesService) { + assertEquals(indexName, indexService.index().getName()); + assertEquals( + numReplicas, + (int) indexService.getIndexSettings().getSettings().getAsInt(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, null) + ); + } + } } 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 7973745ce84b3..50d25b11ef810 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java @@ -946,7 +946,8 @@ static Settings aggregateIndexSettings( if (INDEX_NUMBER_OF_SHARDS_SETTING.exists(indexSettingsBuilder) == false) { indexSettingsBuilder.put(SETTING_NUMBER_OF_SHARDS, INDEX_NUMBER_OF_SHARDS_SETTING.get(settings)); } - if (INDEX_NUMBER_OF_REPLICAS_SETTING.exists(indexSettingsBuilder) == false) { + if (INDEX_NUMBER_OF_REPLICAS_SETTING.exists(indexSettingsBuilder) == false + || indexSettingsBuilder.get(SETTING_NUMBER_OF_REPLICAS) == null) { indexSettingsBuilder.put(SETTING_NUMBER_OF_REPLICAS, DEFAULT_REPLICA_COUNT_SETTING.get(currentState.metadata().settings())); } if (settings.get(SETTING_AUTO_EXPAND_REPLICAS) != null && indexSettingsBuilder.get(SETTING_AUTO_EXPAND_REPLICAS) == null) { 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 0d86cfcca389c..86ca8b3ad6319 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java @@ -2151,6 +2151,33 @@ public void testAsyncDurabilityThrowsExceptionWhenRestrictSettingTrue() { ); } + public void testAggregateIndexSettingsIndexReplicaIsSetToNull() { + // This checks that aggregateIndexSettings works for the case when the index setting `index.number_of_replicas` is set to null + request = new CreateIndexClusterStateUpdateRequest("create index", "test", "test"); + request.settings(Settings.builder().putNull(SETTING_NUMBER_OF_REPLICAS).build()); + Integer clusterDefaultReplicaNumber = 5; + Metadata metadata = new Metadata.Builder().persistentSettings( + Settings.builder().put("cluster.default_number_of_replicas", clusterDefaultReplicaNumber).build() + ).build(); + ClusterState clusterState = ClusterState.builder(org.opensearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .build(); + Settings settings = Settings.builder().put(CLUSTER_REMOTE_INDEX_RESTRICT_ASYNC_DURABILITY_SETTING.getKey(), true).build(); + clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + Settings aggregatedSettings = aggregateIndexSettings( + clusterState, + request, + Settings.EMPTY, + null, + Settings.EMPTY, + IndexScopedSettings.DEFAULT_SCOPED_SETTINGS, + randomShardLimitService(), + Collections.emptySet(), + clusterSettings + ); + assertEquals(clusterDefaultReplicaNumber.toString(), aggregatedSettings.get(SETTING_NUMBER_OF_REPLICAS)); + } + public void testRequestDurabilityWhenRestrictSettingTrue() { // This checks that aggregateIndexSettings works for the case when the cluster setting // cluster.remote_store.index.restrict.async-durability is false or not set, it allows all types of durability modes From f85a58f64e5aaba76eb519e309881f288aff8fa6 Mon Sep 17 00:00:00 2001 From: shailendra0811 <167273922+shailendra0811@users.noreply.github.com> Date: Tue, 23 Jul 2024 18:10:32 +0530 Subject: [PATCH 26/37] [Remote Routing Table] Implement write and read flow for shard diff file. (#14684) * Implement write and read flow to upload/download shard diff file. Signed-off-by: Shailendra Singh --- CHANGELOG.md | 1 + .../remote/RemoteRoutingTableServiceIT.java | 97 +++++- .../routing/RoutingTableIncrementalDiff.java | 168 ++++++++++ .../InternalRemoteRoutingTableService.java | 73 +++- .../remote/NoopRemoteRoutingTableService.java | 33 +- .../remote/RemoteRoutingTableService.java | 48 ++- .../remote/ClusterMetadataManifest.java | 15 +- .../remote/ClusterStateDiffManifest.java | 60 +++- .../RemoteClusterStateCleanupManager.java | 26 ++ .../remote/RemoteClusterStateService.java | 94 +++++- .../remote/RemoteClusterStateUtils.java | 1 + .../remote/RemotePersistenceStats.java | 11 + .../model/RemoteClusterMetadataManifest.java | 7 +- .../routingtable/RemoteRoutingTableDiff.java | 150 +++++++++ .../RemoteRoutingTableServiceTests.java | 165 ++++++++- .../remote/ClusterMetadataManifestTests.java | 81 ++++- ...RemoteClusterStateCleanupManagerTests.java | 146 ++++++++ .../RemoteClusterStateServiceTests.java | 177 +++++++++- .../model/ClusterStateDiffManifestTests.java | 69 +++- .../RemoteIndexRoutingTableDiffTests.java | 317 ++++++++++++++++++ 20 files changed, 1663 insertions(+), 76 deletions(-) create mode 100644 server/src/main/java/org/opensearch/cluster/routing/RoutingTableIncrementalDiff.java create mode 100644 server/src/main/java/org/opensearch/gateway/remote/routingtable/RemoteRoutingTableDiff.java create mode 100644 server/src/test/java/org/opensearch/gateway/remote/routingtable/RemoteIndexRoutingTableDiffTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 5a54c5150da76..c8f185ca2bb3d 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -20,6 +20,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add matchesPluginSystemIndexPattern to SystemIndexRegistry ([#14750](https://github.com/opensearch-project/OpenSearch/pull/14750)) - Add Plugin interface for loading application based configuration templates (([#14659](https://github.com/opensearch-project/OpenSearch/issues/14659))) - Refactor remote-routing-table service inline with remote state interfaces([#14668](https://github.com/opensearch-project/OpenSearch/pull/14668)) +- Add shard-diff path to diff manifest to reduce number of read calls remote store (([#14684](https://github.com/opensearch-project/OpenSearch/pull/14684))) - Add SortResponseProcessor to Search Pipelines (([#14785](https://github.com/opensearch-project/OpenSearch/issues/14785))) - Add prefix mode verification setting for repository verification (([#14790](https://github.com/opensearch-project/OpenSearch/pull/14790))) - Add SplitResponseProcessor to Search Pipelines (([#14800](https://github.com/opensearch-project/OpenSearch/issues/14800))) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteRoutingTableServiceIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteRoutingTableServiceIT.java index 53764c0b4d0e8..b0d046cbdf3db 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteRoutingTableServiceIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/remote/RemoteRoutingTableServiceIT.java @@ -8,6 +8,7 @@ package org.opensearch.gateway.remote; +import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; import org.opensearch.action.admin.cluster.state.ClusterStateRequest; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.routing.IndexRoutingTable; @@ -32,16 +33,19 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import static org.opensearch.common.util.FeatureFlags.REMOTE_PUBLICATION_EXPERIMENTAL; import static org.opensearch.gateway.remote.RemoteClusterStateService.REMOTE_CLUSTER_STATE_ENABLED_SETTING; import static org.opensearch.gateway.remote.routingtable.RemoteIndexRoutingTable.INDEX_ROUTING_TABLE; +import static org.opensearch.indices.IndicesService.CLUSTER_DEFAULT_INDEX_REFRESH_INTERVAL_SETTING; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY; @OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) public class RemoteRoutingTableServiceIT extends RemoteStoreBaseIntegTestCase { private static final String INDEX_NAME = "test-index"; + private static final String INDEX_NAME_1 = "test-index-1"; BlobPath indexRoutingPath; AtomicInteger indexRoutingFiles = new AtomicInteger(); private final RemoteStoreEnums.PathType pathType = RemoteStoreEnums.PathType.HASHED_PREFIX; @@ -72,7 +76,13 @@ public void testRemoteRoutingTableIndexLifecycle() throws Exception { RemoteClusterStateService.class ); RemoteManifestManager remoteManifestManager = remoteClusterStateService.getRemoteManifestManager(); - verifyUpdatesInManifestFile(remoteManifestManager); + Optional latestManifest = remoteManifestManager.getLatestClusterMetadataManifest( + getClusterState().getClusterName().value(), + getClusterState().getMetadata().clusterUUID() + ); + List expectedIndexNames = new ArrayList<>(); + List deletedIndexNames = new ArrayList<>(); + verifyUpdatesInManifestFile(latestManifest, expectedIndexNames, 1, deletedIndexNames, true); List routingTableVersions = getRoutingTableFromAllNodes(); assertTrue(areRoutingTablesSame(routingTableVersions)); @@ -86,7 +96,11 @@ public void testRemoteRoutingTableIndexLifecycle() throws Exception { assertTrue(indexRoutingFilesAfterUpdate >= indexRoutingFiles.get() + 3); }); - verifyUpdatesInManifestFile(remoteManifestManager); + latestManifest = remoteManifestManager.getLatestClusterMetadataManifest( + getClusterState().getClusterName().value(), + getClusterState().getMetadata().clusterUUID() + ); + verifyUpdatesInManifestFile(latestManifest, expectedIndexNames, 1, deletedIndexNames, true); routingTableVersions = getRoutingTableFromAllNodes(); assertTrue(areRoutingTablesSame(routingTableVersions)); @@ -98,6 +112,42 @@ public void testRemoteRoutingTableIndexLifecycle() throws Exception { assertTrue(areRoutingTablesSame(routingTableVersions)); } + public void testRemoteRoutingTableEmptyRoutingTableDiff() throws Exception { + prepareClusterAndVerifyRepository(); + + RemoteClusterStateService remoteClusterStateService = internalCluster().getClusterManagerNodeInstance( + RemoteClusterStateService.class + ); + RemoteManifestManager remoteManifestManager = remoteClusterStateService.getRemoteManifestManager(); + Optional latestManifest = remoteManifestManager.getLatestClusterMetadataManifest( + getClusterState().getClusterName().value(), + getClusterState().getMetadata().clusterUUID() + ); + List expectedIndexNames = new ArrayList<>(); + List deletedIndexNames = new ArrayList<>(); + verifyUpdatesInManifestFile(latestManifest, expectedIndexNames, 1, deletedIndexNames, true); + + List routingTableVersions = getRoutingTableFromAllNodes(); + assertTrue(areRoutingTablesSame(routingTableVersions)); + + // Update cluster settings + ClusterUpdateSettingsResponse response = client().admin() + .cluster() + .prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(CLUSTER_DEFAULT_INDEX_REFRESH_INTERVAL_SETTING.getKey(), 0, TimeUnit.SECONDS)) + .get(); + assertTrue(response.isAcknowledged()); + + latestManifest = remoteManifestManager.getLatestClusterMetadataManifest( + getClusterState().getClusterName().value(), + getClusterState().getMetadata().clusterUUID() + ); + verifyUpdatesInManifestFile(latestManifest, expectedIndexNames, 1, deletedIndexNames, false); + + routingTableVersions = getRoutingTableFromAllNodes(); + assertTrue(areRoutingTablesSame(routingTableVersions)); + } + public void testRemoteRoutingTableIndexNodeRestart() throws Exception { BlobStoreRepository repository = prepareClusterAndVerifyRepository(); @@ -124,10 +174,16 @@ public void testRemoteRoutingTableIndexNodeRestart() throws Exception { RemoteClusterStateService.class ); RemoteManifestManager remoteManifestManager = remoteClusterStateService.getRemoteManifestManager(); - verifyUpdatesInManifestFile(remoteManifestManager); + Optional latestManifest = remoteManifestManager.getLatestClusterMetadataManifest( + getClusterState().getClusterName().value(), + getClusterState().getMetadata().clusterUUID() + ); + List expectedIndexNames = new ArrayList<>(); + List deletedIndexNames = new ArrayList<>(); + verifyUpdatesInManifestFile(latestManifest, expectedIndexNames, 1, deletedIndexNames, true); } - public void testRemoteRoutingTableIndexMasterRestart1() throws Exception { + public void testRemoteRoutingTableIndexMasterRestart() throws Exception { BlobStoreRepository repository = prepareClusterAndVerifyRepository(); List routingTableVersions = getRoutingTableFromAllNodes(); @@ -153,7 +209,13 @@ public void testRemoteRoutingTableIndexMasterRestart1() throws Exception { RemoteClusterStateService.class ); RemoteManifestManager remoteManifestManager = remoteClusterStateService.getRemoteManifestManager(); - verifyUpdatesInManifestFile(remoteManifestManager); + Optional latestManifest = remoteManifestManager.getLatestClusterMetadataManifest( + getClusterState().getClusterName().value(), + getClusterState().getMetadata().clusterUUID() + ); + List expectedIndexNames = new ArrayList<>(); + List deletedIndexNames = new ArrayList<>(); + verifyUpdatesInManifestFile(latestManifest, expectedIndexNames, 1, deletedIndexNames, true); } private BlobStoreRepository prepareClusterAndVerifyRepository() throws Exception { @@ -208,18 +270,23 @@ private BlobPath getIndexRoutingPath(BlobPath indexRoutingPath, String indexUUID ); } - private void verifyUpdatesInManifestFile(RemoteManifestManager remoteManifestManager) { - Optional latestManifest = remoteManifestManager.getLatestClusterMetadataManifest( - getClusterState().getClusterName().value(), - getClusterState().getMetadata().clusterUUID() - ); + private void verifyUpdatesInManifestFile( + Optional latestManifest, + List expectedIndexNames, + int expectedIndicesRoutingFilesInManifest, + List expectedDeletedIndex, + boolean isRoutingTableDiffFileExpected + ) { assertTrue(latestManifest.isPresent()); ClusterMetadataManifest manifest = latestManifest.get(); - assertTrue(manifest.getDiffManifest().getIndicesRoutingUpdated().contains(INDEX_NAME)); - assertTrue(manifest.getDiffManifest().getIndicesDeleted().isEmpty()); - assertFalse(manifest.getIndicesRouting().isEmpty()); - assertEquals(1, manifest.getIndicesRouting().size()); - assertTrue(manifest.getIndicesRouting().get(0).getUploadedFilename().contains(indexRoutingPath.buildAsString())); + + assertEquals(expectedIndexNames, manifest.getDiffManifest().getIndicesRoutingUpdated()); + assertEquals(expectedDeletedIndex, manifest.getDiffManifest().getIndicesDeleted()); + assertEquals(expectedIndicesRoutingFilesInManifest, manifest.getIndicesRouting().size()); + for (ClusterMetadataManifest.UploadedIndexMetadata uploadedFilename : manifest.getIndicesRouting()) { + assertTrue(uploadedFilename.getUploadedFilename().contains(indexRoutingPath.buildAsString())); + } + assertEquals(isRoutingTableDiffFileExpected, manifest.getDiffManifest().getIndicesRoutingDiffPath() != null); } private List getRoutingTableFromAllNodes() throws ExecutionException, InterruptedException { diff --git a/server/src/main/java/org/opensearch/cluster/routing/RoutingTableIncrementalDiff.java b/server/src/main/java/org/opensearch/cluster/routing/RoutingTableIncrementalDiff.java new file mode 100644 index 0000000000000..3d75b22a8ed7f --- /dev/null +++ b/server/src/main/java/org/opensearch/cluster/routing/RoutingTableIncrementalDiff.java @@ -0,0 +1,168 @@ +/* + * 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.routing; + +import org.opensearch.cluster.Diff; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Represents a difference between {@link RoutingTable} objects that can be serialized and deserialized. + */ +public class RoutingTableIncrementalDiff implements Diff { + + private final Map> diffs; + + /** + * Constructs a new RoutingTableIncrementalDiff with the given differences. + * + * @param diffs a map containing the differences of {@link IndexRoutingTable}. + */ + public RoutingTableIncrementalDiff(Map> diffs) { + this.diffs = diffs; + } + + /** + * Gets the map of differences of {@link IndexRoutingTable}. + * + * @return a map containing the differences. + */ + public Map> getDiffs() { + return diffs; + } + + /** + * Reads a {@link RoutingTableIncrementalDiff} from the given {@link StreamInput}. + * + * @param in the input stream to read from. + * @return the deserialized RoutingTableIncrementalDiff. + * @throws IOException if an I/O exception occurs while reading from the stream. + */ + public static RoutingTableIncrementalDiff readFrom(StreamInput in) throws IOException { + int size = in.readVInt(); + Map> diffs = new HashMap<>(); + + for (int i = 0; i < size; i++) { + String key = in.readString(); + Diff diff = IndexRoutingTableIncrementalDiff.readFrom(in); + diffs.put(key, diff); + } + return new RoutingTableIncrementalDiff(diffs); + } + + /** + * Applies the differences to the provided {@link RoutingTable}. + * + * @param part the original RoutingTable to which the differences will be applied. + * @return the updated RoutingTable with the applied differences. + */ + @Override + public RoutingTable apply(RoutingTable part) { + RoutingTable.Builder builder = new RoutingTable.Builder(); + for (IndexRoutingTable indexRoutingTable : part) { + builder.add(indexRoutingTable); // Add existing index routing tables to builder + } + + // Apply the diffs + for (Map.Entry> entry : diffs.entrySet()) { + builder.add(entry.getValue().apply(part.index(entry.getKey()))); + } + + return builder.build(); + } + + /** + * Writes the differences to the given {@link StreamOutput}. + * + * @param out the output stream to write to. + * @throws IOException if an I/O exception occurs while writing to the stream. + */ + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(diffs.size()); + for (Map.Entry> entry : diffs.entrySet()) { + out.writeString(entry.getKey()); + entry.getValue().writeTo(out); + } + } + + /** + * Represents a difference between {@link IndexShardRoutingTable} objects that can be serialized and deserialized. + */ + public static class IndexRoutingTableIncrementalDiff implements Diff { + + private final List indexShardRoutingTables; + + /** + * Constructs a new IndexShardRoutingTableDiff with the given shard routing tables. + * + * @param indexShardRoutingTables a list of IndexShardRoutingTable representing the differences. + */ + public IndexRoutingTableIncrementalDiff(List indexShardRoutingTables) { + this.indexShardRoutingTables = indexShardRoutingTables; + } + + /** + * Applies the differences to the provided {@link IndexRoutingTable}. + * + * @param part the original IndexRoutingTable to which the differences will be applied. + * @return the updated IndexRoutingTable with the applied differences. + */ + @Override + public IndexRoutingTable apply(IndexRoutingTable part) { + IndexRoutingTable.Builder builder = new IndexRoutingTable.Builder(part.getIndex()); + for (IndexShardRoutingTable shardRoutingTable : part) { + builder.addIndexShard(shardRoutingTable); // Add existing shards to builder + } + + // Apply the diff: update or add the new shard routing tables + for (IndexShardRoutingTable diffShard : indexShardRoutingTables) { + builder.addIndexShard(diffShard); + } + return builder.build(); + } + + /** + * Writes the differences to the given {@link StreamOutput}. + * + * @param out the output stream to write to. + * @throws IOException if an I/O exception occurs while writing to the stream. + */ + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(indexShardRoutingTables.size()); + for (IndexShardRoutingTable shardRoutingTable : indexShardRoutingTables) { + IndexShardRoutingTable.Builder.writeTo(shardRoutingTable, out); + } + } + + /** + * Reads a {@link IndexRoutingTableIncrementalDiff} from the given {@link StreamInput}. + * + * @param in the input stream to read from. + * @return the deserialized IndexShardRoutingTableDiff. + * @throws IOException if an I/O exception occurs while reading from the stream. + */ + public static IndexRoutingTableIncrementalDiff readFrom(StreamInput in) throws IOException { + int size = in.readVInt(); + List indexShardRoutingTables = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + IndexShardRoutingTable shardRoutingTable = IndexShardRoutingTable.Builder.readFrom(in); + indexShardRoutingTables.add(shardRoutingTable); + } + return new IndexRoutingTableIncrementalDiff(indexShardRoutingTables); + } + } +} diff --git a/server/src/main/java/org/opensearch/cluster/routing/remote/InternalRemoteRoutingTableService.java b/server/src/main/java/org/opensearch/cluster/routing/remote/InternalRemoteRoutingTableService.java index d7ebc54598b37..3c578a8c5c01f 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/remote/InternalRemoteRoutingTableService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/remote/InternalRemoteRoutingTableService.java @@ -12,9 +12,11 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.action.LatchedActionListener; +import org.opensearch.cluster.Diff; import org.opensearch.cluster.DiffableUtils; import org.opensearch.cluster.routing.IndexRoutingTable; import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.routing.RoutingTableIncrementalDiff; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.lifecycle.AbstractLifecycleComponent; import org.opensearch.common.remote.RemoteWritableEntityStore; @@ -25,8 +27,10 @@ import org.opensearch.core.compress.Compressor; import org.opensearch.gateway.remote.ClusterMetadataManifest; import org.opensearch.gateway.remote.RemoteStateTransferException; +import org.opensearch.gateway.remote.model.RemoteClusterStateBlobStore; import org.opensearch.gateway.remote.model.RemoteRoutingTableBlobStore; import org.opensearch.gateway.remote.routingtable.RemoteIndexRoutingTable; +import org.opensearch.gateway.remote.routingtable.RemoteRoutingTableDiff; import org.opensearch.index.translog.transfer.BlobStoreTransferService; import org.opensearch.node.Node; import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; @@ -58,6 +62,7 @@ public class InternalRemoteRoutingTableService extends AbstractLifecycleComponen private final Supplier repositoriesService; private Compressor compressor; private RemoteWritableEntityStore remoteIndexRoutingTableStore; + private RemoteWritableEntityStore remoteRoutingTableDiffStore; private final ClusterSettings clusterSettings; private BlobStoreRepository blobStoreRepository; private final ThreadPool threadPool; @@ -84,9 +89,10 @@ public List getIndicesRouting(RoutingTable routingTable) { /** * Returns diff between the two routing tables, which includes upserts and deletes. + * * @param before previous routing table - * @param after current routing table - * @return diff of the previous and current routing table + * @param after current routing table + * @return incremental diff of the previous and current routing table */ public DiffableUtils.MapDiff> getIndicesRoutingMapDiff( RoutingTable before, @@ -96,7 +102,7 @@ public DiffableUtils.MapDiff> indexRoutingTableDiff, + LatchedActionListener latchedActionListener + ) { + RoutingTableIncrementalDiff routingTableIncrementalDiff = new RoutingTableIncrementalDiff(indexRoutingTableDiff); + RemoteRoutingTableDiff remoteRoutingTableDiff = new RemoteRoutingTableDiff( + routingTableIncrementalDiff, + clusterUUID, + compressor, + term, + version + ); + + ActionListener completionListener = ActionListener.wrap( + resp -> latchedActionListener.onResponse(remoteRoutingTableDiff.getUploadedMetadata()), + ex -> latchedActionListener.onFailure( + new RemoteStateTransferException("Exception in writing index routing diff to remote store", ex) + ) + ); + + remoteRoutingTableDiffStore.writeAsync(remoteRoutingTableDiff, completionListener); + } + /** * Combines IndicesRoutingMetadata from previous manifest and current uploaded indices, removes deleted indices. * @param previousManifest previous manifest, used to get all existing indices routing paths @@ -171,6 +204,22 @@ public void getAsyncIndexRoutingReadAction( remoteIndexRoutingTableStore.readAsync(remoteIndexRoutingTable, actionListener); } + @Override + public void getAsyncIndexRoutingTableDiffReadAction( + String clusterUUID, + String uploadedFilename, + LatchedActionListener latchedActionListener + ) { + ActionListener actionListener = ActionListener.wrap( + latchedActionListener::onResponse, + latchedActionListener::onFailure + ); + + RemoteRoutingTableDiff remoteRoutingTableDiff = new RemoteRoutingTableDiff(uploadedFilename, clusterUUID, compressor); + + remoteRoutingTableDiffStore.readAsync(remoteRoutingTableDiff, actionListener); + } + @Override public List getUpdatedIndexRoutingTableMetadata( List updatedIndicesRouting, @@ -212,6 +261,14 @@ protected void doStart() { ThreadPool.Names.REMOTE_STATE_READ, clusterSettings ); + + this.remoteRoutingTableDiffStore = new RemoteClusterStateBlobStore<>( + new BlobStoreTransferService(blobStoreRepository.blobStore(), threadPool), + blobStoreRepository, + clusterName, + threadPool, + ThreadPool.Names.REMOTE_STATE_READ + ); } @Override @@ -227,4 +284,14 @@ public void deleteStaleIndexRoutingPaths(List stalePaths) throws IOExcep throw e; } } + + public void deleteStaleIndexRoutingDiffPaths(List stalePaths) throws IOException { + try { + logger.debug(() -> "Deleting stale index routing diff files from remote - " + stalePaths); + blobStoreRepository.blobStore().blobContainer(BlobPath.cleanPath()).deleteBlobsIgnoringIfNotExists(stalePaths); + } catch (IOException e) { + logger.error(() -> new ParameterizedMessage("Failed to delete some stale index routing diff paths from {}", stalePaths), e); + throw e; + } + } } diff --git a/server/src/main/java/org/opensearch/cluster/routing/remote/NoopRemoteRoutingTableService.java b/server/src/main/java/org/opensearch/cluster/routing/remote/NoopRemoteRoutingTableService.java index e6e68e01e761f..1ebf3206212a1 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/remote/NoopRemoteRoutingTableService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/remote/NoopRemoteRoutingTableService.java @@ -9,9 +9,11 @@ package org.opensearch.cluster.routing.remote; import org.opensearch.action.LatchedActionListener; +import org.opensearch.cluster.Diff; import org.opensearch.cluster.DiffableUtils; import org.opensearch.cluster.routing.IndexRoutingTable; import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.routing.RoutingTableIncrementalDiff; import org.opensearch.common.lifecycle.AbstractLifecycleComponent; import org.opensearch.gateway.remote.ClusterMetadataManifest; @@ -34,7 +36,12 @@ public DiffableUtils.MapDiff> indexRoutingTableDiff, + LatchedActionListener latchedActionListener + ) { + // noop + } + @Override public List getAllUploadedIndicesRouting( ClusterMetadataManifest previousManifest, @@ -67,6 +85,15 @@ public void getAsyncIndexRoutingReadAction( // noop } + @Override + public void getAsyncIndexRoutingTableDiffReadAction( + String clusterUUID, + String uploadedFilename, + LatchedActionListener latchedActionListener + ) { + // noop + } + @Override public List getUpdatedIndexRoutingTableMetadata( List updatedIndicesRouting, @@ -95,4 +122,8 @@ protected void doClose() throws IOException { public void deleteStaleIndexRoutingPaths(List stalePaths) throws IOException { // noop } + + public void deleteStaleIndexRoutingDiffPaths(List stalePaths) throws IOException { + // noop + } } diff --git a/server/src/main/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableService.java b/server/src/main/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableService.java index 0b0b4bb7dbc84..0811a5f3010f4 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableService.java @@ -9,15 +9,19 @@ package org.opensearch.cluster.routing.remote; import org.opensearch.action.LatchedActionListener; +import org.opensearch.cluster.Diff; import org.opensearch.cluster.DiffableUtils; import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.IndexShardRoutingTable; import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.routing.RoutingTableIncrementalDiff; import org.opensearch.common.lifecycle.LifecycleComponent; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.gateway.remote.ClusterMetadataManifest; import java.io.IOException; +import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -27,16 +31,36 @@ * @opensearch.internal */ public interface RemoteRoutingTableService extends LifecycleComponent { - public static final DiffableUtils.NonDiffableValueSerializer CUSTOM_ROUTING_TABLE_VALUE_SERIALIZER = - new DiffableUtils.NonDiffableValueSerializer() { + + public static final DiffableUtils.DiffableValueSerializer CUSTOM_ROUTING_TABLE_DIFFABLE_VALUE_SERIALIZER = + new DiffableUtils.DiffableValueSerializer() { + @Override + public IndexRoutingTable read(StreamInput in, String key) throws IOException { + return IndexRoutingTable.readFrom(in); + } + @Override public void write(IndexRoutingTable value, StreamOutput out) throws IOException { value.writeTo(out); } @Override - public IndexRoutingTable read(StreamInput in, String key) throws IOException { - return IndexRoutingTable.readFrom(in); + public Diff readDiff(StreamInput in, String key) throws IOException { + return IndexRoutingTable.readDiffFrom(in); + } + + @Override + public Diff diff(IndexRoutingTable currentState, IndexRoutingTable previousState) { + List diffs = new ArrayList<>(); + for (Map.Entry entry : currentState.getShards().entrySet()) { + Integer index = entry.getKey(); + IndexShardRoutingTable currentShardRoutingTable = entry.getValue(); + IndexShardRoutingTable previousShardRoutingTable = previousState.shard(index); + if (previousShardRoutingTable == null || !previousShardRoutingTable.equals(currentShardRoutingTable)) { + diffs.add(currentShardRoutingTable); + } + } + return new RoutingTableIncrementalDiff.IndexRoutingTableIncrementalDiff(diffs); } }; @@ -48,6 +72,12 @@ void getAsyncIndexRoutingReadAction( LatchedActionListener latchedActionListener ); + void getAsyncIndexRoutingTableDiffReadAction( + String clusterUUID, + String uploadedFilename, + LatchedActionListener latchedActionListener + ); + List getUpdatedIndexRoutingTableMetadata( List updatedIndicesRouting, List allIndicesRouting @@ -66,6 +96,14 @@ void getAsyncIndexRoutingWriteAction( LatchedActionListener latchedActionListener ); + void getAsyncIndexRoutingDiffWriteAction( + String clusterUUID, + long term, + long version, + Map> indexRoutingTableDiff, + LatchedActionListener latchedActionListener + ); + List getAllUploadedIndicesRouting( ClusterMetadataManifest previousManifest, List indicesRoutingUploaded, @@ -74,4 +112,6 @@ List getAllUploadedIndicesRouting public void deleteStaleIndexRoutingPaths(List stalePaths) throws IOException; + public void deleteStaleIndexRoutingDiffPaths(List stalePaths) throws IOException; + } diff --git a/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataManifest.java b/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataManifest.java index 3a66419b1dc20..71815b6ee324c 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataManifest.java +++ b/server/src/main/java/org/opensearch/gateway/remote/ClusterMetadataManifest.java @@ -44,6 +44,7 @@ public class ClusterMetadataManifest implements Writeable, ToXContentFragment { public static final int CODEC_V2 = 2; // In Codec V2, there are separate metadata files rather than a single global metadata file, // also we introduce index routing-metadata, diff and other attributes as part of manifest // required for state publication + public static final int CODEC_V3 = 3; // In Codec V3, we have introduced new diff field in diff-manifest's routing_table_diff private static final ParseField CLUSTER_TERM_FIELD = new ParseField("cluster_term"); private static final ParseField STATE_VERSION_FIELD = new ParseField("state_version"); @@ -109,6 +110,10 @@ private static ClusterMetadataManifest.Builder manifestV2Builder(Object[] fields .clusterStateCustomMetadataMap(clusterStateCustomMetadata(fields)); } + private static ClusterMetadataManifest.Builder manifestV3Builder(Object[] fields) { + return manifestV2Builder(fields); + } + private static long term(Object[] fields) { return (long) fields[0]; } @@ -226,12 +231,18 @@ private static ClusterStateDiffManifest diffManifest(Object[] fields) { fields -> manifestV2Builder(fields).build() ); - private static final ConstructingObjectParser CURRENT_PARSER = PARSER_V2; + private static final ConstructingObjectParser PARSER_V3 = new ConstructingObjectParser<>( + "cluster_metadata_manifest", + fields -> manifestV3Builder(fields).build() + ); + + private static final ConstructingObjectParser CURRENT_PARSER = PARSER_V3; static { declareParser(PARSER_V0, CODEC_V0); declareParser(PARSER_V1, CODEC_V1); declareParser(PARSER_V2, CODEC_V2); + declareParser(PARSER_V3, CODEC_V3); } private static void declareParser(ConstructingObjectParser parser, long codec_version) { @@ -309,7 +320,7 @@ private static void declareParser(ConstructingObjectParser ClusterStateDiffManifest.fromXContent(p), + (p, c) -> ClusterStateDiffManifest.fromXContent(p, codec_version), DIFF_MANIFEST ); } diff --git a/server/src/main/java/org/opensearch/gateway/remote/ClusterStateDiffManifest.java b/server/src/main/java/org/opensearch/gateway/remote/ClusterStateDiffManifest.java index aca53c92781e4..ab7fa1fddf4bf 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/ClusterStateDiffManifest.java +++ b/server/src/main/java/org/opensearch/gateway/remote/ClusterStateDiffManifest.java @@ -32,8 +32,8 @@ import static org.opensearch.cluster.DiffableUtils.NonDiffableValueSerializer.getAbstractInstance; import static org.opensearch.cluster.DiffableUtils.getStringKeySerializer; -import static org.opensearch.cluster.routing.remote.RemoteRoutingTableService.CUSTOM_ROUTING_TABLE_VALUE_SERIALIZER; import static org.opensearch.core.xcontent.XContentParserUtils.ensureExpectedToken; +import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V3; /** * Manifest of diff between two cluster states @@ -53,6 +53,7 @@ public class ClusterStateDiffManifest implements ToXContentFragment, Writeable { private static final String METADATA_CUSTOM_DIFF_FIELD = "metadata_custom_diff"; private static final String UPSERTS_FIELD = "upserts"; private static final String DELETES_FIELD = "deletes"; + private static final String DIFF_FIELD = "diff"; private static final String CLUSTER_BLOCKS_UPDATED_FIELD = "cluster_blocks_diff"; private static final String DISCOVERY_NODES_UPDATED_FIELD = "discovery_nodes_diff"; private static final String ROUTING_TABLE_DIFF = "routing_table_diff"; @@ -72,11 +73,17 @@ public class ClusterStateDiffManifest implements ToXContentFragment, Writeable { private final boolean discoveryNodesUpdated; private final List indicesRoutingUpdated; private final List indicesRoutingDeleted; + private String indicesRoutingDiffPath; private final boolean hashesOfConsistentSettingsUpdated; private final List clusterStateCustomUpdated; private final List clusterStateCustomDeleted; - public ClusterStateDiffManifest(ClusterState state, ClusterState previousState) { + public ClusterStateDiffManifest( + ClusterState state, + ClusterState previousState, + DiffableUtils.MapDiff> routingTableIncrementalDiff, + String indicesRoutingDiffPath + ) { fromStateUUID = previousState.stateUUID(); toStateUUID = state.stateUUID(); coordinationMetadataUpdated = !Metadata.isCoordinationMetadataEqual(state.metadata(), previousState.metadata()); @@ -103,17 +110,13 @@ public ClusterStateDiffManifest(ClusterState state, ClusterState previousState) customMetadataUpdated.addAll(customDiff.getUpserts().keySet()); customMetadataDeleted = customDiff.getDeletes(); - DiffableUtils.MapDiff> routingTableDiff = DiffableUtils.diff( - previousState.getRoutingTable().getIndicesRouting(), - state.getRoutingTable().getIndicesRouting(), - DiffableUtils.getStringKeySerializer(), - CUSTOM_ROUTING_TABLE_VALUE_SERIALIZER - ); - indicesRoutingUpdated = new ArrayList<>(); - routingTableDiff.getUpserts().forEach((k, v) -> indicesRoutingUpdated.add(k)); - - indicesRoutingDeleted = routingTableDiff.getDeletes(); + indicesRoutingDeleted = new ArrayList<>(); + this.indicesRoutingDiffPath = indicesRoutingDiffPath; + if (routingTableIncrementalDiff != null) { + routingTableIncrementalDiff.getUpserts().forEach((k, v) -> indicesRoutingUpdated.add(k)); + indicesRoutingDeleted.addAll(routingTableIncrementalDiff.getDeletes()); + } hashesOfConsistentSettingsUpdated = !state.metadata() .hashesOfConsistentSettings() .equals(previousState.metadata().hashesOfConsistentSettings()); @@ -126,6 +129,7 @@ public ClusterStateDiffManifest(ClusterState state, ClusterState previousState) clusterStateCustomUpdated = new ArrayList<>(clusterStateCustomDiff.getDiffs().keySet()); clusterStateCustomUpdated.addAll(clusterStateCustomDiff.getUpserts().keySet()); clusterStateCustomDeleted = clusterStateCustomDiff.getDeletes(); + List indicie1s = indicesRoutingUpdated; } public ClusterStateDiffManifest( @@ -143,6 +147,7 @@ public ClusterStateDiffManifest( boolean discoveryNodesUpdated, List indicesRoutingUpdated, List indicesRoutingDeleted, + String indicesRoutingDiffPath, boolean hashesOfConsistentSettingsUpdated, List clusterStateCustomUpdated, List clusterStateCustomDeleted @@ -164,6 +169,7 @@ public ClusterStateDiffManifest( this.hashesOfConsistentSettingsUpdated = hashesOfConsistentSettingsUpdated; this.clusterStateCustomUpdated = Collections.unmodifiableList(clusterStateCustomUpdated); this.clusterStateCustomDeleted = Collections.unmodifiableList(clusterStateCustomDeleted); + this.indicesRoutingDiffPath = indicesRoutingDiffPath; } public ClusterStateDiffManifest(StreamInput in) throws IOException { @@ -184,6 +190,7 @@ public ClusterStateDiffManifest(StreamInput in) throws IOException { this.hashesOfConsistentSettingsUpdated = in.readBoolean(); this.clusterStateCustomUpdated = in.readStringList(); this.clusterStateCustomDeleted = in.readStringList(); + this.indicesRoutingDiffPath = in.readString(); } @Override @@ -237,6 +244,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.value(index); } builder.endArray(); + if (indicesRoutingDiffPath != null) { + builder.field(DIFF_FIELD, indicesRoutingDiffPath); + } builder.endObject(); builder.startObject(CLUSTER_STATE_CUSTOM_DIFF_FIELD); builder.startArray(UPSERTS_FIELD); @@ -253,7 +263,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws return builder; } - public static ClusterStateDiffManifest fromXContent(XContentParser parser) throws IOException { + public static ClusterStateDiffManifest fromXContent(XContentParser parser, long codec_version) throws IOException { Builder builder = new Builder(); if (parser.currentToken() == null) { // fresh parser? move to next token parser.nextToken(); @@ -341,6 +351,11 @@ public static ClusterStateDiffManifest fromXContent(XContentParser parser) throw case DELETES_FIELD: builder.indicesRoutingDeleted(convertListToString(parser.listOrderedMap())); break; + case DIFF_FIELD: + if (codec_version >= CODEC_V3) { + builder.indicesRoutingDiffPath(parser.textOrNull()); + } + break; default: throw new XContentParseException("Unexpected field [" + currentFieldName + "]"); } @@ -456,6 +471,10 @@ public List getIndicesRoutingUpdated() { return indicesRoutingUpdated; } + public String getIndicesRoutingDiffPath() { + return indicesRoutingDiffPath; + } + public List getIndicesRoutingDeleted() { return indicesRoutingDeleted; } @@ -468,6 +487,10 @@ public List getClusterStateCustomDeleted() { return clusterStateCustomDeleted; } + public void setIndicesRoutingDiffPath(String indicesRoutingDiffPath) { + this.indicesRoutingDiffPath = indicesRoutingDiffPath; + } + @Override public boolean equals(Object o) { if (this == o) return true; @@ -489,7 +512,8 @@ public boolean equals(Object o) { && Objects.equals(indicesRoutingUpdated, that.indicesRoutingUpdated) && Objects.equals(indicesRoutingDeleted, that.indicesRoutingDeleted) && Objects.equals(clusterStateCustomUpdated, that.clusterStateCustomUpdated) - && Objects.equals(clusterStateCustomDeleted, that.clusterStateCustomDeleted); + && Objects.equals(clusterStateCustomDeleted, that.clusterStateCustomDeleted) + && Objects.equals(indicesRoutingDiffPath, that.indicesRoutingDiffPath); } @Override @@ -538,6 +562,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeBoolean(hashesOfConsistentSettingsUpdated); out.writeStringCollection(clusterStateCustomUpdated); out.writeStringCollection(clusterStateCustomDeleted); + out.writeString(indicesRoutingDiffPath); } /** @@ -560,6 +585,7 @@ public static class Builder { private boolean discoveryNodesUpdated; private List indicesRoutingUpdated; private List indicesRoutingDeleted; + private String indicesRoutingDiff; private boolean hashesOfConsistentSettingsUpdated; private List clusterStateCustomUpdated; private List clusterStateCustomDeleted; @@ -650,6 +676,11 @@ public Builder indicesRoutingDeleted(List indicesRoutingDeleted) { return this; } + public Builder indicesRoutingDiffPath(String indicesRoutingDiffPath) { + this.indicesRoutingDiff = indicesRoutingDiffPath; + return this; + } + public Builder clusterStateCustomUpdated(List clusterStateCustomUpdated) { this.clusterStateCustomUpdated = clusterStateCustomUpdated; return this; @@ -676,6 +707,7 @@ public ClusterStateDiffManifest build() { discoveryNodesUpdated, indicesRoutingUpdated, indicesRoutingDeleted, + indicesRoutingDiff, hashesOfConsistentSettingsUpdated, clusterStateCustomUpdated, clusterStateCustomDeleted diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManager.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManager.java index 99235bc96bfe3..8691187c7fbfa 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManager.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManager.java @@ -179,6 +179,7 @@ void deleteClusterMetadata( Set staleGlobalMetadataPaths = new HashSet<>(); Set staleEphemeralAttributePaths = new HashSet<>(); Set staleIndexRoutingPaths = new HashSet<>(); + Set staleIndexRoutingDiffPaths = new HashSet<>(); activeManifestBlobMetadata.forEach(blobMetadata -> { ClusterMetadataManifest clusterMetadataManifest = remoteManifestManager.fetchRemoteClusterMetadataManifest( clusterName, @@ -222,6 +223,10 @@ void deleteClusterMetadata( clusterMetadataManifest.getIndicesRouting() .forEach(uploadedIndicesRouting -> filesToKeep.add(uploadedIndicesRouting.getUploadedFilename())); } + if (clusterMetadataManifest.getDiffManifest() != null + && clusterMetadataManifest.getDiffManifest().getIndicesRoutingDiffPath() != null) { + filesToKeep.add(clusterMetadataManifest.getDiffManifest().getIndicesRoutingDiffPath()); + } }); staleManifestBlobMetadata.forEach(blobMetadata -> { ClusterMetadataManifest clusterMetadataManifest = remoteManifestManager.fetchRemoteClusterMetadataManifest( @@ -264,6 +269,18 @@ void deleteClusterMetadata( } }); } + if (clusterMetadataManifest.getDiffManifest() != null + && clusterMetadataManifest.getDiffManifest().getIndicesRoutingDiffPath() != null) { + if (!filesToKeep.contains(clusterMetadataManifest.getDiffManifest().getIndicesRoutingDiffPath())) { + staleIndexRoutingDiffPaths.add(clusterMetadataManifest.getDiffManifest().getIndicesRoutingDiffPath()); + logger.debug( + () -> new ParameterizedMessage( + "Indices routing diff paths in stale manifest: {}", + clusterMetadataManifest.getDiffManifest().getIndicesRoutingDiffPath() + ) + ); + } + } clusterMetadataManifest.getIndices().forEach(uploadedIndexMetadata -> { String fileName = RemoteClusterStateUtils.getFormattedIndexFileName(uploadedIndexMetadata.getUploadedFilename()); @@ -316,6 +333,15 @@ void deleteClusterMetadata( ); remoteStateStats.indexRoutingFilesCleanupAttemptFailed(); } + try { + remoteRoutingTableService.deleteStaleIndexRoutingDiffPaths(new ArrayList<>(staleIndexRoutingDiffPaths)); + } catch (IOException e) { + logger.error( + () -> new ParameterizedMessage("Error while deleting stale index routing diff files {}", staleIndexRoutingDiffPaths), + e + ); + remoteStateStats.indicesRoutingDiffFileCleanupAttemptFailed(); + } } catch (IllegalStateException e) { logger.error("Error while fetching Remote Cluster Metadata manifests", e); } catch (IOException e) { 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 b34641f77f607..674279f2251bd 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateService.java @@ -14,6 +14,7 @@ import org.opensearch.action.LatchedActionListener; import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.Diff; import org.opensearch.cluster.DiffableUtils; import org.opensearch.cluster.block.ClusterBlocks; import org.opensearch.cluster.coordination.CoordinationMetadata; @@ -26,6 +27,7 @@ import org.opensearch.cluster.node.DiscoveryNodes.Builder; import org.opensearch.cluster.routing.IndexRoutingTable; import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.routing.RoutingTableIncrementalDiff; import org.opensearch.cluster.routing.remote.RemoteRoutingTableService; import org.opensearch.cluster.routing.remote.RemoteRoutingTableServiceFactory; import org.opensearch.cluster.service.ClusterService; @@ -56,6 +58,7 @@ import org.opensearch.gateway.remote.model.RemoteReadResult; import org.opensearch.gateway.remote.model.RemoteTemplatesMetadata; import org.opensearch.gateway.remote.model.RemoteTransientSettingsMetadata; +import org.opensearch.gateway.remote.routingtable.RemoteRoutingTableDiff; import org.opensearch.index.translog.transfer.BlobStoreTransferService; import org.opensearch.node.Node; import org.opensearch.node.remotestore.RemoteStoreNodeAttribute; @@ -234,13 +237,21 @@ public RemoteClusterStateManifestInfo writeFullMetadata(ClusterState clusterStat isPublicationEnabled, isPublicationEnabled ? clusterState.customs() : Collections.emptyMap(), isPublicationEnabled, - remoteRoutingTableService.getIndicesRouting(clusterState.getRoutingTable()) + remoteRoutingTableService.getIndicesRouting(clusterState.getRoutingTable()), + null + ); + + ClusterStateDiffManifest clusterStateDiffManifest = new ClusterStateDiffManifest( + clusterState, + ClusterState.EMPTY_STATE, + null, + null ); final RemoteClusterStateManifestInfo manifestDetails = remoteManifestManager.uploadManifest( clusterState, uploadedMetadataResults, previousClusterUUID, - new ClusterStateDiffManifest(clusterState, ClusterState.EMPTY_STATE), + clusterStateDiffManifest, false ); @@ -330,10 +341,13 @@ public RemoteClusterStateManifestInfo writeIncrementalMetadata( indicesToBeDeletedFromRemote.remove(indexMetadata.getIndex().getName()); } - final DiffableUtils.MapDiff> routingTableDiff = remoteRoutingTableService - .getIndicesRoutingMapDiff(previousClusterState.getRoutingTable(), clusterState.getRoutingTable()); final List indicesRoutingToUpload = new ArrayList<>(); - routingTableDiff.getUpserts().forEach((k, v) -> indicesRoutingToUpload.add(v)); + final DiffableUtils.MapDiff> routingTableIncrementalDiff = + remoteRoutingTableService.getIndicesRoutingMapDiff(previousClusterState.getRoutingTable(), clusterState.getRoutingTable()); + + Map> indexRoutingTableDiffs = routingTableIncrementalDiff.getDiffs(); + routingTableIncrementalDiff.getDiffs().forEach((k, v) -> indicesRoutingToUpload.add(clusterState.getRoutingTable().index(k))); + routingTableIncrementalDiff.getUpserts().forEach((k, v) -> indicesRoutingToUpload.add(v)); UploadedMetadataResults uploadedMetadataResults; // For migration case from codec V0 or V1 to V2, we have added null check on metadata attribute files, @@ -369,7 +383,8 @@ public RemoteClusterStateManifestInfo writeIncrementalMetadata( updateTransientSettingsMetadata, clusterStateCustomsDiff.getUpserts(), updateHashesOfConsistentSettings, - indicesRoutingToUpload + indicesRoutingToUpload, + indexRoutingTableDiffs ); // update the map if the metadata was uploaded @@ -411,14 +426,23 @@ public RemoteClusterStateManifestInfo writeIncrementalMetadata( uploadedMetadataResults.uploadedIndicesRoutingMetadata = remoteRoutingTableService.getAllUploadedIndicesRouting( previousManifest, uploadedMetadataResults.uploadedIndicesRoutingMetadata, - routingTableDiff.getDeletes() + routingTableIncrementalDiff.getDeletes() + ); + + ClusterStateDiffManifest clusterStateDiffManifest = new ClusterStateDiffManifest( + clusterState, + previousClusterState, + routingTableIncrementalDiff, + uploadedMetadataResults.uploadedIndicesRoutingDiffMetadata != null + ? uploadedMetadataResults.uploadedIndicesRoutingDiffMetadata.getUploadedFilename() + : null ); final RemoteClusterStateManifestInfo manifestDetails = remoteManifestManager.uploadManifest( clusterState, uploadedMetadataResults, previousManifest.getPreviousClusterUUID(), - new ClusterStateDiffManifest(clusterState, previousClusterState), + clusterStateDiffManifest, false ); @@ -488,13 +512,15 @@ UploadedMetadataResults writeMetadataInParallel( boolean uploadTransientSettingMetadata, Map clusterStateCustomToUpload, boolean uploadHashesOfConsistentSettings, - List indicesRoutingToUpload + List indicesRoutingToUpload, + Map> indexRoutingTableDiff ) throws IOException { assert Objects.nonNull(indexMetadataUploadListeners) : "indexMetadataUploadListeners can not be null"; int totalUploadTasks = indexToUpload.size() + indexMetadataUploadListeners.size() + customToUpload.size() + (uploadCoordinationMetadata ? 1 : 0) + (uploadSettingsMetadata ? 1 : 0) + (uploadTemplateMetadata ? 1 : 0) + (uploadDiscoveryNodes ? 1 : 0) + (uploadClusterBlock ? 1 : 0) + (uploadTransientSettingMetadata ? 1 : 0) - + clusterStateCustomToUpload.size() + (uploadHashesOfConsistentSettings ? 1 : 0) + indicesRoutingToUpload.size(); + + clusterStateCustomToUpload.size() + (uploadHashesOfConsistentSettings ? 1 : 0) + indicesRoutingToUpload.size() + + (indexRoutingTableDiff != null && !indexRoutingTableDiff.isEmpty() ? 1 : 0); CountDownLatch latch = new CountDownLatch(totalUploadTasks); List uploadTasks = Collections.synchronizedList(new ArrayList<>(totalUploadTasks)); Map results = new ConcurrentHashMap<>(totalUploadTasks); @@ -664,6 +690,16 @@ UploadedMetadataResults writeMetadataInParallel( listener ); }); + if (indexRoutingTableDiff != null && !indexRoutingTableDiff.isEmpty()) { + uploadTasks.add(RemoteRoutingTableDiff.ROUTING_TABLE_DIFF_FILE); + remoteRoutingTableService.getAsyncIndexRoutingDiffWriteAction( + clusterState.metadata().clusterUUID(), + clusterState.term(), + clusterState.version(), + indexRoutingTableDiff, + listener + ); + } invokeIndexMetadataUploadListeners(indexToUpload, prevIndexMetadataByName, latch, exceptionList); try { @@ -710,6 +746,8 @@ UploadedMetadataResults writeMetadataInParallel( if (uploadedMetadata.getClass().equals(UploadedIndexMetadata.class) && uploadedMetadata.getComponent().contains(INDEX_ROUTING_METADATA_PREFIX)) { response.uploadedIndicesRoutingMetadata.add((UploadedIndexMetadata) uploadedMetadata); + } else if (RemoteRoutingTableDiff.ROUTING_TABLE_DIFF_FILE.equals(name)) { + response.uploadedIndicesRoutingDiffMetadata = (UploadedMetadataAttribute) uploadedMetadata; } else if (name.startsWith(CUSTOM_METADATA)) { // component name for custom metadata will look like custom-- String custom = name.split(DELIMITER)[0].split(CUSTOM_DELIMITER)[1]; @@ -979,16 +1017,18 @@ ClusterState readClusterStateInParallel( List indicesRoutingToRead, boolean readHashesOfConsistentSettings, Map clusterStateCustomToRead, + boolean readIndexRoutingTableDiff, boolean includeEphemeral ) throws IOException { int totalReadTasks = indicesToRead.size() + customToRead.size() + (readCoordinationMetadata ? 1 : 0) + (readSettingsMetadata ? 1 : 0) + (readTemplatesMetadata ? 1 : 0) + (readDiscoveryNodes ? 1 : 0) + (readClusterBlocks ? 1 : 0) + (readTransientSettingsMetadata ? 1 : 0) + (readHashesOfConsistentSettings ? 1 : 0) + clusterStateCustomToRead.size() - + indicesRoutingToRead.size(); + + indicesRoutingToRead.size() + (readIndexRoutingTableDiff ? 1 : 0); CountDownLatch latch = new CountDownLatch(totalReadTasks); List readResults = Collections.synchronizedList(new ArrayList<>()); List readIndexRoutingTableResults = Collections.synchronizedList(new ArrayList<>()); + AtomicReference readIndexRoutingTableDiffResults = new AtomicReference<>(); List exceptionList = Collections.synchronizedList(new ArrayList<>(totalReadTasks)); LatchedActionListener listener = new LatchedActionListener<>(ActionListener.wrap(response -> { @@ -1031,6 +1071,25 @@ ClusterState readClusterStateInParallel( ); } + LatchedActionListener routingTableDiffLatchedActionListener = new LatchedActionListener<>( + ActionListener.wrap(response -> { + logger.debug("Successfully read routing table diff component from remote"); + readIndexRoutingTableDiffResults.set(response); + }, ex -> { + logger.error("Failed to read routing table diff from remote", ex); + exceptionList.add(ex); + }), + latch + ); + + if (readIndexRoutingTableDiff) { + remoteRoutingTableService.getAsyncIndexRoutingTableDiffReadAction( + clusterUUID, + manifest.getDiffManifest().getIndicesRoutingDiffPath(), + routingTableDiffLatchedActionListener + ); + } + for (Map.Entry entry : customToRead.entrySet()) { remoteGlobalMetadataManager.readAsync( entry.getValue().getAttributeName(), @@ -1233,6 +1292,14 @@ ClusterState readClusterStateInParallel( readIndexRoutingTableResults.forEach( indexRoutingTable -> indicesRouting.put(indexRoutingTable.getIndex().getName(), indexRoutingTable) ); + RoutingTableIncrementalDiff routingTableDiff = readIndexRoutingTableDiffResults.get(); + if (routingTableDiff != null) { + routingTableDiff.getDiffs().forEach((key, diff) -> { + IndexRoutingTable previousIndexRoutingTable = indicesRouting.get(key); + IndexRoutingTable updatedTable = diff.apply(previousIndexRoutingTable); + indicesRouting.put(key, updatedTable); + }); + } clusterStateBuilder.routingTable(new RoutingTable(manifest.getRoutingTableVersion(), indicesRouting)); return clusterStateBuilder.build(); @@ -1261,6 +1328,7 @@ public ClusterState getClusterStateForManifest( includeEphemeral ? manifest.getIndicesRouting() : emptyList(), includeEphemeral && manifest.getHashesOfConsistentSettings() != null, includeEphemeral ? manifest.getClusterStateCustomMap() : emptyMap(), + false, includeEphemeral ); } else { @@ -1281,6 +1349,7 @@ public ClusterState getClusterStateForManifest( emptyList(), false, emptyMap(), + false, false ); Metadata.Builder mb = Metadata.builder(remoteGlobalMetadataManager.getGlobalMetadata(manifest.getClusterUUID(), manifest)); @@ -1337,6 +1406,9 @@ public ClusterState getClusterStateUsingDiff(ClusterMetadataManifest manifest, C updatedIndexRouting, diff.isHashesOfConsistentSettingsUpdated(), updatedClusterStateCustom, + manifest.getDiffManifest() != null + && manifest.getDiffManifest().getIndicesRoutingDiffPath() != null + && !manifest.getDiffManifest().getIndicesRoutingDiffPath().isEmpty(), true ); ClusterState.Builder clusterStateBuilder = ClusterState.builder(updatedClusterState); diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateUtils.java b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateUtils.java index f2b93c3784407..74cb838286961 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateUtils.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemoteClusterStateUtils.java @@ -88,6 +88,7 @@ public static class UploadedMetadataResults { ClusterMetadataManifest.UploadedMetadataAttribute uploadedClusterBlocks; List uploadedIndicesRoutingMetadata; ClusterMetadataManifest.UploadedMetadataAttribute uploadedHashesOfConsistentSettings; + ClusterMetadataManifest.UploadedMetadataAttribute uploadedIndicesRoutingDiffMetadata; public UploadedMetadataResults( List uploadedIndexMetadata, diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemotePersistenceStats.java b/server/src/main/java/org/opensearch/gateway/remote/RemotePersistenceStats.java index 36d107a99d258..efd73e11e46b5 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemotePersistenceStats.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemotePersistenceStats.java @@ -20,15 +20,18 @@ public class RemotePersistenceStats extends PersistedStateStats { static final String CLEANUP_ATTEMPT_FAILED_COUNT = "cleanup_attempt_failed_count"; static final String INDEX_ROUTING_FILES_CLEANUP_ATTEMPT_FAILED_COUNT = "index_routing_files_cleanup_attempt_failed_count"; + static final String INDICES_ROUTING_DIFF_FILES_CLEANUP_ATTEMPT_FAILED_COUNT = "indices_routing_diff_files_cleanup_attempt_failed_count"; static final String REMOTE_UPLOAD = "remote_upload"; private AtomicLong cleanupAttemptFailedCount = new AtomicLong(0); private AtomicLong indexRoutingFilesCleanupAttemptFailedCount = new AtomicLong(0); + private AtomicLong indicesRoutingDiffFilesCleanupAttemptFailedCount = new AtomicLong(0); public RemotePersistenceStats() { super(REMOTE_UPLOAD); addToExtendedFields(CLEANUP_ATTEMPT_FAILED_COUNT, cleanupAttemptFailedCount); addToExtendedFields(INDEX_ROUTING_FILES_CLEANUP_ATTEMPT_FAILED_COUNT, indexRoutingFilesCleanupAttemptFailedCount); + addToExtendedFields(INDICES_ROUTING_DIFF_FILES_CLEANUP_ATTEMPT_FAILED_COUNT, indicesRoutingDiffFilesCleanupAttemptFailedCount); } public void cleanUpAttemptFailed() { @@ -46,4 +49,12 @@ public void indexRoutingFilesCleanupAttemptFailed() { public long getIndexRoutingFilesCleanupAttemptFailedCount() { return indexRoutingFilesCleanupAttemptFailedCount.get(); } + + public void indicesRoutingDiffFileCleanupAttemptFailed() { + indexRoutingFilesCleanupAttemptFailedCount.incrementAndGet(); + } + + public long getIndicesRoutingDiffFileCleanupAttemptFailedCount() { + return indexRoutingFilesCleanupAttemptFailedCount.get(); + } } diff --git a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterMetadataManifest.java b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterMetadataManifest.java index 1dc56712d4ab5..acaae3173315a 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterMetadataManifest.java +++ b/server/src/main/java/org/opensearch/gateway/remote/model/RemoteClusterMetadataManifest.java @@ -35,7 +35,7 @@ public class RemoteClusterMetadataManifest extends AbstractRemoteWritableBlobEnt public static final int SPLITTED_MANIFEST_FILE_LENGTH = 6; public static final String METADATA_MANIFEST_NAME_FORMAT = "%s"; - public static final int MANIFEST_CURRENT_CODEC_VERSION = ClusterMetadataManifest.CODEC_V2; + public static final int MANIFEST_CURRENT_CODEC_VERSION = ClusterMetadataManifest.CODEC_V3; public static final String COMMITTED = "C"; public static final String PUBLISHED = "P"; @@ -50,6 +50,9 @@ public class RemoteClusterMetadataManifest extends AbstractRemoteWritableBlobEnt public static final ChecksumBlobStoreFormat CLUSTER_METADATA_MANIFEST_FORMAT_V1 = new ChecksumBlobStoreFormat<>("cluster-metadata-manifest", METADATA_MANIFEST_NAME_FORMAT, ClusterMetadataManifest::fromXContentV1); + public static final ChecksumBlobStoreFormat CLUSTER_METADATA_MANIFEST_FORMAT_V2 = + new ChecksumBlobStoreFormat<>("cluster-metadata-manifest", METADATA_MANIFEST_NAME_FORMAT, ClusterMetadataManifest::fromXContentV2); + /** * Manifest format compatible with codec v2, where we introduced codec versions/global metadata. */ @@ -149,6 +152,8 @@ private ChecksumBlobStoreFormat getClusterMetadataManif long codecVersion = getManifestCodecVersion(); if (codecVersion == MANIFEST_CURRENT_CODEC_VERSION) { return CLUSTER_METADATA_MANIFEST_FORMAT; + } else if (codecVersion == ClusterMetadataManifest.CODEC_V2) { + return CLUSTER_METADATA_MANIFEST_FORMAT_V2; } else if (codecVersion == ClusterMetadataManifest.CODEC_V1) { return CLUSTER_METADATA_MANIFEST_FORMAT_V1; } else if (codecVersion == ClusterMetadataManifest.CODEC_V0) { diff --git a/server/src/main/java/org/opensearch/gateway/remote/routingtable/RemoteRoutingTableDiff.java b/server/src/main/java/org/opensearch/gateway/remote/routingtable/RemoteRoutingTableDiff.java new file mode 100644 index 0000000000000..e876d939490d0 --- /dev/null +++ b/server/src/main/java/org/opensearch/gateway/remote/routingtable/RemoteRoutingTableDiff.java @@ -0,0 +1,150 @@ +/* + * 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.routingtable; + +import org.opensearch.cluster.Diff; +import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.RoutingTableIncrementalDiff; +import org.opensearch.common.io.Streams; +import org.opensearch.common.remote.AbstractRemoteWritableBlobEntity; +import org.opensearch.common.remote.BlobPathParameters; +import org.opensearch.core.compress.Compressor; +import org.opensearch.gateway.remote.ClusterMetadataManifest; +import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.repositories.blobstore.ChecksumWritableBlobStoreFormat; + +import java.io.IOException; +import java.io.InputStream; +import java.util.List; +import java.util.Map; + +import static org.opensearch.gateway.remote.RemoteClusterStateUtils.DELIMITER; + +/** + * Represents a incremental difference between {@link org.opensearch.cluster.routing.RoutingTable} objects that can be serialized and deserialized. + * This class is responsible for writing and reading the differences between RoutingTables to and from an input/output stream. + */ +public class RemoteRoutingTableDiff extends AbstractRemoteWritableBlobEntity { + private final RoutingTableIncrementalDiff routingTableIncrementalDiff; + + private long term; + private long version; + + public static final String ROUTING_TABLE_DIFF = "routing-table-diff"; + + public static final String ROUTING_TABLE_DIFF_METADATA_PREFIX = "routingTableDiff--"; + + public static final String ROUTING_TABLE_DIFF_FILE = "routing_table_diff"; + private static final String codec = "RemoteRoutingTableDiff"; + public static final String ROUTING_TABLE_DIFF_PATH_TOKEN = "routing-table-diff"; + + public static final int VERSION = 1; + + public static final ChecksumWritableBlobStoreFormat REMOTE_ROUTING_TABLE_DIFF_FORMAT = + new ChecksumWritableBlobStoreFormat<>(codec, RoutingTableIncrementalDiff::readFrom); + + /** + * Constructs a new RemoteRoutingTableDiff with the given differences. + * + * @param routingTableIncrementalDiff a RoutingTableIncrementalDiff object containing the differences of {@link IndexRoutingTable}. + * @param clusterUUID the cluster UUID. + * @param compressor the compressor to be used. + * @param term the term of the routing table. + * @param version the version of the routing table. + */ + public RemoteRoutingTableDiff( + RoutingTableIncrementalDiff routingTableIncrementalDiff, + String clusterUUID, + Compressor compressor, + long term, + long version + ) { + super(clusterUUID, compressor); + this.routingTableIncrementalDiff = routingTableIncrementalDiff; + this.term = term; + this.version = version; + } + + /** + * Constructs a new RemoteRoutingTableDiff with the given differences. + * + * @param routingTableIncrementalDiff a RoutingTableIncrementalDiff object containing the differences of {@link IndexRoutingTable}. + * @param clusterUUID the cluster UUID. + * @param compressor the compressor to be used. + */ + public RemoteRoutingTableDiff(RoutingTableIncrementalDiff routingTableIncrementalDiff, String clusterUUID, Compressor compressor) { + super(clusterUUID, compressor); + this.routingTableIncrementalDiff = routingTableIncrementalDiff; + } + + /** + * Constructs a new RemoteIndexRoutingTableDiff with the given blob name, cluster UUID, and compressor. + * + * @param blobName the name of the blob. + * @param clusterUUID the cluster UUID. + * @param compressor the compressor to be used. + */ + public RemoteRoutingTableDiff(String blobName, String clusterUUID, Compressor compressor) { + super(clusterUUID, compressor); + this.routingTableIncrementalDiff = null; + this.blobName = blobName; + } + + /** + * Gets the map of differences of {@link IndexRoutingTable}. + * + * @return a map containing the differences. + */ + public Map> getDiffs() { + assert routingTableIncrementalDiff != null; + return routingTableIncrementalDiff.getDiffs(); + } + + @Override + public BlobPathParameters getBlobPathParameters() { + return new BlobPathParameters(List.of(ROUTING_TABLE_DIFF_PATH_TOKEN), ROUTING_TABLE_DIFF_METADATA_PREFIX); + } + + @Override + public String getType() { + return ROUTING_TABLE_DIFF; + } + + @Override + public String generateBlobFileName() { + if (blobFileName == null) { + blobFileName = String.join( + DELIMITER, + getBlobPathParameters().getFilePrefix(), + RemoteStoreUtils.invertLong(term), + RemoteStoreUtils.invertLong(version), + RemoteStoreUtils.invertLong(System.currentTimeMillis()) + ); + } + return blobFileName; + } + + @Override + public ClusterMetadataManifest.UploadedMetadata getUploadedMetadata() { + assert blobName != null; + return new ClusterMetadataManifest.UploadedMetadataAttribute(ROUTING_TABLE_DIFF_FILE, blobName); + } + + @Override + public InputStream serialize() throws IOException { + assert routingTableIncrementalDiff != null; + return REMOTE_ROUTING_TABLE_DIFF_FORMAT.serialize(routingTableIncrementalDiff, generateBlobFileName(), getCompressor()) + .streamInput(); + } + + @Override + public RoutingTableIncrementalDiff deserialize(InputStream in) throws IOException { + return REMOTE_ROUTING_TABLE_DIFF_FORMAT.deserialize(blobName, Streams.readFully(in)); + } +} diff --git a/server/src/test/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableServiceTests.java b/server/src/test/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableServiceTests.java index f66e096e9b548..74254f1a1987f 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableServiceTests.java @@ -12,12 +12,15 @@ import org.opensearch.action.LatchedActionListener; import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.Diff; import org.opensearch.cluster.DiffableUtils; import org.opensearch.cluster.coordination.CoordinationMetadata; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.IndexShardRoutingTable; import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.routing.RoutingTableIncrementalDiff; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.common.blobstore.BlobPath; @@ -50,8 +53,11 @@ import java.io.IOException; import java.io.InputStream; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; +import java.util.Base64; +import java.util.HashMap; import java.util.List; import java.util.Locale; import java.util.Map; @@ -69,6 +75,10 @@ import static org.opensearch.gateway.remote.routingtable.RemoteIndexRoutingTable.INDEX_ROUTING_METADATA_PREFIX; import static org.opensearch.gateway.remote.routingtable.RemoteIndexRoutingTable.INDEX_ROUTING_TABLE; import static org.opensearch.gateway.remote.routingtable.RemoteIndexRoutingTable.INDEX_ROUTING_TABLE_FORMAT; +import static org.opensearch.gateway.remote.routingtable.RemoteRoutingTableDiff.REMOTE_ROUTING_TABLE_DIFF_FORMAT; +import static org.opensearch.gateway.remote.routingtable.RemoteRoutingTableDiff.ROUTING_TABLE_DIFF_FILE; +import static org.opensearch.gateway.remote.routingtable.RemoteRoutingTableDiff.ROUTING_TABLE_DIFF_METADATA_PREFIX; +import static org.opensearch.gateway.remote.routingtable.RemoteRoutingTableDiff.ROUTING_TABLE_DIFF_PATH_TOKEN; import static org.opensearch.index.remote.RemoteStoreEnums.PathHashAlgorithm.FNV_1A_BASE64; import static org.opensearch.index.remote.RemoteStoreEnums.PathType.HASHED_PREFIX; import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY; @@ -281,10 +291,14 @@ public void testGetIndicesRoutingMapDiffShardChanged() { DiffableUtils.MapDiff> diff = remoteRoutingTableService .getIndicesRoutingMapDiff(routingTable, routingTable2); - assertEquals(1, diff.getUpserts().size()); - assertNotNull(diff.getUpserts().get(indexName)); - assertEquals(noOfShards + 1, diff.getUpserts().get(indexName).getShards().size()); - assertEquals(noOfReplicas + 1, diff.getUpserts().get(indexName).getShards().get(0).getSize()); + assertEquals(0, diff.getUpserts().size()); + assertEquals(1, diff.getDiffs().size()); + assertNotNull(diff.getDiffs().get(indexName)); + assertEquals(noOfShards + 1, diff.getDiffs().get(indexName).apply(routingTable.indicesRouting().get(indexName)).shards().size()); + assertEquals( + noOfReplicas + 1, + diff.getDiffs().get(indexName).apply(routingTable.indicesRouting().get(indexName)).getShards().get(0).getSize() + ); assertEquals(0, diff.getDeletes().size()); final IndexMetadata indexMetadata3 = new IndexMetadata.Builder(indexName).settings( @@ -296,11 +310,14 @@ public void testGetIndicesRoutingMapDiffShardChanged() { RoutingTable routingTable3 = RoutingTable.builder().addAsNew(indexMetadata3).build(); diff = remoteRoutingTableService.getIndicesRoutingMapDiff(routingTable2, routingTable3); - assertEquals(1, diff.getUpserts().size()); - assertNotNull(diff.getUpserts().get(indexName)); - assertEquals(noOfShards + 1, diff.getUpserts().get(indexName).getShards().size()); - assertEquals(noOfReplicas + 2, diff.getUpserts().get(indexName).getShards().get(0).getSize()); - + assertEquals(0, diff.getUpserts().size()); + assertEquals(1, diff.getDiffs().size()); + assertNotNull(diff.getDiffs().get(indexName)); + assertEquals(noOfShards + 1, diff.getDiffs().get(indexName).apply(routingTable.indicesRouting().get(indexName)).shards().size()); + assertEquals( + noOfReplicas + 2, + diff.getDiffs().get(indexName).apply(routingTable.indicesRouting().get(indexName)).getShards().get(0).getSize() + ); assertEquals(0, diff.getDeletes().size()); } @@ -320,10 +337,10 @@ public void testGetIndicesRoutingMapDiffShardDetailChanged() { DiffableUtils.MapDiff> diff = remoteRoutingTableService .getIndicesRoutingMapDiff(routingTable, routingTable2); - assertEquals(1, diff.getUpserts().size()); - assertNotNull(diff.getUpserts().get(indexName)); - assertEquals(noOfShards, diff.getUpserts().get(indexName).getShards().size()); - assertEquals(noOfReplicas + 1, diff.getUpserts().get(indexName).getShards().get(0).getSize()); + assertEquals(1, diff.getDiffs().size()); + assertNotNull(diff.getDiffs().get(indexName)); + assertEquals(noOfShards, diff.getDiffs().get(indexName).apply(routingTable.indicesRouting().get(indexName)).shards().size()); + assertEquals(0, diff.getUpserts().size()); assertEquals(0, diff.getDeletes().size()); } @@ -552,6 +569,44 @@ public void testGetAsyncIndexRoutingReadAction() throws Exception { assertEquals(clusterState.getRoutingTable().getIndicesRouting().get(indexName), indexRoutingTable); } + public void testGetAsyncIndexRoutingTableDiffReadAction() throws Exception { + String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); + ClusterState currentState = createClusterState(indexName); + + // Get the IndexRoutingTable from the current state + IndexRoutingTable indexRoutingTable = currentState.routingTable().index(indexName); + Map shardRoutingTables = indexRoutingTable.getShards(); + + RoutingTableIncrementalDiff.IndexRoutingTableIncrementalDiff indexRoutingTableDiff = + new RoutingTableIncrementalDiff.IndexRoutingTableIncrementalDiff(new ArrayList<>(shardRoutingTables.values())); + + // Create the map for RoutingTableIncrementalDiff + Map> diffs = new HashMap<>(); + diffs.put(indexName, indexRoutingTableDiff); + + RoutingTableIncrementalDiff diff = new RoutingTableIncrementalDiff(diffs); + + String uploadedFileName = String.format(Locale.ROOT, "routing-table-diff/" + indexName); + when(blobContainer.readBlob(indexName)).thenReturn( + REMOTE_ROUTING_TABLE_DIFF_FORMAT.serialize(diff, uploadedFileName, compressor).streamInput() + ); + + TestCapturingListener listener = new TestCapturingListener<>(); + CountDownLatch latch = new CountDownLatch(1); + + remoteRoutingTableService.getAsyncIndexRoutingTableDiffReadAction( + "cluster-uuid", + uploadedFileName, + new LatchedActionListener<>(listener, latch) + ); + latch.await(); + + assertNull(listener.getFailure()); + assertNotNull(listener.getResult()); + RoutingTableIncrementalDiff resultDiff = listener.getResult(); + assertEquals(diff.getDiffs().size(), resultDiff.getDiffs().size()); + } + public void testGetAsyncIndexRoutingWriteAction() throws Exception { String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); ClusterState clusterState = createClusterState(indexName); @@ -604,6 +659,68 @@ public void testGetAsyncIndexRoutingWriteAction() throws Exception { assertThat(RemoteStoreUtils.invertLong(fileNameTokens[3]), lessThanOrEqualTo(System.currentTimeMillis())); } + public void testGetAsyncIndexRoutingDiffWriteAction() throws Exception { + String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); + ClusterState currentState = createClusterState(indexName); + + // Get the IndexRoutingTable from the current state + IndexRoutingTable indexRoutingTable = currentState.routingTable().index(indexName); + Map shardRoutingTables = indexRoutingTable.getShards(); + + RoutingTableIncrementalDiff.IndexRoutingTableIncrementalDiff indexRoutingTableDiff = + new RoutingTableIncrementalDiff.IndexRoutingTableIncrementalDiff(new ArrayList<>(shardRoutingTables.values())); + + // Create the map for RoutingTableIncrementalDiff + Map> diffs = new HashMap<>(); + diffs.put(indexName, indexRoutingTableDiff); + + // RoutingTableIncrementalDiff diff = new RoutingTableIncrementalDiff(diffs); + + Iterable remotePath = new BlobPath().add("base-path") + .add( + Base64.getUrlEncoder() + .withoutPadding() + .encodeToString(currentState.getClusterName().value().getBytes(StandardCharsets.UTF_8)) + ) + .add("cluster-state") + .add(currentState.metadata().clusterUUID()) + .add(ROUTING_TABLE_DIFF_PATH_TOKEN); + + doAnswer(invocationOnMock -> { + invocationOnMock.getArgument(4, ActionListener.class).onResponse(null); + return null; + }).when(blobStoreTransferService) + .uploadBlob(any(InputStream.class), eq(remotePath), anyString(), eq(WritePriority.URGENT), any(ActionListener.class)); + + TestCapturingListener listener = new TestCapturingListener<>(); + CountDownLatch latch = new CountDownLatch(1); + + remoteRoutingTableService.getAsyncIndexRoutingDiffWriteAction( + currentState.metadata().clusterUUID(), + currentState.term(), + currentState.version(), + diffs, + new LatchedActionListener<>(listener, latch) + ); + latch.await(); + assertNull(listener.getFailure()); + assertNotNull(listener.getResult()); + ClusterMetadataManifest.UploadedMetadata uploadedMetadata = listener.getResult(); + + assertEquals(ROUTING_TABLE_DIFF_FILE, uploadedMetadata.getComponent()); + String uploadedFileName = uploadedMetadata.getUploadedFilename(); + String[] pathTokens = uploadedFileName.split(PATH_DELIMITER); + assertEquals(6, pathTokens.length); + assertEquals(pathTokens[0], "base-path"); + String[] fileNameTokens = pathTokens[5].split(DELIMITER); + + assertEquals(4, fileNameTokens.length); + assertEquals(ROUTING_TABLE_DIFF_METADATA_PREFIX, fileNameTokens[0]); + assertEquals(RemoteStoreUtils.invertLong(1L), fileNameTokens[1]); + assertEquals(RemoteStoreUtils.invertLong(2L), fileNameTokens[2]); + assertThat(RemoteStoreUtils.invertLong(fileNameTokens[3]), lessThanOrEqualTo(System.currentTimeMillis())); + } + public void testGetUpdatedIndexRoutingTableMetadataWhenNoChange() { List updatedIndicesRouting = new ArrayList<>(); List indicesRouting = randomUploadedIndexMetadataList(); @@ -687,4 +804,26 @@ public void testDeleteStaleIndexRoutingPathsThrowsIOException() throws IOExcepti verify(blobContainer).deleteBlobsIgnoringIfNotExists(stalePaths); } + public void testDeleteStaleIndexRoutingDiffPaths() throws IOException { + doNothing().when(blobContainer).deleteBlobsIgnoringIfNotExists(any()); + when(blobStore.blobContainer(any())).thenReturn(blobContainer); + List stalePaths = Arrays.asList("path1", "path2"); + remoteRoutingTableService.doStart(); + remoteRoutingTableService.deleteStaleIndexRoutingDiffPaths(stalePaths); + verify(blobContainer).deleteBlobsIgnoringIfNotExists(stalePaths); + } + + public void testDeleteStaleIndexRoutingDiffPathsThrowsIOException() throws IOException { + when(blobStore.blobContainer(any())).thenReturn(blobContainer); + List stalePaths = Arrays.asList("path1", "path2"); + // Simulate an IOException + doThrow(new IOException("test exception")).when(blobContainer).deleteBlobsIgnoringIfNotExists(Mockito.anyList()); + + remoteRoutingTableService.doStart(); + IOException thrown = assertThrows(IOException.class, () -> { + remoteRoutingTableService.deleteStaleIndexRoutingDiffPaths(stalePaths); + }); + assertEquals("test exception", thrown.getMessage()); + verify(blobContainer).deleteBlobsIgnoringIfNotExists(stalePaths); + } } diff --git a/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataManifestTests.java b/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataManifestTests.java index 256161af1a3e2..8a6dd6bc96e72 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataManifestTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/ClusterMetadataManifestTests.java @@ -10,9 +10,11 @@ import org.opensearch.Version; import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.DiffableUtils; import org.opensearch.cluster.metadata.IndexGraveyard; import org.opensearch.cluster.metadata.RepositoriesMetadata; import org.opensearch.cluster.metadata.WeightedRoutingMetadata; +import org.opensearch.cluster.routing.IndexRoutingTable; import org.opensearch.common.xcontent.json.JsonXContent; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; @@ -29,9 +31,12 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.function.Function; import java.util.stream.Collectors; +import org.mockito.Mockito; + import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V0; import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V1; import static org.opensearch.gateway.remote.RemoteClusterStateAttributesManager.CLUSTER_BLOCKS; @@ -157,7 +162,7 @@ public void testClusterMetadataManifestSerializationEqualsHashCode() { .opensearchVersion(Version.CURRENT) .nodeId("B10RX1f5RJenMQvYccCgSQ") .committed(true) - .codecVersion(ClusterMetadataManifest.CODEC_V2) + .codecVersion(ClusterMetadataManifest.CODEC_V3) .indices(randomUploadedIndexMetadataList()) .previousClusterUUID("yfObdx8KSMKKrXf8UyHhM") .clusterUUIDCommitted(true) @@ -191,7 +196,9 @@ public void testClusterMetadataManifestSerializationEqualsHashCode() { .diffManifest( new ClusterStateDiffManifest( RemoteClusterStateServiceTests.generateClusterStateWithOneIndex().build(), - ClusterState.EMPTY_STATE + ClusterState.EMPTY_STATE, + null, + "indicesRoutingDiffPath" ) ) .build(); @@ -523,7 +530,75 @@ public void testClusterMetadataManifestXContentV2() throws IOException { .diffManifest( new ClusterStateDiffManifest( RemoteClusterStateServiceTests.generateClusterStateWithOneIndex().build(), - ClusterState.EMPTY_STATE + ClusterState.EMPTY_STATE, + null, + null + ) + ) + .build(); + 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 testClusterMetadataManifestXContentV3() throws IOException { + UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "test-uuid", "/test/upload/path"); + UploadedMetadataAttribute uploadedMetadataAttribute = new UploadedMetadataAttribute("attribute_name", "testing_attribute"); + final DiffableUtils.MapDiff> routingTableIncrementalDiff = Mockito.mock( + DiffableUtils.MapDiff.class + ); + ClusterMetadataManifest originalManifest = ClusterMetadataManifest.builder() + .clusterTerm(1L) + .stateVersion(1L) + .clusterUUID("test-cluster-uuid") + .stateUUID("test-state-uuid") + .opensearchVersion(Version.CURRENT) + .nodeId("test-node-id") + .committed(false) + .codecVersion(ClusterMetadataManifest.CODEC_V3) + .indices(Collections.singletonList(uploadedIndexMetadata)) + .previousClusterUUID("prev-cluster-uuid") + .clusterUUIDCommitted(true) + .coordinationMetadata(uploadedMetadataAttribute) + .settingMetadata(uploadedMetadataAttribute) + .templatesMetadata(uploadedMetadataAttribute) + .customMetadataMap( + Collections.unmodifiableList( + Arrays.asList( + new UploadedMetadataAttribute( + CUSTOM_METADATA + CUSTOM_DELIMITER + RepositoriesMetadata.TYPE, + "custom--repositories-file" + ), + new UploadedMetadataAttribute( + CUSTOM_METADATA + CUSTOM_DELIMITER + IndexGraveyard.TYPE, + "custom--index_graveyard-file" + ), + new UploadedMetadataAttribute( + CUSTOM_METADATA + CUSTOM_DELIMITER + WeightedRoutingMetadata.TYPE, + "custom--weighted_routing_netadata-file" + ) + ) + ).stream().collect(Collectors.toMap(UploadedMetadataAttribute::getAttributeName, Function.identity())) + ) + .routingTableVersion(1L) + .indicesRouting(Collections.singletonList(uploadedIndexMetadata)) + .discoveryNodesMetadata(uploadedMetadataAttribute) + .clusterBlocksMetadata(uploadedMetadataAttribute) + .transientSettingsMetadata(uploadedMetadataAttribute) + .hashesOfConsistentSettings(uploadedMetadataAttribute) + .clusterStateCustomMetadataMap(Collections.emptyMap()) + .diffManifest( + new ClusterStateDiffManifest( + RemoteClusterStateServiceTests.generateClusterStateWithOneIndex().build(), + ClusterState.EMPTY_STATE, + routingTableIncrementalDiff, + uploadedMetadataAttribute.getUploadedFilename() ) ) .build(); diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerTests.java index ec7e3c1ce81d3..b86f23f3d37aa 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerTests.java @@ -50,6 +50,7 @@ import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V1; import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V2; +import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V3; import static org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedIndexMetadata; import static org.opensearch.gateway.remote.ClusterMetadataManifest.UploadedMetadataAttribute; import static org.opensearch.gateway.remote.RemoteClusterStateCleanupManager.AsyncStaleFileDeletion; @@ -296,6 +297,74 @@ public void testDeleteClusterMetadata() throws IOException { verify(remoteRoutingTableService).deleteStaleIndexRoutingPaths(List.of(index3Metadata.getUploadedFilename())); } + public void testDeleteStaleIndicesRoutingDiffFile() throws IOException { + String clusterUUID = "clusterUUID"; + String clusterName = "test-cluster"; + List inactiveBlobs = Arrays.asList(new PlainBlobMetadata("manifest1.dat", 1L)); + List activeBlobs = Arrays.asList(new PlainBlobMetadata("manifest2.dat", 1L)); + + UploadedMetadataAttribute coordinationMetadata = new UploadedMetadataAttribute(COORDINATION_METADATA, "coordination_metadata"); + UploadedMetadataAttribute templateMetadata = new UploadedMetadataAttribute(TEMPLATES_METADATA, "template_metadata"); + UploadedMetadataAttribute settingMetadata = new UploadedMetadataAttribute(SETTING_METADATA, "settings_metadata"); + UploadedMetadataAttribute coordinationMetadataUpdated = new UploadedMetadataAttribute( + COORDINATION_METADATA, + "coordination_metadata_updated" + ); + + UploadedIndexMetadata index1Metadata = new UploadedIndexMetadata("index1", "indexUUID1", "index_metadata1__2"); + UploadedIndexMetadata index2Metadata = new UploadedIndexMetadata("index2", "indexUUID2", "index_metadata2__2"); + List indicesRouting1 = List.of(index1Metadata); + List indicesRouting2 = List.of(index2Metadata); + ClusterStateDiffManifest diffManifest1 = ClusterStateDiffManifest.builder().indicesRoutingDiffPath("index1RoutingDiffPath").build(); + ClusterStateDiffManifest diffManifest2 = ClusterStateDiffManifest.builder().indicesRoutingDiffPath("index2RoutingDiffPath").build(); + + ClusterMetadataManifest manifest1 = ClusterMetadataManifest.builder() + .indices(List.of(index1Metadata)) + .coordinationMetadata(coordinationMetadataUpdated) + .templatesMetadata(templateMetadata) + .settingMetadata(settingMetadata) + .clusterTerm(1L) + .stateVersion(1L) + .codecVersion(CODEC_V3) + .stateUUID(randomAlphaOfLength(10)) + .clusterUUID(clusterUUID) + .nodeId("nodeA") + .opensearchVersion(VersionUtils.randomOpenSearchVersion(random())) + .previousClusterUUID(ClusterState.UNKNOWN_UUID) + .committed(true) + .routingTableVersion(0L) + .indicesRouting(indicesRouting1) + .diffManifest(diffManifest1) + .build(); + ClusterMetadataManifest manifest2 = ClusterMetadataManifest.builder(manifest1) + .indices(List.of(index2Metadata)) + .indicesRouting(indicesRouting2) + .diffManifest(diffManifest2) + .build(); + + BlobContainer blobContainer = mock(BlobContainer.class); + doThrow(IOException.class).when(blobContainer).delete(); + when(blobStore.blobContainer(any())).thenReturn(blobContainer); + BlobPath blobPath = new BlobPath().add("random-path"); + when((blobStoreRepository.basePath())).thenReturn(blobPath); + remoteClusterStateCleanupManager.start(); + when(remoteManifestManager.getManifestFolderPath(eq(clusterName), eq(clusterUUID))).thenReturn( + new BlobPath().add(encodeString(clusterName)).add(CLUSTER_STATE_PATH_TOKEN).add(clusterUUID).add(MANIFEST) + ); + when(remoteManifestManager.fetchRemoteClusterMetadataManifest(eq(clusterName), eq(clusterUUID), any())).thenReturn( + manifest2, + manifest1 + ); + remoteClusterStateCleanupManager = new RemoteClusterStateCleanupManager( + remoteClusterStateService, + clusterService, + remoteRoutingTableService + ); + remoteClusterStateCleanupManager.start(); + remoteClusterStateCleanupManager.deleteClusterMetadata(clusterName, clusterUUID, activeBlobs, inactiveBlobs); + verify(remoteRoutingTableService).deleteStaleIndexRoutingDiffPaths(List.of("index1RoutingDiffPath")); + } + public void testDeleteClusterMetadataNoOpsRoutingTableService() throws IOException { String clusterUUID = "clusterUUID"; String clusterName = "test-cluster"; @@ -515,6 +584,83 @@ public void testIndexRoutingFilesCleanupFailureStats() throws Exception { }); } + public void testIndicesRoutingDiffFilesCleanupFailureStats() throws Exception { + String clusterUUID = "clusterUUID"; + String clusterName = "test-cluster"; + List inactiveBlobs = Arrays.asList(new PlainBlobMetadata("manifest1.dat", 1L)); + List activeBlobs = Arrays.asList(new PlainBlobMetadata("manifest2.dat", 1L)); + + UploadedMetadataAttribute coordinationMetadata = new UploadedMetadataAttribute(COORDINATION_METADATA, "coordination_metadata"); + UploadedMetadataAttribute templateMetadata = new UploadedMetadataAttribute(TEMPLATES_METADATA, "template_metadata"); + UploadedMetadataAttribute settingMetadata = new UploadedMetadataAttribute(SETTING_METADATA, "settings_metadata"); + UploadedMetadataAttribute coordinationMetadataUpdated = new UploadedMetadataAttribute( + COORDINATION_METADATA, + "coordination_metadata_updated" + ); + + UploadedIndexMetadata index1Metadata = new UploadedIndexMetadata("index1", "indexUUID1", "index_metadata1__2"); + UploadedIndexMetadata index2Metadata = new UploadedIndexMetadata("index2", "indexUUID2", "index_metadata2__2"); + List indicesRouting1 = List.of(index1Metadata); + List indicesRouting2 = List.of(index2Metadata); + ClusterStateDiffManifest diffManifest1 = ClusterStateDiffManifest.builder().indicesRoutingDiffPath("index1RoutingDiffPath").build(); + ClusterStateDiffManifest diffManifest2 = ClusterStateDiffManifest.builder().indicesRoutingDiffPath("index2RoutingDiffPath").build(); + + ClusterMetadataManifest manifest1 = ClusterMetadataManifest.builder() + .indices(List.of(index1Metadata)) + .coordinationMetadata(coordinationMetadataUpdated) + .templatesMetadata(templateMetadata) + .settingMetadata(settingMetadata) + .clusterTerm(1L) + .stateVersion(1L) + .codecVersion(CODEC_V3) + .stateUUID(randomAlphaOfLength(10)) + .clusterUUID(clusterUUID) + .nodeId("nodeA") + .opensearchVersion(VersionUtils.randomOpenSearchVersion(random())) + .previousClusterUUID(ClusterState.UNKNOWN_UUID) + .committed(true) + .routingTableVersion(0L) + .indicesRouting(indicesRouting1) + .diffManifest(diffManifest1) + .build(); + ClusterMetadataManifest manifest2 = ClusterMetadataManifest.builder(manifest1) + .indices(List.of(index2Metadata)) + .indicesRouting(indicesRouting2) + .diffManifest(diffManifest2) + .build(); + + BlobContainer blobContainer = mock(BlobContainer.class); + doThrow(IOException.class).when(blobContainer).delete(); + when(blobStore.blobContainer(any())).thenReturn(blobContainer); + + BlobPath blobPath = new BlobPath().add("random-path"); + when((blobStoreRepository.basePath())).thenReturn(blobPath); + remoteClusterStateCleanupManager.start(); + when(remoteManifestManager.getManifestFolderPath(eq(clusterName), eq(clusterUUID))).thenReturn( + new BlobPath().add(encodeString(clusterName)).add(CLUSTER_STATE_PATH_TOKEN).add(clusterUUID).add(MANIFEST) + ); + when(remoteManifestManager.fetchRemoteClusterMetadataManifest(eq(clusterName), eq(clusterUUID), any())).thenReturn( + manifest1, + manifest2 + ); + doNothing().when(remoteRoutingTableService).deleteStaleIndexRoutingDiffPaths(any()); + + remoteClusterStateCleanupManager.deleteClusterMetadata(clusterName, clusterUUID, activeBlobs, inactiveBlobs); + assertBusy(() -> { + // wait for stats to get updated + assertNotNull(remoteClusterStateCleanupManager.getStats()); + assertEquals(0, remoteClusterStateCleanupManager.getStats().getIndicesRoutingDiffFileCleanupAttemptFailedCount()); + }); + + doThrow(IOException.class).when(remoteRoutingTableService).deleteStaleIndexRoutingPaths(any()); + remoteClusterStateCleanupManager.deleteClusterMetadata(clusterName, clusterUUID, activeBlobs, inactiveBlobs); + assertBusy(() -> { + // wait for stats to get updated + assertNotNull(remoteClusterStateCleanupManager.getStats()); + assertEquals(1, remoteClusterStateCleanupManager.getStats().getIndicesRoutingDiffFileCleanupAttemptFailedCount()); + }); + } + public void testSingleConcurrentExecutionOfStaleManifestCleanup() throws Exception { BlobContainer blobContainer = mock(BlobContainer.class); when(blobStore.blobContainer(any())).thenReturn(blobContainer); diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java index 6c764585c48e7..59ca62dff2aa7 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateServiceTests.java @@ -535,14 +535,15 @@ public void testTimeoutWhileWritingManifestFile() throws IOException { anyBoolean(), anyMap(), anyBoolean(), - anyList() + anyList(), + anyMap() ) ).thenReturn(new RemoteClusterStateUtils.UploadedMetadataResults()); RemoteStateTransferException ex = expectThrows( RemoteStateTransferException.class, () -> spiedService.writeFullMetadata(clusterState, randomAlphaOfLength(10)) ); - assertTrue(ex.getMessage().contains("Timed out waiting for transfer of manifest file to complete")); + assertTrue(ex.getMessage().contains("Timed out waiting for transfer of following metadata to complete")); } public void testWriteFullMetadataInParallelFailureForIndexMetadata() throws IOException { @@ -634,7 +635,8 @@ public void testWriteMetadataInParallelIncompleteUpload() throws IOException { true, clusterState.getCustoms(), true, - emptyList() + emptyList(), + null ) ); assertTrue(exception.getMessage().startsWith("Some metadata components were not uploaded successfully")); @@ -684,7 +686,8 @@ public void testWriteIncrementalMetadataSuccess() throws IOException { eq(false), eq(Collections.emptyMap()), eq(false), - eq(Collections.emptyList()) + eq(Collections.emptyList()), + eq(Collections.emptyMap()) ); assertThat(manifestInfo.getManifestFileName(), notNullValue()); @@ -764,7 +767,8 @@ public void testWriteIncrementalMetadataSuccessWhenPublicationEnabled() throws I eq(false), eq(Collections.emptyMap()), eq(true), - Mockito.anyList() + anyList(), + eq(Collections.emptyMap()) ); assertThat(manifestInfo.getManifestFileName(), notNullValue()); @@ -811,7 +815,8 @@ public void testTimeoutWhileWritingMetadata() throws IOException { true, emptyMap(), true, - emptyList() + emptyList(), + null ) ); assertTrue(exception.getMessage().startsWith("Timed out waiting for transfer of following metadata to complete")); @@ -862,6 +867,7 @@ public void testGetClusterStateForManifest_IncludeEphemeral() throws IOException eq(manifest.getIndicesRouting()), eq(true), eq(manifest.getClusterStateCustomMap()), + eq(false), eq(true) ); } @@ -911,7 +917,9 @@ public void testGetClusterStateForManifest_ExcludeEphemeral() throws IOException eq(emptyList()), eq(false), eq(emptyMap()), + eq(false), eq(false) + ); } @@ -958,6 +966,7 @@ public void testGetClusterStateFromManifest_CodecV1() throws IOException { eq(emptyList()), eq(false), eq(emptyMap()), + eq(false), eq(false) ); verify(mockedGlobalMetadataManager, times(1)).getGlobalMetadata(eq(manifest.getClusterUUID()), eq(manifest)); @@ -1281,6 +1290,7 @@ public void testReadClusterStateInParallel_TimedOut() throws IOException { emptyList(), true, emptyMap(), + false, true ) ); @@ -1312,6 +1322,7 @@ public void testReadClusterStateInParallel_ExceptionDuringRead() throws IOExcept emptyList(), true, emptyMap(), + false, true ) ); @@ -1418,6 +1429,7 @@ public void testReadClusterStateInParallel_UnexpectedResult() throws IOException emptyList(), true, newClusterStateCustoms, + false, true ) ); @@ -1652,6 +1664,7 @@ public void testReadClusterStateInParallel_Success() throws IOException { emptyList(), true, newClusterStateCustoms, + false, true ); @@ -2745,6 +2758,108 @@ public void testWriteIncrementalMetadataSuccessWithRoutingTable() throws IOExcep assertThat(manifest.getIndicesRouting().get(0).getUploadedFilename(), notNullValue()); } + public void testWriteIncrementalMetadataSuccessWithRoutingTableDiff() throws IOException { + initializeRoutingTable(); + final ClusterState clusterState = generateClusterStateWithOneIndex("test-index", 5, 1, false).nodes( + nodesWithLocalNodeClusterManager() + ).build(); + mockBlobStoreObjects(); + List indices = new ArrayList<>(); + final UploadedIndexMetadata uploadedIndiceRoutingMetadata = new UploadedIndexMetadata( + "test-index", + "index-uuid", + "routing-filename", + INDEX_ROUTING_METADATA_PREFIX + ); + indices.add(uploadedIndiceRoutingMetadata); + final ClusterState previousClusterState = generateClusterStateWithOneIndex("test-index", 5, 1, true).nodes( + nodesWithLocalNodeClusterManager() + ).build(); + + final ClusterMetadataManifest previousManifest = ClusterMetadataManifest.builder().indices(indices).build(); + when((blobStoreRepository.basePath())).thenReturn(BlobPath.cleanPath().add("base-path")); + + remoteClusterStateService.start(); + final ClusterMetadataManifest manifest = remoteClusterStateService.writeIncrementalMetadata( + previousClusterState, + clusterState, + previousManifest + ).getClusterMetadataManifest(); + final UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "index-uuid", "metadata-filename"); + final ClusterMetadataManifest expectedManifest = ClusterMetadataManifest.builder() + .indices(List.of(uploadedIndexMetadata)) + .clusterTerm(clusterState.term()) + .stateVersion(1L) + .stateUUID("state-uuid") + .clusterUUID("cluster-uuid") + .previousClusterUUID("prev-cluster-uuid") + .routingTableVersion(1) + .indicesRouting(List.of(uploadedIndiceRoutingMetadata)) + .build(); + + assertThat(manifest.getIndices().size(), is(1)); + assertThat(manifest.getClusterTerm(), is(expectedManifest.getClusterTerm())); + assertThat(manifest.getStateVersion(), is(expectedManifest.getStateVersion())); + assertThat(manifest.getClusterUUID(), is(expectedManifest.getClusterUUID())); + assertThat(manifest.getStateUUID(), is(expectedManifest.getStateUUID())); + assertThat(manifest.getRoutingTableVersion(), is(expectedManifest.getRoutingTableVersion())); + assertThat(manifest.getIndicesRouting().get(0).getIndexName(), is(uploadedIndiceRoutingMetadata.getIndexName())); + assertThat(manifest.getIndicesRouting().get(0).getIndexUUID(), is(uploadedIndiceRoutingMetadata.getIndexUUID())); + assertThat(manifest.getIndicesRouting().get(0).getUploadedFilename(), notNullValue()); + assertThat(manifest.getDiffManifest().getIndicesRoutingDiffPath(), notNullValue()); + } + + public void testWriteIncrementalMetadataSuccessWithRoutingTableDiffNull() throws IOException { + initializeRoutingTable(); + final ClusterState clusterState = generateClusterStateWithOneIndex("test-index", 5, 1, false).nodes( + nodesWithLocalNodeClusterManager() + ).build(); + mockBlobStoreObjects(); + List indices = new ArrayList<>(); + final UploadedIndexMetadata uploadedIndiceRoutingMetadata = new UploadedIndexMetadata( + "test-index", + "index-uuid", + "routing-filename", + INDEX_ROUTING_METADATA_PREFIX + ); + indices.add(uploadedIndiceRoutingMetadata); + final ClusterState previousClusterState = generateClusterStateWithOneIndex("test-index2", 5, 1, false).nodes( + nodesWithLocalNodeClusterManager() + ).build(); + + final ClusterMetadataManifest previousManifest = ClusterMetadataManifest.builder().indices(indices).build(); + when((blobStoreRepository.basePath())).thenReturn(BlobPath.cleanPath().add("base-path")); + + remoteClusterStateService.start(); + final ClusterMetadataManifest manifest = remoteClusterStateService.writeIncrementalMetadata( + previousClusterState, + clusterState, + previousManifest + ).getClusterMetadataManifest(); + final UploadedIndexMetadata uploadedIndexMetadata = new UploadedIndexMetadata("test-index", "index-uuid", "metadata-filename"); + final ClusterMetadataManifest expectedManifest = ClusterMetadataManifest.builder() + .indices(List.of(uploadedIndexMetadata)) + .clusterTerm(clusterState.term()) + .stateVersion(1L) + .stateUUID("state-uuid") + .clusterUUID("cluster-uuid") + .previousClusterUUID("prev-cluster-uuid") + .routingTableVersion(1) + .indicesRouting(List.of(uploadedIndiceRoutingMetadata)) + .build(); + + assertThat(manifest.getIndices().size(), is(1)); + assertThat(manifest.getClusterTerm(), is(expectedManifest.getClusterTerm())); + assertThat(manifest.getStateVersion(), is(expectedManifest.getStateVersion())); + assertThat(manifest.getClusterUUID(), is(expectedManifest.getClusterUUID())); + assertThat(manifest.getStateUUID(), is(expectedManifest.getStateUUID())); + assertThat(manifest.getRoutingTableVersion(), is(expectedManifest.getRoutingTableVersion())); + assertThat(manifest.getIndicesRouting().get(0).getIndexName(), is(uploadedIndiceRoutingMetadata.getIndexName())); + assertThat(manifest.getIndicesRouting().get(0).getIndexUUID(), is(uploadedIndiceRoutingMetadata.getIndexUUID())); + assertThat(manifest.getIndicesRouting().get(0).getUploadedFilename(), notNullValue()); + assertThat(manifest.getDiffManifest().getIndicesRoutingDiffPath(), nullValue()); + } + private void initializeRoutingTable() { Settings newSettings = Settings.builder() .put("node.attr." + REMOTE_STORE_ROUTING_TABLE_REPOSITORY_NAME_ATTRIBUTE_KEY, "routing_repository") @@ -3217,6 +3332,54 @@ static ClusterState.Builder generateClusterStateWithOneIndex() { .routingTable(RoutingTable.builder().addAsNew(indexMetadata).version(1L).build()); } + public static ClusterState.Builder generateClusterStateWithOneIndex( + String indexName, + int primaryShards, + int replicaShards, + boolean addAsNew + ) { + + final Index index = new Index(indexName, "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(primaryShards) + .numberOfReplicas(replicaShards) + .build(); + final CoordinationMetadata coordinationMetadata = CoordinationMetadata.builder().term(1L).build(); + final Settings settings = Settings.builder().put("mock-settings", true).build(); + final TemplatesMetadata templatesMetadata = TemplatesMetadata.builder() + .put(IndexTemplateMetadata.builder("template1").settings(idxSettings).patterns(List.of("test*")).build()) + .build(); + final CustomMetadata1 customMetadata1 = new CustomMetadata1("custom-metadata-1"); + + RoutingTable.Builder routingTableBuilder = RoutingTable.builder(); + if (addAsNew) { + routingTableBuilder.addAsNew(indexMetadata); + } else { + routingTableBuilder.addAsRecovery(indexMetadata); + } + + return ClusterState.builder(ClusterName.DEFAULT) + .version(1L) + .stateUUID("state-uuid") + .metadata( + Metadata.builder() + .version(randomNonNegativeLong()) + .put(indexMetadata, true) + .clusterUUID("cluster-uuid") + .coordinationMetadata(coordinationMetadata) + .persistentSettings(settings) + .templates(templatesMetadata) + .hashesOfConsistentSettings(Map.of("key1", "value1", "key2", "value2")) + .putCustom(customMetadata1.getWriteableName(), customMetadata1) + .build() + ) + .routingTable(routingTableBuilder.version(1L).build()); + } + static ClusterState.Builder generateClusterStateWithAllAttributes() { final Index index = new Index("test-index", "index-uuid"); final Settings idxSettings = Settings.builder() @@ -3296,7 +3459,7 @@ static ClusterMetadataManifest.Builder generateClusterMetadataManifestWithAllAtt ); } - static DiscoveryNodes nodesWithLocalNodeClusterManager() { + public static DiscoveryNodes nodesWithLocalNodeClusterManager() { final DiscoveryNode localNode = new DiscoveryNode("cluster-manager-id", buildNewFakeTransportAddress(), Version.CURRENT); return DiscoveryNodes.builder().clusterManagerNodeId("cluster-manager-id").localNodeId("cluster-manager-id").add(localNode).build(); } diff --git a/server/src/test/java/org/opensearch/gateway/remote/model/ClusterStateDiffManifestTests.java b/server/src/test/java/org/opensearch/gateway/remote/model/ClusterStateDiffManifestTests.java index 897b2f5eeb25d..f89619a09cd52 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/model/ClusterStateDiffManifestTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/model/ClusterStateDiffManifestTests.java @@ -10,6 +10,7 @@ import org.opensearch.Version; import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.DiffableUtils; import org.opensearch.cluster.coordination.CoordinationMetadata; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.IndexTemplateMetadata; @@ -17,6 +18,7 @@ import org.opensearch.cluster.metadata.TemplatesMetadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.IndexRoutingTable; import org.opensearch.common.settings.Settings; import org.opensearch.common.xcontent.json.JsonXContent; import org.opensearch.core.common.bytes.BytesReference; @@ -40,7 +42,11 @@ import static java.util.stream.Collectors.toList; import static org.opensearch.Version.CURRENT; import static org.opensearch.cluster.ClusterState.EMPTY_STATE; +import static org.opensearch.cluster.routing.remote.RemoteRoutingTableService.CUSTOM_ROUTING_TABLE_DIFFABLE_VALUE_SERIALIZER; import static org.opensearch.core.common.transport.TransportAddress.META_ADDRESS; +import static org.opensearch.gateway.remote.ClusterMetadataManifest.CODEC_V3; +import static org.opensearch.gateway.remote.RemoteClusterStateServiceTests.generateClusterStateWithOneIndex; +import static org.opensearch.gateway.remote.RemoteClusterStateServiceTests.nodesWithLocalNodeClusterManager; import static org.opensearch.gateway.remote.model.RemoteClusterBlocksTests.randomClusterBlocks; public class ClusterStateDiffManifestTests extends OpenSearchTestCase { @@ -114,11 +120,70 @@ public void testClusterStateDiffManifestXContent() throws IOException { diffManifest.toXContent(builder, ToXContent.EMPTY_PARAMS); builder.endObject(); try (XContentParser parser = createParser(JsonXContent.jsonXContent, BytesReference.bytes(builder))) { - final ClusterStateDiffManifest parsedManifest = ClusterStateDiffManifest.fromXContent(parser); + final ClusterStateDiffManifest parsedManifest = ClusterStateDiffManifest.fromXContent(parser, CODEC_V3); assertEquals(diffManifest, parsedManifest); } } + public void testClusterStateWithRoutingTableDiffInDiffManifestXContent() throws IOException { + ClusterState initialState = generateClusterStateWithOneIndex("test-index", 5, 1, true).nodes(nodesWithLocalNodeClusterManager()) + .build(); + + ClusterState updatedState = generateClusterStateWithOneIndex("test-index", 5, 2, false).nodes(nodesWithLocalNodeClusterManager()) + .build(); + + ClusterStateDiffManifest diffManifest = verifyRoutingTableDiffManifest(initialState, updatedState); + final XContentBuilder builder = JsonXContent.contentBuilder(); + builder.startObject(); + diffManifest.toXContent(builder, ToXContent.EMPTY_PARAMS); + builder.endObject(); + try (XContentParser parser = createParser(JsonXContent.jsonXContent, BytesReference.bytes(builder))) { + final ClusterStateDiffManifest parsedManifest = ClusterStateDiffManifest.fromXContent(parser, CODEC_V3); + assertEquals(diffManifest, parsedManifest); + } + } + + public void testClusterStateWithRoutingTableDiffInDiffManifestXContent1() throws IOException { + ClusterState initialState = generateClusterStateWithOneIndex("test-index", 5, 1, true).nodes(nodesWithLocalNodeClusterManager()) + .build(); + + ClusterState updatedState = generateClusterStateWithOneIndex("test-index-1", 5, 2, false).nodes(nodesWithLocalNodeClusterManager()) + .build(); + + ClusterStateDiffManifest diffManifest = verifyRoutingTableDiffManifest(initialState, updatedState); + final XContentBuilder builder = JsonXContent.contentBuilder(); + builder.startObject(); + diffManifest.toXContent(builder, ToXContent.EMPTY_PARAMS); + builder.endObject(); + try (XContentParser parser = createParser(JsonXContent.jsonXContent, BytesReference.bytes(builder))) { + final ClusterStateDiffManifest parsedManifest = ClusterStateDiffManifest.fromXContent(parser, CODEC_V3); + assertEquals(diffManifest, parsedManifest); + } + } + + private ClusterStateDiffManifest verifyRoutingTableDiffManifest(ClusterState previousState, ClusterState currentState) { + // Create initial and updated IndexRoutingTable maps + Map initialRoutingTableMap = previousState.getRoutingTable().indicesRouting(); + Map updatedRoutingTableMap = currentState.getRoutingTable().indicesRouting(); + + DiffableUtils.MapDiff> routingTableIncrementalDiff = DiffableUtils.diff( + initialRoutingTableMap, + updatedRoutingTableMap, + DiffableUtils.getStringKeySerializer(), + CUSTOM_ROUTING_TABLE_DIFFABLE_VALUE_SERIALIZER + ); + ClusterStateDiffManifest manifest = new ClusterStateDiffManifest( + currentState, + previousState, + routingTableIncrementalDiff, + "indicesRoutingDiffPath" + ); + assertEquals("indicesRoutingDiffPath", manifest.getIndicesRoutingDiffPath()); + assertEquals(routingTableIncrementalDiff.getUpserts().size(), manifest.getIndicesRoutingUpdated().size()); + assertEquals(routingTableIncrementalDiff.getDeletes().size(), manifest.getIndicesRoutingDeleted().size()); + return manifest; + } + private ClusterStateDiffManifest updateAndVerifyState( ClusterState initialState, List indicesToAdd, @@ -191,7 +256,7 @@ private ClusterStateDiffManifest updateAndVerifyState( } ClusterState updatedClusterState = clusterStateBuilder.metadata(metadataBuilder.build()).build(); - ClusterStateDiffManifest manifest = new ClusterStateDiffManifest(updatedClusterState, initialState); + ClusterStateDiffManifest manifest = new ClusterStateDiffManifest(updatedClusterState, initialState, null, null); assertEquals(indicesToAdd.stream().map(im -> im.getIndex().getName()).collect(toList()), manifest.getIndicesUpdated()); assertEquals(indicesToRemove, manifest.getIndicesDeleted()); assertEquals(new ArrayList<>(customsToAdd.keySet()), manifest.getCustomMetadataUpdated()); diff --git a/server/src/test/java/org/opensearch/gateway/remote/routingtable/RemoteIndexRoutingTableDiffTests.java b/server/src/test/java/org/opensearch/gateway/remote/routingtable/RemoteIndexRoutingTableDiffTests.java new file mode 100644 index 0000000000000..6ffa7fc5cded8 --- /dev/null +++ b/server/src/test/java/org/opensearch/gateway/remote/routingtable/RemoteIndexRoutingTableDiffTests.java @@ -0,0 +1,317 @@ +/* + * 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.routingtable; + +import org.opensearch.Version; +import org.opensearch.cluster.Diff; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.routing.RoutingTableIncrementalDiff; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.compress.DeflateCompressor; +import org.opensearch.common.remote.BlobPathParameters; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.io.stream.NamedWriteableRegistry; +import org.opensearch.core.compress.Compressor; +import org.opensearch.core.compress.NoneCompressor; +import org.opensearch.gateway.remote.ClusterMetadataManifest; +import org.opensearch.index.remote.RemoteStoreUtils; +import org.opensearch.index.translog.transfer.BlobStoreTransferService; +import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; +import org.junit.After; +import org.junit.Before; + +import java.io.IOException; +import java.io.InputStream; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.opensearch.gateway.remote.routingtable.RemoteRoutingTableDiff.ROUTING_TABLE_DIFF_FILE; +import static org.opensearch.gateway.remote.routingtable.RemoteRoutingTableDiff.ROUTING_TABLE_DIFF_METADATA_PREFIX; +import static org.opensearch.gateway.remote.routingtable.RemoteRoutingTableDiff.ROUTING_TABLE_DIFF_PATH_TOKEN; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.Matchers.nullValue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class RemoteIndexRoutingTableDiffTests extends OpenSearchTestCase { + + private static final String TEST_BLOB_NAME = "/test-path/test-blob-name"; + private static final String TEST_BLOB_PATH = "test-path"; + private static final String TEST_BLOB_FILE_NAME = "test-blob-name"; + private static final long STATE_VERSION = 3L; + private static final long STATE_TERM = 2L; + private String clusterUUID; + private BlobStoreRepository blobStoreRepository; + private BlobStoreTransferService blobStoreTransferService; + private ClusterSettings clusterSettings; + private Compressor compressor; + + private String clusterName; + private NamedWriteableRegistry namedWriteableRegistry; + private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); + + @Before + public void setup() { + clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + this.clusterUUID = "test-cluster-uuid"; + this.blobStoreTransferService = mock(BlobStoreTransferService.class); + this.blobStoreRepository = mock(BlobStoreRepository.class); + BlobPath blobPath = new BlobPath().add("/path"); + when(blobStoreRepository.basePath()).thenReturn(blobPath); + when(blobStoreRepository.getCompressor()).thenReturn(new DeflateCompressor()); + compressor = new NoneCompressor(); + namedWriteableRegistry = writableRegistry(); + this.clusterName = "test-cluster-name"; + } + + @After + public void tearDown() throws Exception { + super.tearDown(); + threadPool.shutdown(); + } + + public void testClusterUUID() { + Map> diffs = new HashMap<>(); + String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); + int numberOfShards = randomIntBetween(1, 10); + int numberOfReplicas = randomIntBetween(1, 10); + + IndexMetadata indexMetadata = IndexMetadata.builder(indexName) + .settings(settings(Version.CURRENT)) + .numberOfShards(numberOfShards) + .numberOfReplicas(numberOfReplicas) + .build(); + + IndexRoutingTable indexRoutingTable = IndexRoutingTable.builder(indexMetadata.getIndex()).initializeAsNew(indexMetadata).build(); + + diffs.put(indexName, indexRoutingTable.diff(indexRoutingTable)); + + RoutingTableIncrementalDiff routingTableIncrementalDiff = new RoutingTableIncrementalDiff(diffs); + + RemoteRoutingTableDiff remoteDiffForUpload = new RemoteRoutingTableDiff( + routingTableIncrementalDiff, + clusterUUID, + compressor, + STATE_TERM, + STATE_VERSION + ); + assertEquals(remoteDiffForUpload.clusterUUID(), clusterUUID); + + RemoteRoutingTableDiff remoteDiffForDownload = new RemoteRoutingTableDiff(TEST_BLOB_NAME, clusterUUID, compressor); + assertEquals(remoteDiffForDownload.clusterUUID(), clusterUUID); + } + + public void testFullBlobName() { + Map> diffs = new HashMap<>(); + String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); + int numberOfShards = randomIntBetween(1, 10); + int numberOfReplicas = randomIntBetween(1, 10); + + IndexMetadata indexMetadata = IndexMetadata.builder(indexName) + .settings(settings(Version.CURRENT)) + .numberOfShards(numberOfShards) + .numberOfReplicas(numberOfReplicas) + .build(); + + IndexRoutingTable indexRoutingTable = IndexRoutingTable.builder(indexMetadata.getIndex()).initializeAsNew(indexMetadata).build(); + + diffs.put(indexName, indexRoutingTable.diff(indexRoutingTable)); + RoutingTableIncrementalDiff routingTableIncrementalDiff = new RoutingTableIncrementalDiff(diffs); + + RemoteRoutingTableDiff remoteDiffForUpload = new RemoteRoutingTableDiff( + routingTableIncrementalDiff, + clusterUUID, + compressor, + STATE_TERM, + STATE_VERSION + ); + assertThat(remoteDiffForUpload.getFullBlobName(), nullValue()); + + RemoteRoutingTableDiff remoteDiffForDownload = new RemoteRoutingTableDiff(TEST_BLOB_NAME, clusterUUID, compressor); + assertThat(remoteDiffForDownload.getFullBlobName(), is(TEST_BLOB_NAME)); + } + + public void testBlobFileName() { + Map> diffs = new HashMap<>(); + String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); + int numberOfShards = randomIntBetween(1, 10); + int numberOfReplicas = randomIntBetween(1, 10); + + IndexMetadata indexMetadata = IndexMetadata.builder(indexName) + .settings(settings(Version.CURRENT)) + .numberOfShards(numberOfShards) + .numberOfReplicas(numberOfReplicas) + .build(); + + IndexRoutingTable indexRoutingTable = IndexRoutingTable.builder(indexMetadata.getIndex()).initializeAsNew(indexMetadata).build(); + + diffs.put(indexName, indexRoutingTable.diff(indexRoutingTable)); + RoutingTableIncrementalDiff routingTableIncrementalDiff = new RoutingTableIncrementalDiff(diffs); + + RemoteRoutingTableDiff remoteDiffForUpload = new RemoteRoutingTableDiff( + routingTableIncrementalDiff, + clusterUUID, + compressor, + STATE_TERM, + STATE_VERSION + ); + assertThat(remoteDiffForUpload.getBlobFileName(), nullValue()); + + RemoteRoutingTableDiff remoteDiffForDownload = new RemoteRoutingTableDiff(TEST_BLOB_NAME, clusterUUID, compressor); + assertThat(remoteDiffForDownload.getBlobFileName(), is(TEST_BLOB_FILE_NAME)); + } + + public void testBlobPathParameters() { + Map> diffs = new HashMap<>(); + String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); + int numberOfShards = randomIntBetween(1, 10); + int numberOfReplicas = randomIntBetween(1, 10); + + IndexMetadata indexMetadata = IndexMetadata.builder(indexName) + .settings(settings(Version.CURRENT)) + .numberOfShards(numberOfShards) + .numberOfReplicas(numberOfReplicas) + .build(); + + IndexRoutingTable indexRoutingTable = IndexRoutingTable.builder(indexMetadata.getIndex()).initializeAsNew(indexMetadata).build(); + + diffs.put(indexName, indexRoutingTable.diff(indexRoutingTable)); + RoutingTableIncrementalDiff routingTableIncrementalDiff = new RoutingTableIncrementalDiff(diffs); + + RemoteRoutingTableDiff remoteDiffForUpload = new RemoteRoutingTableDiff( + routingTableIncrementalDiff, + clusterUUID, + compressor, + STATE_TERM, + STATE_VERSION + ); + assertThat(remoteDiffForUpload.getBlobFileName(), nullValue()); + + BlobPathParameters params = remoteDiffForUpload.getBlobPathParameters(); + assertThat(params.getPathTokens(), is(List.of(ROUTING_TABLE_DIFF_PATH_TOKEN))); + String expectedPrefix = ROUTING_TABLE_DIFF_METADATA_PREFIX; + assertThat(params.getFilePrefix(), is(expectedPrefix)); + } + + public void testGenerateBlobFileName() { + Map> diffs = new HashMap<>(); + String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); + int numberOfShards = randomIntBetween(1, 10); + int numberOfReplicas = randomIntBetween(1, 10); + + IndexMetadata indexMetadata = IndexMetadata.builder(indexName) + .settings(settings(Version.CURRENT)) + .numberOfShards(numberOfShards) + .numberOfReplicas(numberOfReplicas) + .build(); + + IndexRoutingTable indexRoutingTable = IndexRoutingTable.builder(indexMetadata.getIndex()).initializeAsNew(indexMetadata).build(); + + diffs.put(indexName, indexRoutingTable.diff(indexRoutingTable)); + RoutingTableIncrementalDiff routingTableIncrementalDiff = new RoutingTableIncrementalDiff(diffs); + + RemoteRoutingTableDiff remoteDiffForUpload = new RemoteRoutingTableDiff( + routingTableIncrementalDiff, + clusterUUID, + compressor, + STATE_TERM, + STATE_VERSION + ); + + String blobFileName = remoteDiffForUpload.generateBlobFileName(); + String[] nameTokens = blobFileName.split("__"); + assertEquals(ROUTING_TABLE_DIFF_METADATA_PREFIX, nameTokens[0]); + assertEquals(RemoteStoreUtils.invertLong(STATE_TERM), nameTokens[1]); + assertEquals(RemoteStoreUtils.invertLong(STATE_VERSION), nameTokens[2]); + assertThat(RemoteStoreUtils.invertLong(nameTokens[3]), lessThanOrEqualTo(System.currentTimeMillis())); + } + + public void testGetUploadedMetadata() throws IOException { + Map> diffs = new HashMap<>(); + String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); + int numberOfShards = randomIntBetween(1, 10); + int numberOfReplicas = randomIntBetween(1, 10); + + IndexMetadata indexMetadata = IndexMetadata.builder(indexName) + .settings(settings(Version.CURRENT)) + .numberOfShards(numberOfShards) + .numberOfReplicas(numberOfReplicas) + .build(); + + IndexRoutingTable indexRoutingTable = IndexRoutingTable.builder(indexMetadata.getIndex()).initializeAsNew(indexMetadata).build(); + + diffs.put(indexName, indexRoutingTable.diff(indexRoutingTable)); + RoutingTableIncrementalDiff routingTableIncrementalDiff = new RoutingTableIncrementalDiff(diffs); + + RemoteRoutingTableDiff remoteDiffForUpload = new RemoteRoutingTableDiff( + routingTableIncrementalDiff, + clusterUUID, + compressor, + STATE_TERM, + STATE_VERSION + ); + + remoteDiffForUpload.setFullBlobName(new BlobPath().add(TEST_BLOB_PATH)); + ClusterMetadataManifest.UploadedMetadata uploadedMetadataAttribute = remoteDiffForUpload.getUploadedMetadata(); + assertEquals(ROUTING_TABLE_DIFF_FILE, uploadedMetadataAttribute.getComponent()); + } + + public void testStreamOperations() throws IOException { + String indexName = randomAlphaOfLength(randomIntBetween(1, 50)); + int numberOfShards = randomIntBetween(1, 10); + int numberOfReplicas = randomIntBetween(1, 10); + + Metadata metadata = Metadata.builder() + .put( + IndexMetadata.builder(indexName) + .settings(settings(Version.CURRENT)) + .numberOfShards(numberOfShards) + .numberOfReplicas(numberOfReplicas) + ) + .build(); + + RoutingTable initialRoutingTable = RoutingTable.builder().addAsNew(metadata.index(indexName)).build(); + Map> diffs = new HashMap<>(); + + initialRoutingTable.getIndicesRouting().values().forEach(indexRoutingTable -> { + diffs.put(indexName, indexRoutingTable.diff(indexRoutingTable)); + RoutingTableIncrementalDiff routingTableIncrementalDiff = new RoutingTableIncrementalDiff(diffs); + + RemoteRoutingTableDiff remoteDiffForUpload = new RemoteRoutingTableDiff( + routingTableIncrementalDiff, + clusterUUID, + compressor, + STATE_TERM, + STATE_VERSION + ); + + assertThrows(AssertionError.class, remoteDiffForUpload::getUploadedMetadata); + + try (InputStream inputStream = remoteDiffForUpload.serialize()) { + remoteDiffForUpload.setFullBlobName(BlobPath.cleanPath()); + assertThat(inputStream.available(), greaterThan(0)); + + routingTableIncrementalDiff = remoteDiffForUpload.deserialize(inputStream); + assertEquals(remoteDiffForUpload.getDiffs().size(), routingTableIncrementalDiff.getDiffs().size()); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } +} From 5026af61b2b8cd5695c3945508a5fae2f4267de8 Mon Sep 17 00:00:00 2001 From: Pranshu Shukla <55992439+Pranshu-S@users.noreply.github.com> Date: Tue, 23 Jul 2024 20:24:00 +0530 Subject: [PATCH 27/37] Optimized ClusterStatsIndices to precomute shard stats (#14426) * Optimize Cluster Stats Indices to precomute node level stats Signed-off-by: Pranshu Shukla --- CHANGELOG.md | 1 + .../admin/cluster/stats/ClusterStatsIT.java | 119 ++++++-- .../cluster/stats/ClusterStatsIndices.java | 67 +++-- .../stats/ClusterStatsNodeResponse.java | 133 ++++++++- .../cluster/stats/ClusterStatsRequest.java | 17 ++ .../stats/ClusterStatsRequestBuilder.java | 5 + .../stats/TransportClusterStatsAction.java | 10 +- .../admin/cluster/RestClusterStatsAction.java | 1 + .../cluster/stats/ClusterStatsNodesTests.java | 269 ++++++++++++++++++ 9 files changed, 584 insertions(+), 38 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index c8f185ca2bb3d..6aa3d7a58dda4 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -30,6 +30,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add persian_stem filter (([#14847](https://github.com/opensearch-project/OpenSearch/pull/14847))) - Create listener to refresh search thread resource usage ([#14832](https://github.com/opensearch-project/OpenSearch/pull/14832)) - Add rest, transport layer changes for hot to warm tiering - dedicated setup (([#13980](https://github.com/opensearch-project/OpenSearch/pull/13980)) +- Optimize Cluster Stats Indices to precomute node level stats ([#14426](https://github.com/opensearch-project/OpenSearch/pull/14426)) ### Dependencies - Bump `org.gradle.test-retry` from 1.5.8 to 1.5.9 ([#13442](https://github.com/opensearch-project/OpenSearch/pull/13442)) diff --git a/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/stats/ClusterStatsIT.java b/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/stats/ClusterStatsIT.java index 085a32593063a..f23cdbb50b37a 100644 --- a/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/stats/ClusterStatsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/stats/ClusterStatsIT.java @@ -88,7 +88,11 @@ public void testNodeCounts() { Map expectedCounts = getExpectedCounts(1, 1, 1, 1, 1, 0, 0); int numNodes = randomIntBetween(1, 5); - ClusterStatsResponse response = client().admin().cluster().prepareClusterStats().get(); + ClusterStatsResponse response = client().admin() + .cluster() + .prepareClusterStats() + .useAggregatedNodeLevelResponses(randomBoolean()) + .get(); assertCounts(response.getNodesStats().getCounts(), total, expectedCounts); for (int i = 0; i < numNodes; i++) { @@ -153,7 +157,11 @@ public void testNodeCountsWithDeprecatedMasterRole() throws ExecutionException, Map expectedCounts = getExpectedCounts(0, 1, 1, 0, 0, 0, 0); Client client = client(); - ClusterStatsResponse response = client.admin().cluster().prepareClusterStats().get(); + ClusterStatsResponse response = client.admin() + .cluster() + .prepareClusterStats() + .useAggregatedNodeLevelResponses(randomBoolean()) + .get(); assertCounts(response.getNodesStats().getCounts(), total, expectedCounts); Set expectedRoles = Set.of(DiscoveryNodeRole.MASTER_ROLE.roleName()); @@ -176,15 +184,60 @@ private void assertShardStats(ClusterStatsIndices.ShardStats stats, int indices, assertThat(stats.getReplication(), Matchers.equalTo(replicationFactor)); } - public void testIndicesShardStats() throws ExecutionException, InterruptedException { + public void testIndicesShardStatsWithoutNodeLevelAggregations() { + internalCluster().startNode(); + ensureGreen(); + ClusterStatsResponse response = client().admin().cluster().prepareClusterStats().useAggregatedNodeLevelResponses(false).get(); + assertThat(response.getStatus(), Matchers.equalTo(ClusterHealthStatus.GREEN)); + + prepareCreate("test1").setSettings(Settings.builder().put("number_of_shards", 2).put("number_of_replicas", 1)).get(); + + response = client().admin().cluster().prepareClusterStats().useAggregatedNodeLevelResponses(false).get(); + assertThat(response.getStatus(), Matchers.equalTo(ClusterHealthStatus.YELLOW)); + assertThat(response.indicesStats.getDocs().getCount(), Matchers.equalTo(0L)); + assertThat(response.indicesStats.getIndexCount(), Matchers.equalTo(1)); + assertShardStats(response.getIndicesStats().getShards(), 1, 2, 2, 0.0); + + // add another node, replicas should get assigned + internalCluster().startNode(); + ensureGreen(); + index("test1", "type", "1", "f", "f"); + refresh(); // make the doc visible + response = client().admin().cluster().prepareClusterStats().useAggregatedNodeLevelResponses(false).get(); + assertThat(response.getStatus(), Matchers.equalTo(ClusterHealthStatus.GREEN)); + assertThat(response.indicesStats.getDocs().getCount(), Matchers.equalTo(1L)); + assertShardStats(response.getIndicesStats().getShards(), 1, 4, 2, 1.0); + + prepareCreate("test2").setSettings(Settings.builder().put("number_of_shards", 3).put("number_of_replicas", 0)).get(); + ensureGreen(); + response = client().admin().cluster().prepareClusterStats().useAggregatedNodeLevelResponses(false).get(); + assertThat(response.getStatus(), Matchers.equalTo(ClusterHealthStatus.GREEN)); + assertThat(response.indicesStats.getIndexCount(), Matchers.equalTo(2)); + assertShardStats(response.getIndicesStats().getShards(), 2, 7, 5, 2.0 / 5); + + assertThat(response.getIndicesStats().getShards().getAvgIndexPrimaryShards(), Matchers.equalTo(2.5)); + assertThat(response.getIndicesStats().getShards().getMinIndexPrimaryShards(), Matchers.equalTo(2)); + assertThat(response.getIndicesStats().getShards().getMaxIndexPrimaryShards(), Matchers.equalTo(3)); + + assertThat(response.getIndicesStats().getShards().getAvgIndexShards(), Matchers.equalTo(3.5)); + assertThat(response.getIndicesStats().getShards().getMinIndexShards(), Matchers.equalTo(3)); + assertThat(response.getIndicesStats().getShards().getMaxIndexShards(), Matchers.equalTo(4)); + + assertThat(response.getIndicesStats().getShards().getAvgIndexReplication(), Matchers.equalTo(0.5)); + assertThat(response.getIndicesStats().getShards().getMinIndexReplication(), Matchers.equalTo(0.0)); + assertThat(response.getIndicesStats().getShards().getMaxIndexReplication(), Matchers.equalTo(1.0)); + + } + + public void testIndicesShardStatsWithNodeLevelAggregations() { internalCluster().startNode(); ensureGreen(); - ClusterStatsResponse response = client().admin().cluster().prepareClusterStats().get(); + ClusterStatsResponse response = client().admin().cluster().prepareClusterStats().useAggregatedNodeLevelResponses(true).get(); assertThat(response.getStatus(), Matchers.equalTo(ClusterHealthStatus.GREEN)); prepareCreate("test1").setSettings(Settings.builder().put("number_of_shards", 2).put("number_of_replicas", 1)).get(); - response = client().admin().cluster().prepareClusterStats().get(); + response = client().admin().cluster().prepareClusterStats().useAggregatedNodeLevelResponses(true).get(); assertThat(response.getStatus(), Matchers.equalTo(ClusterHealthStatus.YELLOW)); assertThat(response.indicesStats.getDocs().getCount(), Matchers.equalTo(0L)); assertThat(response.indicesStats.getIndexCount(), Matchers.equalTo(1)); @@ -195,14 +248,14 @@ public void testIndicesShardStats() throws ExecutionException, InterruptedExcept ensureGreen(); index("test1", "type", "1", "f", "f"); refresh(); // make the doc visible - response = client().admin().cluster().prepareClusterStats().get(); + response = client().admin().cluster().prepareClusterStats().useAggregatedNodeLevelResponses(true).get(); assertThat(response.getStatus(), Matchers.equalTo(ClusterHealthStatus.GREEN)); assertThat(response.indicesStats.getDocs().getCount(), Matchers.equalTo(1L)); assertShardStats(response.getIndicesStats().getShards(), 1, 4, 2, 1.0); prepareCreate("test2").setSettings(Settings.builder().put("number_of_shards", 3).put("number_of_replicas", 0)).get(); ensureGreen(); - response = client().admin().cluster().prepareClusterStats().get(); + response = client().admin().cluster().prepareClusterStats().useAggregatedNodeLevelResponses(true).get(); assertThat(response.getStatus(), Matchers.equalTo(ClusterHealthStatus.GREEN)); assertThat(response.indicesStats.getIndexCount(), Matchers.equalTo(2)); assertShardStats(response.getIndicesStats().getShards(), 2, 7, 5, 2.0 / 5); @@ -225,7 +278,11 @@ public void testValuesSmokeScreen() throws IOException, ExecutionException, Inte internalCluster().startNodes(randomIntBetween(1, 3)); index("test1", "type", "1", "f", "f"); - ClusterStatsResponse response = client().admin().cluster().prepareClusterStats().get(); + ClusterStatsResponse response = client().admin() + .cluster() + .prepareClusterStats() + .useAggregatedNodeLevelResponses(randomBoolean()) + .get(); String msg = response.toString(); assertThat(msg, response.getTimestamp(), Matchers.greaterThan(946681200000L)); // 1 Jan 2000 assertThat(msg, response.indicesStats.getStore().getSizeInBytes(), Matchers.greaterThan(0L)); @@ -265,13 +322,21 @@ public void testAllocatedProcessors() throws Exception { internalCluster().startNode(Settings.builder().put(OpenSearchExecutors.NODE_PROCESSORS_SETTING.getKey(), 7).build()); waitForNodes(1); - ClusterStatsResponse response = client().admin().cluster().prepareClusterStats().get(); + ClusterStatsResponse response = client().admin() + .cluster() + .prepareClusterStats() + .useAggregatedNodeLevelResponses(randomBoolean()) + .get(); assertThat(response.getNodesStats().getOs().getAllocatedProcessors(), equalTo(7)); } public void testClusterStatusWhenStateNotRecovered() throws Exception { internalCluster().startClusterManagerOnlyNode(Settings.builder().put("gateway.recover_after_nodes", 2).build()); - ClusterStatsResponse response = client().admin().cluster().prepareClusterStats().get(); + ClusterStatsResponse response = client().admin() + .cluster() + .prepareClusterStats() + .useAggregatedNodeLevelResponses(randomBoolean()) + .get(); assertThat(response.getStatus(), equalTo(ClusterHealthStatus.RED)); if (randomBoolean()) { @@ -281,14 +346,18 @@ public void testClusterStatusWhenStateNotRecovered() throws Exception { } // wait for the cluster status to settle ensureGreen(); - response = client().admin().cluster().prepareClusterStats().get(); + response = client().admin().cluster().prepareClusterStats().useAggregatedNodeLevelResponses(randomBoolean()).get(); assertThat(response.getStatus(), equalTo(ClusterHealthStatus.GREEN)); } public void testFieldTypes() { internalCluster().startNode(); ensureGreen(); - ClusterStatsResponse response = client().admin().cluster().prepareClusterStats().get(); + ClusterStatsResponse response = client().admin() + .cluster() + .prepareClusterStats() + .useAggregatedNodeLevelResponses(randomBoolean()) + .get(); assertThat(response.getStatus(), Matchers.equalTo(ClusterHealthStatus.GREEN)); assertTrue(response.getIndicesStats().getMappings().getFieldTypeStats().isEmpty()); @@ -301,7 +370,7 @@ public void testFieldTypes() { + "\"eggplant\":{\"type\":\"integer\"}}}}}" ) .get(); - response = client().admin().cluster().prepareClusterStats().get(); + response = client().admin().cluster().prepareClusterStats().useAggregatedNodeLevelResponses(randomBoolean()).get(); assertThat(response.getIndicesStats().getMappings().getFieldTypeStats().size(), equalTo(3)); Set stats = response.getIndicesStats().getMappings().getFieldTypeStats(); for (IndexFeatureStats stat : stats) { @@ -329,7 +398,11 @@ public void testNodeRolesWithMasterLegacySettings() throws ExecutionException, I Map expectedCounts = getExpectedCounts(0, 1, 1, 0, 1, 0, 0); Client client = client(); - ClusterStatsResponse clusterStatsResponse = client.admin().cluster().prepareClusterStats().get(); + ClusterStatsResponse clusterStatsResponse = client.admin() + .cluster() + .prepareClusterStats() + .useAggregatedNodeLevelResponses(randomBoolean()) + .get(); assertCounts(clusterStatsResponse.getNodesStats().getCounts(), total, expectedCounts); Set expectedRoles = Set.of( @@ -359,7 +432,11 @@ public void testNodeRolesWithClusterManagerRole() throws ExecutionException, Int Map expectedCounts = getExpectedCounts(0, 1, 1, 0, 1, 0, 0); Client client = client(); - ClusterStatsResponse clusterStatsResponse = client.admin().cluster().prepareClusterStats().get(); + ClusterStatsResponse clusterStatsResponse = client.admin() + .cluster() + .prepareClusterStats() + .useAggregatedNodeLevelResponses(randomBoolean()) + .get(); assertCounts(clusterStatsResponse.getNodesStats().getCounts(), total, expectedCounts); Set expectedRoles = Set.of( @@ -383,7 +460,11 @@ public void testNodeRolesWithSeedDataNodeLegacySettings() throws ExecutionExcept Map expectedRoleCounts = getExpectedCounts(1, 1, 1, 0, 1, 0, 0); Client client = client(); - ClusterStatsResponse clusterStatsResponse = client.admin().cluster().prepareClusterStats().get(); + ClusterStatsResponse clusterStatsResponse = client.admin() + .cluster() + .prepareClusterStats() + .useAggregatedNodeLevelResponses(randomBoolean()) + .get(); assertCounts(clusterStatsResponse.getNodesStats().getCounts(), total, expectedRoleCounts); Set expectedRoles = Set.of( @@ -410,7 +491,11 @@ public void testNodeRolesWithDataNodeLegacySettings() throws ExecutionException, Map expectedRoleCounts = getExpectedCounts(1, 1, 1, 0, 1, 0, 0); Client client = client(); - ClusterStatsResponse clusterStatsResponse = client.admin().cluster().prepareClusterStats().get(); + ClusterStatsResponse clusterStatsResponse = client.admin() + .cluster() + .prepareClusterStats() + .useAggregatedNodeLevelResponses(randomBoolean()) + .get(); assertCounts(clusterStatsResponse.getNodesStats().getCounts(), total, expectedRoleCounts); Set> expectedNodesRoles = Set.of( diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsIndices.java b/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsIndices.java index 26e554f44fca1..03a73f45ffe81 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsIndices.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsIndices.java @@ -78,26 +78,49 @@ public ClusterStatsIndices(List nodeResponses, Mapping this.segments = new SegmentsStats(); for (ClusterStatsNodeResponse r : nodeResponses) { - for (org.opensearch.action.admin.indices.stats.ShardStats shardStats : r.shardsStats()) { - ShardStats indexShardStats = countsPerIndex.get(shardStats.getShardRouting().getIndexName()); - if (indexShardStats == null) { - indexShardStats = new ShardStats(); - countsPerIndex.put(shardStats.getShardRouting().getIndexName(), indexShardStats); + // Aggregated response from the node + if (r.getAggregatedNodeLevelStats() != null) { + + for (Map.Entry entry : r.getAggregatedNodeLevelStats().indexStatsMap + .entrySet()) { + ShardStats indexShardStats = countsPerIndex.get(entry.getKey()); + if (indexShardStats == null) { + indexShardStats = new ShardStats(entry.getValue()); + countsPerIndex.put(entry.getKey(), indexShardStats); + } else { + indexShardStats.addStatsFrom(entry.getValue()); + } } - indexShardStats.total++; - - CommonStats shardCommonStats = shardStats.getStats(); - - if (shardStats.getShardRouting().primary()) { - indexShardStats.primaries++; - docs.add(shardCommonStats.docs); + docs.add(r.getAggregatedNodeLevelStats().commonStats.docs); + store.add(r.getAggregatedNodeLevelStats().commonStats.store); + fieldData.add(r.getAggregatedNodeLevelStats().commonStats.fieldData); + queryCache.add(r.getAggregatedNodeLevelStats().commonStats.queryCache); + completion.add(r.getAggregatedNodeLevelStats().commonStats.completion); + segments.add(r.getAggregatedNodeLevelStats().commonStats.segments); + } else { + // Default response from the node + for (org.opensearch.action.admin.indices.stats.ShardStats shardStats : r.shardsStats()) { + ShardStats indexShardStats = countsPerIndex.get(shardStats.getShardRouting().getIndexName()); + if (indexShardStats == null) { + indexShardStats = new ShardStats(); + countsPerIndex.put(shardStats.getShardRouting().getIndexName(), indexShardStats); + } + + indexShardStats.total++; + + CommonStats shardCommonStats = shardStats.getStats(); + + if (shardStats.getShardRouting().primary()) { + indexShardStats.primaries++; + docs.add(shardCommonStats.docs); + } + store.add(shardCommonStats.store); + fieldData.add(shardCommonStats.fieldData); + queryCache.add(shardCommonStats.queryCache); + completion.add(shardCommonStats.completion); + segments.add(shardCommonStats.segments); } - store.add(shardCommonStats.store); - fieldData.add(shardCommonStats.fieldData); - queryCache.add(shardCommonStats.queryCache); - completion.add(shardCommonStats.completion); - segments.add(shardCommonStats.segments); } } @@ -202,6 +225,11 @@ public static class ShardStats implements ToXContentFragment { public ShardStats() {} + public ShardStats(ClusterStatsNodeResponse.AggregatedIndexStats aggregatedIndexStats) { + this.total = aggregatedIndexStats.total; + this.primaries = aggregatedIndexStats.primaries; + } + /** * number of indices in the cluster */ @@ -329,6 +357,11 @@ public void addIndexShardCount(ShardStats indexShardCount) { } } + public void addStatsFrom(ClusterStatsNodeResponse.AggregatedIndexStats incomingStats) { + this.total += incomingStats.total; + this.primaries += incomingStats.primaries; + } + /** * Inner Fields used for creating XContent and parsing * diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsNodeResponse.java b/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsNodeResponse.java index 1b25bf84356d6..133cf68f5f8c9 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsNodeResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsNodeResponse.java @@ -32,17 +32,29 @@ package org.opensearch.action.admin.cluster.stats; +import org.opensearch.Version; import org.opensearch.action.admin.cluster.node.info.NodeInfo; import org.opensearch.action.admin.cluster.node.stats.NodeStats; +import org.opensearch.action.admin.indices.stats.CommonStats; import org.opensearch.action.admin.indices.stats.ShardStats; import org.opensearch.action.support.nodes.BaseNodeResponse; import org.opensearch.cluster.health.ClusterHealthStatus; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.common.Nullable; +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; +import org.opensearch.index.cache.query.QueryCacheStats; +import org.opensearch.index.engine.SegmentsStats; +import org.opensearch.index.fielddata.FieldDataStats; +import org.opensearch.index.shard.DocsStats; +import org.opensearch.index.store.StoreStats; +import org.opensearch.search.suggest.completion.CompletionStats; import java.io.IOException; +import java.util.HashMap; +import java.util.Map; /** * Transport action for obtaining cluster stats from node level @@ -55,6 +67,7 @@ public class ClusterStatsNodeResponse extends BaseNodeResponse { private final NodeStats nodeStats; private final ShardStats[] shardsStats; private ClusterHealthStatus clusterStatus; + private AggregatedNodeLevelStats aggregatedNodeLevelStats; public ClusterStatsNodeResponse(StreamInput in) throws IOException { super(in); @@ -64,7 +77,12 @@ public ClusterStatsNodeResponse(StreamInput in) throws IOException { } this.nodeInfo = new NodeInfo(in); this.nodeStats = new NodeStats(in); - shardsStats = in.readArray(ShardStats::new, ShardStats[]::new); + if (in.getVersion().onOrAfter(Version.V_3_0_0)) { + this.shardsStats = in.readOptionalArray(ShardStats::new, ShardStats[]::new); + this.aggregatedNodeLevelStats = in.readOptionalWriteable(AggregatedNodeLevelStats::new); + } else { + this.shardsStats = in.readArray(ShardStats::new, ShardStats[]::new); + } } public ClusterStatsNodeResponse( @@ -81,6 +99,24 @@ public ClusterStatsNodeResponse( this.clusterStatus = clusterStatus; } + public ClusterStatsNodeResponse( + DiscoveryNode node, + @Nullable ClusterHealthStatus clusterStatus, + NodeInfo nodeInfo, + NodeStats nodeStats, + ShardStats[] shardsStats, + boolean useAggregatedNodeLevelResponses + ) { + super(node); + this.nodeInfo = nodeInfo; + this.nodeStats = nodeStats; + if (useAggregatedNodeLevelResponses) { + this.aggregatedNodeLevelStats = new AggregatedNodeLevelStats(node, shardsStats); + } + this.shardsStats = shardsStats; + this.clusterStatus = clusterStatus; + } + public NodeInfo nodeInfo() { return this.nodeInfo; } @@ -101,6 +137,10 @@ public ShardStats[] shardsStats() { return this.shardsStats; } + public AggregatedNodeLevelStats getAggregatedNodeLevelStats() { + return aggregatedNodeLevelStats; + } + public static ClusterStatsNodeResponse readNodeResponse(StreamInput in) throws IOException { return new ClusterStatsNodeResponse(in); } @@ -116,6 +156,95 @@ public void writeTo(StreamOutput out) throws IOException { } nodeInfo.writeTo(out); nodeStats.writeTo(out); - out.writeArray(shardsStats); + if (out.getVersion().onOrAfter(Version.V_3_0_0)) { + if (aggregatedNodeLevelStats != null) { + out.writeOptionalArray(null); + out.writeOptionalWriteable(aggregatedNodeLevelStats); + } else { + out.writeOptionalArray(shardsStats); + out.writeOptionalWriteable(null); + } + } else { + out.writeArray(shardsStats); + } + } + + /** + * Node level statistics used for ClusterStatsIndices for _cluster/stats call. + */ + public class AggregatedNodeLevelStats extends BaseNodeResponse { + + CommonStats commonStats; + Map indexStatsMap; + + protected AggregatedNodeLevelStats(StreamInput in) throws IOException { + super(in); + commonStats = in.readOptionalWriteable(CommonStats::new); + indexStatsMap = in.readMap(StreamInput::readString, AggregatedIndexStats::new); + } + + protected AggregatedNodeLevelStats(DiscoveryNode node, ShardStats[] indexShardsStats) { + super(node); + this.commonStats = new CommonStats(); + this.commonStats.docs = new DocsStats(); + this.commonStats.store = new StoreStats(); + this.commonStats.fieldData = new FieldDataStats(); + this.commonStats.queryCache = new QueryCacheStats(); + this.commonStats.completion = new CompletionStats(); + this.commonStats.segments = new SegmentsStats(); + this.indexStatsMap = new HashMap<>(); + + // Index Level Stats + for (org.opensearch.action.admin.indices.stats.ShardStats shardStats : indexShardsStats) { + AggregatedIndexStats indexShardStats = this.indexStatsMap.get(shardStats.getShardRouting().getIndexName()); + if (indexShardStats == null) { + indexShardStats = new AggregatedIndexStats(); + this.indexStatsMap.put(shardStats.getShardRouting().getIndexName(), indexShardStats); + } + + indexShardStats.total++; + + CommonStats shardCommonStats = shardStats.getStats(); + + if (shardStats.getShardRouting().primary()) { + indexShardStats.primaries++; + this.commonStats.docs.add(shardCommonStats.docs); + } + this.commonStats.store.add(shardCommonStats.store); + this.commonStats.fieldData.add(shardCommonStats.fieldData); + this.commonStats.queryCache.add(shardCommonStats.queryCache); + this.commonStats.completion.add(shardCommonStats.completion); + this.commonStats.segments.add(shardCommonStats.segments); + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeOptionalWriteable(commonStats); + out.writeMap(indexStatsMap, StreamOutput::writeString, (stream, stats) -> stats.writeTo(stream)); + } + } + + /** + * Node level statistics used for ClusterStatsIndices for _cluster/stats call. + */ + @PublicApi(since = "2.16.0") + public static class AggregatedIndexStats implements Writeable { + public int total = 0; + public int primaries = 0; + + public AggregatedIndexStats(StreamInput in) throws IOException { + total = in.readVInt(); + primaries = in.readVInt(); + } + + public AggregatedIndexStats() {} + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(total); + out.writeVInt(primaries); + } } } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsRequest.java index 6a99451c596ed..fdeb82a3466f2 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsRequest.java @@ -32,6 +32,7 @@ package org.opensearch.action.admin.cluster.stats; +import org.opensearch.Version; import org.opensearch.action.support.nodes.BaseNodesRequest; import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; @@ -49,8 +50,13 @@ public class ClusterStatsRequest extends BaseNodesRequest { public ClusterStatsRequest(StreamInput in) throws IOException { super(in); + if (in.getVersion().onOrAfter(Version.V_3_0_0)) { + useAggregatedNodeLevelResponses = in.readOptionalBoolean(); + } } + private Boolean useAggregatedNodeLevelResponses = false; + /** * Get stats from nodes based on the nodes ids specified. If none are passed, stats * based on all nodes will be returned. @@ -59,9 +65,20 @@ public ClusterStatsRequest(String... nodesIds) { super(nodesIds); } + public boolean useAggregatedNodeLevelResponses() { + return useAggregatedNodeLevelResponses; + } + + public void useAggregatedNodeLevelResponses(boolean useAggregatedNodeLevelResponses) { + this.useAggregatedNodeLevelResponses = useAggregatedNodeLevelResponses; + } + @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); + if (out.getVersion().onOrAfter(Version.V_3_0_0)) { + out.writeOptionalBoolean(useAggregatedNodeLevelResponses); + } } } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsRequestBuilder.java b/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsRequestBuilder.java index 0dcb03dc26d0e..4d0932bd3927d 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsRequestBuilder.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsRequestBuilder.java @@ -50,4 +50,9 @@ public class ClusterStatsRequestBuilder extends NodesOperationRequestBuilder< public ClusterStatsRequestBuilder(OpenSearchClient client, ClusterStatsAction action) { super(client, action, new ClusterStatsRequest()); } + + public final ClusterStatsRequestBuilder useAggregatedNodeLevelResponses(boolean useAggregatedNodeLevelResponses) { + request.useAggregatedNodeLevelResponses(useAggregatedNodeLevelResponses); + return this; + } } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java index c7d03596a2a36..be7d41a7ba75e 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java @@ -212,8 +212,14 @@ protected ClusterStatsNodeResponse nodeOperation(ClusterStatsNodeRequest nodeReq clusterStatus = new ClusterStateHealth(clusterService.state()).getStatus(); } - return new ClusterStatsNodeResponse(nodeInfo.getNode(), clusterStatus, nodeInfo, nodeStats, shardsStats.toArray(new ShardStats[0])); - + return new ClusterStatsNodeResponse( + nodeInfo.getNode(), + clusterStatus, + nodeInfo, + nodeStats, + shardsStats.toArray(new ShardStats[0]), + nodeRequest.request.useAggregatedNodeLevelResponses() + ); } /** diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestClusterStatsAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestClusterStatsAction.java index 913db3c81e951..d4426a004af8e 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestClusterStatsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestClusterStatsAction.java @@ -67,6 +67,7 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC ClusterStatsRequest clusterStatsRequest = new ClusterStatsRequest().nodesIds(request.paramAsStringArray("nodeId", null)); clusterStatsRequest.timeout(request.param("timeout")); clusterStatsRequest.setIncludeDiscoveryNodes(false); + clusterStatsRequest.useAggregatedNodeLevelResponses(true); return channel -> client.admin().cluster().clusterStats(clusterStatsRequest, new NodesResponseRestListener<>(channel)); } diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/stats/ClusterStatsNodesTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/stats/ClusterStatsNodesTests.java index 40a30342b86b9..1c4a77905d73f 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/stats/ClusterStatsNodesTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/stats/ClusterStatsNodesTests.java @@ -32,16 +32,38 @@ package org.opensearch.action.admin.cluster.stats; +import org.opensearch.Build; +import org.opensearch.Version; import org.opensearch.action.admin.cluster.node.info.NodeInfo; import org.opensearch.action.admin.cluster.node.stats.NodeStats; import org.opensearch.action.admin.cluster.node.stats.NodeStatsTests; +import org.opensearch.action.admin.indices.stats.CommonStats; +import org.opensearch.action.admin.indices.stats.CommonStatsFlags; +import org.opensearch.action.admin.indices.stats.ShardStats; import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.ShardRoutingState; +import org.opensearch.cluster.routing.TestShardRouting; +import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.network.NetworkModule; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.index.Index; import org.opensearch.core.xcontent.MediaTypeRegistry; +import org.opensearch.index.cache.query.QueryCacheStats; +import org.opensearch.index.engine.SegmentsStats; +import org.opensearch.index.fielddata.FieldDataStats; +import org.opensearch.index.flush.FlushStats; +import org.opensearch.index.shard.DocsStats; +import org.opensearch.index.shard.IndexingStats; +import org.opensearch.index.shard.ShardPath; +import org.opensearch.index.store.StoreStats; +import org.opensearch.search.suggest.completion.CompletionStats; import org.opensearch.test.OpenSearchTestCase; import java.io.IOException; +import java.nio.file.Path; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.Iterator; @@ -158,6 +180,253 @@ public void testIngestStats() throws Exception { ); } + public void testMultiVersionScenarioWithAggregatedNodeLevelStats() { + // Assuming the default behavior will be the type of response expected from a node of version prior to version containing + // aggregated node level information + int numberOfNodes = randomIntBetween(1, 4); + Index testIndex = new Index("test-index", "_na_"); + + List defaultClusterStatsNodeResponses = new ArrayList<>(); + List aggregatedNodeLevelClusterStatsNodeResponses = new ArrayList<>(); + + for (int i = 0; i < numberOfNodes; i++) { + DiscoveryNode node = new DiscoveryNode("node-" + i, buildNewFakeTransportAddress(), Version.CURRENT); + CommonStats commonStats = createRandomCommonStats(); + ShardStats[] shardStats = createshardStats(node, testIndex, commonStats); + ClusterStatsNodeResponse customClusterStatsResponse = createClusterStatsNodeResponse(node, shardStats, testIndex, true, false); + ClusterStatsNodeResponse customNodeLevelAggregatedClusterStatsResponse = createClusterStatsNodeResponse( + node, + shardStats, + testIndex, + false, + true + ); + defaultClusterStatsNodeResponses.add(customClusterStatsResponse); + aggregatedNodeLevelClusterStatsNodeResponses.add(customNodeLevelAggregatedClusterStatsResponse); + } + + ClusterStatsIndices defaultClusterStatsIndices = new ClusterStatsIndices(defaultClusterStatsNodeResponses, null, null); + ClusterStatsIndices aggregatedNodeLevelClusterStatsIndices = new ClusterStatsIndices( + aggregatedNodeLevelClusterStatsNodeResponses, + null, + null + ); + + assertClusterStatsIndicesEqual(defaultClusterStatsIndices, aggregatedNodeLevelClusterStatsIndices); + } + + public void assertClusterStatsIndicesEqual(ClusterStatsIndices first, ClusterStatsIndices second) { + assertEquals(first.getIndexCount(), second.getIndexCount()); + + assertEquals(first.getShards().getIndices(), second.getShards().getIndices()); + assertEquals(first.getShards().getTotal(), second.getShards().getTotal()); + assertEquals(first.getShards().getPrimaries(), second.getShards().getPrimaries()); + assertEquals(first.getShards().getMinIndexShards(), second.getShards().getMaxIndexShards()); + assertEquals(first.getShards().getMinIndexPrimaryShards(), second.getShards().getMinIndexPrimaryShards()); + + // As AssertEquals with double is deprecated and can only be used to compare floating-point numbers + assertTrue(first.getShards().getReplication() == second.getShards().getReplication()); + assertTrue(first.getShards().getAvgIndexShards() == second.getShards().getAvgIndexShards()); + assertTrue(first.getShards().getMaxIndexPrimaryShards() == second.getShards().getMaxIndexPrimaryShards()); + assertTrue(first.getShards().getAvgIndexPrimaryShards() == second.getShards().getAvgIndexPrimaryShards()); + assertTrue(first.getShards().getMinIndexReplication() == second.getShards().getMinIndexReplication()); + assertTrue(first.getShards().getAvgIndexReplication() == second.getShards().getAvgIndexReplication()); + assertTrue(first.getShards().getMaxIndexReplication() == second.getShards().getMaxIndexReplication()); + + // Docs stats + assertEquals(first.getDocs().getAverageSizeInBytes(), second.getDocs().getAverageSizeInBytes()); + assertEquals(first.getDocs().getDeleted(), second.getDocs().getDeleted()); + assertEquals(first.getDocs().getCount(), second.getDocs().getCount()); + assertEquals(first.getDocs().getTotalSizeInBytes(), second.getDocs().getTotalSizeInBytes()); + + // Store Stats + assertEquals(first.getStore().getSizeInBytes(), second.getStore().getSizeInBytes()); + assertEquals(first.getStore().getSize(), second.getStore().getSize()); + assertEquals(first.getStore().getReservedSize(), second.getStore().getReservedSize()); + + // Query Cache + assertEquals(first.getQueryCache().getCacheCount(), second.getQueryCache().getCacheCount()); + assertEquals(first.getQueryCache().getCacheSize(), second.getQueryCache().getCacheSize()); + assertEquals(first.getQueryCache().getEvictions(), second.getQueryCache().getEvictions()); + assertEquals(first.getQueryCache().getHitCount(), second.getQueryCache().getHitCount()); + assertEquals(first.getQueryCache().getTotalCount(), second.getQueryCache().getTotalCount()); + assertEquals(first.getQueryCache().getMissCount(), second.getQueryCache().getMissCount()); + assertEquals(first.getQueryCache().getMemorySize(), second.getQueryCache().getMemorySize()); + assertEquals(first.getQueryCache().getMemorySizeInBytes(), second.getQueryCache().getMemorySizeInBytes()); + + // Completion Stats + assertEquals(first.getCompletion().getSizeInBytes(), second.getCompletion().getSizeInBytes()); + assertEquals(first.getCompletion().getSize(), second.getCompletion().getSize()); + + // Segment Stats + assertEquals(first.getSegments().getBitsetMemory(), second.getSegments().getBitsetMemory()); + assertEquals(first.getSegments().getCount(), second.getSegments().getCount()); + assertEquals(first.getSegments().getBitsetMemoryInBytes(), second.getSegments().getBitsetMemoryInBytes()); + assertEquals(first.getSegments().getFileSizes(), second.getSegments().getFileSizes()); + assertEquals(first.getSegments().getIndexWriterMemoryInBytes(), second.getSegments().getIndexWriterMemoryInBytes()); + assertEquals(first.getSegments().getVersionMapMemory(), second.getSegments().getVersionMapMemory()); + assertEquals(first.getSegments().getVersionMapMemoryInBytes(), second.getSegments().getVersionMapMemoryInBytes()); + } + + public void testNodeIndexShardStatsSuccessfulSerializationDeserialization() throws IOException { + Index testIndex = new Index("test-index", "_na_"); + + DiscoveryNode node = new DiscoveryNode("node", buildNewFakeTransportAddress(), Version.CURRENT); + CommonStats commonStats = createRandomCommonStats(); + ShardStats[] shardStats = createshardStats(node, testIndex, commonStats); + ClusterStatsNodeResponse aggregatedNodeLevelClusterStatsNodeResponse = createClusterStatsNodeResponse( + node, + shardStats, + testIndex, + false, + true + ); + + BytesStreamOutput out = new BytesStreamOutput(); + aggregatedNodeLevelClusterStatsNodeResponse.writeTo(out); + StreamInput in = out.bytes().streamInput(); + + ClusterStatsNodeResponse newClusterStatsNodeRequest = new ClusterStatsNodeResponse(in); + + ClusterStatsIndices beforeSerialization = new ClusterStatsIndices(List.of(aggregatedNodeLevelClusterStatsNodeResponse), null, null); + ClusterStatsIndices afterSerialization = new ClusterStatsIndices(List.of(newClusterStatsNodeRequest), null, null); + + assertClusterStatsIndicesEqual(beforeSerialization, afterSerialization); + + } + + private ClusterStatsNodeResponse createClusterStatsNodeResponse( + DiscoveryNode node, + ShardStats[] shardStats, + Index index, + boolean defaultBehavior, + boolean aggregateNodeLevelStats + ) { + NodeInfo nodeInfo = new NodeInfo( + Version.CURRENT, + Build.CURRENT, + node, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ); + + NodeStats nodeStats = new NodeStats( + node, + randomNonNegativeLong(), + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ); + if (defaultBehavior) { + return new ClusterStatsNodeResponse(node, null, nodeInfo, nodeStats, shardStats); + } else { + return new ClusterStatsNodeResponse(node, null, nodeInfo, nodeStats, shardStats, aggregateNodeLevelStats); + } + + } + + private CommonStats createRandomCommonStats() { + CommonStats commonStats = new CommonStats(CommonStatsFlags.NONE); + commonStats.docs = new DocsStats(randomLongBetween(0, 10000), randomLongBetween(0, 100), randomLongBetween(0, 1000)); + commonStats.store = new StoreStats(randomLongBetween(0, 100), randomLongBetween(0, 1000)); + commonStats.indexing = new IndexingStats(); + commonStats.completion = new CompletionStats(); + commonStats.flush = new FlushStats(randomLongBetween(0, 100), randomLongBetween(0, 100), randomLongBetween(0, 100)); + commonStats.fieldData = new FieldDataStats(randomLongBetween(0, 100), randomLongBetween(0, 100), null); + commonStats.queryCache = new QueryCacheStats( + randomLongBetween(0, 100), + randomLongBetween(0, 100), + randomLongBetween(0, 100), + randomLongBetween(0, 100), + randomLongBetween(0, 100) + ); + commonStats.segments = new SegmentsStats(); + + return commonStats; + } + + private ShardStats[] createshardStats(DiscoveryNode localNode, Index index, CommonStats commonStats) { + List shardStatsList = new ArrayList<>(); + for (int i = 0; i < 2; i++) { + ShardRoutingState shardRoutingState = ShardRoutingState.fromValue((byte) randomIntBetween(2, 3)); + ShardRouting shardRouting = TestShardRouting.newShardRouting( + index.getName(), + i, + localNode.getId(), + randomBoolean(), + shardRoutingState + ); + + Path path = createTempDir().resolve("indices") + .resolve(shardRouting.shardId().getIndex().getUUID()) + .resolve(String.valueOf(shardRouting.shardId().id())); + + ShardStats shardStats = new ShardStats( + shardRouting, + new ShardPath(false, path, path, shardRouting.shardId()), + commonStats, + null, + null, + null + ); + shardStatsList.add(shardStats); + } + + return shardStatsList.toArray(new ShardStats[0]); + } + + private class MockShardStats extends ClusterStatsIndices.ShardStats { + public boolean equals(ClusterStatsIndices.ShardStats shardStats) { + return this.getIndices() == shardStats.getIndices() + && this.getTotal() == shardStats.getTotal() + && this.getPrimaries() == shardStats.getPrimaries() + && this.getReplication() == shardStats.getReplication() + && this.getMaxIndexShards() == shardStats.getMaxIndexShards() + && this.getMinIndexShards() == shardStats.getMinIndexShards() + && this.getAvgIndexShards() == shardStats.getAvgIndexShards() + && this.getMaxIndexPrimaryShards() == shardStats.getMaxIndexPrimaryShards() + && this.getMinIndexPrimaryShards() == shardStats.getMinIndexPrimaryShards() + && this.getAvgIndexPrimaryShards() == shardStats.getAvgIndexPrimaryShards() + && this.getMinIndexReplication() == shardStats.getMinIndexReplication() + && this.getAvgIndexReplication() == shardStats.getAvgIndexReplication() + && this.getMaxIndexReplication() == shardStats.getMaxIndexReplication(); + } + } + private static NodeInfo createNodeInfo(String nodeId, String transportType, String httpType) { Settings.Builder settings = Settings.builder(); if (transportType != null) { From 349708198d01f205293d0ee5ca0bdae7b9ffd76a Mon Sep 17 00:00:00 2001 From: Gaurav Bafna <85113518+gbbafna@users.noreply.github.com> Date: Tue, 23 Jul 2024 21:57:47 +0530 Subject: [PATCH 28/37] Fix constraint bug which allows more primary shards than average primary shards per index (#14908) Signed-off-by: Gaurav Bafna --- .../opensearch/cluster/routing/allocation/ConstraintTypes.java | 2 +- .../cluster/routing/allocation/AllocationConstraintsTests.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/ConstraintTypes.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/ConstraintTypes.java index 08fe8f92d1f80..28ad199218884 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/ConstraintTypes.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/ConstraintTypes.java @@ -70,7 +70,7 @@ public static Predicate isPerIndexPrimaryShardsPerN return (params) -> { int perIndexPrimaryShardCount = params.getNode().numPrimaryShards(params.getIndex()); int perIndexAllowedPrimaryShardCount = (int) Math.ceil(params.getBalancer().avgPrimaryShardsPerNode(params.getIndex())); - return perIndexPrimaryShardCount > perIndexAllowedPrimaryShardCount; + return perIndexPrimaryShardCount >= perIndexAllowedPrimaryShardCount; }; } diff --git a/server/src/test/java/org/opensearch/cluster/routing/allocation/AllocationConstraintsTests.java b/server/src/test/java/org/opensearch/cluster/routing/allocation/AllocationConstraintsTests.java index 90546620e9e3e..4c9fcd1650664 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/allocation/AllocationConstraintsTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/allocation/AllocationConstraintsTests.java @@ -93,7 +93,7 @@ public void testPerIndexPrimaryShardsConstraint() { assertEquals(0, constraints.weight(balancer, node, indexName)); - perIndexPrimaryShardCount = 3; + perIndexPrimaryShardCount = 2; when(node.numPrimaryShards(anyString())).thenReturn(perIndexPrimaryShardCount); assertEquals(CONSTRAINT_WEIGHT, constraints.weight(balancer, node, indexName)); From e46d1d8685a9b90a1f25920989e567373ee23284 Mon Sep 17 00:00:00 2001 From: rishavz_sagar Date: Tue, 23 Jul 2024 22:27:45 +0530 Subject: [PATCH 29/37] Optmising AwarenessAllocationDecider for hashmap.get call (#14761) Signed-off-by: RS146BIJAY --- .../decider/AwarenessAllocationDecider.java | 91 ++++++++++++------- 1 file changed, 58 insertions(+), 33 deletions(-) diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/AwarenessAllocationDecider.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/AwarenessAllocationDecider.java index 5344d95b217a7..16c94acfbb553 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/AwarenessAllocationDecider.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/AwarenessAllocationDecider.java @@ -111,7 +111,6 @@ public class AwarenessAllocationDecider extends AllocationDecider { ); private volatile List awarenessAttributes; - private volatile Map> forcedAwarenessAttributes; public AwarenessAllocationDecider(Settings settings, ClusterSettings clusterSettings) { @@ -163,8 +162,8 @@ private Decision underCapacity(ShardRouting shardRouting, RoutingNode node, Rout IndexMetadata indexMetadata = allocation.metadata().getIndexSafe(shardRouting.index()); int shardCount = indexMetadata.getNumberOfReplicas() + 1; // 1 for primary for (String awarenessAttribute : awarenessAttributes) { - // the node the shard exists on must be associated with an awareness attribute - if (node.node().getAttributes().containsKey(awarenessAttribute) == false) { + // the node the shard exists on must be associated with an awareness attribute. + if (isAwarenessAttributeAssociatedWithNode(node, awarenessAttribute) == false) { return allocation.decision( Decision.NO, NAME, @@ -175,36 +174,10 @@ private Decision underCapacity(ShardRouting shardRouting, RoutingNode node, Rout ); } + int currentNodeCount = getCurrentNodeCountForAttribute(shardRouting, node, allocation, moveToNode, awarenessAttribute); + // build attr_value -> nodes map Set nodesPerAttribute = allocation.routingNodes().nodesPerAttributesCounts(awarenessAttribute); - - // build the count of shards per attribute value - Map shardPerAttribute = new HashMap<>(); - for (ShardRouting assignedShard : allocation.routingNodes().assignedShards(shardRouting.shardId())) { - if (assignedShard.started() || assignedShard.initializing()) { - // Note: this also counts relocation targets as that will be the new location of the shard. - // Relocation sources should not be counted as the shard is moving away - RoutingNode routingNode = allocation.routingNodes().node(assignedShard.currentNodeId()); - shardPerAttribute.merge(routingNode.node().getAttributes().get(awarenessAttribute), 1, Integer::sum); - } - } - - if (moveToNode) { - if (shardRouting.assignedToNode()) { - String nodeId = shardRouting.relocating() ? shardRouting.relocatingNodeId() : shardRouting.currentNodeId(); - if (node.nodeId().equals(nodeId) == false) { - // we work on different nodes, move counts around - shardPerAttribute.compute( - allocation.routingNodes().node(nodeId).node().getAttributes().get(awarenessAttribute), - (k, v) -> (v == null) ? 0 : v - 1 - ); - shardPerAttribute.merge(node.node().getAttributes().get(awarenessAttribute), 1, Integer::sum); - } - } else { - shardPerAttribute.merge(node.node().getAttributes().get(awarenessAttribute), 1, Integer::sum); - } - } - int numberOfAttributes = nodesPerAttribute.size(); List fullValues = forcedAwarenessAttributes.get(awarenessAttribute); @@ -216,9 +189,8 @@ private Decision underCapacity(ShardRouting shardRouting, RoutingNode node, Rout } numberOfAttributes = attributesSet.size(); } - // TODO should we remove ones that are not part of full list? - final int currentNodeCount = shardPerAttribute.get(node.node().getAttributes().get(awarenessAttribute)); + // TODO should we remove ones that are not part of full list? final int maximumNodeCount = (shardCount + numberOfAttributes - 1) / numberOfAttributes; // ceil(shardCount/numberOfAttributes) if (currentNodeCount > maximumNodeCount) { return allocation.decision( @@ -238,4 +210,57 @@ private Decision underCapacity(ShardRouting shardRouting, RoutingNode node, Rout return allocation.decision(Decision.YES, NAME, "node meets all awareness attribute requirements"); } + + private int getCurrentNodeCountForAttribute( + ShardRouting shardRouting, + RoutingNode node, + RoutingAllocation allocation, + boolean moveToNode, + String awarenessAttribute + ) { + // build the count of shards per attribute value + final String shardAttributeForNode = getAttributeValueForNode(node, awarenessAttribute); + int currentNodeCount = 0; + final List assignedShards = allocation.routingNodes().assignedShards(shardRouting.shardId()); + + for (ShardRouting assignedShard : assignedShards) { + if (assignedShard.started() || assignedShard.initializing()) { + // Note: this also counts relocation targets as that will be the new location of the shard. + // Relocation sources should not be counted as the shard is moving away + RoutingNode routingNode = allocation.routingNodes().node(assignedShard.currentNodeId()); + // Increase node count when + if (getAttributeValueForNode(routingNode, awarenessAttribute).equals(shardAttributeForNode)) { + ++currentNodeCount; + } + } + } + + if (moveToNode) { + if (shardRouting.assignedToNode()) { + String nodeId = shardRouting.relocating() ? shardRouting.relocatingNodeId() : shardRouting.currentNodeId(); + if (node.nodeId().equals(nodeId) == false) { + // we work on different nodes, move counts around + if (getAttributeValueForNode(allocation.routingNodes().node(nodeId), awarenessAttribute).equals(shardAttributeForNode) + && currentNodeCount > 0) { + --currentNodeCount; + } + + ++currentNodeCount; + } + } else { + ++currentNodeCount; + } + } + + return currentNodeCount; + } + + private boolean isAwarenessAttributeAssociatedWithNode(RoutingNode node, String awarenessAttribute) { + return node.node().getAttributes().containsKey(awarenessAttribute); + } + + private String getAttributeValueForNode(final RoutingNode node, final String awarenessAttribute) { + return node.node().getAttributes().get(awarenessAttribute); + } + } From 087355f0ee676064ea409ed68090b33e568ea941 Mon Sep 17 00:00:00 2001 From: Andrew Ross Date: Tue, 23 Jul 2024 14:26:22 -0500 Subject: [PATCH 30/37] Fix IngestServiceTests.testBulkRequestExecutionWithFailures (#14918) The test would previously fail if the randomness led to only a single indexing request being included in the bulk payload. This change guarantees multiple indexing requests in order to ensure the batch logic kicks in. Also replace some unneeded mocks with real classes. Signed-off-by: Andrew Ross --- .../opensearch/ingest/IngestServiceTests.java | 47 +++++++++---------- 1 file changed, 22 insertions(+), 25 deletions(-) diff --git a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java index 9d03127692975..166b94966196c 100644 --- a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java +++ b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java @@ -78,6 +78,7 @@ import org.opensearch.test.OpenSearchTestCase; import org.opensearch.threadpool.ThreadPool; import org.opensearch.threadpool.ThreadPool.Names; +import org.hamcrest.MatcherAssert; import org.junit.Before; import java.nio.charset.StandardCharsets; @@ -104,15 +105,16 @@ import static java.util.Collections.emptyMap; import static java.util.Collections.emptySet; +import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; import static org.hamcrest.Matchers.sameInstance; import static org.mockito.Mockito.any; -import static org.mockito.Mockito.anyInt; import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.argThat; import static org.mockito.Mockito.doAnswer; @@ -1106,27 +1108,23 @@ public void testExecuteFailureWithNestedOnFailure() throws Exception { verify(completionHandler, times(1)).accept(Thread.currentThread(), null); } - public void testBulkRequestExecutionWithFailures() throws Exception { + public void testBulkRequestExecutionWithFailures() { BulkRequest bulkRequest = new BulkRequest(); String pipelineId = "_id"; - int numRequest = scaledRandomIntBetween(8, 64); - int numIndexRequests = 0; - for (int i = 0; i < numRequest; i++) { - DocWriteRequest request; + int numIndexRequests = scaledRandomIntBetween(4, 32); + for (int i = 0; i < numIndexRequests; i++) { + IndexRequest indexRequest = new IndexRequest("_index").id("_id").setPipeline(pipelineId).setFinalPipeline("_none"); + indexRequest.source(Requests.INDEX_CONTENT_TYPE, "field1", "value1"); + bulkRequest.add(indexRequest); + } + int numOtherRequests = scaledRandomIntBetween(4, 32); + for (int i = 0; i < numOtherRequests; i++) { if (randomBoolean()) { - if (randomBoolean()) { - request = new DeleteRequest("_index", "_id"); - } else { - request = new UpdateRequest("_index", "_id"); - } + bulkRequest.add(new DeleteRequest("_index", "_id")); } else { - IndexRequest indexRequest = new IndexRequest("_index").id("_id").setPipeline(pipelineId).setFinalPipeline("_none"); - indexRequest.source(Requests.INDEX_CONTENT_TYPE, "field1", "value1"); - request = indexRequest; - numIndexRequests++; + bulkRequest.add(new UpdateRequest("_index", "_id")); } - bulkRequest.add(request); } CompoundProcessor processor = mock(CompoundProcessor.class); @@ -1155,23 +1153,22 @@ public void testBulkRequestExecutionWithFailures() throws Exception { clusterState = IngestService.innerPut(putRequest, clusterState); ingestService.applyClusterState(new ClusterChangedEvent("", clusterState, previousClusterState)); - @SuppressWarnings("unchecked") - BiConsumer requestItemErrorHandler = mock(BiConsumer.class); - @SuppressWarnings("unchecked") - final BiConsumer completionHandler = mock(BiConsumer.class); + final Map errorHandler = new HashMap<>(); + final Map completionHandler = new HashMap<>(); ingestService.executeBulkRequest( - numRequest, + numIndexRequests + numOtherRequests, bulkRequest.requests(), - requestItemErrorHandler, - completionHandler, + errorHandler::put, + completionHandler::put, indexReq -> {}, Names.WRITE, bulkRequest ); - verify(requestItemErrorHandler, times(numIndexRequests)).accept(anyInt(), argThat(o -> o.getCause().equals(error))); + MatcherAssert.assertThat(errorHandler.entrySet(), hasSize(numIndexRequests)); + errorHandler.values().forEach(e -> assertEquals(e.getCause(), error)); - verify(completionHandler, times(1)).accept(Thread.currentThread(), null); + MatcherAssert.assertThat(completionHandler.keySet(), contains(Thread.currentThread())); } public void testBulkRequestExecution() throws Exception { From 312de9947b8848150743623009e8d4b95487e911 Mon Sep 17 00:00:00 2001 From: Bharathwaj G Date: Wed, 24 Jul 2024 08:54:27 +0530 Subject: [PATCH 31/37] [Star tree] Star tree merge changes (#14652) --------- Signed-off-by: Bharathwaj G --- .../composite/Composite99DocValuesReader.java | 10 +- .../composite/Composite99DocValuesWriter.java | 97 +- .../composite/CompositeIndexFieldInfo.java | 37 + .../codec/composite/CompositeIndexReader.java | 5 +- .../datacube/startree/StarTreeValues.java | 47 +- .../aggregators/CountValueAggregator.java | 11 +- .../aggregators/MetricAggregatorInfo.java | 21 +- .../aggregators/SumValueAggregator.java | 17 +- .../startree/aggregators/ValueAggregator.java | 6 +- .../aggregators/ValueAggregatorFactory.java | 9 +- .../startree/builder/BaseStarTreeBuilder.java | 258 +- .../builder/OnHeapStarTreeBuilder.java | 148 +- .../startree/builder/StarTreeBuilder.java | 18 +- .../StarTreeDocValuesIteratorAdapter.java | 82 - .../startree/builder/StarTreesBuilder.java | 61 +- .../datacube/startree/node/StarTreeNode.java | 112 + .../datacube/startree/node/package-info.java | 12 + .../utils/SequentialDocValuesIterator.java | 109 +- .../mapper/CompositeMappedFieldType.java | 4 + .../StarTreeDocValuesFormatTests.java | 172 +- .../CountValueAggregatorTests.java | 8 +- .../MetricAggregatorInfoTests.java | 34 +- .../aggregators/SumValueAggregatorTests.java | 15 +- .../ValueAggregatorFactoryTests.java | 2 +- .../builder/AbstractStarTreeBuilderTests.java | 2251 +++++++++++++++++ .../builder/BaseStarTreeBuilderTests.java | 25 +- .../builder/OnHeapStarTreeBuilderTests.java | 696 +---- ...StarTreeDocValuesIteratorAdapterTests.java | 139 - .../StarTreeValuesIteratorFactoryTests.java | 131 - .../builder/StarTreesBuilderTests.java | 14 +- .../SequentialDocValuesIteratorTests.java | 131 +- .../org/opensearch/index/MapperTestUtils.java | 34 + 32 files changed, 3281 insertions(+), 1435 deletions(-) create mode 100644 server/src/main/java/org/opensearch/index/codec/composite/CompositeIndexFieldInfo.java delete mode 100644 server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeDocValuesIteratorAdapter.java create mode 100644 server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNode.java create mode 100644 server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/package-info.java create mode 100644 server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java delete mode 100644 server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeDocValuesIteratorAdapterTests.java delete mode 100644 server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeValuesIteratorFactoryTests.java diff --git a/server/src/main/java/org/opensearch/index/codec/composite/Composite99DocValuesReader.java b/server/src/main/java/org/opensearch/index/codec/composite/Composite99DocValuesReader.java index 82c844088cfd4..df5008a7f294e 100644 --- a/server/src/main/java/org/opensearch/index/codec/composite/Composite99DocValuesReader.java +++ b/server/src/main/java/org/opensearch/index/codec/composite/Composite99DocValuesReader.java @@ -17,9 +17,9 @@ import org.apache.lucene.index.SortedNumericDocValues; import org.apache.lucene.index.SortedSetDocValues; import org.opensearch.common.annotation.ExperimentalApi; -import org.opensearch.index.mapper.CompositeMappedFieldType; import java.io.IOException; +import java.util.ArrayList; import java.util.List; /** @@ -74,15 +74,13 @@ public void close() throws IOException { } @Override - public List getCompositeIndexFields() { + public List getCompositeIndexFields() { // todo : read from file formats and get the field names. - throw new UnsupportedOperationException(); - + return new ArrayList<>(); } @Override - public CompositeIndexValues getCompositeIndexValues(String field, CompositeMappedFieldType.CompositeFieldType fieldType) - throws IOException { + public CompositeIndexValues getCompositeIndexValues(CompositeIndexFieldInfo compositeIndexFieldInfo) throws IOException { // TODO : read compositeIndexValues [starTreeValues] from star tree files throw new UnsupportedOperationException(); } diff --git a/server/src/main/java/org/opensearch/index/codec/composite/Composite99DocValuesWriter.java b/server/src/main/java/org/opensearch/index/codec/composite/Composite99DocValuesWriter.java index 3753b20a8bea3..3859d3c998573 100644 --- a/server/src/main/java/org/opensearch/index/codec/composite/Composite99DocValuesWriter.java +++ b/server/src/main/java/org/opensearch/index/codec/composite/Composite99DocValuesWriter.java @@ -8,20 +8,29 @@ package org.opensearch.index.codec.composite; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.lucene.codecs.DocValuesConsumer; import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.DocValuesType; +import org.apache.lucene.index.EmptyDocValuesProducer; import org.apache.lucene.index.FieldInfo; import org.apache.lucene.index.MergeState; import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.SortedNumericDocValues; import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.index.codec.composite.datacube.startree.StarTreeValues; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; import org.opensearch.index.compositeindex.datacube.startree.builder.StarTreesBuilder; import org.opensearch.index.mapper.CompositeMappedFieldType; import org.opensearch.index.mapper.MapperService; -import org.opensearch.index.mapper.StarTreeMapper; import java.io.IOException; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicReference; @@ -40,8 +49,10 @@ public class Composite99DocValuesWriter extends DocValuesConsumer { AtomicReference mergeState = new AtomicReference<>(); private final Set compositeMappedFieldTypes; private final Set compositeFieldSet; + private final Set segmentFieldSet; private final Map fieldProducerMap = new HashMap<>(); + private static final Logger logger = LogManager.getLogger(Composite99DocValuesWriter.class); public Composite99DocValuesWriter(DocValuesConsumer delegate, SegmentWriteState segmentWriteState, MapperService mapperService) { @@ -50,6 +61,12 @@ public Composite99DocValuesWriter(DocValuesConsumer delegate, SegmentWriteState this.mapperService = mapperService; this.compositeMappedFieldTypes = mapperService.getCompositeFieldTypes(); compositeFieldSet = new HashSet<>(); + segmentFieldSet = new HashSet<>(); + for (FieldInfo fi : segmentWriteState.fieldInfos) { + if (DocValuesType.SORTED_NUMERIC.equals(fi.getDocValuesType())) { + segmentFieldSet.add(fi.name); + } + } for (CompositeMappedFieldType type : compositeMappedFieldTypes) { compositeFieldSet.addAll(type.fields()); } @@ -95,23 +112,91 @@ private void createCompositeIndicesIfPossible(DocValuesProducer valuesProducer, fieldProducerMap.put(field.name, valuesProducer); compositeFieldSet.remove(field.name); } + segmentFieldSet.remove(field.name); + if (segmentFieldSet.isEmpty()) { + Set compositeFieldSetCopy = new HashSet<>(compositeFieldSet); + for (String compositeField : compositeFieldSetCopy) { + fieldProducerMap.put(compositeField, new EmptyDocValuesProducer() { + @Override + public SortedNumericDocValues getSortedNumeric(FieldInfo field) { + return DocValues.emptySortedNumeric(); + } + }); + compositeFieldSet.remove(compositeField); + } + } // we have all the required fields to build composite fields if (compositeFieldSet.isEmpty()) { for (CompositeMappedFieldType mappedType : compositeMappedFieldTypes) { - if (mappedType instanceof StarTreeMapper.StarTreeFieldType) { - try (StarTreesBuilder starTreesBuilder = new StarTreesBuilder(fieldProducerMap, state, mapperService)) { - starTreesBuilder.build(); + if (mappedType.getCompositeIndexType().equals(CompositeMappedFieldType.CompositeFieldType.STAR_TREE)) { + try (StarTreesBuilder starTreesBuilder = new StarTreesBuilder(state, mapperService)) { + starTreesBuilder.build(fieldProducerMap); } } } } + } @Override public void merge(MergeState mergeState) throws IOException { this.mergeState.compareAndSet(null, mergeState); super.merge(mergeState); - // TODO : handle merge star tree - // mergeStarTreeFields(mergeState); + mergeCompositeFields(mergeState); + } + + /** + * Merges composite fields from multiple segments + * @param mergeState merge state + */ + private void mergeCompositeFields(MergeState mergeState) throws IOException { + mergeStarTreeFields(mergeState); + } + + /** + * Merges star tree data fields from multiple segments + * @param mergeState merge state + */ + private void mergeStarTreeFields(MergeState mergeState) throws IOException { + Map> starTreeSubsPerField = new HashMap<>(); + StarTreeField starTreeField = null; + for (int i = 0; i < mergeState.docValuesProducers.length; i++) { + CompositeIndexReader reader = null; + if (mergeState.docValuesProducers[i] == null) { + continue; + } + if (mergeState.docValuesProducers[i] instanceof CompositeIndexReader) { + reader = (CompositeIndexReader) mergeState.docValuesProducers[i]; + } else { + continue; + } + + List compositeFieldInfo = reader.getCompositeIndexFields(); + for (CompositeIndexFieldInfo fieldInfo : compositeFieldInfo) { + if (fieldInfo.getType().equals(CompositeMappedFieldType.CompositeFieldType.STAR_TREE)) { + CompositeIndexValues compositeIndexValues = reader.getCompositeIndexValues(fieldInfo); + if (compositeIndexValues instanceof StarTreeValues) { + StarTreeValues starTreeValues = (StarTreeValues) compositeIndexValues; + List fieldsList = starTreeSubsPerField.getOrDefault(fieldInfo.getField(), Collections.emptyList()); + if (starTreeField == null) { + starTreeField = starTreeValues.getStarTreeField(); + } + // assert star tree configuration is same across segments + else { + if (starTreeField.equals(starTreeValues.getStarTreeField()) == false) { + throw new IllegalArgumentException( + "star tree field configuration must match the configuration of the field being merged" + ); + } + } + fieldsList.add(starTreeValues); + starTreeSubsPerField.put(fieldInfo.getField(), fieldsList); + } + } + } + } + try (StarTreesBuilder starTreesBuilder = new StarTreesBuilder(state, mapperService)) { + starTreesBuilder.buildDuringMerge(starTreeSubsPerField); + } } } diff --git a/server/src/main/java/org/opensearch/index/codec/composite/CompositeIndexFieldInfo.java b/server/src/main/java/org/opensearch/index/codec/composite/CompositeIndexFieldInfo.java new file mode 100644 index 0000000000000..8193fcc301e67 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/codec/composite/CompositeIndexFieldInfo.java @@ -0,0 +1,37 @@ + +/* + * 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.codec.composite; + +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.index.mapper.CompositeMappedFieldType; + +/** + * Field info details of composite index fields + * + * @opensearch.experimental + */ +@ExperimentalApi +public class CompositeIndexFieldInfo { + private final String field; + private final CompositeMappedFieldType.CompositeFieldType type; + + public CompositeIndexFieldInfo(String field, CompositeMappedFieldType.CompositeFieldType type) { + this.field = field; + this.type = type; + } + + public String getField() { + return field; + } + + public CompositeMappedFieldType.CompositeFieldType getType() { + return type; + } +} diff --git a/server/src/main/java/org/opensearch/index/codec/composite/CompositeIndexReader.java b/server/src/main/java/org/opensearch/index/codec/composite/CompositeIndexReader.java index d02438b75377d..a159b0619bcbb 100644 --- a/server/src/main/java/org/opensearch/index/codec/composite/CompositeIndexReader.java +++ b/server/src/main/java/org/opensearch/index/codec/composite/CompositeIndexReader.java @@ -9,7 +9,6 @@ package org.opensearch.index.codec.composite; import org.opensearch.common.annotation.ExperimentalApi; -import org.opensearch.index.mapper.CompositeMappedFieldType; import java.io.IOException; import java.util.List; @@ -25,10 +24,10 @@ public interface CompositeIndexReader { * Get list of composite index fields from the segment * */ - List getCompositeIndexFields(); + List getCompositeIndexFields(); /** * Get composite index values based on the field name and the field type */ - CompositeIndexValues getCompositeIndexValues(String field, CompositeMappedFieldType.CompositeFieldType fieldType) throws IOException; + CompositeIndexValues getCompositeIndexValues(CompositeIndexFieldInfo fieldInfo) throws IOException; } diff --git a/server/src/main/java/org/opensearch/index/codec/composite/datacube/startree/StarTreeValues.java b/server/src/main/java/org/opensearch/index/codec/composite/datacube/startree/StarTreeValues.java index 2a5b96ce2620a..8378a4063b7ca 100644 --- a/server/src/main/java/org/opensearch/index/codec/composite/datacube/startree/StarTreeValues.java +++ b/server/src/main/java/org/opensearch/index/codec/composite/datacube/startree/StarTreeValues.java @@ -8,10 +8,13 @@ package org.opensearch.index.codec.composite.datacube.startree; +import org.apache.lucene.search.DocIdSetIterator; import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.index.codec.composite.CompositeIndexValues; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; +import org.opensearch.index.compositeindex.datacube.startree.node.StarTreeNode; -import java.util.List; +import java.util.Map; /** * Concrete class that holds the star tree associated values from the segment @@ -20,16 +23,48 @@ */ @ExperimentalApi public class StarTreeValues implements CompositeIndexValues { - private final List dimensionsOrder; + private final StarTreeField starTreeField; + private final StarTreeNode root; + private final Map dimensionDocValuesIteratorMap; + private final Map metricDocValuesIteratorMap; + private final Map attributes; - // TODO : come up with full set of vales such as dimensions and metrics doc values + star tree - public StarTreeValues(List dimensionsOrder) { - super(); - this.dimensionsOrder = List.copyOf(dimensionsOrder); + public StarTreeValues( + StarTreeField starTreeField, + StarTreeNode root, + Map dimensionDocValuesIteratorMap, + Map metricDocValuesIteratorMap, + Map attributes + ) { + this.starTreeField = starTreeField; + this.root = root; + this.dimensionDocValuesIteratorMap = dimensionDocValuesIteratorMap; + this.metricDocValuesIteratorMap = metricDocValuesIteratorMap; + this.attributes = attributes; } @Override public CompositeIndexValues getValues() { return this; } + + public StarTreeField getStarTreeField() { + return starTreeField; + } + + public StarTreeNode getRoot() { + return root; + } + + public Map getDimensionDocValuesIteratorMap() { + return dimensionDocValuesIteratorMap; + } + + public Map getMetricDocValuesIteratorMap() { + return metricDocValuesIteratorMap; + } + + public Map getAttributes() { + return attributes; + } } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/CountValueAggregator.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/CountValueAggregator.java index d72f4a292dc0a..5390b6728b9b6 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/CountValueAggregator.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/CountValueAggregator.java @@ -18,6 +18,11 @@ public class CountValueAggregator implements ValueAggregator { public static final StarTreeNumericType VALUE_AGGREGATOR_TYPE = StarTreeNumericType.LONG; public static final long DEFAULT_INITIAL_VALUE = 1L; + private StarTreeNumericType starTreeNumericType; + + public CountValueAggregator(StarTreeNumericType starTreeNumericType) { + this.starTreeNumericType = starTreeNumericType; + } @Override public MetricStat getAggregationType() { @@ -30,12 +35,12 @@ public StarTreeNumericType getAggregatedValueType() { } @Override - public Long getInitialAggregatedValueForSegmentDocValue(Long segmentDocValue, StarTreeNumericType starTreeNumericType) { + public Long getInitialAggregatedValueForSegmentDocValue(Long segmentDocValue) { return DEFAULT_INITIAL_VALUE; } @Override - public Long mergeAggregatedValueAndSegmentValue(Long value, Long segmentDocValue, StarTreeNumericType starTreeNumericType) { + public Long mergeAggregatedValueAndSegmentValue(Long value, Long segmentDocValue) { return value + 1; } @@ -60,7 +65,7 @@ public Long toLongValue(Long value) { } @Override - public Long toStarTreeNumericTypeValue(Long value, StarTreeNumericType type) { + public Long toStarTreeNumericTypeValue(Long value) { return value; } } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MetricAggregatorInfo.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MetricAggregatorInfo.java index 46f1b1ac11063..a9209a38eca82 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MetricAggregatorInfo.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MetricAggregatorInfo.java @@ -9,7 +9,6 @@ import org.opensearch.index.compositeindex.datacube.MetricStat; import org.opensearch.index.compositeindex.datacube.startree.aggregators.numerictype.StarTreeNumericType; -import org.opensearch.index.compositeindex.datacube.startree.utils.SequentialDocValuesIterator; import org.opensearch.index.fielddata.IndexNumericFieldData; import java.util.Comparator; @@ -17,7 +16,6 @@ /** * Builds aggregation function and doc values field pair to support various aggregations - * * @opensearch.experimental */ public class MetricAggregatorInfo implements Comparable { @@ -29,22 +27,14 @@ public class MetricAggregatorInfo implements Comparable { private final String field; private final ValueAggregator valueAggregators; private final StarTreeNumericType starTreeNumericType; - private final SequentialDocValuesIterator metricStatReader; /** * Constructor for MetricAggregatorInfo */ - public MetricAggregatorInfo( - MetricStat metricStat, - String field, - String starFieldName, - IndexNumericFieldData.NumericType numericType, - SequentialDocValuesIterator metricStatReader - ) { + public MetricAggregatorInfo(MetricStat metricStat, String field, String starFieldName, IndexNumericFieldData.NumericType numericType) { this.metricStat = metricStat; - this.valueAggregators = ValueAggregatorFactory.getValueAggregator(metricStat); this.starTreeNumericType = StarTreeNumericType.fromNumericType(numericType); - this.metricStatReader = metricStatReader; + this.valueAggregators = ValueAggregatorFactory.getValueAggregator(metricStat, this.starTreeNumericType); this.field = field; this.starFieldName = starFieldName; this.metric = toFieldName(); @@ -85,13 +75,6 @@ public StarTreeNumericType getAggregatedValueType() { return starTreeNumericType; } - /** - * @return metric value reader iterator - */ - public SequentialDocValuesIterator getMetricStatReader() { - return metricStatReader; - } - /** * @return field name with metric type and field */ diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/SumValueAggregator.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/SumValueAggregator.java index 543b0f7f42374..385549216e4d6 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/SumValueAggregator.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/SumValueAggregator.java @@ -24,6 +24,12 @@ public class SumValueAggregator implements ValueAggregator { private double compensation = 0; private CompensatedSum kahanSummation = new CompensatedSum(0, 0); + private StarTreeNumericType starTreeNumericType; + + public SumValueAggregator(StarTreeNumericType starTreeNumericType) { + this.starTreeNumericType = starTreeNumericType; + } + @Override public MetricStat getAggregationType() { return MetricStat.SUM; @@ -35,7 +41,7 @@ public StarTreeNumericType getAggregatedValueType() { } @Override - public Double getInitialAggregatedValueForSegmentDocValue(Long segmentDocValue, StarTreeNumericType starTreeNumericType) { + public Double getInitialAggregatedValueForSegmentDocValue(Long segmentDocValue) { kahanSummation.reset(0, 0); kahanSummation.add(starTreeNumericType.getDoubleValue(segmentDocValue)); compensation = kahanSummation.delta(); @@ -44,7 +50,7 @@ public Double getInitialAggregatedValueForSegmentDocValue(Long segmentDocValue, } @Override - public Double mergeAggregatedValueAndSegmentValue(Double value, Long segmentDocValue, StarTreeNumericType starTreeNumericType) { + public Double mergeAggregatedValueAndSegmentValue(Double value, Long segmentDocValue) { assert kahanSummation.value() == value; kahanSummation.reset(sum, compensation); kahanSummation.add(starTreeNumericType.getDoubleValue(segmentDocValue)); @@ -87,9 +93,12 @@ public Long toLongValue(Double value) { } @Override - public Double toStarTreeNumericTypeValue(Long value, StarTreeNumericType type) { + public Double toStarTreeNumericTypeValue(Long value) { try { - return type.getDoubleValue(value); + if (value == null) { + return 0.0; + } + return VALUE_AGGREGATOR_TYPE.getDoubleValue(value); } catch (Exception e) { throw new IllegalStateException("Cannot convert " + value + " to sortable aggregation type", e); } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/ValueAggregator.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/ValueAggregator.java index 3dd1f85845c17..93230ed012b13 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/ValueAggregator.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/ValueAggregator.java @@ -30,12 +30,12 @@ public interface ValueAggregator { /** * Returns the initial aggregated value. */ - A getInitialAggregatedValueForSegmentDocValue(Long segmentDocValue, StarTreeNumericType starTreeNumericType); + A getInitialAggregatedValueForSegmentDocValue(Long segmentDocValue); /** * Applies a segment doc value to the current aggregated value. */ - A mergeAggregatedValueAndSegmentValue(A value, Long segmentDocValue, StarTreeNumericType starTreeNumericType); + A mergeAggregatedValueAndSegmentValue(A value, Long segmentDocValue); /** * Applies an aggregated value to the current aggregated value. @@ -60,5 +60,5 @@ public interface ValueAggregator { /** * Converts an aggregated value from a Long type. */ - A toStarTreeNumericTypeValue(Long rawValue, StarTreeNumericType type); + A toStarTreeNumericTypeValue(Long rawValue); } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/ValueAggregatorFactory.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/ValueAggregatorFactory.java index 4ee0b0b5b13f8..240bbd37a53ee 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/ValueAggregatorFactory.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/ValueAggregatorFactory.java @@ -21,16 +21,17 @@ private ValueAggregatorFactory() {} /** * Returns a new instance of value aggregator for the given aggregation type. * - * @param aggregationType Aggregation type + * @param aggregationType Aggregation type + * @param starTreeNumericType Numeric type associated with star tree field ( as specified in index mapping ) * @return Value aggregator */ - public static ValueAggregator getValueAggregator(MetricStat aggregationType) { + public static ValueAggregator getValueAggregator(MetricStat aggregationType, StarTreeNumericType starTreeNumericType) { switch (aggregationType) { // other metric types (count, min, max, avg) will be supported in the future case SUM: - return new SumValueAggregator(); + return new SumValueAggregator(starTreeNumericType); case COUNT: - return new CountValueAggregator(); + return new CountValueAggregator(starTreeNumericType); default: throw new IllegalStateException("Unsupported aggregation type: " + aggregationType); } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java index 0a363bfad8fe1..7187fade882ea 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilder.java @@ -12,7 +12,11 @@ import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.index.DocValuesType; import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.VectorEncoding; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.opensearch.index.codec.composite.datacube.startree.StarTreeValues; import org.opensearch.index.compositeindex.datacube.Dimension; import org.opensearch.index.compositeindex.datacube.Metric; import org.opensearch.index.compositeindex.datacube.MetricStat; @@ -21,7 +25,6 @@ import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; import org.opensearch.index.compositeindex.datacube.startree.aggregators.MetricAggregatorInfo; import org.opensearch.index.compositeindex.datacube.startree.aggregators.ValueAggregator; -import org.opensearch.index.compositeindex.datacube.startree.aggregators.numerictype.StarTreeNumericType; import org.opensearch.index.compositeindex.datacube.startree.utils.SequentialDocValuesIterator; import org.opensearch.index.compositeindex.datacube.startree.utils.TreeNode; import org.opensearch.index.fielddata.IndexNumericFieldData; @@ -32,11 +35,13 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import static org.opensearch.index.compositeindex.datacube.startree.utils.TreeNode.ALL; @@ -54,8 +59,7 @@ public abstract class BaseStarTreeBuilder implements StarTreeBuilder { /** * Default value for star node */ - public static final int STAR_IN_DOC_VALUES_INDEX = -1; - + public static final Long STAR_IN_DOC_VALUES_INDEX = null; protected final Set skipStarNodeCreationForDimensions; protected final List metricAggregatorInfos; @@ -68,59 +72,41 @@ public abstract class BaseStarTreeBuilder implements StarTreeBuilder { protected final TreeNode rootNode = getNewNode(); - protected SequentialDocValuesIterator[] dimensionReaders; - - // We do not close these producers as they are empty doc value producers (where close() is unsupported) - protected Map fieldProducerMap; - - private final StarTreeDocValuesIteratorAdapter starTreeDocValuesIteratorAdapter; private final StarTreeField starTreeField; + private final MapperService mapperService; + private final SegmentWriteState state; + static String NUM_SEGMENT_DOCS = "numSegmentDocs"; /** * Reads all the configuration related to dimensions and metrics, builds a star-tree based on the different construction parameters. * * @param starTreeField holds the configuration for the star tree - * @param fieldProducerMap helps return the doc values iterator for each type based on field name * @param state stores the segment write state * @param mapperService helps to find the original type of the field */ - protected BaseStarTreeBuilder( - StarTreeField starTreeField, - Map fieldProducerMap, - SegmentWriteState state, - MapperService mapperService - ) throws IOException { - - logger.debug("Building in base star tree builder"); + protected BaseStarTreeBuilder(StarTreeField starTreeField, SegmentWriteState state, MapperService mapperService) { + logger.debug("Building star tree : {}", starTreeField.getName()); this.starTreeField = starTreeField; StarTreeFieldConfiguration starTreeFieldSpec = starTreeField.getStarTreeConfig(); - this.fieldProducerMap = fieldProducerMap; - this.starTreeDocValuesIteratorAdapter = new StarTreeDocValuesIteratorAdapter(); List dimensionsSplitOrder = starTreeField.getDimensionsOrder(); this.numDimensions = dimensionsSplitOrder.size(); this.skipStarNodeCreationForDimensions = new HashSet<>(); this.totalSegmentDocs = state.segmentInfo.maxDoc(); - this.dimensionReaders = new SequentialDocValuesIterator[numDimensions]; + this.mapperService = mapperService; + this.state = state; + Set skipStarNodeCreationForDimensions = starTreeFieldSpec.getSkipStarNodeCreationInDims(); for (int i = 0; i < numDimensions; i++) { - String dimension = dimensionsSplitOrder.get(i).getField(); if (skipStarNodeCreationForDimensions.contains(dimensionsSplitOrder.get(i).getField())) { this.skipStarNodeCreationForDimensions.add(i); } - FieldInfo dimensionFieldInfos = state.fieldInfos.fieldInfo(dimension); - DocValuesType dimensionDocValuesType = dimensionFieldInfos.getDocValuesType(); - dimensionReaders[i] = starTreeDocValuesIteratorAdapter.getDocValuesIterator( - dimensionDocValuesType, - dimensionFieldInfos, - fieldProducerMap.get(dimensionFieldInfos.name) - ); } - this.metricAggregatorInfos = generateMetricAggregatorInfos(mapperService, state); + this.metricAggregatorInfos = generateMetricAggregatorInfos(mapperService); this.numMetrics = metricAggregatorInfos.size(); this.maxLeafDocuments = starTreeFieldSpec.maxLeafDocs(); } @@ -130,13 +116,11 @@ protected BaseStarTreeBuilder( * * @return list of MetricAggregatorInfo */ - public List generateMetricAggregatorInfos(MapperService mapperService, SegmentWriteState state) - throws IOException { + public List generateMetricAggregatorInfos(MapperService mapperService) { List metricAggregatorInfos = new ArrayList<>(); for (Metric metric : this.starTreeField.getMetrics()) { for (MetricStat metricStat : metric.getMetrics()) { IndexNumericFieldData.NumericType numericType; - SequentialDocValuesIterator metricStatReader; Mapper fieldMapper = mapperService.documentMapper().mappers().getMapper(metric.getField()); if (fieldMapper instanceof NumberFieldMapper) { numericType = ((NumberFieldMapper) fieldMapper).fieldType().numericType(); @@ -145,24 +129,11 @@ public List generateMetricAggregatorInfos(MapperService ma throw new IllegalStateException("unsupported mapper type"); } - FieldInfo metricFieldInfos = state.fieldInfos.fieldInfo(metric.getField()); - DocValuesType metricDocValuesType = metricFieldInfos.getDocValuesType(); - if (metricStat != MetricStat.COUNT) { - metricStatReader = starTreeDocValuesIteratorAdapter.getDocValuesIterator( - metricDocValuesType, - metricFieldInfos, - fieldProducerMap.get(metricFieldInfos.name) - ); - } else { - metricStatReader = new SequentialDocValuesIterator(); - } - MetricAggregatorInfo metricAggregatorInfo = new MetricAggregatorInfo( metricStat, metric.getField(), starTreeField.getName(), - numericType, - metricStatReader + numericType ); metricAggregatorInfos.add(metricAggregatorInfo); } @@ -204,12 +175,17 @@ public List generateMetricAggregatorInfos(MapperService ma public abstract Long getDimensionValue(int docId, int dimensionId) throws IOException; /** - * Sorts and aggregates the star-tree document in the segment, and returns a star-tree document iterator for all the - * aggregated star-tree document. + * Sorts and aggregates all the documents in the segment as per the configuration, and returns a star-tree document iterator for all the + * aggregated star-tree documents. * + * @param dimensionReaders List of docValues readers to read dimensions from the segment + * @param metricReaders List of docValues readers to read metrics from the segment * @return Iterator for the aggregated star-tree document */ - public abstract Iterator sortAndAggregateStarTreeDocuments() throws IOException; + public abstract Iterator sortAndAggregateSegmentDocuments( + SequentialDocValuesIterator[] dimensionReaders, + List metricReaders + ) throws IOException; /** * Generates aggregated star-tree documents for star-node. @@ -223,13 +199,16 @@ public abstract Iterator generateStarTreeDocumentsForStarNode( throws IOException; /** - * Returns the star-tree document from the segment + * Returns the star-tree document from the segment based on the current doc id * - * @throws IOException when we are unable to build a star tree document from the segment */ - protected StarTreeDocument getSegmentStarTreeDocument(int currentDocId) throws IOException { - Long[] dimensions = getStarTreeDimensionsFromSegment(currentDocId); - Object[] metrics = getStarTreeMetricsFromSegment(currentDocId); + protected StarTreeDocument getSegmentStarTreeDocument( + int currentDocId, + SequentialDocValuesIterator[] dimensionReaders, + List metricReaders + ) throws IOException { + Long[] dimensions = getStarTreeDimensionsFromSegment(currentDocId, dimensionReaders); + Object[] metrics = getStarTreeMetricsFromSegment(currentDocId, metricReaders); return new StarTreeDocument(dimensions, metrics); } @@ -239,55 +218,48 @@ protected StarTreeDocument getSegmentStarTreeDocument(int currentDocId) throws I * @return dimension values for each of the star-tree dimension * @throws IOException when we are unable to iterate to the next doc for the given dimension readers */ - private Long[] getStarTreeDimensionsFromSegment(int currentDocId) throws IOException { + Long[] getStarTreeDimensionsFromSegment(int currentDocId, SequentialDocValuesIterator[] dimensionReaders) throws IOException { Long[] dimensions = new Long[numDimensions]; for (int i = 0; i < numDimensions; i++) { - try { - dimensions[i] = getValuesFromSegment(dimensionReaders[i], currentDocId); - } catch (Exception e) { - logger.error("unable to read the dimension values from the segment", e); - throw new IllegalStateException("unable to read the dimension values from the segment", e); + if (dimensionReaders[i] != null) { + try { + dimensionReaders[i].nextDoc(currentDocId); + } catch (IOException e) { + logger.error("unable to iterate to next doc", e); + throw new RuntimeException("unable to iterate to next doc", e); + } catch (Exception e) { + logger.error("unable to read the dimension values from the segment", e); + throw new IllegalStateException("unable to read the dimension values from the segment", e); + } + dimensions[i] = dimensionReaders[i].value(currentDocId); + } else { + throw new IllegalStateException("dimension readers are empty"); } - } return dimensions; } - /** - * Returns the next value from the iterator of respective field - * - * @param iterator respective field iterator - * @param currentDocId current document id - * @return the next value for the field - * @throws IOException when we are unable to iterate to the next doc for the given iterator - */ - private Long getValuesFromSegment(SequentialDocValuesIterator iterator, int currentDocId) throws IOException { - try { - starTreeDocValuesIteratorAdapter.nextDoc(iterator, currentDocId); - } catch (IOException e) { - logger.error("unable to iterate to next doc", e); - throw new RuntimeException("unable to iterate to next doc", e); - } - return starTreeDocValuesIteratorAdapter.getNextValue(iterator, currentDocId); - } - /** * Returns the metric values for the next document from the segment * * @return metric values for each of the star-tree metric * @throws IOException when we are unable to iterate to the next doc for the given metric readers */ - private Object[] getStarTreeMetricsFromSegment(int currentDocId) throws IOException { + private Object[] getStarTreeMetricsFromSegment(int currentDocId, List metricsReaders) throws IOException { Object[] metrics = new Object[numMetrics]; for (int i = 0; i < numMetrics; i++) { - SequentialDocValuesIterator metricStatReader = metricAggregatorInfos.get(i).getMetricStatReader(); + SequentialDocValuesIterator metricStatReader = metricsReaders.get(i); if (metricStatReader != null) { try { - metrics[i] = getValuesFromSegment(metricStatReader, currentDocId); + metricStatReader.nextDoc(currentDocId); + } catch (IOException e) { + logger.error("unable to iterate to next doc", e); + throw new RuntimeException("unable to iterate to next doc", e); } catch (Exception e) { logger.error("unable to read the metric values from the segment", e); throw new IllegalStateException("unable to read the metric values from the segment", e); } + metrics[i] = metricStatReader.value(currentDocId); } else { throw new IllegalStateException("metric readers are empty"); } @@ -306,7 +278,8 @@ private Object[] getStarTreeMetricsFromSegment(int currentDocId) throws IOExcept @SuppressWarnings({ "unchecked", "rawtypes" }) protected StarTreeDocument reduceSegmentStarTreeDocuments( StarTreeDocument aggregatedSegmentDocument, - StarTreeDocument segmentDocument + StarTreeDocument segmentDocument, + boolean isMerge ) { if (aggregatedSegmentDocument == null) { Long[] dimensions = Arrays.copyOf(segmentDocument.dimensions, numDimensions); @@ -314,11 +287,12 @@ protected StarTreeDocument reduceSegmentStarTreeDocuments( for (int i = 0; i < numMetrics; i++) { try { ValueAggregator metricValueAggregator = metricAggregatorInfos.get(i).getValueAggregators(); - StarTreeNumericType starTreeNumericType = metricAggregatorInfos.get(i).getAggregatedValueType(); - metrics[i] = metricValueAggregator.getInitialAggregatedValueForSegmentDocValue( - getLong(segmentDocument.metrics[i]), - starTreeNumericType - ); + if (isMerge) { + metrics[i] = metricValueAggregator.getInitialAggregatedValue(segmentDocument.metrics[i]); + } else { + metrics[i] = metricValueAggregator.getInitialAggregatedValueForSegmentDocValue(getLong(segmentDocument.metrics[i])); + } + } catch (Exception e) { logger.error("Cannot parse initial segment doc value", e); throw new IllegalStateException("Cannot parse initial segment doc value [" + segmentDocument.metrics[i] + "]"); @@ -329,12 +303,17 @@ protected StarTreeDocument reduceSegmentStarTreeDocuments( for (int i = 0; i < numMetrics; i++) { try { ValueAggregator metricValueAggregator = metricAggregatorInfos.get(i).getValueAggregators(); - StarTreeNumericType starTreeNumericType = metricAggregatorInfos.get(i).getAggregatedValueType(); - aggregatedSegmentDocument.metrics[i] = metricValueAggregator.mergeAggregatedValueAndSegmentValue( - aggregatedSegmentDocument.metrics[i], - getLong(segmentDocument.metrics[i]), - starTreeNumericType - ); + if (isMerge) { + aggregatedSegmentDocument.metrics[i] = metricValueAggregator.mergeAggregatedValues( + segmentDocument.metrics[i], + aggregatedSegmentDocument.metrics[i] + ); + } else { + aggregatedSegmentDocument.metrics[i] = metricValueAggregator.mergeAggregatedValueAndSegmentValue( + aggregatedSegmentDocument.metrics[i], + getLong(segmentDocument.metrics[i]) + ); + } } catch (Exception e) { logger.error("Cannot apply segment doc value for aggregation", e); throw new IllegalStateException("Cannot apply segment doc value for aggregation [" + segmentDocument.metrics[i] + "]"); @@ -364,7 +343,9 @@ private static long getLong(Object metric) { } if (metricValue == null) { - throw new IllegalStateException("unable to cast segment metric"); + return 0; + // TODO: handle this properly + // throw new IllegalStateException("unable to cast segment metric"); } return metricValue; } @@ -410,25 +391,88 @@ public StarTreeDocument reduceStarTreeDocuments(StarTreeDocument aggregatedDocum } /** - * Builds the star tree using total segment documents + * Builds the star tree from the original segment documents + * + * @param fieldProducerMap contain s the docValues producer to get docValues associated with each field * * @throws IOException when we are unable to build star-tree */ - public void build() throws IOException { + public void build(Map fieldProducerMap) throws IOException { long startTime = System.currentTimeMillis(); logger.debug("Star-tree build is a go with star tree field {}", starTreeField.getName()); - if (totalSegmentDocs == 0) { logger.debug("No documents found in the segment"); return; } - - Iterator starTreeDocumentIterator = sortAndAggregateStarTreeDocuments(); + List metricReaders = getMetricReaders(state, fieldProducerMap); + List dimensionsSplitOrder = starTreeField.getDimensionsOrder(); + SequentialDocValuesIterator[] dimensionReaders = new SequentialDocValuesIterator[dimensionsSplitOrder.size()]; + for (int i = 0; i < numDimensions; i++) { + String dimension = dimensionsSplitOrder.get(i).getField(); + FieldInfo dimensionFieldInfo = state.fieldInfos.fieldInfo(dimension); + if (dimensionFieldInfo == null) { + dimensionFieldInfo = getFieldInfo(dimension); + } + dimensionReaders[i] = new SequentialDocValuesIterator( + fieldProducerMap.get(dimensionFieldInfo.name).getSortedNumeric(dimensionFieldInfo) + ); + } + Iterator starTreeDocumentIterator = sortAndAggregateSegmentDocuments(dimensionReaders, metricReaders); logger.debug("Sorting and aggregating star-tree in ms : {}", (System.currentTimeMillis() - startTime)); build(starTreeDocumentIterator); logger.debug("Finished Building star-tree in ms : {}", (System.currentTimeMillis() - startTime)); } + private static FieldInfo getFieldInfo(String field) { + return new FieldInfo( + field, + 1, + false, + false, + false, + IndexOptions.NONE, + DocValuesType.SORTED_NUMERIC, + -1, + Collections.emptyMap(), + 0, + 0, + 0, + 0, + VectorEncoding.FLOAT32, + VectorSimilarityFunction.EUCLIDEAN, + false, + false + ); + } + + /** + * Generates the configuration required to perform aggregation for all the metrics on a field + * + * @return list of MetricAggregatorInfo + */ + public List getMetricReaders(SegmentWriteState state, Map fieldProducerMap) + throws IOException { + List metricReaders = new ArrayList<>(); + for (Metric metric : this.starTreeField.getMetrics()) { + for (MetricStat metricStat : metric.getMetrics()) { + FieldInfo metricFieldInfo = state.fieldInfos.fieldInfo(metric.getField()); + if (metricFieldInfo == null) { + metricFieldInfo = getFieldInfo(metric.getField()); + } + // TODO + // if (metricStat != MetricStat.COUNT) { + // Need not initialize the metric reader for COUNT metric type + SequentialDocValuesIterator metricReader = new SequentialDocValuesIterator( + fieldProducerMap.get(metricFieldInfo.name).getSortedNumeric(metricFieldInfo) + ); + // } + + metricReaders.add(metricReader); + } + } + return metricReaders; + } + /** * Builds the star tree using Star-Tree Document * @@ -466,7 +510,6 @@ void build(Iterator starTreeDocumentIterator) throws IOExcepti // Create doc values indices in disk // Serialize and save in disk // Write star tree metadata for off heap implementation - } /** @@ -538,10 +581,10 @@ private Map constructNonStarNodes(int startDocId, int endDocId, Long nodeDimensionValue = getDimensionValue(startDocId, dimensionId); for (int i = startDocId + 1; i < endDocId; i++) { Long dimensionValue = getDimensionValue(i, dimensionId); - if (!dimensionValue.equals(nodeDimensionValue)) { + if (Objects.equals(dimensionValue, nodeDimensionValue) == false) { TreeNode child = getNewNode(); child.dimensionId = dimensionId; - child.dimensionValue = nodeDimensionValue; + child.dimensionValue = nodeDimensionValue != null ? nodeDimensionValue : ALL; child.startDocId = nodeStartDocId; child.endDocId = i; nodes.put(nodeDimensionValue, child); @@ -552,7 +595,7 @@ private Map constructNonStarNodes(int startDocId, int endDocId, } TreeNode lastNode = getNewNode(); lastNode.dimensionId = dimensionId; - lastNode.dimensionValue = nodeDimensionValue; + lastNode.dimensionValue = nodeDimensionValue != null ? nodeDimensionValue : ALL; lastNode.startDocId = nodeStartDocId; lastNode.endDocId = endDocId; nodes.put(nodeDimensionValue, lastNode); @@ -607,7 +650,7 @@ private StarTreeDocument createAggregatedDocs(TreeNode node) throws IOException throw new IllegalStateException("aggregated star-tree document is null after reducing the documents"); } for (int i = node.dimensionId + 1; i < numDimensions; i++) { - aggregatedStarTreeDocument.dimensions[i] = Long.valueOf(STAR_IN_DOC_VALUES_INDEX); + aggregatedStarTreeDocument.dimensions[i] = STAR_IN_DOC_VALUES_INDEX; } node.aggregatedDocId = numStarTreeDocs; appendToStarTree(aggregatedStarTreeDocument); @@ -639,7 +682,7 @@ private StarTreeDocument createAggregatedDocs(TreeNode node) throws IOException throw new IllegalStateException("aggregated star-tree document is null after reducing the documents"); } for (int i = node.dimensionId + 1; i < numDimensions; i++) { - aggregatedStarTreeDocument.dimensions[i] = Long.valueOf(STAR_IN_DOC_VALUES_INDEX); + aggregatedStarTreeDocument.dimensions[i] = STAR_IN_DOC_VALUES_INDEX; } node.aggregatedDocId = numStarTreeDocs; appendToStarTree(aggregatedStarTreeDocument); @@ -665,4 +708,5 @@ public void close() throws IOException { } + abstract Iterator mergeStarTrees(List starTreeValues) throws IOException; } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OnHeapStarTreeBuilder.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OnHeapStarTreeBuilder.java index caeb24838da62..1599be2e76a56 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OnHeapStarTreeBuilder.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/OnHeapStarTreeBuilder.java @@ -7,11 +7,14 @@ */ package org.opensearch.index.compositeindex.datacube.startree.builder; -import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.search.DocIdSetIterator; import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.index.codec.composite.datacube.startree.StarTreeValues; +import org.opensearch.index.compositeindex.datacube.Dimension; import org.opensearch.index.compositeindex.datacube.startree.StarTreeDocument; import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; +import org.opensearch.index.compositeindex.datacube.startree.utils.SequentialDocValuesIterator; import org.opensearch.index.mapper.MapperService; import java.io.IOException; @@ -36,27 +39,20 @@ public class OnHeapStarTreeBuilder extends BaseStarTreeBuilder { * Constructor for OnHeapStarTreeBuilder * * @param starTreeField star-tree field - * @param fieldProducerMap helps with document values producer for a particular field * @param segmentWriteState segment write state * @param mapperService helps with the numeric type of field - * @throws IOException throws an exception we are unable to construct an onheap star-tree */ - public OnHeapStarTreeBuilder( - StarTreeField starTreeField, - Map fieldProducerMap, - SegmentWriteState segmentWriteState, - MapperService mapperService - ) throws IOException { - super(starTreeField, fieldProducerMap, segmentWriteState, mapperService); + public OnHeapStarTreeBuilder(StarTreeField starTreeField, SegmentWriteState segmentWriteState, MapperService mapperService) { + super(starTreeField, segmentWriteState, mapperService); } @Override - public void appendStarTreeDocument(StarTreeDocument starTreeDocument) throws IOException { + public void appendStarTreeDocument(StarTreeDocument starTreeDocument) { starTreeDocuments.add(starTreeDocument); } @Override - public StarTreeDocument getStarTreeDocument(int docId) throws IOException { + public StarTreeDocument getStarTreeDocument(int docId) { return starTreeDocuments.get(docId); } @@ -66,34 +62,123 @@ public List getStarTreeDocuments() { } @Override - public Long getDimensionValue(int docId, int dimensionId) throws IOException { + public Long getDimensionValue(int docId, int dimensionId) { return starTreeDocuments.get(docId).dimensions[dimensionId]; } + /** + * Sorts and aggregates all the documents of the segment based on dimension and metrics configuration + * + * @param dimensionReaders List of docValues readers to read dimensions from the segment + * @param metricReaders List of docValues readers to read metrics from the segment + * @return Iterator of star-tree documents + * + */ @Override - public Iterator sortAndAggregateStarTreeDocuments() throws IOException { - int numDocs = totalSegmentDocs; - StarTreeDocument[] starTreeDocuments = new StarTreeDocument[numDocs]; - for (int currentDocId = 0; currentDocId < numDocs; currentDocId++) { - starTreeDocuments[currentDocId] = getSegmentStarTreeDocument(currentDocId); + public Iterator sortAndAggregateSegmentDocuments( + SequentialDocValuesIterator[] dimensionReaders, + List metricReaders + ) throws IOException { + StarTreeDocument[] starTreeDocuments = new StarTreeDocument[totalSegmentDocs]; + for (int currentDocId = 0; currentDocId < totalSegmentDocs; currentDocId++) { + // TODO : we can save empty iterator for dimensions which are not part of segment + starTreeDocuments[currentDocId] = getSegmentStarTreeDocument(currentDocId, dimensionReaders, metricReaders); } - return sortAndAggregateStarTreeDocuments(starTreeDocuments); } + @Override + public void build(List starTreeValuesSubs) throws IOException { + build(mergeStarTrees(starTreeValuesSubs)); + } + + /** + * Sorts and aggregates the star-tree documents from multiple segments and builds star tree based on the newly + * aggregated star-tree documents + * + * @param starTreeValuesSubs StarTreeValues from multiple segments + * @return iterator of star tree documents + */ + @Override + Iterator mergeStarTrees(List starTreeValuesSubs) throws IOException { + return sortAndAggregateStarTreeDocuments(getSegmentsStarTreeDocuments(starTreeValuesSubs), true); + } + + /** + * Returns an array of all the starTreeDocuments from all the segments + * We only take the non-star documents from all the segments. + * + * @param starTreeValuesSubs StarTreeValues from multiple segments + * @return array of star tree documents + */ + StarTreeDocument[] getSegmentsStarTreeDocuments(List starTreeValuesSubs) throws IOException { + List starTreeDocuments = new ArrayList<>(); + for (StarTreeValues starTreeValues : starTreeValuesSubs) { + List dimensionsSplitOrder = starTreeValues.getStarTreeField().getDimensionsOrder(); + SequentialDocValuesIterator[] dimensionReaders = new SequentialDocValuesIterator[dimensionsSplitOrder.size()]; + + for (int i = 0; i < dimensionsSplitOrder.size(); i++) { + String dimension = dimensionsSplitOrder.get(i).getField(); + dimensionReaders[i] = new SequentialDocValuesIterator(starTreeValues.getDimensionDocValuesIteratorMap().get(dimension)); + } + + List metricReaders = new ArrayList<>(); + for (Map.Entry metricDocValuesEntry : starTreeValues.getMetricDocValuesIteratorMap().entrySet()) { + metricReaders.add(new SequentialDocValuesIterator(metricDocValuesEntry.getValue())); + } + + boolean endOfDoc = false; + int currentDocId = 0; + int numSegmentDocs = Integer.parseInt( + starTreeValues.getAttributes().getOrDefault(NUM_SEGMENT_DOCS, String.valueOf(DocIdSetIterator.NO_MORE_DOCS)) + ); + while (currentDocId < numSegmentDocs) { + Long[] dims = new Long[dimensionsSplitOrder.size()]; + int i = 0; + for (SequentialDocValuesIterator dimensionDocValueIterator : dimensionReaders) { + dimensionDocValueIterator.nextDoc(currentDocId); + Long val = dimensionDocValueIterator.value(currentDocId); + dims[i] = val; + i++; + } + i = 0; + Object[] metrics = new Object[metricReaders.size()]; + for (SequentialDocValuesIterator metricDocValuesIterator : metricReaders) { + metricDocValuesIterator.nextDoc(currentDocId); + // As part of merge, we traverse the star tree doc values + // The type of data stored in metric fields is different from the + // actual indexing field they're based on + metrics[i] = metricAggregatorInfos.get(i) + .getValueAggregators() + .toStarTreeNumericTypeValue(metricDocValuesIterator.value(currentDocId)); + i++; + } + StarTreeDocument starTreeDocument = new StarTreeDocument(dims, metrics); + starTreeDocuments.add(starTreeDocument); + currentDocId++; + } + } + StarTreeDocument[] starTreeDocumentsArr = new StarTreeDocument[starTreeDocuments.size()]; + return starTreeDocuments.toArray(starTreeDocumentsArr); + } + + Iterator sortAndAggregateStarTreeDocuments(StarTreeDocument[] starTreeDocuments) { + return sortAndAggregateStarTreeDocuments(starTreeDocuments, false); + } + /** * Sort, aggregates and merges the star-tree documents * * @param starTreeDocuments star-tree documents * @return iterator for star-tree documents */ - Iterator sortAndAggregateStarTreeDocuments(StarTreeDocument[] starTreeDocuments) { + Iterator sortAndAggregateStarTreeDocuments(StarTreeDocument[] starTreeDocuments, boolean isMerge) { // sort all the documents sortStarTreeDocumentsFromDimensionId(starTreeDocuments, 0); // merge the documents - return mergeStarTreeDocuments(starTreeDocuments); + return mergeStarTreeDocuments(starTreeDocuments, isMerge); } /** @@ -102,7 +187,7 @@ Iterator sortAndAggregateStarTreeDocuments(StarTreeDocument[] * @param starTreeDocuments star-tree documents * @return iterator to aggregate star-tree documents */ - private Iterator mergeStarTreeDocuments(StarTreeDocument[] starTreeDocuments) { + private Iterator mergeStarTreeDocuments(StarTreeDocument[] starTreeDocuments, boolean isMerge) { return new Iterator<>() { boolean hasNext = true; StarTreeDocument currentStarTreeDocument = starTreeDocuments[0]; @@ -117,7 +202,7 @@ public boolean hasNext() { @Override public StarTreeDocument next() { // aggregate as we move on to the next doc - StarTreeDocument next = reduceSegmentStarTreeDocuments(null, currentStarTreeDocument); + StarTreeDocument next = reduceSegmentStarTreeDocuments(null, currentStarTreeDocument, isMerge); while (docId < starTreeDocuments.length) { StarTreeDocument starTreeDocument = starTreeDocuments[docId]; docId++; @@ -125,7 +210,7 @@ public StarTreeDocument next() { currentStarTreeDocument = starTreeDocument; return next; } else { - next = reduceSegmentStarTreeDocuments(next, starTreeDocument); + next = reduceSegmentStarTreeDocuments(next, starTreeDocument, isMerge); } } hasNext = false; @@ -141,11 +226,9 @@ public StarTreeDocument next() { * @param endDocId End document id (exclusive) in the star-tree * @param dimensionId Dimension id of the star-node * @return iterator for star-tree documents of star-node - * @throws IOException throws when unable to generate star-tree for star-node */ @Override - public Iterator generateStarTreeDocumentsForStarNode(int startDocId, int endDocId, int dimensionId) - throws IOException { + public Iterator generateStarTreeDocumentsForStarNode(int startDocId, int endDocId, int dimensionId) { int numDocs = endDocId - startDocId; StarTreeDocument[] starTreeDocuments = new StarTreeDocument[numDocs]; for (int i = 0; i < numDocs; i++) { @@ -177,7 +260,7 @@ public boolean hasNext() { @Override public StarTreeDocument next() { StarTreeDocument next = reduceStarTreeDocuments(null, currentStarTreeDocument); - next.dimensions[dimensionId] = Long.valueOf(STAR_IN_DOC_VALUES_INDEX); + next.dimensions[dimensionId] = STAR_IN_DOC_VALUES_INDEX; while (docId < numDocs) { StarTreeDocument starTreeDocument = starTreeDocuments[docId]; docId++; @@ -204,6 +287,15 @@ private void sortStarTreeDocumentsFromDimensionId(StarTreeDocument[] starTreeDoc Arrays.sort(starTreeDocuments, (o1, o2) -> { for (int i = dimensionId; i < numDimensions; i++) { if (!Objects.equals(o1.dimensions[i], o2.dimensions[i])) { + if (o1.dimensions[i] == null && o2.dimensions[i] == null) { + return 0; + } + if (o1.dimensions[i] == null) { + return 1; + } + if (o2.dimensions[i] == null) { + return -1; + } return Long.compare(o1.dimensions[i], o2.dimensions[i]); } } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeBuilder.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeBuilder.java index 20af1b3bc7935..94c9c9f2efb18 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeBuilder.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeBuilder.java @@ -8,10 +8,14 @@ package org.opensearch.index.compositeindex.datacube.startree.builder; +import org.apache.lucene.codecs.DocValuesProducer; import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.index.codec.composite.datacube.startree.StarTreeValues; import java.io.Closeable; import java.io.IOException; +import java.util.List; +import java.util.Map; /** * A star-tree builder that builds a single star-tree. @@ -20,10 +24,20 @@ */ @ExperimentalApi public interface StarTreeBuilder extends Closeable { + /** + * Builds the star tree from the original segment documents + * + * @param fieldProducerMap contains the docValues producer to get docValues associated with each field + * @throws IOException when we are unable to build star-tree + */ + + void build(Map fieldProducerMap) throws IOException; /** - * Builds the star tree based on star-tree field + * Builds the star tree using StarTree values from multiple segments + * + * @param starTreeValuesSubs contains the star tree values from multiple segments * @throws IOException when we are unable to build star-tree */ - void build() throws IOException; + void build(List starTreeValuesSubs) throws IOException; } diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeDocValuesIteratorAdapter.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeDocValuesIteratorAdapter.java deleted file mode 100644 index cb0350bb110b0..0000000000000 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeDocValuesIteratorAdapter.java +++ /dev/null @@ -1,82 +0,0 @@ -/* - * 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.compositeindex.datacube.startree.builder; - -import org.apache.lucene.codecs.DocValuesProducer; -import org.apache.lucene.index.DocValuesType; -import org.apache.lucene.index.FieldInfo; -import org.apache.lucene.index.SortedNumericDocValues; -import org.apache.lucene.search.DocIdSetIterator; -import org.opensearch.common.annotation.ExperimentalApi; -import org.opensearch.index.compositeindex.datacube.startree.utils.SequentialDocValuesIterator; - -import java.io.IOException; - -/** - * A factory class to return respective doc values iterator based on the doc volues type. - * - * @opensearch.experimental - */ -@ExperimentalApi -public class StarTreeDocValuesIteratorAdapter { - - /** - * Creates an iterator for the given doc values type and field using the doc values producer - */ - public SequentialDocValuesIterator getDocValuesIterator(DocValuesType type, FieldInfo field, DocValuesProducer producer) - throws IOException { - switch (type) { - case SORTED_NUMERIC: - return new SequentialDocValuesIterator(producer.getSortedNumeric(field)); - default: - throw new IllegalArgumentException("Unsupported DocValuesType: " + type); - } - } - - /** - * Returns the next value for the given iterator - */ - public Long getNextValue(SequentialDocValuesIterator sequentialDocValuesIterator, int currentDocId) throws IOException { - if (sequentialDocValuesIterator.getDocIdSetIterator() instanceof SortedNumericDocValues) { - SortedNumericDocValues sortedNumericDocValues = (SortedNumericDocValues) sequentialDocValuesIterator.getDocIdSetIterator(); - if (sequentialDocValuesIterator.getDocId() < 0 || sequentialDocValuesIterator.getDocId() == DocIdSetIterator.NO_MORE_DOCS) { - throw new IllegalStateException("invalid doc id to fetch the next value"); - } - - if (sequentialDocValuesIterator.getDocValue() == null) { - sequentialDocValuesIterator.setDocValue(sortedNumericDocValues.nextValue()); - return sequentialDocValuesIterator.getDocValue(); - } - - if (sequentialDocValuesIterator.getDocId() == currentDocId) { - Long nextValue = sequentialDocValuesIterator.getDocValue(); - sequentialDocValuesIterator.setDocValue(null); - return nextValue; - } else { - return null; - } - } else { - throw new IllegalStateException("Unsupported Iterator: " + sequentialDocValuesIterator.getDocIdSetIterator().toString()); - } - } - - /** - * Moves to the next doc in the iterator - * Returns the doc id for the next document from the given iterator - */ - public int nextDoc(SequentialDocValuesIterator iterator, int currentDocId) throws IOException { - if (iterator.getDocValue() != null) { - return iterator.getDocId(); - } - iterator.setDocId(iterator.getDocIdSetIterator().nextDoc()); - iterator.setDocValue(this.getNextValue(iterator, currentDocId)); - return iterator.getDocId(); - } - -} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilder.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilder.java index eaf9ae1dcdaa1..6c3d476aa3a55 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilder.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilder.java @@ -13,6 +13,7 @@ import org.apache.lucene.codecs.DocValuesProducer; import org.apache.lucene.index.SegmentWriteState; import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.index.codec.composite.datacube.startree.StarTreeValues; import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; import org.opensearch.index.mapper.CompositeMappedFieldType; import org.opensearch.index.mapper.MapperService; @@ -37,14 +38,9 @@ public class StarTreesBuilder implements Closeable { private final List starTreeFields; private final SegmentWriteState state; - private final Map fieldProducerMap; private final MapperService mapperService; - public StarTreesBuilder( - Map fieldProducerMap, - SegmentWriteState segmentWriteState, - MapperService mapperService - ) { + public StarTreesBuilder(SegmentWriteState segmentWriteState, MapperService mapperService) { List starTreeFields = new ArrayList<>(); for (CompositeMappedFieldType compositeMappedFieldType : mapperService.getCompositeFieldTypes()) { if (compositeMappedFieldType instanceof StarTreeMapper.StarTreeFieldType) { @@ -59,9 +55,7 @@ public StarTreesBuilder( ); } } - this.starTreeFields = starTreeFields; - this.fieldProducerMap = fieldProducerMap; this.state = segmentWriteState; this.mapperService = mapperService; } @@ -69,38 +63,67 @@ public StarTreesBuilder( /** * Builds the star-trees. */ - public void build() throws IOException { + public void build(Map fieldProducerMap) throws IOException { if (starTreeFields.isEmpty()) { logger.debug("no star-tree fields found, returning from star-tree builder"); return; } long startTime = System.currentTimeMillis(); + int numStarTrees = starTreeFields.size(); logger.debug("Starting building {} star-trees with star-tree fields", numStarTrees); // Build all star-trees for (StarTreeField starTreeField : starTreeFields) { - try (StarTreeBuilder starTreeBuilder = getStarTreeBuilder(starTreeField, fieldProducerMap, state, mapperService)) { - starTreeBuilder.build(); + try (StarTreeBuilder starTreeBuilder = getSingleTreeBuilder(starTreeField, state, mapperService)) { + starTreeBuilder.build(fieldProducerMap); } } - logger.debug("Took {} ms to building {} star-trees with star-tree fields", System.currentTimeMillis() - startTime, numStarTrees); + logger.debug("Took {} ms to build {} star-trees with star-tree fields", System.currentTimeMillis() - startTime, numStarTrees); } @Override public void close() throws IOException { + // TODO : close files + } + /** + * Merges star tree fields from multiple segments + * + * @param starTreeValuesSubsPerField starTreeValuesSubs per field + */ + public void buildDuringMerge(final Map> starTreeValuesSubsPerField) throws IOException { + logger.debug("Starting merge of {} star-trees with star-tree fields", starTreeValuesSubsPerField.size()); + long startTime = System.currentTimeMillis(); + for (Map.Entry> entry : starTreeValuesSubsPerField.entrySet()) { + List starTreeValuesList = entry.getValue(); + if (starTreeValuesList.isEmpty()) { + logger.debug("StarTreeValues is empty for all segments for field : {}", entry.getKey()); + continue; + } + StarTreeField starTreeField = starTreeValuesList.get(0).getStarTreeField(); + StarTreeBuilder builder = getSingleTreeBuilder(starTreeField, state, mapperService); + builder.build(starTreeValuesList); + builder.close(); + } + logger.debug( + "Took {} ms to merge {} star-trees with star-tree fields", + System.currentTimeMillis() - startTime, + starTreeValuesSubsPerField.size() + ); } - StarTreeBuilder getStarTreeBuilder( - StarTreeField starTreeField, - Map fieldProducerMap, - SegmentWriteState state, - MapperService mapperService - ) throws IOException { + /** + * Get star-tree builder based on build mode. + */ + StarTreeBuilder getSingleTreeBuilder(StarTreeField starTreeField, SegmentWriteState state, MapperService mapperService) + throws IOException { switch (starTreeField.getStarTreeConfig().getBuildMode()) { case ON_HEAP: - return new OnHeapStarTreeBuilder(starTreeField, fieldProducerMap, state, mapperService); + return new OnHeapStarTreeBuilder(starTreeField, state, mapperService); + case OFF_HEAP: + // TODO + // return new OffHeapStarTreeBuilder(starTreeField, state, mapperService); default: throw new IllegalArgumentException( String.format( diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNode.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNode.java new file mode 100644 index 0000000000000..59522ffa4be89 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/StarTreeNode.java @@ -0,0 +1,112 @@ +/* + * 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.compositeindex.datacube.startree.node; + +import org.opensearch.common.annotation.ExperimentalApi; + +import java.io.IOException; +import java.util.Iterator; + +/** + * Interface that represents star tree node + * + * @opensearch.experimental + */ +@ExperimentalApi +public interface StarTreeNode { + long ALL = -1l; + + /** + * Returns the dimension ID of the current star-tree node. + * + * @return the dimension ID + * @throws IOException if an I/O error occurs while reading the dimension ID + */ + int getDimensionId() throws IOException; + + /** + * Returns the dimension value of the current star-tree node. + * + * @return the dimension value + * @throws IOException if an I/O error occurs while reading the dimension value + */ + long getDimensionValue() throws IOException; + + /** + * Returns the dimension ID of the child star-tree node. + * + * @return the child dimension ID + * @throws IOException if an I/O error occurs while reading the child dimension ID + */ + int getChildDimensionId() throws IOException; + + /** + * Returns the start document ID of the current star-tree node. + * + * @return the start document ID + * @throws IOException if an I/O error occurs while reading the start document ID + */ + int getStartDocId() throws IOException; + + /** + * Returns the end document ID of the current star-tree node. + * + * @return the end document ID + * @throws IOException if an I/O error occurs while reading the end document ID + */ + int getEndDocId() throws IOException; + + /** + * Returns the aggregated document ID of the current star-tree node. + * + * @return the aggregated document ID + * @throws IOException if an I/O error occurs while reading the aggregated document ID + */ + int getAggregatedDocId() throws IOException; + + /** + * Returns the number of children of the current star-tree node. + * + * @return the number of children + * @throws IOException if an I/O error occurs while reading the number of children + */ + int getNumChildren() throws IOException; + + /** + * Checks if the current node is a leaf star-tree node. + * + * @return true if the node is a leaf node, false otherwise + */ + boolean isLeaf(); + + /** + * Checks if the current node is a star node. + * + * @return true if the node is a star node, false otherwise + * @throws IOException if an I/O error occurs while reading the star node status + */ + boolean isStarNode() throws IOException; + + /** + * Returns the child star-tree node for the given dimension value. + * + * @param dimensionValue the dimension value + * @return the child node for the given dimension value or null if child is not present + * @throws IOException if an I/O error occurs while retrieving the child node + */ + StarTreeNode getChildForDimensionValue(long dimensionValue) throws IOException; + + /** + * Returns an iterator over the children of the current star-tree node. + * + * @return an iterator over the children + * @throws IOException if an I/O error occurs while retrieving the children iterator + */ + Iterator getChildrenIterator() throws IOException; +} diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/package-info.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/package-info.java new file mode 100644 index 0000000000000..516d5b5a012ab --- /dev/null +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/node/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. + */ + +/** + * Holds classes associated with star tree node + */ +package org.opensearch.index.compositeindex.datacube.startree.node; diff --git a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/SequentialDocValuesIterator.java b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/SequentialDocValuesIterator.java index cf5f3e94c1ca6..400d7a1c00104 100644 --- a/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/SequentialDocValuesIterator.java +++ b/server/src/main/java/org/opensearch/index/compositeindex/datacube/startree/utils/SequentialDocValuesIterator.java @@ -1,3 +1,4 @@ + /* * SPDX-License-Identifier: Apache-2.0 * @@ -17,7 +18,6 @@ /** * Coordinates the reading of documents across multiple DocIdSetIterators. * It encapsulates a single DocIdSetIterator and maintains the latest document ID and its associated value. - * * @opensearch.experimental */ @ExperimentalApi @@ -28,15 +28,10 @@ public class SequentialDocValuesIterator { */ private final DocIdSetIterator docIdSetIterator; - /** - * The value associated with the latest document. - */ - private Long docValue; - /** * The id of the latest document. */ - private int docId; + private int docId = -1; /** * Constructs a new SequentialDocValuesIterator instance with the given DocIdSetIterator. @@ -47,85 +42,15 @@ public SequentialDocValuesIterator(DocIdSetIterator docIdSetIterator) { this.docIdSetIterator = docIdSetIterator; } - /** - * Constructs a new SequentialDocValuesIterator instance with the given SortedNumericDocValues. - * - */ - public SequentialDocValuesIterator() { - this.docIdSetIterator = new SortedNumericDocValues() { - @Override - public long nextValue() throws IOException { - return 0; - } - - @Override - public int docValueCount() { - return 0; - } - - @Override - public boolean advanceExact(int i) throws IOException { - return false; - } - - @Override - public int docID() { - return 0; - } - - @Override - public int nextDoc() throws IOException { - return 0; - } - - @Override - public int advance(int i) throws IOException { - return 0; - } - - @Override - public long cost() { - return 0; - } - }; - } - - /** - * Returns the value associated with the latest document. - * - * @return the value associated with the latest document - */ - public Long getDocValue() { - return docValue; - } - - /** - * Sets the value associated with the latest document. - * - * @param docValue the value to be associated with the latest document - */ - public void setDocValue(Long docValue) { - this.docValue = docValue; - } - /** * Returns the id of the latest document. * * @return the id of the latest document */ - public int getDocId() { + int getDocId() { return docId; } - /** - * Sets the id of the latest document. - * - * @param docId the ID of the latest document - */ - public void setDocId(int docId) { - this.docId = docId; - } - /** * Returns the DocIdSetIterator associated with this instance. * @@ -134,4 +59,32 @@ public void setDocId(int docId) { public DocIdSetIterator getDocIdSetIterator() { return docIdSetIterator; } + + public int nextDoc(int currentDocId) throws IOException { + // if doc id stored is less than or equal to the requested doc id , return the stored doc id + if (docId >= currentDocId) { + return docId; + } + docId = this.docIdSetIterator.nextDoc(); + return docId; + } + + public Long value(int currentDocId) throws IOException { + if (this.getDocIdSetIterator() instanceof SortedNumericDocValues) { + SortedNumericDocValues sortedNumericDocValues = (SortedNumericDocValues) this.getDocIdSetIterator(); + if (currentDocId < 0) { + throw new IllegalStateException("invalid doc id to fetch the next value"); + } + if (currentDocId == DocIdSetIterator.NO_MORE_DOCS) { + throw new IllegalStateException("DocValuesIterator is already exhausted"); + } + if (docId == DocIdSetIterator.NO_MORE_DOCS || docId != currentDocId) { + return null; + } + return sortedNumericDocValues.nextValue(); + + } else { + throw new IllegalStateException("Unsupported Iterator requested for SequentialDocValuesIterator"); + } + } } diff --git a/server/src/main/java/org/opensearch/index/mapper/CompositeMappedFieldType.java b/server/src/main/java/org/opensearch/index/mapper/CompositeMappedFieldType.java index e067e70621304..7239ddfb26c0d 100644 --- a/server/src/main/java/org/opensearch/index/mapper/CompositeMappedFieldType.java +++ b/server/src/main/java/org/opensearch/index/mapper/CompositeMappedFieldType.java @@ -72,6 +72,10 @@ public static CompositeFieldType fromName(String name) { } } + public CompositeFieldType getCompositeIndexType() { + return type; + } + public List fields() { return fields; } diff --git a/server/src/test/java/org/opensearch/index/codec/composite/datacube/startree/StarTreeDocValuesFormatTests.java b/server/src/test/java/org/opensearch/index/codec/composite/datacube/startree/StarTreeDocValuesFormatTests.java index 31df9a49bebfb..049d91bc42d9c 100644 --- a/server/src/test/java/org/opensearch/index/codec/composite/datacube/startree/StarTreeDocValuesFormatTests.java +++ b/server/src/test/java/org/opensearch/index/codec/composite/datacube/startree/StarTreeDocValuesFormatTests.java @@ -12,63 +12,165 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.lucene99.Lucene99Codec; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.SortedNumericDocValuesField; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.store.Directory; import org.apache.lucene.tests.index.BaseDocValuesFormatTestCase; +import org.apache.lucene.tests.index.RandomIndexWriter; import org.apache.lucene.tests.util.LuceneTestCase; -import org.opensearch.common.Rounding; +import org.opensearch.Version; +import org.opensearch.cluster.ClusterModule; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.CheckedConsumer; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.FeatureFlags; +import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.index.MapperTestUtils; import org.opensearch.index.codec.composite.Composite99Codec; -import org.opensearch.index.compositeindex.datacube.DateDimension; -import org.opensearch.index.compositeindex.datacube.Dimension; -import org.opensearch.index.compositeindex.datacube.Metric; -import org.opensearch.index.compositeindex.datacube.MetricStat; -import org.opensearch.index.compositeindex.datacube.NumericDimension; -import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; -import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; import org.opensearch.index.mapper.MapperService; -import org.opensearch.index.mapper.StarTreeMapper; +import org.opensearch.indices.IndicesModule; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.BeforeClass; -import java.util.ArrayList; +import java.io.IOException; import java.util.Collections; -import java.util.List; -import java.util.Set; -import org.mockito.Mockito; +import static org.opensearch.common.util.FeatureFlags.STAR_TREE_INDEX; /** * Star tree doc values Lucene tests */ @LuceneTestCase.SuppressSysoutChecks(bugUrl = "we log a lot on purpose") public class StarTreeDocValuesFormatTests extends BaseDocValuesFormatTestCase { + MapperService mapperService = null; + + @BeforeClass + public static void createMapper() throws Exception { + FeatureFlags.initializeFeatureFlags(Settings.builder().put(STAR_TREE_INDEX, "true").build()); + } + + @AfterClass + public static void clearMapper() { + FeatureFlags.initializeFeatureFlags(Settings.EMPTY); + } + + @After + public void teardown() throws IOException { + mapperService.close(); + } + @Override protected Codec getCodec() { - MapperService service = Mockito.mock(MapperService.class); - Mockito.when(service.getCompositeFieldTypes()).thenReturn(Set.of(getStarTreeFieldType())); final Logger testLogger = LogManager.getLogger(StarTreeDocValuesFormatTests.class); - return new Composite99Codec(Lucene99Codec.Mode.BEST_SPEED, service, testLogger); + + try { + createMapperService(getExpandedMapping("status", "size")); + } catch (IOException e) { + throw new RuntimeException(e); + } + Codec codec = new Composite99Codec(Lucene99Codec.Mode.BEST_SPEED, mapperService, testLogger); + return codec; } - private StarTreeMapper.StarTreeFieldType getStarTreeFieldType() { - List m1 = new ArrayList<>(); - m1.add(MetricStat.MAX); - Metric metric = new Metric("sndv", m1); - List d1CalendarIntervals = new ArrayList<>(); - d1CalendarIntervals.add(Rounding.DateTimeUnit.HOUR_OF_DAY); - StarTreeField starTreeField = getStarTreeField(d1CalendarIntervals, metric); + public void testStarTreeDocValues() throws IOException { + Directory directory = newDirectory(); + IndexWriterConfig conf = newIndexWriterConfig(null); + conf.setMergePolicy(newLogMergePolicy()); + RandomIndexWriter iw = new RandomIndexWriter(random(), directory, conf); + Document doc = new Document(); + doc.add(new SortedNumericDocValuesField("sndv", 1)); + doc.add(new SortedNumericDocValuesField("dv", 1)); + doc.add(new SortedNumericDocValuesField("field", 1)); + iw.addDocument(doc); + doc.add(new SortedNumericDocValuesField("sndv", 1)); + doc.add(new SortedNumericDocValuesField("dv", 1)); + doc.add(new SortedNumericDocValuesField("field", 1)); + iw.addDocument(doc); + iw.forceMerge(1); + doc.add(new SortedNumericDocValuesField("sndv", 2)); + doc.add(new SortedNumericDocValuesField("dv", 2)); + doc.add(new SortedNumericDocValuesField("field", 2)); + iw.addDocument(doc); + doc.add(new SortedNumericDocValuesField("sndv", 2)); + doc.add(new SortedNumericDocValuesField("dv", 2)); + doc.add(new SortedNumericDocValuesField("field", 2)); + iw.addDocument(doc); + iw.forceMerge(1); + iw.close(); + + // TODO : validate star tree structures that got created + directory.close(); + } - return new StarTreeMapper.StarTreeFieldType("star_tree", starTreeField); + private XContentBuilder getExpandedMapping(String dim, String metric) throws IOException { + return topMapping(b -> { + b.startObject("composite"); + b.startObject("startree"); + b.field("type", "star_tree"); + b.startObject("config"); + b.field("max_leaf_docs", 100); + b.startArray("ordered_dimensions"); + b.startObject(); + b.field("name", "sndv"); + b.endObject(); + b.startObject(); + b.field("name", "dv"); + b.endObject(); + b.endArray(); + b.startArray("metrics"); + b.startObject(); + b.field("name", "field"); + b.startArray("stats"); + b.value("sum"); + b.value("count"); // TODO : THIS TEST FAILS. + b.endArray(); + b.endObject(); + b.endArray(); + b.endObject(); + b.endObject(); + b.endObject(); + b.startObject("properties"); + b.startObject("sndv"); + b.field("type", "integer"); + b.endObject(); + b.startObject("dv"); + b.field("type", "integer"); + b.endObject(); + b.startObject("field"); + b.field("type", "integer"); + b.endObject(); + b.endObject(); + }); } - private static StarTreeField getStarTreeField(List d1CalendarIntervals, Metric metric1) { - DateDimension d1 = new DateDimension("field", d1CalendarIntervals); - NumericDimension d2 = new NumericDimension("dv"); + private XContentBuilder topMapping(CheckedConsumer buildFields) throws IOException { + XContentBuilder builder = XContentFactory.jsonBuilder().startObject().startObject("_doc"); + buildFields.accept(builder); + return builder.endObject().endObject(); + } - List metrics = List.of(metric1); - List dims = List.of(d1, d2); - StarTreeFieldConfiguration config = new StarTreeFieldConfiguration( - 100, - Collections.emptySet(), - StarTreeFieldConfiguration.StarTreeBuildMode.OFF_HEAP + private void createMapperService(XContentBuilder builder) throws IOException { + IndexMetadata indexMetadata = IndexMetadata.builder("test") + .settings( + Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + ) + .putMapping(builder.toString()) + .build(); + IndicesModule indicesModule = new IndicesModule(Collections.emptyList()); + mapperService = MapperTestUtils.newMapperServiceWithHelperAnalyzer( + new NamedXContentRegistry(ClusterModule.getNamedXWriteables()), + createTempDir(), + Settings.EMPTY, + indicesModule, + "test" ); - - return new StarTreeField("starTree", dims, metrics, config); + mapperService.merge(indexMetadata, MapperService.MergeReason.INDEX_TEMPLATE); } } diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/CountValueAggregatorTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/CountValueAggregatorTests.java index e30e203406a6c..8e6e9e9974646 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/CountValueAggregatorTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/CountValueAggregatorTests.java @@ -13,7 +13,7 @@ import org.opensearch.test.OpenSearchTestCase; public class CountValueAggregatorTests extends OpenSearchTestCase { - private final CountValueAggregator aggregator = new CountValueAggregator(); + private final CountValueAggregator aggregator = new CountValueAggregator(StarTreeNumericType.LONG); public void testGetAggregationType() { assertEquals(MetricStat.COUNT.getTypeName(), aggregator.getAggregationType().getTypeName()); @@ -24,11 +24,11 @@ public void testGetAggregatedValueType() { } public void testGetInitialAggregatedValueForSegmentDocValue() { - assertEquals(1L, aggregator.getInitialAggregatedValueForSegmentDocValue(randomLong(), StarTreeNumericType.LONG), 0.0); + assertEquals(1L, aggregator.getInitialAggregatedValueForSegmentDocValue(randomLong()), 0.0); } public void testMergeAggregatedValueAndSegmentValue() { - assertEquals(3L, aggregator.mergeAggregatedValueAndSegmentValue(2L, 3L, StarTreeNumericType.LONG), 0.0); + assertEquals(3L, aggregator.mergeAggregatedValueAndSegmentValue(2L, 3L), 0.0); } public void testMergeAggregatedValues() { @@ -48,6 +48,6 @@ public void testToLongValue() { } public void testToStarTreeNumericTypeValue() { - assertEquals(3L, aggregator.toStarTreeNumericTypeValue(3L, StarTreeNumericType.LONG), 0.0); + assertEquals(3L, aggregator.toStarTreeNumericTypeValue(3L), 0.0); } } diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MetricAggregatorInfoTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MetricAggregatorInfoTests.java index d08f637a3f0a9..73e6aeb44cfd7 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MetricAggregatorInfoTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/MetricAggregatorInfoTests.java @@ -19,8 +19,7 @@ public void testConstructor() { MetricStat.SUM, "column1", "star_tree_field", - IndexNumericFieldData.NumericType.DOUBLE, - null + IndexNumericFieldData.NumericType.DOUBLE ); assertEquals(MetricStat.SUM, pair.getMetricStat()); assertEquals("column1", pair.getField()); @@ -31,8 +30,7 @@ public void testCountStarConstructor() { MetricStat.COUNT, "anything", "star_tree_field", - IndexNumericFieldData.NumericType.DOUBLE, - null + IndexNumericFieldData.NumericType.DOUBLE ); assertEquals(MetricStat.COUNT, pair.getMetricStat()); assertEquals("anything", pair.getField()); @@ -43,8 +41,7 @@ public void testToFieldName() { MetricStat.SUM, "column2", "star_tree_field", - IndexNumericFieldData.NumericType.DOUBLE, - null + IndexNumericFieldData.NumericType.DOUBLE ); assertEquals("star_tree_field_column2_sum", pair.toFieldName()); } @@ -54,24 +51,22 @@ public void testEquals() { MetricStat.SUM, "column1", "star_tree_field", - IndexNumericFieldData.NumericType.DOUBLE, - null + IndexNumericFieldData.NumericType.DOUBLE ); MetricAggregatorInfo pair2 = new MetricAggregatorInfo( MetricStat.SUM, "column1", "star_tree_field", - IndexNumericFieldData.NumericType.DOUBLE, - null + IndexNumericFieldData.NumericType.DOUBLE ); assertEquals(pair1, pair2); assertNotEquals( pair1, - new MetricAggregatorInfo(MetricStat.COUNT, "column1", "star_tree_field", IndexNumericFieldData.NumericType.DOUBLE, null) + new MetricAggregatorInfo(MetricStat.COUNT, "column1", "star_tree_field", IndexNumericFieldData.NumericType.DOUBLE) ); assertNotEquals( pair1, - new MetricAggregatorInfo(MetricStat.SUM, "column2", "star_tree_field", IndexNumericFieldData.NumericType.DOUBLE, null) + new MetricAggregatorInfo(MetricStat.SUM, "column2", "star_tree_field", IndexNumericFieldData.NumericType.DOUBLE) ); } @@ -80,15 +75,13 @@ public void testHashCode() { MetricStat.SUM, "column1", "star_tree_field", - IndexNumericFieldData.NumericType.DOUBLE, - null + IndexNumericFieldData.NumericType.DOUBLE ); MetricAggregatorInfo pair2 = new MetricAggregatorInfo( MetricStat.SUM, "column1", "star_tree_field", - IndexNumericFieldData.NumericType.DOUBLE, - null + IndexNumericFieldData.NumericType.DOUBLE ); assertEquals(pair1.hashCode(), pair2.hashCode()); } @@ -98,22 +91,19 @@ public void testCompareTo() { MetricStat.SUM, "column1", "star_tree_field", - IndexNumericFieldData.NumericType.DOUBLE, - null + IndexNumericFieldData.NumericType.DOUBLE ); MetricAggregatorInfo pair2 = new MetricAggregatorInfo( MetricStat.SUM, "column2", "star_tree_field", - IndexNumericFieldData.NumericType.DOUBLE, - null + IndexNumericFieldData.NumericType.DOUBLE ); MetricAggregatorInfo pair3 = new MetricAggregatorInfo( MetricStat.COUNT, "column1", "star_tree_field", - IndexNumericFieldData.NumericType.DOUBLE, - null + IndexNumericFieldData.NumericType.DOUBLE ); assertTrue(pair1.compareTo(pair2) < 0); assertTrue(pair2.compareTo(pair1) > 0); diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/SumValueAggregatorTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/SumValueAggregatorTests.java index 3fb627e7cd434..dd66d4344c9e8 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/SumValueAggregatorTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/SumValueAggregatorTests.java @@ -20,7 +20,7 @@ public class SumValueAggregatorTests extends OpenSearchTestCase { @Before public void setup() { - aggregator = new SumValueAggregator(); + aggregator = new SumValueAggregator(StarTreeNumericType.LONG); } public void testGetAggregationType() { @@ -32,21 +32,18 @@ public void testGetAggregatedValueType() { } public void testGetInitialAggregatedValueForSegmentDocValue() { - assertEquals(1.0, aggregator.getInitialAggregatedValueForSegmentDocValue(1L, StarTreeNumericType.LONG), 0.0); - assertThrows( - NullPointerException.class, - () -> aggregator.getInitialAggregatedValueForSegmentDocValue(null, StarTreeNumericType.DOUBLE) - ); + assertEquals(1.0, aggregator.getInitialAggregatedValueForSegmentDocValue(1L), 0.0); + assertThrows(NullPointerException.class, () -> aggregator.getInitialAggregatedValueForSegmentDocValue(null)); } public void testMergeAggregatedValueAndSegmentValue() { aggregator.getInitialAggregatedValue(2.0); - assertEquals(5.0, aggregator.mergeAggregatedValueAndSegmentValue(2.0, 3L, StarTreeNumericType.LONG), 0.0); + assertEquals(5.0, aggregator.mergeAggregatedValueAndSegmentValue(2.0, 3L), 0.0); } public void testMergeAggregatedValueAndSegmentValue_nullSegmentDocValue() { aggregator.getInitialAggregatedValue(2.0); - assertThrows(NullPointerException.class, () -> aggregator.mergeAggregatedValueAndSegmentValue(2.0, null, StarTreeNumericType.LONG)); + assertThrows(NullPointerException.class, () -> aggregator.mergeAggregatedValueAndSegmentValue(2.0, null)); } public void testMergeAggregatedValues() { @@ -67,6 +64,6 @@ public void testToLongValue() { } public void testToStarTreeNumericTypeValue() { - assertEquals(NumericUtils.sortableLongToDouble(3L), aggregator.toStarTreeNumericTypeValue(3L, StarTreeNumericType.DOUBLE), 0.0); + assertEquals(NumericUtils.sortableLongToDouble(3L), aggregator.toStarTreeNumericTypeValue(3L), 0.0); } } diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/ValueAggregatorFactoryTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/ValueAggregatorFactoryTests.java index ce61ab839cc61..428668511fb2e 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/ValueAggregatorFactoryTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/aggregators/ValueAggregatorFactoryTests.java @@ -15,7 +15,7 @@ public class ValueAggregatorFactoryTests extends OpenSearchTestCase { public void testGetValueAggregatorForSumType() { - ValueAggregator aggregator = ValueAggregatorFactory.getValueAggregator(MetricStat.SUM); + ValueAggregator aggregator = ValueAggregatorFactory.getValueAggregator(MetricStat.SUM, StarTreeNumericType.LONG); assertNotNull(aggregator); assertEquals(SumValueAggregator.class, aggregator.getClass()); } diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java new file mode 100644 index 0000000000000..76a7875919a8b --- /dev/null +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/AbstractStarTreeBuilderTests.java @@ -0,0 +1,2251 @@ +/* + * 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.compositeindex.datacube.startree.builder; + +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.codecs.lucene99.Lucene99Codec; +import org.apache.lucene.index.DocValues; +import org.apache.lucene.index.DocValuesType; +import org.apache.lucene.index.EmptyDocValuesProducer; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.FieldInfos; +import org.apache.lucene.index.IndexOptions; +import org.apache.lucene.index.SegmentInfo; +import org.apache.lucene.index.SegmentWriteState; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.index.VectorEncoding; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.sandbox.document.HalfFloatPoint; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.InfoStream; +import org.apache.lucene.util.NumericUtils; +import org.apache.lucene.util.Version; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.codec.composite.datacube.startree.StarTreeValues; +import org.opensearch.index.compositeindex.datacube.Dimension; +import org.opensearch.index.compositeindex.datacube.Metric; +import org.opensearch.index.compositeindex.datacube.MetricStat; +import org.opensearch.index.compositeindex.datacube.NumericDimension; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeDocument; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; +import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; +import org.opensearch.index.compositeindex.datacube.startree.utils.SequentialDocValuesIterator; +import org.opensearch.index.compositeindex.datacube.startree.utils.TreeNode; +import org.opensearch.index.mapper.ContentPath; +import org.opensearch.index.mapper.DocumentMapper; +import org.opensearch.index.mapper.Mapper; +import org.opensearch.index.mapper.MapperService; +import org.opensearch.index.mapper.MappingLookup; +import org.opensearch.index.mapper.NumberFieldMapper; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Queue; +import java.util.Set; +import java.util.UUID; + +import static org.opensearch.index.compositeindex.datacube.startree.builder.BaseStarTreeBuilder.NUM_SEGMENT_DOCS; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public abstract class AbstractStarTreeBuilderTests extends OpenSearchTestCase { + protected MapperService mapperService; + protected List dimensionsOrder; + protected List fields = List.of(); + protected List metrics; + protected Directory directory; + protected FieldInfo[] fieldsInfo; + protected StarTreeField compositeField; + protected Map fieldProducerMap; + protected SegmentWriteState writeState; + private BaseStarTreeBuilder builder; + + @Before + public void setup() throws IOException { + fields = List.of("field1", "field2", "field3", "field4", "field5", "field6", "field7", "field8", "field9", "field10"); + + dimensionsOrder = List.of( + new NumericDimension("field1"), + new NumericDimension("field3"), + new NumericDimension("field5"), + new NumericDimension("field8") + ); + metrics = List.of( + new Metric("field2", List.of(MetricStat.SUM)), + new Metric("field4", List.of(MetricStat.SUM)), + new Metric("field6", List.of(MetricStat.COUNT)) + ); + + DocValuesProducer docValuesProducer = mock(DocValuesProducer.class); + + compositeField = new StarTreeField( + "test", + dimensionsOrder, + metrics, + new StarTreeFieldConfiguration(1, Set.of("field8"), StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP) + ); + directory = newFSDirectory(createTempDir()); + + fieldsInfo = new FieldInfo[fields.size()]; + fieldProducerMap = new HashMap<>(); + for (int i = 0; i < fieldsInfo.length; i++) { + fieldsInfo[i] = new FieldInfo( + fields.get(i), + i, + false, + false, + true, + IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS, + DocValuesType.SORTED_NUMERIC, + -1, + Collections.emptyMap(), + 0, + 0, + 0, + 0, + VectorEncoding.FLOAT32, + VectorSimilarityFunction.EUCLIDEAN, + false, + false + ); + fieldProducerMap.put(fields.get(i), docValuesProducer); + } + writeState = getWriteState(5); + + mapperService = mock(MapperService.class); + DocumentMapper documentMapper = mock(DocumentMapper.class); + when(mapperService.documentMapper()).thenReturn(documentMapper); + Settings settings = Settings.builder().put(settings(org.opensearch.Version.CURRENT).build()).build(); + NumberFieldMapper numberFieldMapper1 = new NumberFieldMapper.Builder("field2", NumberFieldMapper.NumberType.DOUBLE, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper2 = new NumberFieldMapper.Builder("field4", NumberFieldMapper.NumberType.DOUBLE, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper3 = new NumberFieldMapper.Builder("field6", NumberFieldMapper.NumberType.DOUBLE, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + MappingLookup fieldMappers = new MappingLookup( + Set.of(numberFieldMapper1, numberFieldMapper2, numberFieldMapper3), + Collections.emptyList(), + Collections.emptyList(), + 0, + null + ); + when(documentMapper.mappers()).thenReturn(fieldMappers); + } + + private SegmentWriteState getWriteState(int numDocs) { + FieldInfos fieldInfos = new FieldInfos(fieldsInfo); + SegmentInfo segmentInfo = new SegmentInfo( + directory, + Version.LATEST, + Version.LUCENE_9_11_0, + "test_segment", + numDocs, + false, + false, + new Lucene99Codec(), + new HashMap<>(), + UUID.randomUUID().toString().substring(0, 16).getBytes(StandardCharsets.UTF_8), + new HashMap<>(), + null + ); + return new SegmentWriteState(InfoStream.getDefault(), segmentInfo.dir, segmentInfo, fieldInfos, null, newIOContext(random())); + } + + public abstract BaseStarTreeBuilder getStarTreeBuilder( + StarTreeField starTreeField, + SegmentWriteState segmentWriteState, + MapperService mapperService + ) throws IOException; + + public void test_sortAndAggregateStarTreeDocuments() throws IOException { + + int noOfStarTreeDocuments = 5; + StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + + starTreeDocuments[0] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Double[] { 12.0, 10.0, randomDouble() }); + starTreeDocuments[1] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 10.0, 6.0, randomDouble() }); + starTreeDocuments[2] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 14.0, 12.0, randomDouble() }); + starTreeDocuments[3] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Double[] { 9.0, 4.0, randomDouble() }); + starTreeDocuments[4] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 11.0, 16.0, randomDouble() }); + + StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + for (int i = 0; i < noOfStarTreeDocuments; i++) { + long metric1 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[0]); + long metric2 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[1]); + long metric3 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[2]); + segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, new Long[] { metric1, metric2, metric3 }); + } + List inorderStarTreeDocuments = List.of( + new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Object[] { 21.0, 14.0, 2L }), + new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Object[] { 35.0, 34.0, 3L }) + ); + Iterator expectedStarTreeDocumentIterator = inorderStarTreeDocuments.iterator(); + + SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); + List metricsIterators = getMetricIterators(segmentStarTreeDocuments); + builder = getStarTreeBuilder(compositeField, writeState, mapperService); + Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( + dimsIterators, + metricsIterators + ); + + int numOfAggregatedDocuments = 0; + while (segmentStarTreeDocumentIterator.hasNext() && expectedStarTreeDocumentIterator.hasNext()) { + StarTreeDocument resultStarTreeDocument = segmentStarTreeDocumentIterator.next(); + StarTreeDocument expectedStarTreeDocument = expectedStarTreeDocumentIterator.next(); + + assertEquals(expectedStarTreeDocument.dimensions[0], resultStarTreeDocument.dimensions[0]); + assertEquals(expectedStarTreeDocument.dimensions[1], resultStarTreeDocument.dimensions[1]); + assertEquals(expectedStarTreeDocument.dimensions[2], resultStarTreeDocument.dimensions[2]); + assertEquals(expectedStarTreeDocument.dimensions[3], resultStarTreeDocument.dimensions[3]); + assertEquals(expectedStarTreeDocument.metrics[0], resultStarTreeDocument.metrics[0]); + assertEquals(expectedStarTreeDocument.metrics[1], resultStarTreeDocument.metrics[1]); + assertEquals(expectedStarTreeDocument.metrics[2], resultStarTreeDocument.metrics[2]); + + numOfAggregatedDocuments++; + } + + assertEquals(inorderStarTreeDocuments.size(), numOfAggregatedDocuments); + } + + SequentialDocValuesIterator[] getDimensionIterators(StarTreeDocument[] starTreeDocuments) { + SequentialDocValuesIterator[] sequentialDocValuesIterators = + new SequentialDocValuesIterator[starTreeDocuments[0].dimensions.length]; + for (int j = 0; j < starTreeDocuments[0].dimensions.length; j++) { + List dimList = new ArrayList<>(); + List docsWithField = new ArrayList<>(); + + for (int i = 0; i < starTreeDocuments.length; i++) { + if (starTreeDocuments[i].dimensions[j] != null) { + dimList.add(starTreeDocuments[i].dimensions[j]); + docsWithField.add(i); + } + } + sequentialDocValuesIterators[j] = new SequentialDocValuesIterator(getSortedNumericMock(dimList, docsWithField)); + } + return sequentialDocValuesIterators; + } + + List getMetricIterators(StarTreeDocument[] starTreeDocuments) { + List sequentialDocValuesIterators = new ArrayList<>(); + for (int j = 0; j < starTreeDocuments[0].metrics.length; j++) { + List metricslist = new ArrayList<>(); + List docsWithField = new ArrayList<>(); + + for (int i = 0; i < starTreeDocuments.length; i++) { + if (starTreeDocuments[i].metrics[j] != null) { + metricslist.add((long) starTreeDocuments[i].metrics[j]); + docsWithField.add(i); + } + } + sequentialDocValuesIterators.add(new SequentialDocValuesIterator(getSortedNumericMock(metricslist, docsWithField))); + } + return sequentialDocValuesIterators; + } + + public void test_sortAndAggregateStarTreeDocuments_nullMetric() throws IOException { + + int noOfStarTreeDocuments = 5; + StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + + starTreeDocuments[0] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Double[] { 12.0, 10.0, randomDouble() }); + starTreeDocuments[1] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 10.0, 6.0, randomDouble() }); + starTreeDocuments[2] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 14.0, 12.0, randomDouble() }); + starTreeDocuments[3] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Double[] { 9.0, 4.0, randomDouble() }); + starTreeDocuments[4] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 11.0, null, randomDouble() }); + + List inorderStarTreeDocuments = List.of( + new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Object[] { 21.0, 14.0, 2L }), + new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Object[] { 35.0, 18.0, 3L }) + ); + Iterator expectedStarTreeDocumentIterator = inorderStarTreeDocuments.iterator(); + + StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + for (int i = 0; i < noOfStarTreeDocuments; i++) { + Long metric1 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[0]); + Long metric2 = starTreeDocuments[i].metrics[1] != null + ? NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[1]) + : null; + Long metric3 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[2]); + segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, new Object[] { metric1, metric2, metric3 }); + } + SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); + List metricsIterators = getMetricIterators(segmentStarTreeDocuments); + builder = getStarTreeBuilder(compositeField, writeState, mapperService); + Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( + dimsIterators, + metricsIterators + ); + + while (segmentStarTreeDocumentIterator.hasNext() && expectedStarTreeDocumentIterator.hasNext()) { + StarTreeDocument resultStarTreeDocument = segmentStarTreeDocumentIterator.next(); + StarTreeDocument expectedStarTreeDocument = expectedStarTreeDocumentIterator.next(); + assertEquals(expectedStarTreeDocument.dimensions[0], resultStarTreeDocument.dimensions[0]); + assertEquals(expectedStarTreeDocument.dimensions[1], resultStarTreeDocument.dimensions[1]); + assertEquals(expectedStarTreeDocument.dimensions[2], resultStarTreeDocument.dimensions[2]); + assertEquals(expectedStarTreeDocument.dimensions[3], resultStarTreeDocument.dimensions[3]); + assertEquals(expectedStarTreeDocument.metrics[0], resultStarTreeDocument.metrics[0]); + assertEquals(expectedStarTreeDocument.metrics[1], resultStarTreeDocument.metrics[1]); + assertEquals(expectedStarTreeDocument.metrics[2], resultStarTreeDocument.metrics[2]); + } + } + + public void test_sortAndAggregateStarTreeDocuments_nullMetricField() throws IOException { + + int noOfStarTreeDocuments = 5; + StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + // Setting second metric iterator as empty sorted numeric , indicating a metric field is null + starTreeDocuments[0] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Double[] { 12.0, null, randomDouble() }); + starTreeDocuments[1] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 10.0, null, randomDouble() }); + starTreeDocuments[2] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 14.0, null, randomDouble() }); + starTreeDocuments[3] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Double[] { 9.0, null, randomDouble() }); + starTreeDocuments[4] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 11.0, null, randomDouble() }); + + List inorderStarTreeDocuments = List.of( + new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Object[] { 21.0, 0.0, 2L }), + new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Object[] { 35.0, 0.0, 3L }) + ); + Iterator expectedStarTreeDocumentIterator = inorderStarTreeDocuments.iterator(); + + StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + for (int i = 0; i < noOfStarTreeDocuments; i++) { + Long metric1 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[0]); + Long metric2 = starTreeDocuments[i].metrics[1] != null + ? NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[1]) + : null; + Long metric3 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[2]); + segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, new Object[] { metric1, metric2, metric3 }); + } + SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); + List metricsIterators = getMetricIterators(segmentStarTreeDocuments); + builder = getStarTreeBuilder(compositeField, writeState, mapperService); + Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( + dimsIterators, + metricsIterators + ); + + while (segmentStarTreeDocumentIterator.hasNext() && expectedStarTreeDocumentIterator.hasNext()) { + StarTreeDocument resultStarTreeDocument = segmentStarTreeDocumentIterator.next(); + StarTreeDocument expectedStarTreeDocument = expectedStarTreeDocumentIterator.next(); + assertEquals(expectedStarTreeDocument.dimensions[0], resultStarTreeDocument.dimensions[0]); + assertEquals(expectedStarTreeDocument.dimensions[1], resultStarTreeDocument.dimensions[1]); + assertEquals(expectedStarTreeDocument.dimensions[2], resultStarTreeDocument.dimensions[2]); + assertEquals(expectedStarTreeDocument.dimensions[3], resultStarTreeDocument.dimensions[3]); + assertEquals(expectedStarTreeDocument.metrics[0], resultStarTreeDocument.metrics[0]); + assertEquals(expectedStarTreeDocument.metrics[1], resultStarTreeDocument.metrics[1]); + assertEquals(expectedStarTreeDocument.metrics[2], resultStarTreeDocument.metrics[2]); + } + } + + public void test_sortAndAggregateStarTreeDocuments_nullDimensionField() throws IOException { + int noOfStarTreeDocuments = 5; + StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + // Setting second metric iterator as empty sorted numeric , indicating a metric field is null + starTreeDocuments[0] = new StarTreeDocument(new Long[] { 2L, null, 3L, 4L }, new Double[] { 12.0, null, randomDouble() }); + starTreeDocuments[1] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 10.0, null, randomDouble() }); + starTreeDocuments[2] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 14.0, null, randomDouble() }); + starTreeDocuments[3] = new StarTreeDocument(new Long[] { 2L, null, 3L, 4L }, new Double[] { 9.0, null, randomDouble() }); + starTreeDocuments[4] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 11.0, null, randomDouble() }); + + List inorderStarTreeDocuments = List.of( + new StarTreeDocument(new Long[] { 2L, null, 3L, 4L }, new Object[] { 21.0, 0.0, 2L }), + new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Object[] { 35.0, 0.0, 3L }) + ); + Iterator expectedStarTreeDocumentIterator = inorderStarTreeDocuments.iterator(); + + StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + for (int i = 0; i < noOfStarTreeDocuments; i++) { + Long metric1 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[0]); + Long metric2 = starTreeDocuments[i].metrics[1] != null + ? NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[1]) + : null; + Long metric3 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[2]); + segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, new Object[] { metric1, metric2, metric3 }); + } + SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); + List metricsIterators = getMetricIterators(segmentStarTreeDocuments); + builder = getStarTreeBuilder(compositeField, writeState, mapperService); + Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( + dimsIterators, + metricsIterators + ); + + while (segmentStarTreeDocumentIterator.hasNext() && expectedStarTreeDocumentIterator.hasNext()) { + StarTreeDocument resultStarTreeDocument = segmentStarTreeDocumentIterator.next(); + StarTreeDocument expectedStarTreeDocument = expectedStarTreeDocumentIterator.next(); + assertEquals(expectedStarTreeDocument.dimensions[0], resultStarTreeDocument.dimensions[0]); + assertEquals(expectedStarTreeDocument.dimensions[1], resultStarTreeDocument.dimensions[1]); + assertEquals(expectedStarTreeDocument.dimensions[2], resultStarTreeDocument.dimensions[2]); + assertEquals(expectedStarTreeDocument.dimensions[3], resultStarTreeDocument.dimensions[3]); + assertEquals(expectedStarTreeDocument.metrics[0], resultStarTreeDocument.metrics[0]); + assertEquals(expectedStarTreeDocument.metrics[1], resultStarTreeDocument.metrics[1]); + assertEquals(expectedStarTreeDocument.metrics[2], resultStarTreeDocument.metrics[2]); + } + } + + public void test_sortAndAggregateStarTreeDocuments_nullDimensionsAndNullMetrics() throws IOException { + int noOfStarTreeDocuments = 5; + StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + // Setting second metric iterator as empty sorted numeric , indicating a metric field is null + starTreeDocuments[0] = new StarTreeDocument(new Long[] { null, null, null, null }, new Double[] { null, null, null }); + starTreeDocuments[1] = new StarTreeDocument(new Long[] { null, null, null, null }, new Double[] { null, null, null }); + starTreeDocuments[2] = new StarTreeDocument(new Long[] { null, null, null, null }, new Double[] { null, null, null }); + starTreeDocuments[3] = new StarTreeDocument(new Long[] { null, null, null, null }, new Double[] { null, null, null }); + starTreeDocuments[4] = new StarTreeDocument(new Long[] { null, null, null, null }, new Double[] { null, null, null }); + + List inorderStarTreeDocuments = List.of(); + Iterator expectedStarTreeDocumentIterator = inorderStarTreeDocuments.iterator(); + + StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + for (int i = 0; i < noOfStarTreeDocuments; i++) { + Long metric1 = starTreeDocuments[i].metrics[1] != null + ? NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[0]) + : null; + Long metric2 = starTreeDocuments[i].metrics[1] != null + ? NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[1]) + : null; + Long metric3 = starTreeDocuments[i].metrics[1] != null + ? NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[2]) + : null; + segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, new Object[] { metric1, metric2, metric3 }); + } + SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); + List metricsIterators = getMetricIterators(segmentStarTreeDocuments); + builder = getStarTreeBuilder(compositeField, writeState, mapperService); + Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( + dimsIterators, + metricsIterators + ); + + while (segmentStarTreeDocumentIterator.hasNext() && expectedStarTreeDocumentIterator.hasNext()) { + StarTreeDocument resultStarTreeDocument = segmentStarTreeDocumentIterator.next(); + StarTreeDocument expectedStarTreeDocument = expectedStarTreeDocumentIterator.next(); + assertEquals(expectedStarTreeDocument.dimensions[0], resultStarTreeDocument.dimensions[0]); + assertEquals(expectedStarTreeDocument.dimensions[1], resultStarTreeDocument.dimensions[1]); + assertEquals(expectedStarTreeDocument.dimensions[2], resultStarTreeDocument.dimensions[2]); + assertEquals(expectedStarTreeDocument.dimensions[3], resultStarTreeDocument.dimensions[3]); + assertEquals(expectedStarTreeDocument.metrics[0], resultStarTreeDocument.metrics[0]); + assertEquals(expectedStarTreeDocument.metrics[1], resultStarTreeDocument.metrics[1]); + assertEquals(expectedStarTreeDocument.metrics[2], resultStarTreeDocument.metrics[2]); + } + } + + public void test_sortAndAggregateStarTreeDocuments_emptyDimensions() throws IOException { + + int noOfStarTreeDocuments = 5; + StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + // Setting second metric iterator as empty sorted numeric , indicating a metric field is null + starTreeDocuments[0] = new StarTreeDocument(new Long[] { null, null, null, null }, new Double[] { 12.0, null, randomDouble() }); + starTreeDocuments[1] = new StarTreeDocument(new Long[] { null, null, null, null }, new Double[] { 10.0, null, randomDouble() }); + starTreeDocuments[2] = new StarTreeDocument(new Long[] { null, null, null, null }, new Double[] { 14.0, null, randomDouble() }); + starTreeDocuments[3] = new StarTreeDocument(new Long[] { null, null, null, null }, new Double[] { 9.0, null, randomDouble() }); + starTreeDocuments[4] = new StarTreeDocument(new Long[] { null, null, null, null }, new Double[] { 11.0, null, randomDouble() }); + + List inorderStarTreeDocuments = List.of( + new StarTreeDocument(new Long[] { null, null, null, null }, new Object[] { 56.0, 0.0, 5L }) + ); + Iterator expectedStarTreeDocumentIterator = inorderStarTreeDocuments.iterator(); + + StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + for (int i = 0; i < noOfStarTreeDocuments; i++) { + Long metric1 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[0]); + Long metric2 = starTreeDocuments[i].metrics[1] != null + ? NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[1]) + : null; + Long metric3 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[2]); + segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, new Object[] { metric1, metric2, metric3 }); + } + SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); + List metricsIterators = getMetricIterators(segmentStarTreeDocuments); + builder = getStarTreeBuilder(compositeField, writeState, mapperService); + Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( + dimsIterators, + metricsIterators + ); + + while (segmentStarTreeDocumentIterator.hasNext() && expectedStarTreeDocumentIterator.hasNext()) { + StarTreeDocument resultStarTreeDocument = segmentStarTreeDocumentIterator.next(); + StarTreeDocument expectedStarTreeDocument = expectedStarTreeDocumentIterator.next(); + assertEquals(expectedStarTreeDocument.dimensions[0], resultStarTreeDocument.dimensions[0]); + assertEquals(expectedStarTreeDocument.dimensions[1], resultStarTreeDocument.dimensions[1]); + assertEquals(expectedStarTreeDocument.dimensions[2], resultStarTreeDocument.dimensions[2]); + assertEquals(expectedStarTreeDocument.dimensions[3], resultStarTreeDocument.dimensions[3]); + assertEquals(expectedStarTreeDocument.metrics[0], resultStarTreeDocument.metrics[0]); + assertEquals(expectedStarTreeDocument.metrics[1], resultStarTreeDocument.metrics[1]); + assertEquals(expectedStarTreeDocument.metrics[2], resultStarTreeDocument.metrics[2]); + } + } + + public void test_sortAndAggregateStarTreeDocument_longMaxAndLongMinDimensions() throws IOException { + + int noOfStarTreeDocuments = 5; + StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + + starTreeDocuments[0] = new StarTreeDocument(new Long[] { Long.MIN_VALUE, 4L, 3L, 4L }, new Double[] { 12.0, 10.0, randomDouble() }); + starTreeDocuments[1] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, Long.MAX_VALUE }, new Double[] { 10.0, 6.0, randomDouble() }); + starTreeDocuments[2] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, Long.MAX_VALUE }, new Double[] { 14.0, 12.0, randomDouble() }); + starTreeDocuments[3] = new StarTreeDocument(new Long[] { Long.MIN_VALUE, 4L, 3L, 4L }, new Double[] { 9.0, 4.0, randomDouble() }); + starTreeDocuments[4] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, Long.MAX_VALUE }, new Double[] { 11.0, 16.0, randomDouble() }); + + List inorderStarTreeDocuments = List.of( + new StarTreeDocument(new Long[] { Long.MIN_VALUE, 4L, 3L, 4L }, new Object[] { 21.0, 14.0, 2L }), + new StarTreeDocument(new Long[] { 3L, 4L, 2L, Long.MAX_VALUE }, new Object[] { 35.0, 34.0, 3L }) + ); + Iterator expectedStarTreeDocumentIterator = inorderStarTreeDocuments.iterator(); + + StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + for (int i = 0; i < noOfStarTreeDocuments; i++) { + long metric1 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[0]); + long metric2 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[1]); + long metric3 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[2]); + segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, new Long[] { metric1, metric2, metric3 }); + } + + SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); + List metricsIterators = getMetricIterators(segmentStarTreeDocuments); + builder = getStarTreeBuilder(compositeField, writeState, mapperService); + Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( + dimsIterators, + metricsIterators + ); + int numOfAggregatedDocuments = 0; + while (segmentStarTreeDocumentIterator.hasNext() && expectedStarTreeDocumentIterator.hasNext()) { + StarTreeDocument resultStarTreeDocument = segmentStarTreeDocumentIterator.next(); + StarTreeDocument expectedStarTreeDocument = expectedStarTreeDocumentIterator.next(); + + assertEquals(expectedStarTreeDocument.dimensions[0], resultStarTreeDocument.dimensions[0]); + assertEquals(expectedStarTreeDocument.dimensions[1], resultStarTreeDocument.dimensions[1]); + assertEquals(expectedStarTreeDocument.dimensions[2], resultStarTreeDocument.dimensions[2]); + assertEquals(expectedStarTreeDocument.dimensions[3], resultStarTreeDocument.dimensions[3]); + assertEquals(expectedStarTreeDocument.metrics[0], resultStarTreeDocument.metrics[0]); + assertEquals(expectedStarTreeDocument.metrics[1], resultStarTreeDocument.metrics[1]); + assertEquals(expectedStarTreeDocument.metrics[2], resultStarTreeDocument.metrics[2]); + + numOfAggregatedDocuments++; + } + + assertEquals(inorderStarTreeDocuments.size(), numOfAggregatedDocuments); + + } + + public void test_sortAndAggregateStarTreeDocument_DoubleMaxAndDoubleMinMetrics() throws IOException { + + int noOfStarTreeDocuments = 5; + StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + + starTreeDocuments[0] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Double[] { Double.MAX_VALUE, 10.0, randomDouble() }); + starTreeDocuments[1] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 10.0, 6.0, randomDouble() }); + starTreeDocuments[2] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 14.0, Double.MIN_VALUE, randomDouble() }); + starTreeDocuments[3] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Double[] { 9.0, 4.0, randomDouble() }); + starTreeDocuments[4] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 11.0, 16.0, randomDouble() }); + + List inorderStarTreeDocuments = List.of( + new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Object[] { Double.MAX_VALUE + 9, 14.0, 2L }), + new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Object[] { 35.0, Double.MIN_VALUE + 22, 3L }) + ); + Iterator expectedStarTreeDocumentIterator = inorderStarTreeDocuments.iterator(); + + StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + for (int i = 0; i < noOfStarTreeDocuments; i++) { + long metric1 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[0]); + long metric2 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[1]); + long metric3 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[2]); + segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, new Long[] { metric1, metric2, metric3 }); + } + + SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); + List metricsIterators = getMetricIterators(segmentStarTreeDocuments); + builder = getStarTreeBuilder(compositeField, writeState, mapperService); + Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( + dimsIterators, + metricsIterators + ); + int numOfAggregatedDocuments = 0; + while (segmentStarTreeDocumentIterator.hasNext() && expectedStarTreeDocumentIterator.hasNext()) { + StarTreeDocument resultStarTreeDocument = segmentStarTreeDocumentIterator.next(); + StarTreeDocument expectedStarTreeDocument = expectedStarTreeDocumentIterator.next(); + + assertEquals(expectedStarTreeDocument.dimensions[0], resultStarTreeDocument.dimensions[0]); + assertEquals(expectedStarTreeDocument.dimensions[1], resultStarTreeDocument.dimensions[1]); + assertEquals(expectedStarTreeDocument.dimensions[2], resultStarTreeDocument.dimensions[2]); + assertEquals(expectedStarTreeDocument.dimensions[3], resultStarTreeDocument.dimensions[3]); + assertEquals(expectedStarTreeDocument.metrics[0], resultStarTreeDocument.metrics[0]); + assertEquals(expectedStarTreeDocument.metrics[1], resultStarTreeDocument.metrics[1]); + assertEquals(expectedStarTreeDocument.metrics[2], resultStarTreeDocument.metrics[2]); + + numOfAggregatedDocuments++; + } + + assertEquals(inorderStarTreeDocuments.size(), numOfAggregatedDocuments); + + } + + public void test_build_halfFloatMetrics() throws IOException { + + mapperService = mock(MapperService.class); + DocumentMapper documentMapper = mock(DocumentMapper.class); + when(mapperService.documentMapper()).thenReturn(documentMapper); + Settings settings = Settings.builder().put(settings(org.opensearch.Version.CURRENT).build()).build(); + NumberFieldMapper numberFieldMapper1 = new NumberFieldMapper.Builder("field2", NumberFieldMapper.NumberType.HALF_FLOAT, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper2 = new NumberFieldMapper.Builder("field4", NumberFieldMapper.NumberType.HALF_FLOAT, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper3 = new NumberFieldMapper.Builder("field6", NumberFieldMapper.NumberType.HALF_FLOAT, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + MappingLookup fieldMappers = new MappingLookup( + Set.of(numberFieldMapper1, numberFieldMapper2, numberFieldMapper3), + Collections.emptyList(), + Collections.emptyList(), + 0, + null + ); + when(documentMapper.mappers()).thenReturn(fieldMappers); + + int noOfStarTreeDocuments = 5; + StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + + starTreeDocuments[0] = new StarTreeDocument( + new Long[] { 2L, 4L, 3L, 4L }, + new HalfFloatPoint[] { new HalfFloatPoint("hf1", 12), new HalfFloatPoint("hf6", 10), new HalfFloatPoint("field6", 10) } + ); + starTreeDocuments[1] = new StarTreeDocument( + new Long[] { 3L, 4L, 2L, 1L }, + new HalfFloatPoint[] { new HalfFloatPoint("hf2", 10), new HalfFloatPoint("hf7", 6), new HalfFloatPoint("field6", 10) } + ); + starTreeDocuments[2] = new StarTreeDocument( + new Long[] { 3L, 4L, 2L, 1L }, + new HalfFloatPoint[] { new HalfFloatPoint("hf3", 14), new HalfFloatPoint("hf8", 12), new HalfFloatPoint("field6", 10) } + ); + starTreeDocuments[3] = new StarTreeDocument( + new Long[] { 2L, 4L, 3L, 4L }, + new HalfFloatPoint[] { new HalfFloatPoint("hf4", 9), new HalfFloatPoint("hf9", 4), new HalfFloatPoint("field6", 10) } + ); + starTreeDocuments[4] = new StarTreeDocument( + new Long[] { 3L, 4L, 2L, 1L }, + new HalfFloatPoint[] { new HalfFloatPoint("hf5", 11), new HalfFloatPoint("hf10", 16), new HalfFloatPoint("field6", 10) } + ); + + StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + for (int i = 0; i < noOfStarTreeDocuments; i++) { + long metric1 = HalfFloatPoint.halfFloatToSortableShort( + ((HalfFloatPoint) starTreeDocuments[i].metrics[0]).numericValue().floatValue() + ); + long metric2 = HalfFloatPoint.halfFloatToSortableShort( + ((HalfFloatPoint) starTreeDocuments[i].metrics[1]).numericValue().floatValue() + ); + long metric3 = HalfFloatPoint.halfFloatToSortableShort( + ((HalfFloatPoint) starTreeDocuments[i].metrics[2]).numericValue().floatValue() + ); + segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, new Long[] { metric1, metric2, metric3 }); + } + + SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); + List metricsIterators = getMetricIterators(segmentStarTreeDocuments); + builder = getStarTreeBuilder(compositeField, writeState, mapperService); + + Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( + dimsIterators, + metricsIterators + ); + builder.build(segmentStarTreeDocumentIterator); + List resultStarTreeDocuments = builder.getStarTreeDocuments(); + assertEquals(7, resultStarTreeDocuments.size()); + + Iterator expectedStarTreeDocumentIterator = getExpectedStarTreeDocumentIterator(); + assertStarTreeDocuments(resultStarTreeDocuments, expectedStarTreeDocumentIterator); + } + + public void test_build_floatMetrics() throws IOException { + + mapperService = mock(MapperService.class); + DocumentMapper documentMapper = mock(DocumentMapper.class); + when(mapperService.documentMapper()).thenReturn(documentMapper); + Settings settings = Settings.builder().put(settings(org.opensearch.Version.CURRENT).build()).build(); + NumberFieldMapper numberFieldMapper1 = new NumberFieldMapper.Builder("field2", NumberFieldMapper.NumberType.FLOAT, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper2 = new NumberFieldMapper.Builder("field4", NumberFieldMapper.NumberType.FLOAT, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper3 = new NumberFieldMapper.Builder("field6", NumberFieldMapper.NumberType.FLOAT, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + MappingLookup fieldMappers = new MappingLookup( + Set.of(numberFieldMapper1, numberFieldMapper2, numberFieldMapper3), + Collections.emptyList(), + Collections.emptyList(), + 0, + null + ); + when(documentMapper.mappers()).thenReturn(fieldMappers); + + int noOfStarTreeDocuments = 5; + StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + + starTreeDocuments[0] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Float[] { 12.0F, 10.0F, randomFloat() }); + starTreeDocuments[1] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Float[] { 10.0F, 6.0F, randomFloat() }); + starTreeDocuments[2] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Float[] { 14.0F, 12.0F, randomFloat() }); + starTreeDocuments[3] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Float[] { 9.0F, 4.0F, randomFloat() }); + starTreeDocuments[4] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Float[] { 11.0F, 16.0F, randomFloat() }); + + StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + for (int i = 0; i < noOfStarTreeDocuments; i++) { + long metric1 = NumericUtils.floatToSortableInt((Float) starTreeDocuments[i].metrics[0]); + long metric2 = NumericUtils.floatToSortableInt((Float) starTreeDocuments[i].metrics[1]); + long metric3 = NumericUtils.floatToSortableInt((Float) starTreeDocuments[i].metrics[2]); + segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, new Long[] { metric1, metric2, metric3 }); + } + + SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); + List metricsIterators = getMetricIterators(segmentStarTreeDocuments); + builder = getStarTreeBuilder(compositeField, writeState, mapperService); + Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( + dimsIterators, + metricsIterators + ); + builder.build(segmentStarTreeDocumentIterator); + + List resultStarTreeDocuments = builder.getStarTreeDocuments(); + assertEquals(7, resultStarTreeDocuments.size()); + + Iterator expectedStarTreeDocumentIterator = getExpectedStarTreeDocumentIterator(); + assertStarTreeDocuments(resultStarTreeDocuments, expectedStarTreeDocumentIterator); + } + + public void test_build_longMetrics() throws IOException { + + mapperService = mock(MapperService.class); + DocumentMapper documentMapper = mock(DocumentMapper.class); + when(mapperService.documentMapper()).thenReturn(documentMapper); + Settings settings = Settings.builder().put(settings(org.opensearch.Version.CURRENT).build()).build(); + NumberFieldMapper numberFieldMapper1 = new NumberFieldMapper.Builder("field2", NumberFieldMapper.NumberType.LONG, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper2 = new NumberFieldMapper.Builder("field4", NumberFieldMapper.NumberType.LONG, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + NumberFieldMapper numberFieldMapper3 = new NumberFieldMapper.Builder("field6", NumberFieldMapper.NumberType.LONG, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + MappingLookup fieldMappers = new MappingLookup( + Set.of(numberFieldMapper1, numberFieldMapper2, numberFieldMapper3), + Collections.emptyList(), + Collections.emptyList(), + 0, + null + ); + when(documentMapper.mappers()).thenReturn(fieldMappers); + + int noOfStarTreeDocuments = 5; + StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + + starTreeDocuments[0] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Long[] { 12L, 10L, randomLong() }); + starTreeDocuments[1] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Long[] { 10L, 6L, randomLong() }); + starTreeDocuments[2] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Long[] { 14L, 12L, randomLong() }); + starTreeDocuments[3] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Long[] { 9L, 4L, randomLong() }); + starTreeDocuments[4] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Long[] { 11L, 16L, randomLong() }); + + StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + for (int i = 0; i < noOfStarTreeDocuments; i++) { + long metric1 = (Long) starTreeDocuments[i].metrics[0]; + long metric2 = (Long) starTreeDocuments[i].metrics[1]; + long metric3 = (Long) starTreeDocuments[i].metrics[2]; + segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, new Long[] { metric1, metric2, metric3 }); + } + + SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); + List metricsIterators = getMetricIterators(segmentStarTreeDocuments); + builder = getStarTreeBuilder(compositeField, writeState, mapperService); + Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( + dimsIterators, + metricsIterators + ); + builder.build(segmentStarTreeDocumentIterator); + + List resultStarTreeDocuments = builder.getStarTreeDocuments(); + assertEquals(7, resultStarTreeDocuments.size()); + + Iterator expectedStarTreeDocumentIterator = getExpectedStarTreeDocumentIterator(); + assertStarTreeDocuments(resultStarTreeDocuments, expectedStarTreeDocumentIterator); + } + + private static Iterator getExpectedStarTreeDocumentIterator() { + List expectedStarTreeDocuments = List.of( + new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Object[] { 21.0, 14.0, 2L }), + new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Object[] { 35.0, 34.0, 3L }), + new StarTreeDocument(new Long[] { null, 4L, 2L, 1L }, new Object[] { 35.0, 34.0, 3L }), + new StarTreeDocument(new Long[] { null, 4L, 3L, 4L }, new Object[] { 21.0, 14.0, 2L }), + new StarTreeDocument(new Long[] { null, 4L, null, 1L }, new Object[] { 35.0, 34.0, 3L }), + new StarTreeDocument(new Long[] { null, 4L, null, 4L }, new Object[] { 21.0, 14.0, 2L }), + new StarTreeDocument(new Long[] { null, 4L, null, null }, new Object[] { 56.0, 48.0, 5L }), + new StarTreeDocument(new Long[] { null, null, null, null }, new Object[] { 56.0, 48.0, 5L }) + ); + return expectedStarTreeDocuments.iterator(); + } + + public void test_build() throws IOException { + + int noOfStarTreeDocuments = 5; + StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + + starTreeDocuments[0] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Double[] { 12.0, 10.0, randomDouble() }); + starTreeDocuments[1] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 10.0, 6.0, randomDouble() }); + starTreeDocuments[2] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 14.0, 12.0, randomDouble() }); + starTreeDocuments[3] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Double[] { 9.0, 4.0, randomDouble() }); + starTreeDocuments[4] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 11.0, 16.0, randomDouble() }); + + StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + for (int i = 0; i < noOfStarTreeDocuments; i++) { + long metric1 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[0]); + long metric2 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[1]); + long metric3 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[2]); + segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, new Long[] { metric1, metric2, metric3 }); + } + + SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); + List metricsIterators = getMetricIterators(segmentStarTreeDocuments); + builder = getStarTreeBuilder(compositeField, writeState, mapperService); + Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( + dimsIterators, + metricsIterators + ); + builder.build(segmentStarTreeDocumentIterator); + + List resultStarTreeDocuments = builder.getStarTreeDocuments(); + assertEquals(7, resultStarTreeDocuments.size()); + + Iterator expectedStarTreeDocumentIterator = getExpectedStarTreeDocumentIterator(); + assertStarTreeDocuments(resultStarTreeDocuments, expectedStarTreeDocumentIterator); + } + + private void assertStarTreeDocuments( + List resultStarTreeDocuments, + Iterator expectedStarTreeDocumentIterator + ) { + Iterator resultStarTreeDocumentIterator = resultStarTreeDocuments.iterator(); + while (resultStarTreeDocumentIterator.hasNext() && expectedStarTreeDocumentIterator.hasNext()) { + StarTreeDocument resultStarTreeDocument = resultStarTreeDocumentIterator.next(); + StarTreeDocument expectedStarTreeDocument = expectedStarTreeDocumentIterator.next(); + + assertEquals(expectedStarTreeDocument.dimensions[0], resultStarTreeDocument.dimensions[0]); + assertEquals(expectedStarTreeDocument.dimensions[1], resultStarTreeDocument.dimensions[1]); + assertEquals(expectedStarTreeDocument.dimensions[2], resultStarTreeDocument.dimensions[2]); + assertEquals(expectedStarTreeDocument.dimensions[3], resultStarTreeDocument.dimensions[3]); + assertEquals(expectedStarTreeDocument.metrics[0], resultStarTreeDocument.metrics[0]); + assertEquals(expectedStarTreeDocument.metrics[1], resultStarTreeDocument.metrics[1]); + assertEquals(expectedStarTreeDocument.metrics[2], resultStarTreeDocument.metrics[2]); + } + } + + public void test_build_starTreeDataset() throws IOException { + + fields = List.of("fieldC", "fieldB", "fieldL", "fieldI"); + + dimensionsOrder = List.of(new NumericDimension("fieldC"), new NumericDimension("fieldB"), new NumericDimension("fieldL")); + metrics = List.of(new Metric("fieldI", List.of(MetricStat.SUM))); + + DocValuesProducer docValuesProducer = mock(DocValuesProducer.class); + + compositeField = new StarTreeField( + "test", + dimensionsOrder, + metrics, + new StarTreeFieldConfiguration(1, Set.of(), StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP) + ); + SegmentInfo segmentInfo = new SegmentInfo( + directory, + Version.LATEST, + Version.LUCENE_9_11_0, + "test_segment", + 7, + false, + false, + new Lucene99Codec(), + new HashMap<>(), + UUID.randomUUID().toString().substring(0, 16).getBytes(StandardCharsets.UTF_8), + new HashMap<>(), + null + ); + + fieldsInfo = new FieldInfo[fields.size()]; + fieldProducerMap = new HashMap<>(); + for (int i = 0; i < fieldsInfo.length; i++) { + fieldsInfo[i] = new FieldInfo( + fields.get(i), + i, + false, + false, + true, + IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS, + DocValuesType.SORTED_NUMERIC, + -1, + Collections.emptyMap(), + 0, + 0, + 0, + 0, + VectorEncoding.FLOAT32, + VectorSimilarityFunction.EUCLIDEAN, + false, + false + ); + fieldProducerMap.put(fields.get(i), docValuesProducer); + } + FieldInfos fieldInfos = new FieldInfos(fieldsInfo); + writeState = new SegmentWriteState(InfoStream.getDefault(), segmentInfo.dir, segmentInfo, fieldInfos, null, newIOContext(random())); + + mapperService = mock(MapperService.class); + DocumentMapper documentMapper = mock(DocumentMapper.class); + when(mapperService.documentMapper()).thenReturn(documentMapper); + Settings settings = Settings.builder().put(settings(org.opensearch.Version.CURRENT).build()).build(); + NumberFieldMapper numberFieldMapper1 = new NumberFieldMapper.Builder("fieldI", NumberFieldMapper.NumberType.DOUBLE, false, true) + .build(new Mapper.BuilderContext(settings, new ContentPath())); + MappingLookup fieldMappers = new MappingLookup( + Set.of(numberFieldMapper1), + Collections.emptyList(), + Collections.emptyList(), + 0, + null + ); + when(documentMapper.mappers()).thenReturn(fieldMappers); + + int noOfStarTreeDocuments = 7; + StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + starTreeDocuments[0] = new StarTreeDocument(new Long[] { 1L, 11L, 21L }, new Double[] { 400.0 }); + starTreeDocuments[1] = new StarTreeDocument(new Long[] { 1L, 12L, 22L }, new Double[] { 200.0 }); + starTreeDocuments[2] = new StarTreeDocument(new Long[] { 2L, 13L, 23L }, new Double[] { 300.0 }); + starTreeDocuments[3] = new StarTreeDocument(new Long[] { 2L, 13L, 21L }, new Double[] { 100.0 }); + starTreeDocuments[4] = new StarTreeDocument(new Long[] { 3L, 11L, 21L }, new Double[] { 600.0 }); + starTreeDocuments[5] = new StarTreeDocument(new Long[] { 3L, 12L, 23L }, new Double[] { 200.0 }); + starTreeDocuments[6] = new StarTreeDocument(new Long[] { 3L, 12L, 21L }, new Double[] { 400.0 }); + + StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; + for (int i = 0; i < noOfStarTreeDocuments; i++) { + long metric1 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[0]); + segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, new Long[] { metric1 }); + } + + SequentialDocValuesIterator[] dimsIterators = getDimensionIterators(segmentStarTreeDocuments); + List metricsIterators = getMetricIterators(segmentStarTreeDocuments); + builder = getStarTreeBuilder(compositeField, writeState, mapperService); + Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( + dimsIterators, + metricsIterators + ); + builder.build(segmentStarTreeDocumentIterator); + + List resultStarTreeDocuments = builder.getStarTreeDocuments(); + Iterator expectedStarTreeDocumentIterator = expectedStarTreeDocuments(); + Iterator resultStarTreeDocumentIterator = resultStarTreeDocuments.iterator(); + Map> dimValueToDocIdMap = new HashMap<>(); + builder.rootNode.isStarNode = true; + traverseStarTree(builder.rootNode, dimValueToDocIdMap, true); + + Map> expectedDimToValueMap = getExpectedDimToValueMap(); + for (Map.Entry> entry : dimValueToDocIdMap.entrySet()) { + int dimId = entry.getKey(); + if (dimId == -1) continue; + Map map = expectedDimToValueMap.get(dimId); + for (Map.Entry dimValueToDocIdEntry : entry.getValue().entrySet()) { + long dimValue = dimValueToDocIdEntry.getKey(); + int docId = dimValueToDocIdEntry.getValue(); + if (map.get(dimValue) != null) { + assertEquals(map.get(dimValue), resultStarTreeDocuments.get(docId).metrics[0]); + } + } + } + + while (resultStarTreeDocumentIterator.hasNext() && expectedStarTreeDocumentIterator.hasNext()) { + StarTreeDocument resultStarTreeDocument = resultStarTreeDocumentIterator.next(); + StarTreeDocument expectedStarTreeDocument = expectedStarTreeDocumentIterator.next(); + assertEquals(expectedStarTreeDocument.dimensions[0], resultStarTreeDocument.dimensions[0]); + assertEquals(expectedStarTreeDocument.dimensions[1], resultStarTreeDocument.dimensions[1]); + assertEquals(expectedStarTreeDocument.dimensions[2], resultStarTreeDocument.dimensions[2]); + assertEquals(expectedStarTreeDocument.metrics[0], resultStarTreeDocument.metrics[0]); + } + } + + private static Map> getExpectedDimToValueMap() { + Map> expectedDimToValueMap = new HashMap<>(); + Map dimValueMap = new HashMap<>(); + dimValueMap.put(1L, 600.0); + dimValueMap.put(2L, 400.0); + dimValueMap.put(3L, 1200.0); + expectedDimToValueMap.put(0, dimValueMap); + + dimValueMap = new HashMap<>(); + dimValueMap.put(11L, 1000.0); + dimValueMap.put(12L, 800.0); + dimValueMap.put(13L, 400.0); + expectedDimToValueMap.put(1, dimValueMap); + + dimValueMap = new HashMap<>(); + dimValueMap.put(21L, 1500.0); + dimValueMap.put(22L, 200.0); + dimValueMap.put(23L, 500.0); + expectedDimToValueMap.put(2, dimValueMap); + return expectedDimToValueMap; + } + + private Iterator expectedStarTreeDocuments() { + List expectedStarTreeDocuments = List.of( + new StarTreeDocument(new Long[] { 1L, 11L, 21L }, new Object[] { 400.0 }), + new StarTreeDocument(new Long[] { 1L, 12L, 22L }, new Object[] { 200.0 }), + new StarTreeDocument(new Long[] { 2L, 13L, 21L }, new Object[] { 100.0 }), + new StarTreeDocument(new Long[] { 2L, 13L, 23L }, new Object[] { 300.0 }), + new StarTreeDocument(new Long[] { 3L, 11L, 21L }, new Object[] { 600.0 }), + new StarTreeDocument(new Long[] { 3L, 12L, 21L }, new Object[] { 400.0 }), + new StarTreeDocument(new Long[] { 3L, 12L, 23L }, new Object[] { 200.0 }), + new StarTreeDocument(new Long[] { null, 11L, 21L }, new Object[] { 1000.0 }), + new StarTreeDocument(new Long[] { null, 12L, 21L }, new Object[] { 400.0 }), + new StarTreeDocument(new Long[] { null, 12L, 22L }, new Object[] { 200.0 }), + new StarTreeDocument(new Long[] { null, 12L, 23L }, new Object[] { 200.0 }), + new StarTreeDocument(new Long[] { null, 13L, 21L }, new Object[] { 100.0 }), + new StarTreeDocument(new Long[] { null, 13L, 23L }, new Object[] { 300.0 }), + new StarTreeDocument(new Long[] { null, null, 21L }, new Object[] { 1500.0 }), + new StarTreeDocument(new Long[] { null, null, 22L }, new Object[] { 200.0 }), + new StarTreeDocument(new Long[] { null, null, 23L }, new Object[] { 500.0 }), + new StarTreeDocument(new Long[] { null, null, null }, new Object[] { 2200.0 }), + new StarTreeDocument(new Long[] { null, 12L, null }, new Object[] { 800.0 }), + new StarTreeDocument(new Long[] { null, 13L, null }, new Object[] { 400.0 }), + new StarTreeDocument(new Long[] { 1L, null, 21L }, new Object[] { 400.0 }), + new StarTreeDocument(new Long[] { 1L, null, 22L }, new Object[] { 200.0 }), + new StarTreeDocument(new Long[] { 1L, null, null }, new Object[] { 600.0 }), + new StarTreeDocument(new Long[] { 2L, 13L, null }, new Object[] { 400.0 }), + new StarTreeDocument(new Long[] { 3L, null, 21L }, new Object[] { 1000.0 }), + new StarTreeDocument(new Long[] { 3L, null, 23L }, new Object[] { 200.0 }), + new StarTreeDocument(new Long[] { 3L, null, null }, new Object[] { 1200.0 }), + new StarTreeDocument(new Long[] { 3L, 12L, null }, new Object[] { 600.0 }) + ); + + return expectedStarTreeDocuments.iterator(); + } + + public void testFlushFlow() throws IOException { + List dimList = List.of(0L, 1L, 3L, 4L, 5L); + List docsWithField = List.of(0, 1, 3, 4, 5); + List dimList2 = List.of(0L, 1L, 2L, 3L, 4L, 5L); + List docsWithField2 = List.of(0, 1, 2, 3, 4, 5); + + List metricsList = List.of( + getLongFromDouble(0.0), + getLongFromDouble(10.0), + getLongFromDouble(20.0), + getLongFromDouble(30.0), + getLongFromDouble(40.0), + getLongFromDouble(50.0) + ); + List metricsWithField = List.of(0, 1, 2, 3, 4, 5); + + StarTreeField sf = getStarTreeFieldWithMultipleMetrics(); + SortedNumericDocValues d1sndv = getSortedNumericMock(dimList, docsWithField); + SortedNumericDocValues d2sndv = getSortedNumericMock(dimList2, docsWithField2); + SortedNumericDocValues m1sndv = getSortedNumericMock(metricsList, metricsWithField); + SortedNumericDocValues m2sndv = getSortedNumericMock(metricsList, metricsWithField); + + OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(sf, getWriteState(6), mapperService); + SequentialDocValuesIterator[] dimDvs = { new SequentialDocValuesIterator(d1sndv), new SequentialDocValuesIterator(d2sndv) }; + Iterator starTreeDocumentIterator = builder.sortAndAggregateSegmentDocuments( + dimDvs, + List.of(new SequentialDocValuesIterator(m1sndv), new SequentialDocValuesIterator(m2sndv)) + ); + /** + * Asserting following dim / metrics [ dim1, dim2 / Sum [metric], count [metric] ] + [0, 0] | [0.0, 1] + [1, 1] | [10.0, 1] + [3, 3] | [30.0, 1] + [4, 4] | [40.0, 1] + [5, 5] | [50.0, 1] + [null, 2] | [20.0, 1] + */ + int count = 0; + while (starTreeDocumentIterator.hasNext()) { + count++; + StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); + assertEquals( + starTreeDocument.dimensions[0] != null ? starTreeDocument.dimensions[0] * 1 * 10.0 : 20.0, + starTreeDocument.metrics[0] + ); + assertEquals(1L, starTreeDocument.metrics[1]); + } + assertEquals(6, count); + } + + public void testFlushFlowBuild() throws IOException { + List dimList = new ArrayList<>(100); + List docsWithField = new ArrayList<>(100); + for (int i = 0; i < 100; i++) { + dimList.add((long) i); + docsWithField.add(i); + } + + List dimList2 = new ArrayList<>(100); + List docsWithField2 = new ArrayList<>(100); + for (int i = 0; i < 100; i++) { + dimList2.add((long) i); + docsWithField2.add(i); + } + + List metricsList = new ArrayList<>(100); + List metricsWithField = new ArrayList<>(100); + for (int i = 0; i < 100; i++) { + metricsList.add(getLongFromDouble(i * 10.0)); + metricsWithField.add(i); + } + + Dimension d1 = new NumericDimension("field1"); + Dimension d2 = new NumericDimension("field3"); + Metric m1 = new Metric("field2", List.of(MetricStat.SUM)); + List dims = List.of(d1, d2); + List metrics = List.of(m1); + StarTreeFieldConfiguration c = new StarTreeFieldConfiguration( + 1, + new HashSet<>(), + StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP + ); + StarTreeField sf = new StarTreeField("sf", dims, metrics, c); + SortedNumericDocValues d1sndv = getSortedNumericMock(dimList, docsWithField); + SortedNumericDocValues d2sndv = getSortedNumericMock(dimList2, docsWithField2); + SortedNumericDocValues m1sndv = getSortedNumericMock(metricsList, metricsWithField); + + BaseStarTreeBuilder builder = getStarTreeBuilder(sf, getWriteState(100), mapperService); + + DocValuesProducer d1vp = getDocValuesProducer(d1sndv); + DocValuesProducer d2vp = getDocValuesProducer(d2sndv); + DocValuesProducer m1vp = getDocValuesProducer(m1sndv); + Map fieldProducerMap = Map.of("field1", d1vp, "field3", d2vp, "field2", m1vp); + builder.build(fieldProducerMap); + /** + * Asserting following dim / metrics [ dim1, dim2 / Sum [ metric] ] + [0, 0] | [0.0] + [1, 1] | [10.0] + [2, 2] | [20.0] + [3, 3] | [30.0] + [4, 4] | [40.0] + .... + [null, 0] | [0.0] + [null, 1] | [10.0] + ... + [null, null] | [49500.0] + */ + List starTreeDocuments = builder.getStarTreeDocuments(); + for (StarTreeDocument starTreeDocument : starTreeDocuments) { + assertEquals( + starTreeDocument.dimensions[1] != null ? starTreeDocument.dimensions[1] * 10.0 : 49500.0, + starTreeDocument.metrics[0] + ); + } + builder.close(); + } + + private static DocValuesProducer getDocValuesProducer(SortedNumericDocValues sndv) { + return new EmptyDocValuesProducer() { + @Override + public SortedNumericDocValues getSortedNumeric(FieldInfo field) throws IOException { + return sndv; + } + }; + } + + private static StarTreeField getStarTreeFieldWithMultipleMetrics() { + Dimension d1 = new NumericDimension("field1"); + Dimension d2 = new NumericDimension("field3"); + Metric m1 = new Metric("field2", List.of(MetricStat.SUM)); + Metric m2 = new Metric("field2", List.of(MetricStat.COUNT)); + List dims = List.of(d1, d2); + List metrics = List.of(m1, m2); + StarTreeFieldConfiguration c = new StarTreeFieldConfiguration( + 1000, + new HashSet<>(), + StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP + ); + StarTreeField sf = new StarTreeField("sf", dims, metrics, c); + return sf; + } + + public void testMergeFlowWithSum() throws IOException { + List dimList = List.of(0L, 1L, 3L, 4L, 5L, 6L); + List docsWithField = List.of(0, 1, 3, 4, 5, 6); + List dimList2 = List.of(0L, 1L, 2L, 3L, 4L, 5L, -1L); + List docsWithField2 = List.of(0, 1, 2, 3, 4, 5, 6); + + List metricsList = List.of( + getLongFromDouble(0.0), + getLongFromDouble(10.0), + getLongFromDouble(20.0), + getLongFromDouble(30.0), + getLongFromDouble(40.0), + getLongFromDouble(50.0), + getLongFromDouble(60.0) + + ); + List metricsWithField = List.of(0, 1, 2, 3, 4, 5, 6); + + StarTreeField sf = getStarTreeField(MetricStat.SUM); + StarTreeValues starTreeValues = getStarTreeValues( + getSortedNumericMock(dimList, docsWithField), + getSortedNumericMock(dimList2, docsWithField2), + getSortedNumericMock(metricsList, metricsWithField), + sf, + "6" + ); + + StarTreeValues starTreeValues2 = getStarTreeValues( + getSortedNumericMock(dimList, docsWithField), + getSortedNumericMock(dimList2, docsWithField2), + getSortedNumericMock(metricsList, metricsWithField), + sf, + "6" + ); + OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(sf, getWriteState(6), mapperService); + Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); + /** + * Asserting following dim / metrics [ dim1, dim2 / Sum [ metric] ] + * [0, 0] | [0.0] + * [1, 1] | [20.0] + * [3, 3] | [60.0] + * [4, 4] | [80.0] + * [5, 5] | [100.0] + * [null, 2] | [40.0] + * ------------------ We only take non star docs + * [6,-1] | [120.0] + */ + int count = 0; + while (starTreeDocumentIterator.hasNext()) { + count++; + StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); + assertEquals( + starTreeDocument.dimensions[0] != null ? starTreeDocument.dimensions[0] * 2 * 10.0 : 40.0, + starTreeDocument.metrics[0] + ); + } + assertEquals(6, count); + } + + public void testMergeFlowWithCount() throws IOException { + List dimList = List.of(0L, 1L, 3L, 4L, 5L, 6L); + List docsWithField = List.of(0, 1, 3, 4, 5, 6); + List dimList2 = List.of(0L, 1L, 2L, 3L, 4L, 5L, -1L); + List docsWithField2 = List.of(0, 1, 2, 3, 4, 5, 6); + + List metricsList = List.of(0L, 1L, 2L, 3L, 4L, 5L, 6L); + List metricsWithField = List.of(0, 1, 2, 3, 4, 5, 6); + + StarTreeField sf = getStarTreeField(MetricStat.COUNT); + StarTreeValues starTreeValues = getStarTreeValues( + getSortedNumericMock(dimList, docsWithField), + getSortedNumericMock(dimList2, docsWithField2), + getSortedNumericMock(metricsList, metricsWithField), + sf, + "6" + ); + + StarTreeValues starTreeValues2 = getStarTreeValues( + getSortedNumericMock(dimList, docsWithField), + getSortedNumericMock(dimList2, docsWithField2), + getSortedNumericMock(metricsList, metricsWithField), + sf, + "6" + ); + OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(sf, getWriteState(6), mapperService); + Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); + /** + * Asserting following dim / metrics [ dim1, dim2 / Count [ metric] ] + [0, 0] | [0] + [1, 1] | [2] + [3, 3] | [6] + [4, 4] | [8] + [5, 5] | [10] + [null, 2] | [4] + --------------- + [6,-1] | [12] + */ + int count = 0; + while (starTreeDocumentIterator.hasNext()) { + count++; + StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); + assertEquals(starTreeDocument.dimensions[0] != null ? starTreeDocument.dimensions[0] * 2 : 4, starTreeDocument.metrics[0]); + } + assertEquals(6, count); + } + + private StarTreeValues getStarTreeValues( + SortedNumericDocValues dimList, + SortedNumericDocValues dimList2, + SortedNumericDocValues metricsList, + StarTreeField sf, + String number + ) { + SortedNumericDocValues d1sndv = dimList; + SortedNumericDocValues d2sndv = dimList2; + SortedNumericDocValues m1sndv = metricsList; + Map dimDocIdSetIterators = Map.of("field1", d1sndv, "field3", d2sndv); + Map metricDocIdSetIterators = Map.of("field2", m1sndv); + StarTreeValues starTreeValues = new StarTreeValues( + sf, + null, + dimDocIdSetIterators, + metricDocIdSetIterators, + Map.of("numSegmentDocs", number) + ); + return starTreeValues; + } + + public void testMergeFlowWithDifferentDocsFromSegments() throws IOException { + List dimList = List.of(0L, 1L, 3L, 4L, 5L, 6L); + List docsWithField = List.of(0, 1, 3, 4, 5, 6); + List dimList2 = List.of(0L, 1L, 2L, 3L, 4L, 5L, -1L); + List docsWithField2 = List.of(0, 1, 2, 3, 4, 5, 6); + + List metricsList = List.of(0L, 1L, 2L, 3L, 4L, 5L, 6L); + List metricsWithField = List.of(0, 1, 2, 3, 4, 5, 6); + + List dimList3 = List.of(5L, 6L, 8L, -1L); + List docsWithField3 = List.of(0, 1, 3, 4); + List dimList4 = List.of(5L, 6L, 7L, 8L, -1L); + List docsWithField4 = List.of(0, 1, 2, 3, 4); + + List metricsList2 = List.of(5L, 6L, 7L, 8L, 9L); + List metricsWithField2 = List.of(0, 1, 2, 3, 4); + + StarTreeField sf = getStarTreeField(MetricStat.COUNT); + StarTreeValues starTreeValues = getStarTreeValues( + getSortedNumericMock(dimList, docsWithField), + getSortedNumericMock(dimList2, docsWithField2), + getSortedNumericMock(metricsList, metricsWithField), + sf, + "6" + ); + + StarTreeValues starTreeValues2 = getStarTreeValues( + getSortedNumericMock(dimList3, docsWithField3), + getSortedNumericMock(dimList4, docsWithField4), + getSortedNumericMock(metricsList2, metricsWithField2), + sf, + "4" + ); + OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(sf, getWriteState(4), mapperService); + Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); + /** + * Asserting following dim / metrics [ dim1, dim2 / Count [ metric] ] + [0, 0] | [0] + [1, 1] | [1] + [3, 3] | [3] + [4, 4] | [4] + [5, 5] | [10] + [6, 6] | [6] + [8, 8] | [8] + [null, 2] | [2] + [null, 7] | [7] + */ + int count = 0; + while (starTreeDocumentIterator.hasNext()) { + count++; + StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); + if (Objects.equals(starTreeDocument.dimensions[0], 5L)) { + assertEquals(starTreeDocument.dimensions[0] * 2, starTreeDocument.metrics[0]); + } else { + assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); + } + } + assertEquals(9, count); + } + + public void testMergeFlowWithMissingDocs() throws IOException { + List dimList = List.of(0L, 1L, 2L, 3L, 4L, 6L); + List docsWithField = List.of(0, 1, 2, 3, 4, 6); + List dimList2 = List.of(0L, 1L, 2L, 3L, 4L, 5L, -1L); + List docsWithField2 = List.of(0, 1, 2, 3, 4, 5, 6); + + List metricsList = List.of(0L, 1L, 2L, 3L, 4L, 5L, 6L); + List metricsWithField = List.of(0, 1, 2, 3, 4, 5, 6); + + List dimList3 = List.of(5L, 6L, 8L, -1L); + List docsWithField3 = List.of(0, 1, 3, 4); + List dimList4 = List.of(5L, 6L, 7L, 8L, -1L); + List docsWithField4 = List.of(0, 1, 2, 3, 4); + + List metricsList2 = List.of(5L, 6L, 7L, 8L, 9L); + List metricsWithField2 = List.of(0, 1, 2, 3, 4); + + StarTreeField sf = getStarTreeField(MetricStat.COUNT); + StarTreeValues starTreeValues = getStarTreeValues( + getSortedNumericMock(dimList, docsWithField), + getSortedNumericMock(dimList2, docsWithField2), + getSortedNumericMock(metricsList, metricsWithField), + sf, + "6" + ); + + StarTreeValues starTreeValues2 = getStarTreeValues( + getSortedNumericMock(dimList3, docsWithField3), + getSortedNumericMock(dimList4, docsWithField4), + getSortedNumericMock(metricsList2, metricsWithField2), + sf, + "4" + ); + OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(sf, getWriteState(4), mapperService); + Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); + /** + * Asserting following dim / metrics [ dim1, dim2 / Count [ metric] ] + [0, 0] | [0] + [1, 1] | [1] + [2, 2] | [2] + [3, 3] | [3] + [4, 4] | [4] + [5, 5] | [5] + [6, 6] | [6] + [8, 8] | [8] + [null, 5] | [5] + [null, 7] | [7] + */ + int count = 0; + while (starTreeDocumentIterator.hasNext()) { + count++; + StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); + if (starTreeDocument.dimensions[0] == null) { + assertTrue(List.of(5L, 7L).contains(starTreeDocument.dimensions[1])); + } + assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); + } + assertEquals(10, count); + } + + public void testMergeFlowWithMissingDocsInSecondDim() throws IOException { + List dimList2 = List.of(0L, 1L, 2L, 3L, 4L, 6L); + List docsWithField2 = List.of(0, 1, 2, 3, 4, 6); + List dimList = List.of(0L, 1L, 2L, 3L, 4L, 5L, -1L); + List docsWithField = List.of(0, 1, 2, 3, 4, 5, 6); + + List metricsList = List.of(0L, 1L, 2L, 3L, 4L, 5L, 6L); + List metricsWithField = List.of(0, 1, 2, 3, 4, 5, 6); + + List dimList3 = List.of(5L, 6L, 8L, -1L); + List docsWithField3 = List.of(0, 1, 3, 4); + List dimList4 = List.of(5L, 6L, 7L, 8L, -1L); + List docsWithField4 = List.of(0, 1, 2, 3, 4); + + List metricsList2 = List.of(5L, 6L, 7L, 8L, 9L); + List metricsWithField2 = List.of(0, 1, 2, 3, 4); + + StarTreeField sf = getStarTreeField(MetricStat.COUNT); + StarTreeValues starTreeValues = getStarTreeValues( + getSortedNumericMock(dimList, docsWithField), + getSortedNumericMock(dimList2, docsWithField2), + getSortedNumericMock(metricsList, metricsWithField), + sf, + "6" + ); + + StarTreeValues starTreeValues2 = getStarTreeValues( + getSortedNumericMock(dimList3, docsWithField3), + getSortedNumericMock(dimList4, docsWithField4), + getSortedNumericMock(metricsList2, metricsWithField2), + sf, + "4" + ); + OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(sf, getWriteState(4), mapperService); + Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); + /** + * Asserting following dim / metrics [ dim1, dim2 / Count [ metric] ] + [0, 0] | [0] + [1, 1] | [1] + [2, 2] | [2] + [3, 3] | [3] + [4, 4] | [4] + [5, 5] | [5] + [5, null] | [5] + [6, 6] | [6] + [8, 8] | [8] + [null, 7] | [7] + */ + int count = 0; + while (starTreeDocumentIterator.hasNext()) { + count++; + StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); + if (starTreeDocument.dimensions[0] != null && starTreeDocument.dimensions[0] == 5) { + assertEquals(starTreeDocument.dimensions[0], starTreeDocument.metrics[0]); + } else { + assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); + } + } + assertEquals(10, count); + } + + public void testMergeFlowWithDocsMissingAtTheEnd() throws IOException { + List dimList = List.of(0L, 1L, 2L, 3L, 4L); + List docsWithField = List.of(0, 1, 2, 3, 4); + List dimList2 = List.of(0L, 1L, 2L, 3L, 4L, 5L, -1L); + List docsWithField2 = List.of(0, 1, 2, 3, 4, 5, 6); + + List metricsList = List.of(0L, 1L, 2L, 3L, 4L, 5L, 6L); + List metricsWithField = List.of(0, 1, 2, 3, 4, 5, 6); + + List dimList3 = List.of(5L, 6L, 8L, -1L); + List docsWithField3 = List.of(0, 1, 3, 4); + List dimList4 = List.of(5L, 6L, 7L, 8L, -1L); + List docsWithField4 = List.of(0, 1, 2, 3, 4); + + List metricsList2 = List.of(5L, 6L, 7L, 8L, 9L); + List metricsWithField2 = List.of(0, 1, 2, 3, 4); + + StarTreeField sf = getStarTreeField(MetricStat.COUNT); + StarTreeValues starTreeValues = getStarTreeValues( + getSortedNumericMock(dimList, docsWithField), + getSortedNumericMock(dimList2, docsWithField2), + getSortedNumericMock(metricsList, metricsWithField), + sf, + "6" + ); + + StarTreeValues starTreeValues2 = getStarTreeValues( + getSortedNumericMock(dimList3, docsWithField3), + getSortedNumericMock(dimList4, docsWithField4), + getSortedNumericMock(metricsList2, metricsWithField2), + sf, + "4" + ); + OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(sf, writeState, mapperService); + Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); + /** + * Asserting following dim / metrics [ dim1, dim2 / Count [ metric] ] + [0, 0] | [0] + [1, 1] | [1] + [2, 2] | [2] + [3, 3] | [3] + [4, 4] | [4] + [5, 5] | [5] + [6, 6] | [6] + [8, 8] | [8] + [null, 5] | [5] + [null, 7] | [7] + */ + int count = 0; + while (starTreeDocumentIterator.hasNext()) { + count++; + StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); + if (starTreeDocument.dimensions[0] == null) { + assertTrue(List.of(5L, 7L).contains(starTreeDocument.dimensions[1])); + } + assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); + } + assertEquals(10, count); + } + + public void testMergeFlowWithEmptyFieldsInOneSegment() throws IOException { + List dimList = List.of(0L, 1L, 2L, 3L, 4L); + List docsWithField = List.of(0, 1, 2, 3, 4); + List dimList2 = List.of(0L, 1L, 2L, 3L, 4L, 5L, -1L); + List docsWithField2 = List.of(0, 1, 2, 3, 4, 5, 6); + + List metricsList = List.of(0L, 1L, 2L, 3L, 4L, 5L, 6L); + List metricsWithField = List.of(0, 1, 2, 3, 4, 5, 6); + + StarTreeField sf = getStarTreeField(MetricStat.COUNT); + StarTreeValues starTreeValues = getStarTreeValues( + getSortedNumericMock(dimList, docsWithField), + getSortedNumericMock(dimList2, docsWithField2), + getSortedNumericMock(metricsList, metricsWithField), + sf, + "6" + ); + + StarTreeValues starTreeValues2 = getStarTreeValues( + DocValues.emptySortedNumeric(), + DocValues.emptySortedNumeric(), + DocValues.emptySortedNumeric(), + sf, + "0" + ); + OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(sf, getWriteState(0), mapperService); + Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); + /** + * Asserting following dim / metrics [ dim1, dim2 / Count [ metric] ] + [0, 0] | [0] + [1, 1] | [1] + [2, 2] | [2] + [3, 3] | [3] + [4, 4] | [4] + [null, 5] | [5] + */ + int count = 0; + while (starTreeDocumentIterator.hasNext()) { + count++; + StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); + if (starTreeDocument.dimensions[0] == null) { + assertEquals(5L, (long) starTreeDocument.dimensions[1]); + } + assertEquals(starTreeDocument.dimensions[1], starTreeDocument.metrics[0]); + } + assertEquals(6, count); + } + + public void testMergeFlowWithDuplicateDimensionValues() throws IOException { + List dimList1 = new ArrayList<>(500); + List docsWithField1 = new ArrayList<>(500); + for (int i = 0; i < 100; i++) { + for (int j = 0; j < 5; j++) { + dimList1.add((long) i); + docsWithField1.add(i * 5 + j); + } + } + + List dimList2 = new ArrayList<>(500); + List docsWithField2 = new ArrayList<>(500); + for (int i = 0; i < 100; i++) { + for (int j = 0; j < 5; j++) { + dimList2.add((long) i); + docsWithField2.add(i * 5 + j); + } + } + + List dimList3 = new ArrayList<>(500); + List docsWithField3 = new ArrayList<>(500); + for (int i = 0; i < 100; i++) { + for (int j = 0; j < 5; j++) { + dimList3.add((long) i); + docsWithField3.add(i * 5 + j); + } + } + + List dimList4 = new ArrayList<>(500); + List docsWithField4 = new ArrayList<>(500); + for (int i = 0; i < 100; i++) { + for (int j = 0; j < 5; j++) { + dimList4.add((long) i); + docsWithField4.add(i * 5 + j); + } + } + + List metricsList = new ArrayList<>(100); + List metricsWithField = new ArrayList<>(100); + for (int i = 0; i < 500; i++) { + metricsList.add(getLongFromDouble(i * 10.0)); + metricsWithField.add(i); + } + + StarTreeField sf = getStarTreeField(1); + StarTreeValues starTreeValues = getStarTreeValues( + dimList1, + docsWithField1, + dimList2, + docsWithField2, + dimList3, + docsWithField3, + dimList4, + docsWithField4, + metricsList, + metricsWithField, + sf + ); + + StarTreeValues starTreeValues2 = getStarTreeValues( + dimList1, + docsWithField1, + dimList2, + docsWithField2, + dimList3, + docsWithField3, + dimList4, + docsWithField4, + metricsList, + metricsWithField, + sf + ); + OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(sf, writeState, mapperService); + builder.build(List.of(starTreeValues, starTreeValues2)); + List starTreeDocuments = builder.getStarTreeDocuments(); + assertEquals(401, starTreeDocuments.size()); + int count = 0; + double sum = 0; + /** + 401 docs get generated + [0, 0, 0, 0] | [200.0] + [1, 1, 1, 1] | [700.0] + [2, 2, 2, 2] | [1200.0] + [3, 3, 3, 3] | [1700.0] + [4, 4, 4, 4] | [2200.0] + ..... + [null, null, null, 99] | [49700.0] + [null, null, null, null] | [2495000.0] + */ + for (StarTreeDocument starTreeDocument : starTreeDocuments) { + if (starTreeDocument.dimensions[3] == null) { + assertEquals(sum, starTreeDocument.metrics[0]); + } else { + if (starTreeDocument.dimensions[0] != null) { + sum += (double) starTreeDocument.metrics[0]; + } + assertEquals(starTreeDocument.dimensions[3] * 500 + 200.0, starTreeDocument.metrics[0]); + } + count++; + } + assertEquals(401, count); + builder.close(); + } + + public void testMergeFlowWithMaxLeafDocs() throws IOException { + List dimList1 = new ArrayList<>(500); + List docsWithField1 = new ArrayList<>(500); + + for (int i = 0; i < 20; i++) { + for (int j = 0; j < 20; j++) { + dimList1.add((long) i); + docsWithField1.add(i * 20 + j); + } + } + for (int i = 80; i < 100; i++) { + for (int j = 0; j < 5; j++) { + dimList1.add((long) i); + docsWithField1.add(i * 5 + j); + } + } + List dimList3 = new ArrayList<>(500); + List docsWithField3 = new ArrayList<>(500); + for (int i = 0; i < 100; i++) { + for (int j = 0; j < 5; j++) { + dimList3.add((long) i); + docsWithField3.add(i * 5 + j); + } + } + List dimList2 = new ArrayList<>(500); + List docsWithField2 = new ArrayList<>(500); + for (int i = 0; i < 10; i++) { + for (int j = 0; j < 50; j++) { + dimList2.add((long) i); + docsWithField2.add(i * 50 + j); + } + } + + List dimList4 = new ArrayList<>(500); + List docsWithField4 = new ArrayList<>(500); + for (int i = 0; i < 100; i++) { + for (int j = 0; j < 5; j++) { + dimList4.add((long) i); + docsWithField4.add(i * 5 + j); + } + } + + List metricsList = new ArrayList<>(100); + List metricsWithField = new ArrayList<>(100); + for (int i = 0; i < 500; i++) { + metricsList.add(getLongFromDouble(i * 10.0)); + metricsWithField.add(i); + } + + StarTreeField sf = getStarTreeField(3); + StarTreeValues starTreeValues = getStarTreeValues( + dimList1, + docsWithField1, + dimList2, + docsWithField2, + dimList3, + docsWithField3, + dimList4, + docsWithField4, + metricsList, + metricsWithField, + sf + ); + + StarTreeValues starTreeValues2 = getStarTreeValues( + dimList1, + docsWithField1, + dimList2, + docsWithField2, + dimList3, + docsWithField3, + dimList4, + docsWithField4, + metricsList, + metricsWithField, + sf + ); + + OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(sf, writeState, mapperService); + builder.build(List.of(starTreeValues, starTreeValues2)); + List starTreeDocuments = builder.getStarTreeDocuments(); + /** + 635 docs get generated + [0, 0, 0, 0] | [200.0] + [1, 1, 1, 1] | [700.0] + [2, 2, 2, 2] | [1200.0] + [3, 3, 3, 3] | [1700.0] + [4, 4, 4, 4] | [2200.0] + ..... + [null, null, null, 99] | [49700.0] + ..... + [null, null, null, null] | [2495000.0] + */ + assertEquals(635, starTreeDocuments.size()); + builder.close(); + } + + private StarTreeValues getStarTreeValues( + List dimList1, + List docsWithField1, + List dimList2, + List docsWithField2, + List dimList3, + List docsWithField3, + List dimList4, + List docsWithField4, + List metricsList, + List metricsWithField, + StarTreeField sf + ) { + SortedNumericDocValues d1sndv = getSortedNumericMock(dimList1, docsWithField1); + SortedNumericDocValues d2sndv = getSortedNumericMock(dimList2, docsWithField2); + SortedNumericDocValues d3sndv = getSortedNumericMock(dimList3, docsWithField3); + SortedNumericDocValues d4sndv = getSortedNumericMock(dimList4, docsWithField4); + SortedNumericDocValues m1sndv = getSortedNumericMock(metricsList, metricsWithField); + Map dimDocIdSetIterators = Map.of("field1", d1sndv, "field3", d2sndv, "field5", d3sndv, "field8", d4sndv); + Map metricDocIdSetIterators = Map.of("field2", m1sndv); + StarTreeValues starTreeValues = new StarTreeValues(sf, null, dimDocIdSetIterators, metricDocIdSetIterators, getAttributes(500)); + return starTreeValues; + } + + public void testMergeFlowWithDuplicateDimensionValueWithMaxLeafDocs() throws IOException { + List dimList1 = new ArrayList<>(500); + List docsWithField1 = new ArrayList<>(500); + + for (int i = 0; i < 20; i++) { + for (int j = 0; j < 20; j++) { + dimList1.add((long) i); + docsWithField1.add(i * 20 + j); + } + } + for (int i = 80; i < 100; i++) { + for (int j = 0; j < 5; j++) { + dimList1.add((long) i); + docsWithField1.add(i * 5 + j); + } + } + List dimList3 = new ArrayList<>(500); + List docsWithField3 = new ArrayList<>(500); + for (int i = 0; i < 100; i++) { + for (int j = 0; j < 5; j++) { + dimList3.add((long) i); + docsWithField3.add(i * 5 + j); + } + } + List dimList2 = new ArrayList<>(500); + List docsWithField2 = new ArrayList<>(500); + for (int i = 0; i < 500; i++) { + dimList2.add((long) 1); + docsWithField2.add(i); + } + + List dimList4 = new ArrayList<>(500); + List docsWithField4 = new ArrayList<>(500); + for (int i = 0; i < 100; i++) { + for (int j = 0; j < 5; j++) { + dimList4.add((long) i); + docsWithField4.add(i * 5 + j); + } + } + + List metricsList = new ArrayList<>(100); + List metricsWithField = new ArrayList<>(100); + for (int i = 0; i < 500; i++) { + metricsList.add(getLongFromDouble(i * 10.0)); + metricsWithField.add(i); + } + + StarTreeField sf = getStarTreeField(3); + StarTreeValues starTreeValues = getStarTreeValues( + dimList1, + docsWithField1, + dimList2, + docsWithField2, + dimList3, + docsWithField3, + dimList4, + docsWithField4, + metricsList, + metricsWithField, + sf + ); + + StarTreeValues starTreeValues2 = getStarTreeValues( + dimList1, + docsWithField1, + dimList2, + docsWithField2, + dimList3, + docsWithField3, + dimList4, + docsWithField4, + metricsList, + metricsWithField, + sf + ); + OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(sf, writeState, mapperService); + builder.build(List.of(starTreeValues, starTreeValues2)); + List starTreeDocuments = builder.getStarTreeDocuments(); + assertEquals(401, starTreeDocuments.size()); + builder.close(); + } + + public static long getLongFromDouble(double value) { + return Double.doubleToLongBits(value); + } + + public void testMergeFlowWithMaxLeafDocsAndStarTreeNodesAssertion() throws IOException { + List dimList1 = new ArrayList<>(500); + List docsWithField1 = new ArrayList<>(500); + Map> expectedDimToValueMap = new HashMap<>(); + Map dimValueMap = new HashMap<>(); + for (int i = 0; i < 20; i++) { + for (int j = 0; j < 20; j++) { + dimList1.add((long) i); + docsWithField1.add(i * 20 + j); + } + // metric = no of docs * 10.0 + dimValueMap.put((long) i, 200.0); + } + for (int i = 80; i < 100; i++) { + for (int j = 0; j < 5; j++) { + dimList1.add((long) i); + docsWithField1.add(i * 5 + j); + } + // metric = no of docs * 10.0 + dimValueMap.put((long) i, 50.0); + } + dimValueMap.put(Long.MAX_VALUE, 5000.0); + expectedDimToValueMap.put(0, dimValueMap); + dimValueMap = new HashMap<>(); + List dimList3 = new ArrayList<>(500); + List docsWithField3 = new ArrayList<>(500); + for (int i = 0; i < 500; i++) { + dimList3.add((long) 1); + docsWithField3.add(i); + dimValueMap.put((long) i, 10.0); + } + dimValueMap.put(Long.MAX_VALUE, 5000.0); + expectedDimToValueMap.put(2, dimValueMap); + dimValueMap = new HashMap<>(); + List dimList2 = new ArrayList<>(500); + List docsWithField2 = new ArrayList<>(500); + for (int i = 0; i < 500; i++) { + dimList2.add((long) i); + docsWithField2.add(i); + dimValueMap.put((long) i, 10.0); + } + dimValueMap.put(Long.MAX_VALUE, 200.0); + expectedDimToValueMap.put(1, dimValueMap); + dimValueMap = new HashMap<>(); + List dimList4 = new ArrayList<>(500); + List docsWithField4 = new ArrayList<>(500); + for (int i = 0; i < 500; i++) { + dimList4.add((long) 1); + docsWithField4.add(i); + dimValueMap.put((long) i, 10.0); + } + dimValueMap.put(Long.MAX_VALUE, 5000.0); + expectedDimToValueMap.put(3, dimValueMap); + List metricsList = new ArrayList<>(100); + List metricsWithField = new ArrayList<>(100); + for (int i = 0; i < 500; i++) { + metricsList.add(getLongFromDouble(10.0)); + metricsWithField.add(i); + } + + StarTreeField sf = getStarTreeField(10); + StarTreeValues starTreeValues = getStarTreeValues( + dimList1, + docsWithField1, + dimList2, + docsWithField2, + dimList3, + docsWithField3, + dimList4, + docsWithField4, + metricsList, + metricsWithField, + sf + ); + + StarTreeValues starTreeValues2 = getStarTreeValues( + dimList1, + docsWithField1, + dimList2, + docsWithField2, + dimList3, + docsWithField3, + dimList4, + docsWithField4, + metricsList, + metricsWithField, + sf + ); + OnHeapStarTreeBuilder builder = new OnHeapStarTreeBuilder(sf, writeState, mapperService); + builder.build(List.of(starTreeValues, starTreeValues2)); + List starTreeDocuments = builder.getStarTreeDocuments(); + Map> dimValueToDocIdMap = new HashMap<>(); + traverseStarTree(builder.rootNode, dimValueToDocIdMap, true); + for (Map.Entry> entry : dimValueToDocIdMap.entrySet()) { + int dimId = entry.getKey(); + if (dimId == -1) continue; + Map map = expectedDimToValueMap.get(dimId); + for (Map.Entry dimValueToDocIdEntry : entry.getValue().entrySet()) { + long dimValue = dimValueToDocIdEntry.getKey(); + int docId = dimValueToDocIdEntry.getValue(); + assertEquals(map.get(dimValue) * 2, starTreeDocuments.get(docId).metrics[0]); + } + } + assertEquals(1041, starTreeDocuments.size()); + builder.close(); + } + + private static StarTreeField getStarTreeField(int maxLeafDocs) { + Dimension d1 = new NumericDimension("field1"); + Dimension d2 = new NumericDimension("field3"); + Dimension d3 = new NumericDimension("field5"); + Dimension d4 = new NumericDimension("field8"); + List dims = List.of(d1, d2, d3, d4); + Metric m1 = new Metric("field2", List.of(MetricStat.SUM)); + List metrics = List.of(m1); + StarTreeFieldConfiguration c = new StarTreeFieldConfiguration( + maxLeafDocs, + new HashSet<>(), + StarTreeFieldConfiguration.StarTreeBuildMode.OFF_HEAP + ); + StarTreeField sf = new StarTreeField("sf", dims, metrics, c); + return sf; + } + + private void traverseStarTree(TreeNode root, Map> dimValueToDocIdMap, boolean traverStarNodes) { + TreeNode starTree = root; + // Use BFS to traverse the star tree + Queue queue = new ArrayDeque<>(); + queue.add(starTree); + int currentDimensionId = -1; + TreeNode starTreeNode; + List docIds = new ArrayList<>(); + while ((starTreeNode = queue.poll()) != null) { + int dimensionId = starTreeNode.dimensionId; + if (dimensionId > currentDimensionId) { + currentDimensionId = dimensionId; + } + + // store aggregated document of the node + int docId = starTreeNode.aggregatedDocId; + Map map = dimValueToDocIdMap.getOrDefault(dimensionId, new HashMap<>()); + if (starTreeNode.isStarNode) { + map.put(Long.MAX_VALUE, docId); + } else { + map.put(starTreeNode.dimensionValue, docId); + } + dimValueToDocIdMap.put(dimensionId, map); + + if (starTreeNode.children != null && (!traverStarNodes || starTreeNode.isStarNode)) { + Iterator childrenIterator = starTreeNode.children.values().iterator(); + while (childrenIterator.hasNext()) { + TreeNode childNode = childrenIterator.next(); + queue.add(childNode); + } + } + } + } + + public void testMergeFlow() throws IOException { + List dimList1 = new ArrayList<>(1000); + List docsWithField1 = new ArrayList<>(1000); + for (int i = 0; i < 1000; i++) { + dimList1.add((long) i); + docsWithField1.add(i); + } + + List dimList2 = new ArrayList<>(1000); + List docsWithField2 = new ArrayList<>(1000); + for (int i = 0; i < 1000; i++) { + dimList2.add((long) i); + docsWithField2.add(i); + } + + List dimList3 = new ArrayList<>(1000); + List docsWithField3 = new ArrayList<>(1000); + for (int i = 0; i < 1000; i++) { + dimList3.add((long) i); + docsWithField3.add(i); + } + + List dimList4 = new ArrayList<>(1000); + List docsWithField4 = new ArrayList<>(1000); + for (int i = 0; i < 1000; i++) { + dimList4.add((long) i); + docsWithField4.add(i); + } + + List dimList5 = new ArrayList<>(1000); + List docsWithField5 = new ArrayList<>(1000); + for (int i = 0; i < 1000; i++) { + dimList5.add((long) i); + docsWithField5.add(i); + } + + List metricsList = new ArrayList<>(1000); + List metricsWithField = new ArrayList<>(1000); + for (int i = 0; i < 1000; i++) { + metricsList.add(getLongFromDouble(i * 10.0)); + metricsWithField.add(i); + } + + Dimension d1 = new NumericDimension("field1"); + Dimension d2 = new NumericDimension("field3"); + Dimension d3 = new NumericDimension("field5"); + Dimension d4 = new NumericDimension("field8"); + // Dimension d5 = new NumericDimension("field5"); + Metric m1 = new Metric("field2", List.of(MetricStat.SUM)); + List dims = List.of(d1, d2, d3, d4); + List metrics = List.of(m1); + StarTreeFieldConfiguration c = new StarTreeFieldConfiguration( + 1, + new HashSet<>(), + StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP + ); + StarTreeField sf = new StarTreeField("sf", dims, metrics, c); + SortedNumericDocValues d1sndv = getSortedNumericMock(dimList1, docsWithField1); + SortedNumericDocValues d2sndv = getSortedNumericMock(dimList2, docsWithField2); + SortedNumericDocValues d3sndv = getSortedNumericMock(dimList3, docsWithField3); + SortedNumericDocValues d4sndv = getSortedNumericMock(dimList4, docsWithField4); + SortedNumericDocValues m1sndv = getSortedNumericMock(metricsList, metricsWithField); + Map dimDocIdSetIterators = Map.of("field1", d1sndv, "field3", d2sndv, "field5", d3sndv, "field8", d4sndv); + Map metricDocIdSetIterators = Map.of("field2", m1sndv); + StarTreeValues starTreeValues = new StarTreeValues(sf, null, dimDocIdSetIterators, metricDocIdSetIterators, getAttributes(1000)); + + SortedNumericDocValues f2d1sndv = getSortedNumericMock(dimList1, docsWithField1); + SortedNumericDocValues f2d2sndv = getSortedNumericMock(dimList2, docsWithField2); + SortedNumericDocValues f2d3sndv = getSortedNumericMock(dimList3, docsWithField3); + SortedNumericDocValues f2d4sndv = getSortedNumericMock(dimList4, docsWithField4); + SortedNumericDocValues f2m1sndv = getSortedNumericMock(metricsList, metricsWithField); + Map f2dimDocIdSetIterators = Map.of( + "field1", + f2d1sndv, + "field3", + f2d2sndv, + "field5", + f2d3sndv, + "field8", + f2d4sndv + ); + Map f2metricDocIdSetIterators = Map.of("field2", f2m1sndv); + StarTreeValues starTreeValues2 = new StarTreeValues( + sf, + null, + f2dimDocIdSetIterators, + f2metricDocIdSetIterators, + getAttributes(1000) + ); + + BaseStarTreeBuilder builder = getStarTreeBuilder(sf, writeState, mapperService); + Iterator starTreeDocumentIterator = builder.mergeStarTrees(List.of(starTreeValues, starTreeValues2)); + /** + [0, 0, 0, 0] | [0.0] + [1, 1, 1, 1] | [20.0] + [2, 2, 2, 2] | [40.0] + [3, 3, 3, 3] | [60.0] + [4, 4, 4, 4] | [80.0] + [5, 5, 5, 5] | [100.0] + ... + [999, 999, 999, 999] | [19980.0] + */ + while (starTreeDocumentIterator.hasNext()) { + StarTreeDocument starTreeDocument = starTreeDocumentIterator.next(); + assertEquals(starTreeDocument.dimensions[0] * 20.0, starTreeDocument.metrics[0]); + } + builder.close(); + } + + Map getAttributes(int numSegmentDocs) { + return Map.of(String.valueOf(NUM_SEGMENT_DOCS), String.valueOf(numSegmentDocs)); + } + + private static StarTreeField getStarTreeField(MetricStat count) { + Dimension d1 = new NumericDimension("field1"); + Dimension d2 = new NumericDimension("field3"); + Metric m1 = new Metric("field2", List.of(count)); + List dims = List.of(d1, d2); + List metrics = List.of(m1); + StarTreeFieldConfiguration c = new StarTreeFieldConfiguration( + 1000, + new HashSet<>(), + StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP + ); + return new StarTreeField("sf", dims, metrics, c); + } + + private Long getLongFromDouble(Double num) { + if (num == null) { + return null; + } + return NumericUtils.doubleToSortableLong(num); + } + + SortedNumericDocValues getSortedNumericMock(List dimList, List docsWithField) { + return new SortedNumericDocValues() { + int index = -1; + + @Override + public long nextValue() { + return dimList.get(index); + } + + @Override + public int docValueCount() { + return 0; + } + + @Override + public boolean advanceExact(int target) { + return false; + } + + @Override + public int docID() { + return index; + } + + @Override + public int nextDoc() { + if (index == docsWithField.size() - 1) { + return NO_MORE_DOCS; + } + index++; + return docsWithField.get(index); + } + + @Override + public int advance(int target) { + return 0; + } + + @Override + public long cost() { + return 0; + } + }; + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + if (builder != null) { + builder.close(); + } + directory.close(); + } +} diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilderTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilderTests.java index b78130e72aba1..51ebc02ea8243 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilderTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/BaseStarTreeBuilderTests.java @@ -22,6 +22,7 @@ import org.apache.lucene.util.InfoStream; import org.apache.lucene.util.Version; import org.opensearch.common.settings.Settings; +import org.opensearch.index.codec.composite.datacube.startree.StarTreeValues; import org.opensearch.index.compositeindex.datacube.Dimension; import org.opensearch.index.compositeindex.datacube.Metric; import org.opensearch.index.compositeindex.datacube.MetricStat; @@ -30,6 +31,7 @@ import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; import org.opensearch.index.compositeindex.datacube.startree.aggregators.MetricAggregatorInfo; +import org.opensearch.index.compositeindex.datacube.startree.utils.SequentialDocValuesIterator; import org.opensearch.index.fielddata.IndexNumericFieldData; import org.opensearch.index.mapper.ContentPath; import org.opensearch.index.mapper.DocumentMapper; @@ -155,7 +157,10 @@ public static void setup() throws IOException { ); when(documentMapper.mappers()).thenReturn(fieldMappers); - builder = new BaseStarTreeBuilder(starTreeField, fieldProducerMap, state, mapperService) { + builder = new BaseStarTreeBuilder(starTreeField, state, mapperService) { + @Override + public void build(List starTreeValuesSubs) throws IOException {} + @Override public void appendStarTreeDocument(StarTreeDocument starTreeDocument) throws IOException {} @@ -171,11 +176,14 @@ public List getStarTreeDocuments() { @Override public Long getDimensionValue(int docId, int dimensionId) throws IOException { - return 0L; + return 0l; } @Override - public Iterator sortAndAggregateStarTreeDocuments() throws IOException { + public Iterator sortAndAggregateSegmentDocuments( + SequentialDocValuesIterator[] dimensionReaders, + List metricReaders + ) throws IOException { return null; } @@ -184,14 +192,19 @@ public Iterator generateStarTreeDocumentsForStarNode(int start throws IOException { return null; } + + @Override + Iterator mergeStarTrees(List starTreeValues) throws IOException { + return null; + } }; } public void test_generateMetricAggregatorInfos() throws IOException { - List metricAggregatorInfos = builder.generateMetricAggregatorInfos(mapperService, state); + List metricAggregatorInfos = builder.generateMetricAggregatorInfos(mapperService); List expectedMetricAggregatorInfos = List.of( - new MetricAggregatorInfo(MetricStat.SUM, "field2", starTreeField.getName(), IndexNumericFieldData.NumericType.DOUBLE, null), - new MetricAggregatorInfo(MetricStat.SUM, "field4", starTreeField.getName(), IndexNumericFieldData.NumericType.DOUBLE, null) + new MetricAggregatorInfo(MetricStat.SUM, "field2", starTreeField.getName(), IndexNumericFieldData.NumericType.DOUBLE), + new MetricAggregatorInfo(MetricStat.SUM, "field4", starTreeField.getName(), IndexNumericFieldData.NumericType.DOUBLE) ); assertEquals(metricAggregatorInfos, expectedMetricAggregatorInfos); } diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/OnHeapStarTreeBuilderTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/OnHeapStarTreeBuilderTests.java index 4e107e78d27be..aed08b7727be7 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/OnHeapStarTreeBuilderTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/OnHeapStarTreeBuilderTests.java @@ -8,699 +8,17 @@ package org.opensearch.index.compositeindex.datacube.startree.builder; -import org.apache.lucene.codecs.DocValuesProducer; -import org.apache.lucene.codecs.lucene99.Lucene99Codec; -import org.apache.lucene.index.DocValuesType; -import org.apache.lucene.index.FieldInfo; -import org.apache.lucene.index.FieldInfos; -import org.apache.lucene.index.IndexOptions; -import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentWriteState; -import org.apache.lucene.index.VectorEncoding; -import org.apache.lucene.index.VectorSimilarityFunction; -import org.apache.lucene.sandbox.document.HalfFloatPoint; -import org.apache.lucene.store.Directory; -import org.apache.lucene.util.InfoStream; -import org.apache.lucene.util.NumericUtils; -import org.apache.lucene.util.Version; -import org.opensearch.common.settings.Settings; -import org.opensearch.index.compositeindex.datacube.Dimension; -import org.opensearch.index.compositeindex.datacube.Metric; -import org.opensearch.index.compositeindex.datacube.MetricStat; -import org.opensearch.index.compositeindex.datacube.NumericDimension; -import org.opensearch.index.compositeindex.datacube.startree.StarTreeDocument; import org.opensearch.index.compositeindex.datacube.startree.StarTreeField; -import org.opensearch.index.compositeindex.datacube.startree.StarTreeFieldConfiguration; -import org.opensearch.index.mapper.ContentPath; -import org.opensearch.index.mapper.DocumentMapper; -import org.opensearch.index.mapper.Mapper; import org.opensearch.index.mapper.MapperService; -import org.opensearch.index.mapper.MappingLookup; -import org.opensearch.index.mapper.NumberFieldMapper; -import org.opensearch.test.OpenSearchTestCase; -import org.junit.Before; - -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.UUID; - -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -public class OnHeapStarTreeBuilderTests extends OpenSearchTestCase { - - private OnHeapStarTreeBuilder builder; - private MapperService mapperService; - private List dimensionsOrder; - private List fields = List.of(); - private List metrics; - private Directory directory; - private FieldInfo[] fieldsInfo; - private StarTreeField compositeField; - private Map fieldProducerMap; - private SegmentWriteState writeState; - - @Before - public void setup() throws IOException { - fields = List.of("field1", "field2", "field3", "field4", "field5", "field6", "field7", "field8", "field9", "field10"); - - dimensionsOrder = List.of( - new NumericDimension("field1"), - new NumericDimension("field3"), - new NumericDimension("field5"), - new NumericDimension("field8") - ); - metrics = List.of( - new Metric("field2", List.of(MetricStat.SUM)), - new Metric("field4", List.of(MetricStat.SUM)), - new Metric("field6", List.of(MetricStat.COUNT)) - ); - - DocValuesProducer docValuesProducer = mock(DocValuesProducer.class); - - compositeField = new StarTreeField( - "test", - dimensionsOrder, - metrics, - new StarTreeFieldConfiguration(1, Set.of("field8"), StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP) - ); - directory = newFSDirectory(createTempDir()); - SegmentInfo segmentInfo = new SegmentInfo( - directory, - Version.LATEST, - Version.LUCENE_9_11_0, - "test_segment", - 5, - false, - false, - new Lucene99Codec(), - new HashMap<>(), - UUID.randomUUID().toString().substring(0, 16).getBytes(StandardCharsets.UTF_8), - new HashMap<>(), - null - ); - - fieldsInfo = new FieldInfo[fields.size()]; - fieldProducerMap = new HashMap<>(); - for (int i = 0; i < fieldsInfo.length; i++) { - fieldsInfo[i] = new FieldInfo( - fields.get(i), - i, - false, - false, - true, - IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS, - DocValuesType.SORTED_NUMERIC, - -1, - Collections.emptyMap(), - 0, - 0, - 0, - 0, - VectorEncoding.FLOAT32, - VectorSimilarityFunction.EUCLIDEAN, - false, - false - ); - fieldProducerMap.put(fields.get(i), docValuesProducer); - } - FieldInfos fieldInfos = new FieldInfos(fieldsInfo); - writeState = new SegmentWriteState(InfoStream.getDefault(), segmentInfo.dir, segmentInfo, fieldInfos, null, newIOContext(random())); - - mapperService = mock(MapperService.class); - DocumentMapper documentMapper = mock(DocumentMapper.class); - when(mapperService.documentMapper()).thenReturn(documentMapper); - Settings settings = Settings.builder().put(settings(org.opensearch.Version.CURRENT).build()).build(); - NumberFieldMapper numberFieldMapper1 = new NumberFieldMapper.Builder("field2", NumberFieldMapper.NumberType.DOUBLE, false, true) - .build(new Mapper.BuilderContext(settings, new ContentPath())); - NumberFieldMapper numberFieldMapper2 = new NumberFieldMapper.Builder("field4", NumberFieldMapper.NumberType.DOUBLE, false, true) - .build(new Mapper.BuilderContext(settings, new ContentPath())); - NumberFieldMapper numberFieldMapper3 = new NumberFieldMapper.Builder("field6", NumberFieldMapper.NumberType.DOUBLE, false, true) - .build(new Mapper.BuilderContext(settings, new ContentPath())); - MappingLookup fieldMappers = new MappingLookup( - Set.of(numberFieldMapper1, numberFieldMapper2, numberFieldMapper3), - Collections.emptyList(), - Collections.emptyList(), - 0, - null - ); - when(documentMapper.mappers()).thenReturn(fieldMappers); - builder = new OnHeapStarTreeBuilder(compositeField, fieldProducerMap, writeState, mapperService); - } - - public void test_sortAndAggregateStarTreeDocuments() throws IOException { - - int noOfStarTreeDocuments = 5; - StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; - - starTreeDocuments[0] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Double[] { 12.0, 10.0, randomDouble() }); - starTreeDocuments[1] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 10.0, 6.0, randomDouble() }); - starTreeDocuments[2] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 14.0, 12.0, randomDouble() }); - starTreeDocuments[3] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Double[] { 9.0, 4.0, randomDouble() }); - starTreeDocuments[4] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 11.0, 16.0, randomDouble() }); - - List inorderStarTreeDocuments = List.of( - new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Object[] { 21.0, 14.0, 2L }), - new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Object[] { 35.0, 34.0, 3L }) - ); - Iterator expectedStarTreeDocumentIterator = inorderStarTreeDocuments.iterator(); - - StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; - for (int i = 0; i < noOfStarTreeDocuments; i++) { - long metric1 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[0]); - long metric2 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[1]); - long metric3 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[2]); - segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, new Long[] { metric1, metric2, metric3 }); - } - - Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateStarTreeDocuments(segmentStarTreeDocuments); - int numOfAggregatedDocuments = 0; - while (segmentStarTreeDocumentIterator.hasNext() && expectedStarTreeDocumentIterator.hasNext()) { - StarTreeDocument resultStarTreeDocument = segmentStarTreeDocumentIterator.next(); - StarTreeDocument expectedStarTreeDocument = expectedStarTreeDocumentIterator.next(); - - assertEquals(expectedStarTreeDocument.dimensions[0], resultStarTreeDocument.dimensions[0]); - assertEquals(expectedStarTreeDocument.dimensions[1], resultStarTreeDocument.dimensions[1]); - assertEquals(expectedStarTreeDocument.dimensions[2], resultStarTreeDocument.dimensions[2]); - assertEquals(expectedStarTreeDocument.dimensions[3], resultStarTreeDocument.dimensions[3]); - assertEquals(expectedStarTreeDocument.metrics[0], resultStarTreeDocument.metrics[0]); - assertEquals(expectedStarTreeDocument.metrics[1], resultStarTreeDocument.metrics[1]); - assertEquals(expectedStarTreeDocument.metrics[2], resultStarTreeDocument.metrics[2]); - - numOfAggregatedDocuments++; - } - - assertEquals(inorderStarTreeDocuments.size(), numOfAggregatedDocuments); - - } - - public void test_sortAndAggregateStarTreeDocuments_nullMetric() throws IOException { - - int noOfStarTreeDocuments = 5; - StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; - - starTreeDocuments[0] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Double[] { 12.0, 10.0, randomDouble() }); - starTreeDocuments[1] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 10.0, 6.0, randomDouble() }); - starTreeDocuments[2] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 14.0, 12.0, randomDouble() }); - starTreeDocuments[3] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Double[] { 9.0, 4.0, randomDouble() }); - starTreeDocuments[4] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 11.0, null, randomDouble() }); - StarTreeDocument expectedStarTreeDocument = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Double[] { 21.0, 14.0, 2.0 }); - - StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; - for (int i = 0; i < noOfStarTreeDocuments; i++) { - Long metric1 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[0]); - Long metric2 = starTreeDocuments[i].metrics[1] != null - ? NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[1]) - : null; - Long metric3 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[2]); - segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, new Object[] { metric1, metric2, metric3 }); - } - - Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateStarTreeDocuments(segmentStarTreeDocuments); - - StarTreeDocument resultStarTreeDocument = segmentStarTreeDocumentIterator.next(); - assertEquals(expectedStarTreeDocument.dimensions[0], resultStarTreeDocument.dimensions[0]); - assertEquals(expectedStarTreeDocument.dimensions[1], resultStarTreeDocument.dimensions[1]); - assertEquals(expectedStarTreeDocument.dimensions[2], resultStarTreeDocument.dimensions[2]); - assertEquals(expectedStarTreeDocument.dimensions[3], resultStarTreeDocument.dimensions[3]); - assertEquals(expectedStarTreeDocument.metrics[0], resultStarTreeDocument.metrics[0]); - assertEquals(expectedStarTreeDocument.metrics[1], resultStarTreeDocument.metrics[1]); - - assertThrows( - "Null metric should have resulted in IllegalStateException", - IllegalStateException.class, - segmentStarTreeDocumentIterator::next - ); - - } - - public void test_sortAndAggregateStarTreeDocument_longMaxAndLongMinDimensions() throws IOException { - - int noOfStarTreeDocuments = 5; - StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; - - starTreeDocuments[0] = new StarTreeDocument(new Long[] { Long.MIN_VALUE, 4L, 3L, 4L }, new Double[] { 12.0, 10.0, randomDouble() }); - starTreeDocuments[1] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, Long.MAX_VALUE }, new Double[] { 10.0, 6.0, randomDouble() }); - starTreeDocuments[2] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, Long.MAX_VALUE }, new Double[] { 14.0, 12.0, randomDouble() }); - starTreeDocuments[3] = new StarTreeDocument(new Long[] { Long.MIN_VALUE, 4L, 3L, 4L }, new Double[] { 9.0, 4.0, randomDouble() }); - starTreeDocuments[4] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, Long.MAX_VALUE }, new Double[] { 11.0, 16.0, randomDouble() }); - - List inorderStarTreeDocuments = List.of( - new StarTreeDocument(new Long[] { Long.MIN_VALUE, 4L, 3L, 4L }, new Object[] { 21.0, 14.0, 2L }), - new StarTreeDocument(new Long[] { 3L, 4L, 2L, Long.MAX_VALUE }, new Object[] { 35.0, 34.0, 3L }) - ); - Iterator expectedStarTreeDocumentIterator = inorderStarTreeDocuments.iterator(); - - StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; - for (int i = 0; i < noOfStarTreeDocuments; i++) { - long metric1 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[0]); - long metric2 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[1]); - long metric3 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[2]); - segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, new Long[] { metric1, metric2, metric3 }); - } - - Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateStarTreeDocuments(segmentStarTreeDocuments); - int numOfAggregatedDocuments = 0; - while (segmentStarTreeDocumentIterator.hasNext() && expectedStarTreeDocumentIterator.hasNext()) { - StarTreeDocument resultStarTreeDocument = segmentStarTreeDocumentIterator.next(); - StarTreeDocument expectedStarTreeDocument = expectedStarTreeDocumentIterator.next(); - - assertEquals(expectedStarTreeDocument.dimensions[0], resultStarTreeDocument.dimensions[0]); - assertEquals(expectedStarTreeDocument.dimensions[1], resultStarTreeDocument.dimensions[1]); - assertEquals(expectedStarTreeDocument.dimensions[2], resultStarTreeDocument.dimensions[2]); - assertEquals(expectedStarTreeDocument.dimensions[3], resultStarTreeDocument.dimensions[3]); - assertEquals(expectedStarTreeDocument.metrics[0], resultStarTreeDocument.metrics[0]); - assertEquals(expectedStarTreeDocument.metrics[1], resultStarTreeDocument.metrics[1]); - assertEquals(expectedStarTreeDocument.metrics[2], resultStarTreeDocument.metrics[2]); - - numOfAggregatedDocuments++; - } - - assertEquals(inorderStarTreeDocuments.size(), numOfAggregatedDocuments); - - } - - public void test_sortAndAggregateStarTreeDocument_DoubleMaxAndDoubleMinMetrics() throws IOException { - - int noOfStarTreeDocuments = 5; - StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; - - starTreeDocuments[0] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Double[] { Double.MAX_VALUE, 10.0, randomDouble() }); - starTreeDocuments[1] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 10.0, 6.0, randomDouble() }); - starTreeDocuments[2] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 14.0, Double.MIN_VALUE, randomDouble() }); - starTreeDocuments[3] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Double[] { 9.0, 4.0, randomDouble() }); - starTreeDocuments[4] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 11.0, 16.0, randomDouble() }); - - List inorderStarTreeDocuments = List.of( - new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Object[] { Double.MAX_VALUE + 9, 14.0, 2L }), - new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Object[] { 35.0, Double.MIN_VALUE + 22, 3L }) - ); - Iterator expectedStarTreeDocumentIterator = inorderStarTreeDocuments.iterator(); - - StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; - for (int i = 0; i < noOfStarTreeDocuments; i++) { - long metric1 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[0]); - long metric2 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[1]); - long metric3 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[2]); - segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, new Long[] { metric1, metric2, metric3 }); - } - - Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateStarTreeDocuments(segmentStarTreeDocuments); - int numOfAggregatedDocuments = 0; - while (segmentStarTreeDocumentIterator.hasNext() && expectedStarTreeDocumentIterator.hasNext()) { - StarTreeDocument resultStarTreeDocument = segmentStarTreeDocumentIterator.next(); - StarTreeDocument expectedStarTreeDocument = expectedStarTreeDocumentIterator.next(); - - assertEquals(expectedStarTreeDocument.dimensions[0], resultStarTreeDocument.dimensions[0]); - assertEquals(expectedStarTreeDocument.dimensions[1], resultStarTreeDocument.dimensions[1]); - assertEquals(expectedStarTreeDocument.dimensions[2], resultStarTreeDocument.dimensions[2]); - assertEquals(expectedStarTreeDocument.dimensions[3], resultStarTreeDocument.dimensions[3]); - assertEquals(expectedStarTreeDocument.metrics[0], resultStarTreeDocument.metrics[0]); - assertEquals(expectedStarTreeDocument.metrics[1], resultStarTreeDocument.metrics[1]); - assertEquals(expectedStarTreeDocument.metrics[2], resultStarTreeDocument.metrics[2]); - - numOfAggregatedDocuments++; - } - - assertEquals(inorderStarTreeDocuments.size(), numOfAggregatedDocuments); - - } - - public void test_build_halfFloatMetrics() throws IOException { - - mapperService = mock(MapperService.class); - DocumentMapper documentMapper = mock(DocumentMapper.class); - when(mapperService.documentMapper()).thenReturn(documentMapper); - Settings settings = Settings.builder().put(settings(org.opensearch.Version.CURRENT).build()).build(); - NumberFieldMapper numberFieldMapper1 = new NumberFieldMapper.Builder("field2", NumberFieldMapper.NumberType.HALF_FLOAT, false, true) - .build(new Mapper.BuilderContext(settings, new ContentPath())); - NumberFieldMapper numberFieldMapper2 = new NumberFieldMapper.Builder("field4", NumberFieldMapper.NumberType.HALF_FLOAT, false, true) - .build(new Mapper.BuilderContext(settings, new ContentPath())); - NumberFieldMapper numberFieldMapper3 = new NumberFieldMapper.Builder("field6", NumberFieldMapper.NumberType.HALF_FLOAT, false, true) - .build(new Mapper.BuilderContext(settings, new ContentPath())); - MappingLookup fieldMappers = new MappingLookup( - Set.of(numberFieldMapper1, numberFieldMapper2, numberFieldMapper3), - Collections.emptyList(), - Collections.emptyList(), - 0, - null - ); - when(documentMapper.mappers()).thenReturn(fieldMappers); - builder = new OnHeapStarTreeBuilder(compositeField, fieldProducerMap, writeState, mapperService); - - int noOfStarTreeDocuments = 5; - StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; - - starTreeDocuments[0] = new StarTreeDocument( - new Long[] { 2L, 4L, 3L, 4L }, - new HalfFloatPoint[] { new HalfFloatPoint("hf1", 12), new HalfFloatPoint("hf6", 10), new HalfFloatPoint("field6", 10) } - ); - starTreeDocuments[1] = new StarTreeDocument( - new Long[] { 3L, 4L, 2L, 1L }, - new HalfFloatPoint[] { new HalfFloatPoint("hf2", 10), new HalfFloatPoint("hf7", 6), new HalfFloatPoint("field6", 10) } - ); - starTreeDocuments[2] = new StarTreeDocument( - new Long[] { 3L, 4L, 2L, 1L }, - new HalfFloatPoint[] { new HalfFloatPoint("hf3", 14), new HalfFloatPoint("hf8", 12), new HalfFloatPoint("field6", 10) } - ); - starTreeDocuments[3] = new StarTreeDocument( - new Long[] { 2L, 4L, 3L, 4L }, - new HalfFloatPoint[] { new HalfFloatPoint("hf4", 9), new HalfFloatPoint("hf9", 4), new HalfFloatPoint("field6", 10) } - ); - starTreeDocuments[4] = new StarTreeDocument( - new Long[] { 3L, 4L, 2L, 1L }, - new HalfFloatPoint[] { new HalfFloatPoint("hf5", 11), new HalfFloatPoint("hf10", 16), new HalfFloatPoint("field6", 10) } - ); - - StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; - for (int i = 0; i < noOfStarTreeDocuments; i++) { - long metric1 = HalfFloatPoint.halfFloatToSortableShort( - ((HalfFloatPoint) starTreeDocuments[i].metrics[0]).numericValue().floatValue() - ); - long metric2 = HalfFloatPoint.halfFloatToSortableShort( - ((HalfFloatPoint) starTreeDocuments[i].metrics[1]).numericValue().floatValue() - ); - long metric3 = HalfFloatPoint.halfFloatToSortableShort( - ((HalfFloatPoint) starTreeDocuments[i].metrics[2]).numericValue().floatValue() - ); - segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, new Long[] { metric1, metric2, metric3 }); - } - - Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateStarTreeDocuments(segmentStarTreeDocuments); - builder.build(segmentStarTreeDocumentIterator); - - List resultStarTreeDocuments = builder.getStarTreeDocuments(); - assertEquals(7, resultStarTreeDocuments.size()); - - Iterator expectedStarTreeDocumentIterator = getExpectedStarTreeDocumentIterator(); - assertStarTreeDocuments(resultStarTreeDocuments, expectedStarTreeDocumentIterator); - } - - public void test_build_floatMetrics() throws IOException { - - mapperService = mock(MapperService.class); - DocumentMapper documentMapper = mock(DocumentMapper.class); - when(mapperService.documentMapper()).thenReturn(documentMapper); - Settings settings = Settings.builder().put(settings(org.opensearch.Version.CURRENT).build()).build(); - NumberFieldMapper numberFieldMapper1 = new NumberFieldMapper.Builder("field2", NumberFieldMapper.NumberType.FLOAT, false, true) - .build(new Mapper.BuilderContext(settings, new ContentPath())); - NumberFieldMapper numberFieldMapper2 = new NumberFieldMapper.Builder("field4", NumberFieldMapper.NumberType.FLOAT, false, true) - .build(new Mapper.BuilderContext(settings, new ContentPath())); - NumberFieldMapper numberFieldMapper3 = new NumberFieldMapper.Builder("field6", NumberFieldMapper.NumberType.FLOAT, false, true) - .build(new Mapper.BuilderContext(settings, new ContentPath())); - MappingLookup fieldMappers = new MappingLookup( - Set.of(numberFieldMapper1, numberFieldMapper2, numberFieldMapper3), - Collections.emptyList(), - Collections.emptyList(), - 0, - null - ); - when(documentMapper.mappers()).thenReturn(fieldMappers); - builder = new OnHeapStarTreeBuilder(compositeField, fieldProducerMap, writeState, mapperService); - - int noOfStarTreeDocuments = 5; - StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; - - starTreeDocuments[0] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Float[] { 12.0F, 10.0F, randomFloat() }); - starTreeDocuments[1] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Float[] { 10.0F, 6.0F, randomFloat() }); - starTreeDocuments[2] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Float[] { 14.0F, 12.0F, randomFloat() }); - starTreeDocuments[3] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Float[] { 9.0F, 4.0F, randomFloat() }); - starTreeDocuments[4] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Float[] { 11.0F, 16.0F, randomFloat() }); - - StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; - for (int i = 0; i < noOfStarTreeDocuments; i++) { - long metric1 = NumericUtils.floatToSortableInt((Float) starTreeDocuments[i].metrics[0]); - long metric2 = NumericUtils.floatToSortableInt((Float) starTreeDocuments[i].metrics[1]); - long metric3 = NumericUtils.floatToSortableInt((Float) starTreeDocuments[i].metrics[2]); - segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, new Long[] { metric1, metric2, metric3 }); - } - - Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateStarTreeDocuments(segmentStarTreeDocuments); - builder.build(segmentStarTreeDocumentIterator); - - List resultStarTreeDocuments = builder.getStarTreeDocuments(); - assertEquals(7, resultStarTreeDocuments.size()); - - Iterator expectedStarTreeDocumentIterator = getExpectedStarTreeDocumentIterator(); - assertStarTreeDocuments(resultStarTreeDocuments, expectedStarTreeDocumentIterator); - } - - public void test_build_longMetrics() throws IOException { - - mapperService = mock(MapperService.class); - DocumentMapper documentMapper = mock(DocumentMapper.class); - when(mapperService.documentMapper()).thenReturn(documentMapper); - Settings settings = Settings.builder().put(settings(org.opensearch.Version.CURRENT).build()).build(); - NumberFieldMapper numberFieldMapper1 = new NumberFieldMapper.Builder("field2", NumberFieldMapper.NumberType.LONG, false, true) - .build(new Mapper.BuilderContext(settings, new ContentPath())); - NumberFieldMapper numberFieldMapper2 = new NumberFieldMapper.Builder("field4", NumberFieldMapper.NumberType.LONG, false, true) - .build(new Mapper.BuilderContext(settings, new ContentPath())); - NumberFieldMapper numberFieldMapper3 = new NumberFieldMapper.Builder("field6", NumberFieldMapper.NumberType.LONG, false, true) - .build(new Mapper.BuilderContext(settings, new ContentPath())); - MappingLookup fieldMappers = new MappingLookup( - Set.of(numberFieldMapper1, numberFieldMapper2, numberFieldMapper3), - Collections.emptyList(), - Collections.emptyList(), - 0, - null - ); - when(documentMapper.mappers()).thenReturn(fieldMappers); - builder = new OnHeapStarTreeBuilder(compositeField, fieldProducerMap, writeState, mapperService); - - int noOfStarTreeDocuments = 5; - StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; - - starTreeDocuments[0] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Long[] { 12L, 10L, randomLong() }); - starTreeDocuments[1] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Long[] { 10L, 6L, randomLong() }); - starTreeDocuments[2] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Long[] { 14L, 12L, randomLong() }); - starTreeDocuments[3] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Long[] { 9L, 4L, randomLong() }); - starTreeDocuments[4] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Long[] { 11L, 16L, randomLong() }); - - StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; - for (int i = 0; i < noOfStarTreeDocuments; i++) { - long metric1 = (Long) starTreeDocuments[i].metrics[0]; - long metric2 = (Long) starTreeDocuments[i].metrics[1]; - long metric3 = (Long) starTreeDocuments[i].metrics[2]; - segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, new Long[] { metric1, metric2, metric3 }); - } - - Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateStarTreeDocuments(segmentStarTreeDocuments); - builder.build(segmentStarTreeDocumentIterator); - - List resultStarTreeDocuments = builder.getStarTreeDocuments(); - assertEquals(7, resultStarTreeDocuments.size()); - - Iterator expectedStarTreeDocumentIterator = getExpectedStarTreeDocumentIterator(); - assertStarTreeDocuments(resultStarTreeDocuments, expectedStarTreeDocumentIterator); - } - - private static Iterator getExpectedStarTreeDocumentIterator() { - List expectedStarTreeDocuments = List.of( - new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Object[] { 21.0, 14.0, 2L }), - new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Object[] { 35.0, 34.0, 3L }), - new StarTreeDocument(new Long[] { -1L, 4L, 2L, 1L }, new Object[] { 35.0, 34.0, 3L }), - new StarTreeDocument(new Long[] { -1L, 4L, 3L, 4L }, new Object[] { 21.0, 14.0, 2L }), - new StarTreeDocument(new Long[] { -1L, 4L, -1L, 1L }, new Object[] { 35.0, 34.0, 3L }), - new StarTreeDocument(new Long[] { -1L, 4L, -1L, 4L }, new Object[] { 21.0, 14.0, 2L }), - new StarTreeDocument(new Long[] { -1L, 4L, -1L, -1L }, new Object[] { 56.0, 48.0, 5L }) - ); - return expectedStarTreeDocuments.iterator(); - } - - public void test_build() throws IOException { - - int noOfStarTreeDocuments = 5; - StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; - - starTreeDocuments[0] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Double[] { 12.0, 10.0, randomDouble() }); - starTreeDocuments[1] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 10.0, 6.0, randomDouble() }); - starTreeDocuments[2] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 14.0, 12.0, randomDouble() }); - starTreeDocuments[3] = new StarTreeDocument(new Long[] { 2L, 4L, 3L, 4L }, new Double[] { 9.0, 4.0, randomDouble() }); - starTreeDocuments[4] = new StarTreeDocument(new Long[] { 3L, 4L, 2L, 1L }, new Double[] { 11.0, 16.0, randomDouble() }); - - StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; - for (int i = 0; i < noOfStarTreeDocuments; i++) { - long metric1 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[0]); - long metric2 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[1]); - long metric3 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[2]); - segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, new Long[] { metric1, metric2, metric3 }); - } - - Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateStarTreeDocuments(segmentStarTreeDocuments); - builder.build(segmentStarTreeDocumentIterator); - - List resultStarTreeDocuments = builder.getStarTreeDocuments(); - assertEquals(7, resultStarTreeDocuments.size()); - - Iterator expectedStarTreeDocumentIterator = getExpectedStarTreeDocumentIterator(); - assertStarTreeDocuments(resultStarTreeDocuments, expectedStarTreeDocumentIterator); - } - - private void assertStarTreeDocuments( - List resultStarTreeDocuments, - Iterator expectedStarTreeDocumentIterator - ) { - Iterator resultStarTreeDocumentIterator = resultStarTreeDocuments.iterator(); - while (resultStarTreeDocumentIterator.hasNext() && expectedStarTreeDocumentIterator.hasNext()) { - StarTreeDocument resultStarTreeDocument = resultStarTreeDocumentIterator.next(); - StarTreeDocument expectedStarTreeDocument = expectedStarTreeDocumentIterator.next(); - - assertEquals(expectedStarTreeDocument.dimensions[0], resultStarTreeDocument.dimensions[0]); - assertEquals(expectedStarTreeDocument.dimensions[1], resultStarTreeDocument.dimensions[1]); - assertEquals(expectedStarTreeDocument.dimensions[2], resultStarTreeDocument.dimensions[2]); - assertEquals(expectedStarTreeDocument.dimensions[3], resultStarTreeDocument.dimensions[3]); - assertEquals(expectedStarTreeDocument.metrics[0], resultStarTreeDocument.metrics[0]); - assertEquals(expectedStarTreeDocument.metrics[1], resultStarTreeDocument.metrics[1]); - assertEquals(expectedStarTreeDocument.metrics[2], resultStarTreeDocument.metrics[2]); - } - } - - public void test_build_starTreeDataset() throws IOException { - - fields = List.of("fieldC", "fieldB", "fieldL", "fieldI"); - - dimensionsOrder = List.of(new NumericDimension("fieldC"), new NumericDimension("fieldB"), new NumericDimension("fieldL")); - metrics = List.of(new Metric("fieldI", List.of(MetricStat.SUM))); - - DocValuesProducer docValuesProducer = mock(DocValuesProducer.class); - - compositeField = new StarTreeField( - "test", - dimensionsOrder, - metrics, - new StarTreeFieldConfiguration(1, Set.of(), StarTreeFieldConfiguration.StarTreeBuildMode.ON_HEAP) - ); - SegmentInfo segmentInfo = new SegmentInfo( - directory, - Version.LATEST, - Version.LUCENE_9_11_0, - "test_segment", - 7, - false, - false, - new Lucene99Codec(), - new HashMap<>(), - UUID.randomUUID().toString().substring(0, 16).getBytes(StandardCharsets.UTF_8), - new HashMap<>(), - null - ); - - fieldsInfo = new FieldInfo[fields.size()]; - fieldProducerMap = new HashMap<>(); - for (int i = 0; i < fieldsInfo.length; i++) { - fieldsInfo[i] = new FieldInfo( - fields.get(i), - i, - false, - false, - true, - IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS, - DocValuesType.SORTED_NUMERIC, - -1, - Collections.emptyMap(), - 0, - 0, - 0, - 0, - VectorEncoding.FLOAT32, - VectorSimilarityFunction.EUCLIDEAN, - false, - false - ); - fieldProducerMap.put(fields.get(i), docValuesProducer); - } - FieldInfos fieldInfos = new FieldInfos(fieldsInfo); - writeState = new SegmentWriteState(InfoStream.getDefault(), segmentInfo.dir, segmentInfo, fieldInfos, null, newIOContext(random())); - - mapperService = mock(MapperService.class); - DocumentMapper documentMapper = mock(DocumentMapper.class); - when(mapperService.documentMapper()).thenReturn(documentMapper); - Settings settings = Settings.builder().put(settings(org.opensearch.Version.CURRENT).build()).build(); - NumberFieldMapper numberFieldMapper1 = new NumberFieldMapper.Builder("fieldI", NumberFieldMapper.NumberType.DOUBLE, false, true) - .build(new Mapper.BuilderContext(settings, new ContentPath())); - MappingLookup fieldMappers = new MappingLookup( - Set.of(numberFieldMapper1), - Collections.emptyList(), - Collections.emptyList(), - 0, - null - ); - when(documentMapper.mappers()).thenReturn(fieldMappers); - builder = new OnHeapStarTreeBuilder(compositeField, fieldProducerMap, writeState, mapperService); - - int noOfStarTreeDocuments = 7; - StarTreeDocument[] starTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; - starTreeDocuments[0] = new StarTreeDocument(new Long[] { 1L, 11L, 21L }, new Double[] { 400.0 }); - starTreeDocuments[1] = new StarTreeDocument(new Long[] { 1L, 12L, 22L }, new Double[] { 200.0 }); - starTreeDocuments[2] = new StarTreeDocument(new Long[] { 2L, 13L, 23L }, new Double[] { 300.0 }); - starTreeDocuments[3] = new StarTreeDocument(new Long[] { 2L, 13L, 21L }, new Double[] { 100.0 }); - starTreeDocuments[4] = new StarTreeDocument(new Long[] { 3L, 11L, 21L }, new Double[] { 600.0 }); - starTreeDocuments[5] = new StarTreeDocument(new Long[] { 3L, 12L, 23L }, new Double[] { 200.0 }); - starTreeDocuments[6] = new StarTreeDocument(new Long[] { 3L, 12L, 21L }, new Double[] { 400.0 }); - - StarTreeDocument[] segmentStarTreeDocuments = new StarTreeDocument[noOfStarTreeDocuments]; - for (int i = 0; i < noOfStarTreeDocuments; i++) { - long metric1 = NumericUtils.doubleToSortableLong((Double) starTreeDocuments[i].metrics[0]); - segmentStarTreeDocuments[i] = new StarTreeDocument(starTreeDocuments[i].dimensions, new Long[] { metric1 }); - } - - Iterator segmentStarTreeDocumentIterator = builder.sortAndAggregateStarTreeDocuments(segmentStarTreeDocuments); - builder.build(segmentStarTreeDocumentIterator); - - List resultStarTreeDocuments = builder.getStarTreeDocuments(); - List expectedStarTreeDocuments = List.of( - new StarTreeDocument(new Long[] { 1L, 11L, 21L }, new Object[] { 400.0 }), - new StarTreeDocument(new Long[] { 1L, 12L, 22L }, new Object[] { 200.0 }), - new StarTreeDocument(new Long[] { 2L, 13L, 21L }, new Object[] { 100.0 }), - new StarTreeDocument(new Long[] { 2L, 13L, 23L }, new Object[] { 300.0 }), - new StarTreeDocument(new Long[] { 3L, 11L, 21L }, new Object[] { 600.0 }), - new StarTreeDocument(new Long[] { 3L, 12L, 21L }, new Object[] { 400.0 }), - new StarTreeDocument(new Long[] { 3L, 12L, 23L }, new Object[] { 200.0 }), - new StarTreeDocument(new Long[] { -1L, 11L, 21L }, new Object[] { 1000.0 }), - new StarTreeDocument(new Long[] { -1L, 12L, 21L }, new Object[] { 400.0 }), - new StarTreeDocument(new Long[] { -1L, 12L, 22L }, new Object[] { 200.0 }), - new StarTreeDocument(new Long[] { -1L, 12L, 23L }, new Object[] { 200.0 }), - new StarTreeDocument(new Long[] { -1L, 13L, 21L }, new Object[] { 100.0 }), - new StarTreeDocument(new Long[] { -1L, 13L, 23L }, new Object[] { 300.0 }), - new StarTreeDocument(new Long[] { -1L, -1L, 21L }, new Object[] { 1500.0 }), - new StarTreeDocument(new Long[] { -1L, -1L, 22L }, new Object[] { 200.0 }), - new StarTreeDocument(new Long[] { -1L, -1L, 23L }, new Object[] { 500.0 }), - new StarTreeDocument(new Long[] { -1L, -1L, -1L }, new Object[] { 2200.0 }), - new StarTreeDocument(new Long[] { -1L, 12L, -1L }, new Object[] { 800.0 }), - new StarTreeDocument(new Long[] { -1L, 13L, -1L }, new Object[] { 400.0 }), - new StarTreeDocument(new Long[] { 1L, -1L, 21L }, new Object[] { 400.0 }), - new StarTreeDocument(new Long[] { 1L, -1L, 22L }, new Object[] { 200.0 }), - new StarTreeDocument(new Long[] { 1L, -1L, -1L }, new Object[] { 600.0 }), - new StarTreeDocument(new Long[] { 2L, 13L, -1L }, new Object[] { 400.0 }), - new StarTreeDocument(new Long[] { 3L, -1L, 21L }, new Object[] { 1000.0 }), - new StarTreeDocument(new Long[] { 3L, -1L, 23L }, new Object[] { 200.0 }), - new StarTreeDocument(new Long[] { 3L, -1L, -1L }, new Object[] { 1200.0 }), - new StarTreeDocument(new Long[] { 3L, 12L, -1L }, new Object[] { 600.0 }) - ); - - Iterator expectedStarTreeDocumentIterator = expectedStarTreeDocuments.iterator(); - Iterator resultStarTreeDocumentIterator = resultStarTreeDocuments.iterator(); - while (resultStarTreeDocumentIterator.hasNext() && expectedStarTreeDocumentIterator.hasNext()) { - StarTreeDocument resultStarTreeDocument = resultStarTreeDocumentIterator.next(); - StarTreeDocument expectedStarTreeDocument = expectedStarTreeDocumentIterator.next(); - - assertEquals(expectedStarTreeDocument.dimensions[0], resultStarTreeDocument.dimensions[0]); - assertEquals(expectedStarTreeDocument.dimensions[1], resultStarTreeDocument.dimensions[1]); - assertEquals(expectedStarTreeDocument.dimensions[2], resultStarTreeDocument.dimensions[2]); - assertEquals(expectedStarTreeDocument.metrics[0], resultStarTreeDocument.metrics[0]); - } - - } +public class OnHeapStarTreeBuilderTests extends AbstractStarTreeBuilderTests { @Override - public void tearDown() throws Exception { - super.tearDown(); - directory.close(); + public BaseStarTreeBuilder getStarTreeBuilder( + StarTreeField starTreeField, + SegmentWriteState segmentWriteState, + MapperService mapperService + ) { + return new OnHeapStarTreeBuilder(starTreeField, segmentWriteState, mapperService); } } diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeDocValuesIteratorAdapterTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeDocValuesIteratorAdapterTests.java deleted file mode 100644 index 9c2621401faa4..0000000000000 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeDocValuesIteratorAdapterTests.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * 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.compositeindex.datacube.startree.builder; - -import org.apache.lucene.codecs.DocValuesProducer; -import org.apache.lucene.index.DocValuesType; -import org.apache.lucene.index.FieldInfo; -import org.apache.lucene.index.IndexOptions; -import org.apache.lucene.index.SortedNumericDocValues; -import org.apache.lucene.index.VectorEncoding; -import org.apache.lucene.index.VectorSimilarityFunction; -import org.apache.lucene.search.DocIdSetIterator; -import org.opensearch.index.compositeindex.datacube.startree.utils.SequentialDocValuesIterator; -import org.opensearch.test.OpenSearchTestCase; - -import java.io.IOException; -import java.util.Collections; - -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -public class StarTreeDocValuesIteratorAdapterTests extends OpenSearchTestCase { - - private StarTreeDocValuesIteratorAdapter adapter; - - @Override - public void setUp() throws Exception { - super.setUp(); - adapter = new StarTreeDocValuesIteratorAdapter(); - } - - public void testGetDocValuesIterator() throws IOException { - DocValuesProducer mockProducer = mock(DocValuesProducer.class); - SortedNumericDocValues mockSortedNumericDocValues = mock(SortedNumericDocValues.class); - - when(mockProducer.getSortedNumeric(any())).thenReturn(mockSortedNumericDocValues); - - SequentialDocValuesIterator iterator = adapter.getDocValuesIterator(DocValuesType.SORTED_NUMERIC, any(), mockProducer); - - assertNotNull(iterator); - assertEquals(mockSortedNumericDocValues, iterator.getDocIdSetIterator()); - } - - public void testGetDocValuesIteratorWithUnsupportedType() { - DocValuesProducer mockProducer = mock(DocValuesProducer.class); - FieldInfo fieldInfo = new FieldInfo( - "random_field", - 0, - false, - false, - true, - IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS, - DocValuesType.SORTED_NUMERIC, - -1, - Collections.emptyMap(), - 0, - 0, - 0, - 0, - VectorEncoding.FLOAT32, - VectorSimilarityFunction.EUCLIDEAN, - false, - false - ); - IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> { - adapter.getDocValuesIterator(DocValuesType.BINARY, fieldInfo, mockProducer); - }); - - assertEquals("Unsupported DocValuesType: BINARY", exception.getMessage()); - } - - public void testGetNextValue() throws IOException { - SortedNumericDocValues mockSortedNumericDocValues = mock(SortedNumericDocValues.class); - SequentialDocValuesIterator iterator = new SequentialDocValuesIterator(mockSortedNumericDocValues); - iterator.setDocId(1); - when(mockSortedNumericDocValues.nextValue()).thenReturn(42L); - - Long nextValue = adapter.getNextValue(iterator, 1); - - assertEquals(Long.valueOf(42L), nextValue); - assertEquals(Long.valueOf(42L), iterator.getDocValue()); - } - - public void testGetNextValueWithInvalidDocId() { - SortedNumericDocValues mockSortedNumericDocValues = mock(SortedNumericDocValues.class); - SequentialDocValuesIterator iterator = new SequentialDocValuesIterator(mockSortedNumericDocValues); - iterator.setDocId(DocIdSetIterator.NO_MORE_DOCS); - - IllegalStateException exception = expectThrows(IllegalStateException.class, () -> { adapter.getNextValue(iterator, 1); }); - - assertEquals("invalid doc id to fetch the next value", exception.getMessage()); - } - - public void testGetNextValueWithUnsupportedIterator() { - DocIdSetIterator mockIterator = mock(DocIdSetIterator.class); - SequentialDocValuesIterator iterator = new SequentialDocValuesIterator(mockIterator); - - IllegalStateException exception = expectThrows(IllegalStateException.class, () -> { adapter.getNextValue(iterator, 1); }); - - assertEquals("Unsupported Iterator: " + mockIterator.toString(), exception.getMessage()); - } - - public void testNextDoc() throws IOException { - SortedNumericDocValues mockSortedNumericDocValues = mock(SortedNumericDocValues.class); - SequentialDocValuesIterator iterator = new SequentialDocValuesIterator(mockSortedNumericDocValues); - when(mockSortedNumericDocValues.nextDoc()).thenReturn(2, 3, DocIdSetIterator.NO_MORE_DOCS); - when(mockSortedNumericDocValues.nextValue()).thenReturn(42L, 32L); - - int nextDocId = adapter.nextDoc(iterator, 1); - assertEquals(2, nextDocId); - assertEquals(Long.valueOf(42L), adapter.getNextValue(iterator, nextDocId)); - - nextDocId = adapter.nextDoc(iterator, 2); - assertEquals(3, nextDocId); - when(mockSortedNumericDocValues.nextValue()).thenReturn(42L, 32L); - - } - - public void testNextDoc_noMoreDocs() throws IOException { - SortedNumericDocValues mockSortedNumericDocValues = mock(SortedNumericDocValues.class); - SequentialDocValuesIterator iterator = new SequentialDocValuesIterator(mockSortedNumericDocValues); - when(mockSortedNumericDocValues.nextDoc()).thenReturn(2, DocIdSetIterator.NO_MORE_DOCS); - when(mockSortedNumericDocValues.nextValue()).thenReturn(42L, 32L); - - int nextDocId = adapter.nextDoc(iterator, 1); - assertEquals(2, nextDocId); - assertEquals(Long.valueOf(42L), adapter.getNextValue(iterator, nextDocId)); - - assertThrows(IllegalStateException.class, () -> adapter.nextDoc(iterator, 2)); - - } -} diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeValuesIteratorFactoryTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeValuesIteratorFactoryTests.java deleted file mode 100644 index 1aba67533d52e..0000000000000 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreeValuesIteratorFactoryTests.java +++ /dev/null @@ -1,131 +0,0 @@ -/* - * 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.compositeindex.datacube.startree.builder; - -import org.apache.lucene.codecs.DocValuesProducer; -import org.apache.lucene.index.DocValuesType; -import org.apache.lucene.index.FieldInfo; -import org.apache.lucene.index.IndexOptions; -import org.apache.lucene.index.SortedNumericDocValues; -import org.apache.lucene.index.VectorEncoding; -import org.apache.lucene.index.VectorSimilarityFunction; -import org.apache.lucene.search.DocIdSetIterator; -import org.opensearch.index.compositeindex.datacube.startree.utils.SequentialDocValuesIterator; -import org.opensearch.test.OpenSearchTestCase; -import org.junit.BeforeClass; - -import java.io.IOException; -import java.util.Collections; - -import org.mockito.Mockito; - -import static org.mockito.Mockito.when; - -public class StarTreeValuesIteratorFactoryTests extends OpenSearchTestCase { - - private static StarTreeDocValuesIteratorAdapter starTreeDocValuesIteratorAdapter; - private static FieldInfo mockFieldInfo; - - @BeforeClass - public static void setup() { - starTreeDocValuesIteratorAdapter = new StarTreeDocValuesIteratorAdapter(); - mockFieldInfo = new FieldInfo( - "field", - 1, - false, - false, - true, - IndexOptions.NONE, - DocValuesType.NONE, - -1, - Collections.emptyMap(), - 0, - 0, - 0, - 0, - VectorEncoding.FLOAT32, - VectorSimilarityFunction.EUCLIDEAN, - false, - false - ); - } - - public void testCreateIterator_SortedNumeric() throws IOException { - DocValuesProducer producer = Mockito.mock(DocValuesProducer.class); - SortedNumericDocValues iterator = Mockito.mock(SortedNumericDocValues.class); - when(producer.getSortedNumeric(mockFieldInfo)).thenReturn(iterator); - SequentialDocValuesIterator result = starTreeDocValuesIteratorAdapter.getDocValuesIterator( - DocValuesType.SORTED_NUMERIC, - mockFieldInfo, - producer - ); - assertEquals(iterator.getClass(), result.getDocIdSetIterator().getClass()); - } - - public void testCreateIterator_UnsupportedType() { - DocValuesProducer producer = Mockito.mock(DocValuesProducer.class); - IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> { - starTreeDocValuesIteratorAdapter.getDocValuesIterator(DocValuesType.BINARY, mockFieldInfo, producer); - }); - assertEquals("Unsupported DocValuesType: BINARY", exception.getMessage()); - } - - public void testGetNextValue_SortedNumeric() throws IOException { - SortedNumericDocValues iterator = Mockito.mock(SortedNumericDocValues.class); - when(iterator.nextDoc()).thenReturn(0); - when(iterator.nextValue()).thenReturn(123L); - SequentialDocValuesIterator sequentialDocValuesIterator = new SequentialDocValuesIterator(iterator); - sequentialDocValuesIterator.getDocIdSetIterator().nextDoc(); - long result = starTreeDocValuesIteratorAdapter.getNextValue(sequentialDocValuesIterator, 0); - assertEquals(123L, result); - } - - public void testGetNextValue_UnsupportedIterator() { - DocIdSetIterator iterator = Mockito.mock(DocIdSetIterator.class); - SequentialDocValuesIterator sequentialDocValuesIterator = new SequentialDocValuesIterator(iterator); - - IllegalStateException exception = expectThrows(IllegalStateException.class, () -> { - starTreeDocValuesIteratorAdapter.getNextValue(sequentialDocValuesIterator, 0); - }); - assertEquals("Unsupported Iterator: " + iterator.toString(), exception.getMessage()); - } - - public void testNextDoc() throws IOException { - SortedNumericDocValues iterator = Mockito.mock(SortedNumericDocValues.class); - SequentialDocValuesIterator sequentialDocValuesIterator = new SequentialDocValuesIterator(iterator); - when(iterator.nextDoc()).thenReturn(5); - - int result = starTreeDocValuesIteratorAdapter.nextDoc(sequentialDocValuesIterator, 5); - assertEquals(5, result); - } - - public void test_multipleCoordinatedDocumentReader() throws IOException { - SortedNumericDocValues iterator1 = Mockito.mock(SortedNumericDocValues.class); - SortedNumericDocValues iterator2 = Mockito.mock(SortedNumericDocValues.class); - - SequentialDocValuesIterator sequentialDocValuesIterator1 = new SequentialDocValuesIterator(iterator1); - SequentialDocValuesIterator sequentialDocValuesIterator2 = new SequentialDocValuesIterator(iterator2); - - when(iterator1.nextDoc()).thenReturn(0); - when(iterator2.nextDoc()).thenReturn(1); - - when(iterator1.nextValue()).thenReturn(9L); - when(iterator2.nextValue()).thenReturn(9L); - - starTreeDocValuesIteratorAdapter.nextDoc(sequentialDocValuesIterator1, 0); - starTreeDocValuesIteratorAdapter.nextDoc(sequentialDocValuesIterator2, 0); - assertEquals(0, sequentialDocValuesIterator1.getDocId()); - assertEquals(9L, (long) sequentialDocValuesIterator1.getDocValue()); - assertNotEquals(0, sequentialDocValuesIterator2.getDocId()); - assertEquals(1, sequentialDocValuesIterator2.getDocId()); - assertEquals(9L, (long) sequentialDocValuesIterator2.getDocValue()); - - } - -} diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilderTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilderTests.java index 518c6729c2e1a..564ab110fa7a5 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilderTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/builder/StarTreesBuilderTests.java @@ -88,16 +88,16 @@ public void setUp() throws Exception { public void test_buildWithNoStarTreeFields() throws IOException { when(mapperService.getCompositeFieldTypes()).thenReturn(new HashSet<>()); - StarTreesBuilder starTreesBuilder = new StarTreesBuilder(fieldProducerMap, segmentWriteState, mapperService); - starTreesBuilder.build(); + StarTreesBuilder starTreesBuilder = new StarTreesBuilder(segmentWriteState, mapperService); + starTreesBuilder.build(fieldProducerMap); verifyNoInteractions(docValuesProducer); } public void test_getStarTreeBuilder() throws IOException { when(mapperService.getCompositeFieldTypes()).thenReturn(Set.of(starTreeFieldType)); - StarTreesBuilder starTreesBuilder = new StarTreesBuilder(fieldProducerMap, segmentWriteState, mapperService); - StarTreeBuilder starTreeBuilder = starTreesBuilder.getStarTreeBuilder(starTreeField, fieldProducerMap, segmentWriteState, mapperService); + StarTreesBuilder starTreesBuilder = new StarTreesBuilder(segmentWriteState, mapperService); + StarTreeBuilder starTreeBuilder = starTreesBuilder.getSingleTreeBuilder(starTreeField, segmentWriteState, mapperService); assertTrue(starTreeBuilder instanceof OnHeapStarTreeBuilder); } @@ -105,8 +105,8 @@ public void test_getStarTreeBuilder_illegalArgument() { when(mapperService.getCompositeFieldTypes()).thenReturn(Set.of(starTreeFieldType)); StarTreeFieldConfiguration starTreeFieldConfiguration = new StarTreeFieldConfiguration(1, new HashSet<>(), StarTreeFieldConfiguration.StarTreeBuildMode.OFF_HEAP); StarTreeField starTreeField = new StarTreeField("star_tree", new ArrayList<>(), new ArrayList<>(), starTreeFieldConfiguration); - StarTreesBuilder starTreesBuilder = new StarTreesBuilder(fieldProducerMap, segmentWriteState, mapperService); - assertThrows(IllegalArgumentException.class, () -> starTreesBuilder.getStarTreeBuilder(starTreeField, fieldProducerMap, segmentWriteState, mapperService)); + StarTreesBuilder starTreesBuilder = new StarTreesBuilder(segmentWriteState, mapperService); + assertThrows(IllegalArgumentException.class, () -> starTreesBuilder.getSingleTreeBuilder(starTreeField, segmentWriteState, mapperService)); } public void test_closeWithNoStarTreeFields() throws IOException { @@ -118,7 +118,7 @@ public void test_closeWithNoStarTreeFields() throws IOException { StarTreeField starTreeField = new StarTreeField("star_tree", new ArrayList<>(), new ArrayList<>(), starTreeFieldConfiguration); starTreeFieldType = new StarTreeMapper.StarTreeFieldType("star_tree", starTreeField); when(mapperService.getCompositeFieldTypes()).thenReturn(Set.of(starTreeFieldType)); - StarTreesBuilder starTreesBuilder = new StarTreesBuilder(fieldProducerMap, segmentWriteState, mapperService); + StarTreesBuilder starTreesBuilder = new StarTreesBuilder(segmentWriteState, mapperService); starTreesBuilder.close(); verifyNoInteractions(docValuesProducer); diff --git a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/SequentialDocValuesIteratorTests.java b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/SequentialDocValuesIteratorTests.java index 76b612e3677f7..dfc83125b2806 100644 --- a/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/SequentialDocValuesIteratorTests.java +++ b/server/src/test/java/org/opensearch/index/compositeindex/datacube/startree/utils/SequentialDocValuesIteratorTests.java @@ -8,39 +8,126 @@ package org.opensearch.index.compositeindex.datacube.startree.utils; +import org.apache.lucene.codecs.DocValuesProducer; +import org.apache.lucene.index.BinaryDocValues; +import org.apache.lucene.index.DocValuesType; +import org.apache.lucene.index.FieldInfo; +import org.apache.lucene.index.IndexOptions; import org.apache.lucene.index.SortedNumericDocValues; -import org.opensearch.index.fielddata.AbstractNumericDocValues; +import org.apache.lucene.index.VectorEncoding; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.util.BytesRef; import org.opensearch.test.OpenSearchTestCase; +import org.junit.BeforeClass; import java.io.IOException; +import java.util.Collections; + +import org.mockito.Mockito; + +import static org.mockito.Mockito.when; public class SequentialDocValuesIteratorTests extends OpenSearchTestCase { - public void test_sequentialDocValuesIterator() { - SequentialDocValuesIterator sequentialDocValuesIterator = new SequentialDocValuesIterator(new AbstractNumericDocValues() { - @Override - public long longValue() throws IOException { - return 0; - } - - @Override - public boolean advanceExact(int i) throws IOException { - return false; - } - - @Override - public int docID() { - return 0; - } + private static FieldInfo mockFieldInfo; + + @BeforeClass + public static void setup() { + mockFieldInfo = new FieldInfo( + "field", + 1, + false, + false, + true, + IndexOptions.NONE, + DocValuesType.NONE, + -1, + Collections.emptyMap(), + 0, + 0, + 0, + 0, + VectorEncoding.FLOAT32, + VectorSimilarityFunction.EUCLIDEAN, + false, + false + ); + } + + public void testCreateIterator_SortedNumeric() throws IOException { + DocValuesProducer producer = Mockito.mock(DocValuesProducer.class); + SortedNumericDocValues iterator = Mockito.mock(SortedNumericDocValues.class); + when(producer.getSortedNumeric(mockFieldInfo)).thenReturn(iterator); + SequentialDocValuesIterator result = new SequentialDocValuesIterator(producer.getSortedNumeric(mockFieldInfo)); + assertEquals(iterator.getClass(), result.getDocIdSetIterator().getClass()); + } + + public void testCreateIterator_UnsupportedType() throws IOException { + DocValuesProducer producer = Mockito.mock(DocValuesProducer.class); + BinaryDocValues iterator = Mockito.mock(BinaryDocValues.class); + when(producer.getBinary(mockFieldInfo)).thenReturn(iterator); + SequentialDocValuesIterator result = new SequentialDocValuesIterator(producer.getBinary(mockFieldInfo)); + assertEquals(iterator.getClass(), result.getDocIdSetIterator().getClass()); + when(iterator.nextDoc()).thenReturn(0); + when(iterator.binaryValue()).thenReturn(new BytesRef("123")); + + IllegalStateException exception = expectThrows(IllegalStateException.class, () -> { + result.nextDoc(0); + result.value(0); }); + assertEquals("Unsupported Iterator requested for SequentialDocValuesIterator", exception.getMessage()); + } + + public void testGetNextValue_SortedNumeric() throws IOException { + SortedNumericDocValues iterator = Mockito.mock(SortedNumericDocValues.class); + when(iterator.nextDoc()).thenReturn(0); + when(iterator.nextValue()).thenReturn(123L); + SequentialDocValuesIterator sequentialDocValuesIterator = new SequentialDocValuesIterator(iterator); + sequentialDocValuesIterator.nextDoc(0); + long result = sequentialDocValuesIterator.value(0); + assertEquals(123L, result); + } + + public void testGetNextValue_UnsupportedIterator() { + DocIdSetIterator iterator = Mockito.mock(DocIdSetIterator.class); + SequentialDocValuesIterator sequentialDocValuesIterator = new SequentialDocValuesIterator(iterator); + + IllegalStateException exception = expectThrows(IllegalStateException.class, () -> { sequentialDocValuesIterator.value(0); }); + assertEquals("Unsupported Iterator requested for SequentialDocValuesIterator", exception.getMessage()); + } + + public void testNextDoc() throws IOException { + SortedNumericDocValues iterator = Mockito.mock(SortedNumericDocValues.class); + SequentialDocValuesIterator sequentialDocValuesIterator = new SequentialDocValuesIterator(iterator); + when(iterator.nextDoc()).thenReturn(5); - assertTrue(sequentialDocValuesIterator.getDocIdSetIterator() instanceof AbstractNumericDocValues); - assertEquals(sequentialDocValuesIterator.getDocId(), 0); + int result = sequentialDocValuesIterator.nextDoc(5); + assertEquals(5, result); } - public void test_sequentialDocValuesIterator_default() { - SequentialDocValuesIterator sequentialDocValuesIterator = new SequentialDocValuesIterator(); - assertTrue(sequentialDocValuesIterator.getDocIdSetIterator() instanceof SortedNumericDocValues); + public void test_multipleCoordinatedDocumentReader() throws IOException { + SortedNumericDocValues iterator1 = Mockito.mock(SortedNumericDocValues.class); + SortedNumericDocValues iterator2 = Mockito.mock(SortedNumericDocValues.class); + + SequentialDocValuesIterator sequentialDocValuesIterator1 = new SequentialDocValuesIterator(iterator1); + SequentialDocValuesIterator sequentialDocValuesIterator2 = new SequentialDocValuesIterator(iterator2); + + when(iterator1.nextDoc()).thenReturn(0); + when(iterator2.nextDoc()).thenReturn(1); + + when(iterator1.nextValue()).thenReturn(9L); + when(iterator2.nextValue()).thenReturn(9L); + + sequentialDocValuesIterator1.nextDoc(0); + sequentialDocValuesIterator2.nextDoc(0); + assertEquals(0, sequentialDocValuesIterator1.getDocId()); + assertEquals(9L, (long) sequentialDocValuesIterator1.value(0)); + assertNull(sequentialDocValuesIterator2.value(0)); + assertNotEquals(0, sequentialDocValuesIterator2.getDocId()); + assertEquals(1, sequentialDocValuesIterator2.getDocId()); + assertEquals(9L, (long) sequentialDocValuesIterator2.value(1)); + } } diff --git a/test/framework/src/main/java/org/opensearch/index/MapperTestUtils.java b/test/framework/src/main/java/org/opensearch/index/MapperTestUtils.java index 108492c1cf8f9..302180fcf95df 100644 --- a/test/framework/src/main/java/org/opensearch/index/MapperTestUtils.java +++ b/test/framework/src/main/java/org/opensearch/index/MapperTestUtils.java @@ -38,6 +38,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.Environment; +import org.opensearch.index.analysis.AnalysisTestsHelper; import org.opensearch.index.analysis.IndexAnalyzers; import org.opensearch.index.mapper.DocumentMapper; import org.opensearch.index.mapper.DocumentMapperParser; @@ -46,6 +47,7 @@ import org.opensearch.index.similarity.SimilarityService; import org.opensearch.indices.IndicesModule; import org.opensearch.indices.mapper.MapperRegistry; +import org.opensearch.plugins.AnalysisPlugin; import org.opensearch.test.IndexSettingsModule; import java.io.IOException; @@ -97,6 +99,38 @@ public static MapperService newMapperService( ); } + public static MapperService newMapperServiceWithHelperAnalyzer( + NamedXContentRegistry xContentRegistry, + Path tempDir, + Settings settings, + IndicesModule indicesModule, + String indexName + ) throws IOException { + Settings.Builder settingsBuilder = Settings.builder().put(Environment.PATH_HOME_SETTING.getKey(), tempDir).put(settings); + if (settings.get(IndexMetadata.SETTING_VERSION_CREATED) == null) { + settingsBuilder.put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT); + } + Settings finalSettings = settingsBuilder.build(); + MapperRegistry mapperRegistry = indicesModule.getMapperRegistry(); + IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(indexName, finalSettings); + IndexAnalyzers indexAnalyzers = createMockTestAnalysis(finalSettings); + SimilarityService similarityService = new SimilarityService(indexSettings, null, Collections.emptyMap()); + return new MapperService( + indexSettings, + indexAnalyzers, + xContentRegistry, + similarityService, + mapperRegistry, + () -> null, + () -> false, + null + ); + } + + public static IndexAnalyzers createMockTestAnalysis(Settings nodeSettings, AnalysisPlugin... analysisPlugins) throws IOException { + return AnalysisTestsHelper.createTestAnalysisFromSettings(nodeSettings, analysisPlugins).indexAnalyzers; + } + public static void assertConflicts(String mapping1, String mapping2, DocumentMapperParser parser, String... conflicts) throws IOException { DocumentMapper docMapper = parser.parse("type", new CompressedXContent(mapping1)); From e749424db053ad31db1c4f1ab9374251ca9b737d Mon Sep 17 00:00:00 2001 From: Rishabh Singh Date: Tue, 23 Jul 2024 20:24:35 -0700 Subject: [PATCH 32/37] Security fixes and updates (#14928) Signed-off-by: Rishabh Singh --- .github/workflows/add-performance-comment.yml | 5 ++- .github/workflows/benchmark-pull-request.yml | 34 +++++++++---------- 2 files changed, 21 insertions(+), 18 deletions(-) diff --git a/.github/workflows/add-performance-comment.yml b/.github/workflows/add-performance-comment.yml index b522d348c84b2..fc272714c5628 100644 --- a/.github/workflows/add-performance-comment.yml +++ b/.github/workflows/add-performance-comment.yml @@ -6,7 +6,10 @@ on: jobs: add-comment: - if: github.event.label.name == 'Performance' + if: | + github.event.label.name == 'Performance' || + github.event.label.name == 'Search:Performance' || + github.event.label.name == 'Indexing:Performance' runs-on: ubuntu-latest permissions: pull-requests: write diff --git a/.github/workflows/benchmark-pull-request.yml b/.github/workflows/benchmark-pull-request.yml index 9d83331e81d5a..47abcc1178572 100644 --- a/.github/workflows/benchmark-pull-request.yml +++ b/.github/workflows/benchmark-pull-request.yml @@ -77,18 +77,6 @@ jobs: run: | echo "Invalid comment format detected. Failing the workflow." exit 1 - - id: get_approvers - run: | - echo "approvers=$(cat .github/CODEOWNERS | grep '^\*' | tr -d '* ' | sed 's/@/,/g' | sed 's/,//1')" >> $GITHUB_OUTPUT - - uses: trstringer/manual-approval@v1 - if: (!contains(steps.get_approvers.outputs.approvers, github.event.comment.user.login)) - with: - secret: ${{ github.TOKEN }} - approvers: ${{ steps.get_approvers.outputs.approvers }} - minimum-approvals: 1 - issue-title: 'Request to approve/deny benchmark run for PR #${{ env.PR_NUMBER }}' - issue-body: "Please approve or deny the benchmark run for PR #${{ env.PR_NUMBER }}" - exclude-workflow-initiator-as-approver: false - name: Get PR Details id: get_pr uses: actions/github-script@v7 @@ -106,21 +94,33 @@ jobs: return { "headRepoFullName": pull_request.head.repo.full_name, - "headRef": pull_request.head.ref + "headRefSha": pull_request.head.sha }; - name: Set pr details env vars run: | echo '${{ steps.get_pr.outputs.result }}' | jq -r '.headRepoFullName' - echo '${{ steps.get_pr.outputs.result }}' | jq -r '.headRef' + echo '${{ steps.get_pr.outputs.result }}' | jq -r '.headRefSha' headRepo=$(echo '${{ steps.get_pr.outputs.result }}' | jq -r '.headRepoFullName') - headRef=$(echo '${{ steps.get_pr.outputs.result }}' | jq -r '.headRef') + headRefSha=$(echo '${{ steps.get_pr.outputs.result }}' | jq -r '.headRefSha') echo "prHeadRepo=$headRepo" >> $GITHUB_ENV - echo "prHeadRef=$headRef" >> $GITHUB_ENV + echo "prHeadRefSha=$headRefSha" >> $GITHUB_ENV + - id: get_approvers + run: | + echo "approvers=$(cat .github/CODEOWNERS | grep '^\*' | tr -d '* ' | sed 's/@/,/g' | sed 's/,//1')" >> $GITHUB_OUTPUT + - uses: trstringer/manual-approval@v1 + if: (!contains(steps.get_approvers.outputs.approvers, github.event.comment.user.login)) + with: + secret: ${{ github.TOKEN }} + approvers: ${{ steps.get_approvers.outputs.approvers }} + minimum-approvals: 1 + issue-title: 'Request to approve/deny benchmark run for PR #${{ env.PR_NUMBER }}' + issue-body: "Please approve or deny the benchmark run for PR #${{ env.PR_NUMBER }}" + exclude-workflow-initiator-as-approver: false - name: Checkout PR Repo uses: actions/checkout@v4 with: repository: ${{ env.prHeadRepo }} - ref: ${{ env.prHeadRef }} + ref: ${{ env.prHeadRefSha }} token: ${{ secrets.GITHUB_TOKEN }} - name: Setup Java uses: actions/setup-java@v1 From 2def4fd302b71a6d3ed2ce3efc3cce6800fbdd3f Mon Sep 17 00:00:00 2001 From: Sooraj Sinha <81695996+soosinha@users.noreply.github.com> Date: Wed, 24 Jul 2024 10:36:06 +0530 Subject: [PATCH 33/37] Create new IndexInput for multi part upload (#14888) * Create new IndexInput for multi part upload Signed-off-by: Sooraj Sinha --- .../transfer/BlobStoreTransferService.java | 35 ++++++++-------- .../blobstore/ChecksumBlobStoreFormat.java | 35 ++++++++-------- .../blobstore/ConfigBlobStoreFormat.java | 40 +++++++++++-------- .../BlobStoreTransferServiceTests.java | 30 +++++++++++++- 4 files changed, 86 insertions(+), 54 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java index d55abb40dec48..22bb4cf0514bf 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/BlobStoreTransferService.java @@ -131,20 +131,18 @@ public void uploadBlob( } final String resourceDescription = "BlobStoreTransferService.uploadBlob(blob=\"" + fileName + "\")"; byte[] bytes = inputStream.readAllBytes(); - try (IndexInput input = new ByteArrayIndexInput(resourceDescription, bytes)) { - long expectedChecksum = computeChecksum(input, resourceDescription); - uploadBlobAsyncInternal( - fileName, - fileName, - bytes.length, - blobPath, - writePriority, - (size, position) -> new OffsetRangeIndexInputStream(input, size, position), - expectedChecksum, - listener, - null - ); - } + long expectedChecksum = computeChecksum(bytes, resourceDescription); + uploadBlobAsyncInternal( + fileName, + fileName, + bytes.length, + blobPath, + writePriority, + (size, position) -> new OffsetRangeIndexInputStream(new ByteArrayIndexInput(resourceDescription, bytes), size, position), + expectedChecksum, + listener, + null + ); } // Builds a metadata map containing the Base64-encoded checkpoint file data associated with a translog file. @@ -220,7 +218,8 @@ private void uploadBlob( } - private void uploadBlobAsyncInternal( + // package private for testing + void uploadBlobAsyncInternal( String fileName, String remoteFileName, long contentLength, @@ -335,10 +334,10 @@ public void listAllInSortedOrderAsync( threadPool.executor(threadpoolName).execute(() -> { listAllInSortedOrder(path, filenamePrefix, limit, listener); }); } - private static long computeChecksum(IndexInput indexInput, String resourceDescription) throws ChecksumCombinationException { + private static long computeChecksum(byte[] bytes, String resourceDescription) throws ChecksumCombinationException { long expectedChecksum; - try { - expectedChecksum = checksumOfChecksum(indexInput.clone(), CHECKSUM_BYTES_LENGTH); + try (IndexInput indexInput = new ByteArrayIndexInput(resourceDescription, bytes)) { + expectedChecksum = checksumOfChecksum(indexInput, CHECKSUM_BYTES_LENGTH); } 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 e567e1b626c5a..3a49fed4be282 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumBlobStoreFormat.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/ChecksumBlobStoreFormat.java @@ -223,10 +223,11 @@ private void writeAsyncWithPriority( return; } final String blobName = blobName(name); - final BytesReference bytes = serialize(obj, blobName, compressor, params); + final BytesReference bytesReference = serialize(obj, blobName, compressor, params); final String resourceDescription = "ChecksumBlobStoreFormat.writeAsyncWithPriority(blob=\"" + blobName + "\")"; - try (IndexInput input = new ByteArrayIndexInput(resourceDescription, BytesReference.toBytes(bytes))) { - long expectedChecksum; + byte[] bytes = BytesReference.toBytes(bytesReference); + long expectedChecksum; + try (IndexInput input = new ByteArrayIndexInput(resourceDescription, bytes)) { try { expectedChecksum = checksumOfChecksum(input.clone(), 8); } catch (Exception e) { @@ -237,21 +238,21 @@ private void writeAsyncWithPriority( e ); } + } - try ( - RemoteTransferContainer remoteTransferContainer = new RemoteTransferContainer( - blobName, - blobName, - bytes.length(), - true, - priority, - (size, position) -> new OffsetRangeIndexInputStream(input, size, position), - expectedChecksum, - ((AsyncMultiStreamBlobContainer) blobContainer).remoteIntegrityCheckSupported() - ) - ) { - ((AsyncMultiStreamBlobContainer) blobContainer).asyncBlobUpload(remoteTransferContainer.createWriteContext(), listener); - } + try ( + RemoteTransferContainer remoteTransferContainer = new RemoteTransferContainer( + blobName, + blobName, + bytes.length, + true, + priority, + (size, position) -> new OffsetRangeIndexInputStream(new ByteArrayIndexInput(resourceDescription, bytes), size, position), + expectedChecksum, + ((AsyncMultiStreamBlobContainer) blobContainer).remoteIntegrityCheckSupported() + ) + ) { + ((AsyncMultiStreamBlobContainer) blobContainer).asyncBlobUpload(remoteTransferContainer.createWriteContext(), listener); } } diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/ConfigBlobStoreFormat.java b/server/src/main/java/org/opensearch/repositories/blobstore/ConfigBlobStoreFormat.java index 18c718ca2110e..8127bf8c2a2a2 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/ConfigBlobStoreFormat.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/ConfigBlobStoreFormat.java @@ -8,7 +8,6 @@ package org.opensearch.repositories.blobstore; -import org.apache.lucene.store.IndexInput; import org.opensearch.common.blobstore.AsyncMultiStreamBlobContainer; import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.common.blobstore.stream.write.WritePriority; @@ -51,23 +50,30 @@ public void writeAsyncWithUrgentPriority(T obj, BlobContainer blobContainer, Str return; } String blobName = blobName(name); - BytesReference bytes = serialize(obj, blobName, new NoneCompressor(), ToXContent.EMPTY_PARAMS, XContentType.JSON, null, null); + BytesReference bytesReference = serialize( + obj, + blobName, + new NoneCompressor(), + ToXContent.EMPTY_PARAMS, + XContentType.JSON, + null, + null + ); String resourceDescription = "BlobStoreFormat.writeAsyncWithPriority(blob=\"" + blobName + "\")"; - try (IndexInput input = new ByteArrayIndexInput(resourceDescription, BytesReference.toBytes(bytes))) { - try ( - RemoteTransferContainer remoteTransferContainer = new RemoteTransferContainer( - blobName, - blobName, - bytes.length(), - true, - WritePriority.URGENT, - (size, position) -> new OffsetRangeIndexInputStream(input, size, position), - null, - false - ) - ) { - ((AsyncMultiStreamBlobContainer) blobContainer).asyncBlobUpload(remoteTransferContainer.createWriteContext(), listener); - } + byte[] bytes = BytesReference.toBytes(bytesReference); + try ( + RemoteTransferContainer remoteTransferContainer = new RemoteTransferContainer( + blobName, + blobName, + bytes.length, + true, + WritePriority.URGENT, + (size, position) -> new OffsetRangeIndexInputStream(new ByteArrayIndexInput(resourceDescription, bytes), size, position), + null, + false + ) + ) { + ((AsyncMultiStreamBlobContainer) blobContainer).asyncBlobUpload(remoteTransferContainer.createWriteContext(), listener); } } } diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceTests.java index cd78aead80923..10e4cc6cfb1ef 100644 --- a/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceTests.java +++ b/server/src/test/java/org/opensearch/index/translog/transfer/BlobStoreTransferServiceTests.java @@ -22,6 +22,8 @@ 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.transfer.RemoteTransferContainer; +import org.opensearch.common.blobstore.transfer.stream.OffsetRangeInputStream; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.core.action.ActionListener; @@ -54,9 +56,13 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; + import static org.opensearch.index.translog.transfer.TranslogTransferManager.CHECKPOINT_FILE_DATA_KEY; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; public class BlobStoreTransferServiceTests extends OpenSearchTestCase { @@ -139,8 +145,28 @@ public void testUploadBlobFromInputStreamAsyncFSRepo() throws IOException, Inter FsBlobStore fsBlobStore = mock(FsBlobStore.class); when(fsBlobStore.blobContainer(any())).thenReturn(mockAsyncFsContainer); - TransferService transferService = new BlobStoreTransferService(fsBlobStore, threadPool); - uploadBlobFromInputStream(transferService); + BlobStoreTransferService transferServiceSpy = Mockito.spy(new BlobStoreTransferService(fsBlobStore, threadPool)); + uploadBlobFromInputStream(transferServiceSpy); + + ArgumentCaptor inputStreamCaptor = ArgumentCaptor.forClass( + RemoteTransferContainer.OffsetRangeInputStreamSupplier.class + ); + verify(transferServiceSpy).uploadBlobAsyncInternal( + Mockito.anyString(), + Mockito.anyString(), + Mockito.anyLong(), + Mockito.any(), + Mockito.any(), + inputStreamCaptor.capture(), + Mockito.anyLong(), + Mockito.any(), + Mockito.any() + ); + RemoteTransferContainer.OffsetRangeInputStreamSupplier inputStreamSupplier = inputStreamCaptor.getValue(); + OffsetRangeInputStream inputStream1 = inputStreamSupplier.get(1, 0); + OffsetRangeInputStream inputStream2 = inputStreamSupplier.get(1, 2); + assertNotEquals(inputStream1, inputStream2); + assertNotEquals(inputStream1.getFilePointer(), inputStream2.getFilePointer()); } private IndexMetadata getIndexMetadata() { From 7673a7733ccecc8730e8a3ecff898b72dc3deaa6 Mon Sep 17 00:00:00 2001 From: Pranshu Shukla <55992439+Pranshu-S@users.noreply.github.com> Date: Wed, 24 Jul 2024 10:54:22 +0530 Subject: [PATCH 34/37] Updating Cluster Stats Optimisation Versions to 2.16 (#14914) * Updating Cluster Stats Optimisation Versions to 2.16 Signed-off-by: Pranshu Shukla --- .../action/admin/cluster/stats/ClusterStatsNodeResponse.java | 4 ++-- .../action/admin/cluster/stats/ClusterStatsRequest.java | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsNodeResponse.java b/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsNodeResponse.java index 133cf68f5f8c9..6ed3ca7c409e7 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsNodeResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsNodeResponse.java @@ -77,7 +77,7 @@ public ClusterStatsNodeResponse(StreamInput in) throws IOException { } this.nodeInfo = new NodeInfo(in); this.nodeStats = new NodeStats(in); - if (in.getVersion().onOrAfter(Version.V_3_0_0)) { + if (in.getVersion().onOrAfter(Version.V_2_16_0)) { this.shardsStats = in.readOptionalArray(ShardStats::new, ShardStats[]::new); this.aggregatedNodeLevelStats = in.readOptionalWriteable(AggregatedNodeLevelStats::new); } else { @@ -156,7 +156,7 @@ public void writeTo(StreamOutput out) throws IOException { } nodeInfo.writeTo(out); nodeStats.writeTo(out); - if (out.getVersion().onOrAfter(Version.V_3_0_0)) { + if (out.getVersion().onOrAfter(Version.V_2_16_0)) { if (aggregatedNodeLevelStats != null) { out.writeOptionalArray(null); out.writeOptionalWriteable(aggregatedNodeLevelStats); diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsRequest.java index fdeb82a3466f2..bd75b2210e474 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsRequest.java @@ -50,7 +50,7 @@ public class ClusterStatsRequest extends BaseNodesRequest { public ClusterStatsRequest(StreamInput in) throws IOException { super(in); - if (in.getVersion().onOrAfter(Version.V_3_0_0)) { + if (in.getVersion().onOrAfter(Version.V_2_16_0)) { useAggregatedNodeLevelResponses = in.readOptionalBoolean(); } } @@ -76,7 +76,7 @@ public void useAggregatedNodeLevelResponses(boolean useAggregatedNodeLevelRespon @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - if (out.getVersion().onOrAfter(Version.V_3_0_0)) { + if (out.getVersion().onOrAfter(Version.V_2_16_0)) { out.writeOptionalBoolean(useAggregatedNodeLevelResponses); } } From 5744eae80dfe466397f4254acf995794855db370 Mon Sep 17 00:00:00 2001 From: shailendra0811 <167273922+shailendra0811@users.noreply.github.com> Date: Wed, 24 Jul 2024 14:59:30 +0530 Subject: [PATCH 35/37] Fix read/write method for Diff Manifest in case Shard diff file is null. (#14938) Signed-off-by: Shailendra Singh --- .../gateway/remote/ClusterStateDiffManifest.java | 8 ++++---- .../opensearch/gateway/remote/RemotePersistenceStats.java | 4 ++-- .../remote/RemoteClusterStateCleanupManagerTests.java | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/opensearch/gateway/remote/ClusterStateDiffManifest.java b/server/src/main/java/org/opensearch/gateway/remote/ClusterStateDiffManifest.java index ab7fa1fddf4bf..a3b36ddcff1a7 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/ClusterStateDiffManifest.java +++ b/server/src/main/java/org/opensearch/gateway/remote/ClusterStateDiffManifest.java @@ -129,7 +129,6 @@ public ClusterStateDiffManifest( clusterStateCustomUpdated = new ArrayList<>(clusterStateCustomDiff.getDiffs().keySet()); clusterStateCustomUpdated.addAll(clusterStateCustomDiff.getUpserts().keySet()); clusterStateCustomDeleted = clusterStateCustomDiff.getDeletes(); - List indicie1s = indicesRoutingUpdated; } public ClusterStateDiffManifest( @@ -190,7 +189,7 @@ public ClusterStateDiffManifest(StreamInput in) throws IOException { this.hashesOfConsistentSettingsUpdated = in.readBoolean(); this.clusterStateCustomUpdated = in.readStringList(); this.clusterStateCustomDeleted = in.readStringList(); - this.indicesRoutingDiffPath = in.readString(); + this.indicesRoutingDiffPath = in.readOptionalString(); } @Override @@ -535,7 +534,8 @@ public int hashCode() { indicesRoutingDeleted, hashesOfConsistentSettingsUpdated, clusterStateCustomUpdated, - clusterStateCustomDeleted + clusterStateCustomDeleted, + indicesRoutingDiffPath ); } @@ -562,7 +562,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeBoolean(hashesOfConsistentSettingsUpdated); out.writeStringCollection(clusterStateCustomUpdated); out.writeStringCollection(clusterStateCustomDeleted); - out.writeString(indicesRoutingDiffPath); + out.writeOptionalString(indicesRoutingDiffPath); } /** diff --git a/server/src/main/java/org/opensearch/gateway/remote/RemotePersistenceStats.java b/server/src/main/java/org/opensearch/gateway/remote/RemotePersistenceStats.java index efd73e11e46b5..1e7f8f278fb0f 100644 --- a/server/src/main/java/org/opensearch/gateway/remote/RemotePersistenceStats.java +++ b/server/src/main/java/org/opensearch/gateway/remote/RemotePersistenceStats.java @@ -51,10 +51,10 @@ public long getIndexRoutingFilesCleanupAttemptFailedCount() { } public void indicesRoutingDiffFileCleanupAttemptFailed() { - indexRoutingFilesCleanupAttemptFailedCount.incrementAndGet(); + indicesRoutingDiffFilesCleanupAttemptFailedCount.incrementAndGet(); } public long getIndicesRoutingDiffFileCleanupAttemptFailedCount() { - return indexRoutingFilesCleanupAttemptFailedCount.get(); + return indicesRoutingDiffFilesCleanupAttemptFailedCount.get(); } } diff --git a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerTests.java b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerTests.java index b86f23f3d37aa..920a48f02b99a 100644 --- a/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerTests.java +++ b/server/src/test/java/org/opensearch/gateway/remote/RemoteClusterStateCleanupManagerTests.java @@ -652,7 +652,7 @@ public void testIndicesRoutingDiffFilesCleanupFailureStats() throws Exception { assertEquals(0, remoteClusterStateCleanupManager.getStats().getIndicesRoutingDiffFileCleanupAttemptFailedCount()); }); - doThrow(IOException.class).when(remoteRoutingTableService).deleteStaleIndexRoutingPaths(any()); + doThrow(IOException.class).when(remoteRoutingTableService).deleteStaleIndexRoutingDiffPaths(any()); remoteClusterStateCleanupManager.deleteClusterMetadata(clusterName, clusterUUID, activeBlobs, inactiveBlobs); assertBusy(() -> { // wait for stats to get updated From 2a14c2772cc53bf2941e80c911307eaaacca055d Mon Sep 17 00:00:00 2001 From: Bukhtawar Khan Date: Wed, 24 Jul 2024 17:23:55 +0530 Subject: [PATCH 36/37] Make reroute iteration time-bound for large shard allocations (#14848) * Make reroute iteration time-bound for large shard allocations Signed-off-by: Bukhtawar Khan Co-authored-by: Rishab Nahata --- CHANGELOG.md | 1 + .../gateway/RecoveryFromGatewayIT.java | 128 +++++++++++++++++- .../routing/allocation/AllocationService.java | 5 +- .../allocation/ExistingShardsAllocator.java | 7 +- .../common/settings/ClusterSettings.java | 2 + .../common/util/BatchRunnableExecutor.java | 66 +++++++++ .../util/concurrent/TimeoutAwareRunnable.java | 19 +++ .../gateway/BaseGatewayShardAllocator.java | 21 +++ .../gateway/ShardsBatchGatewayAllocator.java | 86 ++++++++++-- .../ExistingShardsAllocatorTests.java | 118 ++++++++++++++++ .../util/BatchRunnableExecutorTests.java | 97 +++++++++++++ .../gateway/GatewayAllocatorTests.java | 32 +++++ .../PrimaryShardBatchAllocatorTests.java | 47 +++++++ .../ReplicaShardBatchAllocatorTests.java | 27 ++++ .../TestShardBatchGatewayAllocator.java | 5 +- 15 files changed, 645 insertions(+), 16 deletions(-) create mode 100644 server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java create mode 100644 server/src/main/java/org/opensearch/common/util/concurrent/TimeoutAwareRunnable.java create mode 100644 server/src/test/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocatorTests.java create mode 100644 server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 6aa3d7a58dda4..edc0ca2732f25 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -62,6 +62,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Allow @InternalApi annotation on classes not meant to be constructed outside of the OpenSearch core ([#14575](https://github.com/opensearch-project/OpenSearch/pull/14575)) - Add @InternalApi annotation to japicmp exclusions ([#14597](https://github.com/opensearch-project/OpenSearch/pull/14597)) - Allow system index warning in OpenSearchRestTestCase.refreshAllIndices ([#14635](https://github.com/opensearch-project/OpenSearch/pull/14635)) +- Make reroute iteration time-bound for large shard allocations ([#14848](https://github.com/opensearch-project/OpenSearch/pull/14848)) ### Deprecated - Deprecate batch_size parameter on bulk API ([#14725](https://github.com/opensearch-project/OpenSearch/pull/14725)) diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index 6296608c64d37..4085cc3890f30 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -769,7 +769,7 @@ public void testMessyElectionsStillMakeClusterGoGreen() throws Exception { ensureGreen("test"); } - public void testBatchModeEnabled() throws Exception { + public void testBatchModeEnabledWithoutTimeout() throws Exception { internalCluster().startClusterManagerOnlyNodes( 1, Settings.builder().put(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.getKey(), true).build() @@ -810,6 +810,132 @@ public void testBatchModeEnabled() throws Exception { assertEquals(0, gatewayAllocator.getNumberOfInFlightFetches()); } + public void testBatchModeEnabledWithSufficientTimeoutAndClusterGreen() throws Exception { + internalCluster().startClusterManagerOnlyNodes( + 1, + Settings.builder() + .put(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.getKey(), true) + .put(ShardsBatchGatewayAllocator.PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING.getKey(), "20s") + .put(ShardsBatchGatewayAllocator.REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING.getKey(), "20s") + .build() + ); + List dataOnlyNodes = internalCluster().startDataOnlyNodes(2); + createIndex( + "test", + Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1).build() + ); + ensureGreen("test"); + Settings node0DataPathSettings = internalCluster().dataPathSettings(dataOnlyNodes.get(0)); + Settings node1DataPathSettings = internalCluster().dataPathSettings(dataOnlyNodes.get(1)); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(dataOnlyNodes.get(0))); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(dataOnlyNodes.get(1))); + ensureRed("test"); + ensureStableCluster(1); + + logger.info("--> Now do a protective reroute"); + ClusterRerouteResponse clusterRerouteResponse = client().admin().cluster().prepareReroute().setRetryFailed(true).get(); + assertTrue(clusterRerouteResponse.isAcknowledged()); + + ShardsBatchGatewayAllocator gatewayAllocator = internalCluster().getInstance( + ShardsBatchGatewayAllocator.class, + internalCluster().getClusterManagerName() + ); + assertTrue(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.get(internalCluster().clusterService().getSettings())); + assertEquals(1, gatewayAllocator.getNumberOfStartedShardBatches()); + assertEquals(1, gatewayAllocator.getNumberOfStoreShardBatches()); + + // Now start both data nodes and ensure batch mode is working + logger.info("--> restarting the stopped nodes"); + internalCluster().startDataOnlyNode(Settings.builder().put("node.name", dataOnlyNodes.get(0)).put(node0DataPathSettings).build()); + internalCluster().startDataOnlyNode(Settings.builder().put("node.name", dataOnlyNodes.get(1)).put(node1DataPathSettings).build()); + ensureStableCluster(3); + ensureGreen("test"); + assertEquals(0, gatewayAllocator.getNumberOfStartedShardBatches()); + assertEquals(0, gatewayAllocator.getNumberOfStoreShardBatches()); + assertEquals(0, gatewayAllocator.getNumberOfInFlightFetches()); + } + + public void testBatchModeEnabledWithInSufficientTimeoutButClusterGreen() throws Exception { + + internalCluster().startClusterManagerOnlyNodes( + 1, + Settings.builder().put(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.getKey(), true).build() + ); + List dataOnlyNodes = internalCluster().startDataOnlyNodes(2); + createNIndices(50, "test"); // this will create 50p, 50r shards + ensureStableCluster(3); + IndicesStatsResponse indicesStats = dataNodeClient().admin().indices().prepareStats().get(); + assertThat(indicesStats.getSuccessfulShards(), equalTo(100)); + ClusterHealthResponse health = client().admin() + .cluster() + .health(Requests.clusterHealthRequest().waitForGreenStatus().timeout("1m")) + .actionGet(); + assertFalse(health.isTimedOut()); + assertEquals(GREEN, health.getStatus()); + + String clusterManagerName = internalCluster().getClusterManagerName(); + Settings clusterManagerDataPathSettings = internalCluster().dataPathSettings(clusterManagerName); + Settings node0DataPathSettings = internalCluster().dataPathSettings(dataOnlyNodes.get(0)); + Settings node1DataPathSettings = internalCluster().dataPathSettings(dataOnlyNodes.get(1)); + + internalCluster().stopCurrentClusterManagerNode(); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(dataOnlyNodes.get(0))); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(dataOnlyNodes.get(1))); + + // Now start cluster manager node and post that verify batches created + internalCluster().startClusterManagerOnlyNodes( + 1, + Settings.builder() + .put("node.name", clusterManagerName) + .put(clusterManagerDataPathSettings) + .put(ShardsBatchGatewayAllocator.GATEWAY_ALLOCATOR_BATCH_SIZE.getKey(), 5) + .put(ShardsBatchGatewayAllocator.PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING.getKey(), "10ms") + .put(ShardsBatchGatewayAllocator.REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING.getKey(), "10ms") + .put(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.getKey(), true) + .build() + ); + ensureStableCluster(1); + + logger.info("--> Now do a protective reroute"); // to avoid any race condition in test + ClusterRerouteResponse clusterRerouteResponse = client().admin().cluster().prepareReroute().setRetryFailed(true).get(); + assertTrue(clusterRerouteResponse.isAcknowledged()); + + ShardsBatchGatewayAllocator gatewayAllocator = internalCluster().getInstance( + ShardsBatchGatewayAllocator.class, + internalCluster().getClusterManagerName() + ); + + assertTrue(ExistingShardsAllocator.EXISTING_SHARDS_ALLOCATOR_BATCH_MODE.get(internalCluster().clusterService().getSettings())); + assertEquals(10, gatewayAllocator.getNumberOfStartedShardBatches()); + assertEquals(10, gatewayAllocator.getNumberOfStoreShardBatches()); + health = client(internalCluster().getClusterManagerName()).admin().cluster().health(Requests.clusterHealthRequest()).actionGet(); + assertFalse(health.isTimedOut()); + assertEquals(RED, health.getStatus()); + assertEquals(100, health.getUnassignedShards()); + assertEquals(0, health.getInitializingShards()); + assertEquals(0, health.getActiveShards()); + assertEquals(0, health.getRelocatingShards()); + assertEquals(0, health.getNumberOfDataNodes()); + + // Now start both data nodes and ensure batch mode is working + logger.info("--> restarting the stopped nodes"); + internalCluster().startDataOnlyNode(Settings.builder().put("node.name", dataOnlyNodes.get(0)).put(node0DataPathSettings).build()); + internalCluster().startDataOnlyNode(Settings.builder().put("node.name", dataOnlyNodes.get(1)).put(node1DataPathSettings).build()); + ensureStableCluster(3); + + // wait for cluster to turn green + health = client().admin().cluster().health(Requests.clusterHealthRequest().waitForGreenStatus().timeout("5m")).actionGet(); + assertFalse(health.isTimedOut()); + assertEquals(GREEN, health.getStatus()); + assertEquals(0, health.getUnassignedShards()); + assertEquals(0, health.getInitializingShards()); + assertEquals(100, health.getActiveShards()); + assertEquals(0, health.getRelocatingShards()); + assertEquals(2, health.getNumberOfDataNodes()); + assertEquals(0, gatewayAllocator.getNumberOfStartedShardBatches()); + assertEquals(0, gatewayAllocator.getNumberOfStoreShardBatches()); + } + public void testBatchModeDisabled() throws Exception { internalCluster().startClusterManagerOnlyNodes( 1, diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java index 5ad3a2fd47ce3..e29a81a2c131f 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/AllocationService.java @@ -72,6 +72,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; @@ -617,10 +618,10 @@ private void allocateExistingUnassignedShards(RoutingAllocation allocation) { private void allocateAllUnassignedShards(RoutingAllocation allocation) { ExistingShardsAllocator allocator = existingShardsAllocators.get(ShardsBatchGatewayAllocator.ALLOCATOR_NAME); - allocator.allocateAllUnassignedShards(allocation, true); + Optional.ofNullable(allocator.allocateAllUnassignedShards(allocation, true)).ifPresent(Runnable::run); allocator.afterPrimariesBeforeReplicas(allocation); // Replicas Assignment - allocator.allocateAllUnassignedShards(allocation, false); + Optional.ofNullable(allocator.allocateAllUnassignedShards(allocation, false)).ifPresent(Runnable::run); } private void disassociateDeadNodes(RoutingAllocation allocation) { diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java index fb2a37237f8b6..eb7a1e7209c37 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocator.java @@ -41,6 +41,7 @@ import org.opensearch.gateway.GatewayAllocator; import org.opensearch.gateway.ShardsBatchGatewayAllocator; +import java.util.ArrayList; import java.util.List; /** @@ -108,14 +109,16 @@ void allocateUnassigned( * * Allocation service will currently run the default implementation of it implemented by {@link ShardsBatchGatewayAllocator} */ - default void allocateAllUnassignedShards(RoutingAllocation allocation, boolean primary) { + default Runnable allocateAllUnassignedShards(RoutingAllocation allocation, boolean primary) { RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); + List runnables = new ArrayList<>(); while (iterator.hasNext()) { ShardRouting shardRouting = iterator.next(); if (shardRouting.primary() == primary) { - allocateUnassigned(shardRouting, allocation, iterator); + runnables.add(() -> allocateUnassigned(shardRouting, allocation, iterator)); } } + return () -> runnables.forEach(Runnable::run); } /** 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 49801fd3834b8..2f60c731bc554 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -343,6 +343,8 @@ public void apply(Settings value, Settings current, Settings previous) { GatewayService.RECOVER_AFTER_NODES_SETTING, GatewayService.RECOVER_AFTER_TIME_SETTING, ShardsBatchGatewayAllocator.GATEWAY_ALLOCATOR_BATCH_SIZE, + ShardsBatchGatewayAllocator.PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING, + ShardsBatchGatewayAllocator.REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING, PersistedClusterStateService.SLOW_WRITE_LOGGING_THRESHOLD, NetworkModule.HTTP_DEFAULT_TYPE_SETTING, NetworkModule.TRANSPORT_DEFAULT_TYPE_SETTING, diff --git a/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java new file mode 100644 index 0000000000000..d3d3304cb909a --- /dev/null +++ b/server/src/main/java/org/opensearch/common/util/BatchRunnableExecutor.java @@ -0,0 +1,66 @@ +/* + * 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.util; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.common.Randomness; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.concurrent.TimeoutAwareRunnable; + +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; + +/** + * A {@link Runnable} that iteratively executes a batch of {@link TimeoutAwareRunnable}s. If the elapsed time exceeds the timeout defined by {@link TimeValue} timeout, then all subsequent {@link TimeoutAwareRunnable}s will have their {@link TimeoutAwareRunnable#onTimeout} method invoked and will not be run. + * + * @opensearch.internal + */ +public class BatchRunnableExecutor implements Runnable { + + private final Supplier timeoutSupplier; + + private final List timeoutAwareRunnables; + + private static final Logger logger = LogManager.getLogger(BatchRunnableExecutor.class); + + public BatchRunnableExecutor(List timeoutAwareRunnables, Supplier timeoutSupplier) { + this.timeoutSupplier = timeoutSupplier; + this.timeoutAwareRunnables = timeoutAwareRunnables; + } + + // for tests + public List getTimeoutAwareRunnables() { + return this.timeoutAwareRunnables; + } + + @Override + public void run() { + logger.debug("Starting execution of runnable of size [{}]", timeoutAwareRunnables.size()); + long startTime = System.nanoTime(); + if (timeoutAwareRunnables.isEmpty()) { + return; + } + Randomness.shuffle(timeoutAwareRunnables); + for (TimeoutAwareRunnable runnable : timeoutAwareRunnables) { + if (timeoutSupplier.get().nanos() < 0 || System.nanoTime() - startTime < timeoutSupplier.get().nanos()) { + runnable.run(); + } else { + logger.debug("Executing timeout for runnable of size [{}]", timeoutAwareRunnables.size()); + runnable.onTimeout(); + } + } + logger.debug( + "Time taken to execute timed runnables in this cycle:[{}ms]", + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime) + ); + } + +} diff --git a/server/src/main/java/org/opensearch/common/util/concurrent/TimeoutAwareRunnable.java b/server/src/main/java/org/opensearch/common/util/concurrent/TimeoutAwareRunnable.java new file mode 100644 index 0000000000000..8d3357ad93095 --- /dev/null +++ b/server/src/main/java/org/opensearch/common/util/concurrent/TimeoutAwareRunnable.java @@ -0,0 +1,19 @@ +/* + * 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.util.concurrent; + +/** + * Runnable that is aware of a timeout + * + * @opensearch.internal + */ +public interface TimeoutAwareRunnable extends Runnable { + + void onTimeout(); +} diff --git a/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java b/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java index 58982e869794f..0d6af943d39e0 100644 --- a/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/BaseGatewayShardAllocator.java @@ -36,6 +36,7 @@ import org.apache.logging.log4j.Logger; import org.opensearch.cluster.routing.RecoverySource; import org.opensearch.cluster.routing.RoutingNode; +import org.opensearch.cluster.routing.RoutingNodes; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; import org.opensearch.cluster.routing.allocation.AllocationDecision; @@ -43,9 +44,12 @@ import org.opensearch.cluster.routing.allocation.NodeAllocationResult; import org.opensearch.cluster.routing.allocation.RoutingAllocation; import org.opensearch.cluster.routing.allocation.decider.Decision; +import org.opensearch.core.index.shard.ShardId; import java.util.ArrayList; +import java.util.HashSet; import java.util.List; +import java.util.Set; /** * An abstract class that implements basic functionality for allocating @@ -78,6 +82,23 @@ public void allocateUnassigned( executeDecision(shardRouting, allocateUnassignedDecision, allocation, unassignedAllocationHandler); } + protected void allocateUnassignedBatchOnTimeout(List shardRoutings, RoutingAllocation allocation, boolean primary) { + Set shardIdsFromBatch = new HashSet<>(); + for (ShardRouting shardRouting : shardRoutings) { + ShardId shardId = shardRouting.shardId(); + shardIdsFromBatch.add(shardId); + } + RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); + while (iterator.hasNext()) { + ShardRouting unassignedShard = iterator.next(); + AllocateUnassignedDecision allocationDecision; + if (unassignedShard.primary() == primary && shardIdsFromBatch.contains(unassignedShard.shardId())) { + allocationDecision = AllocateUnassignedDecision.throttle(null); + executeDecision(unassignedShard, allocationDecision, allocation, iterator); + } + } + } + protected void executeDecision( ShardRouting shardRouting, AllocateUnassignedDecision allocateUnassignedDecision, diff --git a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java index 3c0797cd450d2..55f5388d8f454 100644 --- a/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ShardsBatchGatewayAllocator.java @@ -27,9 +27,13 @@ import org.opensearch.common.UUIDs; import org.opensearch.common.inject.Inject; import org.opensearch.common.lease.Releasables; +import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.BatchRunnableExecutor; import org.opensearch.common.util.concurrent.ConcurrentCollections; +import org.opensearch.common.util.concurrent.TimeoutAwareRunnable; import org.opensearch.common.util.set.Sets; import org.opensearch.core.action.ActionListener; import org.opensearch.core.index.shard.ShardId; @@ -41,6 +45,7 @@ import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper; import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -68,6 +73,14 @@ public class ShardsBatchGatewayAllocator implements ExistingShardsAllocator { private final long maxBatchSize; private static final short DEFAULT_SHARD_BATCH_SIZE = 2000; + private static final String PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING_KEY = + "cluster.routing.allocation.shards_batch_gateway_allocator.primary_allocator_timeout"; + private static final String REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING_KEY = + "cluster.routing.allocation.shards_batch_gateway_allocator.replica_allocator_timeout"; + + private TimeValue primaryShardsBatchGatewayAllocatorTimeout; + private TimeValue replicaShardsBatchGatewayAllocatorTimeout; + /** * Number of shards we send in one batch to data nodes for fetching metadata */ @@ -79,6 +92,20 @@ public class ShardsBatchGatewayAllocator implements ExistingShardsAllocator { Setting.Property.NodeScope ); + public static final Setting PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING = Setting.timeSetting( + PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING_KEY, + TimeValue.MINUS_ONE, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + + public static final Setting REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING = Setting.timeSetting( + REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING_KEY, + TimeValue.MINUS_ONE, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + private final RerouteService rerouteService; private final PrimaryShardBatchAllocator primaryShardBatchAllocator; private final ReplicaShardBatchAllocator replicaShardBatchAllocator; @@ -97,7 +124,8 @@ public ShardsBatchGatewayAllocator( RerouteService rerouteService, TransportNodesListGatewayStartedShardsBatch batchStartedAction, TransportNodesListShardStoreMetadataBatch batchStoreAction, - Settings settings + Settings settings, + ClusterSettings clusterSettings ) { this.rerouteService = rerouteService; this.primaryShardBatchAllocator = new InternalPrimaryBatchShardAllocator(); @@ -105,6 +133,10 @@ public ShardsBatchGatewayAllocator( this.batchStartedAction = batchStartedAction; this.batchStoreAction = batchStoreAction; this.maxBatchSize = GATEWAY_ALLOCATOR_BATCH_SIZE.get(settings); + this.primaryShardsBatchGatewayAllocatorTimeout = PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING.get(settings); + clusterSettings.addSettingsUpdateConsumer(PRIMARY_BATCH_ALLOCATOR_TIMEOUT_SETTING, this::setPrimaryBatchAllocatorTimeout); + this.replicaShardsBatchGatewayAllocatorTimeout = REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING.get(settings); + clusterSettings.addSettingsUpdateConsumer(REPLICA_BATCH_ALLOCATOR_TIMEOUT_SETTING, this::setReplicaBatchAllocatorTimeout); } @Override @@ -127,7 +159,10 @@ protected ShardsBatchGatewayAllocator(long batchSize) { this.batchStoreAction = null; this.replicaShardBatchAllocator = null; this.maxBatchSize = batchSize; + this.primaryShardsBatchGatewayAllocatorTimeout = null; + this.replicaShardsBatchGatewayAllocatorTimeout = null; } + // for tests @Override @@ -187,14 +222,14 @@ public void allocateUnassigned( } @Override - public void allocateAllUnassignedShards(final RoutingAllocation allocation, boolean primary) { + public BatchRunnableExecutor allocateAllUnassignedShards(final RoutingAllocation allocation, boolean primary) { assert primaryShardBatchAllocator != null; assert replicaShardBatchAllocator != null; - innerAllocateUnassignedBatch(allocation, primaryShardBatchAllocator, replicaShardBatchAllocator, primary); + return innerAllocateUnassignedBatch(allocation, primaryShardBatchAllocator, replicaShardBatchAllocator, primary); } - protected void innerAllocateUnassignedBatch( + protected BatchRunnableExecutor innerAllocateUnassignedBatch( RoutingAllocation allocation, PrimaryShardBatchAllocator primaryBatchShardAllocator, ReplicaShardBatchAllocator replicaBatchShardAllocator, @@ -203,20 +238,45 @@ protected void innerAllocateUnassignedBatch( // create batches for unassigned shards Set batchesToAssign = createAndUpdateBatches(allocation, primary); if (batchesToAssign.isEmpty()) { - return; + return null; } + List runnables = new ArrayList<>(); if (primary) { batchIdToStartedShardBatch.values() .stream() .filter(batch -> batchesToAssign.contains(batch.batchId)) - .forEach( - shardsBatch -> primaryBatchShardAllocator.allocateUnassignedBatch(shardsBatch.getBatchedShardRoutings(), allocation) - ); + .forEach(shardsBatch -> runnables.add(new TimeoutAwareRunnable() { + @Override + public void onTimeout() { + primaryBatchShardAllocator.allocateUnassignedBatchOnTimeout( + shardsBatch.getBatchedShardRoutings(), + allocation, + true + ); + } + + @Override + public void run() { + primaryBatchShardAllocator.allocateUnassignedBatch(shardsBatch.getBatchedShardRoutings(), allocation); + } + })); + return new BatchRunnableExecutor(runnables, () -> primaryShardsBatchGatewayAllocatorTimeout); } else { batchIdToStoreShardBatch.values() .stream() .filter(batch -> batchesToAssign.contains(batch.batchId)) - .forEach(batch -> replicaBatchShardAllocator.allocateUnassignedBatch(batch.getBatchedShardRoutings(), allocation)); + .forEach(batch -> runnables.add(new TimeoutAwareRunnable() { + @Override + public void onTimeout() { + replicaBatchShardAllocator.allocateUnassignedBatchOnTimeout(batch.getBatchedShardRoutings(), allocation, false); + } + + @Override + public void run() { + replicaBatchShardAllocator.allocateUnassignedBatch(batch.getBatchedShardRoutings(), allocation); + } + })); + return new BatchRunnableExecutor(runnables, () -> replicaShardsBatchGatewayAllocatorTimeout); } } @@ -721,4 +781,12 @@ public int getNumberOfStartedShardBatches() { public int getNumberOfStoreShardBatches() { return batchIdToStoreShardBatch.size(); } + + private void setPrimaryBatchAllocatorTimeout(TimeValue primaryShardsBatchGatewayAllocatorTimeout) { + this.primaryShardsBatchGatewayAllocatorTimeout = primaryShardsBatchGatewayAllocatorTimeout; + } + + private void setReplicaBatchAllocatorTimeout(TimeValue replicaShardsBatchGatewayAllocatorTimeout) { + this.replicaShardsBatchGatewayAllocatorTimeout = replicaShardsBatchGatewayAllocatorTimeout; + } } diff --git a/server/src/test/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocatorTests.java b/server/src/test/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocatorTests.java new file mode 100644 index 0000000000000..1da8f5ef7f695 --- /dev/null +++ b/server/src/test/java/org/opensearch/cluster/routing/allocation/ExistingShardsAllocatorTests.java @@ -0,0 +1,118 @@ +/* + * 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.routing.allocation; + +import org.opensearch.Version; +import org.opensearch.cluster.ClusterName; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.OpenSearchAllocationTestCase; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.common.settings.Settings; + +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +public class ExistingShardsAllocatorTests extends OpenSearchAllocationTestCase { + + public void testRunnablesExecutedForUnassignedShards() throws InterruptedException { + + Metadata metadata = Metadata.builder() + .put(IndexMetadata.builder("test").settings(settings(Version.CURRENT)).numberOfShards(3).numberOfReplicas(2)) + .build(); + RoutingTable initialRoutingTable = RoutingTable.builder().addAsNew(metadata.index("test")).build(); + + ClusterState clusterState = ClusterState.builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(initialRoutingTable) + .build(); + clusterState = ClusterState.builder(clusterState) + .nodes(DiscoveryNodes.builder().add(newNode("node1")).add(newNode("node2")).add(newNode("node3"))) + .build(); + RoutingAllocation allocation = new RoutingAllocation( + yesAllocationDeciders(), + clusterState.getRoutingNodes(), + clusterState, + null, + null, + 0L + ); + CountDownLatch expectedStateLatch = new CountDownLatch(3); + TestAllocator testAllocator = new TestAllocator(expectedStateLatch); + testAllocator.allocateAllUnassignedShards(allocation, true).run(); + // if the below condition is passed, then we are sure runnable executed for all primary shards + assertTrue(expectedStateLatch.await(30, TimeUnit.SECONDS)); + + expectedStateLatch = new CountDownLatch(6); + testAllocator = new TestAllocator(expectedStateLatch); + testAllocator.allocateAllUnassignedShards(allocation, false).run(); + // if the below condition is passed, then we are sure runnable executed for all replica shards + assertTrue(expectedStateLatch.await(30, TimeUnit.SECONDS)); + } + + private static class TestAllocator implements ExistingShardsAllocator { + + final CountDownLatch countDownLatch; + + TestAllocator(CountDownLatch latch) { + this.countDownLatch = latch; + } + + @Override + public void beforeAllocation(RoutingAllocation allocation) { + + } + + @Override + public void afterPrimariesBeforeReplicas(RoutingAllocation allocation) { + + } + + @Override + public void allocateUnassigned( + ShardRouting shardRouting, + RoutingAllocation allocation, + UnassignedAllocationHandler unassignedAllocationHandler + ) { + countDownLatch.countDown(); + } + + @Override + public AllocateUnassignedDecision explainUnassignedShardAllocation( + ShardRouting unassignedShard, + RoutingAllocation routingAllocation + ) { + return null; + } + + @Override + public void cleanCaches() { + + } + + @Override + public void applyStartedShards(List startedShards, RoutingAllocation allocation) { + + } + + @Override + public void applyFailedShards(List failedShards, RoutingAllocation allocation) { + + } + + @Override + public int getNumberOfInFlightFetches() { + return 0; + } + } +} diff --git a/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTests.java b/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTests.java new file mode 100644 index 0000000000000..269f89faec54d --- /dev/null +++ b/server/src/test/java/org/opensearch/common/util/BatchRunnableExecutorTests.java @@ -0,0 +1,97 @@ +/* + * 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.util; + +import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.concurrent.TimeoutAwareRunnable; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.function.Supplier; + +import static org.mockito.Mockito.atMost; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +public class BatchRunnableExecutorTests extends OpenSearchTestCase { + private Supplier timeoutSupplier; + private TimeoutAwareRunnable runnable1; + private TimeoutAwareRunnable runnable2; + private TimeoutAwareRunnable runnable3; + private List runnableList; + + public void setupRunnables() { + timeoutSupplier = mock(Supplier.class); + runnable1 = mock(TimeoutAwareRunnable.class); + runnable2 = mock(TimeoutAwareRunnable.class); + runnable3 = mock(TimeoutAwareRunnable.class); + runnableList = Arrays.asList(runnable1, runnable2, runnable3); + } + + public void testRunWithoutTimeout() { + setupRunnables(); + timeoutSupplier = () -> TimeValue.timeValueSeconds(1); + BatchRunnableExecutor executor = new BatchRunnableExecutor(runnableList, timeoutSupplier); + executor.run(); + verify(runnable1, times(1)).run(); + verify(runnable2, times(1)).run(); + verify(runnable3, times(1)).run(); + verify(runnable1, never()).onTimeout(); + verify(runnable2, never()).onTimeout(); + verify(runnable3, never()).onTimeout(); + } + + public void testRunWithTimeout() { + setupRunnables(); + timeoutSupplier = () -> TimeValue.timeValueNanos(1); + BatchRunnableExecutor executor = new BatchRunnableExecutor(runnableList, timeoutSupplier); + executor.run(); + verify(runnable1, times(1)).onTimeout(); + verify(runnable2, times(1)).onTimeout(); + verify(runnable3, times(1)).onTimeout(); + verify(runnable1, never()).run(); + verify(runnable2, never()).run(); + verify(runnable3, never()).run(); + } + + public void testRunWithPartialTimeout() { + setupRunnables(); + timeoutSupplier = () -> TimeValue.timeValueMillis(50); + BatchRunnableExecutor executor = new BatchRunnableExecutor(runnableList, timeoutSupplier); + doAnswer(invocation -> { + Thread.sleep(100); + return null; + }).when(runnable1).run(); + executor.run(); + verify(runnable1, atMost(1)).run(); + verify(runnable2, atMost(1)).run(); + verify(runnable3, atMost(1)).run(); + verify(runnable2, atMost(1)).onTimeout(); + verify(runnable3, atMost(1)).onTimeout(); + verify(runnable2, atMost(1)).onTimeout(); + verify(runnable3, atMost(1)).onTimeout(); + } + + public void testRunWithEmptyRunnableList() { + setupRunnables(); + BatchRunnableExecutor executor = new BatchRunnableExecutor(Collections.emptyList(), timeoutSupplier); + executor.run(); + verify(runnable1, never()).onTimeout(); + verify(runnable2, never()).onTimeout(); + verify(runnable3, never()).onTimeout(); + verify(runnable1, never()).run(); + verify(runnable2, never()).run(); + verify(runnable3, never()).run(); + } +} diff --git a/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java index aa31c710c1fbd..bd56123f6df1f 100644 --- a/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/GatewayAllocatorTests.java @@ -32,6 +32,7 @@ import org.opensearch.cluster.routing.allocation.decider.AllocationDeciders; import org.opensearch.common.collect.Tuple; import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.BatchRunnableExecutor; import org.opensearch.common.util.set.Sets; import org.opensearch.core.index.shard.ShardId; import org.opensearch.snapshots.SnapshotShardSizeInfo; @@ -61,6 +62,13 @@ public void setUp() throws Exception { testShardsBatchGatewayAllocator = new TestShardBatchGatewayAllocator(); } + public void testExecutorNotNull() { + createIndexAndUpdateClusterState(1, 3, 1); + createBatchesAndAssert(1); + BatchRunnableExecutor executor = testShardsBatchGatewayAllocator.allocateAllUnassignedShards(testAllocation, true); + assertNotNull(executor); + } + public void testSingleBatchCreation() { createIndexAndUpdateClusterState(1, 3, 1); createBatchesAndAssert(1); @@ -336,6 +344,30 @@ public void testGetBatchIdNonExisting() { allShardRoutings.forEach(shard -> assertNull(testShardsBatchGatewayAllocator.getBatchId(shard, shard.primary()))); } + public void testCreatePrimaryAndReplicaExecutorOfSizeOne() { + createIndexAndUpdateClusterState(1, 3, 2); + BatchRunnableExecutor executor = testShardsBatchGatewayAllocator.allocateAllUnassignedShards(testAllocation, true); + assertEquals(executor.getTimeoutAwareRunnables().size(), 1); + executor = testShardsBatchGatewayAllocator.allocateAllUnassignedShards(testAllocation, false); + assertEquals(executor.getTimeoutAwareRunnables().size(), 1); + } + + public void testCreatePrimaryExecutorOfSizeOneAndReplicaExecutorOfSizeZero() { + createIndexAndUpdateClusterState(1, 3, 0); + BatchRunnableExecutor executor = testShardsBatchGatewayAllocator.allocateAllUnassignedShards(testAllocation, true); + assertEquals(executor.getTimeoutAwareRunnables().size(), 1); + executor = testShardsBatchGatewayAllocator.allocateAllUnassignedShards(testAllocation, false); + assertNull(executor); + } + + public void testCreatePrimaryAndReplicaExecutorOfSizeTwo() { + createIndexAndUpdateClusterState(2, 1001, 1); + BatchRunnableExecutor executor = testShardsBatchGatewayAllocator.allocateAllUnassignedShards(testAllocation, true); + assertEquals(executor.getTimeoutAwareRunnables().size(), 2); + executor = testShardsBatchGatewayAllocator.allocateAllUnassignedShards(testAllocation, false); + assertEquals(executor.getTimeoutAwareRunnables().size(), 2); + } + private void createIndexAndUpdateClusterState(int count, int numberOfShards, int numberOfReplicas) { if (count == 0) return; Metadata.Builder metadata = Metadata.builder(); diff --git a/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java index 8ad8bcda95f40..270cf465d0f80 100644 --- a/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/PrimaryShardBatchAllocatorTests.java @@ -41,6 +41,7 @@ import org.junit.Before; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -256,6 +257,52 @@ public void testAllocateUnassignedBatchThrottlingAllocationDeciderIsHonoured() { assertEquals(UnassignedInfo.AllocationStatus.DECIDERS_THROTTLED, ignoredShards.get(0).unassignedInfo().getLastAllocationStatus()); } + public void testAllocateUnassignedBatchOnTimeoutWithMatchingPrimaryShards() { + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + AllocationDeciders allocationDeciders = randomAllocationDeciders(Settings.builder().build(), clusterSettings, random()); + setUpShards(1); + final RoutingAllocation routingAllocation = routingAllocationWithOnePrimary(allocationDeciders, CLUSTER_RECOVERED, "allocId-0"); + ShardRouting shardRouting = routingAllocation.routingTable().getIndicesRouting().get("test").shard(shardId.id()).primaryShard(); + + List shardRoutings = Arrays.asList(shardRouting); + batchAllocator.allocateUnassignedBatchOnTimeout(shardRoutings, routingAllocation, true); + + List ignoredShards = routingAllocation.routingNodes().unassigned().ignored(); + assertEquals(1, ignoredShards.size()); + assertEquals(UnassignedInfo.AllocationStatus.DECIDERS_THROTTLED, ignoredShards.get(0).unassignedInfo().getLastAllocationStatus()); + } + + public void testAllocateUnassignedBatchOnTimeoutWithNoMatchingPrimaryShards() { + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + AllocationDeciders allocationDeciders = randomAllocationDeciders(Settings.builder().build(), clusterSettings, random()); + setUpShards(1); + final RoutingAllocation routingAllocation = routingAllocationWithOnePrimary(allocationDeciders, CLUSTER_RECOVERED, "allocId-0"); + List shardRoutings = new ArrayList<>(); + batchAllocator.allocateUnassignedBatchOnTimeout(shardRoutings, routingAllocation, true); + + List ignoredShards = routingAllocation.routingNodes().unassigned().ignored(); + assertEquals(0, ignoredShards.size()); + } + + public void testAllocateUnassignedBatchOnTimeoutWithNonPrimaryShards() { + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + AllocationDeciders allocationDeciders = randomAllocationDeciders(Settings.builder().build(), clusterSettings, random()); + setUpShards(1); + final RoutingAllocation routingAllocation = routingAllocationWithOnePrimary(allocationDeciders, CLUSTER_RECOVERED, "allocId-0"); + + ShardRouting shardRouting = routingAllocation.routingTable() + .getIndicesRouting() + .get("test") + .shard(shardId.id()) + .replicaShards() + .get(0); + List shardRoutings = Arrays.asList(shardRouting); + batchAllocator.allocateUnassignedBatchOnTimeout(shardRoutings, routingAllocation, false); + + List ignoredShards = routingAllocation.routingNodes().unassigned().ignored(); + assertEquals(1, ignoredShards.size()); + } + private RoutingAllocation routingAllocationWithOnePrimary( AllocationDeciders deciders, UnassignedInfo.Reason reason, diff --git a/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java b/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java index 526a3990955b8..435fd78be2bcd 100644 --- a/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java +++ b/server/src/test/java/org/opensearch/gateway/ReplicaShardBatchAllocatorTests.java @@ -717,6 +717,33 @@ public void testAllocateUnassignedBatchThrottlingAllocationDeciderIsHonoured() t assertEquals(UnassignedInfo.AllocationStatus.DECIDERS_THROTTLED, allocateUnassignedDecision.getAllocationStatus()); } + public void testAllocateUnassignedBatchOnTimeoutWithUnassignedReplicaShard() { + RoutingAllocation allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders()); + final RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); + List shards = new ArrayList<>(); + while (iterator.hasNext()) { + shards.add(iterator.next()); + } + testBatchAllocator.allocateUnassignedBatchOnTimeout(shards, allocation, false); + assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); + assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); + assertEquals( + UnassignedInfo.AllocationStatus.NO_ATTEMPT, + allocation.routingNodes().unassigned().ignored().get(0).unassignedInfo().getLastAllocationStatus() + ); + } + + public void testAllocateUnassignedBatchOnTimeoutWithAlreadyRecoveringReplicaShard() { + RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders()); + final RoutingNodes.UnassignedShards.UnassignedIterator iterator = allocation.routingNodes().unassigned().iterator(); + List shards = new ArrayList<>(); + while (iterator.hasNext()) { + shards.add(iterator.next()); + } + testBatchAllocator.allocateUnassignedBatchOnTimeout(shards, allocation, false); + assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(0)); + } + private RoutingAllocation onePrimaryOnNode1And1Replica(AllocationDeciders deciders) { return onePrimaryOnNode1And1Replica(deciders, Settings.EMPTY, UnassignedInfo.Reason.CLUSTER_RECOVERED); } diff --git a/test/framework/src/main/java/org/opensearch/test/gateway/TestShardBatchGatewayAllocator.java b/test/framework/src/main/java/org/opensearch/test/gateway/TestShardBatchGatewayAllocator.java index fbb39c284f0ff..0eb4bb6935bac 100644 --- a/test/framework/src/main/java/org/opensearch/test/gateway/TestShardBatchGatewayAllocator.java +++ b/test/framework/src/main/java/org/opensearch/test/gateway/TestShardBatchGatewayAllocator.java @@ -13,6 +13,7 @@ import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.allocation.AllocateUnassignedDecision; import org.opensearch.cluster.routing.allocation.RoutingAllocation; +import org.opensearch.common.util.BatchRunnableExecutor; import org.opensearch.core.index.shard.ShardId; import org.opensearch.gateway.AsyncShardFetch; import org.opensearch.gateway.PrimaryShardBatchAllocator; @@ -102,9 +103,9 @@ protected boolean hasInitiatedFetching(ShardRouting shard) { }; @Override - public void allocateAllUnassignedShards(RoutingAllocation allocation, boolean primary) { + public BatchRunnableExecutor allocateAllUnassignedShards(RoutingAllocation allocation, boolean primary) { currentNodes = allocation.nodes(); - innerAllocateUnassignedBatch(allocation, primaryBatchShardAllocator, replicaBatchShardAllocator, primary); + return innerAllocateUnassignedBatch(allocation, primaryBatchShardAllocator, replicaBatchShardAllocator, primary); } @Override From 1fe58b5d712cfef525abfbd2dfaf398c0368745f Mon Sep 17 00:00:00 2001 From: gaobinlong Date: Wed, 24 Jul 2024 19:54:04 +0800 Subject: [PATCH 37/37] Fix the documentation url of the Create or Update alias API in rest-api-spec (#14935) Signed-off-by: Gao Binlong --- .../src/main/resources/rest-api-spec/api/indices.put_alias.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_alias.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_alias.json index d99edcf5513f9..14427b00f1bb3 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_alias.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_alias.json @@ -1,7 +1,7 @@ { "indices.put_alias":{ "documentation":{ - "url":"https://opensearch.org/docs/latest/api-reference/index-apis/alias/", + "url":"https://opensearch.org/docs/latest/api-reference/index-apis/update-alias/", "description":"Creates or updates an alias." }, "stability":"stable",