diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/SnapshotStatusApisIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/SnapshotStatusApisIT.java index 8b6869aa1d81a..123277a3780a2 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/SnapshotStatusApisIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/SnapshotStatusApisIT.java @@ -619,9 +619,9 @@ public void testSnapshotStatusApiFailureForTooManyShardsAcrossSnapshots() throws ); assertEquals(exception.status(), RestStatus.TOO_MANY_REQUESTS); assertTrue( - exception.getMessage().endsWith(" is more than the maximum allowed value of shard count [2] for snapshot status request") + exception.getMessage().contains(" is more than the maximum allowed value of shard count [2] for snapshot status request") ); - }, 1, TimeUnit.MINUTES); + }); // across multiple snapshots assertBusy(() -> { @@ -636,13 +636,13 @@ public void testSnapshotStatusApiFailureForTooManyShardsAcrossSnapshots() throws ); assertEquals(exception.status(), RestStatus.TOO_MANY_REQUESTS); assertTrue( - exception.getMessage().endsWith(" is more than the maximum allowed value of shard count [2] for snapshot status request") + exception.getMessage().contains(" is more than the maximum allowed value of shard count [2] for snapshot status request") ); - }, 1, TimeUnit.MINUTES); + }); logger.info("Reset MAX_SHARDS_ALLOWED_IN_STATUS_API to default value"); updateSettingsRequest.persistentSettings(Settings.builder().putNull(MAX_SHARDS_ALLOWED_IN_STATUS_API.getKey())); - assertAcked(internalCluster().client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); + assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); } public void testSnapshotStatusForIndexFilter() throws Exception { @@ -666,6 +666,7 @@ public void testSnapshotStatusForIndexFilter() throws Exception { String snapshot = "test-snap-1"; createSnapshot(repositoryName, snapshot, List.of(index1, index2, index3)); + // for a completed snapshot assertBusy(() -> { SnapshotStatus snapshotsStatus = client().admin() .cluster() @@ -682,6 +683,96 @@ public void testSnapshotStatusForIndexFilter() throws Exception { }, 1, TimeUnit.MINUTES); } + public void testSnapshotStatusForIndexFilterForInProgressSnapshot() throws Exception { + String repositoryName = "test-repo"; + createRepository(repositoryName, "mock", Settings.builder().put("location", randomRepoPath()).put("block_on_data", true)); + + logger.info("Create indices"); + String index1 = "test-idx-1"; + String index2 = "test-idx-2"; + String index3 = "test-idx-3"; + createIndex(index1, index2, index3); + ensureGreen(); + + logger.info("Indexing some data"); + for (int i = 0; i < 10; i++) { + index(index1, "_doc", Integer.toString(i), "foo", "bar" + i); + index(index2, "_doc", Integer.toString(i), "foo", "baz" + i); + index(index3, "_doc", Integer.toString(i), "foo", "baz" + i); + } + refresh(); + String inProgressSnapshot = "test-in-progress-snapshot"; + + logger.info("Create snapshot"); + ActionFuture createSnapshotResponseActionFuture = startFullSnapshot(repositoryName, inProgressSnapshot); + + logger.info("Block data node"); + waitForBlockOnAnyDataNode(repositoryName, TimeValue.timeValueMinutes(1)); + awaitNumberOfSnapshotsInProgress(1); + + // test normal functioning of index filter for in progress snapshot + assertBusy(() -> { + SnapshotStatus snapshotsStatus = client().admin() + .cluster() + .prepareSnapshotStatus(repositoryName) + .setSnapshots(inProgressSnapshot) + .setIndices(index1, index2) + .get() + .getSnapshots() + .get(0); + Map snapshotIndexStatusMap = snapshotsStatus.getIndices(); + // Although the snapshot contains 3 indices, the response of status api call only contains results for 2 + assertEquals(snapshotIndexStatusMap.size(), 2); + assertEquals(snapshotIndexStatusMap.keySet(), Set.of(index1, index2)); + }); + + // when a non-existent index is requested in the index-filter + assertBusy(() -> { + // failure due to index not found in snapshot + final String nonExistentIndex1 = "non-existent-index-1"; + final String nonExistentIndex2 = "non-existent-index-2"; + Exception ex = expectThrows( + Exception.class, + () -> client().admin() + .cluster() + .prepareSnapshotStatus(repositoryName) + .setSnapshots(inProgressSnapshot) + .setIndices(index1, index2, nonExistentIndex1, nonExistentIndex2) + .execute() + .actionGet() + ); + String cause = String.format( + Locale.ROOT, + "indices [%s] missing in snapshot [%s] of repository [%s]", + String.join(", ", List.of(nonExistentIndex2, nonExistentIndex1)), + inProgressSnapshot, + repositoryName + ); + assertEquals(cause, ex.getCause().getMessage()); + + // no error for ignore_unavailable = true and status response contains only the found indices + SnapshotStatus snapshotsStatus = client().admin() + .cluster() + .prepareSnapshotStatus(repositoryName) + .setSnapshots(inProgressSnapshot) + .setIndices(index1, index2, nonExistentIndex1, nonExistentIndex2) + .setIgnoreUnavailable(true) + .get() + .getSnapshots() + .get(0); + + Map snapshotIndexStatusMap = snapshotsStatus.getIndices(); + assertEquals(snapshotIndexStatusMap.size(), 2); + assertEquals(snapshotIndexStatusMap.keySet(), Set.of(index1, index2)); + }); + + logger.info("Unblock data node"); + unblockAllDataNodes(repositoryName); + + logger.info("Wait for snapshot to finish"); + waitForCompletion(repositoryName, inProgressSnapshot, TimeValue.timeValueSeconds(60)); + } + public void testSnapshotStatusFailuresWithIndexFilter() throws Exception { String repositoryName = "test-repo"; String index1 = "test-idx-1"; @@ -705,6 +796,39 @@ public void testSnapshotStatusFailuresWithIndexFilter() throws Exception { createSnapshot(repositoryName, snapshot1, List.of(index1, index2, index3)); createSnapshot(repositoryName, snapshot2, List.of(index1)); + assertBusy(() -> { + // failure due to passing index filter for _all value of repository param + Exception ex = expectThrows( + Exception.class, + () -> client().admin() + .cluster() + .prepareSnapshotStatus("_all") + .setSnapshots(snapshot1) + .setIndices(index1, index2, index3) + .execute() + .actionGet() + ); + String cause = + "index list filter is supported only when a single 'repository' is passed, but found 'repository' param = [_all]"; + assertTrue(ex.getMessage().contains(cause)); + }); + + assertBusy(() -> { + // failure due to passing index filter for _all value of snapshot param --> gets translated as a blank array + Exception ex = expectThrows( + Exception.class, + () -> client().admin() + .cluster() + .prepareSnapshotStatus(repositoryName) + .setSnapshots() + .setIndices(index1, index2, index3) + .execute() + .actionGet() + ); + String cause = "index list filter is supported only when a single 'snapshot' is passed, but found 'snapshot' param = [_all]"; + assertTrue(ex.getMessage().contains(cause)); + }); + assertBusy(() -> { // failure due to passing index filter for multiple snapshots ActionRequestValidationException ex = expectThrows( @@ -717,9 +841,10 @@ public void testSnapshotStatusFailuresWithIndexFilter() throws Exception { .execute() .actionGet() ); - String cause = "index list filter is supported only for a single snapshot"; + String cause = + "index list filter is supported only when a single 'snapshot' is passed, but found 'snapshot' param = [[test-snap-1, test-snap-2]]"; assertTrue(ex.getMessage().contains(cause)); - }, 1, TimeUnit.MINUTES); + }); assertBusy(() -> { // failure due to index not found in snapshot @@ -743,7 +868,18 @@ public void testSnapshotStatusFailuresWithIndexFilter() throws Exception { ); assertEquals(cause, ex.getCause().getMessage()); - }, 1, TimeUnit.MINUTES); + // no error for ignore_unavailable = true and status response contains only the found indices + SnapshotStatus snapshotsStatus = client().admin() + .cluster() + .prepareSnapshotStatus(repositoryName) + .setSnapshots(snapshot2) + .setIndices(index1, index2, index3) + .setIgnoreUnavailable(true) + .get() + .getSnapshots() + .get(0); + assertEquals(1, snapshotsStatus.getIndices().size()); + }); assertBusy(() -> { // failure due to too many shards requested @@ -763,12 +899,148 @@ public void testSnapshotStatusFailuresWithIndexFilter() throws Exception { .actionGet() ); assertEquals(ex.status(), RestStatus.TOO_MANY_REQUESTS); - assertTrue(ex.getMessage().endsWith(" is more than the maximum allowed value of shard count [2] for snapshot status request")); + assertTrue(ex.getMessage().contains(" is more than the maximum allowed value of shard count [2] for snapshot status request")); logger.info("Reset MAX_SHARDS_ALLOWED_IN_STATUS_API to default value"); updateSettingsRequest.persistentSettings(Settings.builder().putNull(MAX_SHARDS_ALLOWED_IN_STATUS_API.getKey())); - assertAcked(internalCluster().client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); - }, 2, TimeUnit.MINUTES); + assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); + }); + } + + public void testSnapshotStatusShardLimitOfResponseForInProgressSnapshot() throws Exception { + logger.info("Create repository"); + String repositoryName = "test-repo"; + createRepository( + repositoryName, + "mock", + Settings.builder() + .put("location", randomRepoPath()) + .put("compress", false) + .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES) + .put("wait_after_unblock", 200) + ); + + logger.info("Create indices"); + String index1 = "test-idx-1"; + String index2 = "test-idx-2"; + String index3 = "test-idx-3"; + assertAcked(prepareCreate(index1, 1, Settings.builder().put("number_of_shards", 1).put("number_of_replicas", 0))); + assertAcked(prepareCreate(index2, 1, Settings.builder().put("number_of_shards", 1).put("number_of_replicas", 0))); + assertAcked(prepareCreate(index3, 1, Settings.builder().put("number_of_shards", 1).put("number_of_replicas", 0))); + ensureGreen(); + + logger.info("Index some data"); + indexRandomDocs(index1, 10); + indexRandomDocs(index2, 10); + indexRandomDocs(index3, 10); + + logger.info("Create completed snapshot"); + String completedSnapshot = "test-completed-snapshot"; + String blockedNode = blockNodeWithIndex(repositoryName, index1); + client().admin().cluster().prepareCreateSnapshot(repositoryName, completedSnapshot).setWaitForCompletion(false).get(); + waitForBlock(blockedNode, repositoryName, TimeValue.timeValueSeconds(60)); + unblockNode(repositoryName, blockedNode); + waitForCompletion(repositoryName, completedSnapshot, TimeValue.timeValueSeconds(60)); + + logger.info("Index some more data"); + indexRandomDocs(index1, 10); + indexRandomDocs(index2, 10); + indexRandomDocs(index3, 10); + refresh(); + + logger.info("Create in-progress snapshot"); + String inProgressSnapshot = "test-in-progress-snapshot"; + blockedNode = blockNodeWithIndex(repositoryName, index1); + client().admin().cluster().prepareCreateSnapshot(repositoryName, inProgressSnapshot).setWaitForCompletion(false).get(); + waitForBlock(blockedNode, repositoryName, TimeValue.timeValueSeconds(60)); + List snapshotStatuses = client().admin() + .cluster() + .prepareSnapshotStatus(repositoryName) + .setSnapshots(inProgressSnapshot, completedSnapshot) + .get() + .getSnapshots(); + + assertEquals(2, snapshotStatuses.size()); + assertEquals(SnapshotsInProgress.State.STARTED, snapshotStatuses.get(0).getState()); + assertEquals(SnapshotsInProgress.State.SUCCESS, snapshotStatuses.get(1).getState()); + + logger.info("Set MAX_SHARDS_ALLOWED_IN_STATUS_API to a low value"); + ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest(); + updateSettingsRequest.persistentSettings(Settings.builder().put(MAX_SHARDS_ALLOWED_IN_STATUS_API.getKey(), 1)); + assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); + + // shard limit exceeded due to inProgress snapshot alone @ without index-filter + assertBusy(() -> { + CircuitBreakingException exception = expectThrows( + CircuitBreakingException.class, + () -> client().admin() + .cluster() + .prepareSnapshotStatus(repositoryName) + .setSnapshots(inProgressSnapshot) + .execute() + .actionGet() + ); + assertEquals(exception.status(), RestStatus.TOO_MANY_REQUESTS); + assertTrue( + exception.getMessage().contains(" is more than the maximum allowed value of shard count [1] for snapshot status request") + ); + }); + + // shard limit exceeded due to inProgress snapshot alone @ with index-filter + assertBusy(() -> { + CircuitBreakingException exception = expectThrows( + CircuitBreakingException.class, + () -> client().admin() + .cluster() + .prepareSnapshotStatus(repositoryName) + .setSnapshots(inProgressSnapshot) + .setIndices(index1, index2) + .execute() + .actionGet() + ); + assertEquals(exception.status(), RestStatus.TOO_MANY_REQUESTS); + assertTrue( + exception.getMessage().contains(" is more than the maximum allowed value of shard count [1] for snapshot status request") + ); + }); + + logger.info("Set MAX_SHARDS_ALLOWED_IN_STATUS_API to a slightly higher value"); + updateSettingsRequest.persistentSettings(Settings.builder().put(MAX_SHARDS_ALLOWED_IN_STATUS_API.getKey(), 5)); + assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); + + // shard limit exceeded due to passing for inProgress but failing for current + completed + + assertBusy(() -> { + SnapshotStatus inProgressSnapshotStatus = client().admin() + .cluster() + .prepareSnapshotStatus(repositoryName) + .setSnapshots(inProgressSnapshot) + .get() + .getSnapshots() + .get(0); + assertEquals(3, inProgressSnapshotStatus.getShards().size()); + + CircuitBreakingException exception = expectThrows( + CircuitBreakingException.class, + () -> client().admin() + .cluster() + .prepareSnapshotStatus(repositoryName) + .setSnapshots(inProgressSnapshot, completedSnapshot) + .execute() + .actionGet() + ); + assertEquals(exception.status(), RestStatus.TOO_MANY_REQUESTS); + assertTrue( + exception.getMessage().contains(" is more than the maximum allowed value of shard count [5] for snapshot status request") + ); + }); + + unblockNode(repositoryName, blockedNode); + waitForCompletion(repositoryName, inProgressSnapshot, TimeValue.timeValueSeconds(60)); + + logger.info("Reset MAX_SHARDS_ALLOWED_IN_STATUS_API to default value"); + updateSettingsRequest.persistentSettings(Settings.builder().putNull(MAX_SHARDS_ALLOWED_IN_STATUS_API.getKey())); + assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); } private static SnapshotIndexShardStatus stateFirstShard(SnapshotStatus snapshotStatus, String indexName) { diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotsStatusRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotsStatusRequest.java index 3d7fb5b6beb56..a270dcfa53474 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotsStatusRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotsStatusRequest.java @@ -41,6 +41,7 @@ import org.opensearch.core.common.io.stream.StreamOutput; import java.io.IOException; +import java.util.Arrays; import static org.opensearch.action.ValidateActions.addValidationError; @@ -124,8 +125,20 @@ public ActionRequestValidationException validate() { if (snapshots == null) { validationException = addValidationError("snapshots is null", validationException); } - if (indices.length != 0 && snapshots.length != 1) { - validationException = addValidationError("index list filter is supported only for a single snapshot", validationException); + if (indices.length != 0) { + if (repository.equals("_all")) { + String error = + "index list filter is supported only when a single 'repository' is passed, but found 'repository' param = [_all]"; + validationException = addValidationError(error, validationException); + } + if (snapshots.length != 1) { + // snapshot param was '_all' (length = 0) or a list of snapshots (length > 1) + String snapshotParamValue = snapshots.length == 0 ? "_all" : Arrays.toString(snapshots); + String error = "index list filter is supported only when a single 'snapshot' is passed, but found 'snapshot' param = [" + + snapshotParamValue + + "]"; + validationException = addValidationError(error, validationException); + } } return validationException; } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java index 8228cb6301c8c..2c8b06bb5e8fe 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java @@ -99,8 +99,14 @@ public class TransportSnapshotsStatusAction extends TransportClusterManagerNodeA private final TransportNodesSnapshotsStatus transportNodesSnapshotsStatus; + private Set requestedIndexNames; + private long maximumAllowedShardCount; + private int totalShardsRequiredInResponse; + + private boolean requestUsesIndexFilter; + @Inject public TransportSnapshotsStatusAction( TransportService transportService, @@ -145,25 +151,21 @@ protected void clusterManagerOperation( final ClusterState state, final ActionListener listener ) throws Exception { + setupForRequest(request); + final SnapshotsInProgress snapshotsInProgress = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); List currentSnapshots = SnapshotsService.currentSnapshots( snapshotsInProgress, request.repository(), Arrays.asList(request.snapshots()) ); + if (currentSnapshots.isEmpty()) { buildResponse(snapshotsInProgress, request, currentSnapshots, null, listener); return; } - Set nodesIds = new HashSet<>(); - for (SnapshotsInProgress.Entry entry : currentSnapshots) { - for (final SnapshotsInProgress.ShardSnapshotStatus status : entry.shards().values()) { - if (status.nodeId() != null) { - nodesIds.add(status.nodeId()); - } - } - } + Set nodesIds = getNodeIdsOfCurrentSnapshots(request, currentSnapshots); if (!nodesIds.isEmpty()) { // There are still some snapshots running - check their progress @@ -192,6 +194,97 @@ protected void clusterManagerOperation( } + private void setupForRequest(SnapshotsStatusRequest request) { + requestedIndexNames = new HashSet<>(Arrays.asList(request.indices())); + requestUsesIndexFilter = requestedIndexNames.isEmpty() == false; + totalShardsRequiredInResponse = 0; + maximumAllowedShardCount = clusterService.getClusterSettings().get(MAX_SHARDS_ALLOWED_IN_STATUS_API); + } + + /* + * To get the node IDs of the relevant (according to the index filter) shards which are part of current snapshots + * It also deals with any missing indices (for index-filter case) and calculates the number of shards contributed by all + * the current snapshots to the total count (irrespective of index-filter) + * If this count exceeds the limit, CircuitBreakingException is thrown + * */ + private Set getNodeIdsOfCurrentSnapshots(final SnapshotsStatusRequest request, List currentSnapshots) + throws CircuitBreakingException { + Set nodesIdsOfCurrentSnapshotShards = new HashSet<>(); + int totalShardsAcrossCurrentSnapshots = 0; + + for (SnapshotsInProgress.Entry currentSnapshotEntry : currentSnapshots) { + if (currentSnapshotEntry.remoteStoreIndexShallowCopyV2()) { + // skip current shallow v2 snapshots + continue; + } + if (requestUsesIndexFilter) { + // index-filter is allowed only for a single snapshot, which has to be this one + // first check if any requested indices are missing from this current snapshot + + final Set indicesInCurrentSnapshot = currentSnapshotEntry.indices() + .stream() + .map(IndexId::getName) + .collect(Collectors.toSet()); + + final Set indicesNotFound = requestedIndexNames.stream() + .filter(index -> indicesInCurrentSnapshot.contains(index) == false) + .collect(Collectors.toSet()); + + if (indicesNotFound.isEmpty() == false) { + handleIndexNotFound( + requestedIndexNames, + indicesNotFound, + request, + currentSnapshotEntry.snapshot().getSnapshotId().getName(), + false + ); + } + // the actual no. of shards contributed by this current snapshot will now be calculated + } else { + // all shards of this current snapshot are required in response + totalShardsAcrossCurrentSnapshots += currentSnapshotEntry.shards().size(); + } + + for (final Map.Entry shardStatusEntry : currentSnapshotEntry.shards() + .entrySet()) { + SnapshotsInProgress.ShardSnapshotStatus shardStatus = shardStatusEntry.getValue(); + boolean indexPresentInFilter = requestedIndexNames.contains(shardStatusEntry.getKey().getIndexName()); + + if (requestUsesIndexFilter && indexPresentInFilter) { + // count only those shards whose index belongs to the index-filter + totalShardsAcrossCurrentSnapshots++; + + // for non-index filter case, we already counted all the shards of this current snapshot (non-shallow v2) + } + + if (shardStatus.nodeId() != null) { + if (requestUsesIndexFilter) { + if (indexPresentInFilter) { + // include node only if the index containing this shard belongs to the index filter + nodesIdsOfCurrentSnapshotShards.add(shardStatus.nodeId()); + } + } else { + nodesIdsOfCurrentSnapshotShards.add(shardStatus.nodeId()); + } + } + } + } + + totalShardsRequiredInResponse += totalShardsAcrossCurrentSnapshots; + if (totalShardsRequiredInResponse > maximumAllowedShardCount) { + // index-filter is allowed only for a single snapshot. If index-filter is being used and limit got exceeded, + // this snapshot is current and its relevant indices contribute more shards than the limit + + // if index-filter is not being used and limit got exceed, there could be more shards required in response coming from completed + // snapshots + // but since the limit is already exceeded, we can fail request here + boolean couldInvolveMoreShards = requestUsesIndexFilter == false; + handleMaximumAllowedShardCountExceeded(request.repository(), totalShardsRequiredInResponse, couldInvolveMoreShards); + } + + return nodesIdsOfCurrentSnapshotShards; + } + private void buildResponse( SnapshotsInProgress snapshotsInProgress, SnapshotsStatusRequest request, @@ -215,6 +308,10 @@ private void buildResponse( List shardStatusBuilder = new ArrayList<>(); Map indexIdLookup = null; for (final Map.Entry shardEntry : entry.shards().entrySet()) { + if (requestUsesIndexFilter && requestedIndexNames.contains(shardEntry.getKey().getIndexName()) == false) { + // skip shard if its index does not belong to the index-filter + continue; + } SnapshotsInProgress.ShardSnapshotStatus status = shardEntry.getValue(); if (status.nodeId() != null) { // We should have information about this shard from the shard: @@ -320,7 +417,6 @@ private void loadRepositoryData( String repositoryName, ActionListener listener ) { - maximumAllowedShardCount = clusterService.getClusterSettings().get(MAX_SHARDS_ALLOWED_IN_STATUS_API); final StepListener repositoryDataListener = new StepListener<>(); repositoriesService.getRepositoryData(repositoryName, repositoryDataListener); repositoryDataListener.whenComplete(repositoryData -> { @@ -343,8 +439,7 @@ private void loadRepositoryData( snapshotInfo ); boolean isShallowV2Snapshot = snapshotInfo.getPinnedTimestamp() > 0; - long initialSnapshotTotalSize = 0; - if (isShallowV2Snapshot && request.indices().length == 0) { + if (isShallowV2Snapshot && requestUsesIndexFilter == false) { // TODO: add primary store size in bytes at the snapshot level } @@ -430,7 +525,10 @@ private Map snapshotsInfo( .stream() .filter(s -> requestedSnapshotNames.contains(s.getName())) .collect(Collectors.toMap(SnapshotId::getName, Function.identity())); + + // for no index filter-case and excludes shards from shallow v2 snapshots int totalShardsAcrossSnapshots = 0; + for (final String snapshotName : request.snapshots()) { if (currentSnapshotNames.contains(snapshotName)) { // we've already found this snapshot in the current snapshot entries, so skip over @@ -453,23 +551,15 @@ private Map snapshotsInfo( } SnapshotInfo snapshotInfo = snapshot(snapshotsInProgress, repositoryName, snapshotId); boolean isV2Snapshot = snapshotInfo.getPinnedTimestamp() > 0; - if (isV2Snapshot == false && request.indices().length == 0) { + if (isV2Snapshot == false && requestUsesIndexFilter == false) { totalShardsAcrossSnapshots += snapshotInfo.totalShards(); } snapshotsInfoMap.put(snapshotId, snapshotInfo); } - if (totalShardsAcrossSnapshots > maximumAllowedShardCount && request.indices().length == 0) { - String message = "[" - + repositoryName - + ":" - + String.join(", ", request.snapshots()) - + "]" - + " Total shard count [" - + totalShardsAcrossSnapshots - + "] is more than the maximum allowed value of shard count [" - + maximumAllowedShardCount - + "] for snapshot status request"; - throw new CircuitBreakingException(message, CircuitBreaker.Durability.PERMANENT); + totalShardsRequiredInResponse += totalShardsAcrossSnapshots; + if (totalShardsRequiredInResponse > maximumAllowedShardCount && requestUsesIndexFilter == false) { + // includes shard contributions from all snapshots (current and completed) + handleMaximumAllowedShardCountExceeded(repositoryName, totalShardsRequiredInResponse, false); } return unmodifiableMap(snapshotsInfoMap); } @@ -492,52 +582,46 @@ private Map snapshotShards( final RepositoryData repositoryData, final SnapshotInfo snapshotInfo ) throws IOException { - final Set requestedIndexNames = Sets.newHashSet(request.indices()); String snapshotName = snapshotInfo.snapshotId().getName(); - Set indices = Sets.newHashSet(snapshotInfo.indices()); - if (requestedIndexNames.isEmpty() == false) { - Set finalIndices = indices; - List indicesNotFound = requestedIndexNames.stream() - .filter(i -> finalIndices.contains(i) == false) - .collect(Collectors.toList()); + Set indicesToProcess; + if (requestUsesIndexFilter) { + Set snapshotIndices = Sets.newHashSet(snapshotInfo.indices()); + Set indicesNotFound = requestedIndexNames.stream() + .filter(index -> snapshotIndices.contains(index) == false) + .collect(Collectors.toSet()); if (indicesNotFound.isEmpty() == false) { - handleIndexNotFound(String.join(", ", indicesNotFound), request, snapshotName, repositoryName); + boolean moreMissingIndicesPossible = indicesNotFound.size() == requestedIndexNames.size(); + handleIndexNotFound(requestedIndexNames, indicesNotFound, request, snapshotName, moreMissingIndicesPossible); } - indices = requestedIndexNames; + indicesToProcess = requestedIndexNames; + } else { + // all indices of this snapshot + indicesToProcess = Sets.newHashSet(snapshotInfo.indices()); } final Repository repository = repositoriesService.repository(repositoryName); boolean isV2Snapshot = snapshotInfo.getPinnedTimestamp() > 0; + + // for index filter-case and excludes shards from shallow v2 snapshots int totalShardsAcrossIndices = 0; final Map indexMetadataMap = new HashMap<>(); - - for (String index : indices) { + for (String index : indicesToProcess) { IndexId indexId = repositoryData.resolveIndexId(index); IndexMetadata indexMetadata = repository.getSnapshotIndexMetaData(repositoryData, snapshotInfo.snapshotId(), indexId); if (indexMetadata != null) { - if (requestedIndexNames.isEmpty() == false && isV2Snapshot == false) { + if (requestUsesIndexFilter && isV2Snapshot == false) { totalShardsAcrossIndices += indexMetadata.getNumberOfShards(); } indexMetadataMap.put(indexId, indexMetadata); - } else if (requestedIndexNames.isEmpty() == false) { - handleIndexNotFound(index, request, snapshotName, repositoryName); + } else if (requestUsesIndexFilter) { + handleIndexNotFound(indicesToProcess, Collections.singleton(index), request, snapshotName, true); } } - if (totalShardsAcrossIndices > maximumAllowedShardCount && requestedIndexNames.isEmpty() == false && isV2Snapshot == false) { - String message = "[" - + repositoryName - + ":" - + String.join(", ", request.snapshots()) - + "]" - + " Total shard count [" - + totalShardsAcrossIndices - + "] across the requested indices [" - + requestedIndexNames.stream().collect(Collectors.joining(", ")) - + "] is more than the maximum allowed value of shard count [" - + maximumAllowedShardCount - + "] for snapshot status request"; - throw new CircuitBreakingException(message, CircuitBreaker.Durability.PERMANENT); + totalShardsRequiredInResponse += totalShardsAcrossIndices; + if (totalShardsRequiredInResponse > maximumAllowedShardCount && requestUsesIndexFilter && isV2Snapshot == false) { + // index-filter is allowed only for a single snapshot, which has to be this one + handleMaximumAllowedShardCountExceeded(request.repository(), totalShardsRequiredInResponse, false); } final Map shardStatus = new HashMap<>(); @@ -563,7 +647,6 @@ private Map snapshotShards( // could not be taken due to partial being set to false. shardSnapshotStatus = IndexShardSnapshotStatus.newFailed("skipped"); } else { - // TODO: to be refactored later if (isV2Snapshot) { shardSnapshotStatus = IndexShardSnapshotStatus.newDone(0, 0, 0, 0, 0, 0, null); } else { @@ -578,21 +661,55 @@ private Map snapshotShards( return unmodifiableMap(shardStatus); } - private void handleIndexNotFound(String index, SnapshotsStatusRequest request, String snapshotName, String repositoryName) { + private void handleIndexNotFound( + Set indicesToProcess, + Set indicesNotFound, + SnapshotsStatusRequest request, + String snapshotName, + boolean moreMissingIndicesPossible + ) throws IndexNotFoundException { + String indices = String.join(", ", indicesNotFound); + if (moreMissingIndicesPossible) { + indices = indices.concat(" and possibly more indices"); + } if (request.ignoreUnavailable()) { - // ignoring unavailable index + // ignoring unavailable indices logger.debug( "snapshot status request ignoring indices [{}], not found in snapshot[{}] in repository [{}]", - index, + indices, snapshotName, - repositoryName + request.repository() ); + + // remove unavailable indices from the set to be processed + indicesToProcess.removeAll(indicesNotFound); } else { - String cause = "indices [" + index + "] missing in snapshot [" + snapshotName + "] of repository [" + repositoryName + "]"; - throw new IndexNotFoundException(index, new IllegalArgumentException(cause)); + String cause = "indices [" + + indices + + "] missing in snapshot [" + + snapshotName + + "] of repository [" + + request.repository() + + "]"; + throw new IndexNotFoundException(indices, new IllegalArgumentException(cause)); } } + private void handleMaximumAllowedShardCountExceeded(String repositoryName, int totalContributingShards, boolean couldInvolveMoreShards) + throws CircuitBreakingException { + String shardCount = "[" + totalContributingShards + (couldInvolveMoreShards ? "+" : "") + "]"; + String message = "[" + + repositoryName + + "] Total shard count " + + shardCount + + " is more than the maximum allowed value of shard count [" + + maximumAllowedShardCount + + "] for snapshot status request. Try narrowing down the request by using a snapshot list or " + + "an index list for a singular snapshot."; + + throw new CircuitBreakingException(message, CircuitBreaker.Durability.PERMANENT); + } + private static SnapshotShardFailure findShardFailure(List shardFailures, ShardId shardId) { for (SnapshotShardFailure shardFailure : shardFailures) { if (shardId.getIndexName().equals(shardFailure.index()) && shardId.getId() == shardFailure.shardId()) { diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestSnapshotsStatusAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestSnapshotsStatusAction.java index 502be16f2fa8e..50380c506358f 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestSnapshotsStatusAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestSnapshotsStatusAction.java @@ -82,6 +82,9 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC snapshots = Strings.EMPTY_ARRAY; } String[] indices = request.paramAsStringArray("index", Strings.EMPTY_ARRAY); + if (indices.length == 1 && "_all".equalsIgnoreCase(indices[0])) { + indices = Strings.EMPTY_ARRAY; + } SnapshotsStatusRequest snapshotsStatusRequest = snapshotsStatusRequest(repository).snapshots(snapshots).indices(indices); snapshotsStatusRequest.ignoreUnavailable(request.paramAsBoolean("ignore_unavailable", snapshotsStatusRequest.ignoreUnavailable()));