diff --git a/CHANGELOG.md b/CHANGELOG.md index 976017cae8da0..0d4fcf94e9e58 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/), - Support AutoExpand for SearchReplica ([#17741](https://github.com/opensearch-project/OpenSearch/pull/17741)) - Implement fixed interval refresh task scheduling ([#17777](https://github.com/opensearch-project/OpenSearch/pull/17777)) - Add GRPC DocumentService and Bulk endpoint ([#17727](https://github.com/opensearch-project/OpenSearch/pull/17727)) +- Added scale to zero (`search_only` mode) support for OpenSearch reader writer separation ([#17299](https://github.com/opensearch-project/OpenSearch/pull/17299) ### Changed - Migrate BC libs to their FIPS counterparts ([#14912](https://github.com/opensearch-project/OpenSearch/pull/14912)) diff --git a/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexIT.java b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexIT.java new file mode 100644 index 0000000000000..3e22084db96d8 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexIT.java @@ -0,0 +1,366 @@ +/* + * 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.scale.searchonly; + +import org.opensearch.action.admin.indices.settings.get.GetSettingsResponse; +import org.opensearch.action.index.IndexResponse; +import org.opensearch.action.search.SearchResponse; +import org.opensearch.action.support.WriteRequest; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.IndexShardRoutingTable; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.FeatureFlags; +import org.opensearch.core.rest.RestStatus; +import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.remotestore.RemoteStoreBaseIntegTestCase; +import org.opensearch.test.InternalTestCluster; +import org.opensearch.test.OpenSearchIntegTestCase; + +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SEARCH_REPLICAS; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount; + +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) +public class ScaleIndexIT extends RemoteStoreBaseIntegTestCase { + + private static final String TEST_INDEX = "test_scale_index"; + + @Override + protected Settings featureFlagSettings() { + return Settings.builder().put(super.featureFlagSettings()).put(FeatureFlags.READER_WRITER_SPLIT_EXPERIMENTAL, Boolean.TRUE).build(); + } + + public Settings indexSettings() { + return Settings.builder().put(SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT).build(); + } + + /** + * Tests the full lifecycle of scaling an index down to search-only mode, + * scaling search replicas while in search-only mode, verifying cluster health in + * various states, and then scaling back up to normal mode. + */ + public void testFullSearchOnlyReplicasFullLifecycle() throws Exception { + internalCluster().startClusterManagerOnlyNode(); + internalCluster().startDataOnlyNodes(2); + internalCluster().startSearchOnlyNodes(3); + + Settings specificSettings = Settings.builder() + .put(indexSettings()) + .put(SETTING_NUMBER_OF_SHARDS, 1) + .put(SETTING_NUMBER_OF_REPLICAS, 1) + .put(SETTING_NUMBER_OF_SEARCH_REPLICAS, 1) + .build(); + + createIndex(TEST_INDEX, specificSettings); + ensureGreen(TEST_INDEX); + + for (int i = 0; i < 10; i++) { + IndexResponse indexResponse = client().prepareIndex(TEST_INDEX) + .setId(Integer.toString(i)) + .setSource("field1", "value" + i) + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) + .get(); + assertEquals(RestStatus.CREATED, indexResponse.status()); + } + + assertBusy(() -> { + SearchResponse searchResponse = client().prepareSearch(TEST_INDEX).get(); + assertHitCount(searchResponse, 10); + assertSearchNodeDocCounts(10, TEST_INDEX); + }, 30, TimeUnit.SECONDS); + + ensureGreen(TEST_INDEX); + + // Scale down to search-only mode + assertAcked(client().admin().indices().prepareScaleSearchOnly(TEST_INDEX, true).get()); + + // Verify search-only setting is enabled + GetSettingsResponse settingsResponse = client().admin().indices().prepareGetSettings(TEST_INDEX).get(); + assertTrue(settingsResponse.getSetting(TEST_INDEX, IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey()).equals("true")); + + // Verify that write operations are blocked during scale-down + assertBusy(() -> { + try { + // Attempt to index a document while scale-down is in progress + client().prepareIndex(TEST_INDEX) + .setId("sample-write-after-search-only-block") + .setSource("field1", "value1") + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) + .get(); + fail("Write operation should be blocked during scale-down"); + } catch (Exception e) { + assertTrue( + "Exception should indicate index scaled down", + e.getMessage().contains("blocked by: [FORBIDDEN/20/index scaled down]") + ); + } + }, 10, TimeUnit.SECONDS); + + ensureGreen(TEST_INDEX); + + // Verify search still works on all search nodes + assertSearchNodeDocCounts(10, TEST_INDEX); + + // Scale up search replicas while in search-only mode + assertAcked( + client().admin() + .indices() + .prepareUpdateSettings(TEST_INDEX) + .setSettings(Settings.builder().put(SETTING_NUMBER_OF_SEARCH_REPLICAS, 3).build()) + .get() + ); + + ensureGreen(TEST_INDEX); + + // Verify search still works on all search nodes + assertBusy(() -> { assertSearchNodeDocCounts(10, TEST_INDEX); }, 30, TimeUnit.SECONDS); + + // Scale down search replicas while still in search-only mode + assertAcked( + client().admin() + .indices() + .prepareUpdateSettings(TEST_INDEX) + .setSettings(Settings.builder().put(SETTING_NUMBER_OF_SEARCH_REPLICAS, 2).build()) + .get() + ); + + ensureGreen(TEST_INDEX); + + // Verify search still works on all search nodes + assertBusy(() -> { assertSearchNodeDocCounts(10, TEST_INDEX); }, 30, TimeUnit.SECONDS); + + // Test cluster health when one search replica is down + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(findNodesWithSearchOnlyReplicas()[0])); + + assertEquals( + "Index health should be YELLOW with one search replica down", + "YELLOW", + client().admin().cluster().prepareHealth(TEST_INDEX).get().getStatus().name() + ); + + // Start a replacement search node and wait for recovery + internalCluster().startSearchOnlyNode(); + ensureGreen(TEST_INDEX); + + // Scale back up to normal mode + assertAcked(client().admin().indices().prepareScaleSearchOnly(TEST_INDEX, false).get()); + ensureGreen(TEST_INDEX); + + // Verify search-only setting is disabled + settingsResponse = client().admin().indices().prepareGetSettings(TEST_INDEX).get(); + assertFalse(settingsResponse.getSetting(TEST_INDEX, IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey()).equals("true")); + + // Verify search still works after scale-up + assertBusy(() -> { + SearchResponse response = client().prepareSearch(TEST_INDEX).get(); + assertHitCount(response, 10); + assertSearchNodeDocCounts(10, TEST_INDEX); + }, 30, TimeUnit.SECONDS); + + // Verify writes work again after scale-up + IndexResponse indexResponse = client().prepareIndex(TEST_INDEX) + .setId("new-doc") + .setSource("field1", "new-value") + .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) + .get(); + assertEquals(RestStatus.CREATED, indexResponse.status()); + + // Verify new document is searchable + assertBusy(() -> { + SearchResponse finalResponse = client().prepareSearch(TEST_INDEX).get(); + assertHitCount(finalResponse, 11); + assertSearchNodeDocCounts(11, TEST_INDEX); + }); + } + + /** + * Tests scaling down an index to search-only mode when there are no search replicas. + */ + public void testScaleDownValidationWithoutSearchReplicas() { + internalCluster().startClusterManagerOnlyNode(); + internalCluster().startDataOnlyNodes(2); + internalCluster().startSearchOnlyNode(); + + Settings specificSettings = Settings.builder() + .put(indexSettings()) + .put(SETTING_NUMBER_OF_SHARDS, 2) + .put(SETTING_NUMBER_OF_REPLICAS, 1) + .build(); + + createIndex(TEST_INDEX, specificSettings); + ensureYellow(TEST_INDEX); + + IllegalArgumentException exception = expectThrows( + IllegalArgumentException.class, + () -> client().admin().indices().prepareScaleSearchOnly(TEST_INDEX, true).get() + ); + + assertTrue( + "Expected error about missing search replicas", + exception.getMessage().contains("Cannot scale to zero without search replicas for index:") + ); + } + + /** + * Scenario 1: Tests search-only replicas recovery with persistent data directory + * and cluster.remote_store.state.enabled=false + */ + public void testSearchOnlyRecoveryWithPersistentData() throws Exception { + internalCluster().startClusterManagerOnlyNode(); + internalCluster().startDataOnlyNodes(2); + internalCluster().startSearchOnlyNode(); + + Settings specificSettings = Settings.builder() + .put(indexSettings()) + .put(SETTING_NUMBER_OF_SHARDS, 2) + .put(SETTING_NUMBER_OF_REPLICAS, 1) + .put(SETTING_NUMBER_OF_SEARCH_REPLICAS, 1) + .build(); + + createIndex(TEST_INDEX, specificSettings); + ensureGreen(TEST_INDEX); + + assertAcked(client().admin().indices().prepareScaleSearchOnly(TEST_INDEX, true).get()); + + assertBusy(() -> { + ClusterState state = client().admin().cluster().prepareState().get().getState(); + IndexRoutingTable routingTable = state.routingTable().index(TEST_INDEX); + + for (IndexShardRoutingTable shardTable : routingTable) { + assertNull("Primary should be null", shardTable.primaryShard()); + assertTrue("No writer replicas should exist", shardTable.writerReplicas().isEmpty()); + assertEquals( + "One search replica should be active", + 1, + shardTable.searchOnlyReplicas().stream().filter(ShardRouting::active).count() + ); + } + }); + } + + /** + * Scenario 2: Tests behavior with cluster.remote_store.state.enabled=true + * but without data directory preservation + */ + public void testClusterRemoteStoreStateEnabled() throws Exception { + Settings remoteStoreSettings = Settings.builder().put(nodeSettings(0)).put("cluster.remote_store.state.enabled", true).build(); + + internalCluster().startClusterManagerOnlyNode(remoteStoreSettings); + internalCluster().startDataOnlyNodes(2); + internalCluster().startSearchOnlyNode(); + + Settings specificSettings = Settings.builder() + .put(indexSettings()) + .put(SETTING_NUMBER_OF_SHARDS, 2) + .put(SETTING_NUMBER_OF_REPLICAS, 1) + .put(SETTING_NUMBER_OF_SEARCH_REPLICAS, 1) + .build(); + + createIndex(TEST_INDEX, specificSettings); + ensureGreen(TEST_INDEX); + + assertAcked(client().admin().indices().prepareScaleSearchOnly(TEST_INDEX, true).get()); + + internalCluster().stopAllNodes(); + + internalCluster().startClusterManagerOnlyNode(remoteStoreSettings); + internalCluster().startDataOnlyNodes(2); + internalCluster().startSearchOnlyNode(); + + assertBusy(() -> { + ClusterState state = client().admin().cluster().prepareState().get().getState(); + IndexRoutingTable routingTable = state.routingTable().index(TEST_INDEX); + + for (IndexShardRoutingTable shardTable : routingTable) { + assertTrue( + "Only search replicas should be active", + shardTable.searchOnlyReplicas().stream().anyMatch(ShardRouting::active) + ); + } + }); + } + + /** + * Scenario 3: Tests recovery with persistent data directory and remote store state + */ + public void testRecoveryWithPersistentDataAndRemoteStore() throws Exception { + Settings remoteStoreSettings = Settings.builder().put(nodeSettings(0)).put("cluster.remote_store.state.enabled", true).build(); + + internalCluster().startClusterManagerOnlyNode(remoteStoreSettings); + internalCluster().startDataOnlyNodes(2); + internalCluster().startSearchOnlyNode(); + + Settings specificSettings = Settings.builder() + .put(indexSettings()) + .put(SETTING_NUMBER_OF_SHARDS, 2) + .put(SETTING_NUMBER_OF_REPLICAS, 1) + .put(SETTING_NUMBER_OF_SEARCH_REPLICAS, 1) + .build(); + + createIndex(TEST_INDEX, specificSettings); + ensureGreen(TEST_INDEX); + + internalCluster().fullRestart(); + + ensureGreen(TEST_INDEX); + assertAcked(client().admin().indices().prepareScaleSearchOnly(TEST_INDEX, true).get()); + + assertBusy(() -> { assertEquals("One search replica should be active", 1, findNodesWithSearchOnlyReplicas().length); }); + } + + /** + * Helper method to find all nodes that contain active search-only replica shards + * @return Array of node names that have active search-only replicas + */ + private String[] findNodesWithSearchOnlyReplicas() { + ClusterState state = client().admin().cluster().prepareState().get().getState(); + IndexRoutingTable indexRoutingTable = state.routingTable().index(TEST_INDEX); + + // Use a set to avoid duplicates if multiple shards are on the same node + Set nodeNames = new HashSet<>(); + + for (IndexShardRoutingTable shardTable : indexRoutingTable) { + for (ShardRouting searchReplica : shardTable.searchOnlyReplicas()) { + if (searchReplica.active()) { + String nodeId = searchReplica.currentNodeId(); + nodeNames.add(state.nodes().get(nodeId).getName()); + } + } + } + + if (nodeNames.isEmpty()) { + throw new AssertionError("Could not find any node with active search-only replica"); + } + + return nodeNames.toArray(new String[0]); + } + + /** + * Assert that documents are accessible and have the expected count across all search nodes + * @param expectedDocCount Expected number of documents in the index + * @param index The index name to search + */ + protected void assertSearchNodeDocCounts(int expectedDocCount, String index) { + // Check on all nodes that have search-only replicas + String[] searchNodes = findNodesWithSearchOnlyReplicas(); + for (String node : searchNodes) { + assertHitCount(client(node).prepareSearch(index).setSize(0).get(), expectedDocCount); + } + } +} diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SearchReplicaRestoreIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SearchReplicaRestoreIT.java index af911162d4458..b544a8b602da6 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SearchReplicaRestoreIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SearchReplicaRestoreIT.java @@ -8,7 +8,11 @@ package org.opensearch.indices.replication; +import org.opensearch.action.admin.cluster.remotestore.restore.RestoreRemoteStoreRequest; +import org.opensearch.action.admin.cluster.remotestore.restore.RestoreRemoteStoreResponse; +import org.opensearch.action.admin.indices.settings.get.GetSettingsResponse; import org.opensearch.action.search.SearchResponse; +import org.opensearch.action.support.PlainActionFuture; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.common.settings.Settings; @@ -99,6 +103,24 @@ public void testSearchReplicaRestore_WhenSnapshotOnSegRepWithSearchReplica_Resto assertTrue(exception.getMessage().contains(getSnapshotExceptionMessage(ReplicationType.SEGMENT, ReplicationType.DOCUMENT))); } + public void testRemoteStoreRestoreFailsForSearchOnlyIndex() throws Exception { + bootstrapIndexWithSearchReplicas(); + assertAcked(client().admin().indices().prepareScaleSearchOnly(INDEX_NAME, true).get()); + + GetSettingsResponse settingsResponse = client().admin().indices().prepareGetSettings(INDEX_NAME).get(); + assertEquals("true", settingsResponse.getSetting(INDEX_NAME, IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey())); + + IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> { + PlainActionFuture future = PlainActionFuture.newFuture(); + client().admin().cluster().restoreRemoteStore(new RestoreRemoteStoreRequest().indices(INDEX_NAME), future); + future.actionGet(); + }); + + assertTrue( + exception.getMessage().contains("Cannot use _remotestore/_restore on search_only mode enabled index [" + INDEX_NAME + "].") + ); + } + private void bootstrapIndexWithOutSearchReplicas(ReplicationType replicationType) throws InterruptedException { internalCluster().startNodes(2); diff --git a/server/src/main/java/org/opensearch/action/ActionModule.java b/server/src/main/java/org/opensearch/action/ActionModule.java index f1cc400d1a4f8..fa504466e0a4a 100644 --- a/server/src/main/java/org/opensearch/action/ActionModule.java +++ b/server/src/main/java/org/opensearch/action/ActionModule.java @@ -184,6 +184,8 @@ import org.opensearch.action.admin.indices.resolve.ResolveIndexAction; import org.opensearch.action.admin.indices.rollover.RolloverAction; import org.opensearch.action.admin.indices.rollover.TransportRolloverAction; +import org.opensearch.action.admin.indices.scale.searchonly.ScaleIndexAction; +import org.opensearch.action.admin.indices.scale.searchonly.TransportScaleIndexAction; import org.opensearch.action.admin.indices.segments.IndicesSegmentsAction; import org.opensearch.action.admin.indices.segments.PitSegmentsAction; import org.opensearch.action.admin.indices.segments.TransportIndicesSegmentsAction; @@ -430,6 +432,7 @@ import org.opensearch.rest.action.admin.indices.RestResolveIndexAction; import org.opensearch.rest.action.admin.indices.RestResumeIngestionAction; import org.opensearch.rest.action.admin.indices.RestRolloverIndexAction; +import org.opensearch.rest.action.admin.indices.RestScaleIndexAction; import org.opensearch.rest.action.admin.indices.RestSimulateIndexTemplateAction; import org.opensearch.rest.action.admin.indices.RestSimulateTemplateAction; import org.opensearch.rest.action.admin.indices.RestSyncedFlushAction; @@ -696,6 +699,7 @@ public void reg actions.register(AutoPutMappingAction.INSTANCE, TransportAutoPutMappingAction.class); actions.register(IndicesAliasesAction.INSTANCE, TransportIndicesAliasesAction.class); actions.register(UpdateSettingsAction.INSTANCE, TransportUpdateSettingsAction.class); + actions.register(ScaleIndexAction.INSTANCE, TransportScaleIndexAction.class); actions.register(AnalyzeAction.INSTANCE, TransportAnalyzeAction.class); actions.register(PutIndexTemplateAction.INSTANCE, TransportPutIndexTemplateAction.class); actions.register(GetIndexTemplatesAction.INSTANCE, TransportGetIndexTemplatesAction.class); @@ -923,7 +927,7 @@ public void initRestHandlers(Supplier nodesInCluster) { registerHandler.accept(new RestUpgradeAction()); registerHandler.accept(new RestUpgradeStatusAction()); registerHandler.accept(new RestClearIndicesCacheAction()); - + registerHandler.accept(new RestScaleIndexAction()); registerHandler.accept(new RestIndexAction()); registerHandler.accept(new CreateHandler()); registerHandler.accept(new AutoIdHandler(nodesInCluster)); diff --git a/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexAction.java b/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexAction.java new file mode 100644 index 0000000000000..9b79dd7bba632 --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexAction.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.action.admin.indices.scale.searchonly; + +import org.opensearch.action.ActionType; +import org.opensearch.action.support.clustermanager.AcknowledgedResponse; + +/** + * Action type for search-only scale operations on indices. + *

+ * This action type represents the administrative operation to transition indices + * between normal operation and search-only mode. It provides the action name constant + * used for transport actions and request routing. The action produces an + * {@link AcknowledgedResponse} to indicate success or failure. + *

+ * The action is registered with the name "indices:admin/scale/search_only" in the + * transport action registry. + */ +public class ScaleIndexAction extends ActionType { + /** + * Singleton instance of the SearchOnlyScaleAction. + */ + public static final ScaleIndexAction INSTANCE = new ScaleIndexAction(); + + /** + * The name of this action, used for transport action registration and routing. + *

+ * This action name follows the OpenSearch convention of prefixing administrative + * index actions with "indices:admin/". + */ + public static final String NAME = "indices:admin/scale/search_only"; + + /** + * Private constructor to enforce singleton pattern. + *

+ * Initializes the action with the appropriate name and response reader. + */ + private ScaleIndexAction() { + super(NAME, AcknowledgedResponse::new); + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexClusterStateBuilder.java b/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexClusterStateBuilder.java new file mode 100644 index 0000000000000..e376e766e29fd --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexClusterStateBuilder.java @@ -0,0 +1,273 @@ +/* + * 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.scale.searchonly; + +import org.opensearch.Version; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.block.ClusterBlock; +import org.opensearch.cluster.block.ClusterBlocks; +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.RecoverySource; +import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.UnassignedInfo; +import org.opensearch.common.UUIDs; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.index.Index; +import org.opensearch.core.rest.RestStatus; +import org.opensearch.repositories.IndexId; + +import java.util.EnumSet; +import java.util.Map; +import java.util.UUID; + +import static org.opensearch.cluster.block.ClusterBlockLevel.WRITE; +import static org.opensearch.cluster.metadata.IndexMetadata.INDEX_SEARCH_ONLY_BLOCK; +import static org.opensearch.cluster.metadata.IndexMetadata.INDEX_SEARCH_ONLY_BLOCK_ID; + +/** + * Utility class responsible for constructing new cluster states during scale operations. + *

+ * This builder constructs cluster states for both scale-up and scale-down operations, handling + * the transition of indices between normal operation and search-only mode. It manages: + *

    + *
  • Adding temporary write blocks during scale-down preparation
  • + *
  • Updating index metadata and settings when finalizing scale operations
  • + *
  • Modifying routing tables to add/remove shards based on scaling direction
  • + *
+ *

+ * The cluster state modifications ensure proper synchronization of operations and maintain + * data integrity throughout the scaling process. + */ +class ScaleIndexClusterStateBuilder { + + /** + * Builds the new cluster state by adding a temporary scale-down block on the target index. + *

+ * This temporary block prevents writes to the index during the preparation phase of scaling down, + * allowing existing operations to complete before transitioning to search-only mode. + * + * @param currentState the current cluster state + * @param index the name of the index being scaled down + * @param blockedIndices map to store the association between indices and their scale blocks + * @return the modified cluster state with temporary write blocks applied + */ + ClusterState buildScaleDownState(ClusterState currentState, String index, Map blockedIndices) { + Metadata.Builder metadataBuilder = Metadata.builder(currentState.metadata()); + ClusterBlocks.Builder blocksBuilder = ClusterBlocks.builder().blocks(currentState.blocks()); + RoutingTable.Builder routingTableBuilder = RoutingTable.builder(currentState.routingTable()); + + IndexMetadata indexMetadata = currentState.metadata().index(index); + Index idx = indexMetadata.getIndex(); + ClusterBlock scaleBlock = createScaleDownBlock(); + + blocksBuilder.addIndexBlock(index, scaleBlock); + blockedIndices.put(idx, scaleBlock); + + return ClusterState.builder(currentState) + .metadata(metadataBuilder) + .blocks(blocksBuilder) + .routingTable(routingTableBuilder.build()) + .build(); + } + + /** + * Builds the final cluster state for completing a scale-down operation. + *

+ * This state modification: + *

    + *
  • Removes the temporary scale-down block
  • + *
  • Updates index metadata to mark it as search-only
  • + *
  • Adds a permanent search-only block
  • + *
  • Updates the routing table to maintain only search replicas
  • + *
+ * + * @param currentState the current cluster state + * @param index the name of the index being scaled down + * @return the modified cluster state with finalized search-only configuration + * @throws IllegalStateException if the specified index is not found + */ + ClusterState buildFinalScaleDownState(ClusterState currentState, String index) { + ClusterBlocks.Builder blocksBuilder = ClusterBlocks.builder().blocks(currentState.blocks()); + RoutingTable.Builder routingTableBuilder = RoutingTable.builder(currentState.routingTable()); + Metadata.Builder metadataBuilder = Metadata.builder(currentState.metadata()); + + IndexMetadata indexMetadata = currentState.metadata().index(index); + if (indexMetadata == null) { + throw new IllegalStateException("Index " + index + " not found"); + } + + blocksBuilder.removeIndexBlockWithId(index, INDEX_SEARCH_ONLY_BLOCK_ID); + + Settings updatedSettings = Settings.builder() + .put(indexMetadata.getSettings()) + .put(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), true) + .build(); + + metadataBuilder.put( + IndexMetadata.builder(indexMetadata).settings(updatedSettings).settingsVersion(indexMetadata.getSettingsVersion() + 1) + ); + + blocksBuilder.addIndexBlock(index, INDEX_SEARCH_ONLY_BLOCK); + + updateRoutingTableForScaleDown(routingTableBuilder, currentState, index); + + return ClusterState.builder(currentState) + .metadata(metadataBuilder) + .blocks(blocksBuilder) + .routingTable(routingTableBuilder.build()) + .build(); + } + + /** + * Updates the routing table for a scale-down operation, removing non-search-only shards. + *

+ * This method preserves only the search-only replica shards in the routing table, + * effectively removing primary shards and standard replicas from the allocation. + * + * @param routingTableBuilder the routing table builder to modify + * @param currentState the current cluster state + * @param index the name of the index being scaled down + */ + private void updateRoutingTableForScaleDown(RoutingTable.Builder routingTableBuilder, ClusterState currentState, String index) { + IndexRoutingTable indexRoutingTable = currentState.routingTable().index(index); + if (indexRoutingTable != null) { + IndexRoutingTable.Builder indexBuilder = new IndexRoutingTable.Builder(indexRoutingTable.getIndex()); + for (IndexShardRoutingTable shardTable : indexRoutingTable) { + IndexShardRoutingTable.Builder shardBuilder = new IndexShardRoutingTable.Builder(shardTable.shardId()); + for (ShardRouting shardRouting : shardTable) { + if (shardRouting.isSearchOnly()) { + shardBuilder.addShard(shardRouting); + } + } + indexBuilder.addIndexShard(shardBuilder.build()); + } + routingTableBuilder.add(indexBuilder.build()); + } + } + + /** + * Builds a new routing table for scaling up an index from search-only mode to normal operation. + *

+ * This method: + *

    + *
  • Preserves existing search-only replicas
  • + *
  • Creates new unassigned primary shards with remote store recovery source
  • + *
  • Creates new unassigned replica shards configured for peer recovery
  • + *
+ *

+ * The resulting routing table allows the cluster allocator to restore normal index operation + * by recovering shards from remote storage. + * + * @param currentState the current cluster state + * @param index the name of the index being scaled up + * @return the modified routing table with newly added primary and replica shards + */ + RoutingTable buildScaleUpRoutingTable(ClusterState currentState, String index) { + RoutingTable.Builder routingTableBuilder = RoutingTable.builder(currentState.routingTable()); + IndexRoutingTable indexRoutingTable = currentState.routingTable().index(index); + IndexMetadata indexMetadata = currentState.metadata().index(index); + + if (indexRoutingTable != null && indexMetadata != null) { + IndexRoutingTable.Builder indexBuilder = new IndexRoutingTable.Builder(indexRoutingTable.getIndex()); + + for (IndexShardRoutingTable shardTable : indexRoutingTable) { + indexBuilder.addIndexShard(buildShardTableForScaleUp(shardTable, indexMetadata)); + } + routingTableBuilder.add(indexBuilder.build()); + } + + return routingTableBuilder.build(); + } + + /** + * Builds a shard routing table for a scale-up operation. + *

+ * For each shard, this method: + *

    + *
  • Preserves all existing search-only replicas
  • + *
  • Creates a new unassigned primary shard configured to recover from remote store
  • + *
  • Creates a new unassigned replica shard configured to recover from peers
  • + *
+ * + * @param shardTable the current shard routing table + * @return a new shard routing table with both search replicas and newly added shards + */ + + private IndexShardRoutingTable buildShardTableForScaleUp(IndexShardRoutingTable shardTable, IndexMetadata indexMetadata) { + IndexShardRoutingTable.Builder shardBuilder = new IndexShardRoutingTable.Builder(shardTable.shardId()); + + // Keep existing search-only shards + for (ShardRouting shardRouting : shardTable) { + if (shardRouting.isSearchOnly()) { + shardBuilder.addShard(shardRouting); + } + } + + RecoverySource.RemoteStoreRecoverySource remoteStoreRecoverySource = new RecoverySource.RemoteStoreRecoverySource( + UUID.randomUUID().toString(), + Version.CURRENT, + new IndexId(shardTable.shardId().getIndex().getName(), shardTable.shardId().getIndex().getUUID()) + ); + + // Get replica settings from index metadata + int numberOfReplicas = indexMetadata.getNumberOfReplicas(); + + // Create primary shard + ShardRouting primaryShard = ShardRouting.newUnassigned( + shardTable.shardId(), + true, + remoteStoreRecoverySource, + new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "Restoring primary shard") + ); + shardBuilder.addShard(primaryShard); + + // Create the correct number of replica shards + for (int i = 0; i < numberOfReplicas; i++) { + ShardRouting replicaShard = ShardRouting.newUnassigned( + shardTable.shardId(), + false, + RecoverySource.PeerRecoverySource.INSTANCE, + new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "Restoring replica shard") + ); + shardBuilder.addShard(replicaShard); + } + + return shardBuilder.build(); + } + + /** + * Creates a temporary cluster block used during scale-down preparation. + *

+ * This block: + *

    + *
  • Prevents write operations to the index
  • + *
  • Uses a unique ID to track the block through the scaling process
  • + *
  • Returns a 403 Forbidden status for write attempts
  • + *
  • Includes a descriptive message
  • + *
+ * + * @return a cluster block for temporary use during scale-down + */ + static ClusterBlock createScaleDownBlock() { + return new ClusterBlock( + INDEX_SEARCH_ONLY_BLOCK_ID, + UUIDs.randomBase64UUID(), + "index preparing to scale down", + false, + false, + false, + RestStatus.FORBIDDEN, + EnumSet.of(WRITE) + ); + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexNodeRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexNodeRequest.java new file mode 100644 index 0000000000000..73ef7d1b97fc6 --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexNodeRequest.java @@ -0,0 +1,87 @@ +/* + * 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.scale.searchonly; + +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.transport.TransportRequest; + +import java.io.IOException; +import java.util.List; + +/** + * A transport request sent to nodes to facilitate shard synchronization during search-only scaling operations. + *

+ * This request is sent from the cluster manager to data nodes that host primary shards for the target index + * during scale operations. It contains the index name and a list of shard IDs that need to be synchronized + * before completing a scale-down operation. + *

+ * When a node receives this request, it performs final sync and flush operations on the specified shards, + * ensuring all operations are committed and the remote store is synced. This is a crucial step in + * the scale-down process to ensure no data loss occurs when the index transitions to search-only mode. + */ +class ScaleIndexNodeRequest extends TransportRequest { + private final String index; + private final List shardIds; + + /** + * Constructs a new NodeSearchOnlyRequest. + * + * @param index the name of the index being scaled + * @param shardIds the list of shard IDs to be synchronized on the target node + */ + ScaleIndexNodeRequest(String index, List shardIds) { + this.index = index; + this.shardIds = shardIds; + } + + /** + * Deserialization constructor. + * + * @param in the stream input to read from + * @throws IOException if there is an I/O error during deserialization + */ + ScaleIndexNodeRequest(StreamInput in) throws IOException { + super(in); + this.index = in.readString(); + this.shardIds = in.readList(ShardId::new); + } + + /** + * Serializes this request to the given output stream. + * + * @param out the output stream to write to + * @throws IOException if there is an I/O error during serialization + */ + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(index); + out.writeList(shardIds); + } + + /** + * Returns the index name associated with this request. + * + * @return the index name + */ + String getIndex() { + return index; + } + + /** + * Returns the list of shard IDs to be synchronized. + * + * @return the list of shard IDs + */ + List getShardIds() { + return shardIds; + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexNodeResponse.java b/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexNodeResponse.java new file mode 100644 index 0000000000000..5b0f6846d8c3b --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexNodeResponse.java @@ -0,0 +1,89 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.action.admin.indices.scale.searchonly; + +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.transport.TransportResponse; + +import java.io.IOException; +import java.util.List; + +/** + * Response sent from nodes after processing a {@link ScaleIndexNodeRequest} during search-only scaling operations. + *

+ * This response contains information about the node that processed the request and the results of + * synchronization attempts for each requested shard. The cluster manager uses these responses to + * determine whether it's safe to proceed with finalizing a scale-down operation. + *

+ * Each response includes details about whether shards have any uncommitted operations or need + * additional synchronization, which would indicate the scale operation should be delayed until + * the cluster reaches a stable state. + */ +class ScaleIndexNodeResponse extends TransportResponse { + private final DiscoveryNode node; + private final List shardResponses; + + /** + * Constructs a new NodeSearchOnlyResponse. + * + * @param node the node that processed the synchronization request + * @param shardResponses the list of responses from individual shard synchronization attempts + */ + ScaleIndexNodeResponse(DiscoveryNode node, List shardResponses) { + this.node = node; + this.shardResponses = shardResponses; + } + + /** + * Deserialization constructor. + * + * @param in the stream input to read from + * @throws IOException if there is an I/O error during deserialization + */ + ScaleIndexNodeResponse(StreamInput in) throws IOException { + node = new DiscoveryNode(in); + shardResponses = in.readList(ScaleIndexShardResponse::new); + } + + /** + * Serializes this response to the given output stream. + * + * @param out the output stream to write to + * @throws IOException if there is an I/O error during serialization + */ + @Override + public void writeTo(StreamOutput out) throws IOException { + node.writeTo(out); + out.writeList(shardResponses); + } + + /** + * Returns the node that processed the synchronization request. + * + * @return the discovery node information + */ + public DiscoveryNode getNode() { + return node; + } + + /** + * Returns the list of shard-level synchronization responses. + *

+ * These responses contain critical information about the state of each shard, + * including whether there are uncommitted operations or if additional synchronization + * is needed before the scale operation can safely proceed. + * + * @return the list of shard responses + */ + public List getShardResponses() { + return shardResponses; + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexOperationValidator.java b/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexOperationValidator.java new file mode 100644 index 0000000000000..a749f8b3311c9 --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexOperationValidator.java @@ -0,0 +1,88 @@ +/* + * 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.scale.searchonly; + +import org.opensearch.action.support.clustermanager.AcknowledgedResponse; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.core.action.ActionListener; +import org.opensearch.indices.replication.common.ReplicationType; + +/** + * Validates that indices meet the prerequisites for search-only scale operations. + *

+ * This validator ensures that indexes being scaled up or down satisfy all the + * necessary conditions for a safe scaling operation. It checks for required settings, + * index state compatibility, and configuration prerequisites such as remote store + * and segment replication settings. + */ +class ScaleIndexOperationValidator { + + /** + * Validates that the given index meets the prerequisites for the scale operation. + *

+ * For scale-down operations, this method verifies: + *

    + *
  • The index exists
  • + *
  • The index is not already in search-only mode
  • + *
  • The index has at least one search-only replica configured
  • + *
  • Remote store is enabled for the index
  • + *
  • Segment replication is enabled for the index
  • + *
+ *

+ * For scale-up operations, this method verifies: + *

    + *
  • The index exists
  • + *
  • The index is currently in search-only mode
  • + *
+ * + * @param indexMetadata the metadata of the index to validate + * @param index the name of the index being validated + * @param listener the action listener to notify in case of validation failure + * @param isScaleDown true if validating for scale-down, false for scale-up + * @return true if validation succeeds, false if validation fails (and listener is notified) + */ + boolean validateScalePrerequisites( + IndexMetadata indexMetadata, + String index, + ActionListener listener, + boolean isScaleDown + ) { + try { + if (indexMetadata == null) { + throw new IllegalArgumentException("Index [" + index + "] not found"); + } + if (isScaleDown) { + if (indexMetadata.getSettings().getAsBoolean(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), false)) { + throw new IllegalStateException("Index [" + index + "] is already in search-only mode"); + } + + if (indexMetadata.getNumberOfSearchOnlyReplicas() == 0) { + throw new IllegalArgumentException("Cannot scale to zero without search replicas for index: " + index); + } + if (indexMetadata.getSettings().getAsBoolean(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, false) == false) { + throw new IllegalArgumentException( + "To scale to zero, " + IndexMetadata.SETTING_REMOTE_STORE_ENABLED + " must be enabled for index: " + index + ); + } + if (ReplicationType.SEGMENT.toString() + .equals(indexMetadata.getSettings().get(IndexMetadata.SETTING_REPLICATION_TYPE)) == false) { + throw new IllegalArgumentException("To scale to zero, segment replication must be enabled for index: " + index); + } + } else { + if (indexMetadata.getSettings().getAsBoolean(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), false) == false) { + throw new IllegalStateException("Index [" + index + "] is not in search-only mode"); + } + } + return true; + } catch (Exception e) { + listener.onFailure(e); + return false; + } + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexRequest.java new file mode 100644 index 0000000000000..a8ec2754e8414 --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexRequest.java @@ -0,0 +1,166 @@ +/* + * 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.scale.searchonly; + +import org.opensearch.action.ActionRequestValidationException; +import org.opensearch.action.ValidateActions; +import org.opensearch.action.support.IndicesOptions; +import org.opensearch.action.support.clustermanager.AcknowledgedRequest; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.io.stream.StreamOutput; + +import java.io.IOException; +import java.util.Objects; + +/** + * A request for search-only scale operations (up or down) on an index. + *

+ * This request represents an administrative operation to either: + *

    + *
  • Scale an index down to search-only mode, removing write capability while preserving search replicas
  • + *
  • Scale an index up from search-only mode back to full read-write operation
  • + *
+ *

+ * The request is processed by the cluster manager node, which coordinates the necessary + * cluster state changes, shard synchronization, and recovery operations needed to transition + * an index between normal and search-only states. + */ +class ScaleIndexRequest extends AcknowledgedRequest { + private final String index; + private boolean scaleDown; + private IndicesOptions indicesOptions = IndicesOptions.strictExpandOpen(); + + /** + * Constructs a new SearchOnlyRequest. + * + * @param index the name of the index to scale + * @param scaleDown true to scale down to search-only mode, false to scale up to normal operation + */ + ScaleIndexRequest(String index, boolean scaleDown) { + super(); + this.index = index; + this.scaleDown = scaleDown; + } + + /** + * Deserialization constructor. + * + * @param in the stream input to read from + * @throws IOException if there is an I/O error during deserialization + */ + ScaleIndexRequest(StreamInput in) throws IOException { + super(in); + this.index = in.readString(); + this.scaleDown = in.readBoolean(); + this.indicesOptions = IndicesOptions.readIndicesOptions(in); + } + + /** + * Validates this request. + *

+ * Ensures that the index name is provided and not empty. + * + * @return validation exception if invalid, null otherwise + */ + @Override + public ActionRequestValidationException validate() { + ActionRequestValidationException validationException = null; + if (index == null || index.trim().isEmpty()) { + validationException = ValidateActions.addValidationError("index is required", validationException); + } + return validationException; + } + + /** + * Serializes this request to the given output stream. + * + * @param out the output stream to write to + * @throws IOException if there is an I/O error during serialization + */ + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(index); + out.writeBoolean(scaleDown); + indicesOptions.writeIndicesOptions(out); + } + + /** + * Checks if this request equals another object. + * + * @param o the object to compare with + * @return true if equal, false otherwise + */ + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ScaleIndexRequest that = (ScaleIndexRequest) o; + return scaleDown == that.scaleDown && Objects.equals(index, that.index) && Objects.equals(indicesOptions, that.indicesOptions); + } + + /** + * Returns a hash code for this request. + * + * @return the hash code + */ + @Override + public int hashCode() { + return Objects.hash(index, scaleDown, indicesOptions); + } + + /** + * Returns the name of the index to scale. + * + * @return the index name + */ + public String getIndex() { + return index; + } + + /** + * Returns whether this is a scale-down operation. + * + * @return true if scaling down to search-only mode, false if scaling up to normal operation + */ + public boolean isScaleDown() { + return scaleDown; + } + + /** + * Returns the indices options that specify how to resolve indices and handle unavailable indices. + * + * @return the indices options + */ + public IndicesOptions indicesOptions() { + return indicesOptions; + } + + /** + * Sets the indices options for this request. + * + * @param indicesOptions the indices options to use + * @return this request (for method chaining) + */ + public ScaleIndexRequest indicesOptions(IndicesOptions indicesOptions) { + this.indicesOptions = indicesOptions; + return this; + } + + /** + * Sets whether this is a scale down operation. + * + * @param scaleDown true if scaling down, false if scaling up + * @return this request (for method chaining) + */ + public ScaleIndexRequest scaleDown(boolean scaleDown) { + this.scaleDown = scaleDown; + return this; + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexRequestBuilder.java b/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexRequestBuilder.java new file mode 100644 index 0000000000000..f18cdbe5c19a0 --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexRequestBuilder.java @@ -0,0 +1,38 @@ +/* + * 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.scale.searchonly; + +import org.opensearch.action.ActionRequestBuilder; +import org.opensearch.action.support.clustermanager.AcknowledgedResponse; +import org.opensearch.common.annotation.PublicApi; +import org.opensearch.transport.client.OpenSearchClient; + +/** + * A builder for constructing {@link ScaleIndexRequest} objects to perform search-only scale operations. + *

+ * This builder simplifies the creation of requests to scale indices up or down for search-only mode. + * It provides methods to configure the scaling direction and follows the builder pattern to allow + * method chaining for constructing requests. + *

+ * The builder is part of the public API since OpenSearch 3.0.0. + */ +@PublicApi(since = "3.0.0") +public class ScaleIndexRequestBuilder extends ActionRequestBuilder { + + /** + * Constructs a new builder for scaling an index, allowing explicit direction specification. + * + * @param client the client to use for executing the request + * @param searchOnly true for scaling down to search-only mode, false for scaling up to normal mode + * @param index the name of the index to scale + */ + public ScaleIndexRequestBuilder(OpenSearchClient client, boolean searchOnly, String index) { + super(client, ScaleIndexAction.INSTANCE, new ScaleIndexRequest(index, searchOnly)); + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexResponse.java b/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexResponse.java new file mode 100644 index 0000000000000..a81551a3c9898 --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexResponse.java @@ -0,0 +1,136 @@ +/* + * 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.scale.searchonly; + +import org.opensearch.core.action.ActionResponse; +import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; + +/** + * Response for search-only scale operations, containing information about shards' synchronization status. + *

+ * This response aggregates the results of shard synchronization attempts from multiple nodes + * during a scale operation. It tracks: + *

    + *
  • Whether any shards have uncommitted operations
  • + *
  • Whether any shards still need synchronization
  • + *
  • Detailed failure reasons if the scale operation cannot proceed
  • + *
+ *

+ * The response is used by the cluster manager to determine whether a scale operation + * can be finalized or needs to be retried after more time is allowed for synchronization. + */ +class ScaleIndexResponse extends ActionResponse implements ToXContent { + private final Collection nodeResponses; + private final String failureReason; + private final boolean hasFailures; + + /** + * Constructs a new SearchOnlyResponse by aggregating responses from multiple nodes. + *

+ * This constructor analyzes the responses to determine if any shards report conditions + * that would prevent safely finalizing a scale operation, such as uncommitted operations + * or pending synchronization tasks. + * + * @param responses the collection of node responses containing shard status information + */ + ScaleIndexResponse(Collection responses) { + this.nodeResponses = responses; + this.hasFailures = responses.stream() + .anyMatch(r -> r.getShardResponses().stream().anyMatch(s -> s.hasUncommittedOperations() || s.needsSync())); + this.failureReason = buildFailureReason(); + } + + /** + * Serializes this response to the given output stream. + * + * @param out the output stream to write to + * @throws IOException if there is an I/O error during serialization + */ + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeList(new ArrayList<>(nodeResponses)); + out.writeBoolean(hasFailures); + out.writeOptionalString(failureReason); + } + + /** + * Indicates whether any shards reported conditions that would prevent + * safely finalizing the scale operation. + * + * @return true if any shard has uncommitted operations or needs sync, false otherwise + */ + boolean hasFailures() { + return hasFailures; + } + + /** + * Builds a detailed description of failure reasons if the scale operation cannot proceed. + *

+ * This method constructs a human-readable string explaining which shards on which nodes + * reported conditions that prevent the scale operation from being finalized, including + * whether they have uncommitted operations or need additional synchronization. + * + * @return a detailed failure description, or null if no failures were detected + */ + String buildFailureReason() { + if (!hasFailures) { + return null; + } + StringBuilder reason = new StringBuilder(); + for (ScaleIndexNodeResponse nodeResponse : nodeResponses) { + for (ScaleIndexShardResponse shardResponse : nodeResponse.getShardResponses()) { + if (shardResponse.hasUncommittedOperations() || shardResponse.needsSync()) { + reason.append("Shard ") + .append(shardResponse.getShardId()) + .append(" on node ") + .append(nodeResponse.getNode()) + .append(": "); + if (shardResponse.hasUncommittedOperations()) { + reason.append("has uncommitted operations "); + } + if (shardResponse.needsSync()) { + reason.append("needs sync "); + } + reason.append("; "); + } + } + } + return reason.toString(); + } + + /** + * Converts this response to XContent format for API responses. + *

+ * The generated content includes: + *

    + *
  • Whether any failures were detected
  • + *
  • Detailed failure reasons if applicable
  • + *
+ * + * @param builder the XContentBuilder to use + * @param params parameters for XContent generation + * @return the XContentBuilder with response data added + * @throws IOException if there is an error generating the XContent + */ + @Override + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + builder.startObject(); + if (failureReason != null) { + builder.field("failure_reason", failureReason); + } + builder.endObject(); + return builder; + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexShardResponse.java b/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexShardResponse.java new file mode 100644 index 0000000000000..8c7759e44e12f --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexShardResponse.java @@ -0,0 +1,109 @@ +/* + * 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.scale.searchonly; + +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.index.shard.ShardId; + +import java.io.IOException; + +/** + * Response containing synchronization status for a single shard during search-only scale operations. + *

+ * This response captures the critical state information needed to determine if a shard + * is ready for a scale operation to proceed, including: + *

    + *
  • Whether the shard has uncommitted operations that need to be persisted
  • + *
  • Whether the shard needs additional synchronization with remote storage
  • + *
+ *

+ * The cluster manager uses this information from all primary shards to decide + * whether it's safe to finalize a scale-down operation. + */ +class ScaleIndexShardResponse implements Writeable { + private final ShardId shardId; + private final boolean needsSync; + private final int uncommittedOperations; + + /** + * Constructs a new ShardSearchOnlyResponse. + * + * @param shardId the ID of the shard that was synchronized + * @param needsSync whether the shard still needs additional synchronization + * @param uncommittedOperations the number of operations not yet committed to the transaction log + */ + ScaleIndexShardResponse(ShardId shardId, boolean needsSync, int uncommittedOperations) { + this.shardId = shardId; + this.needsSync = needsSync; + this.uncommittedOperations = uncommittedOperations; + } + + /** + * Deserialization constructor. + * + * @param in the stream input to read from + * @throws IOException if there is an I/O error during deserialization + */ + ScaleIndexShardResponse(StreamInput in) throws IOException { + this.shardId = new ShardId(in); + this.needsSync = in.readBoolean(); + this.uncommittedOperations = in.readVInt(); + } + + /** + * Serializes this response to the given output stream. + * + * @param out the output stream to write to + * @throws IOException if there is an I/O error during serialization + */ + @Override + public void writeTo(StreamOutput out) throws IOException { + shardId.writeTo(out); + out.writeBoolean(needsSync); + out.writeVInt(uncommittedOperations); + } + + /** + * Returns the shard ID associated with this response. + * + * @return the shard ID + */ + ShardId getShardId() { + return shardId; + } + + /** + * Indicates whether the shard needs additional synchronization before scaling. + *

+ * A shard may need synchronization if: + *

    + *
  • It has pending operations that need to be synced to remote storage
  • + *
  • The local and remote states don't match
  • + *
+ * + * @return true if additional synchronization is needed, false otherwise + */ + boolean needsSync() { + return needsSync; + } + + /** + * Indicates whether the shard has operations that haven't been committed to the transaction log. + *

+ * Uncommitted operations represent recent writes that haven't been fully persisted, + * making it unsafe to proceed with a scale-down operation until they are committed. + * + * @return true if there are uncommitted operations, false otherwise + */ + boolean hasUncommittedOperations() { + return uncommittedOperations > 0; + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexShardSyncManager.java b/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexShardSyncManager.java new file mode 100644 index 0000000000000..596bbc2c5270f --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexShardSyncManager.java @@ -0,0 +1,217 @@ +/* + * 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.scale.searchonly; + +import org.opensearch.action.support.GroupedActionListener; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportException; +import org.opensearch.transport.TransportResponseHandler; +import org.opensearch.transport.TransportService; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Manages shard synchronization for scale operations + */ +/** + * Manages shard synchronization operations during search-only scaling. + *

+ * This manager coordinates the necessary synchronization across nodes to ensure all shards + * are in a consistent state before finalizing scale operations. It handles: + *

    + *
  • Dispatching sync requests to nodes hosting primary shards
  • + *
  • Aggregating responses from multiple nodes
  • + *
  • Validating that shards are ready for scale operations
  • + *
  • Tracking primary shard assignments across the cluster
  • + *
+ *

+ * The synchronization process is a critical safety mechanism that prevents data loss + * during transitions between normal and search-only modes. + */ +class ScaleIndexShardSyncManager { + + private final ClusterService clusterService; + private final TransportService transportService; + private final String transportActionName; + + /** + * Constructs a new ShardSyncManager. + * + * @param clusterService the cluster service for accessing cluster state + * @param transportService the transport service for sending requests to other nodes + * @param transportActionName the transport action name for shard sync requests + */ + ScaleIndexShardSyncManager(ClusterService clusterService, TransportService transportService, String transportActionName) { + this.clusterService = clusterService; + this.transportService = transportService; + this.transportActionName = transportActionName; + } + + /** + * Sends shard sync requests to each node that holds a primary shard. + *

+ * This method: + *

    + *
  • Groups shards by node to minimize network requests
  • + *
  • Creates a grouped listener to aggregate responses
  • + *
  • Dispatches sync requests to each relevant node
  • + *
+ *

+ * The listener is notified once all nodes have responded or if any errors occur. + * + * @param index the name of the index being scaled + * @param primaryShardsNodes map of shard IDs to node IDs for all primary shards + * @param listener the listener to notify when all nodes have responded + */ + void sendShardSyncRequests( + String index, + Map primaryShardsNodes, + ActionListener> listener + ) { + if (primaryShardsNodes.isEmpty()) { + listener.onFailure(new IllegalStateException("No primary shards found for index " + index)); + return; + } + + Map> nodeShardGroups = primaryShardsNodes.entrySet() + .stream() + .collect(Collectors.groupingBy(Map.Entry::getValue, Collectors.mapping(Map.Entry::getKey, Collectors.toList()))); + + final GroupedActionListener groupedListener = new GroupedActionListener<>(listener, nodeShardGroups.size()); + + for (Map.Entry> entry : nodeShardGroups.entrySet()) { + final String nodeId = entry.getKey(); + final List shards = entry.getValue(); + final DiscoveryNode targetNode = clusterService.state().nodes().get(nodeId); + + if (targetNode == null) { + groupedListener.onFailure(new IllegalStateException("Node [" + nodeId + "] not found")); + continue; + } + + sendNodeRequest(targetNode, index, shards, groupedListener); + } + } + + /** + * Sends a sync request to a specific node for the given shards. + *

+ * This method creates and sends a transport request to perform shard synchronization + * on a target node, registering appropriate response and error handlers. + * + * @param targetNode the node to send the request to + * @param index the name of the index being scaled + * @param shards the list of shards to synchronize on the target node + * @param listener the listener to notify with the response + */ + void sendNodeRequest(DiscoveryNode targetNode, String index, List shards, ActionListener listener) { + transportService.sendRequest( + targetNode, + transportActionName, + new ScaleIndexNodeRequest(index, shards), + new TransportResponseHandler() { + @Override + public ScaleIndexNodeResponse read(StreamInput in) throws IOException { + return new ScaleIndexNodeResponse(in); + } + + @Override + public void handleResponse(ScaleIndexNodeResponse response) { + listener.onResponse(response); + } + + @Override + public void handleException(TransportException exp) { + listener.onFailure(exp); + } + + @Override + public String executor() { + return ThreadPool.Names.SAME; + } + } + ); + } + + /** + * Aggregates node responses and verifies that no shard reports uncommitted operations or a pending sync. + *

+ * This validation ensures that all shards are in a consistent state before proceeding with + * a scale operation. If any shard reports conditions that would make scaling unsafe, the + * operation is failed with detailed information about which shards need more time. + * + * @param responses the collection of responses from all nodes + * @param listener the listener to notify with the aggregated result + */ + void validateNodeResponses(Collection responses, ActionListener listener) { + boolean hasUncommittedOps = false; + boolean needsSync = false; + List failedShards = new ArrayList<>(); + + for (ScaleIndexNodeResponse nodeResponse : responses) { + for (ScaleIndexShardResponse shardResponse : nodeResponse.getShardResponses()) { + if (shardResponse.hasUncommittedOperations()) { + hasUncommittedOps = true; + failedShards.add(shardResponse.getShardId().toString()); + } + if (shardResponse.needsSync()) { + needsSync = true; + failedShards.add(shardResponse.getShardId().toString()); + } + } + } + + if (hasUncommittedOps || needsSync) { + String errorDetails = "Pre-scale sync failed for shards: " + + String.join(", ", failedShards) + + (hasUncommittedOps ? " - uncommitted operations remain" : "") + + (needsSync ? " - sync needed" : ""); + listener.onFailure(new IllegalStateException(errorDetails)); + } else { + listener.onResponse(new ScaleIndexResponse(responses)); + } + } + + /** + * Returns the primary shard node assignments for a given index. + *

+ * Builds a mapping between shard IDs and the node IDs hosting their primary copies. + * This mapping is used to determine which nodes need to be contacted for shard + * synchronization during scale operations. + * + * @param indexMetadata the metadata of the index + * @param state the current cluster state + * @return a map of shard IDs to node IDs for all assigned primary shards + */ + Map getPrimaryShardAssignments(IndexMetadata indexMetadata, ClusterState state) { + Map assignments = new HashMap<>(); + for (int i = 0; i < indexMetadata.getNumberOfShards(); i++) { + ShardId shardId = new ShardId(indexMetadata.getIndex(), i); + ShardRouting primaryShard = state.routingTable().index(indexMetadata.getIndex().getName()).shard(i).primaryShard(); + if (primaryShard != null && primaryShard.assignedToNode()) { + assignments.put(shardId, primaryShard.currentNodeId()); + } + } + return assignments; + } +} diff --git a/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/TransportScaleIndexAction.java b/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/TransportScaleIndexAction.java new file mode 100644 index 0000000000000..7384f35a600f7 --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/TransportScaleIndexAction.java @@ -0,0 +1,481 @@ +/* + * 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.scale.searchonly; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.action.admin.indices.flush.FlushRequest; +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.ChannelActionListener; +import org.opensearch.action.support.GroupedActionListener; +import org.opensearch.action.support.clustermanager.AcknowledgedResponse; +import org.opensearch.action.support.clustermanager.TransportClusterManagerNodeAction; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.ClusterStateUpdateTask; +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.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.IndexNameExpressionResolver; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.routing.allocation.AllocationService; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.Priority; +import org.opensearch.common.inject.Inject; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.index.Index; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.IndexService; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.indices.IndicesService; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportChannel; +import org.opensearch.transport.TransportService; + +import java.io.IOException; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.opensearch.cluster.metadata.IndexMetadata.INDEX_SEARCH_ONLY_BLOCK_ID; + +/** + * Transport action implementation for search-only scale operations. + *

+ * This class coordinates the entire process of scaling indices up or down between normal + * and search-only modes. It manages the multistep process including: + *

    + *
  • Validating prerequisites for scale operations
  • + *
  • Adding temporary write blocks during scale-down preparation
  • + *
  • Coordinating shard synchronization across nodes
  • + *
  • Modifying cluster state to transition indices between modes
  • + *
  • Handling synchronization requests from other nodes
  • + *
+ *

+ * The scale operation is implemented as a series of cluster state update tasks to ensure + * atomicity and consistency throughout the transition. + */ +public class TransportScaleIndexAction extends TransportClusterManagerNodeAction { + + private static final Logger logger = LogManager.getLogger(TransportScaleIndexAction.class); + /** + * Transport action name for shard sync requests + */ + public static final String NAME = ScaleIndexAction.NAME + "[s]"; + + public static final String SHARD_SYNC_EXECUTOR = ThreadPool.Names.MANAGEMENT; + + private final AllocationService allocationService; + private final IndicesService indicesService; + private final ThreadPool threadPool; + + private final ScaleIndexOperationValidator validator; + private final ScaleIndexClusterStateBuilder scaleIndexClusterStateBuilder; + private final ScaleIndexShardSyncManager scaleIndexShardSyncManager; + + /** + * Constructs a new TransportSearchOnlyAction. + * + * @param transportService the transport service for network communication + * @param clusterService the cluster service for accessing cluster state + * @param threadPool the thread pool for executing operations + * @param actionFilters filters for action requests + * @param indexNameExpressionResolver resolver for index names and expressions + * @param allocationService service for shard allocation decisions + * @param indicesService service for accessing index shards + */ + @Inject + public TransportScaleIndexAction( + TransportService transportService, + ClusterService clusterService, + ThreadPool threadPool, + ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver, + AllocationService allocationService, + IndicesService indicesService + ) { + super( + ScaleIndexAction.NAME, + transportService, + clusterService, + threadPool, + actionFilters, + ScaleIndexRequest::new, + indexNameExpressionResolver + ); + this.allocationService = allocationService; + this.indicesService = indicesService; + this.threadPool = threadPool; + this.validator = new ScaleIndexOperationValidator(); + this.scaleIndexClusterStateBuilder = new ScaleIndexClusterStateBuilder(); + this.scaleIndexShardSyncManager = new ScaleIndexShardSyncManager(clusterService, transportService, NAME); + + transportService.registerRequestHandler( + NAME, + ThreadPool.Names.SAME, + ScaleIndexNodeRequest::new, + (request, channel, task) -> handleShardSyncRequest(request, channel) + ); + } + + /** + * Returns the executor name for this transport action. + * + * @return the executor name + */ + @Override + protected String executor() { + return ThreadPool.Names.MANAGEMENT; + } + + /** + * Deserializes the response from stream. + * + * @param in the stream input + * @return the deserialized response + * @throws IOException if an I/O error occurs + */ + @Override + protected AcknowledgedResponse read(StreamInput in) throws IOException { + return new AcknowledgedResponse(in); + } + + /** + * Handles the search-only request on the cluster manager node. + *

+ * This method determines whether to execute a scale-up or scale-down operation + * based on the request parameters, and submits the appropriate cluster state update task. + * + * @param request the search-only scale request + * @param state the current cluster state + * @param listener the listener to notify with the operation result + */ + @Override + protected void clusterManagerOperation(ScaleIndexRequest request, ClusterState state, ActionListener listener) { + try { + String index = request.getIndex(); + if (request.isScaleDown()) { + submitScaleDownTask(index, listener); + } else { + submitScaleUpTask(index, state, listener); + } + } catch (Exception e) { + logger.error("Failed to execute cluster manager operation", e); + listener.onFailure(e); + } + } + + /** + * Submits the scale-down update task: it first adds a temporary block to the indices and then initiates shard synchronization. + */ + private void submitScaleDownTask(final String index, final ActionListener listener) { + final Map blockedIndices = new HashMap<>(); + + clusterService.submitStateUpdateTask( + "add-block-index-to-scale " + index, + new AddBlockClusterStateUpdateTask(index, blockedIndices, listener) + ); + } + + /** + * Sends shard sync requests to each node that holds a primary shard. + */ + private void proceedWithScaleDown( + String index, + Map primaryShardsNodes, + ActionListener listener + ) { + scaleIndexShardSyncManager.sendShardSyncRequests( + index, + primaryShardsNodes, + ActionListener.wrap(responses -> handleShardSyncResponses(responses, index, listener), listener::onFailure) + ); + } + + private void handleShardSyncResponses( + Collection responses, + String index, + ActionListener listener + ) { + scaleIndexShardSyncManager.validateNodeResponses( + responses, + ActionListener.wrap(searchOnlyResponse -> finalizeScaleDown(index, listener), listener::onFailure) + ); + } + + /** + * Finalizes scale-down by updating the metadata and routing table: + * removes the temporary block and adds a permanent search-only block. + */ + private void finalizeScaleDown(String index, ActionListener listener) { + clusterService.submitStateUpdateTask("finalize-scale-down", new FinalizeScaleDownTask(index, listener)); + } + + /** + * Handles an incoming shard sync request from another node. + */ + void handleShardSyncRequest(ScaleIndexNodeRequest request, TransportChannel channel) { + ClusterState state = clusterService.state(); + + IndexMetadata indexMetadata = state.metadata().index(request.getIndex()); + if (indexMetadata == null) { + throw new IllegalStateException("Index " + request.getIndex() + " not found"); + } + + IndexService indexService = getIndexService(indexMetadata); + ChannelActionListener listener = new ChannelActionListener<>( + channel, + "sync_shard", + request + ); + + syncShards(indexService, request.getShardIds(), listener); + } + + private IndexService getIndexService(IndexMetadata indexMetadata) { + IndexService indexService = indicesService.indexService(indexMetadata.getIndex()); + if (indexService == null) { + throw new IllegalStateException("IndexService not found for index " + indexMetadata.getIndex().getName()); + } + return indexService; + } + + private void syncShards(IndexService indexService, List shardIds, ActionListener listener) { + + GroupedActionListener groupedActionListener = new GroupedActionListener<>(new ActionListener<>() { + @Override + public void onResponse(Collection shardResponses) { + listener.onResponse(new ScaleIndexNodeResponse(clusterService.localNode(), shardResponses.stream().toList())); + } + + @Override + public void onFailure(Exception e) { + listener.onFailure(e); + } + }, shardIds.size()); + + for (ShardId shardId : shardIds) { + IndexShard shard = indexService.getShardOrNull(shardId.id()); + if (shard == null || shard.routingEntry().primary() == false) { + groupedActionListener.onFailure(new IllegalStateException("Attempting to scale down a replica shard")); + break; + } + threadPool.executor(SHARD_SYNC_EXECUTOR).execute(() -> { syncSingleShard(shard, groupedActionListener); }); + } + } + + void syncSingleShard(IndexShard shard, ActionListener listener) { + shard.acquireAllPrimaryOperationsPermits(new ActionListener<>() { + @Override + public void onResponse(Releasable releasable) { + logger.info("Performing final sync and flush for shard {}", shard.shardId()); + try { + shard.sync(); + shard.flush(new FlushRequest().force(true).waitIfOngoing(true)); + shard.waitForRemoteStoreSync(); + listener.onResponse( + new ScaleIndexShardResponse(shard.shardId(), shard.isSyncNeeded(), shard.translogStats().getUncommittedOperations()) + ); + } catch (IOException e) { + listener.onFailure(e); + } + } + + @Override + public void onFailure(Exception e) { + listener.onFailure(e); + } + }, TimeValue.timeValueSeconds(30)); + } + + /** + * Submits the scale-up update task that rebuilds the routing table and updates index metadata. + */ + private void submitScaleUpTask( + final String index, + final ClusterState currentState, + final ActionListener listener + ) { + IndexMetadata indexMetadata = currentState.metadata().index(index); + if (!validator.validateScalePrerequisites(indexMetadata, index, listener, false)) { + return; + } + + clusterService.submitStateUpdateTask("scale-up-index", new ScaleUpClusterStateUpdateTask(index, listener)); + } + + @Override + protected ClusterBlockException checkBlock(ScaleIndexRequest request, ClusterState state) { + return state.blocks() + .indicesBlockedException( + ClusterBlockLevel.METADATA_WRITE, + indexNameExpressionResolver.concreteIndexNames(state, request.indicesOptions(), request.getIndex()) + ); + } + + /** + * Cluster state update task for adding a temporary block during the initial phase of scaling down. + *

+ * This task: + *

    + *
  • Validates that the index meets prerequisites for scaling down
  • + *
  • Adds a temporary write block to prevent new operations during scaling
  • + *
  • Initiates shard synchronization after the block is applied
  • + *
+ */ + class AddBlockClusterStateUpdateTask extends ClusterStateUpdateTask { + private final String index; + private final Map blockedIndices; + private final ActionListener listener; + + AddBlockClusterStateUpdateTask( + String index, + Map blockedIndices, + ActionListener listener + ) { + super(Priority.URGENT); + this.index = index; + this.blockedIndices = blockedIndices; + this.listener = listener; + } + + @Override + public ClusterState execute(final ClusterState currentState) { + IndexMetadata indexMetadata = currentState.metadata().index(index); + try { + validator.validateScalePrerequisites(indexMetadata, index, listener, true); + return scaleIndexClusterStateBuilder.buildScaleDownState(currentState, index, blockedIndices); + } catch (Exception e) { + return currentState; + } + } + + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + if (oldState == newState) { + listener.onResponse(new AcknowledgedResponse(true)); + return; + } + + IndexMetadata indexMetadata = newState.metadata().index(index); + if (indexMetadata != null) { + Map primaryShardsNodes = scaleIndexShardSyncManager.getPrimaryShardAssignments(indexMetadata, newState); + proceedWithScaleDown(index, primaryShardsNodes, listener); + } + } + + @Override + public void onFailure(String source, Exception e) { + logger.error("Failed to process cluster state update for scale down", e); + listener.onFailure(e); + } + } + + /** + * Cluster state update task for finalizing a scale-down operation. + *

+ * This task: + *

    + *
  • Removes the temporary scale-down preparation block
  • + *
  • Updates index metadata to mark it as search-only
  • + *
  • Applies a permanent search-only block
  • + *
  • Updates the routing table to remove non-search-only shards
  • + *
+ */ + class FinalizeScaleDownTask extends ClusterStateUpdateTask { + private final String index; + private final ActionListener listener; + + FinalizeScaleDownTask(String index, ActionListener listener) { + super(Priority.URGENT); + this.index = index; + this.listener = listener; + } + + @Override + public ClusterState execute(ClusterState currentState) { + return scaleIndexClusterStateBuilder.buildFinalScaleDownState(currentState, index); + } + + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + listener.onResponse(new AcknowledgedResponse(true)); + } + + @Override + public void onFailure(String source, Exception e) { + logger.error("Failed to finalize scale-down operation", e); + listener.onFailure(e); + } + } + + /** + * Cluster state update task for scaling up an index from search-only mode to normal operation. + *

+ * This task: + *

    + *
  • Rebuilds the routing table to add primary and replica shards
  • + *
  • Removes the search-only block
  • + *
  • Updates index settings to disable search-only mode
  • + *
  • Triggers routing table updates to allocate the new shards
  • + *
+ */ + private class ScaleUpClusterStateUpdateTask extends ClusterStateUpdateTask { + private final String index; + private final ActionListener listener; + + ScaleUpClusterStateUpdateTask(String index, ActionListener listener) { + this.index = index; + this.listener = listener; + } + + @Override + public ClusterState execute(ClusterState currentState) { + RoutingTable newRoutingTable = scaleIndexClusterStateBuilder.buildScaleUpRoutingTable(currentState, index); + ClusterState tempState = ClusterState.builder(currentState).routingTable(newRoutingTable).build(); + + ClusterBlocks.Builder blocksBuilder = ClusterBlocks.builder().blocks(tempState.blocks()); + Metadata.Builder metadataBuilder = Metadata.builder(tempState.metadata()); + + blocksBuilder.removeIndexBlockWithId(index, INDEX_SEARCH_ONLY_BLOCK_ID); + IndexMetadata indexMetadata = tempState.metadata().index(index); + Settings updatedSettings = Settings.builder() + .put(indexMetadata.getSettings()) + .put(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), false) + .build(); + metadataBuilder.put( + IndexMetadata.builder(indexMetadata).settings(updatedSettings).settingsVersion(indexMetadata.getSettingsVersion() + 1) + ); + + return allocationService.reroute( + ClusterState.builder(tempState).blocks(blocksBuilder).metadata(metadataBuilder).build(), + "restore indexing shards" + ); + } + + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + listener.onResponse(new AcknowledgedResponse(true)); + } + + @Override + public void onFailure(String source, Exception e) { + logger.error("Failed to execute cluster state update for scale up", e); + listener.onFailure(e); + } + } + +} diff --git a/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/package-info.java b/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/package-info.java new file mode 100644 index 0000000000000..cd1d28c0d798d --- /dev/null +++ b/server/src/main/java/org/opensearch/action/admin/indices/scale/searchonly/package-info.java @@ -0,0 +1,10 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +/** Index Scaling transport handlers for managing search-only mode. */ +package org.opensearch.action.admin.indices.scale.searchonly; 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 1a3c657f5b1b8..fad504a476511 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 @@ -146,7 +146,7 @@ protected void clusterManagerOperation( final String customDataPath = IndexMetadata.INDEX_DATA_PATH_SETTING.get(state.metadata().index(index).getSettings()); for (IndexShardRoutingTable routing : indexShardRoutingTables) { final int shardId = routing.shardId().id(); - ClusterShardHealth shardHealth = new ClusterShardHealth(shardId, routing); + ClusterShardHealth shardHealth = new ClusterShardHealth(shardId, routing, state.metadata().index(index)); if (request.shardStatuses().contains(shardHealth.getStatus())) { shardsToFetch.add(Tuple.tuple(routing.shardId(), customDataPath)); } diff --git a/server/src/main/java/org/opensearch/cluster/block/ClusterBlockLevel.java b/server/src/main/java/org/opensearch/cluster/block/ClusterBlockLevel.java index 5d3bf94aedb19..2164fc48e4ba5 100644 --- a/server/src/main/java/org/opensearch/cluster/block/ClusterBlockLevel.java +++ b/server/src/main/java/org/opensearch/cluster/block/ClusterBlockLevel.java @@ -51,4 +51,5 @@ public enum ClusterBlockLevel { public static final EnumSet ALL = EnumSet.allOf(ClusterBlockLevel.class); public static final EnumSet READ_WRITE = EnumSet.of(READ, WRITE); + public static final EnumSet WRITE_BLOCK = EnumSet.of(WRITE); } diff --git a/server/src/main/java/org/opensearch/cluster/block/ClusterBlocks.java b/server/src/main/java/org/opensearch/cluster/block/ClusterBlocks.java index c894fa5dce714..4e119418f6917 100644 --- a/server/src/main/java/org/opensearch/cluster/block/ClusterBlocks.java +++ b/server/src/main/java/org/opensearch/cluster/block/ClusterBlocks.java @@ -432,6 +432,9 @@ public Builder addBlocks(IndexMetadata indexMetadata) { if (indexMetadata.isRemoteSnapshot()) { addIndexBlock(indexName, IndexMetadata.REMOTE_READ_ONLY_ALLOW_DELETE); } + if (IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.get(indexMetadata.getSettings())) { + addIndexBlock(indexName, IndexMetadata.APIBlock.SEARCH_ONLY.getBlock()); + } return this; } diff --git a/server/src/main/java/org/opensearch/cluster/health/ClusterIndexHealth.java b/server/src/main/java/org/opensearch/cluster/health/ClusterIndexHealth.java index 77d96cb0af792..fae586e1adf03 100644 --- a/server/src/main/java/org/opensearch/cluster/health/ClusterIndexHealth.java +++ b/server/src/main/java/org/opensearch/cluster/health/ClusterIndexHealth.java @@ -156,7 +156,7 @@ public ClusterIndexHealth(final IndexMetadata indexMetadata, final IndexRoutingT shards = new HashMap<>(); for (IndexShardRoutingTable shardRoutingTable : indexRoutingTable) { int shardId = shardRoutingTable.shardId().id(); - shards.put(shardId, new ClusterShardHealth(shardId, shardRoutingTable)); + shards.put(shardId, new ClusterShardHealth(shardId, shardRoutingTable, indexMetadata)); } // update the index status @@ -212,11 +212,13 @@ public ClusterIndexHealth( int computeUnassignedShards = 0; int computeDelayedUnassignedShards = 0; + boolean isSearchOnlyClusterBlockEnabled = indexMetadata.getSettings() + .getAsBoolean(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), false); boolean isShardLevelHealthRequired = healthLevel == ClusterHealthRequest.Level.SHARDS; if (isShardLevelHealthRequired) { for (IndexShardRoutingTable indexShardRoutingTable : indexRoutingTable) { int shardId = indexShardRoutingTable.shardId().id(); - ClusterShardHealth shardHealth = new ClusterShardHealth(shardId, indexShardRoutingTable); + ClusterShardHealth shardHealth = new ClusterShardHealth(shardId, indexShardRoutingTable, indexMetadata); if (shardHealth.isPrimaryActive()) { computeActivePrimaryShards++; } @@ -252,15 +254,25 @@ public ClusterIndexHealth( } } ShardRouting primaryShard = indexShardRoutingTable.primaryShard(); - if (primaryShard.active()) { - computeActivePrimaryShards++; + + if (primaryShard == null) { + if (isSearchOnlyClusterBlockEnabled) { + computeStatus = getIndexHealthStatus(ClusterHealthStatus.GREEN, computeStatus); + } else { + computeStatus = getIndexHealthStatus(ClusterHealthStatus.RED, computeStatus); + } + } else { + if (primaryShard.active()) { + computeActivePrimaryShards++; + } + ClusterHealthStatus shardHealth = ClusterShardHealth.getShardHealth( + primaryShard, + activeShardsPerShardId, + shardRoutingCountPerShardId, + indexMetadata + ); + computeStatus = getIndexHealthStatus(shardHealth, computeStatus); } - ClusterHealthStatus shardHealth = ClusterShardHealth.getShardHealth( - primaryShard, - activeShardsPerShardId, - shardRoutingCountPerShardId - ); - computeStatus = getIndexHealthStatus(shardHealth, computeStatus); } } diff --git a/server/src/main/java/org/opensearch/cluster/health/ClusterShardHealth.java b/server/src/main/java/org/opensearch/cluster/health/ClusterShardHealth.java index ace4537a5e291..15f255d78ae0d 100644 --- a/server/src/main/java/org/opensearch/cluster/health/ClusterShardHealth.java +++ b/server/src/main/java/org/opensearch/cluster/health/ClusterShardHealth.java @@ -32,6 +32,7 @@ package org.opensearch.cluster.health; +import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.routing.IndexShardRoutingTable; import org.opensearch.cluster.routing.RecoverySource; import org.opensearch.cluster.routing.ShardRouting; @@ -113,7 +114,11 @@ public final class ClusterShardHealth implements Writeable, ToXContentFragment { private int delayedUnassignedShards; private final boolean primaryActive; - public ClusterShardHealth(final int shardId, final IndexShardRoutingTable shardRoutingTable) { + public ClusterShardHealth( + final int shardId, + final IndexShardRoutingTable shardRoutingTable, + final boolean isSearchOnlyClusterBlockEnabled + ) { this.shardId = shardId; int computeActiveShards = 0; int computeRelocatingShards = 0; @@ -126,7 +131,6 @@ public ClusterShardHealth(final int shardId, final IndexShardRoutingTable shardR if (shardRouting.active()) { computeActiveShards++; if (shardRouting.relocating()) { - // the shard is relocating, the one it is relocating to will be in initializing state, so we don't count it computeRelocatingShards++; } } else if (shardRouting.initializing()) { @@ -139,13 +143,22 @@ public ClusterShardHealth(final int shardId, final IndexShardRoutingTable shardR } } final ShardRouting primaryRouting = shardRoutingTable.primaryShard(); - this.status = getShardHealth(primaryRouting, computeActiveShards, shardRoutingTable.size()); + this.status = getShardHealth(primaryRouting, computeActiveShards, shardRoutingTable.size(), isSearchOnlyClusterBlockEnabled); this.activeShards = computeActiveShards; this.relocatingShards = computeRelocatingShards; this.initializingShards = computeInitializingShards; this.unassignedShards = computeUnassignedShards; this.delayedUnassignedShards = computeDelayedUnassignedShards; - this.primaryActive = primaryRouting.active(); + this.primaryActive = primaryRouting != null && primaryRouting.active(); + } + + // Original constructor can call the new one + public ClusterShardHealth(final int shardId, final IndexShardRoutingTable shardRoutingTable, final IndexMetadata indexMetadata) { + this( + shardId, + shardRoutingTable, + indexMetadata.getSettings().getAsBoolean(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), false) + ); } public ClusterShardHealth(final StreamInput in) throws IOException { @@ -229,9 +242,40 @@ public void writeTo(final StreamOutput out) throws IOException { * Shard health is YELLOW when primary shard is active but at-least one replica shard is inactive. * Shard health is RED when the primary is not active. *

+ *

+ * In search-only mode (when {@code isSearchOnlyClusterBlockEnabled} is {@code true}): + *

+ *
    + *
  • Shard health is GREEN when all expected search replicas are active
  • + *
  • Shard health is YELLOW when some (but not all) search replicas are active
  • + *
  • Shard health is RED when no search replicas are active
  • + *
+ * + * @param primaryRouting the routing entry for the primary shard, may be null + * @param activeShards the number of active shards (primary and replicas) + * @param totalShards the total number of shards (primary and replicas) + * @param isSearchOnlyClusterBlockEnabled whether the index is in search-only mode + * @return the health status for the shard */ - public static ClusterHealthStatus getShardHealth(final ShardRouting primaryRouting, final int activeShards, final int totalShards) { - assert primaryRouting != null : "Primary shard routing can't be null"; + public static ClusterHealthStatus getShardHealth( + final ShardRouting primaryRouting, + final int activeShards, + final int totalShards, + final boolean isSearchOnlyClusterBlockEnabled + ) { + + if (primaryRouting == null) { + if (isSearchOnlyClusterBlockEnabled) { + if (activeShards == 0) { + return ClusterHealthStatus.RED; + } else { + return (activeShards < totalShards) ? ClusterHealthStatus.YELLOW : ClusterHealthStatus.GREEN; + } + } else { + return ClusterHealthStatus.RED; + } + } + if (primaryRouting.active()) { if (activeShards == totalShards) { return ClusterHealthStatus.GREEN; @@ -243,6 +287,35 @@ public static ClusterHealthStatus getShardHealth(final ShardRouting primaryRouti } } + /** + * Computes the shard health of an index. + *

+ * Shard health is GREEN when all primary and replica shards of the indices are active. + * Shard health is YELLOW when primary shard is active but at-least one replica shard is inactive. + * Shard health is RED when the primary is not active. + *

+ *

+ * In search-only mode (when {@link IndexMetadata#INDEX_BLOCKS_SEARCH_ONLY_SETTING} is enabled): + *

+ *
    + *
  • Shard health is GREEN when all expected search replicas are active
  • + *
  • Shard health is YELLOW when some (but not all) search replicas are active
  • + *
  • Shard health is RED when no search replicas are active
  • + *
+ */ + public static ClusterHealthStatus getShardHealth( + final ShardRouting primaryRouting, + final int activeShards, + final int totalShards, + final IndexMetadata indexMetadata + ) { + + boolean isSearchOnlyClusterBlockEnabled = indexMetadata.getSettings() + .getAsBoolean(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), false); + + return getShardHealth(primaryRouting, activeShards, totalShards, isSearchOnlyClusterBlockEnabled); + } + /** * Checks if an inactive primary shard should cause the cluster health to go RED. *

diff --git a/server/src/main/java/org/opensearch/cluster/health/ClusterStateHealth.java b/server/src/main/java/org/opensearch/cluster/health/ClusterStateHealth.java index 5eeac822e7c3e..ea09555d9d537 100644 --- a/server/src/main/java/org/opensearch/cluster/health/ClusterStateHealth.java +++ b/server/src/main/java/org/opensearch/cluster/health/ClusterStateHealth.java @@ -93,6 +93,7 @@ public ClusterStateHealth(final ClusterState clusterState, final String[] concre numberOfDataNodes = clusterState.nodes().getDataNodes().size(); hasDiscoveredClusterManager = clusterState.nodes().getClusterManagerNodeId() != null; indices = new HashMap<>(); + for (String index : concreteIndices) { IndexRoutingTable indexRoutingTable = clusterState.routingTable().index(index); IndexMetadata indexMetadata = clusterState.metadata().index(index); @@ -101,7 +102,6 @@ public ClusterStateHealth(final ClusterState clusterState, final String[] concre } ClusterIndexHealth indexHealth = new ClusterIndexHealth(indexMetadata, indexRoutingTable); - indices.put(indexHealth.getIndex(), indexHealth); } @@ -123,6 +123,21 @@ public ClusterStateHealth(final ClusterState clusterState, final String[] concre computeStatus = getClusterHealthStatus(indexHealth, computeStatus); } + Map searchOnlyIndices = collectSearchOnlyIndices(clusterState, concreteIndices, indices); + + if (searchOnlyIndices.isEmpty() == false) { + for (ClusterIndexHealth indexHealth : searchOnlyIndices.values()) { + if (indexHealth.getStatus() == ClusterHealthStatus.RED) { + computeStatus = ClusterHealthStatus.RED; + break; + } + if (indexHealth.getUnassignedShards() > 0 && indexHealth.getActiveShards() == 0) { + computeStatus = ClusterHealthStatus.RED; + break; + } + } + } + if (clusterState.blocks().hasGlobalBlockWithStatus(RestStatus.SERVICE_UNAVAILABLE)) { computeStatus = ClusterHealthStatus.RED; } @@ -177,7 +192,12 @@ public ClusterStateHealth( continue; } - ClusterIndexHealth indexHealth = new ClusterIndexHealth(indexMetadata, indexRoutingTable, healthLevel); + ClusterHealthRequest.Level indexHealthLevel = healthLevel; + if (healthLevel == ClusterHealthRequest.Level.CLUSTER && isSearchOnlyClusterBlockEnabled(indexMetadata)) { + indexHealthLevel = ClusterHealthRequest.Level.SHARDS; + } + + ClusterIndexHealth indexHealth = new ClusterIndexHealth(indexMetadata, indexRoutingTable, indexHealthLevel); computeActivePrimaryShards += indexHealth.getActivePrimaryShards(); computeActiveShards += indexHealth.getActiveShards(); computeRelocatingShards += indexHealth.getRelocatingShards(); @@ -186,12 +206,25 @@ public ClusterStateHealth( computeDelayedUnassignedShards += indexHealth.getDelayedUnassignedShards(); computeStatus = getClusterHealthStatus(indexHealth, computeStatus); - if (isIndexOrShardLevelHealthRequired) { - // Store ClusterIndexHealth only when the health is requested at Index or Shard level + if (isIndexOrShardLevelHealthRequired + || (isSearchOnlyClusterBlockEnabled(indexMetadata) && indexHealthLevel == ClusterHealthRequest.Level.SHARDS)) { + // Store ClusterIndexHealth when: + // 1. Health is requested at Index or Shard level, OR + // 2. This is a search_only index we're examining at SHARDS level indices.put(indexHealth.getIndex(), indexHealth); } } + Map searchOnlyIndices = collectSearchOnlyIndices(clusterState, concreteIndices, indices); + if (searchOnlyIndices.isEmpty() == false) { + for (ClusterIndexHealth indexHealth : searchOnlyIndices.values()) { + if (indexHealth.getUnassignedShards() > 0 && indexHealth.getActiveShards() == 0) { + computeStatus = ClusterHealthStatus.RED; + break; + } + } + } + if (clusterState.blocks().hasGlobalBlockWithStatus(RestStatus.SERVICE_UNAVAILABLE)) { computeStatus = ClusterHealthStatus.RED; } @@ -219,6 +252,46 @@ public ClusterStateHealth( } } + /** + * Checks if an index has search-only mode enabled. + * + * @param indexMetadata The index metadata + * @return true if search-only mode is enabled, false otherwise + */ + private static boolean isSearchOnlyClusterBlockEnabled(IndexMetadata indexMetadata) { + return indexMetadata.getSettings().getAsBoolean(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), false); + } + + /** + * Collects health information for search-only indices. + * + * @param clusterState The current cluster state + * @param concreteIndices Array of index names + * @param healthIndices Map of existing index health objects + * @return Map of index health objects for search-only indices + */ + private static Map collectSearchOnlyIndices( + final ClusterState clusterState, + final String[] concreteIndices, + final Map healthIndices + ) { + + Map searchOnlyIndices = new HashMap<>(); + for (String index : concreteIndices) { + IndexMetadata indexMetadata = clusterState.metadata().index(index); + if (indexMetadata == null) continue; + + if (isSearchOnlyClusterBlockEnabled(indexMetadata)) { + String indexName = indexMetadata.getIndex().getName(); + ClusterIndexHealth indexHealth = healthIndices.get(indexName); + if (indexHealth != null) { + searchOnlyIndices.put(indexName, indexHealth); + } + } + } + return searchOnlyIndices; + } + private static ClusterHealthStatus getClusterHealthStatus(ClusterIndexHealth indexHealth, ClusterHealthStatus computeStatus) { switch (indexHealth.getStatus()) { case RED: diff --git a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java index 9005c830167f9..8782e06e68419 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java @@ -166,6 +166,24 @@ public class IndexMetadata implements Diffable, ToXContentFragmen EnumSet.of(ClusterBlockLevel.METADATA_WRITE, ClusterBlockLevel.WRITE) ); + // Block ID and block for scale operations (IDs 20-29 reserved for scaling) + public static final int INDEX_SEARCH_ONLY_BLOCK_ID = 20; + + /** + * Permanent cluster block applied to indices in search-only mode. + *

+ * This block prevents write operations to the index while allowing read operations. + */ + public static final ClusterBlock INDEX_SEARCH_ONLY_BLOCK = new ClusterBlock( + INDEX_SEARCH_ONLY_BLOCK_ID, + "index scaled down", + false, + false, + false, + RestStatus.FORBIDDEN, + EnumSet.of(ClusterBlockLevel.WRITE) + ); + /** * The state of the index. * @@ -507,7 +525,8 @@ public enum APIBlock implements Writeable { READ("read", INDEX_READ_BLOCK), WRITE("write", INDEX_WRITE_BLOCK), METADATA("metadata", INDEX_METADATA_BLOCK), - READ_ONLY_ALLOW_DELETE("read_only_allow_delete", INDEX_READ_ONLY_ALLOW_DELETE_BLOCK); + READ_ONLY_ALLOW_DELETE("read_only_allow_delete", INDEX_READ_ONLY_ALLOW_DELETE_BLOCK), + SEARCH_ONLY("search_only", INDEX_SEARCH_ONLY_BLOCK); final String name; final String settingName; @@ -576,6 +595,8 @@ public static APIBlock readFrom(StreamInput input) throws IOException { public static final String SETTING_READ_ONLY_ALLOW_DELETE = APIBlock.READ_ONLY_ALLOW_DELETE.settingName(); public static final Setting INDEX_BLOCKS_READ_ONLY_ALLOW_DELETE_SETTING = APIBlock.READ_ONLY_ALLOW_DELETE.setting(); + public static final Setting INDEX_BLOCKS_SEARCH_ONLY_SETTING = APIBlock.SEARCH_ONLY.setting(); + public static final String SETTING_VERSION_CREATED = "index.version.created"; public static final Setting SETTING_INDEX_VERSION_CREATED = Setting.versionSetting( diff --git a/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java b/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java index 08574dddc007c..d1bb689672cca 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java +++ b/server/src/main/java/org/opensearch/cluster/routing/IndexRoutingTable.java @@ -122,11 +122,11 @@ public Index getIndex() { boolean validate(Metadata metadata) { // check index exists if (!metadata.hasIndex(index.getName())) { - throw new IllegalStateException(index + " exists in routing does not exists in metadata"); + throw new IllegalStateException(index + " exists in routing but does not exist in metadata"); } IndexMetadata indexMetadata = metadata.index(index.getName()); if (indexMetadata.getIndexUUID().equals(index.getUUID()) == false) { - throw new IllegalStateException(index.getName() + " exists in routing does not exists in metadata with the same uuid"); + throw new IllegalStateException(index.getName() + " exists in routing but does not exist in metadata with the same uuid"); } // check the number of shards @@ -141,27 +141,33 @@ boolean validate(Metadata metadata) { throw new IllegalStateException("Wrong number of shards in routing table, missing: " + expected); } - // check the replicas + boolean isSearchOnlyClusterBlockEnabled = indexMetadata.getSettings() + .getAsBoolean(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), false); + for (IndexShardRoutingTable indexShardRoutingTable : this) { int routingNumberOfReplicas = indexShardRoutingTable.size() - 1; - if (routingNumberOfReplicas != indexMetadata.getNumberOfReplicas() + indexMetadata.getNumberOfSearchOnlyReplicas()) { + int expectedReplicas = indexMetadata.getNumberOfReplicas() + indexMetadata.getNumberOfSearchOnlyReplicas(); + + // Only throw if we are NOT in search-only mode. Otherwise, we ignore or log the mismatch. + if (routingNumberOfReplicas != expectedReplicas && isSearchOnlyClusterBlockEnabled == false) { throw new IllegalStateException( "Shard [" + indexShardRoutingTable.shardId().id() + "] routing table has wrong number of replicas, expected [" - + "Replicas: " + + "Replicas: " + indexMetadata.getNumberOfReplicas() - + "Search Replicas: " + + ", Search Replicas: " + indexMetadata.getNumberOfSearchOnlyReplicas() + "], got [" + routingNumberOfReplicas + "]" ); } + for (ShardRouting shardRouting : indexShardRoutingTable) { if (!shardRouting.index().equals(index)) { throw new IllegalStateException( - "shard routing has an index [" + shardRouting.index() + "] that is different " + "from the routing table" + "shard routing has an index [" + shardRouting.index() + "] that is different from the routing table" ); } final Set inSyncAllocationIds = indexMetadata.inSyncAllocationIds(shardRouting.id()); @@ -185,8 +191,8 @@ boolean validate(Metadata metadata) { throw new IllegalStateException( "a primary shard routing " + shardRouting - + " is a primary that is recovering from a stale primary has unexpected allocation ids in in-sync " - + "allocation set " + + " is a primary that is recovering " + + "from a stale primary but has unexpected allocation ids in the in-sync set " + inSyncAllocationIds ); } @@ -194,8 +200,8 @@ boolean validate(Metadata metadata) { throw new IllegalStateException( "a primary shard routing " + shardRouting - + " is a primary that is recovering from a known allocation id but has no corresponding entry in the in-sync " - + "allocation set " + + " is a primary that is recovering " + + "from a known allocation id but has no corresponding entry in the in-sync set " + inSyncAllocationIds ); } @@ -425,6 +431,7 @@ public Builder initializeAsNew(IndexMetadata indexMetadata) { /** * Initializes an existing index. */ + public Builder initializeAsRecovery(IndexMetadata indexMetadata) { return initializeEmpty(indexMetadata, new UnassignedInfo(UnassignedInfo.Reason.CLUSTER_RECOVERED, null)); } @@ -602,6 +609,9 @@ private Builder initializeAsRestore( return this; } + /** + * Initializes a new empty index, with an option to control if its from an API or not. + */ /** * Initializes a new empty index, with an option to control if its from an API or not. */ @@ -610,6 +620,34 @@ private Builder initializeEmpty(IndexMetadata indexMetadata, UnassignedInfo unas if (!shards.isEmpty()) { throw new IllegalStateException("trying to initialize an index with fresh shards, but already has shards created"); } + + // Check if search-only mode is enabled + boolean isSearchOnly = indexMetadata.getSettings().getAsBoolean(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), false); + + // For search-only mode, only initialize search replicas + if (isSearchOnly) { + for (int shardNumber = 0; shardNumber < indexMetadata.getNumberOfShards(); shardNumber++) { + ShardId shardId = new ShardId(index, shardNumber); + IndexShardRoutingTable.Builder indexShardRoutingBuilder = new IndexShardRoutingTable.Builder(shardId); + + // Add only search replicas + for (int i = 0; i < indexMetadata.getNumberOfSearchOnlyReplicas(); i++) { + indexShardRoutingBuilder.addShard( + ShardRouting.newUnassigned( + shardId, + false, + true, + RecoverySource.EmptyStoreRecoverySource.INSTANCE, + unassignedInfo + ) + ); + } + shards.put(shardNumber, indexShardRoutingBuilder.build()); + } + return this; + } + + // Standard initialization for non-search-only mode for (int shardNumber = 0; shardNumber < indexMetadata.getNumberOfShards(); shardNumber++) { ShardId shardId = new ShardId(index, shardNumber); final RecoverySource primaryRecoverySource; diff --git a/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java b/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java index 6db70cc5f4fc5..4d2232afd9a5e 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java +++ b/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java @@ -128,7 +128,15 @@ public RoutingNodes(ClusterState clusterState, boolean readOnly) { // also fill replicaSet information for (final IndexRoutingTable indexRoutingTable : routingTable.indicesRouting().values()) { for (IndexShardRoutingTable indexShard : indexRoutingTable) { - assert indexShard.primary != null; + IndexMetadata idxMetadata = metadata.index(indexShard.shardId().getIndex()); + boolean isSearchOnlyClusterBlockEnabled = false; + if (idxMetadata != null) { + isSearchOnlyClusterBlockEnabled = idxMetadata.getSettings() + .getAsBoolean(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), false); + } + if (isSearchOnlyClusterBlockEnabled == false) { + assert indexShard.primary != null : "Primary shard routing can't be null for non-search-only indices"; + } for (ShardRouting shard : indexShard) { // to get all the shards belonging to an index, including the replicas, // we define a replica set and keep track of it. A replica set is identified @@ -184,8 +192,18 @@ private void updateRecoveryCounts(final ShardRouting routing, final boolean incr final int howMany = increment ? 1 : -1; assert routing.initializing() : "routing must be initializing: " + routing; + + IndexMetadata idxMetadata = metadata.index(routing.index()); + boolean isSearchOnlyClusterBlockEnabled = false; + if (idxMetadata != null) { + isSearchOnlyClusterBlockEnabled = idxMetadata.getSettings() + .getAsBoolean(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), false); + } + // TODO: check primary == null || primary.active() after all tests properly add ReplicaAfterPrimaryActiveAllocationDecider - assert primary == null || primary.assignedToNode() : "shard is initializing but its primary is not assigned to a node"; + if (isSearchOnlyClusterBlockEnabled == false) { + assert primary == null || primary.assignedToNode() : "shard is initializing but its primary is not assigned to a node"; + } // Primary shard routing, excluding the relocating primaries. if (routing.primary() && (primary == null || primary == routing)) { diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/ReplicaAfterPrimaryActiveAllocationDecider.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/ReplicaAfterPrimaryActiveAllocationDecider.java index 52b89f5c403e8..c516b247ee0b0 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/ReplicaAfterPrimaryActiveAllocationDecider.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/ReplicaAfterPrimaryActiveAllocationDecider.java @@ -32,6 +32,7 @@ package org.opensearch.cluster.routing.allocation.decider; +import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.routing.RoutingNode; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.allocation.RoutingAllocation; @@ -57,8 +58,17 @@ public Decision canAllocate(ShardRouting shardRouting, RoutingAllocation allocat } ShardRouting primary = allocation.routingNodes().activePrimary(shardRouting.shardId()); if (primary == null) { - return allocation.decision(Decision.NO, NAME, "primary shard for this replica is not yet active"); + boolean isSearchOnlyClusterBlockEnabled = allocation.metadata() + .getIndexSafe(shardRouting.index()) + .getSettings() + .getAsBoolean(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), false); + if (shardRouting.isSearchOnly() && isSearchOnlyClusterBlockEnabled) { + return allocation.decision(Decision.YES, NAME, "search only: both shard and index are marked search-only"); + } else { + return allocation.decision(Decision.NO, NAME, "primary shard for this replica is not yet active"); + } } + return allocation.decision(Decision.YES, NAME, "primary shard for this replica is already active"); } } 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 3793b9b09e3b2..5353ad9776522 100644 --- a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java @@ -109,6 +109,7 @@ public final class IndexScopedSettings extends AbstractScopedSettings { IndexMetadata.INDEX_BLOCKS_METADATA_SETTING, IndexMetadata.INDEX_BLOCKS_READ_ONLY_ALLOW_DELETE_SETTING, IndexMetadata.INDEX_PRIORITY_SETTING, + IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING, IndexMetadata.INDEX_DATA_PATH_SETTING, IndexMetadata.INDEX_FORMAT_SETTING, IndexMetadata.INDEX_HIDDEN_SETTING, diff --git a/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java b/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java index c30ee8479ac97..2844a378df571 100644 --- a/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/ReplicaShardAllocator.java @@ -55,6 +55,7 @@ import org.opensearch.indices.store.TransportNodesListShardStoreMetadataHelper.StoreFilesMetadata; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; @@ -248,14 +249,30 @@ protected AllocateUnassignedDecision getAllocationDecision( final RoutingNodes routingNodes = allocation.routingNodes(); final boolean explain = allocation.debugDecision(); ShardRouting primaryShard = routingNodes.activePrimary(unassignedShard.shardId()); + if (primaryShard == null) { - assert explain : "primary should only be null here if we are in explain mode, so we didn't " - + "exit early when canBeAllocatedToAtLeastOneNode didn't return a YES decision"; - return AllocateUnassignedDecision.no( - UnassignedInfo.AllocationStatus.fromDecision(allocationDecision.v1().type()), - new ArrayList<>(allocationDecision.v2().values()) - ); + // Determine if the index is configured for search-only. + + if (unassignedShard.isSearchOnly()) { + boolean isSearchOnlyClusterBlockEnabled = allocation.metadata() + .getIndexSafe(unassignedShard.index()) + .getSettings() + .getAsBoolean(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), false); + + if (isSearchOnlyClusterBlockEnabled) { + return getSearchReplicaAllocationDecision(unassignedShard, allocation); + } + } else { + // For non-search-only replicas, if there is no active primary we do not attempt an allocation. + assert explain : "primary should only be null here if we are in explain mode, so we didn't " + + "exit early when canBeAllocatedToAtLeastOneNode didn't return a YES decision"; + return AllocateUnassignedDecision.no( + UnassignedInfo.AllocationStatus.fromDecision(allocationDecision.v1().type()), + new ArrayList<>(allocationDecision.v2().values()) + ); + } } + assert primaryShard.currentNodeId() != null; final DiscoveryNode primaryNode = allocation.nodes().get(primaryShard.currentNodeId()); final StoreFilesMetadata primaryStore = findStore(primaryNode, nodeShardStores); @@ -331,6 +348,33 @@ protected AllocateUnassignedDecision getAllocationDecision( return AllocateUnassignedDecision.NOT_TAKEN; } + /** + * Handles allocation decisions for search-only replica shards + */ + private AllocateUnassignedDecision getSearchReplicaAllocationDecision(ShardRouting unassignedShard, RoutingAllocation allocation) { + // Obtain the collection of data nodes once + Collection dataNodes = allocation.nodes().getDataNodes().values(); + + // Use a stream to find the first candidate node where the allocation decider returns YES + DiscoveryNode selectedCandidate = dataNodes.stream() + .filter(candidate -> allocation.routingNodes().node(candidate.getId()) != null) + .filter(candidate -> { + RoutingNode node = allocation.routingNodes().node(candidate.getId()); + Decision decision = allocation.deciders().canAllocate(unassignedShard, node, allocation); + return decision.type() == Decision.Type.YES; + }) + .findFirst() + .orElse(null); + + // If a candidate was found, return a YES allocation decision + if (selectedCandidate != null) { + return AllocateUnassignedDecision.yes(selectedCandidate, null, new ArrayList<>(), false); + } + + // If there are no data nodes available, delay allocation + return AllocateUnassignedDecision.delayed(0L, 0L, null); + } + /** * Determines if the shard can be allocated on at least one node based on the allocation deciders. *

diff --git a/server/src/main/java/org/opensearch/index/recovery/RemoteStoreRestoreService.java b/server/src/main/java/org/opensearch/index/recovery/RemoteStoreRestoreService.java index 73a79a54ca588..9ddb84f58543e 100644 --- a/server/src/main/java/org/opensearch/index/recovery/RemoteStoreRestoreService.java +++ b/server/src/main/java/org/opensearch/index/recovery/RemoteStoreRestoreService.java @@ -49,7 +49,6 @@ import java.util.function.Function; import java.util.stream.Collectors; -import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_ENABLED; import static org.opensearch.common.util.IndexUtils.filterIndices; import static org.opensearch.repositories.blobstore.BlobStoreRepository.SYSTEM_REPOSITORY_SETTING; @@ -145,14 +144,13 @@ public RemoteRestoreResult restore( boolean metadataFromRemoteStore = (restoreClusterUUID == null || restoreClusterUUID.isEmpty() || restoreClusterUUID.isBlank()) == false; + if (metadataFromRemoteStore) { try { - // Restore with current cluster UUID will fail as same indices would be present in the cluster which we are trying to - // restore if (currentState.metadata().clusterUUID().equals(restoreClusterUUID)) { - throw new IllegalArgumentException("clusterUUID to restore from should be different from current cluster UUID"); + throw new IllegalArgumentException("Cluster UUID for restore must be different from the current cluster UUID."); } - logger.info("Restoring cluster state from remote store from cluster UUID : [{}]", restoreClusterUUID); + logger.info("Restoring cluster state from remote store for cluster UUID: [{}]", restoreClusterUUID); remoteState = remoteClusterStateService.getLatestClusterState( currentState.getClusterName().value(), restoreClusterUUID, @@ -170,13 +168,21 @@ public RemoteRestoreResult restore( indexNames, IndicesOptions.fromOptions(true, true, true, true) ); + for (String indexName : filteredIndices) { IndexMetadata indexMetadata = currentState.metadata().index(indexName); if (indexMetadata == null) { logger.warn("Index restore is not supported for non-existent index. Skipping: {}", indexName); - } else if (indexMetadata.getSettings().getAsBoolean(SETTING_REMOTE_STORE_ENABLED, false) == false) { - logger.warn("Remote store is not enabled for index: {}", indexName); - } else if (restoreAllShards && IndexMetadata.State.CLOSE.equals(indexMetadata.getState()) == false) { + continue; + } + boolean isSearchOnlyClusterBlockEnabled = indexMetadata.getSettings() + .getAsBoolean(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), false); + if (isSearchOnlyClusterBlockEnabled) { + throw new IllegalArgumentException( + String.format(Locale.ROOT, "Cannot use _remotestore/_restore on search_only mode enabled index [%s].", indexName) + ); + } + if (restoreAllShards && IndexMetadata.State.CLOSE.equals(indexMetadata.getState()) == false) { throw new IllegalStateException( String.format( Locale.ROOT, @@ -184,9 +190,8 @@ public RemoteRestoreResult restore( indexName ) + " Close the existing index." ); - } else { - indexMetadataMap.put(indexName, new Tuple<>(false, indexMetadata)); } + indexMetadataMap.put(indexName, new Tuple<>(false, indexMetadata)); } } return executeRestore(currentState, indexMetadataMap, restoreAllShards, remoteState); diff --git a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java index 30d9c362b6269..bb79fab32cf9f 100644 --- a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java @@ -67,13 +67,32 @@ public void getCheckpointMetadata( try (final GatedCloseable segmentInfosSnapshot = indexShard.getSegmentInfosSnapshot()) { final Version version = segmentInfosSnapshot.get().getCommitLuceneVersion(); final RemoteSegmentMetadata mdFile = getRemoteSegmentMetadata(); - // During initial recovery flow, the remote store might not - // have metadata as primary hasn't uploaded anything yet. - if (mdFile == null && indexShard.state().equals(IndexShardState.STARTED) == false) { - listener.onResponse(new CheckpointInfoResponse(checkpoint, Collections.emptyMap(), null)); - return; + + // Handle null metadata file case + if (mdFile == null) { + // During initial recovery flow, the remote store might not + // have metadata as primary hasn't uploaded anything yet. + if (indexShard.state().equals(IndexShardState.STARTED) == false) { + // Non-started shard during recovery + listener.onResponse(new CheckpointInfoResponse(checkpoint, Collections.emptyMap(), null)); + return; + } else if (indexShard.routingEntry().isSearchOnly()) { + // Allow search-only replicas to become active without metadata + logger.debug("Search-only replica proceeding without remote metadata: {}", indexShard.shardId()); + listener.onResponse( + new CheckpointInfoResponse(indexShard.getLatestReplicationCheckpoint(), Collections.emptyMap(), null) + ); + return; + } else { + // Regular replicas should not be active without metadata + listener.onFailure( + new IllegalStateException("Remote metadata file can't be null if shard is active: " + indexShard.shardId()) + ); + return; + } } - assert mdFile != null : "Remote metadata file can't be null if shard is active " + indexShard.state(); + + // Process metadata when it exists metadataMap = mdFile.getMetadata() .entrySet() .stream() diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java index b8a5774c21c1f..c35898cfffe1e 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicator.java @@ -345,7 +345,7 @@ SegmentReplicationTarget get(ShardId shardId) { } ReplicationCheckpoint getPrimaryCheckpoint(ShardId shardId) { - return primaryCheckpoint.getOrDefault(shardId, ReplicationCheckpoint.empty(shardId)); + return primaryCheckpoint.get(shardId); } ReplicationCollection.ReplicationRef get(long id) { diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestScaleIndexAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestScaleIndexAction.java new file mode 100644 index 0000000000000..c36b3c16198c9 --- /dev/null +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestScaleIndexAction.java @@ -0,0 +1,88 @@ +/* + * 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.rest.action.admin.indices; + +import org.opensearch.rest.BaseRestHandler; +import org.opensearch.rest.RestRequest; +import org.opensearch.rest.action.RestToXContentListener; +import org.opensearch.transport.client.node.NodeClient; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import static java.util.Arrays.asList; +import static java.util.Collections.unmodifiableList; +import static org.opensearch.rest.RestRequest.Method.POST; + +/** + * Rest action for scaling index operations + * + * @opensearch.internal + */ +public class RestScaleIndexAction extends BaseRestHandler { + + private static final String SEARCH_ONLY_FIELD = "search_only"; + + @Override + public List routes() { + return unmodifiableList(asList(new Route(POST, "/{index}/_scale"))); + } + + @Override + public String getName() { + return "search_only_index_action"; + } + + @Override + protected RestChannelConsumer prepareRequest(final RestRequest request, NodeClient client) throws IOException { + String index = request.param("index"); + if (index == null || index.trim().isEmpty()) { + throw new IllegalArgumentException("index is required"); + } + + // Parse the request body first to get the scale down value + final boolean searchOnly = parseSearchOnlyValue(request); + + // Then use the final value in the lambda + return channel -> client.admin().indices().prepareScaleSearchOnly(index, searchOnly).execute(new RestToXContentListener<>(channel)); + } + + /** + * Parses and validates the search_only parameter from the request body. + */ + private boolean parseSearchOnlyValue(RestRequest request) { + try { + Map source; + try { + source = request.contentParser().map(); + } catch (Exception e) { + throw new IllegalArgumentException("Request body must be valid JSON", e); + } + for (String key : source.keySet()) { + if (SEARCH_ONLY_FIELD.equals(key) == false) { + throw new IllegalArgumentException("Unknown parameter [" + key + "]. Only [" + SEARCH_ONLY_FIELD + "] is allowed."); + } + } + if (source.containsKey(SEARCH_ONLY_FIELD) == false) { + throw new IllegalArgumentException("Parameter [" + SEARCH_ONLY_FIELD + "] is required"); + } + Object value = source.get(SEARCH_ONLY_FIELD); + if ((value instanceof Boolean) == false) { + throw new IllegalArgumentException("Parameter [" + SEARCH_ONLY_FIELD + "] must be a boolean (true or false)"); + } + return (Boolean) value; + } catch (Exception e) { + if (e instanceof IllegalArgumentException) { + throw e; + } + throw new IllegalArgumentException("Request body must be valid JSON", e); + } + } +} diff --git a/server/src/main/java/org/opensearch/transport/client/IndicesAdminClient.java b/server/src/main/java/org/opensearch/transport/client/IndicesAdminClient.java index 2beec71785d91..6b8d168ecbbda 100644 --- a/server/src/main/java/org/opensearch/transport/client/IndicesAdminClient.java +++ b/server/src/main/java/org/opensearch/transport/client/IndicesAdminClient.java @@ -92,6 +92,7 @@ import org.opensearch.action.admin.indices.rollover.RolloverRequest; import org.opensearch.action.admin.indices.rollover.RolloverRequestBuilder; import org.opensearch.action.admin.indices.rollover.RolloverResponse; +import org.opensearch.action.admin.indices.scale.searchonly.ScaleIndexRequestBuilder; import org.opensearch.action.admin.indices.segments.IndicesSegmentResponse; import org.opensearch.action.admin.indices.segments.IndicesSegmentsRequest; import org.opensearch.action.admin.indices.segments.IndicesSegmentsRequestBuilder; @@ -889,4 +890,13 @@ public interface IndicesAdminClient extends OpenSearchClient { /** Get ingestion state */ void getIngestionState(GetIngestionStateRequest request, ActionListener listener); + + /** + * Prepares a request to scale an index between normal and search-only modes. + * + * @param index The name of the index to scale + * @param searchOnly Whether to scale to search-only mode (true) or back to normal mode (false) + * @return The request builder configured with the specified scaling direction + */ + ScaleIndexRequestBuilder prepareScaleSearchOnly(String index, boolean searchOnly); } diff --git a/server/src/main/java/org/opensearch/transport/client/support/AbstractClient.java b/server/src/main/java/org/opensearch/transport/client/support/AbstractClient.java index 317613fd2a86e..9c408a82402b5 100644 --- a/server/src/main/java/org/opensearch/transport/client/support/AbstractClient.java +++ b/server/src/main/java/org/opensearch/transport/client/support/AbstractClient.java @@ -268,6 +268,7 @@ import org.opensearch.action.admin.indices.rollover.RolloverRequest; import org.opensearch.action.admin.indices.rollover.RolloverRequestBuilder; import org.opensearch.action.admin.indices.rollover.RolloverResponse; +import org.opensearch.action.admin.indices.scale.searchonly.ScaleIndexRequestBuilder; import org.opensearch.action.admin.indices.segments.IndicesSegmentResponse; import org.opensearch.action.admin.indices.segments.IndicesSegmentsAction; import org.opensearch.action.admin.indices.segments.IndicesSegmentsRequest; @@ -2150,6 +2151,10 @@ public void updateView(CreateViewAction.Request request, ActionListener updateView(CreateViewAction.Request request) { return execute(UpdateViewAction.INSTANCE, request); diff --git a/server/src/test/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexClusterStateBuilderTests.java b/server/src/test/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexClusterStateBuilderTests.java new file mode 100644 index 0000000000000..9e0ce2a339a66 --- /dev/null +++ b/server/src/test/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexClusterStateBuilderTests.java @@ -0,0 +1,169 @@ +/* + * 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.scale.searchonly; + +import org.opensearch.Version; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.block.ClusterBlock; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.routing.IndexRoutingTable; +import org.opensearch.cluster.routing.RecoverySource; +import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.routing.UnassignedInfo; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.index.Index; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.HashMap; +import java.util.Map; + +import static org.opensearch.cluster.metadata.IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING; +import static org.opensearch.cluster.metadata.IndexMetadata.INDEX_SEARCH_ONLY_BLOCK; +import static org.opensearch.cluster.metadata.IndexMetadata.INDEX_SEARCH_ONLY_BLOCK_ID; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_CREATION_DATE; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_INDEX_UUID; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SEARCH_REPLICAS; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_ENABLED; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_VERSION_CREATED; +import static org.opensearch.cluster.routing.ShardRoutingState.UNASSIGNED; + +public class ScaleIndexClusterStateBuilderTests extends OpenSearchTestCase { + + private ScaleIndexClusterStateBuilder builder; + private ClusterState initialState; + private String testIndex; + private IndexMetadata indexMetadata; + + @Override + public void setUp() throws Exception { + super.setUp(); + builder = new ScaleIndexClusterStateBuilder(); + testIndex = "test_index"; + + // Create basic index metadata with segment replication enabled + Settings indexSettings = Settings.builder() + .put(SETTING_VERSION_CREATED, Version.CURRENT) + .put(SETTING_INDEX_UUID, randomAlphaOfLength(8)) + .put(SETTING_NUMBER_OF_SHARDS, 1) + .put(SETTING_NUMBER_OF_REPLICAS, 1) + .put(SETTING_NUMBER_OF_SEARCH_REPLICAS, 1) // Add search replicas + .put(SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) // Enable segment replication + .put(SETTING_REMOTE_STORE_ENABLED, true) // Enable remote store + .put(SETTING_CREATION_DATE, System.currentTimeMillis()) + .build(); + + indexMetadata = IndexMetadata.builder(testIndex).settings(indexSettings).build(); + + // Create initial cluster state with routing table + Metadata metadata = Metadata.builder().put(indexMetadata, true).build(); + + initialState = ClusterState.builder(ClusterState.EMPTY_STATE) + .metadata(metadata) + .routingTable(RoutingTable.builder().addAsNew(indexMetadata).build()) + .build(); + } + + public void testBuildScaleDownState() { + Map blockedIndices = new HashMap<>(); + + // Execute scale down state build + ClusterState newState = builder.buildScaleDownState(initialState, testIndex, blockedIndices); + + // Verify block was added + assertTrue("Scale down block should be present", newState.blocks().hasIndexBlockWithId(testIndex, INDEX_SEARCH_ONLY_BLOCK_ID)); + + // Verify blocked indices map was updated + assertFalse("Blocked indices map should not be empty", blockedIndices.isEmpty()); + assertEquals("Should have one blocked index", 1, blockedIndices.size()); + assertTrue("Index should be in blocked indices map", blockedIndices.containsKey(indexMetadata.getIndex())); + } + + public void testBuildFinalScaleDownState() { + Map blockedIndices = new HashMap<>(); + ClusterState stateWithBlock = builder.buildScaleDownState(initialState, testIndex, blockedIndices); + + ClusterState finalState = builder.buildFinalScaleDownState(stateWithBlock, testIndex); + + // Verify blocks + assertFalse( + "Temporary block should be removed", + finalState.blocks().hasIndexBlock(testIndex, blockedIndices.get(indexMetadata.getIndex())) + ); + assertTrue("Search-only block should be present", finalState.blocks().hasIndexBlock(testIndex, INDEX_SEARCH_ONLY_BLOCK)); + + // Verify metadata was updated + IndexMetadata updatedMetadata = finalState.metadata().index(testIndex); + assertTrue( + "Index should be marked as search-only", + updatedMetadata.getSettings().getAsBoolean(INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), false) + ); + } + + public void testBuildScaleUpRoutingTable() { + // Prepare a proper search-only state + Settings scaleUpSettings = Settings.builder() + .put(indexMetadata.getSettings()) + .put(INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), true) + .build(); + + IndexMetadata searchOnlyMetadata = IndexMetadata.builder(indexMetadata).settings(scaleUpSettings).build(); + + // Create search-only shard routing + ShardRouting searchOnlyShard = ShardRouting.newUnassigned( + new ShardId(searchOnlyMetadata.getIndex(), 0), + false, // not primary + true, // search only + RecoverySource.EmptyStoreRecoverySource.INSTANCE, + new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "test") + ); + + // Build routing table with search-only shard + IndexRoutingTable.Builder routingTableBuilder = new IndexRoutingTable.Builder(searchOnlyMetadata.getIndex()).addShard( + searchOnlyShard + ); + + ClusterState searchOnlyState = ClusterState.builder(initialState) + .metadata(Metadata.builder(initialState.metadata()).put(searchOnlyMetadata, true)) + .routingTable(RoutingTable.builder().add(routingTableBuilder.build()).build()) + .build(); + + // Execute scale up + RoutingTable newRoutingTable = builder.buildScaleUpRoutingTable(searchOnlyState, testIndex); + + // Verify routing table + IndexRoutingTable indexRoutingTable = newRoutingTable.index(testIndex); + assertNotNull("Index routing table should exist", indexRoutingTable); + + // Verify primary shard was added + boolean hasPrimary = indexRoutingTable.shardsWithState(UNASSIGNED).stream().anyMatch(ShardRouting::primary); + assertTrue("Should have an unassigned primary shard", hasPrimary); + + // Verify regular replicas were added (excluding search replicas) + long replicaCount = indexRoutingTable.shardsWithState(UNASSIGNED) + .stream() + .filter(shard -> !shard.primary() && !shard.isSearchOnly()) + .count(); + assertEquals("Should have correct number of replica shards", indexMetadata.getNumberOfReplicas(), replicaCount); + + // Verify search replicas were preserved + long searchReplicaCount = indexRoutingTable.shardsWithState(UNASSIGNED).stream().filter(ShardRouting::isSearchOnly).count(); + assertEquals("Should preserve search replicas", indexMetadata.getNumberOfSearchOnlyReplicas(), searchReplicaCount); + } + + public void testBuildFinalScaleDownStateWithInvalidIndex() { + expectThrows(IllegalStateException.class, () -> builder.buildFinalScaleDownState(initialState, "nonexistent_index")); + } +} diff --git a/server/src/test/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexNodeRequestTests.java b/server/src/test/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexNodeRequestTests.java new file mode 100644 index 0000000000000..f2974c97f1e24 --- /dev/null +++ b/server/src/test/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexNodeRequestTests.java @@ -0,0 +1,100 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.action.admin.indices.scale.searchonly; + +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.index.Index; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +public class ScaleIndexNodeRequestTests extends OpenSearchTestCase { + + public void testConstructorAndGetters() { + String indexName = "test_index"; + List shardIds = createTestShardIds(indexName, 3); + + ScaleIndexNodeRequest request = new ScaleIndexNodeRequest(indexName, shardIds); + + assertEquals("Index name should match", indexName, request.getIndex()); + assertEquals("Shard IDs should match", shardIds, request.getShardIds()); + } + + public void testSerializationRoundTrip() throws IOException { + String indexName = "test_index"; + List shardIds = createTestShardIds(indexName, 3); + + ScaleIndexNodeRequest originalRequest = new ScaleIndexNodeRequest(indexName, shardIds); + + BytesStreamOutput output = new BytesStreamOutput(); + originalRequest.writeTo(output); + + StreamInput input = output.bytes().streamInput(); + ScaleIndexNodeRequest deserializedRequest = new ScaleIndexNodeRequest(input); + + assertEquals("Index name should survive serialization", originalRequest.getIndex(), deserializedRequest.getIndex()); + assertEquals("Shard IDs should survive serialization", originalRequest.getShardIds(), deserializedRequest.getShardIds()); + } + + public void testSerializationWithEmptyShardList() throws IOException { + String indexName = "test_index"; + List emptyShardIds = new ArrayList<>(); + + ScaleIndexNodeRequest originalRequest = new ScaleIndexNodeRequest(indexName, emptyShardIds); + + BytesStreamOutput output = new BytesStreamOutput(); + originalRequest.writeTo(output); + + StreamInput input = output.bytes().streamInput(); + ScaleIndexNodeRequest deserializedRequest = new ScaleIndexNodeRequest(input); + + assertEquals("Index name should survive serialization", originalRequest.getIndex(), deserializedRequest.getIndex()); + assertTrue("Empty shard list should survive serialization", deserializedRequest.getShardIds().isEmpty()); + } + + public void testSerializationWithMultipleShards() throws IOException { + String indexName = "test_index"; + List shardIds = createTestShardIds(indexName, 5); + + ScaleIndexNodeRequest originalRequest = new ScaleIndexNodeRequest(indexName, shardIds); + + BytesStreamOutput output = new BytesStreamOutput(); + originalRequest.writeTo(output); + + StreamInput input = output.bytes().streamInput(); + ScaleIndexNodeRequest deserializedRequest = new ScaleIndexNodeRequest(input); + + assertEquals( + "Should have correct number of shards after deserialization", + shardIds.size(), + deserializedRequest.getShardIds().size() + ); + + for (int i = 0; i < shardIds.size(); i++) { + ShardId original = shardIds.get(i); + ShardId deserialized = deserializedRequest.getShardIds().get(i); + + assertEquals("Shard ID should match", original.id(), deserialized.id()); + assertEquals("Index name should match", original.getIndexName(), deserialized.getIndexName()); + } + } + + private List createTestShardIds(String indexName, int count) { + List shardIds = new ArrayList<>(count); + Index index = new Index(indexName, "uuid"); + for (int i = 0; i < count; i++) { + shardIds.add(new ShardId(index, i)); + } + return shardIds; + } +} diff --git a/server/src/test/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexOperationValidatorTests.java b/server/src/test/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexOperationValidatorTests.java new file mode 100644 index 0000000000000..71976f17499df --- /dev/null +++ b/server/src/test/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexOperationValidatorTests.java @@ -0,0 +1,159 @@ +/* + * 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.scale.searchonly; + +import org.opensearch.Version; +import org.opensearch.action.support.clustermanager.AcknowledgedResponse; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.action.ActionListener; +import org.opensearch.test.OpenSearchTestCase; + +import org.mockito.ArgumentMatcher; +import org.mockito.Mockito; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.argThat; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; + +public class ScaleIndexOperationValidatorTests extends OpenSearchTestCase { + + private ScaleIndexOperationValidator validator; + private ActionListener listener; + + @Override + public void setUp() throws Exception { + super.setUp(); + validator = new ScaleIndexOperationValidator(); + // Create a mock listener so we can verify onFailure is called with the expected exception. + listener = Mockito.mock(ActionListener.class); + } + + public void testValidateScalePrerequisites_NullIndexMetadata() { + // When index metadata is null, validation should fail. + boolean result = validator.validateScalePrerequisites(null, "test-index", listener, true); + assertFalse(result); + verify(listener).onFailure(argThat(new ExceptionMatcher("Index [test-index] not found"))); + } + + public void testValidateScalePrerequisites_ScaleDown_AlreadySearchOnly() { + // For scale-down, if the index is already marked as search-only, validation should fail. + Settings settings = Settings.builder().put(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), true).build(); + IndexMetadata indexMetadata = createTestIndexMetadata("test-index", settings, 1); + boolean result = validator.validateScalePrerequisites(indexMetadata, "test-index", listener, true); + assertFalse(result); + verify(listener).onFailure(argThat(new ExceptionMatcher("already in search-only mode"))); + } + + public void testValidateScalePrerequisites_ScaleDown_NoSearchOnlyReplicas() { + // If there are zero search-only replicas, validation should fail. + Settings settings = Settings.builder() + .put(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), false) + .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) + .put(IndexMetadata.SETTING_REPLICATION_TYPE, "SEGMENT") + .build(); + // Pass zero for the number of search-only replicas. + IndexMetadata indexMetadata = createTestIndexMetadata("test-index", settings, 0); + boolean result = validator.validateScalePrerequisites(indexMetadata, "test-index", listener, true); + assertFalse(result); + verify(listener).onFailure(argThat(new ExceptionMatcher("Cannot scale to zero without search replicas"))); + } + + public void testValidateScalePrerequisites_ScaleDown_RemoteStoreNotEnabled() { + // If remote store is not enabled, validation should fail. + Settings settings = Settings.builder() + .put(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), false) + .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, false) + .put(IndexMetadata.SETTING_REPLICATION_TYPE, "SEGMENT") + .build(); + IndexMetadata indexMetadata = createTestIndexMetadata("test-index", settings, 1); + boolean result = validator.validateScalePrerequisites(indexMetadata, "test-index", listener, true); + assertFalse(result); + verify(listener).onFailure(argThat(new ExceptionMatcher(IndexMetadata.SETTING_REMOTE_STORE_ENABLED))); + } + + public void testValidateScalePrerequisites_ScaleDown_InvalidReplicationType() { + // If the replication type is not SEGMENT, validation should fail. + Settings settings = Settings.builder() + .put(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), false) + .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) + .put(IndexMetadata.SETTING_REPLICATION_TYPE, "OTHER") + .build(); + IndexMetadata indexMetadata = createTestIndexMetadata("test-index", settings, 1); + boolean result = validator.validateScalePrerequisites(indexMetadata, "test-index", listener, true); + assertFalse(result); + verify(listener).onFailure(argThat(new ExceptionMatcher("segment replication must be enabled"))); + } + + public void testValidateScalePrerequisites_ScaleDown_Valid() { + // All prerequisites for scaling down are met. + Settings settings = Settings.builder() + .put(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), false) + .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) + .put(IndexMetadata.SETTING_REPLICATION_TYPE, "SEGMENT") + .build(); + IndexMetadata indexMetadata = createTestIndexMetadata("test-index", settings, 1); + boolean result = validator.validateScalePrerequisites(indexMetadata, "test-index", listener, true); + assertTrue(result); + verify(listener, never()).onFailure(any()); + } + + public void testValidateScalePrerequisites_ScaleUp_NotSearchOnly() { + // For scale-up, the index must be in search-only mode. + Settings settings = Settings.builder().put(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), false).build(); + IndexMetadata indexMetadata = createTestIndexMetadata("test-index", settings, 1); + boolean result = validator.validateScalePrerequisites(indexMetadata, "test-index", listener, false); + assertFalse(result); + verify(listener).onFailure(argThat(new ExceptionMatcher("not in search-only mode"))); + } + + public void testValidateScalePrerequisites_ScaleUp_Valid() { + // Valid scale-up: the index is in search-only mode. + Settings settings = Settings.builder().put(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), true).build(); + IndexMetadata indexMetadata = createTestIndexMetadata("test-index", settings, 1); + boolean result = validator.validateScalePrerequisites(indexMetadata, "test-index", listener, false); + assertTrue(result); + verify(listener, never()).onFailure(any()); + } + + /** + * Helper method to create a dummy IndexMetadata. + * Adjust this helper to match your actual IndexMetadata builder. + */ + private IndexMetadata createTestIndexMetadata(String indexName, Settings settings, int searchOnlyReplicas) { + Settings updatedSettings = Settings.builder() + .put(settings) + // Add the required index version setting. You can use a hardcoded value or Version.CURRENT.toString() + .put("index.version.created", Version.CURRENT) + .build(); + return IndexMetadata.builder(indexName) + .settings(updatedSettings) + .numberOfShards(1) + .numberOfReplicas(1) + .numberOfSearchReplicas(searchOnlyReplicas) + .build(); + } + + /** + * A custom ArgumentMatcher to check that an exception’s message contains a given substring. + */ + private static class ExceptionMatcher implements ArgumentMatcher { + private final String substring; + + ExceptionMatcher(String substring) { + this.substring = substring; + } + + @Override + public boolean matches(Exception e) { + return e != null && e.getMessage() != null && e.getMessage().contains(substring); + } + } +} diff --git a/server/src/test/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexRequestTests.java b/server/src/test/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexRequestTests.java new file mode 100644 index 0000000000000..c34de67290edf --- /dev/null +++ b/server/src/test/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexRequestTests.java @@ -0,0 +1,56 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.action.admin.indices.scale.searchonly; + +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; + +public class ScaleIndexRequestTests extends OpenSearchTestCase { + + public void testSerialization() throws IOException { + ScaleIndexRequest request = new ScaleIndexRequest("test_index", true); + + BytesStreamOutput out = new BytesStreamOutput(); + request.writeTo(out); + + StreamInput in = out.bytes().streamInput(); + ScaleIndexRequest deserializedRequest = new ScaleIndexRequest(in); + + assertEquals(request.getIndex(), deserializedRequest.getIndex()); + assertEquals(request.isScaleDown(), deserializedRequest.isScaleDown()); + } + + public void testValidation() { + ScaleIndexRequest request = new ScaleIndexRequest(null, true); + assertNotNull(request.validate()); + + request = new ScaleIndexRequest("", true); + assertNotNull(request.validate()); + + request = new ScaleIndexRequest(" ", true); + assertNotNull(request.validate()); + + request = new ScaleIndexRequest("test_index", true); + assertNull(request.validate()); + } + + public void testEquals() { + ScaleIndexRequest request1 = new ScaleIndexRequest("test_index", true); + ScaleIndexRequest request2 = new ScaleIndexRequest("test_index", true); + ScaleIndexRequest request3 = new ScaleIndexRequest("other_index", true); + ScaleIndexRequest request4 = new ScaleIndexRequest("test_index", false); + + assertEquals(request1, request2); + assertNotEquals(request1, request3); + assertNotEquals(request1, request4); + } +} diff --git a/server/src/test/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexResponseTests.java b/server/src/test/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexResponseTests.java new file mode 100644 index 0000000000000..ba377e5e887be --- /dev/null +++ b/server/src/test/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexResponseTests.java @@ -0,0 +1,234 @@ +/* + * 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.scale.searchonly; + +import org.opensearch.Version; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodeRole; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.common.transport.TransportAddress; +import org.opensearch.core.index.Index; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.test.OpenSearchTestCase; + +import java.net.InetAddress; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Set; + +public class ScaleIndexResponseTests extends OpenSearchTestCase { + + private DiscoveryNode createTestNode() throws Exception { + return new DiscoveryNode( + "test_node", + "test_node_id", + new TransportAddress(InetAddress.getByName("127.0.0.1"), 9300), + Collections.emptyMap(), + Set.of(DiscoveryNodeRole.DATA_ROLE), + Version.CURRENT + ); + } + + public void testSuccessfulResponse() throws Exception { + // Create test node and responses with no failures + DiscoveryNode node = createTestNode(); + List shardResponses = new ArrayList<>(); + + // Add successful shard responses + shardResponses.add( + new ScaleIndexShardResponse( + new ShardId(new Index("test_index", "test_uuid"), 0), + false, // doesn't need sync + 0 // no uncommitted operations + ) + ); + + List nodeResponses = Collections.singletonList(new ScaleIndexNodeResponse(node, shardResponses)); + + ScaleIndexResponse response = new ScaleIndexResponse(nodeResponses); + + // Verify response state + assertFalse("Response should not have failures", response.hasFailures()); + assertNull("Failure reason should be null", response.buildFailureReason()); + } + + public void testResponseWithFailures() throws Exception { + DiscoveryNode node = createTestNode(); + List shardResponses = new ArrayList<>(); + + // Create an Index instance + Index index = new Index("test_index", "test_uuid"); + + // Add a failed shard response (needs sync) + ShardId shardId0 = new ShardId(index, 0); + shardResponses.add( + new ScaleIndexShardResponse( + shardId0, + true, // needs sync + 0 // no uncommitted operations + ) + ); + + // Add another failed shard response (has uncommitted operations) + ShardId shardId1 = new ShardId(index, 1); + shardResponses.add( + new ScaleIndexShardResponse( + shardId1, + false, // doesn't need sync + 5 // has uncommitted operations + ) + ); + + List nodeResponses = Collections.singletonList(new ScaleIndexNodeResponse(node, shardResponses)); + + ScaleIndexResponse response = new ScaleIndexResponse(nodeResponses); + + // Verify response state + assertTrue("Response should have failures", response.hasFailures()); + assertNotNull("Failure reason should not be null", response.buildFailureReason()); + String failureReason = response.buildFailureReason(); + + // Verify the exact shard IDs appear in the failure reason + assertTrue("Failure reason should mention shard 0", failureReason.contains("Shard " + shardId0)); + assertTrue("Failure reason should mention shard 1", failureReason.contains("Shard " + shardId1)); + assertTrue("Failure reason should mention sync needed", failureReason.contains("needs sync")); + assertTrue("Failure reason should mention uncommitted operations", failureReason.contains("has uncommitted operations")); + } + + public void testSerialization() throws Exception { + DiscoveryNode node = createTestNode(); + List shardResponses = new ArrayList<>(); + + // Add mixed success/failure responses + shardResponses.add( + new ScaleIndexShardResponse( + new ShardId(new Index("test_index", "test_uuid"), 0), + false, // doesn't need sync + 0 // no uncommitted operations + ) + ); + shardResponses.add( + new ScaleIndexShardResponse( + new ShardId(new Index("test_index", "test_uuid"), 1), + true, // needs sync + 3 // has uncommitted operations + ) + ); + + List nodeResponses = Collections.singletonList(new ScaleIndexNodeResponse(node, shardResponses)); + + ScaleIndexResponse originalResponse = new ScaleIndexResponse(nodeResponses); + + // Serialize + BytesStreamOutput output = new BytesStreamOutput(); + originalResponse.writeTo(output); + + // Deserialize - first read the node responses + StreamInput input = output.bytes().streamInput(); + List deserializedNodeResponses = input.readList(ScaleIndexNodeResponse::new); + ScaleIndexResponse deserializedResponse = new ScaleIndexResponse(deserializedNodeResponses); + + // Verify serialization preserved state + assertEquals("Failure state should match after serialization", originalResponse.hasFailures(), deserializedResponse.hasFailures()); + assertEquals( + "Failure reason should match after serialization", + originalResponse.buildFailureReason(), + deserializedResponse.buildFailureReason() + ); + } + + public void testToXContent() throws Exception { + DiscoveryNode node = createTestNode(); + List shardResponses = new ArrayList<>(); + + // Add a failed shard response + shardResponses.add( + new ScaleIndexShardResponse( + new ShardId(new Index("test_index", "test_uuid"), 0), + true, // needs sync + 2 // has uncommitted operations + ) + ); + + List nodeResponses = Collections.singletonList(new ScaleIndexNodeResponse(node, shardResponses)); + + ScaleIndexResponse response = new ScaleIndexResponse(nodeResponses); + + // Convert to XContent + XContentBuilder builder = XContentFactory.jsonBuilder(); + response.toXContent(builder, null); + String json = builder.toString(); + + // Verify XContent output contains only the fields defined in toXContent() + assertTrue("XContent should contain failure_reason field", json.contains("\"failure_reason\"")); + // The failure reason will contain details about the failure + assertTrue("XContent should contain failure details", json.contains("Shard") && json.contains("needs sync")); + } + + public void testEmptyResponse() throws Exception { + // Create response with empty node responses + ScaleIndexResponse response = new ScaleIndexResponse(Collections.emptyList()); + + // Verify empty response state + assertFalse("Empty response should not have failures", response.hasFailures()); + assertNull("Empty response should have null failure reason", response.buildFailureReason()); + } + + public void testMultiNodeResponse() throws Exception { + List nodeResponses = new ArrayList<>(); + + // Create two nodes + DiscoveryNode node1 = createTestNode(); + DiscoveryNode node2 = new DiscoveryNode( + "test_node2", + "test_node_id2", + new TransportAddress(InetAddress.getByName("127.0.0.2"), 9300), + Collections.emptyMap(), + Set.of(DiscoveryNodeRole.DATA_ROLE), + Version.CURRENT + ); + + // Create index and shards + Index index = new Index("test_index", "test_uuid"); + ShardId shardId0 = new ShardId(index, 0); + ShardId shardId1 = new ShardId(index, 1); + + // Add responses from both nodes + List shardResponses1 = Collections.singletonList( + new ScaleIndexShardResponse( + shardId0, + false, // doesn't need sync + 0 // no uncommitted operations + ) + ); + + List shardResponses2 = Collections.singletonList( + new ScaleIndexShardResponse( + shardId1, + true, // needs sync + 0 // no uncommitted operations + ) + ); + + nodeResponses.add(new ScaleIndexNodeResponse(node1, shardResponses1)); + nodeResponses.add(new ScaleIndexNodeResponse(node2, shardResponses2)); + + ScaleIndexResponse response = new ScaleIndexResponse(nodeResponses); + + // Verify multi-node response + assertTrue("Response should have failures due to node2", response.hasFailures()); + String failureReason = response.buildFailureReason(); + assertTrue("Failure reason should mention node2", failureReason.contains("test_node2")); + assertTrue("Failure reason should mention shard 1", failureReason.contains("Shard " + shardId1)); + } +} diff --git a/server/src/test/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexShardResponseTests.java b/server/src/test/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexShardResponseTests.java new file mode 100644 index 0000000000000..6ec24514cfcf5 --- /dev/null +++ b/server/src/test/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexShardResponseTests.java @@ -0,0 +1,104 @@ +/* + * 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.scale.searchonly; + +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.core.common.io.stream.StreamInput; +import org.opensearch.core.index.Index; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; + +public class ScaleIndexShardResponseTests extends OpenSearchTestCase { + + public void testConstructorAndGetters() { + ShardId shardId = createTestShardId(); + boolean needsSync = randomBoolean(); + int uncommittedOps = randomIntBetween(0, 100); + + ScaleIndexShardResponse response = new ScaleIndexShardResponse(shardId, needsSync, uncommittedOps); + + assertEquals("Shard ID should match", shardId, response.getShardId()); + assertEquals("Needs sync flag should match", needsSync, response.needsSync()); + assertEquals("Uncommitted operations status should match", uncommittedOps > 0, response.hasUncommittedOperations()); + } + + public void testSerializationRoundTrip() throws IOException { + ShardId shardId = createTestShardId(); + boolean needsSync = randomBoolean(); + int uncommittedOps = randomIntBetween(0, 100); + + ScaleIndexShardResponse originalResponse = new ScaleIndexShardResponse(shardId, needsSync, uncommittedOps); + + BytesStreamOutput output = new BytesStreamOutput(); + originalResponse.writeTo(output); + + StreamInput input = output.bytes().streamInput(); + ScaleIndexShardResponse deserializedResponse = new ScaleIndexShardResponse(input); + + assertEquals("Shard ID should survive serialization", originalResponse.getShardId(), deserializedResponse.getShardId()); + assertEquals("Needs sync flag should survive serialization", originalResponse.needsSync(), deserializedResponse.needsSync()); + assertEquals( + "Uncommitted operations status should survive serialization", + originalResponse.hasUncommittedOperations(), + deserializedResponse.hasUncommittedOperations() + ); + } + + public void testZeroUncommittedOperations() { + ShardId shardId = createTestShardId(); + ScaleIndexShardResponse response = new ScaleIndexShardResponse(shardId, randomBoolean(), 0); + + assertFalse("Should report no uncommitted operations when count is 0", response.hasUncommittedOperations()); + } + + public void testNonZeroUncommittedOperations() { + ShardId shardId = createTestShardId(); + int uncommittedOps = randomIntBetween(1, 100); + ScaleIndexShardResponse response = new ScaleIndexShardResponse(shardId, randomBoolean(), uncommittedOps); + + assertTrue("Should report uncommitted operations when count is > 0", response.hasUncommittedOperations()); + } + + public void testSerializationWithExtremeValues() throws IOException { + ShardId shardId = createTestShardId(); + + // Test with Integer.MAX_VALUE uncommitted operations + ScaleIndexShardResponse originalResponse = new ScaleIndexShardResponse(shardId, true, Integer.MAX_VALUE); + + BytesStreamOutput output = new BytesStreamOutput(); + originalResponse.writeTo(output); + + StreamInput input = output.bytes().streamInput(); + ScaleIndexShardResponse deserializedResponse = new ScaleIndexShardResponse(input); + + assertTrue("Max value should be preserved and indicate uncommitted operations", deserializedResponse.hasUncommittedOperations()); + } + + public void testSerializationWithVariousShardIds() throws IOException { + // Test with different shard numbers + for (int shardNum : new int[] { 0, 1, 100, Integer.MAX_VALUE }) { + ShardId shardId = new ShardId(new Index("test_index", "uuid"), shardNum); + ScaleIndexShardResponse originalResponse = new ScaleIndexShardResponse(shardId, randomBoolean(), randomIntBetween(0, 100)); + + BytesStreamOutput output = new BytesStreamOutput(); + originalResponse.writeTo(output); + + StreamInput input = output.bytes().streamInput(); + ScaleIndexShardResponse deserializedResponse = new ScaleIndexShardResponse(input); + + assertEquals("Shard number should survive serialization", shardId.id(), deserializedResponse.getShardId().id()); + } + } + + private ShardId createTestShardId() { + return new ShardId(new Index("test_index", "uuid"), randomIntBetween(0, 10)); + } +} diff --git a/server/src/test/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexShardSyncManagerTests.java b/server/src/test/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexShardSyncManagerTests.java new file mode 100644 index 0000000000000..9f40d24fbb992 --- /dev/null +++ b/server/src/test/java/org/opensearch/action/admin/indices/scale/searchonly/ScaleIndexShardSyncManagerTests.java @@ -0,0 +1,340 @@ +/* + * 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.scale.searchonly; + +import org.opensearch.Version; +import org.opensearch.cluster.ClusterName; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.node.DiscoveryNode; +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.ShardRouting; +import org.opensearch.cluster.routing.ShardRoutingState; +import org.opensearch.cluster.routing.TestShardRouting; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.settings.Settings; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.index.Index; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.transport.TransportException; +import org.opensearch.transport.TransportResponseHandler; +import org.opensearch.transport.TransportService; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class ScaleIndexShardSyncManagerTests extends OpenSearchTestCase { + + private ClusterService clusterService; + private TransportService transportService; + private ScaleIndexShardSyncManager syncManager; + private final String transportActionName = "dummyAction"; + + @Override + public void setUp() throws Exception { + super.setUp(); + clusterService = mock(ClusterService.class); + transportService = mock(TransportService.class); + syncManager = new ScaleIndexShardSyncManager(clusterService, transportService, transportActionName); + } + + public void testSendShardSyncRequests_emptyPrimaryShards() { + ActionListener> listener = new ActionListener<>() { + @Override + public void onResponse(Collection responses) { + fail("Expected failure when primary shards map is empty"); + } + + @Override + public void onFailure(Exception e) { + assertTrue(e instanceof IllegalStateException); + assertEquals("No primary shards found for index test_index", e.getMessage()); + } + }; + syncManager.sendShardSyncRequests("test_index", Collections.emptyMap(), listener); + } + + public void testSendShardSyncRequests_nodeNotFound() { + // Prepare a mapping: one shard assigned to node "node1" + ShardId shardId = new ShardId(new Index("test_index", "uuid"), 0); + Map primaryShardsNodes = Collections.singletonMap(shardId, "node1"); + + // Set cluster state with empty discovery nodes so "node1" is missing. + DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().build(); + ClusterState clusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build(); + when(clusterService.state()).thenReturn(clusterState); + + ActionListener> listener = new ActionListener<>() { + @Override + public void onResponse(Collection responses) { + fail("Expected failure due to missing node"); + } + + @Override + public void onFailure(Exception e) { + assertTrue(e.getMessage().contains("Node [node1] not found")); + } + }; + + syncManager.sendShardSyncRequests("test_index", primaryShardsNodes, listener); + } + + public void testSendShardSyncRequests_success() throws Exception { + // Prepare a mapping: one shard assigned to node "node1" + ShardId shardId = new ShardId(new Index("test_index", "uuid"), 0); + Map primaryShardsNodes = Collections.singletonMap(shardId, "node1"); + + // Build cluster state with discovery node "node1" + DiscoveryNode node = new DiscoveryNode("node1", buildNewFakeTransportAddress(), Version.CURRENT); + DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().add(node).localNodeId("node1").build(); + ClusterState clusterState = ClusterState.builder(new ClusterName("test")).nodes(discoveryNodes).build(); + when(clusterService.state()).thenReturn(clusterState); + + // Stub transportService.sendRequest to return a dummy response. + doAnswer(invocation -> { + TransportResponseHandler handler = invocation.getArgument(3); + handler.handleResponse(new ScaleIndexNodeResponse(node, Collections.emptyList())); + return null; + }).when(transportService) + .sendRequest( + any(DiscoveryNode.class), + eq(transportActionName), + any(ScaleIndexNodeRequest.class), + any(TransportResponseHandler.class) + ); + + CountDownLatch latch = new CountDownLatch(1); + AtomicReference> responseRef = new AtomicReference<>(); + AtomicReference exceptionRef = new AtomicReference<>(); + + ActionListener> listener = new ActionListener<>() { + @Override + public void onResponse(Collection responses) { + responseRef.set(responses); + latch.countDown(); + } + + @Override + public void onFailure(Exception e) { + exceptionRef.set(e); + latch.countDown(); + } + }; + + syncManager.sendShardSyncRequests("test_index", primaryShardsNodes, listener); + assertTrue(latch.await(5, TimeUnit.SECONDS)); + assertNull(exceptionRef.get()); + Collection responses = responseRef.get(); + assertNotNull(responses); + // We expect one response since there's one node. + assertEquals(1, responses.size()); + } + + public void testSendNodeRequest_success() throws Exception { + DiscoveryNode node = new DiscoveryNode("node1", buildNewFakeTransportAddress(), Version.CURRENT); + String index = "test_index"; + List shards = Collections.singletonList(new ShardId(new Index("test_index", "uuid"), 0)); + + CountDownLatch latch = new CountDownLatch(1); + AtomicReference responseRef = new AtomicReference<>(); + + ActionListener listener = new ActionListener<>() { + @Override + public void onResponse(ScaleIndexNodeResponse response) { + responseRef.set(response); + latch.countDown(); + } + + @Override + public void onFailure(Exception e) { + fail("Unexpected failure: " + e.getMessage()); + } + }; + + doAnswer(invocation -> { + TransportResponseHandler handler = invocation.getArgument(3); + handler.handleResponse(new ScaleIndexNodeResponse(node, Collections.emptyList())); + return null; + }).when(transportService) + .sendRequest(eq(node), eq(transportActionName), any(ScaleIndexNodeRequest.class), any(TransportResponseHandler.class)); + + syncManager.sendNodeRequest(node, index, shards, listener); + assertTrue(latch.await(5, TimeUnit.SECONDS)); + assertNotNull(responseRef.get()); + } + + public void testSendNodeRequest_failure() throws Exception { + DiscoveryNode node = new DiscoveryNode("node1", buildNewFakeTransportAddress(), Version.CURRENT); + String index = "test_index"; + List shards = Collections.singletonList(new ShardId(new Index("test_index", "uuid"), 0)); + + CountDownLatch latch = new CountDownLatch(1); + AtomicReference exceptionRef = new AtomicReference<>(); + + ActionListener listener = new ActionListener<>() { + @Override + public void onResponse(ScaleIndexNodeResponse response) { + fail("Expected failure"); + } + + @Override + public void onFailure(Exception e) { + exceptionRef.set(e); + latch.countDown(); + } + }; + + // Use a dummy Throwable as the cause instead of passing the node. + doAnswer(invocation -> { + TransportResponseHandler handler = invocation.getArgument(3); + handler.handleException(new TransportException("Test exception", new Exception("dummy cause"))); + return null; + }).when(transportService) + .sendRequest(eq(node), eq(transportActionName), any(ScaleIndexNodeRequest.class), any(TransportResponseHandler.class)); + + syncManager.sendNodeRequest(node, index, shards, listener); + assertTrue(latch.await(5, TimeUnit.SECONDS)); + assertNotNull(exceptionRef.get()); + assertTrue(exceptionRef.get() instanceof TransportException); + } + + public void testValidateNodeResponses_success() { + // Create a shard response with no failures. + ShardId shardId = new ShardId(new Index("test_index", "uuid"), 0); + ScaleIndexShardResponse shardResponse = new ScaleIndexShardResponse(shardId, false, 0); + ScaleIndexNodeResponse nodeResponse = new ScaleIndexNodeResponse( + new DiscoveryNode("node1", buildNewFakeTransportAddress(), Version.CURRENT), + Collections.singletonList(shardResponse) + ); + + List responses = Collections.singletonList(nodeResponse); + AtomicReference responseRef = new AtomicReference<>(); + AtomicReference exceptionRef = new AtomicReference<>(); + + syncManager.validateNodeResponses(responses, new ActionListener() { + @Override + public void onResponse(ScaleIndexResponse response) { + responseRef.set(response); + } + + @Override + public void onFailure(Exception e) { + exceptionRef.set(e); + } + }); + + assertNull(exceptionRef.get()); + assertNotNull(responseRef.get()); + } + + public void testValidateNodeResponses_failure_uncommitted() { + // Create a shard response indicating uncommitted operations. + ShardId shardId = new ShardId(new Index("test_index", "uuid"), 0); + ScaleIndexShardResponse shardResponse = new ScaleIndexShardResponse(shardId, false, 5); + ScaleIndexNodeResponse nodeResponse = new ScaleIndexNodeResponse( + new DiscoveryNode("node1", buildNewFakeTransportAddress(), Version.CURRENT), + Collections.singletonList(shardResponse) + ); + + List responses = Collections.singletonList(nodeResponse); + AtomicReference exceptionRef = new AtomicReference<>(); + + syncManager.validateNodeResponses(responses, new ActionListener() { + @Override + public void onResponse(ScaleIndexResponse response) { + fail("Expected failure due to uncommitted operations"); + } + + @Override + public void onFailure(Exception e) { + exceptionRef.set(e); + } + }); + + assertNotNull(exceptionRef.get()); + assertTrue(exceptionRef.get().getMessage().contains("uncommitted operations")); + } + + public void testValidateNodeResponses_failure_needsSync() { + // Create a shard response indicating that a shard needs sync. + ShardId shardId = new ShardId(new Index("test_index", "uuid"), 0); + ScaleIndexShardResponse shardResponse = new ScaleIndexShardResponse(shardId, true, 0); + ScaleIndexNodeResponse nodeResponse = new ScaleIndexNodeResponse( + new DiscoveryNode("node1", buildNewFakeTransportAddress(), Version.CURRENT), + Collections.singletonList(shardResponse) + ); + + List responses = Collections.singletonList(nodeResponse); + AtomicReference exceptionRef = new AtomicReference<>(); + + syncManager.validateNodeResponses(responses, new ActionListener() { + @Override + public void onResponse(ScaleIndexResponse response) { + fail("Expected failure due to sync needed"); + } + + @Override + public void onFailure(Exception e) { + exceptionRef.set(e); + } + }); + + assertNotNull(exceptionRef.get()); + assertTrue(exceptionRef.get().getMessage().contains("sync needed")); + } + + public void testGetPrimaryShardAssignments_withRouting() { + // Create index settings with an explicit uuid. + Settings indexSettings = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put("index.uuid", "uuid") + .build(); + // Build IndexMetadata using the index name. The builder will pick up the uuid from the settings. + IndexMetadata indexMetadata = IndexMetadata.builder("test_index") + .settings(indexSettings) + .numberOfShards(1) + .numberOfReplicas(1) + .build(); + + // Build a minimal routing table using the same index name and uuid. + Index index = new Index("test_index", "uuid"); + ShardId shardId = new ShardId(index, 0); + ShardRouting primaryShardRouting = TestShardRouting.newShardRouting(shardId, "node1", true, ShardRoutingState.STARTED); + IndexShardRoutingTable shardRoutingTable = new IndexShardRoutingTable.Builder(shardId).addShard(primaryShardRouting).build(); + IndexRoutingTable indexRoutingTable = new IndexRoutingTable.Builder(index).addIndexShard(shardRoutingTable).build(); + RoutingTable routingTable = RoutingTable.builder().add(indexRoutingTable).build(); + + // Build a cluster state that contains the routing table. + ClusterState state = ClusterState.builder(new ClusterName("test")).routingTable(routingTable).build(); + + // Invoke the method under test. + Map assignments = syncManager.getPrimaryShardAssignments(indexMetadata, state); + // We expect one mapping: shard0 -> "node1" + assertEquals(1, assignments.size()); + // Construct the expected shard id using the same Index (name and uuid). + ShardId expectedShardId = new ShardId(new Index("test_index", "uuid"), 0); + assertEquals("node1", assignments.get(expectedShardId)); + } + +} diff --git a/server/src/test/java/org/opensearch/action/admin/indices/scale/searchonly/TransportScaleIndexActionTests.java b/server/src/test/java/org/opensearch/action/admin/indices/scale/searchonly/TransportScaleIndexActionTests.java new file mode 100644 index 0000000000000..c5a0e810db3b5 --- /dev/null +++ b/server/src/test/java/org/opensearch/action/admin/indices/scale/searchonly/TransportScaleIndexActionTests.java @@ -0,0 +1,798 @@ +/* + * 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.scale.searchonly; + +import org.opensearch.Version; +import org.opensearch.action.admin.indices.flush.FlushRequest; +import org.opensearch.action.support.ActionFilters; +import org.opensearch.action.support.clustermanager.AcknowledgedResponse; +import org.opensearch.cluster.ClusterName; +import org.opensearch.cluster.ClusterState; +import org.opensearch.cluster.ClusterStateUpdateTask; +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.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.IndexNameExpressionResolver; +import org.opensearch.cluster.metadata.Metadata; +import org.opensearch.cluster.node.DiscoveryNode; +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.ShardRouting; +import org.opensearch.cluster.routing.ShardRoutingState; +import org.opensearch.cluster.routing.TestShardRouting; +import org.opensearch.cluster.routing.allocation.AllocationService; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.core.action.ActionListener; +import org.opensearch.core.index.Index; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.core.rest.RestStatus; +import org.opensearch.index.IndexService; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.translog.TranslogStats; +import org.opensearch.indices.IndicesService; +import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportChannel; +import org.opensearch.transport.TransportResponseHandler; +import org.opensearch.transport.TransportService; +import org.junit.After; +import org.junit.Before; + +import java.util.Collection; +import java.util.Collections; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +import org.mockito.ArgumentCaptor; + +import static org.opensearch.cluster.metadata.IndexMetadata.INDEX_SEARCH_ONLY_BLOCK_ID; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.clearInvocations; +import static org.mockito.Mockito.doAnswer; +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 TransportScaleIndexActionTests extends OpenSearchTestCase { + + private TransportService transportService; + private ClusterService clusterService; + private AllocationService allocationService; + private IndicesService indicesService; + private ThreadPool threadPool; + private TransportScaleIndexAction action; + + @Before + public void setUp() throws Exception { + super.setUp(); + threadPool = new TestThreadPool("ScaleIndexActionTests"); + transportService = mock(TransportService.class); + clusterService = mock(ClusterService.class); + allocationService = mock(AllocationService.class); + indicesService = mock(IndicesService.class); + + action = new TransportScaleIndexAction( + transportService, + clusterService, + threadPool, + new ActionFilters(Collections.emptySet()), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), + allocationService, + indicesService + ); + + // Setup basic cluster state + ClusterState.Builder stateBuilder = ClusterState.builder(new ClusterName("test")); + stateBuilder.nodes(DiscoveryNodes.builder().build()); + when(clusterService.state()).thenReturn(stateBuilder.build()); + } + + @After + public void tearDown() throws Exception { + ThreadPool.terminate(threadPool, 30, TimeUnit.SECONDS); + super.tearDown(); + } + + public void testScaleDownValidation() { + String indexName = "test_index"; + ScaleIndexRequest request = new ScaleIndexRequest(indexName, true); + + // Test validation when index doesn't exist + ClusterState state = createClusterStateWithoutIndex(indexName); + when(clusterService.state()).thenReturn(state); + + ActionListener listener = new ActionListener<>() { + @Override + public void onResponse(AcknowledgedResponse response) { + fail("Expected validation to fail"); + } + + @Override + public void onFailure(Exception e) { + assertTrue(e instanceof IllegalArgumentException); + assertEquals("Index [" + indexName + "] not found", e.getMessage()); + } + }; + + action.clusterManagerOperation(request, state, listener); + } + + public void testScaleDownWithSearchOnlyAlreadyEnabled() { + String indexName = "test_index"; + ScaleIndexRequest request = new ScaleIndexRequest(indexName, true); + + // Create cluster state with search-only already enabled + ClusterState state = createClusterStateWithSearchOnlyEnabled(indexName); + when(clusterService.state()).thenReturn(state); + + ActionListener listener = new ActionListener<>() { + @Override + public void onResponse(AcknowledgedResponse response) { + fail("Expected validation to fail"); + } + + @Override + public void onFailure(Exception e) { + assertTrue(e instanceof IllegalStateException); + assertEquals("Index [" + indexName + "] is already in search-only mode", e.getMessage()); + } + }; + + action.clusterManagerOperation(request, state, listener); + } + + public void testScaleUpValidation() { + String indexName = "test_index"; + ScaleIndexRequest request = new ScaleIndexRequest(indexName, false); + + // Test validation when index is not in search-only mode + ClusterState state = createClusterStateWithoutSearchOnly(indexName); + when(clusterService.state()).thenReturn(state); + + ActionListener listener = new ActionListener() { + @Override + public void onResponse(AcknowledgedResponse response) { + fail("Expected validation to fail"); + } + + @Override + public void onFailure(Exception e) { + assertTrue(e instanceof IllegalStateException); + assertEquals("Index [" + indexName + "] is not in search-only mode", e.getMessage()); + } + }; + + action.clusterManagerOperation(request, state, listener); + } + + private ClusterState createClusterStateWithoutIndex(String indexName) { + return ClusterState.builder(new ClusterName("test")).metadata(Metadata.builder().build()).build(); + } + + private ClusterState createClusterStateWithSearchOnlyEnabled(String indexName) { + Settings indexSettings = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), true) + .build(); + + IndexMetadata indexMetadata = IndexMetadata.builder(indexName) + .settings(indexSettings) + .numberOfShards(1) + .numberOfReplicas(1) + .build(); + + return ClusterState.builder(new ClusterName("test")).metadata(Metadata.builder().put(indexMetadata, true).build()).build(); + } + + private ClusterState createClusterStateWithoutSearchOnly(String indexName) { + Settings indexSettings = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), false) + .build(); + + IndexMetadata indexMetadata = IndexMetadata.builder(indexName) + .settings(indexSettings) + .numberOfShards(1) + .numberOfReplicas(1) + .build(); + + return ClusterState.builder(new ClusterName("test")).metadata(Metadata.builder().put(indexMetadata, true).build()).build(); + } + + public void testAddBlockClusterStateUpdateTask() { + String indexName = "test_index"; + ScaleIndexRequest request = new ScaleIndexRequest(indexName, true); + + // Create initial cluster state with necessary index metadata + Settings indexSettings = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) + .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) + .put(IndexMetadata.SETTING_NUMBER_OF_SEARCH_REPLICAS, 1) + .build(); + + IndexMetadata indexMetadata = IndexMetadata.builder(indexName) + .settings(indexSettings) + .numberOfShards(1) + .numberOfReplicas(1) + .build(); + + ClusterState initialState = ClusterState.builder(new ClusterName("test")) + .metadata(Metadata.builder().put(indexMetadata, true).build()) + .build(); + + when(clusterService.state()).thenReturn(initialState); + + ActionListener listener = new ActionListener<>() { + @Override + public void onResponse(AcknowledgedResponse response) { + assertTrue("Expected block to be added successfully", response.isAcknowledged()); + } + + @Override + public void onFailure(Exception e) { + fail("Should not fail: " + e.getMessage()); + } + }; + + action.clusterManagerOperation(request, initialState, listener); + + // Verify that the appropriate block was added + verify(clusterService).submitStateUpdateTask(eq("add-block-index-to-scale " + indexName), any()); + } + + public void testFinalizeScaleDownTaskSimple() throws Exception { + String indexName = "test_index"; + ScaleIndexRequest request = new ScaleIndexRequest(indexName, true); + + // Create minimal index metadata that meets scale-down prerequisites. + Settings indexSettings = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) + .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) + .put(IndexMetadata.SETTING_NUMBER_OF_SEARCH_REPLICAS, 1) + .build(); + IndexMetadata indexMetadata = IndexMetadata.builder(indexName) + .settings(indexSettings) + .numberOfShards(1) + .numberOfReplicas(1) + .build(); + + // Build a minimal routing table for the index. + Index index = indexMetadata.getIndex(); + ShardId shardId = new ShardId(index, 0); + ShardRouting primaryShardRouting = TestShardRouting.newShardRouting(shardId, "node1", true, ShardRoutingState.STARTED); + IndexShardRoutingTable shardRoutingTable = new IndexShardRoutingTable.Builder(shardId).addShard(primaryShardRouting).build(); + IndexRoutingTable indexRoutingTable = new IndexRoutingTable.Builder(index).addIndexShard(shardRoutingTable).build(); + RoutingTable routingTable = RoutingTable.builder().add(indexRoutingTable).build(); + + // Create a DiscoveryNode and include it in the cluster state's nodes. + DiscoveryNode node = new DiscoveryNode("node1", buildNewFakeTransportAddress(), Version.CURRENT); + DiscoveryNodes discoveryNodes = DiscoveryNodes.builder().add(node).localNodeId("node1").build(); + + // Build the complete cluster state with metadata, routing table, and nodes. + ClusterState initialState = ClusterState.builder(new ClusterName("test")) + .metadata(Metadata.builder().put(indexMetadata, true).build()) + .routingTable(routingTable) + .nodes(discoveryNodes) + .build(); + when(clusterService.state()).thenReturn(initialState); + + // Stub transportService.sendRequest so that any shard sync request immediately succeeds. + doAnswer(invocation -> { + TransportResponseHandler handler = invocation.getArgument(3); + handler.handleResponse(new ScaleIndexNodeResponse(node, Collections.emptyList())); + return null; + }).when(transportService) + .sendRequest( + any(DiscoveryNode.class), + eq(TransportScaleIndexAction.NAME), + any(ScaleIndexNodeRequest.class), + any(TransportResponseHandler.class) + ); + + // Execute the scale-down operation. + action.clusterManagerOperation(request, initialState, new ActionListener<>() { + @Override + public void onResponse(AcknowledgedResponse response) {} + + @Override + public void onFailure(Exception e) { + fail("Operation should not fail: " + e.getMessage()); + } + }); + + // Capture the add-block task submitted by the action. + ArgumentCaptor captor = ArgumentCaptor.forClass(ClusterStateUpdateTask.class); + verify(clusterService).submitStateUpdateTask(eq("add-block-index-to-scale " + indexName), captor.capture()); + ClusterStateUpdateTask addBlockTask = captor.getValue(); + + // Create a new cluster state that is different from initialState. + // For example, add a dummy block to the index. + ClusterBlocks.Builder blocksBuilder = ClusterBlocks.builder().blocks(initialState.blocks()); + blocksBuilder.addIndexBlock( + indexName, + new ClusterBlock(123, "dummy", false, false, false, RestStatus.FORBIDDEN, EnumSet.of(ClusterBlockLevel.WRITE)) + ); + ClusterState newState = ClusterState.builder(initialState).blocks(blocksBuilder).build(); + + // Simulate the add-block task callback (with a changed state) to trigger finalize. + addBlockTask.clusterStateProcessed("test-source", initialState, newState); + + // Verify that the finalize-scale-down update task was submitted. + verify(clusterService).submitStateUpdateTask(eq("finalize-scale-down"), any(ClusterStateUpdateTask.class)); + } + + public void testScaleUpClusterStateUpdateTask() throws Exception { + String indexName = "test_index"; + ScaleIndexRequest request = new ScaleIndexRequest(indexName, false); + + // Create index metadata with search-only mode enabled. + Settings indexSettings = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), true) + .build(); + IndexMetadata indexMetadata = IndexMetadata.builder(indexName) + .settings(indexSettings) + .numberOfShards(1) + .numberOfReplicas(1) + .build(); + + // Build a minimal routing table for the index. + Index index = indexMetadata.getIndex(); + ShardId shardId = new ShardId(index, 0); + // Create a dummy shard routing in STARTED state. + ShardRouting searchOnlyShardRouting = TestShardRouting.newShardRouting(shardId, "node1", true, ShardRoutingState.STARTED); + IndexShardRoutingTable shardRoutingTable = new IndexShardRoutingTable.Builder(shardId).addShard(searchOnlyShardRouting).build(); + IndexRoutingTable indexRoutingTable = new IndexRoutingTable.Builder(index).addIndexShard(shardRoutingTable).build(); + RoutingTable routingTable = RoutingTable.builder().add(indexRoutingTable).build(); + + // Build the complete cluster state with metadata and the routing table. + ClusterState initialState = ClusterState.builder(new ClusterName("test")) + .metadata(Metadata.builder().put(indexMetadata, true).build()) + .routingTable(routingTable) + .build(); + + // Stub allocationService.reroute to return a valid state. + ClusterState stateAfterReroute = ClusterState.builder(initialState).build(); + when(allocationService.reroute(any(ClusterState.class), anyString())).thenReturn(stateAfterReroute); + when(clusterService.state()).thenReturn(initialState); + + ActionListener listener = new ActionListener<>() { + @Override + public void onResponse(AcknowledgedResponse response) { + assertTrue("Expected scale up to complete successfully", response.isAcknowledged()); + } + + @Override + public void onFailure(Exception e) { + fail("Should not fail: " + e.getMessage()); + } + }; + + // Trigger the scale-up operation. + action.clusterManagerOperation(request, initialState, listener); + + // Capture the update task submitted for scaling up. + ArgumentCaptor captor = ArgumentCaptor.forClass(ClusterStateUpdateTask.class); + verify(clusterService).submitStateUpdateTask(eq("scale-up-index"), captor.capture()); + ClusterStateUpdateTask scaleUpTask = captor.getValue(); + + // Manually simulate execution of the scale-up task. + ClusterState updatedState = scaleUpTask.execute(initialState); + scaleUpTask.clusterStateProcessed("test-source", initialState, updatedState); + + // Verify that allocationService.reroute was called with the expected reason. + verify(allocationService).reroute(any(ClusterState.class), eq("restore indexing shards")); + } + + public void testScaleDownWithMissingIndex() { + String indexName = "non_existent_index"; + ScaleIndexRequest request = new ScaleIndexRequest(indexName, true); + + ClusterState state = ClusterState.builder(new ClusterName("test")).metadata(Metadata.builder().build()).build(); + + ActionListener listener = new ActionListener<>() { + @Override + public void onResponse(AcknowledgedResponse response) { + fail("Should fail for missing index"); + } + + @Override + public void onFailure(Exception e) { + assertTrue(e instanceof IllegalArgumentException); + assertEquals("Index [" + indexName + "] not found", e.getMessage()); + } + }; + + action.clusterManagerOperation(request, state, listener); + } + + public void testScaleUpWithSearchOnlyNotEnabled() { + String indexName = "test_index"; + ScaleIndexRequest request = new ScaleIndexRequest(indexName, false); + + Settings indexSettings = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), false) + .build(); + + IndexMetadata indexMetadata = IndexMetadata.builder(indexName) + .settings(indexSettings) + .numberOfShards(1) + .numberOfReplicas(1) + .build(); + + ClusterState state = ClusterState.builder(new ClusterName("test")) + .metadata(Metadata.builder().put(indexMetadata, true).build()) + .build(); + + ActionListener listener = new ActionListener<>() { + @Override + public void onResponse(AcknowledgedResponse response) { + fail("Should fail when search-only is not enabled"); + } + + @Override + public void onFailure(Exception e) { + assertTrue(e instanceof IllegalStateException); + assertEquals("Index [" + indexName + "] is not in search-only mode", e.getMessage()); + } + }; + action.clusterManagerOperation(request, state, listener); + } + + public void testHandleShardSyncRequest() throws Exception { + // Mock dependencies + TransportService transportService = mock(TransportService.class); + ClusterService clusterService = mock(ClusterService.class); + IndicesService indicesService = mock(IndicesService.class); + TransportChannel channel = mock(TransportChannel.class); + DiscoveryNode localNode = new DiscoveryNode("local", buildNewFakeTransportAddress(), Version.CURRENT); + + // Use a real ThreadPool but with a controlled executor + ThreadPool threadPool = new TestThreadPool("testHandleShardSyncRequest"); + + try { + // Create test data + String indexName = "test_index"; + Index index = new Index(indexName, "_na_"); + ShardId shardId = new ShardId(index, 0); + List shardIds = Collections.singletonList(shardId); + ScaleIndexNodeRequest request = new ScaleIndexNodeRequest(indexName, shardIds); + + // Mock cluster state + ClusterState clusterState = mock(ClusterState.class); + Metadata metadata = mock(Metadata.class); + IndexMetadata indexMetadata = mock(IndexMetadata.class); + when(clusterService.state()).thenReturn(clusterState); + when(clusterState.metadata()).thenReturn(metadata); + when(metadata.index(indexName)).thenReturn(indexMetadata); + when(indexMetadata.getIndex()).thenReturn(index); + when(clusterService.localNode()).thenReturn(localNode); + + // Mock index service and shard + IndexService indexService = mock(IndexService.class); + IndexShard indexShard = mock(IndexShard.class); + TranslogStats translogStats = mock(TranslogStats.class); + + when(indicesService.indexService(any(Index.class))).thenReturn(indexService); + when(indexService.getShardOrNull(anyInt())).thenReturn(indexShard); + when(indexShard.shardId()).thenReturn(shardId); + when(indexShard.translogStats()).thenReturn(translogStats); + when(translogStats.getUncommittedOperations()).thenReturn(0); + when(indexShard.isSyncNeeded()).thenReturn(false); + + // Mock shard routing to return a primary routing + ShardRouting shardRouting = mock(ShardRouting.class); + when(shardRouting.primary()).thenReturn(true); + when(indexShard.routingEntry()).thenReturn(shardRouting); + + // Mock the acquireAllPrimaryOperationsPermits method to immediately call the listener + doAnswer(invocation -> { + ActionListener listener = invocation.getArgument(0); + Releasable releasable = mock(Releasable.class); + listener.onResponse(releasable); + return null; + }).when(indexShard).acquireAllPrimaryOperationsPermits(any(ActionListener.class), any(TimeValue.class)); + + // Create action instance with the real ThreadPool + TransportScaleIndexAction action = new TransportScaleIndexAction( + transportService, + clusterService, + threadPool, + new ActionFilters(Collections.emptySet()), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), + allocationService, + indicesService + ); + + // Call handleShardSyncRequest + action.handleShardSyncRequest(request, channel); + + // Wait a short time for the async task to execute + assertBusy(() -> { verify(channel).sendResponse(any(ScaleIndexNodeResponse.class)); }, 5, TimeUnit.SECONDS); + } finally { + ThreadPool.terminate(threadPool, 30, TimeUnit.SECONDS); + } + } + + public void testSyncSingleShard() throws Exception { + IndexShard shard = mock(IndexShard.class); + ShardId shardId = new ShardId(new Index("test_index", "_na_"), 0); + TranslogStats translogStats = mock(TranslogStats.class); + + when(shard.shardId()).thenReturn(shardId); + when(shard.translogStats()).thenReturn(translogStats); + + TransportScaleIndexAction action = new TransportScaleIndexAction( + transportService, + clusterService, + threadPool, + new ActionFilters(Collections.emptySet()), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), + allocationService, + indicesService + ); + + when(translogStats.getUncommittedOperations()).thenReturn(0); + when(shard.isSyncNeeded()).thenReturn(false); + + final AtomicReference successResponseRef = new AtomicReference<>(); + final AtomicReference successExceptionRef = new AtomicReference<>(); + final CountDownLatch successLatch = new CountDownLatch(1); + + action.syncSingleShard(shard, new ActionListener<>() { + @Override + public void onResponse(ScaleIndexShardResponse response) { + successResponseRef.set(response); + successLatch.countDown(); + } + + @Override + public void onFailure(Exception e) { + successExceptionRef.set(e); + successLatch.countDown(); + } + }); + + ArgumentCaptor successPermitCaptor = ArgumentCaptor.forClass(ActionListener.class); + verify(shard).acquireAllPrimaryOperationsPermits(successPermitCaptor.capture(), any(TimeValue.class)); + successPermitCaptor.getValue().onResponse(mock(Releasable.class)); + + assertTrue(successLatch.await(1, TimeUnit.SECONDS)); + + assertNull("No exception expected", successExceptionRef.get()); + assertNotNull("Response should not be null", successResponseRef.get()); + assertFalse("Response should not indicate sync needed", successResponseRef.get().needsSync()); + assertFalse("Response should not indicate uncommitted operations", successResponseRef.get().hasUncommittedOperations()); + + verify(shard, times(1)).sync(); + verify(shard, times(1)).flush(any(FlushRequest.class)); + verify(shard, times(1)).waitForRemoteStoreSync(); + + clearInvocations(shard); + + when(translogStats.getUncommittedOperations()).thenReturn(5); + when(shard.isSyncNeeded()).thenReturn(true); + + final AtomicReference responseRef = new AtomicReference<>(); + final AtomicReference exceptionRef = new AtomicReference<>(); + final CountDownLatch latch = new CountDownLatch(1); + + action.syncSingleShard(shard, new ActionListener<>() { + @Override + public void onResponse(ScaleIndexShardResponse response) { + responseRef.set(response); + latch.countDown(); + } + + @Override + public void onFailure(Exception e) { + exceptionRef.set(e); + latch.countDown(); + } + }); + + ArgumentCaptor permitListenerCaptor = ArgumentCaptor.forClass(ActionListener.class); + verify(shard).acquireAllPrimaryOperationsPermits(permitListenerCaptor.capture(), any(TimeValue.class)); + permitListenerCaptor.getValue().onResponse(mock(Releasable.class)); + + assertTrue(latch.await(1, TimeUnit.SECONDS)); + + assertNull("No exception expected", exceptionRef.get()); + assertNotNull("Response should not be null", responseRef.get()); + assertTrue("Response should indicate uncommitted operations", responseRef.get().hasUncommittedOperations()); + assertTrue("Response should indicate sync needed", responseRef.get().needsSync()); + + verify(shard, times(1)).sync(); + verify(shard, times(1)).flush(any(FlushRequest.class)); + verify(shard, times(1)).waitForRemoteStoreSync(); + } + + public void testCheckBlock() { + // Mock dependencies + TransportScaleIndexAction action = new TransportScaleIndexAction( + transportService, + clusterService, + threadPool, + new ActionFilters(Collections.emptySet()), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), + allocationService, + indicesService + ); + + // Create test data + String indexName = "test_index"; + ScaleIndexRequest request = new ScaleIndexRequest(indexName, true); + + // Create index metadata + Settings indexSettings = Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT).build(); + IndexMetadata indexMetadata = IndexMetadata.builder(indexName) + .settings(indexSettings) + .numberOfShards(1) + .numberOfReplicas(1) + .build(); + + // Test with no blocks + ClusterState state = ClusterState.builder(new ClusterName("test")) + .metadata(Metadata.builder().put(indexMetadata, true)) + .blocks(ClusterBlocks.builder().build()) + .build(); + assertNull(action.checkBlock(request, state)); + + // Test with metadata write block + ClusterBlocks.Builder blocksBuilder = ClusterBlocks.builder(); + ClusterBlock metadataBlock = new ClusterBlock( + 1, + "test block", + false, + false, + false, + RestStatus.FORBIDDEN, + EnumSet.of(ClusterBlockLevel.METADATA_WRITE) + ); + blocksBuilder.addGlobalBlock(metadataBlock); + state = ClusterState.builder(new ClusterName("test")) + .metadata(Metadata.builder().put(indexMetadata, true)) + .blocks(blocksBuilder.build()) + .build(); + + ClusterBlockException exception = action.checkBlock(request, state); + assertNotNull(exception); + assertTrue(exception.blocks().contains(metadataBlock)); + } + + public void testAddBlockClusterStateUpdateTaskExecute() { + // Mock dependencies + String indexName = "test_index"; + ActionListener listener = mock(ActionListener.class); + Map blockedIndices = new HashMap<>(); + + // Create initial cluster state + Settings indexSettings = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) + .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) + .put(IndexMetadata.SETTING_NUMBER_OF_SEARCH_REPLICAS, 1) + .build(); + + IndexMetadata indexMetadata = IndexMetadata.builder(indexName) + .settings(indexSettings) + .numberOfShards(1) + .numberOfReplicas(1) + .build(); + + ClusterState initialState = ClusterState.builder(new ClusterName("test")) + .metadata(Metadata.builder().put(indexMetadata, true).build()) + .build(); + + // Create action and task + TransportScaleIndexAction action = new TransportScaleIndexAction( + transportService, + clusterService, + threadPool, + new ActionFilters(Collections.emptySet()), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), + allocationService, + indicesService + ); + + TransportScaleIndexAction.AddBlockClusterStateUpdateTask task = action.new AddBlockClusterStateUpdateTask( + indexName, blockedIndices, listener + ); + + // Test successful execution + ClusterState newState = task.execute(initialState); + assertNotEquals(initialState, newState); + + // Verify that a block with the correct ID was added + Collection indexBlocks = newState.blocks().indices().get(indexName); + assertNotNull("Index blocks should not be null", indexBlocks); + assertTrue("Index should have at least one block", !indexBlocks.isEmpty()); + boolean hasBlockWithCorrectId = indexBlocks.stream().anyMatch(block -> block.id() == INDEX_SEARCH_ONLY_BLOCK_ID); + assertTrue("Should find a block with ID " + INDEX_SEARCH_ONLY_BLOCK_ID, hasBlockWithCorrectId); + + // Test execution with missing index + initialState = ClusterState.builder(new ClusterName("test")).metadata(Metadata.builder().build()).build(); + + ClusterState resultState = task.execute(initialState); + assertEquals(initialState, resultState); + + // Test onFailure + Exception testException = new Exception("Test failure"); + task.onFailure("test", testException); + verify(listener).onFailure(testException); + } + + public void testFinalizeScaleDownTaskFailure() { + // Mock dependencies + String indexName = "test_index"; + ActionListener listener = mock(ActionListener.class); + + TransportScaleIndexAction action = new TransportScaleIndexAction( + transportService, + clusterService, + threadPool, + new ActionFilters(Collections.emptySet()), + new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), + allocationService, + indicesService + ); + + TransportScaleIndexAction.FinalizeScaleDownTask task = action.new FinalizeScaleDownTask(indexName, listener); + + // Test onFailure + Exception testException = new Exception("Test failure"); + task.onFailure("test", testException); + verify(listener).onFailure(testException); + + // Test successful processing + ClusterState state = mock(ClusterState.class); + task.clusterStateProcessed("test", state, state); + verify(listener).onResponse(any(AcknowledgedResponse.class)); + } + + public void testThreadPoolConstantValidity() { + ThreadPool threadPool = new TestThreadPool("testThreadPoolConstantValidity"); + try { + // Verify that our constant points to a valid thread pool + assertNotNull("Thread pool executor should exist", threadPool.executor(TransportScaleIndexAction.SHARD_SYNC_EXECUTOR)); + + // Verify SHARD_SYNC_EXECUTOR is using the MANAGEMENT pool as expected + assertEquals( + "SHARD_SYNC_EXECUTOR should be set to MANAGEMENT", + ThreadPool.Names.MANAGEMENT, + TransportScaleIndexAction.SHARD_SYNC_EXECUTOR + ); + } finally { + ThreadPool.terminate(threadPool, 30, TimeUnit.SECONDS); + } + } + +} diff --git a/server/src/test/java/org/opensearch/cluster/health/ClusterShardHealthTests.java b/server/src/test/java/org/opensearch/cluster/health/ClusterShardHealthTests.java index 01319867bcd86..0d71f449f9b4f 100644 --- a/server/src/test/java/org/opensearch/cluster/health/ClusterShardHealthTests.java +++ b/server/src/test/java/org/opensearch/cluster/health/ClusterShardHealthTests.java @@ -31,12 +31,15 @@ package org.opensearch.cluster.health; +import org.opensearch.Version; +import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.routing.IndexShardRoutingTable; import org.opensearch.cluster.routing.RecoverySource; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.ShardRoutingState; import org.opensearch.cluster.routing.TestShardRouting; import org.opensearch.cluster.routing.UnassignedInfo; +import org.opensearch.common.settings.Settings; import org.opensearch.core.common.io.stream.Writeable; import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; @@ -64,8 +67,19 @@ public void testClusterShardGreenHealth() { indexShardRoutingBuilder.addShard( TestShardRouting.newShardRouting(indexName, shardID, "node_1", null, false, ShardRoutingState.STARTED) ); + + IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(indexName) + .settings( + Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + ) + .creationDate(System.currentTimeMillis()); + IndexMetadata indexMetadata = indexMetadataBuilder.build(); IndexShardRoutingTable indexShardRoutingTable = indexShardRoutingBuilder.build(); - ClusterShardHealth clusterShardHealth = new ClusterShardHealth(shardID, indexShardRoutingTable); + + ClusterShardHealth clusterShardHealth = new ClusterShardHealth(shardID, indexShardRoutingTable, indexMetadata); assertEquals(2, clusterShardHealth.getActiveShards()); assertEquals(0, clusterShardHealth.getInitializingShards()); assertEquals(0, clusterShardHealth.getRelocatingShards()); @@ -112,7 +126,18 @@ public void testClusterShardYellowHealth() { ) ); IndexShardRoutingTable indexShardRoutingTable = indexShardRoutingBuilder.build(); - ClusterShardHealth clusterShardHealth = new ClusterShardHealth(shardID, indexShardRoutingTable); + + IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(indexName) + .settings( + Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + ) + .creationDate(System.currentTimeMillis()); + IndexMetadata indexMetadata = indexMetadataBuilder.build(); + + ClusterShardHealth clusterShardHealth = new ClusterShardHealth(shardID, indexShardRoutingTable, indexMetadata); assertEquals(2, clusterShardHealth.getActiveShards()); assertEquals(1, clusterShardHealth.getInitializingShards()); assertEquals(1, clusterShardHealth.getRelocatingShards()); @@ -150,7 +175,18 @@ public void testClusterShardRedHealth() { TestShardRouting.newShardRouting(indexName, shardID, null, null, false, ShardRoutingState.UNASSIGNED) ); IndexShardRoutingTable indexShardRoutingTable = indexShardRoutingBuilder.build(); - ClusterShardHealth clusterShardHealth = new ClusterShardHealth(shardID, indexShardRoutingTable); + + IndexMetadata.Builder indexMetadataBuilder = IndexMetadata.builder(indexName) + .settings( + Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + ) + .creationDate(System.currentTimeMillis()); + IndexMetadata indexMetadata = indexMetadataBuilder.build(); + + ClusterShardHealth clusterShardHealth = new ClusterShardHealth(shardID, indexShardRoutingTable, indexMetadata); assertEquals(0, clusterShardHealth.getActiveShards()); assertEquals(0, clusterShardHealth.getInitializingShards()); assertEquals(0, clusterShardHealth.getRelocatingShards()); @@ -161,7 +197,30 @@ public void testClusterShardRedHealth() { } public void testShardRoutingNullCheck() { - assertThrows(AssertionError.class, () -> ClusterShardHealth.getShardHealth(null, 0, 0)); + IndexMetadata.Builder indexMetadataBuilder = 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) + ); + IndexMetadata indexMetadata = indexMetadataBuilder.build(); + + // When search-only is not enabled (default), expect RED status + assertEquals(ClusterHealthStatus.RED, ClusterShardHealth.getShardHealth(null, 0, 0, indexMetadata)); + + // When search-only is enabled, expect RED status + IndexMetadata.Builder searchOnlyMetadataBuilder = 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) + .put(IndexMetadata.INDEX_BLOCKS_SEARCH_ONLY_SETTING.getKey(), true) + ); + IndexMetadata searchOnlyMetadata = searchOnlyMetadataBuilder.build(); + + assertEquals(ClusterHealthStatus.RED, ClusterShardHealth.getShardHealth(null, 0, 0, searchOnlyMetadata)); } @Override diff --git a/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java b/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java index 287962b158c79..88c0046796ce6 100644 --- a/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java @@ -11,6 +11,7 @@ import org.apache.lucene.store.FilterDirectory; import org.opensearch.action.support.PlainActionFuture; import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.common.settings.Settings; import org.opensearch.index.engine.InternalEngineFactory; import org.opensearch.index.engine.NRTReplicationEngineFactory; @@ -137,29 +138,43 @@ public void testGetSegmentFilesReturnEmptyResponse() throws ExecutionException, assert (response.files.isEmpty()); } - public void testGetCheckpointMetadataEmpty() throws ExecutionException, InterruptedException, IOException { + public void testGetCheckpointMetadataEmpty() throws ExecutionException, InterruptedException { IndexShard mockShard = mock(IndexShard.class); - // Build mockShard to return replicaShard directory so that empty metadata file is returned. + // Build mockShard to return replicaShard directory so that an empty metadata file is returned. buildIndexShardBehavior(mockShard, replicaShard); replicationSource = new RemoteStoreReplicationSource(mockShard); - // Mock replica shard state to RECOVERING so that getCheckpointInfo return empty map + // For a RECOVERING shard, the response should have an empty metadata map. final ReplicationCheckpoint checkpoint = replicaShard.getLatestReplicationCheckpoint(); final PlainActionFuture res = PlainActionFuture.newFuture(); when(mockShard.state()).thenReturn(IndexShardState.RECOVERING); - replicationSource = new RemoteStoreReplicationSource(mockShard); - // Recovering shard should just do a noop and return empty metadata map. replicationSource.getCheckpointMetadata(REPLICATION_ID, checkpoint, res); CheckpointInfoResponse response = res.get(); - assert (response.getCheckpoint().equals(checkpoint)); - assert (response.getMetadataMap().isEmpty()); + assertTrue(response.getCheckpoint().equals(checkpoint)); + assertTrue(response.getMetadataMap().isEmpty()); - // Started shard should fail with assertion error. + // For a STARTED shard, the new behavior needs mock routing entry when(mockShard.state()).thenReturn(IndexShardState.STARTED); - expectThrows(AssertionError.class, () -> { - final PlainActionFuture res2 = PlainActionFuture.newFuture(); - replicationSource.getCheckpointMetadata(REPLICATION_ID, checkpoint, res2); - }); + // Mock a routing entry for the search-only condition + ShardRouting mockRouting = mock(ShardRouting.class); + when(mockRouting.isSearchOnly()).thenReturn(true); // Make it a search-only replica + when(mockShard.routingEntry()).thenReturn(mockRouting); + + // Ensure the mock returns the expected checkpoint when getLatestReplicationCheckpoint is called. + when(mockShard.getLatestReplicationCheckpoint()).thenReturn(replicaShard.getLatestReplicationCheckpoint()); + final PlainActionFuture res2 = PlainActionFuture.newFuture(); + replicationSource.getCheckpointMetadata(REPLICATION_ID, checkpoint, res2); + CheckpointInfoResponse response2 = res2.get(); + assertTrue(response2.getCheckpoint().equals(replicaShard.getLatestReplicationCheckpoint())); + assertTrue(response2.getMetadataMap().isEmpty()); + + // Additional test for non-search-only replica (should fail with exception) + when(mockRouting.isSearchOnly()).thenReturn(false); + final PlainActionFuture res3 = PlainActionFuture.newFuture(); + replicationSource.getCheckpointMetadata(REPLICATION_ID, checkpoint, res3); + ExecutionException exception = assertThrows(ExecutionException.class, () -> res3.get()); + assertTrue(exception.getCause() instanceof IllegalStateException); + assertTrue(exception.getCause().getMessage().contains("Remote metadata file can't be null if shard is active")); } private void buildIndexShardBehavior(IndexShard mockShard, IndexShard indexShard) { diff --git a/server/src/test/java/org/opensearch/rest/action/admin/indices/RestScaleIndexActionTests.java b/server/src/test/java/org/opensearch/rest/action/admin/indices/RestScaleIndexActionTests.java new file mode 100644 index 0000000000000..c8f5921f8b0ab --- /dev/null +++ b/server/src/test/java/org/opensearch/rest/action/admin/indices/RestScaleIndexActionTests.java @@ -0,0 +1,161 @@ +/* + * 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.rest.action.admin.indices; + +import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.bytes.BytesArray; +import org.opensearch.rest.RestRequest; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.test.rest.FakeRestRequest; +import org.opensearch.transport.client.node.NodeClient; + +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.notNullValue; +import static org.mockito.Mockito.mock; + +public class RestScaleIndexActionTests extends OpenSearchTestCase { + + private RestScaleIndexAction action; + + @Override + public void setUp() throws Exception { + super.setUp(); + action = new RestScaleIndexAction(); + } + + public void testMissingIndexParam() { + // Build a fake request with no "index" parameter + Map params = new HashMap<>(); + FakeRestRequest restRequest = new FakeRestRequest.Builder(xContentRegistry()).withParams(params) + .withMethod(RestRequest.Method.POST) + .build(); + IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> action.prepareRequest(restRequest, mock(NodeClient.class)) + ); + assertThat(e.getMessage(), containsString("index is required")); + } + + public void testEmptyIndexParam() { + // Build a fake request with an empty "index" parameter + Map params = new HashMap<>(); + params.put("index", " "); + FakeRestRequest restRequest = new FakeRestRequest.Builder(xContentRegistry()).withParams(params) + .withMethod(RestRequest.Method.POST) + .build(); + IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> action.prepareRequest(restRequest, mock(NodeClient.class)) + ); + assertThat(e.getMessage(), containsString("index is required")); + } + + public void testUnknownParameterInBody() { + String json = "{\"unknown\": \"value\"}"; + Map params = new HashMap<>(); + params.put("index", "test-index"); + FakeRestRequest restRequest = new FakeRestRequest.Builder(xContentRegistry()).withParams(params) + .withMethod(RestRequest.Method.POST) + .withContent(new BytesArray(json), XContentType.JSON) + .build(); + IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> action.prepareRequest(restRequest, mock(NodeClient.class)) + ); + assertThat(e.getMessage(), containsString("Unknown parameter [unknown]. Only [search_only] is allowed.")); + } + + public void testEmptyBody() { + String json = "{}"; + Map params = new HashMap<>(); + params.put("index", "test-index"); + FakeRestRequest restRequest = new FakeRestRequest.Builder(xContentRegistry()).withParams(params) + .withMethod(RestRequest.Method.POST) + .withContent(new BytesArray(json), XContentType.JSON) + .build(); + IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> action.prepareRequest(restRequest, mock(NodeClient.class)) + ); + assertThat(e.getMessage(), containsString("Parameter [search_only] is required")); + } + + public void testInvalidSearchOnlyType() { + String json = "{\"search_only\": \"not_a_boolean\"}"; + Map params = new HashMap<>(); + params.put("index", "test-index"); + FakeRestRequest restRequest = new FakeRestRequest.Builder(xContentRegistry()).withParams(params) + .withMethod(RestRequest.Method.POST) + .withContent(new BytesArray(json), XContentType.JSON) + .build(); + IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> action.prepareRequest(restRequest, mock(NodeClient.class)) + ); + assertThat(e.getMessage(), containsString("Parameter [search_only] must be a boolean (true or false)")); + } + + public void testValidRequestWithSearchOnlyTrue() throws Exception { + String json = "{\"search_only\": true}"; + Map params = new HashMap<>(); + params.put("index", "test-index"); + FakeRestRequest restRequest = new FakeRestRequest.Builder(xContentRegistry()).withParams(params) + .withMethod(RestRequest.Method.POST) + .withContent(new BytesArray(json), XContentType.JSON) + .build(); + Object consumer = action.prepareRequest(restRequest, mock(NodeClient.class)); + assertThat(consumer, notNullValue()); + } + + public void testValidRequestWithSearchOnlyFalse() throws Exception { + String json = "{\"search_only\": false}"; + Map params = new HashMap<>(); + params.put("index", "test-index"); + FakeRestRequest restRequest = new FakeRestRequest.Builder(xContentRegistry()).withParams(params) + .withMethod(RestRequest.Method.POST) + .withContent(new BytesArray(json), XContentType.JSON) + .build(); + Object consumer = action.prepareRequest(restRequest, mock(NodeClient.class)); + assertThat(consumer, notNullValue()); + } + + public void testInvalidJson() { + String json = "{\"search_only\": fa}"; // Invalid JSON + Map params = new HashMap<>(); + params.put("index", "test-index"); + FakeRestRequest restRequest = new FakeRestRequest.Builder(xContentRegistry()).withParams(params) + .withMethod(RestRequest.Method.POST) + .withContent(new BytesArray(json), XContentType.JSON) + .build(); + IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> action.prepareRequest(restRequest, mock(NodeClient.class)) + ); + assertThat(e.getMessage(), containsString("Request body must be valid JSON")); + } + + public void testParseScaleDownValueMultipleFields() { + String json = "{\"search_only\": true, \"unknown_field\": \"value\"}"; + Map params = new HashMap<>(); + params.put("index", "test-index"); + FakeRestRequest restRequest = new FakeRestRequest.Builder(xContentRegistry()).withParams(params) + .withMethod(RestRequest.Method.POST) + .withContent(new BytesArray(json), XContentType.JSON) + .build(); + + IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> action.prepareRequest(restRequest, mock(NodeClient.class)) + ); + assertThat(e.getMessage(), containsString("Unknown parameter [unknown_field]. Only [search_only] is allowed.")); + } +}