diff --git a/docs/reference/modules/threadpool.asciidoc b/docs/reference/modules/threadpool.asciidoc index f3ecc98156115..5a6e59039b17f 100644 --- a/docs/reference/modules/threadpool.asciidoc +++ b/docs/reference/modules/threadpool.asciidoc @@ -43,6 +43,11 @@ There are several thread pools, but the important ones include: keep-alive of `5m` and a max of `min(5, (`<>`) / 2)`. +`snapshot_meta`:: + For snapshot repository metadata read operations. Thread pool type is `scaling` with a + keep-alive of `5m` and a max of `min(50, (`<>` pass:[ * ]3))`. + `warmer`:: For segment warm-up operations. Thread pool type is `scaling` with a keep-alive of `5m` and a max of `min(5, (`<, + "version": , + "indices": [], + "data_streams": [], + "feature_states": [], + "include_global_state": true, + "state": "SUCCESS", + "start_time": "2020-07-06T21:55:18.129Z", + "start_time_in_millis": 1593093628850, + "end_time": "2020-07-06T21:55:18.129Z", + "end_time_in_millis": 1593094752018, + "duration_in_millis": 0, + "failures": [], + "shards": { + "total": 0, + "failed": 0, + "successful": 0 + } + }, + { + "snapshot": "snapshot_2", + "uuid": "vdRctLCxSketdKb54xw67g", + "repository": "my_repository", + "version_id": , + "version": , + "indices": [], + "data_streams": [], + "feature_states": [], + "include_global_state": true, + "state": "SUCCESS", + "start_time": "2020-07-06T21:55:18.130Z", + "start_time_in_millis": 1593093628851, + "end_time": "2020-07-06T21:55:18.130Z", + "end_time_in_millis": 1593094752019, + "duration_in_millis": 1, + "failures": [], + "shards": { + "total": 0, + "failed": 0, + "successful": 0 + } + } + ], + "next": "c25hcHNob3RfMixteV9yZXBvc2l0b3J5LHNuYXBzaG90XzI=" +} +---- +// TESTRESPONSE[s/"uuid": "dKb54xw67gvdRctLCxSket"/"uuid": $body.snapshots.0.uuid/] +// TESTRESPONSE[s/"uuid": "vdRctLCxSketdKb54xw67g"/"uuid": $body.snapshots.1.uuid/] +// TESTRESPONSE[s/"version_id": /"version_id": $body.snapshots.0.version_id/] +// TESTRESPONSE[s/"version": /"version": $body.snapshots.0.version/] +// TESTRESPONSE[s/"start_time": "2020-07-06T21:55:18.129Z"/"start_time": $body.snapshots.0.start_time/] +// TESTRESPONSE[s/"start_time": "2020-07-06T21:55:18.130Z"/"start_time": $body.snapshots.1.start_time/] +// TESTRESPONSE[s/"start_time_in_millis": 1593093628850/"start_time_in_millis": $body.snapshots.0.start_time_in_millis/] +// TESTRESPONSE[s/"start_time_in_millis": 1593093628851/"start_time_in_millis": $body.snapshots.1.start_time_in_millis/] +// TESTRESPONSE[s/"end_time": "2020-07-06T21:55:18.129Z"/"end_time": $body.snapshots.0.end_time/] +// TESTRESPONSE[s/"end_time": "2020-07-06T21:55:18.130Z"/"end_time": $body.snapshots.1.end_time/] +// TESTRESPONSE[s/"end_time_in_millis": 1593094752018/"end_time_in_millis": $body.snapshots.0.end_time_in_millis/] +// TESTRESPONSE[s/"end_time_in_millis": 1593094752019/"end_time_in_millis": $body.snapshots.1.end_time_in_millis/] +// TESTRESPONSE[s/"duration_in_millis": 0/"duration_in_millis": $body.snapshots.0.duration_in_millis/] +// TESTRESPONSE[s/"duration_in_millis": 1/"duration_in_millis": $body.snapshots.1.duration_in_millis/] + +A subsequent request for the remaining snapshots can then be made using the `next` value from the previous response as `after` parameter. + +[source,console] +---- +GET /_snapshot/my_repository/snapshot*?size=2&sort=name&after=c25hcHNob3RfMixteV9yZXBvc2l0b3J5LHNuYXBzaG90XzI= +---- + +The API returns the following response: + +[source,console-result] +---- +{ + "snapshots": [ + { + "snapshot": "snapshot_3", + "uuid": "dRctdKb54xw67gvLCxSket", + "repository": "my_repository", + "version_id": , + "version": , + "indices": [], + "data_streams": [], + "feature_states": [], + "include_global_state": true, + "state": "SUCCESS", + "start_time": "2020-07-06T21:55:18.129Z", + "start_time_in_millis": 1593093628850, + "end_time": "2020-07-06T21:55:18.129Z", + "end_time_in_millis": 1593094752018, + "duration_in_millis": 0, + "failures": [], + "shards": { + "total": 0, + "failed": 0, + "successful": 0 + } + } + ] +} +---- +// TESTRESPONSE[s/"uuid": "dRctdKb54xw67gvLCxSket"/"uuid": $body.snapshots.0.uuid/] +// TESTRESPONSE[s/"version_id": /"version_id": $body.snapshots.0.version_id/] +// TESTRESPONSE[s/"version": /"version": $body.snapshots.0.version/] +// TESTRESPONSE[s/"start_time": "2020-07-06T21:55:18.129Z"/"start_time": $body.snapshots.0.start_time/] +// TESTRESPONSE[s/"start_time_in_millis": 1593093628850/"start_time_in_millis": $body.snapshots.0.start_time_in_millis/] +// TESTRESPONSE[s/"end_time": "2020-07-06T21:55:18.129Z"/"end_time": $body.snapshots.0.end_time/] +// TESTRESPONSE[s/"end_time_in_millis": 1593094752018/"end_time_in_millis": $body.snapshots.0.end_time_in_millis/] +// TESTRESPONSE[s/"duration_in_millis": 0/"duration_in_millis": $body.snapshots.0.duration_in_millis/] \ No newline at end of file diff --git a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/snapshots/RestGetSnapshotsIT.java b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/snapshots/RestGetSnapshotsIT.java new file mode 100644 index 0000000000000..95496e3e537c6 --- /dev/null +++ b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/snapshots/RestGetSnapshotsIT.java @@ -0,0 +1,247 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.http.snapshots; + +import org.apache.http.client.methods.HttpGet; +import org.elasticsearch.action.ActionFuture; +import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse; +import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsRequest; +import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsResponse; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.Response; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.DeprecationHandler; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.json.JsonXContent; +import org.elasticsearch.core.Tuple; +import org.elasticsearch.search.sort.SortOrder; +import org.elasticsearch.snapshots.AbstractSnapshotIntegTestCase; +import org.elasticsearch.snapshots.SnapshotInfo; +import org.elasticsearch.threadpool.ThreadPool; + +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; + +import static org.elasticsearch.snapshots.AbstractSnapshotIntegTestCase.assertSnapshotListSorted; +import static org.hamcrest.Matchers.in; +import static org.hamcrest.Matchers.is; + +// TODO: dry up duplication across this suite and org.elasticsearch.snapshots.GetSnapshotsIT more +public class RestGetSnapshotsIT extends AbstractSnapshotRestTestCase { + + @Override + protected Settings nodeSettings(int nodeOrdinal, Settings otherSettings) { + return Settings.builder().put(super.nodeSettings(nodeOrdinal, otherSettings)) + .put(ThreadPool.ESTIMATED_TIME_INTERVAL_SETTING.getKey(), 0) // We have tests that check by-timestamp order + .build(); + } + + public void testSortOrder() throws Exception { + final String repoName = "test-repo"; + AbstractSnapshotIntegTestCase.createRepository(logger, repoName, "fs"); + final List snapshotNamesWithoutIndex = + AbstractSnapshotIntegTestCase.createNSnapshots(logger, repoName, randomIntBetween(3, 20)); + + createIndexWithContent("test-index"); + + final List snapshotNamesWithIndex = + AbstractSnapshotIntegTestCase.createNSnapshots(logger, repoName, randomIntBetween(3, 20)); + + final Collection allSnapshotNames = new HashSet<>(snapshotNamesWithIndex); + allSnapshotNames.addAll(snapshotNamesWithoutIndex); + doTestSortOrder(repoName, allSnapshotNames, SortOrder.ASC); + doTestSortOrder(repoName, allSnapshotNames, SortOrder.DESC); + } + + private void doTestSortOrder(String repoName, Collection allSnapshotNames, SortOrder order) throws IOException { + final List defaultSorting = clusterAdmin().prepareGetSnapshots(repoName).setOrder(order).get().getSnapshots(); + assertSnapshotListSorted(defaultSorting, null, order); + assertSnapshotListSorted( + allSnapshotsSorted(allSnapshotNames, repoName, GetSnapshotsRequest.SortBy.NAME, order), + GetSnapshotsRequest.SortBy.NAME, + order + ); + assertSnapshotListSorted( + allSnapshotsSorted(allSnapshotNames, repoName, GetSnapshotsRequest.SortBy.DURATION, order), + GetSnapshotsRequest.SortBy.DURATION, + order + ); + assertSnapshotListSorted( + allSnapshotsSorted(allSnapshotNames, repoName, GetSnapshotsRequest.SortBy.INDICES, order), + GetSnapshotsRequest.SortBy.INDICES, + order + ); + assertSnapshotListSorted( + allSnapshotsSorted(allSnapshotNames, repoName, GetSnapshotsRequest.SortBy.START_TIME, order), + GetSnapshotsRequest.SortBy.START_TIME, + order + ); + } + + public void testResponseSizeLimit() throws Exception { + final String repoName = "test-repo"; + AbstractSnapshotIntegTestCase.createRepository(logger, repoName, "fs"); + final List names = AbstractSnapshotIntegTestCase.createNSnapshots(logger, repoName, randomIntBetween(6, 20)); + for (GetSnapshotsRequest.SortBy sort : GetSnapshotsRequest.SortBy.values()) { + for (SortOrder order : SortOrder.values()) { + logger.info("--> testing pagination for [{}] [{}]", sort, order); + doTestPagination(repoName, names, sort, order); + } + } + } + + private void doTestPagination(String repoName, + List names, + GetSnapshotsRequest.SortBy sort, + SortOrder order) throws IOException { + final List allSnapshotsSorted = allSnapshotsSorted(names, repoName, sort, order); + final Tuple> batch1 = sortedWithLimit(repoName, sort, null, 2, order); + assertEquals(allSnapshotsSorted.subList(0, 2), batch1.v2()); + final Tuple> batch2 = sortedWithLimit(repoName, sort, batch1.v1(), 2, order); + assertEquals(allSnapshotsSorted.subList(2, 4), batch2.v2()); + final int lastBatch = names.size() - batch1.v2().size() - batch2.v2().size(); + final Tuple> batch3 = sortedWithLimit(repoName, sort, batch2.v1(), lastBatch, order); + assertEquals(batch3.v2(), allSnapshotsSorted.subList(batch1.v2().size() + batch2.v2().size(), names.size())); + final Tuple> batch3NoLimit = sortedWithLimit( + repoName, + sort, + batch2.v1(), + GetSnapshotsRequest.NO_LIMIT, + order + ); + assertNull(batch3NoLimit.v1()); + assertEquals(batch3.v2(), batch3NoLimit.v2()); + final Tuple> batch3LargeLimit = sortedWithLimit( + repoName, + sort, + batch2.v1(), + lastBatch + randomIntBetween(1, 100), + order + ); + assertEquals(batch3.v2(), batch3LargeLimit.v2()); + assertNull(batch3LargeLimit.v1()); + } + + public void testSortAndPaginateWithInProgress() throws Exception { + final String repoName = "test-repo"; + AbstractSnapshotIntegTestCase.createRepository(logger, repoName, "mock"); + final Collection allSnapshotNames = + new HashSet<>(AbstractSnapshotIntegTestCase.createNSnapshots(logger, repoName, randomIntBetween(3, 20))); + createIndexWithContent("test-index-1"); + allSnapshotNames.addAll(AbstractSnapshotIntegTestCase.createNSnapshots(logger, repoName, randomIntBetween(3, 20))); + createIndexWithContent("test-index-2"); + + final int inProgressCount = randomIntBetween(6, 20); + final List> inProgressSnapshots = new ArrayList<>(inProgressCount); + AbstractSnapshotIntegTestCase.blockAllDataNodes(repoName); + for (int i = 0; i < inProgressCount; i++) { + final String snapshotName = "snap-" + i; + allSnapshotNames.add(snapshotName); + inProgressSnapshots.add(AbstractSnapshotIntegTestCase.startFullSnapshot(logger, repoName, snapshotName, false)); + } + AbstractSnapshotIntegTestCase.awaitNumberOfSnapshotsInProgress(logger, inProgressCount); + + assertStablePagination(repoName, allSnapshotNames, GetSnapshotsRequest.SortBy.START_TIME); + assertStablePagination(repoName, allSnapshotNames, GetSnapshotsRequest.SortBy.NAME); + assertStablePagination(repoName, allSnapshotNames, GetSnapshotsRequest.SortBy.INDICES); + + AbstractSnapshotIntegTestCase.unblockAllDataNodes(repoName); + for (ActionFuture inProgressSnapshot : inProgressSnapshots) { + AbstractSnapshotIntegTestCase.assertSuccessful(logger, inProgressSnapshot); + } + + assertStablePagination(repoName, allSnapshotNames, GetSnapshotsRequest.SortBy.START_TIME); + assertStablePagination(repoName, allSnapshotNames, GetSnapshotsRequest.SortBy.NAME); + assertStablePagination(repoName, allSnapshotNames, GetSnapshotsRequest.SortBy.INDICES); + } + + private void createIndexWithContent(String indexName) { + logger.info("--> creating index [{}]", indexName); + createIndex(indexName, AbstractSnapshotIntegTestCase.SINGLE_SHARD_NO_REPLICA); + ensureGreen(indexName); + indexDoc(indexName, "some_id", "foo", "bar"); + } + + private static void assertStablePagination(String repoName, + Collection allSnapshotNames, + GetSnapshotsRequest.SortBy sort) throws IOException { + final SortOrder order = randomFrom(SortOrder.values()); + final List allSorted = allSnapshotsSorted(allSnapshotNames, repoName, sort, order); + + for (int i = 1; i <= allSnapshotNames.size(); i++) { + final List subsetSorted = sortedWithLimit(repoName, sort, null, i, order).v2(); + assertEquals(subsetSorted, allSorted.subList(0, i)); + } + + for (int j = 0; j < allSnapshotNames.size(); j++) { + final SnapshotInfo after = allSorted.get(j); + for (int i = 1; i < allSnapshotNames.size() - j; i++) { + final List subsetSorted = sortedWithLimit( + repoName, sort, GetSnapshotsRequest.After.from(after, sort).asQueryParam(), i, order).v2(); + assertEquals(subsetSorted, allSorted.subList(j + 1, j + i + 1)); + } + } + } + + private static List allSnapshotsSorted(Collection allSnapshotNames, + String repoName, + GetSnapshotsRequest.SortBy sortBy, + SortOrder order) throws IOException { + final Request request = baseGetSnapshotsRequest(repoName); + request.addParameter("sort", sortBy.toString()); + if (order == SortOrder.DESC || randomBoolean()) { + request.addParameter("order", order.toString()); + } + final Response response = getRestClient().performRequest(request); + final List snapshotInfos = readSnapshotInfos(response).v2(); + assertEquals(snapshotInfos.size(), allSnapshotNames.size()); + for (SnapshotInfo snapshotInfo : snapshotInfos) { + assertThat(snapshotInfo.snapshotId().getName(), is(in(allSnapshotNames))); + } + return snapshotInfos; + } + + private static Request baseGetSnapshotsRequest(String repoName) { + return new Request(HttpGet.METHOD_NAME, "/_snapshot/" + repoName + "/*"); + } + + private static Tuple> readSnapshotInfos(Response response) throws IOException { + try (InputStream input = response.getEntity().getContent(); + XContentParser parser = JsonXContent.jsonXContent.createParser( + NamedXContentRegistry.EMPTY, DeprecationHandler.THROW_UNSUPPORTED_OPERATION, input)) { + final GetSnapshotsResponse getSnapshotsResponse = GetSnapshotsResponse.fromXContent(parser); + return Tuple.tuple(getSnapshotsResponse.next(), getSnapshotsResponse.getSnapshots()); + } + } + + private static Tuple> sortedWithLimit(String repoName, + GetSnapshotsRequest.SortBy sortBy, + String after, + int size, + SortOrder order) throws IOException { + final Request request = baseGetSnapshotsRequest(repoName); + request.addParameter("sort", sortBy.toString()); + if (size != GetSnapshotsRequest.NO_LIMIT || randomBoolean()) { + request.addParameter("size", String.valueOf(size)); + } + if (after != null) { + request.addParameter("after", after); + } + if (order == SortOrder.DESC || randomBoolean()) { + request.addParameter("order", order.toString()); + } + final Response response = getRestClient().performRequest(request); + return readSnapshotInfos(response); + } +} diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/CloneSnapshotIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/CloneSnapshotIT.java index d994220b1d462..02b66df66c8d1 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/CloneSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/CloneSnapshotIT.java @@ -726,6 +726,7 @@ private static BlobStoreIndexShardSnapshots readShardGeneration( ActionRunnable.supply( f, () -> BlobStoreRepository.INDEX_SHARD_SNAPSHOTS_FORMAT.read( + repository.getMetadata().name(), repository.shardContainer(repositoryShardId.index(), repositoryShardId.shardId()), generation, NamedXContentRegistry.EMPTY diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java index 21f9cc10ea0f0..3bd3df0b9927c 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java @@ -1061,6 +1061,7 @@ public void onRequestSent( final ActionFuture deleteResponse = startDeleteSnapshot(repoName, snapshotName); awaitClusterState( + logger, otherDataNode, state -> state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY) .entries() diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/GetSnapshotsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/GetSnapshotsIT.java new file mode 100644 index 0000000000000..93c217e3cd6a0 --- /dev/null +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/GetSnapshotsIT.java @@ -0,0 +1,252 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.snapshots; + +import org.elasticsearch.action.ActionFuture; +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse; +import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsRequest; +import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsRequestBuilder; +import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsResponse; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.Tuple; +import org.elasticsearch.search.sort.SortOrder; +import org.elasticsearch.threadpool.ThreadPool; + +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; + +import static org.hamcrest.Matchers.in; +import static org.hamcrest.Matchers.is; + +public class GetSnapshotsIT extends AbstractSnapshotIntegTestCase { + + @Override + protected Settings nodeSettings(int nodeOrdinal, Settings otherSettings) { + return Settings.builder() + .put(super.nodeSettings(nodeOrdinal, otherSettings)) + .put(ThreadPool.ESTIMATED_TIME_INTERVAL_SETTING.getKey(), 0) // We have tests that check by-timestamp order + .build(); + } + + public void testSortBy() throws Exception { + final String repoName = "test-repo"; + final Path repoPath = randomRepoPath(); + createRepository(repoName, "fs", repoPath); + maybeInitWithOldSnapshotVersion(repoName, repoPath); + final List snapshotNamesWithoutIndex = createNSnapshots(repoName, randomIntBetween(3, 20)); + + createIndexWithContent("test-index"); + + final List snapshotNamesWithIndex = createNSnapshots(repoName, randomIntBetween(3, 20)); + + final Collection allSnapshotNames = new HashSet<>(snapshotNamesWithIndex); + allSnapshotNames.addAll(snapshotNamesWithoutIndex); + + doTestSortOrder(repoName, allSnapshotNames, SortOrder.ASC); + doTestSortOrder(repoName, allSnapshotNames, SortOrder.DESC); + } + + private void doTestSortOrder(String repoName, Collection allSnapshotNames, SortOrder order) { + final List defaultSorting = clusterAdmin().prepareGetSnapshots(repoName).setOrder(order).get().getSnapshots(); + assertSnapshotListSorted(defaultSorting, null, order); + assertSnapshotListSorted( + allSnapshotsSorted(allSnapshotNames, repoName, GetSnapshotsRequest.SortBy.NAME, order), + GetSnapshotsRequest.SortBy.NAME, + order + ); + assertSnapshotListSorted( + allSnapshotsSorted(allSnapshotNames, repoName, GetSnapshotsRequest.SortBy.DURATION, order), + GetSnapshotsRequest.SortBy.DURATION, + order + ); + assertSnapshotListSorted( + allSnapshotsSorted(allSnapshotNames, repoName, GetSnapshotsRequest.SortBy.INDICES, order), + GetSnapshotsRequest.SortBy.INDICES, + order + ); + assertSnapshotListSorted( + allSnapshotsSorted(allSnapshotNames, repoName, GetSnapshotsRequest.SortBy.START_TIME, order), + GetSnapshotsRequest.SortBy.START_TIME, + order + ); + } + + public void testResponseSizeLimit() throws Exception { + final String repoName = "test-repo"; + final Path repoPath = randomRepoPath(); + createRepository(repoName, "fs", repoPath); + maybeInitWithOldSnapshotVersion(repoName, repoPath); + final List names = createNSnapshots(repoName, randomIntBetween(6, 20)); + for (GetSnapshotsRequest.SortBy sort : GetSnapshotsRequest.SortBy.values()) { + for (SortOrder order : SortOrder.values()) { + logger.info("--> testing pagination for [{}] [{}]", sort, order); + doTestPagination(repoName, names, sort, order); + } + } + } + + private void doTestPagination(String repoName, List names, GetSnapshotsRequest.SortBy sort, SortOrder order) { + final List allSnapshotsSorted = allSnapshotsSorted(names, repoName, sort, order); + final Tuple> batch1 = sortedWithLimit(repoName, sort, null, 2, order); + assertEquals(allSnapshotsSorted.subList(0, 2), batch1.v2()); + final Tuple> batch2 = sortedWithLimit(repoName, sort, batch1.v1(), 2, order); + assertEquals(allSnapshotsSorted.subList(2, 4), batch2.v2()); + final int lastBatch = names.size() - batch1.v2().size() - batch2.v2().size(); + final Tuple> batch3 = sortedWithLimit(repoName, sort, batch2.v1(), lastBatch, order); + assertEquals(batch3.v2(), allSnapshotsSorted.subList(batch1.v2().size() + batch2.v2().size(), names.size())); + final Tuple> batch3NoLimit = sortedWithLimit( + repoName, + sort, + batch2.v1(), + GetSnapshotsRequest.NO_LIMIT, + order + ); + assertNull(batch3NoLimit.v1()); + assertEquals(batch3.v2(), batch3NoLimit.v2()); + final Tuple> batch3LargeLimit = sortedWithLimit( + repoName, + sort, + batch2.v1(), + lastBatch + randomIntBetween(1, 100), + order + ); + assertEquals(batch3.v2(), batch3LargeLimit.v2()); + assertNull(batch3LargeLimit.v1()); + } + + public void testSortAndPaginateWithInProgress() throws Exception { + final String repoName = "test-repo"; + final Path repoPath = randomRepoPath(); + createRepository(repoName, "mock", repoPath); + maybeInitWithOldSnapshotVersion(repoName, repoPath); + final Collection allSnapshotNames = new HashSet<>(createNSnapshots(repoName, randomIntBetween(3, 20))); + createIndexWithContent("test-index-1"); + allSnapshotNames.addAll(createNSnapshots(repoName, randomIntBetween(3, 20))); + createIndexWithContent("test-index-2"); + + final int inProgressCount = randomIntBetween(6, 20); + final List> inProgressSnapshots = new ArrayList<>(inProgressCount); + blockAllDataNodes(repoName); + for (int i = 0; i < inProgressCount; i++) { + final String snapshotName = "snap-" + i; + allSnapshotNames.add(snapshotName); + inProgressSnapshots.add(startFullSnapshot(repoName, snapshotName)); + } + awaitNumberOfSnapshotsInProgress(inProgressCount); + + assertStablePagination(repoName, allSnapshotNames, GetSnapshotsRequest.SortBy.START_TIME); + assertStablePagination(repoName, allSnapshotNames, GetSnapshotsRequest.SortBy.NAME); + assertStablePagination(repoName, allSnapshotNames, GetSnapshotsRequest.SortBy.INDICES); + + unblockAllDataNodes(repoName); + for (ActionFuture inProgressSnapshot : inProgressSnapshots) { + assertSuccessful(inProgressSnapshot); + } + + assertStablePagination(repoName, allSnapshotNames, GetSnapshotsRequest.SortBy.START_TIME); + assertStablePagination(repoName, allSnapshotNames, GetSnapshotsRequest.SortBy.NAME); + assertStablePagination(repoName, allSnapshotNames, GetSnapshotsRequest.SortBy.INDICES); + } + + public void testPaginationRequiresVerboseListing() throws Exception { + final String repoName = "tst-repo"; + createRepository(repoName, "fs"); + createNSnapshots(repoName, randomIntBetween(1, 5)); + expectThrows( + ActionRequestValidationException.class, + () -> clusterAdmin().prepareGetSnapshots(repoName) + .setVerbose(false) + .setSort(GetSnapshotsRequest.SortBy.DURATION) + .setSize(GetSnapshotsRequest.NO_LIMIT) + .execute() + .actionGet() + ); + expectThrows( + ActionRequestValidationException.class, + () -> clusterAdmin().prepareGetSnapshots(repoName) + .setVerbose(false) + .setSort(GetSnapshotsRequest.SortBy.START_TIME) + .setSize(randomIntBetween(1, 100)) + .execute() + .actionGet() + ); + } + + private static void assertStablePagination(String repoName, Collection allSnapshotNames, GetSnapshotsRequest.SortBy sort) { + final SortOrder order = randomFrom(SortOrder.values()); + final List allSorted = allSnapshotsSorted(allSnapshotNames, repoName, sort, order); + + for (int i = 1; i <= allSnapshotNames.size(); i++) { + final Tuple> subsetSorted = sortedWithLimit(repoName, sort, null, i, order); + assertEquals(allSorted.subList(0, i), subsetSorted.v2()); + } + + for (int j = 0; j < allSnapshotNames.size(); j++) { + final SnapshotInfo after = allSorted.get(j); + for (int i = 1; i < allSnapshotNames.size() - j; i++) { + final List subsetSorted = sortedWithLimit( + repoName, + sort, + GetSnapshotsRequest.After.from(after, sort).asQueryParam(), + i, + order + ).v2(); + assertEquals(subsetSorted, allSorted.subList(j + 1, j + i + 1)); + } + } + } + + private static List allSnapshotsSorted( + Collection allSnapshotNames, + String repoName, + GetSnapshotsRequest.SortBy sortBy, + SortOrder order + ) { + final List snapshotInfos = sortedWithLimit(repoName, sortBy, null, GetSnapshotsRequest.NO_LIMIT, order).v2(); + assertEquals(snapshotInfos.size(), allSnapshotNames.size()); + for (SnapshotInfo snapshotInfo : snapshotInfos) { + assertThat(snapshotInfo.snapshotId().getName(), is(in(allSnapshotNames))); + } + return snapshotInfos; + } + + private static Tuple> sortedWithLimit( + String repoName, + GetSnapshotsRequest.SortBy sortBy, + String after, + int size, + SortOrder order + ) { + final GetSnapshotsResponse response = baseGetSnapshotsRequest(repoName).setAfter(after) + .setSort(sortBy) + .setSize(size) + .setOrder(order) + .get(); + return Tuple.tuple(response.next(), response.getSnapshots()); + } + + private static GetSnapshotsRequestBuilder baseGetSnapshotsRequest(String repoName) { + final GetSnapshotsRequestBuilder builder = clusterAdmin().prepareGetSnapshots(repoName); + // exclude old version snapshot from test assertions every time and do a prefixed query in either case half the time + if (randomBoolean() + || clusterAdmin().prepareGetSnapshots(repoName) + .setSnapshots(AbstractSnapshotIntegTestCase.OLD_VERSION_SNAPSHOT_PREFIX + "*") + .setIgnoreUnavailable(true) + .get() + .getSnapshots() + .isEmpty() == false) { + builder.setSnapshots(RANDOM_SNAPSHOT_NAME_PREFIX + "*"); + } + return builder; + } +} diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SystemIndicesSnapshotIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SystemIndicesSnapshotIT.java index 8d16dc824196a..fa73d9ef54b33 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SystemIndicesSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SystemIndicesSnapshotIT.java @@ -13,7 +13,6 @@ import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse; import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsResponse; import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; -import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.logging.Loggers; @@ -935,10 +934,6 @@ private long getDocCount(String indexName) { return client().admin().indices().prepareStats(indexName).get().getPrimaries().getDocs().getCount(); } - private IndexResponse indexDoc(String index, String id, Object... source) { - return client().prepareIndex(index, "_doc").setId(id).setSource(source).execute().actionGet(); - } - public static class SystemIndexTestPlugin extends Plugin implements SystemIndexPlugin { public static final String SYSTEM_INDEX_NAME = ".test-system-idx"; diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/TransportGetRepositoriesAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/TransportGetRepositoriesAction.java index c2dd55eaee700..3cb65c6b794a7 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/TransportGetRepositoriesAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/get/TransportGetRepositoriesAction.java @@ -66,14 +66,23 @@ protected void masterOperation( ClusterState state, final ActionListener listener ) { + listener.onResponse(new GetRepositoriesResponse(new RepositoriesMetadata(getRepositories(state, request.repositories())))); + } + + /** + * Get repository metadata for given repository names from given cluster state. + * + * @param state Cluster state + * @param repoNames Repository names or patterns to get metadata for + * @return list of repository metadata + */ + public static List getRepositories(ClusterState state, String[] repoNames) { RepositoriesMetadata repositories = state.metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); - if (request.repositories().length == 0 - || (request.repositories().length == 1 - && ("_all".equals(request.repositories()[0]) || "*".equals(request.repositories()[0])))) { - listener.onResponse(new GetRepositoriesResponse(repositories)); + if (repoNames.length == 0 || (repoNames.length == 1 && ("_all".equals(repoNames[0]) || "*".equals(repoNames[0])))) { + return repositories.repositories(); } else { Set repositoriesToGet = new LinkedHashSet<>(); // to keep insertion order - for (String repositoryOrPattern : request.repositories()) { + for (String repositoryOrPattern : repoNames) { if (Regex.isSimpleMatchPattern(repositoryOrPattern) == false) { repositoriesToGet.add(repositoryOrPattern); } else { @@ -88,12 +97,11 @@ protected void masterOperation( for (String repository : repositoriesToGet) { RepositoryMetadata repositoryMetadata = repositories.repository(repository); if (repositoryMetadata == null) { - listener.onFailure(new RepositoryMissingException(repository)); - return; + throw new RepositoryMissingException(repository); } repositoryListBuilder.add(repositoryMetadata); } - listener.onResponse(new GetRepositoriesResponse(new RepositoriesMetadata(repositoryListBuilder))); + return repositoryListBuilder; } } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequest.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequest.java index 2ff264ec577cb..33027d4456757 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequest.java @@ -8,16 +8,24 @@ package org.elasticsearch.action.admin.cluster.snapshots.get; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.support.master.MasterNodeRequest; import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.regex.Regex; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.search.sort.SortOrder; +import org.elasticsearch.snapshots.SnapshotInfo; import org.elasticsearch.tasks.CancellableTask; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Base64; import java.util.Map; import static org.elasticsearch.action.ValidateActions.addValidationError; @@ -31,7 +39,25 @@ public class GetSnapshotsRequest extends MasterNodeRequest public static final String CURRENT_SNAPSHOT = "_current"; public static final boolean DEFAULT_VERBOSE_MODE = true; - private String repository; + public static final Version MULTIPLE_REPOSITORIES_SUPPORT_ADDED = Version.V_7_14_0; + + public static final Version PAGINATED_GET_SNAPSHOTS_VERSION = Version.V_7_14_0; + + public static final int NO_LIMIT = -1; + + /** + * Number of snapshots to fetch information for or {@link #NO_LIMIT} for fetching all snapshots matching the request. + */ + private int size = NO_LIMIT; + + @Nullable + private After after; + + private SortBy sort = SortBy.START_TIME; + + private SortOrder order = SortOrder.ASC; + + private String[] repositories; private String[] snapshots = Strings.EMPTY_ARRAY; @@ -48,45 +74,122 @@ public GetSnapshotsRequest() {} * @param snapshots list of snapshots */ public GetSnapshotsRequest(String repository, String[] snapshots) { - this.repository = repository; + this.repositories = new String[] { repository }; this.snapshots = snapshots; } /** - * Constructs a new get snapshots request with given repository name + * Constructs a new get snapshots request with given repository names and list of snapshots * - * @param repository repository name + * @param repositories repository names + * @param snapshots list of snapshots */ - public GetSnapshotsRequest(String repository) { - this.repository = repository; + public GetSnapshotsRequest(String[] repositories, String[] snapshots) { + this.repositories = repositories; + this.snapshots = snapshots; + } + + /** + * Constructs a new get snapshots request with given repository names + * + * @param repositories repository names + */ + public GetSnapshotsRequest(String... repositories) { + this.repositories = repositories; } public GetSnapshotsRequest(StreamInput in) throws IOException { super(in); - repository = in.readString(); + if (in.getVersion().onOrAfter(MULTIPLE_REPOSITORIES_SUPPORT_ADDED)) { + repositories = in.readStringArray(); + } else { + repositories = new String[] { in.readString() }; + } snapshots = in.readStringArray(); ignoreUnavailable = in.readBoolean(); verbose = in.readBoolean(); + if (in.getVersion().onOrAfter(PAGINATED_GET_SNAPSHOTS_VERSION)) { + after = in.readOptionalWriteable(After::new); + sort = in.readEnum(SortBy.class); + size = in.readVInt(); + order = SortOrder.readFromStream(in); + } } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeString(repository); + if (out.getVersion().onOrAfter(MULTIPLE_REPOSITORIES_SUPPORT_ADDED)) { + out.writeStringArray(repositories); + } else { + if (repositories.length != 1) { + throw new IllegalArgumentException( + "Requesting snapshots from multiple repositories is not supported in versions prior " + + "to " + + MULTIPLE_REPOSITORIES_SUPPORT_ADDED.toString() + ); + } + out.writeString(repositories[0]); + } out.writeStringArray(snapshots); out.writeBoolean(ignoreUnavailable); out.writeBoolean(verbose); + if (out.getVersion().onOrAfter(PAGINATED_GET_SNAPSHOTS_VERSION)) { + out.writeOptionalWriteable(after); + out.writeEnum(sort); + out.writeVInt(size); + order.writeTo(out); + } else if (sort != SortBy.START_TIME || size != NO_LIMIT || after != null || order != SortOrder.ASC) { + throw new IllegalArgumentException("can't use paginated get snapshots request with node version [" + out.getVersion() + "]"); + } } @Override public ActionRequestValidationException validate() { ActionRequestValidationException validationException = null; - if (repository == null) { - validationException = addValidationError("repository is missing", validationException); + if (repositories == null || repositories.length == 0) { + validationException = addValidationError("repositories are missing", validationException); + } + if (size == 0 || size < NO_LIMIT) { + validationException = addValidationError("size must be -1 or greater than 0", validationException); + } + if (verbose == false) { + if (sort != SortBy.START_TIME) { + validationException = addValidationError("can't use non-default sort with verbose=false", validationException); + } + if (size > 0) { + validationException = addValidationError("can't use size limit with verbose=false", validationException); + } + if (after != null) { + validationException = addValidationError("can't use after with verbose=false", validationException); + } + if (order != SortOrder.ASC) { + validationException = addValidationError("can't use non-default sort order with verbose=false", validationException); + } } return validationException; } + /** + * Sets repository names + * + * @param repositories repository names + * @return this request + */ + public GetSnapshotsRequest repositories(String... repositories) { + this.repositories = repositories; + return this; + } + + /** + * Returns repository names + * + * @return repository names + */ + public String[] repositories() { + return this.repositories; + } + /** * Sets repository name * @@ -94,8 +197,7 @@ public ActionRequestValidationException validate() { * @return this request */ public GetSnapshotsRequest repository(String repository) { - this.repository = repository; - return this; + return repositories(repository); } /** @@ -104,7 +206,20 @@ public GetSnapshotsRequest repository(String repository) { * @return repository name */ public String repository() { - return this.repository; + if (repositories == null || repositories.length == 0) { + return null; + } + if (repositories.length != 1) { + throw new IllegalStateException("more than a single repository set for request"); + } + return this.repositories[0]; + } + + public boolean isSingleRepositoryRequest() { + return repositories.length == 1 + && repositories[0] != null + && "_all".equals(repositories[0]) == false + && Regex.isSimpleMatchPattern(repositories[0]) == false; } /** @@ -157,6 +272,42 @@ public GetSnapshotsRequest verbose(boolean verbose) { return this; } + public After after() { + return after; + } + + public SortBy sort() { + return sort; + } + + public GetSnapshotsRequest after(@Nullable After after) { + this.after = after; + return this; + } + + public GetSnapshotsRequest sort(SortBy sort) { + this.sort = sort; + return this; + } + + public GetSnapshotsRequest size(int size) { + this.size = size; + return this; + } + + public int size() { + return size; + } + + public SortOrder order() { + return order; + } + + public GetSnapshotsRequest order(SortOrder order) { + this.order = order; + return this; + } + /** * Returns whether the request will return a verbose response. */ @@ -168,4 +319,112 @@ public boolean verbose() { public Task createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { return new CancellableTask(id, type, action, getDescription(), parentTaskId, headers); } + + public enum SortBy { + START_TIME("start_time"), + NAME("name"), + DURATION("duration"), + INDICES("index_count"); + + private final String param; + + SortBy(String param) { + this.param = param; + } + + @Override + public String toString() { + return param; + } + + public static SortBy of(String value) { + switch (value) { + case "start_time": + return START_TIME; + case "name": + return NAME; + case "duration": + return DURATION; + case "index_count": + return INDICES; + default: + throw new IllegalArgumentException("unknown sort order [" + value + "]"); + } + } + } + + public static final class After implements Writeable { + + private final String value; + + private final String repoName; + + private final String snapshotName; + + After(StreamInput in) throws IOException { + this(in.readString(), in.readString(), in.readString()); + } + + public static After fromQueryParam(String param) { + final String[] parts = new String(Base64.getUrlDecoder().decode(param), StandardCharsets.UTF_8).split(","); + if (parts.length != 3) { + throw new IllegalArgumentException("invalid ?after parameter [" + param + "]"); + } + return new After(parts[0], parts[1], parts[2]); + } + + @Nullable + public static After from(@Nullable SnapshotInfo snapshotInfo, SortBy sortBy) { + if (snapshotInfo == null) { + return null; + } + final String afterValue; + switch (sortBy) { + case START_TIME: + afterValue = String.valueOf(snapshotInfo.startTime()); + break; + case NAME: + afterValue = snapshotInfo.snapshotId().getName(); + break; + case DURATION: + afterValue = String.valueOf(snapshotInfo.endTime() - snapshotInfo.startTime()); + break; + case INDICES: + afterValue = String.valueOf(snapshotInfo.indices().size()); + break; + default: + throw new AssertionError("unknown sort column [" + sortBy + "]"); + } + return new After(afterValue, snapshotInfo.repository(), snapshotInfo.snapshotId().getName()); + } + + public After(String value, String repoName, String snapshotName) { + this.value = value; + this.repoName = repoName; + this.snapshotName = snapshotName; + } + + public String value() { + return value; + } + + public String snapshotName() { + return snapshotName; + } + + public String repoName() { + return repoName; + } + + public String asQueryParam() { + return Base64.getUrlEncoder().encodeToString((value + "," + repoName + "," + snapshotName).getBytes(StandardCharsets.UTF_8)); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(value); + out.writeString(repoName); + out.writeString(snapshotName); + } + } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequestBuilder.java index ea16940ff69eb..ebe0df0227bf5 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequestBuilder.java @@ -11,6 +11,8 @@ import org.elasticsearch.action.support.master.MasterNodeOperationRequestBuilder; import org.elasticsearch.client.ElasticsearchClient; import org.elasticsearch.common.util.ArrayUtils; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.search.sort.SortOrder; /** * Get snapshots request builder @@ -34,6 +36,13 @@ public GetSnapshotsRequestBuilder(ElasticsearchClient client, GetSnapshotsAction super(client, action, new GetSnapshotsRequest(repository)); } + /** + * Constructs the new get snapshot request with specified repositories + */ + public GetSnapshotsRequestBuilder(ElasticsearchClient client, GetSnapshotsAction action, String... repositories) { + super(client, action, new GetSnapshotsRequest(repositories)); + } + /** * Sets the repository name * @@ -45,6 +54,17 @@ public GetSnapshotsRequestBuilder setRepository(String repository) { return this; } + /** + * Sets the repository names + * + * @param repositories repository names + * @return this builder + */ + public GetSnapshotsRequestBuilder setRepositories(String... repositories) { + request.repositories(repositories); + return this; + } + /** * Sets list of snapshots to return * @@ -102,4 +122,28 @@ public GetSnapshotsRequestBuilder setVerbose(boolean verbose) { return this; } + public GetSnapshotsRequestBuilder setAfter(String after) { + return setAfter(after == null ? null : GetSnapshotsRequest.After.fromQueryParam(after)); + } + + public GetSnapshotsRequestBuilder setAfter(@Nullable GetSnapshotsRequest.After after) { + request.after(after); + return this; + } + + public GetSnapshotsRequestBuilder setSort(GetSnapshotsRequest.SortBy sort) { + request.sort(sort); + return this; + } + + public GetSnapshotsRequestBuilder setSize(int size) { + request.size(size); + return this; + } + + public GetSnapshotsRequestBuilder setOrder(SortOrder order) { + request.order(order); + return this; + } + } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java index ac8c8c95d293f..3f16f1469b653 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java @@ -8,6 +8,7 @@ package org.elasticsearch.action.admin.cluster.snapshots.get; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; @@ -18,11 +19,14 @@ import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.core.Nullable; import org.elasticsearch.snapshots.SnapshotInfo; import java.io.IOException; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Objects; /** @@ -34,7 +38,7 @@ public class GetSnapshotsResponse extends ActionResponse implements ToXContentOb private static final ConstructingObjectParser GET_SNAPSHOT_PARSER = new ConstructingObjectParser<>( GetSnapshotsResponse.class.getName(), true, - (args) -> new GetSnapshotsResponse((List) args[0]) + (args) -> new GetSnapshotsResponse((List) args[0], (Map) args[1], (String) args[2]) ); static { @@ -43,17 +47,37 @@ public class GetSnapshotsResponse extends ActionResponse implements ToXContentOb (p, c) -> SnapshotInfo.SNAPSHOT_INFO_PARSER.apply(p, c).build(), new ParseField("snapshots") ); + GET_SNAPSHOT_PARSER.declareObject( + ConstructingObjectParser.optionalConstructorArg(), + (p, c) -> p.map(HashMap::new, ElasticsearchException::fromXContent), + new ParseField("failures") + ); + GET_SNAPSHOT_PARSER.declareStringOrNull(ConstructingObjectParser.optionalConstructorArg(), new ParseField("next")); } private final List snapshots; - public GetSnapshotsResponse(List snapshots) { - this.snapshots = Collections.unmodifiableList(snapshots); + private final Map failures; + + @Nullable + private final String next; + + public GetSnapshotsResponse(List snapshots, Map failures, @Nullable String next) { + this.snapshots = org.elasticsearch.core.List.copyOf(snapshots); + this.failures = failures == null ? org.elasticsearch.core.Map.of() : org.elasticsearch.core.Map.copyOf(failures); + this.next = next; } - GetSnapshotsResponse(StreamInput in) throws IOException { - super(in); - snapshots = Collections.unmodifiableList(in.readList(SnapshotInfo::readFrom)); + public GetSnapshotsResponse(StreamInput in) throws IOException { + this.snapshots = in.readList(SnapshotInfo::readFrom); + if (in.getVersion().onOrAfter(GetSnapshotsRequest.MULTIPLE_REPOSITORIES_SUPPORT_ADDED)) { + final Map failedResponses = in.readMap(StreamInput::readString, StreamInput::readException); + this.failures = Collections.unmodifiableMap(failedResponses); + this.next = in.readOptionalString(); + } else { + this.failures = Collections.emptyMap(); + this.next = null; + } } /** @@ -65,9 +89,37 @@ public List getSnapshots() { return snapshots; } + /** + * Returns a map of repository name to {@link ElasticsearchException} for each unsuccessful response. + */ + public Map getFailures() { + return failures; + } + + @Nullable + public String next() { + return next; + } + + /** + * Returns true if there is a least one failed response. + */ + public boolean isFailed() { + return failures.isEmpty() == false; + } + @Override public void writeTo(StreamOutput out) throws IOException { out.writeList(snapshots); + if (out.getVersion().onOrAfter(GetSnapshotsRequest.MULTIPLE_REPOSITORIES_SUPPORT_ADDED)) { + out.writeMap(failures, StreamOutput::writeString, StreamOutput::writeException); + out.writeOptionalString(next); + } else { + if (failures.isEmpty() == false) { + assert false : "transport action should have thrown directly for old version but saw " + failures; + throw failures.values().iterator().next(); + } + } } @Override @@ -78,6 +130,21 @@ public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params par snapshotInfo.toXContent(builder, params); } builder.endArray(); + if (failures.isEmpty() == false) { + builder.startObject("failures"); + for (Map.Entry error : failures.entrySet()) { + builder.field(error.getKey(), (b, pa) -> { + b.startObject(); + error.getValue().toXContent(b, pa); + b.endObject(); + return b; + }); + } + builder.endObject(); + } + if (next != null) { + builder.field("next", next); + } builder.endObject(); return builder; } @@ -91,12 +158,12 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; GetSnapshotsResponse that = (GetSnapshotsResponse) o; - return Objects.equals(snapshots, that.snapshots); + return Objects.equals(snapshots, that.snapshots) && Objects.equals(failures, that.failures) && Objects.equals(next, that.next); } @Override public int hashCode() { - return Objects.hash(snapshots); + return Objects.hash(snapshots, failures, next); } @Override diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java index a3e943f40b885..057f6201b5198 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/get/TransportGetSnapshotsAction.java @@ -10,27 +10,33 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.util.CollectionUtil; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.StepListener; +import org.elasticsearch.action.admin.cluster.repositories.get.TransportGetRepositoriesAction; import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.GroupedActionListener; import org.elasticsearch.action.support.master.TransportMasterNodeAction; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.SnapshotsInProgress; import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.metadata.RepositoryMetadata; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.regex.Regex; import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.Tuple; +import org.elasticsearch.repositories.GetSnapshotInfoContext; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.RepositoryData; -import org.elasticsearch.snapshots.SnapshotException; +import org.elasticsearch.repositories.RepositoryMissingException; +import org.elasticsearch.search.sort.SortOrder; +import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.snapshots.SnapshotInfo; import org.elasticsearch.snapshots.SnapshotMissingException; @@ -42,15 +48,19 @@ import org.elasticsearch.transport.TransportService; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; +import java.util.function.Predicate; +import java.util.function.ToLongFunction; import java.util.stream.Collectors; - -import static java.util.Collections.unmodifiableList; +import java.util.stream.Stream; /** * Transport Action for get snapshots operation @@ -102,16 +112,108 @@ protected void masterOperation( final ClusterState state, final ActionListener listener ) { - final String repo = request.repository(); - final String[] snapshots = request.snapshots(); - final SnapshotsInProgress snapshotsInProgress = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); assert task instanceof CancellableTask : task + " not cancellable"; - final Map allSnapshotIds = new HashMap<>(); + getMultipleReposSnapshotInfo( + request.isSingleRepositoryRequest() == false, + state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY), + TransportGetRepositoriesAction.getRepositories(state, request.repositories()), + request.snapshots(), + request.ignoreUnavailable(), + request.verbose(), + (CancellableTask) task, + request.sort(), + request.after(), + request.size(), + request.order(), + listener + ); + } + + private void getMultipleReposSnapshotInfo( + boolean isMultiRepoRequest, + SnapshotsInProgress snapshotsInProgress, + List repos, + String[] snapshots, + boolean ignoreUnavailable, + boolean verbose, + CancellableTask cancellableTask, + GetSnapshotsRequest.SortBy sortBy, + @Nullable GetSnapshotsRequest.After after, + int size, + SortOrder order, + ActionListener listener + ) { + // short-circuit if there are no repos, because we can not create GroupedActionListener of size 0 + if (repos.isEmpty()) { + listener.onResponse(new GetSnapshotsResponse(Collections.emptyList(), Collections.emptyMap(), null)); + return; + } + final GroupedActionListener, SnapshotsInRepo>> groupedActionListener = + new GroupedActionListener<>(listener.map(responses -> { + assert repos.size() == responses.size(); + final List allSnapshots = responses.stream() + .map(Tuple::v2) + .filter(Objects::nonNull) + .flatMap(snapshotsInRepo -> snapshotsInRepo.snapshotInfos.stream()) + .collect(Collectors.toList()); + final Map failures = responses.stream() + .map(Tuple::v1) + .filter(Objects::nonNull) + .collect(Collectors.toMap(Tuple::v1, Tuple::v2)); + final SnapshotsInRepo snInfos = sortSnapshots(allSnapshots, sortBy, after, size, order); + final List snapshotInfos = snInfos.snapshotInfos; + return new GetSnapshotsResponse( + snapshotInfos, + failures, + snInfos.hasMore || responses.stream().anyMatch(r -> r.v2() != null && r.v2().hasMore) + ? GetSnapshotsRequest.After.from(snapshotInfos.get(snapshotInfos.size() - 1), sortBy).asQueryParam() + : null + ); + }), repos.size()); + + for (final RepositoryMetadata repo : repos) { + final String repoName = repo.name(); + getSingleRepoSnapshotInfo( + snapshotsInProgress, + repoName, + snapshots, + ignoreUnavailable, + verbose, + cancellableTask, + sortBy, + after, + size, + order, + groupedActionListener.delegateResponse((groupedListener, e) -> { + if (isMultiRepoRequest && e instanceof ElasticsearchException) { + groupedListener.onResponse(Tuple.tuple(Tuple.tuple(repoName, (ElasticsearchException) e), null)); + } else { + groupedListener.onFailure(e); + } + }).map(snInfos -> Tuple.tuple(null, snInfos)) + ); + } + } + + private void getSingleRepoSnapshotInfo( + SnapshotsInProgress snapshotsInProgress, + String repo, + String[] snapshots, + boolean ignoreUnavailable, + boolean verbose, + CancellableTask task, + GetSnapshotsRequest.SortBy sortBy, + @Nullable final GetSnapshotsRequest.After after, + int size, + SortOrder order, + ActionListener listener + ) { + final Map allSnapshotIds = new HashMap<>(); final List currentSnapshots = new ArrayList<>(); - for (SnapshotInfo snapshotInfo : sortedCurrentSnapshots(snapshotsInProgress, repo)) { - SnapshotId snapshotId = snapshotInfo.snapshotId(); - allSnapshotIds.put(snapshotId.getName(), snapshotId); + for (SnapshotInfo snapshotInfo : sortedCurrentSnapshots(snapshotsInProgress, repo, sortBy, after, size, order).snapshotInfos) { + Snapshot snapshot = snapshotInfo.snapshot(); + allSnapshotIds.put(snapshot.getSnapshotId().getName(), snapshot); currentSnapshots.add(snapshotInfo); } @@ -127,13 +229,17 @@ protected void masterOperation( snapshotsInProgress, repo, snapshots, - request.ignoreUnavailable(), - request.verbose(), + ignoreUnavailable, + verbose, allSnapshotIds, currentSnapshots, repositoryData, - (CancellableTask) task, - listener.map(GetSnapshotsResponse::new) + task, + sortBy, + after, + size, + order, + listener ), listener::onFailure ); @@ -146,7 +252,14 @@ protected void masterOperation( * @param repositoryName repository name * @return list of snapshots */ - private static List sortedCurrentSnapshots(SnapshotsInProgress snapshotsInProgress, String repositoryName) { + private static SnapshotsInRepo sortedCurrentSnapshots( + SnapshotsInProgress snapshotsInProgress, + String repositoryName, + GetSnapshotsRequest.SortBy sortBy, + @Nullable final GetSnapshotsRequest.After after, + int size, + SortOrder order + ) { List snapshotList = new ArrayList<>(); List entries = SnapshotsService.currentSnapshots( snapshotsInProgress, @@ -156,8 +269,7 @@ private static List sortedCurrentSnapshots(SnapshotsInProgress sna for (SnapshotsInProgress.Entry entry : entries) { snapshotList.add(new SnapshotInfo(entry)); } - CollectionUtil.timSort(snapshotList); - return unmodifiableList(snapshotList); + return sortSnapshots(snapshotList, sortBy, after, size, order); } private void loadSnapshotInfos( @@ -166,11 +278,15 @@ private void loadSnapshotInfos( String[] snapshots, boolean ignoreUnavailable, boolean verbose, - Map allSnapshotIds, + Map allSnapshotIds, List currentSnapshots, @Nullable RepositoryData repositoryData, CancellableTask task, - ActionListener> listener + GetSnapshotsRequest.SortBy sortBy, + @Nullable final GetSnapshotsRequest.After after, + int size, + SortOrder order, + ActionListener listener ) { if (task.isCancelled()) { listener.onFailure(new TaskCancelledException("task cancelled")); @@ -179,17 +295,17 @@ private void loadSnapshotInfos( if (repositoryData != null) { for (SnapshotId snapshotId : repositoryData.getSnapshotIds()) { - allSnapshotIds.put(snapshotId.getName(), snapshotId); + allSnapshotIds.put(snapshotId.getName(), new Snapshot(repo, snapshotId)); } } - final Set toResolve = new HashSet<>(); + final Set toResolve = new HashSet<>(); if (isAllSnapshots(snapshots)) { toResolve.addAll(allSnapshotIds.values()); } else { for (String snapshotOrPattern : snapshots) { if (GetSnapshotsRequest.CURRENT_SNAPSHOT.equalsIgnoreCase(snapshotOrPattern)) { - toResolve.addAll(currentSnapshots.stream().map(SnapshotInfo::snapshotId).collect(Collectors.toList())); + toResolve.addAll(currentSnapshots.stream().map(SnapshotInfo::snapshot).collect(Collectors.toList())); } else if (Regex.isSimpleMatchPattern(snapshotOrPattern) == false) { if (allSnapshotIds.containsKey(snapshotOrPattern)) { toResolve.add(allSnapshotIds.get(snapshotOrPattern)); @@ -197,7 +313,7 @@ private void loadSnapshotInfos( throw new SnapshotMissingException(repo, snapshotOrPattern); } } else { - for (Map.Entry entry : allSnapshotIds.entrySet()) { + for (Map.Entry entry : allSnapshotIds.entrySet()) { if (Regex.simpleMatch(snapshotOrPattern, entry.getKey())) { toResolve.add(entry.getValue()); } @@ -211,22 +327,32 @@ private void loadSnapshotInfos( } if (verbose) { - threadPool.generic() - .execute( - ActionRunnable.supply( - listener, - () -> snapshots(snapshotsInProgress, repo, new ArrayList<>(toResolve), ignoreUnavailable, task) - ) - ); + snapshots( + snapshotsInProgress, + repo, + Collections.unmodifiableList(toResolve.stream().map(Snapshot::getSnapshotId).collect(Collectors.toList())), + ignoreUnavailable, + task, + sortBy, + after, + size, + order, + listener + ); } else { - final List snapshotInfos; + final SnapshotsInRepo snapshotInfos; if (repositoryData != null) { // want non-current snapshots as well, which are found in the repository data - snapshotInfos = buildSimpleSnapshotInfos(toResolve, repositoryData, currentSnapshots); + snapshotInfos = buildSimpleSnapshotInfos(toResolve, repo, repositoryData, currentSnapshots, sortBy, after, size, order); } else { // only want current snapshots - snapshotInfos = currentSnapshots.stream().map(SnapshotInfo::basic).collect(Collectors.toList()); - CollectionUtil.timSort(snapshotInfos); + snapshotInfos = sortSnapshots( + currentSnapshots.stream().map(SnapshotInfo::basic).collect(Collectors.toList()), + sortBy, + after, + size, + order + ); } listener.onResponse(snapshotInfos); } @@ -234,23 +360,26 @@ private void loadSnapshotInfos( /** * Returns a list of snapshots from repository sorted by snapshot creation date - * - * @param snapshotsInProgress snapshots in progress in the cluster state + * @param snapshotsInProgress snapshots in progress in the cluster state * @param repositoryName repository name * @param snapshotIds snapshots for which to fetch snapshot information * @param ignoreUnavailable if true, snapshots that could not be read will only be logged with a warning, - * if false, they will throw an error - * @return list of snapshots */ - private List snapshots( + private void snapshots( SnapshotsInProgress snapshotsInProgress, String repositoryName, - List snapshotIds, + Collection snapshotIds, boolean ignoreUnavailable, - CancellableTask task + CancellableTask task, + GetSnapshotsRequest.SortBy sortBy, + @Nullable GetSnapshotsRequest.After after, + int size, + SortOrder order, + ActionListener listener ) { if (task.isCancelled()) { - throw new TaskCancelledException("task cancelled"); + listener.onFailure(new TaskCancelledException("task cancelled")); + return; } final Set snapshotSet = new HashSet<>(); final Set snapshotIdsToIterate = new HashSet<>(snapshotIds); @@ -265,28 +394,49 @@ private List snapshots( snapshotSet.add(new SnapshotInfo(entry)); } } - // then, look in the repository - final Repository repository = repositoriesService.repository(repositoryName); - for (SnapshotId snapshotId : snapshotIdsToIterate) { - if (task.isCancelled()) { - throw new TaskCancelledException("task cancelled"); - } - try { - snapshotSet.add(repository.getSnapshotInfo(snapshotId)); - } catch (Exception ex) { - if (ignoreUnavailable) { - logger.warn(() -> new ParameterizedMessage("failed to get snapshot [{}]", snapshotId), ex); - } else { - if (ex instanceof SnapshotException) { - throw ex; - } - throw new SnapshotException(repositoryName, snapshotId, "Snapshot could not be read", ex); - } - } + // then, look in the repository if there's any matching snapshots left + final List snapshotInfos; + if (snapshotIdsToIterate.isEmpty()) { + snapshotInfos = Collections.emptyList(); + } else { + snapshotInfos = Collections.synchronizedList(new ArrayList<>()); + } + final ActionListener allDoneListener = listener.delegateFailure((l, v) -> { + final ArrayList snapshotList = new ArrayList<>(snapshotInfos); + snapshotList.addAll(snapshotSet); + listener.onResponse(sortSnapshots(snapshotList, sortBy, after, size, order)); + }); + if (snapshotIdsToIterate.isEmpty()) { + allDoneListener.onResponse(null); + return; + } + final Repository repository; + try { + repository = repositoriesService.repository(repositoryName); + } catch (RepositoryMissingException e) { + listener.onFailure(e); + return; } - final ArrayList snapshotList = new ArrayList<>(snapshotSet); - CollectionUtil.timSort(snapshotList); - return unmodifiableList(snapshotList); + repository.getSnapshotInfo( + new GetSnapshotInfoContext( + snapshotIdsToIterate, + ignoreUnavailable == false, + task::isCancelled, + (context, snapshotInfo) -> snapshotInfos.add(snapshotInfo), + ignoreUnavailable ? ActionListener.runAfter(new ActionListener() { + @Override + public void onResponse(Void unused) { + logger.trace("done fetching snapshot infos [{}]", snapshotIdsToIterate); + } + + @Override + public void onFailure(Exception e) { + assert false : new AssertionError("listener should always complete successfully for ignoreUnavailable=true", e); + logger.warn("failed to fetch snapshot info for some snapshots", e); + } + }, () -> allDoneListener.onResponse(null)) : allDoneListener + ) + ); } private boolean isAllSnapshots(String[] snapshots) { @@ -297,39 +447,164 @@ private boolean isCurrentSnapshotsOnly(String[] snapshots) { return (snapshots.length == 1 && GetSnapshotsRequest.CURRENT_SNAPSHOT.equalsIgnoreCase(snapshots[0])); } - private static List buildSimpleSnapshotInfos( - final Set toResolve, + private static SnapshotsInRepo buildSimpleSnapshotInfos( + final Set toResolve, + final String repoName, final RepositoryData repositoryData, - final List currentSnapshots + final List currentSnapshots, + final GetSnapshotsRequest.SortBy sortBy, + @Nullable final GetSnapshotsRequest.After after, + final int size, + final SortOrder order ) { List snapshotInfos = new ArrayList<>(); for (SnapshotInfo snapshotInfo : currentSnapshots) { - if (toResolve.remove(snapshotInfo.snapshotId())) { + if (toResolve.remove(snapshotInfo.snapshot())) { snapshotInfos.add(snapshotInfo.basic()); } } Map> snapshotsToIndices = new HashMap<>(); for (IndexId indexId : repositoryData.getIndices().values()) { for (SnapshotId snapshotId : repositoryData.getSnapshots(indexId)) { - if (toResolve.contains(snapshotId)) { + if (toResolve.contains(new Snapshot(repoName, snapshotId))) { snapshotsToIndices.computeIfAbsent(snapshotId, (k) -> new ArrayList<>()).add(indexId.getName()); } } } - for (SnapshotId snapshotId : toResolve) { - final List indices = snapshotsToIndices.getOrDefault(snapshotId, Collections.emptyList()); + for (Snapshot snapshot : toResolve) { + final List indices = snapshotsToIndices.getOrDefault(snapshot.getSnapshotId(), Collections.emptyList()); CollectionUtil.timSort(indices); snapshotInfos.add( new SnapshotInfo( - snapshotId, + snapshot, indices, Collections.emptyList(), Collections.emptyList(), - repositoryData.getSnapshotState(snapshotId) + repositoryData.getSnapshotState(snapshot.getSnapshotId()) ) ); } - CollectionUtil.timSort(snapshotInfos); - return Collections.unmodifiableList(snapshotInfos); + return sortSnapshots(snapshotInfos, sortBy, after, size, order); + } + + private static final Comparator BY_START_TIME = Comparator.comparingLong(SnapshotInfo::startTime) + .thenComparing(SnapshotInfo::snapshotId); + + private static final Comparator BY_DURATION = Comparator.comparingLong( + sni -> sni.endTime() - sni.startTime() + ).thenComparing(SnapshotInfo::snapshotId); + + private static final Comparator BY_INDICES_COUNT = Comparator.comparingInt(sni -> sni.indices().size()) + .thenComparing(SnapshotInfo::snapshotId); + + private static final Comparator BY_NAME = Comparator.comparing(sni -> sni.snapshotId().getName()); + + private static SnapshotsInRepo sortSnapshots( + List snapshotInfos, + GetSnapshotsRequest.SortBy sortBy, + @Nullable GetSnapshotsRequest.After after, + int size, + SortOrder order + ) { + final Comparator comparator; + switch (sortBy) { + case START_TIME: + comparator = BY_START_TIME; + break; + case NAME: + comparator = BY_NAME; + break; + case DURATION: + comparator = BY_DURATION; + break; + case INDICES: + comparator = BY_INDICES_COUNT; + break; + default: + throw new AssertionError("unexpected sort column [" + sortBy + "]"); + } + + Stream infos = snapshotInfos.stream(); + + if (after != null) { + final Predicate isAfter; + final String snapshotName = after.snapshotName(); + final String repoName = after.repoName(); + switch (sortBy) { + case START_TIME: + isAfter = filterByLongOffset(SnapshotInfo::startTime, Long.parseLong(after.value()), snapshotName, repoName, order); + break; + case NAME: + isAfter = order == SortOrder.ASC + ? (info -> compareName(snapshotName, repoName, info) < 0) + : (info -> compareName(snapshotName, repoName, info) > 0); + break; + case DURATION: + isAfter = filterByLongOffset( + info -> info.endTime() - info.startTime(), + Long.parseLong(after.value()), + snapshotName, + repoName, + order + ); + break; + case INDICES: + isAfter = filterByLongOffset( + info -> info.indices().size(), + Integer.parseInt(after.value()), + snapshotName, + repoName, + order + ); + break; + default: + throw new AssertionError("unexpected sort column [" + sortBy + "]"); + } + infos = infos.filter(isAfter); + } + infos = infos.sorted(order == SortOrder.DESC ? comparator.reversed() : comparator); + if (size != GetSnapshotsRequest.NO_LIMIT) { + infos = infos.limit(size + 1); + } + final List snapshots = Collections.unmodifiableList(infos.collect(Collectors.toList())); + boolean hasMore = size != GetSnapshotsRequest.NO_LIMIT && size < snapshots.size(); + return new SnapshotsInRepo(hasMore ? snapshots.subList(0, size) : snapshots, hasMore); + } + + private static Predicate filterByLongOffset( + ToLongFunction extractor, + long after, + String snapshotName, + String repoName, + SortOrder order + ) { + return order == SortOrder.ASC ? info -> { + final long val = extractor.applyAsLong(info); + + return after < val || (after == val && compareName(snapshotName, repoName, info) < 0); + } : info -> { + final long val = extractor.applyAsLong(info); + return after > val || (after == val && compareName(snapshotName, repoName, info) > 0); + }; + } + + private static int compareName(String name, String repoName, SnapshotInfo info) { + final int res = name.compareTo(info.snapshotId().getName()); + if (res != 0) { + return res; + } + return repoName.compareTo(info.repository()); + } + + private static final class SnapshotsInRepo { + + private final boolean hasMore; + + private final List snapshotInfos; + + SnapshotsInRepo(List snapshotInfos, boolean hasMore) { + this.hasMore = hasMore; + this.snapshotInfos = snapshotInfos; + } } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java index ac77539b02011..7e5b71b4668bd 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java @@ -14,7 +14,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionRunnable; +import org.elasticsearch.action.StepListener; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.TransportMasterNodeAction; import org.elasticsearch.cluster.ClusterState; @@ -27,10 +27,10 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.util.CollectionUtils; -import org.elasticsearch.common.util.concurrent.ListenableFuture; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; +import org.elasticsearch.repositories.GetSnapshotInfoContext; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.repositories.Repository; @@ -52,6 +52,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -62,6 +63,7 @@ import java.util.stream.Collectors; import static java.util.Collections.unmodifiableMap; +import static org.elasticsearch.cluster.SnapshotsInProgress.ShardState.SUCCESS; public class TransportSnapshotsStatusAction extends TransportMasterNodeAction { @@ -147,13 +149,14 @@ protected void masterOperation( new TransportNodesSnapshotsStatus.Request(nodesIds.toArray(Strings.EMPTY_ARRAY)).snapshots(snapshots) .timeout(request.masterNodeTimeout()), ActionListener.wrap( - nodeSnapshotStatuses -> threadPool.generic() - .execute( - ActionRunnable.wrap( - listener, - l -> buildResponse(snapshotsInProgress, request, currentSnapshots, nodeSnapshotStatuses, cancellableTask, l) - ) - ), + nodeSnapshotStatuses -> buildResponse( + snapshotsInProgress, + request, + currentSnapshots, + nodeSnapshotStatuses, + cancellableTask, + listener + ), listener::onFailure ) ); @@ -197,8 +200,7 @@ private void buildResponse( SnapshotIndexShardStatus shardStatus = shardStatues.get(shardEntry.key); if (shardStatus != null) { // We have full information about this shard - if (shardStatus.getStage() == SnapshotIndexShardStage.DONE - && shardEntry.value.state() != SnapshotsInProgress.ShardState.SUCCESS) { + if (shardStatus.getStage() == SnapshotIndexShardStage.DONE && shardEntry.value.state() != SUCCESS) { // Unlikely edge case: // Data node has finished snapshotting the shard but the cluster state has not yet been updated // to reflect this. We adjust the status to show up as snapshot metadata being written because @@ -291,9 +293,10 @@ private void loadRepositoryData( ActionListener listener ) { final Set requestedSnapshotNames = Sets.newHashSet(request.snapshots()); - final ListenableFuture repositoryDataListener = new ListenableFuture<>(); + final StepListener repositoryDataListener = new StepListener<>(); repositoriesService.getRepositoryData(repositoryName, repositoryDataListener); - repositoryDataListener.addListener(ActionListener.wrap(repositoryData -> { + final Collection snapshotIdsToLoad = new ArrayList<>(); + repositoryDataListener.whenComplete(repositoryData -> { ensureNotCancelled(task); final Map matchedSnapshotIds = repositoryData.getSnapshotIds() .stream() @@ -319,73 +322,62 @@ private void loadRepositoryData( throw new SnapshotMissingException(repositoryName, snapshotName); } } - SnapshotInfo snapshotInfo = snapshot(snapshotsInProgress, repositoryName, snapshotId); - List shardStatusBuilder = new ArrayList<>(); - if (snapshotInfo.state().completed()) { - Map shardStatuses = snapshotShards( - repositoryName, - repositoryData, - task, - snapshotInfo - ); - for (Map.Entry shardStatus : shardStatuses.entrySet()) { - IndexShardSnapshotStatus.Copy lastSnapshotStatus = shardStatus.getValue().asCopy(); - shardStatusBuilder.add(new SnapshotIndexShardStatus(shardStatus.getKey(), lastSnapshotStatus)); - } - final SnapshotsInProgress.State state; - switch (snapshotInfo.state()) { - case FAILED: - state = SnapshotsInProgress.State.FAILED; - break; - case SUCCESS: - case PARTIAL: - // Translating both PARTIAL and SUCCESS to SUCCESS for now - // TODO: add the differentiation on the metadata level in the next major release - state = SnapshotsInProgress.State.SUCCESS; - break; - default: - throw new IllegalArgumentException("Unknown snapshot state " + snapshotInfo.state()); - } - final long startTime = snapshotInfo.startTime(); - final long endTime = snapshotInfo.endTime(); - assert endTime >= startTime || (endTime == 0L && snapshotInfo.state().completed() == false) - : "Inconsistent timestamps found in SnapshotInfo [" + snapshotInfo + "]"; - builder.add( - new SnapshotStatus( - new Snapshot(repositoryName, snapshotId), - state, - Collections.unmodifiableList(shardStatusBuilder), - snapshotInfo.includeGlobalState(), - startTime, - // Use current time to calculate overall runtime for in-progress snapshots that have endTime == 0 - (endTime == 0 ? threadPool.absoluteTimeInMillis() : endTime) - startTime - ) - ); + if (snapshotsInProgress.snapshot(new Snapshot(repositoryName, snapshotId)) == null) { + snapshotIdsToLoad.add(snapshotId); } } - listener.onResponse(new SnapshotsStatusResponse(Collections.unmodifiableList(builder))); - }, listener::onFailure), threadPool.generic(), null); - } - /** - * Retrieves snapshot from repository - * - * @param snapshotsInProgress snapshots in progress in the cluster state - * @param repositoryName repository name - * @param snapshotId snapshot id - * @return snapshot - * @throws SnapshotMissingException if snapshot is not found - */ - private SnapshotInfo snapshot(SnapshotsInProgress snapshotsInProgress, String repositoryName, SnapshotId snapshotId) { - List entries = SnapshotsService.currentSnapshots( - snapshotsInProgress, - repositoryName, - Collections.singletonList(snapshotId.getName()) - ); - if (entries.isEmpty() == false) { - return new SnapshotInfo(entries.iterator().next()); - } - return repositoriesService.repository(repositoryName).getSnapshotInfo(snapshotId); + if (snapshotIdsToLoad.isEmpty()) { + listener.onResponse(new SnapshotsStatusResponse(Collections.unmodifiableList(builder))); + } else { + final List threadSafeBuilder = Collections.synchronizedList(builder); + repositoriesService.repository(repositoryName) + .getSnapshotInfo(new GetSnapshotInfoContext(snapshotIdsToLoad, true, task::isCancelled, (context, snapshotInfo) -> { + List shardStatusBuilder = new ArrayList<>(); + final Map shardStatuses; + try { + shardStatuses = snapshotShards(repositoryName, repositoryData, task, snapshotInfo); + } catch (Exception e) { + // stops all further fetches of snapshotInfo since context is fail-fast + context.onFailure(e); + return; + } + for (Map.Entry shardStatus : shardStatuses.entrySet()) { + IndexShardSnapshotStatus.Copy lastSnapshotStatus = shardStatus.getValue().asCopy(); + shardStatusBuilder.add(new SnapshotIndexShardStatus(shardStatus.getKey(), lastSnapshotStatus)); + } + final SnapshotsInProgress.State state; + switch (snapshotInfo.state()) { + case FAILED: + state = SnapshotsInProgress.State.FAILED; + break; + case SUCCESS: + case PARTIAL: + // Translating both PARTIAL and SUCCESS to SUCCESS for now + // TODO: add the differentiation on the metadata level in the next major release + state = SnapshotsInProgress.State.SUCCESS; + break; + default: + throw new IllegalArgumentException("Unknown snapshot state " + snapshotInfo.state()); + } + final long startTime = snapshotInfo.startTime(); + final long endTime = snapshotInfo.endTime(); + assert endTime >= startTime || (endTime == 0L && snapshotInfo.state().completed() == false) + : "Inconsistent timestamps found in SnapshotInfo [" + snapshotInfo + "]"; + threadSafeBuilder.add( + new SnapshotStatus( + new Snapshot(repositoryName, snapshotInfo.snapshotId()), + state, + Collections.unmodifiableList(shardStatusBuilder), + snapshotInfo.includeGlobalState(), + startTime, + // Use current time to calculate overall runtime for in-progress snapshots that have endTime == 0 + (endTime == 0 ? threadPool.absoluteTimeInMillis() : endTime) - startTime + ) + ); + }, listener.map(v -> new SnapshotsStatusResponse(org.elasticsearch.core.List.copyOf(threadSafeBuilder))))); + } + }, listener::onFailure); } /** diff --git a/server/src/main/java/org/elasticsearch/client/Requests.java b/server/src/main/java/org/elasticsearch/client/Requests.java index a8b9f8dd83670..7bebab2017b32 100644 --- a/server/src/main/java/org/elasticsearch/client/Requests.java +++ b/server/src/main/java/org/elasticsearch/client/Requests.java @@ -494,13 +494,13 @@ public static CreateSnapshotRequest createSnapshotRequest(String repository, Str } /** - * Gets snapshots from repository + * Gets snapshots from repositories * - * @param repository repository name + * @param repositories repository names * @return get snapshot request */ - public static GetSnapshotsRequest getSnapshotsRequest(String repository) { - return new GetSnapshotsRequest(repository); + public static GetSnapshotsRequest getSnapshotsRequest(String... repositories) { + return new GetSnapshotsRequest(repositories); } /** diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java index c9260f9e9803e..19dac8a30f2f5 100644 --- a/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java +++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java @@ -558,14 +558,14 @@ private Entry(Snapshot snapshot, boolean includeGlobalState, boolean partial, St this.snapshot = snapshot; this.includeGlobalState = includeGlobalState; this.partial = partial; - this.indices = indices; - this.dataStreams = dataStreams; + this.indices = org.elasticsearch.core.Map.copyOf(indices); + this.dataStreams = org.elasticsearch.core.List.copyOf(dataStreams); this.featureStates = Collections.unmodifiableList(featureStates); this.startTime = startTime; this.shards = shards; this.repositoryStateId = repositoryStateId; this.failure = failure; - this.userMetadata = userMetadata; + this.userMetadata = userMetadata == null ? null : org.elasticsearch.core.Map.copyOf(userMetadata); this.version = version; this.source = source; if (source == null) { diff --git a/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java b/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java index 6bd06477c6500..fe87c65820a83 100644 --- a/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java @@ -50,8 +50,8 @@ public RepositoryMetadata getMetadata() { } @Override - public SnapshotInfo getSnapshotInfo(SnapshotId snapshotId) { - return in.getSnapshotInfo(snapshotId); + public void getSnapshotInfo(GetSnapshotInfoContext context) { + in.getSnapshotInfo(context); } @Override diff --git a/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java b/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java new file mode 100644 index 0000000000000..8205b59cf848a --- /dev/null +++ b/server/src/main/java/org/elasticsearch/repositories/GetSnapshotInfoContext.java @@ -0,0 +1,148 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ +package org.elasticsearch.repositories; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.util.concurrent.CountDown; +import org.elasticsearch.snapshots.SnapshotId; +import org.elasticsearch.snapshots.SnapshotInfo; +import org.elasticsearch.threadpool.ThreadPool; + +import java.util.Collection; +import java.util.List; +import java.util.function.BiConsumer; +import java.util.function.BooleanSupplier; + +/** + * Describes the context of fetching one or more {@link SnapshotInfo} via {@link Repository#getSnapshotInfo(GetSnapshotInfoContext)}. + */ +public final class GetSnapshotInfoContext implements ActionListener { + + private static final Logger logger = LogManager.getLogger(GetSnapshotInfoContext.class); + + /** + * Snapshot ids to fetch info for. + */ + private final List snapshotIds; + + /** + * Stop fetching additional {@link SnapshotInfo} if an exception is encountered. + */ + private final boolean abortOnFailure; + + /** + * If this supplier returns true, indicates that the task that initiated this context has been cancelled and that not further fetching + * of {@link SnapshotInfo} should be started. + */ + private final BooleanSupplier isCancelled; + + /** + * Listener resolved when fetching {@link SnapshotInfo} has completed. If resolved successfully, no more calls to + * {@link #consumer} will be made. Only resolves exceptionally if {@link #abortOnFailure} is true in case one or more + * {@link SnapshotInfo} failed to be fetched. + * This listener is always invoked on the {@link ThreadPool.Names#SNAPSHOT_META} pool. + */ + private final ActionListener doneListener; + + /** + * {@link BiConsumer} invoked for each {@link SnapshotInfo} that is fetched with this instance and the {@code SnapshotInfo} as + * arguments. This consumer is always invoked on the {@link ThreadPool.Names#SNAPSHOT_META} pool. + */ + private final BiConsumer consumer; + + private final CountDown counter; + + public GetSnapshotInfoContext( + Collection snapshotIds, + boolean abortOnFailure, + BooleanSupplier isCancelled, + BiConsumer consumer, + ActionListener listener + ) { + if (snapshotIds.isEmpty()) { + throw new IllegalArgumentException("no snapshot ids to fetch given"); + } + this.snapshotIds = org.elasticsearch.core.List.copyOf(snapshotIds); + this.counter = new CountDown(snapshotIds.size()); + this.abortOnFailure = abortOnFailure; + this.isCancelled = isCancelled; + this.consumer = consumer; + this.doneListener = listener; + } + + public List snapshotIds() { + return snapshotIds; + } + + /** + * @return true if fetching {@link SnapshotInfo} should be stopped after encountering any exception + */ + public boolean abortOnFailure() { + return abortOnFailure; + } + + /** + * @return true if fetching {@link SnapshotInfo} has been cancelled + */ + public boolean isCancelled() { + return isCancelled.getAsBoolean(); + } + + /** + * @return true if fetching {@link SnapshotInfo} is either complete or should be stopped because of an error + */ + public boolean done() { + return counter.isCountedDown(); + } + + @Override + public void onResponse(SnapshotInfo snapshotInfo) { + assert Repository.assertSnapshotMetaThread(); + try { + consumer.accept(this, snapshotInfo); + } catch (Exception e) { + assert false : e; + onFailure(e); + return; + } + if (counter.countDown()) { + try { + doneListener.onResponse(null); + } catch (Exception e) { + assert false : e; + failDoneListener(e); + } + } + } + + @Override + public void onFailure(Exception e) { + assert Repository.assertSnapshotMetaThread(); + if (abortOnFailure) { + if (counter.fastForward()) { + failDoneListener(e); + } + } else { + logger.warn("failed to fetch snapshot info", e); + if (counter.countDown()) { + doneListener.onResponse(null); + } + } + } + + private void failDoneListener(Exception failure) { + try { + doneListener.onFailure(failure); + } catch (Exception ex) { + assert false : ex; + throw ex; + } + } +} diff --git a/server/src/main/java/org/elasticsearch/repositories/Repository.java b/server/src/main/java/org/elasticsearch/repositories/Repository.java index a548ccb330d03..dcde761beafd3 100644 --- a/server/src/main/java/org/elasticsearch/repositories/Repository.java +++ b/server/src/main/java/org/elasticsearch/repositories/Repository.java @@ -25,9 +25,11 @@ import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.snapshots.SnapshotInfo; +import org.elasticsearch.threadpool.ThreadPool; import java.io.IOException; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; @@ -71,12 +73,34 @@ default Repository create(RepositoryMetadata metadata, Function listener) { + getSnapshotInfo(new GetSnapshotInfoContext(Collections.singletonList(snapshotId), true, () -> false, (context, snapshotInfo) -> { + assert Repository.assertSnapshotMetaThread(); + listener.onResponse(snapshotInfo); + }, new ActionListener() { + @Override + public void onResponse(Void o) { + // ignored + } + + @Override + public void onFailure(Exception e) { + listener.onFailure(e); + } + })); + } /** * Returns global metadata associated with the snapshot. @@ -310,4 +334,11 @@ void cloneShardSnapshot( default Map adaptUserMetadata(Map userMetadata) { return userMetadata; } + + static boolean assertSnapshotMetaThread() { + final String threadName = Thread.currentThread().getName(); + assert threadName.contains('[' + ThreadPool.Names.SNAPSHOT_META + ']') || threadName.startsWith("TEST-") + : "Expected current thread [" + Thread.currentThread() + "] to be a snapshot meta thread."; + return true; + } } diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index c1121695c6cde..63cdece736485 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -90,6 +90,7 @@ import org.elasticsearch.index.store.StoreFileMetadata; import org.elasticsearch.indices.recovery.RecoverySettings; import org.elasticsearch.indices.recovery.RecoveryState; +import org.elasticsearch.repositories.GetSnapshotInfoContext; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.IndexMetaDataGenerations; import org.elasticsearch.repositories.RepositoriesService; @@ -112,6 +113,7 @@ import org.elasticsearch.snapshots.SnapshotInfo; import org.elasticsearch.snapshots.SnapshotMissingException; import org.elasticsearch.snapshots.SnapshotsService; +import org.elasticsearch.tasks.TaskCancelledException; import org.elasticsearch.threadpool.ThreadPool; import java.io.FilterInputStream; @@ -262,13 +264,13 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp public static final ChecksumBlobStoreFormat GLOBAL_METADATA_FORMAT = new ChecksumBlobStoreFormat<>( "metadata", METADATA_NAME_FORMAT, - Metadata::fromXContent + (repoName, parser) -> Metadata.fromXContent(parser) ); public static final ChecksumBlobStoreFormat INDEX_METADATA_FORMAT = new ChecksumBlobStoreFormat<>( "index-metadata", METADATA_NAME_FORMAT, - IndexMetadata::fromXContent + (repoName, parser) -> IndexMetadata.fromXContent(parser) ); private static final String SNAPSHOT_CODEC = "snapshot"; @@ -282,13 +284,13 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp public static final ChecksumBlobStoreFormat INDEX_SHARD_SNAPSHOT_FORMAT = new ChecksumBlobStoreFormat<>( SNAPSHOT_CODEC, SNAPSHOT_NAME_FORMAT, - BlobStoreIndexShardSnapshot::fromXContent + (repoName, parser) -> BlobStoreIndexShardSnapshot.fromXContent(parser) ); public static final ChecksumBlobStoreFormat INDEX_SHARD_SNAPSHOTS_FORMAT = new ChecksumBlobStoreFormat<>( "snapshots", SNAPSHOT_INDEX_NAME_FORMAT, - BlobStoreIndexShardSnapshots::fromXContent + (repoName, parser) -> BlobStoreIndexShardSnapshots.fromXContent(parser) ); public static final Setting MAX_SNAPSHOT_BYTES_PER_SEC = Setting.byteSizeSetting( @@ -1022,7 +1024,8 @@ private void writeUpdatedShardMetaDataAndComputeDeletes( for (String indexMetaGeneration : indexMetaGenerations) { executor.execute(ActionRunnable.supply(allShardCountsListener, () -> { try { - return INDEX_METADATA_FORMAT.read(indexContainer, indexMetaGeneration, namedXContentRegistry).getNumberOfShards(); + return INDEX_METADATA_FORMAT.read(metadata.name(), indexContainer, indexMetaGeneration, namedXContentRegistry) + .getNumberOfShards(); } catch (Exception ex) { logger.warn( () -> new ParameterizedMessage( @@ -1477,20 +1480,63 @@ private void cleanupOldShardGens(RepositoryData existingRepositoryData, Reposito } @Override - public SnapshotInfo getSnapshotInfo(final SnapshotId snapshotId) { - try { - return SNAPSHOT_FORMAT.read(blobContainer(), snapshotId.getUUID(), namedXContentRegistry); - } catch (NoSuchFileException ex) { - throw new SnapshotMissingException(metadata.name(), snapshotId, ex); - } catch (IOException | NotXContentException ex) { - throw new SnapshotException(metadata.name(), snapshotId, "failed to get snapshots", ex); + public void getSnapshotInfo(GetSnapshotInfoContext context) { + // put snapshot info downloads into a task queue instead of pushing them all into the queue to not completely monopolize the + // snapshot meta pool for a single request + final int workers = Math.min(threadPool.info(ThreadPool.Names.SNAPSHOT_META).getMax(), context.snapshotIds().size()); + final BlockingQueue queue = new LinkedBlockingQueue<>(context.snapshotIds()); + for (int i = 0; i < workers; i++) { + getOneSnapshotInfo(queue, context); } } + /** + * Tries to poll a {@link SnapshotId} to load {@link SnapshotInfo} for from the given {@code queue}. + */ + private void getOneSnapshotInfo(BlockingQueue queue, GetSnapshotInfoContext context) { + final SnapshotId snapshotId = queue.poll(); + if (snapshotId == null) { + return; + } + threadPool.executor(ThreadPool.Names.SNAPSHOT_META).execute(() -> { + if (context.done()) { + return; + } + if (context.isCancelled()) { + queue.clear(); + context.onFailure(new TaskCancelledException("task cancelled")); + return; + } + Exception failure = null; + SnapshotInfo snapshotInfo = null; + try { + snapshotInfo = SNAPSHOT_FORMAT.read(metadata.name(), blobContainer(), snapshotId.getUUID(), namedXContentRegistry); + } catch (NoSuchFileException ex) { + failure = new SnapshotMissingException(metadata.name(), snapshotId, ex); + } catch (IOException | NotXContentException ex) { + failure = new SnapshotException(metadata.name(), snapshotId, "failed to get snapshot info" + snapshotId, ex); + } catch (Exception e) { + failure = e instanceof SnapshotException + ? e + : new SnapshotException(metadata.name(), snapshotId, "Snapshot could not be read", e); + } + if (failure != null) { + if (context.abortOnFailure()) { + queue.clear(); + } + context.onFailure(failure); + } else { + assert snapshotInfo != null; + context.onResponse(snapshotInfo); + } + getOneSnapshotInfo(queue, context); + }); + } + @Override public Metadata getSnapshotGlobalMetadata(final SnapshotId snapshotId) { try { - return GLOBAL_METADATA_FORMAT.read(blobContainer(), snapshotId.getUUID(), namedXContentRegistry); + return GLOBAL_METADATA_FORMAT.read(metadata.name(), blobContainer(), snapshotId.getUUID(), namedXContentRegistry); } catch (NoSuchFileException ex) { throw new SnapshotMissingException(metadata.name(), snapshotId, ex); } catch (IOException ex) { @@ -1502,6 +1548,7 @@ public Metadata getSnapshotGlobalMetadata(final SnapshotId snapshotId) { public IndexMetadata getSnapshotIndexMetaData(RepositoryData repositoryData, SnapshotId snapshotId, IndexId index) throws IOException { try { return INDEX_METADATA_FORMAT.read( + metadata.name(), indexContainer(index), repositoryData.indexMetaDataGenerations().indexMetaBlobId(snapshotId, index), namedXContentRegistry @@ -1577,6 +1624,7 @@ public long getRestoreThrottleTimeInNanos() { protected void assertSnapshotOrGenericThread() { assert Thread.currentThread().getName().contains('[' + ThreadPool.Names.SNAPSHOT + ']') + || Thread.currentThread().getName().contains('[' + ThreadPool.Names.SNAPSHOT_META + ']') || Thread.currentThread().getName().contains('[' + ThreadPool.Names.GENERIC + ']') : "Expected current thread [" + Thread.currentThread() + "] to be the snapshot or generic thread."; } @@ -1656,13 +1704,14 @@ && isReadOnly() == false // Don't deduplicate repo data loading if we don't have strong consistency guarantees between the repo and the cluster state // Also, if we are not caching repository data (for tests) we assume that the contents of the repository data at a given // generation may change + final Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT_META); if (bestEffortConsistency || cacheRepositoryData == false) { - threadPool.generic().execute(ActionRunnable.wrap(listener, this::doGetRepositoryData)); + executor.execute(ActionRunnable.wrap(listener, this::doGetRepositoryData)); } else { repoDataDeduplicator.executeOnce( metadata, listener, - (metadata, l) -> threadPool.generic().execute(ActionRunnable.wrap(l, this::doGetRepositoryData)) + (metadata, l) -> executor.execute(ActionRunnable.wrap(l, this::doGetRepositoryData)) ); } } @@ -2174,43 +2223,40 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS .collect(Collectors.toList()); if (snapshotIdsWithMissingDetails.isEmpty() == false) { final Map extraDetailsMap = new ConcurrentHashMap<>(); - final GroupedActionListener loadExtraDetailsListener = new GroupedActionListener<>( - ActionListener.runAfter(new ActionListener>() { - @Override - public void onResponse(Collection voids) { - logger.info( - "Successfully loaded all snapshots' detailed information for {} from snapshot metadata", - AllocationService.firstListElementsToCommaDelimitedString( - snapshotIdsWithMissingDetails, - SnapshotId::toString, - logger.isDebugEnabled() - ) - ); - } - - @Override - public void onFailure(Exception e) { - logger.warn("Failure when trying to load missing details from snapshot metadata", e); - } - }, () -> filterRepositoryDataStep.onResponse(repositoryData.withExtraDetails(extraDetailsMap))), - snapshotIdsWithMissingDetails.size() - ); - for (SnapshotId snapshotId : snapshotIdsWithMissingDetails) { - // Just spawn all the download jobs at the same time: this is pretty important, executes only rarely (typically once - // after an upgrade) and each job is only a small download so this shouldn't block other SNAPSHOT activities for long. - threadPool().executor(ThreadPool.Names.SNAPSHOT).execute(ActionRunnable.run(loadExtraDetailsListener, () -> { - final SnapshotInfo snapshotInfo = getSnapshotInfo(snapshotId); - extraDetailsMap.put( - snapshotId, + getSnapshotInfo( + new GetSnapshotInfoContext( + snapshotIdsWithMissingDetails, + false, + () -> false, + (context, snapshotInfo) -> extraDetailsMap.put( + snapshotInfo.snapshotId(), new SnapshotDetails( snapshotInfo.state(), snapshotInfo.version(), snapshotInfo.startTime(), snapshotInfo.endTime() ) - ); - })); - } + ), + ActionListener.runAfter(new ActionListener() { + @Override + public void onResponse(Void aVoid) { + logger.info( + "Successfully loaded all snapshots' detailed information for {} from snapshot metadata", + AllocationService.firstListElementsToCommaDelimitedString( + snapshotIdsWithMissingDetails, + SnapshotId::toString, + logger.isDebugEnabled() + ) + ); + } + + @Override + public void onFailure(Exception e) { + logger.warn("Failure when trying to load missing details from snapshot metadata", e); + } + }, () -> filterRepositoryDataStep.onResponse(repositoryData.withExtraDetails(extraDetailsMap))) + ) + ); } else { filterRepositoryDataStep.onResponse(repositoryData); } @@ -3198,7 +3244,7 @@ private static List unusedBlobs( */ public BlobStoreIndexShardSnapshot loadShardSnapshot(BlobContainer shardContainer, SnapshotId snapshotId) { try { - return INDEX_SHARD_SNAPSHOT_FORMAT.read(shardContainer, snapshotId.getUUID(), namedXContentRegistry); + return INDEX_SHARD_SNAPSHOT_FORMAT.read(metadata.name(), shardContainer, snapshotId.getUUID(), namedXContentRegistry); } catch (NoSuchFileException ex) { throw new SnapshotMissingException(metadata.name(), snapshotId, ex); } catch (IOException ex) { @@ -3230,7 +3276,10 @@ private Tuple buildBlobStoreIndexShardSnap if (generation.equals(ShardGenerations.NEW_SHARD_GEN)) { return new Tuple<>(BlobStoreIndexShardSnapshots.EMPTY, ShardGenerations.NEW_SHARD_GEN); } - return new Tuple<>(INDEX_SHARD_SNAPSHOTS_FORMAT.read(shardContainer, generation, namedXContentRegistry), generation); + return new Tuple<>( + INDEX_SHARD_SNAPSHOTS_FORMAT.read(metadata.name(), shardContainer, generation, namedXContentRegistry), + generation + ); } final Tuple legacyIndex = buildBlobStoreIndexShardSnapshots(blobs, shardContainer); return new Tuple<>(legacyIndex.v1(), String.valueOf(legacyIndex.v2())); @@ -3247,6 +3296,7 @@ private Tuple buildBlobStoreIndexShardSnapsh long latest = latestGeneration(blobs); if (latest >= 0) { final BlobStoreIndexShardSnapshots shardSnapshots = INDEX_SHARD_SNAPSHOTS_FORMAT.read( + metadata.name(), shardContainer, Long.toString(latest), namedXContentRegistry diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java index b304b40ce1285..00f136b6413fa 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/ChecksumBlobStoreFormat.java @@ -15,6 +15,7 @@ import org.apache.lucene.store.InputStreamDataInput; import org.apache.lucene.store.OutputStreamIndexOutput; import org.elasticsearch.cluster.metadata.Metadata; +import org.elasticsearch.common.CheckedBiFunction; import org.elasticsearch.common.Numbers; import org.elasticsearch.common.blobstore.BlobContainer; import org.elasticsearch.common.bytes.BytesArray; @@ -32,7 +33,6 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.core.CheckedConsumer; -import org.elasticsearch.core.CheckedFunction; import org.elasticsearch.gateway.CorruptStateException; import java.io.FilterInputStream; @@ -64,14 +64,14 @@ public final class ChecksumBlobStoreFormat { private final String blobNameFormat; - private final CheckedFunction reader; + private final CheckedBiFunction reader; /** * @param codec codec name * @param blobNameFormat format of the blobname in {@link String#format} format * @param reader prototype object that can deserialize T from XContent */ - public ChecksumBlobStoreFormat(String codec, String blobNameFormat, CheckedFunction reader) { + public ChecksumBlobStoreFormat(String codec, String blobNameFormat, CheckedBiFunction reader) { this.reader = reader; this.blobNameFormat = blobNameFormat; this.codec = codec; @@ -84,10 +84,11 @@ public ChecksumBlobStoreFormat(String codec, String blobNameFormat, CheckedFunct * @param name name to be translated into * @return parsed blob object */ - public T read(BlobContainer blobContainer, String name, NamedXContentRegistry namedXContentRegistry) throws IOException { + public T read(String repoName, BlobContainer blobContainer, String name, NamedXContentRegistry namedXContentRegistry) + throws IOException { String blobName = blobName(name); try (InputStream in = blobContainer.readBlob(blobName)) { - return deserialize(namedXContentRegistry, in); + return deserialize(repoName, namedXContentRegistry, in); } } @@ -95,7 +96,7 @@ public String blobName(String name) { return String.format(Locale.ROOT, blobNameFormat, name); } - public T deserialize(NamedXContentRegistry namedXContentRegistry, InputStream input) throws IOException { + public T deserialize(String repoName, NamedXContentRegistry namedXContentRegistry, InputStream input) throws IOException { final DeserializeMetaBlobInputStream deserializeMetaBlobInputStream = new DeserializeMetaBlobInputStream(input); try { CodecUtil.checkHeader(new InputStreamDataInput(deserializeMetaBlobInputStream), codec, VERSION, VERSION); @@ -110,7 +111,7 @@ public T deserialize(NamedXContentRegistry namedXContentRegistry, InputStream in XContentParser parser = XContentType.SMILE.xContent() .createParser(namedXContentRegistry, LoggingDeprecationHandler.INSTANCE, wrappedStream) ) { - result = reader.apply(parser); + result = reader.apply(repoName, parser); } deserializeMetaBlobInputStream.verifyFooter(); return result; @@ -295,7 +296,7 @@ public void serialize( CodecUtil.writeHeader(indexOutput, codec, VERSION); try (OutputStream indexOutputOutputStream = new IndexOutputOutputStream(indexOutput) { @Override - public void close() throws IOException { + public void close() { // this is important since some of the XContentBuilders write bytes on close. // in order to write the footer we need to prevent closing the actual index input. } diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestGetSnapshotsAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestGetSnapshotsAction.java index 306943a2c26da..a4284711928fb 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestGetSnapshotsAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestGetSnapshotsAction.java @@ -15,15 +15,16 @@ import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.action.RestToXContentListener; import org.elasticsearch.rest.action.RestCancellableNodeClient; +import org.elasticsearch.search.sort.SortOrder; import java.io.IOException; -import java.util.Collections; import java.util.List; import java.util.Set; import static java.util.Collections.singletonList; import static org.elasticsearch.client.Requests.getSnapshotsRequest; import static org.elasticsearch.rest.RestRequest.Method.GET; +import static org.elasticsearch.snapshots.SnapshotInfo.INCLUDE_REPOSITORY_XCONTENT_PARAM; import static org.elasticsearch.snapshots.SnapshotInfo.INDEX_DETAILS_XCONTENT_PARAM; /** @@ -43,17 +44,27 @@ public String getName() { @Override protected Set responseParams() { - return Collections.singleton(INDEX_DETAILS_XCONTENT_PARAM); + return org.elasticsearch.core.Set.of(INDEX_DETAILS_XCONTENT_PARAM, INCLUDE_REPOSITORY_XCONTENT_PARAM); } @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { - String repository = request.param("repository"); + String[] repositories = request.paramAsStringArray("repository", Strings.EMPTY_ARRAY); String[] snapshots = request.paramAsStringArray("snapshot", Strings.EMPTY_ARRAY); - GetSnapshotsRequest getSnapshotsRequest = getSnapshotsRequest(repository).snapshots(snapshots); + GetSnapshotsRequest getSnapshotsRequest = getSnapshotsRequest(repositories).snapshots(snapshots); getSnapshotsRequest.ignoreUnavailable(request.paramAsBoolean("ignore_unavailable", getSnapshotsRequest.ignoreUnavailable())); getSnapshotsRequest.verbose(request.paramAsBoolean("verbose", getSnapshotsRequest.verbose())); + final GetSnapshotsRequest.SortBy sort = GetSnapshotsRequest.SortBy.of(request.param("sort", getSnapshotsRequest.sort().toString())); + getSnapshotsRequest.sort(sort); + final int size = request.paramAsInt("size", getSnapshotsRequest.size()); + getSnapshotsRequest.size(size); + final String afterString = request.param("after"); + if (afterString != null) { + getSnapshotsRequest.after(GetSnapshotsRequest.After.fromQueryParam(afterString)); + } + final SortOrder order = SortOrder.fromString(request.param("order", getSnapshotsRequest.order().toString())); + getSnapshotsRequest.order(order); getSnapshotsRequest.masterNodeTimeout(request.paramAsTime("master_timeout", getSnapshotsRequest.masterNodeTimeout())); return channel -> new RestCancellableNodeClient(client, request.getHttpChannel()).admin().cluster() .getSnapshots(getSnapshotsRequest, new RestToXContentListener<>(channel)); diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/RestSnapshotAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/RestSnapshotAction.java index 2f452312ac854..3046105550a99 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/cat/RestSnapshotAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/cat/RestSnapshotAction.java @@ -9,9 +9,11 @@ package org.elasticsearch.rest.action.cat; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsRequest; import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsResponse; import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.Table; import org.elasticsearch.common.time.DateFormatter; import org.elasticsearch.core.TimeValue; @@ -76,6 +78,7 @@ protected Table getTableWithHeader(RestRequest request) { return new Table() .startHeaders() .addCell("id", "alias:id,snapshot;desc:unique snapshot") + .addCell("repository", "alias:re,repo;desc:repository name") .addCell("status", "alias:s,status;text-align:right;desc:snapshot name") .addCell("start_epoch", "alias:ste,startEpoch;desc:start time in seconds since 1970-01-01 00:00:00") .addCell("start_time", "alias:sti,startTime;desc:start time in HH:MM:SS") @@ -94,10 +97,28 @@ protected Table getTableWithHeader(RestRequest request) { private Table buildTable(RestRequest req, GetSnapshotsResponse getSnapshotsResponse) { Table table = getTableWithHeader(req); - for (SnapshotInfo snapshotStatus : getSnapshotsResponse.getSnapshots()) { + + if (getSnapshotsResponse.isFailed()) { + ElasticsearchException causes = null; + + for (ElasticsearchException e : getSnapshotsResponse.getFailures().values()) { + if (causes == null) { + causes = e; + } else { + causes.addSuppressed(e); + } + } + throw new ElasticsearchException( + "Repositories [" + + Strings.collectionToCommaDelimitedString(getSnapshotsResponse.getFailures().keySet()) + + "] failed to retrieve snapshots", causes); + } + + for (SnapshotInfo snapshotStatus: getSnapshotsResponse.getSnapshots()) { table.startRow(); table.addCell(snapshotStatus.snapshotId().getName()); + table.addCell(snapshotStatus.repository()); table.addCell(snapshotStatus.state()); table.addCell(TimeUnit.SECONDS.convert(snapshotStatus.startTime(), TimeUnit.MILLISECONDS)); table.addCell(FORMATTER.format(Instant.ofEpochMilli(snapshotStatus.startTime()))); diff --git a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java index ca86573ddfc15..9f82368c8faea 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java @@ -17,7 +17,6 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.StepListener; import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotRequest; import org.elasticsearch.action.support.GroupedActionListener; @@ -235,10 +234,7 @@ public void restoreSnapshot( final StepListener repositoryDataListener = new StepListener<>(); repository.getRepositoryData(repositoryDataListener); - repositoryDataListener.whenComplete(repositoryData -> repositoryUuidRefreshListener.whenComplete(ignored -> - // fork handling to the generic pool since it loads various pieces of metadata from the repository over a longer period - // of time - clusterService.getClusterApplierService().threadPool().generic().execute(ActionRunnable.wrap(listener, l -> { + repositoryDataListener.whenComplete(repositoryData -> repositoryUuidRefreshListener.whenComplete(ignored -> { final String snapshotName = request.snapshot(); final Optional matchingSnapshotId = repositoryData.getSnapshotIds() .stream() @@ -256,8 +252,14 @@ public void restoreSnapshot( "snapshot UUID mismatch: expected [" + request.snapshotUuid() + "] but got [" + snapshotId.getUUID() + "]" ); } - startRestore(repository.getSnapshotInfo(snapshotId), repository, request, repositoryData, updater, l); - })), listener::onFailure), listener::onFailure); + repository.getSnapshotInfo( + snapshotId, + ActionListener.wrap( + snapshotInfo -> startRestore(snapshotInfo, repository, request, repositoryData, updater, listener), + listener::onFailure + ) + ); + }, listener::onFailure), listener::onFailure); } catch (Exception e) { logger.warn( () -> new ParameterizedMessage("[{}] failed to restore snapshot", request.repository() + ":" + request.snapshot()), @@ -289,6 +291,7 @@ private void startRestore( BiConsumer updater, ActionListener listener ) throws IOException { + assert Repository.assertSnapshotMetaThread(); final SnapshotId snapshotId = snapshotInfo.snapshotId(); final String repositoryName = repository.getMetadata().name(); final Snapshot snapshot = new Snapshot(repositoryName, snapshotId); diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java index 40de0dc0b24a8..4add6d73fdb68 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotInfo.java @@ -52,10 +52,12 @@ public final class SnapshotInfo implements Comparable, ToXContent, public static final Version METADATA_FIELD_INTRODUCED = Version.V_7_3_0; public static final String INDEX_DETAILS_XCONTENT_PARAM = "index_details"; + public static final String INCLUDE_REPOSITORY_XCONTENT_PARAM = "include_repository"; private static final DateFormatter DATE_TIME_FORMATTER = DateFormatter.forPattern("strict_date_optional_time"); private static final String SNAPSHOT = "snapshot"; private static final String UUID = "uuid"; + private static final String REPOSITORY = "repository"; private static final String INDICES = "indices"; private static final String DATA_STREAMS = "data_streams"; private static final String STATE = "state"; @@ -81,12 +83,15 @@ public final class SnapshotInfo implements Comparable, ToXContent, private static final String FEATURE_STATES = "feature_states"; private static final String INDEX_DETAILS = "index_details"; + private static final String UNKNOWN_REPO_NAME = "_na_"; + private static final Comparator COMPARATOR = Comparator.comparing(SnapshotInfo::startTime) .thenComparing(SnapshotInfo::snapshotId); public static final class SnapshotInfoBuilder { private String snapshotName = null; private String snapshotUUID = null; + private String repository = UNKNOWN_REPO_NAME; private String state = null; private String reason = null; private List indices = null; @@ -109,6 +114,10 @@ private void setSnapshotUUID(String snapshotUUID) { this.snapshotUUID = snapshotUUID; } + private void setRepository(String repository) { + this.repository = repository; + } + private void setState(String state) { this.state = state; } @@ -162,7 +171,7 @@ private void setShardFailures(List shardFailures) { } public SnapshotInfo build() { - SnapshotId snapshotId = new SnapshotId(snapshotName, snapshotUUID); + final Snapshot snapshot = new Snapshot(repository, new SnapshotId(snapshotName, snapshotUUID)); if (indices == null) { indices = Collections.emptyList(); @@ -191,7 +200,7 @@ public SnapshotInfo build() { } return new SnapshotInfo( - snapshotId, + snapshot, indices, dataStreams, featureStates, @@ -246,6 +255,7 @@ int getSuccessfulShards() { static { SNAPSHOT_INFO_PARSER.declareString(SnapshotInfoBuilder::setSnapshotName, new ParseField(SNAPSHOT)); SNAPSHOT_INFO_PARSER.declareString(SnapshotInfoBuilder::setSnapshotUUID, new ParseField(UUID)); + SNAPSHOT_INFO_PARSER.declareString(SnapshotInfoBuilder::setRepository, new ParseField(REPOSITORY)); SNAPSHOT_INFO_PARSER.declareString(SnapshotInfoBuilder::setState, new ParseField(STATE)); SNAPSHOT_INFO_PARSER.declareString(SnapshotInfoBuilder::setReason, new ParseField(REASON)); SNAPSHOT_INFO_PARSER.declareStringArray(SnapshotInfoBuilder::setIndices, new ParseField(INDICES)); @@ -276,7 +286,7 @@ int getSuccessfulShards() { SHARD_STATS_PARSER.declareInt(ShardStatsBuilder::setSuccessfulShards, new ParseField(SUCCESSFUL)); } - private final SnapshotId snapshotId; + private final Snapshot snapshot; @Nullable private final SnapshotState state; @@ -312,14 +322,14 @@ int getSuccessfulShards() { private final Map indexSnapshotDetails; public SnapshotInfo( - SnapshotId snapshotId, + Snapshot snapshot, List indices, List dataStreams, List featureStates, SnapshotState state ) { this( - snapshotId, + snapshot, indices, dataStreams, featureStates, @@ -338,7 +348,7 @@ public SnapshotInfo( } public SnapshotInfo( - SnapshotId snapshotId, + Snapshot snapshot, List indices, List dataStreams, List featureStates, @@ -346,7 +356,7 @@ public SnapshotInfo( SnapshotState state ) { this( - snapshotId, + snapshot, indices, dataStreams, featureStates, @@ -366,7 +376,7 @@ public SnapshotInfo( public SnapshotInfo(SnapshotsInProgress.Entry entry) { this( - entry.snapshot().getSnapshotId(), + entry.snapshot(), org.elasticsearch.core.List.copyOf(entry.indices().keySet()), entry.dataStreams(), entry.featureStates(), @@ -385,7 +395,7 @@ public SnapshotInfo(SnapshotsInProgress.Entry entry) { } public SnapshotInfo( - SnapshotId snapshotId, + Snapshot snapshot, List indices, List dataStreams, List featureStates, @@ -399,7 +409,7 @@ public SnapshotInfo( Map indexSnapshotDetails ) { this( - snapshotId, + snapshot, indices, dataStreams, featureStates, @@ -418,7 +428,7 @@ public SnapshotInfo( } SnapshotInfo( - SnapshotId snapshotId, + Snapshot snapshot, List indices, List dataStreams, List featureStates, @@ -434,7 +444,7 @@ public SnapshotInfo( SnapshotState state, Map indexSnapshotDetails ) { - this.snapshotId = Objects.requireNonNull(snapshotId); + this.snapshot = Objects.requireNonNull(snapshot); this.indices = org.elasticsearch.core.List.copyOf(indices); this.dataStreams = org.elasticsearch.core.List.copyOf(dataStreams); this.featureStates = org.elasticsearch.core.List.copyOf(featureStates); @@ -455,7 +465,12 @@ public SnapshotInfo( * Constructs snapshot information from stream input */ public static SnapshotInfo readFrom(final StreamInput in) throws IOException { - final SnapshotId snapshotId = new SnapshotId(in); + final Snapshot snapshot; + if (in.getVersion().onOrAfter(GetSnapshotsRequest.PAGINATED_GET_SNAPSHOTS_VERSION)) { + snapshot = new Snapshot(in); + } else { + snapshot = new Snapshot(UNKNOWN_REPO_NAME, new SnapshotId(in)); + } final List indices = in.readStringList(); final SnapshotState state = in.readBoolean() ? SnapshotState.fromValue(in.readByte()) : null; final String reason = in.readOptionalString(); @@ -491,7 +506,7 @@ public static SnapshotInfo readFrom(final StreamInput in) throws IOException { indexSnapshotDetails = Collections.emptyMap(); } return new SnapshotInfo( - snapshotId, + snapshot, indices, dataStreams, featureStates, @@ -514,7 +529,11 @@ public static SnapshotInfo readFrom(final StreamInput in) throws IOException { * all information stripped out except the snapshot id, state, and indices. */ public SnapshotInfo basic() { - return new SnapshotInfo(snapshotId, indices, Collections.emptyList(), featureStates, state); + return new SnapshotInfo(snapshot, indices, Collections.emptyList(), featureStates, state); + } + + public Snapshot snapshot() { + return snapshot; } /** @@ -523,7 +542,11 @@ public SnapshotInfo basic() { * @return snapshot id */ public SnapshotId snapshotId() { - return snapshotId; + return snapshot.getSnapshotId(); + } + + public String repository() { + return snapshot.getRepository(); } /** @@ -669,8 +692,8 @@ public int compareTo(final SnapshotInfo o) { @Override public String toString() { return "SnapshotInfo{" - + "snapshotId=" - + snapshotId + + "snapshot=" + + snapshot + ", state=" + state + ", reason='" @@ -726,8 +749,14 @@ public XContentBuilder toXContent(final XContentBuilder builder, final Params pa final boolean verbose = params.paramAsBoolean("verbose", GetSnapshotsRequest.DEFAULT_VERBOSE_MODE); // write snapshot info for the API and any other situations builder.startObject(); + final SnapshotId snapshotId = snapshot.getSnapshotId(); builder.field(SNAPSHOT, snapshotId.getName()); builder.field(UUID, snapshotId.getUUID()); + + if (params.paramAsBoolean(INCLUDE_REPOSITORY_XCONTENT_PARAM, true) && UNKNOWN_REPO_NAME.equals(snapshot.getRepository()) == false) { + builder.field(REPOSITORY, snapshot.getRepository()); + } + if (version != null) { builder.field(VERSION_ID, version.id); builder.field(VERSION, version.toString()); @@ -801,6 +830,7 @@ public XContentBuilder toXContent(final XContentBuilder builder, final Params pa private XContentBuilder toXContentInternal(final XContentBuilder builder, final ToXContent.Params params) throws IOException { builder.startObject(SNAPSHOT); + final SnapshotId snapshotId = snapshot.getSnapshotId(); builder.field(NAME, snapshotId.getName()); builder.field(UUID, snapshotId.getUUID()); assert version != null : "version must always be known when writing a snapshot metadata blob"; @@ -856,7 +886,7 @@ private XContentBuilder toXContentInternal(final XContentBuilder builder, final * handle x-content written with the external version as external x-content * is only for display purposes and does not need to be parsed. */ - public static SnapshotInfo fromXContentInternal(final XContentParser parser) throws IOException { + public static SnapshotInfo fromXContentInternal(final String repoName, final XContentParser parser) throws IOException { String name = null; String uuid = null; Version version = Version.CURRENT; @@ -951,7 +981,7 @@ public static SnapshotInfo fromXContentInternal(final XContentParser parser) thr uuid = name; } return new SnapshotInfo( - new SnapshotId(name, uuid), + new Snapshot(repoName, new SnapshotId(name, uuid)), indices, dataStreams, featureStates, @@ -971,7 +1001,11 @@ public static SnapshotInfo fromXContentInternal(final XContentParser parser) thr @Override public void writeTo(final StreamOutput out) throws IOException { - snapshotId.writeTo(out); + if (out.getVersion().onOrAfter(GetSnapshotsRequest.PAGINATED_GET_SNAPSHOTS_VERSION)) { + snapshot.writeTo(out); + } else { + snapshot.getSnapshotId().writeTo(out); + } out.writeStringCollection(indices); if (state != null) { out.writeBoolean(true); @@ -1027,7 +1061,7 @@ public boolean equals(Object o) { && endTime == that.endTime && totalShards == that.totalShards && successfulShards == that.successfulShards - && Objects.equals(snapshotId, that.snapshotId) + && Objects.equals(snapshot, that.snapshot) && state == that.state && Objects.equals(reason, that.reason) && Objects.equals(indices, that.indices) @@ -1043,7 +1077,7 @@ public boolean equals(Object o) { @Override public int hashCode() { return Objects.hash( - snapshotId, + snapshot, state, reason, indices, diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index 5f314308b3284..a01af577bcd64 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -773,7 +773,7 @@ private void startCloning(Repository repository, SnapshotsInProgress.Entry clone // 1. step, load SnapshotInfo to make sure that source snapshot was successful for the indices we want to clone // TODO: we could skip this step for snapshots with state SUCCESS final StepListener snapshotInfoListener = new StepListener<>(); - executor.execute(ActionRunnable.supply(snapshotInfoListener, () -> repository.getSnapshotInfo(sourceSnapshot))); + repository.getSnapshotInfo(sourceSnapshot, snapshotInfoListener); final StepListener>> allShardCountsListener = new StepListener<>(); final GroupedActionListener> shardCountListener = new GroupedActionListener<>( @@ -1965,7 +1965,7 @@ private void finalizeSnapshotEntry(SnapshotsInProgress.Entry entry, Metadata met indexSnapshotDetails.entrySet().removeIf(e -> e.getValue().getShardCount() == 0); final SnapshotInfo snapshotInfo = new SnapshotInfo( - snapshot.getSnapshotId(), + snapshot, finalIndices, entry.dataStreams().stream().filter(metaForSnapshot.dataStreams()::containsKey).collect(Collectors.toList()), entry.partial() ? onlySuccessfulFeatureStates(entry, finalIndices) : entry.featureStates(), @@ -1974,7 +1974,9 @@ private void finalizeSnapshotEntry(SnapshotsInProgress.Entry entry, Metadata met entry.partial() ? shardGenerations.totalShards() : entry.shards().size(), shardFailures, entry.includeGlobalState(), - entry.userMetadata(), + // TODO: remove this hack making the metadata mutable once + // https://github.com/elastic/elasticsearch/pull/72776 has been merged + entry.userMetadata() == null ? null : new HashMap<>(entry.userMetadata()), entry.startTime(), indexSnapshotDetails ); diff --git a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java index 7b9f6a66f4323..afce1c14a4895 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java +++ b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java @@ -68,6 +68,7 @@ public static class Names { public static final String REFRESH = "refresh"; public static final String WARMER = "warmer"; public static final String SNAPSHOT = "snapshot"; + public static final String SNAPSHOT_META = "snapshot_meta"; public static final String FORCE_MERGE = "force_merge"; public static final String FETCH_SHARD_STARTED = "fetch_shard_started"; public static final String FETCH_SHARD_STORE = "fetch_shard_store"; @@ -128,6 +129,7 @@ public static ThreadPoolType fromType(String type) { map.put(Names.REFRESH, ThreadPoolType.SCALING); map.put(Names.WARMER, ThreadPoolType.SCALING); map.put(Names.SNAPSHOT, ThreadPoolType.SCALING); + map.put(Names.SNAPSHOT_META, ThreadPoolType.SCALING); map.put(Names.FORCE_MERGE, ThreadPoolType.FIXED); map.put(Names.FETCH_SHARD_STARTED, ThreadPoolType.SCALING); map.put(Names.FETCH_SHARD_STORE, ThreadPoolType.SCALING); @@ -147,6 +149,7 @@ public static ThreadPoolType fromType(String type) { private final ThreadContext threadContext; + @SuppressWarnings("rawtypes") private final Map builders; private final ScheduledThreadPoolExecutor scheduler; @@ -179,6 +182,7 @@ public Collection builders() { Setting.Property.NodeScope ); + @SuppressWarnings({"rawtypes", "unchecked"}) public ThreadPool(final Settings settings, final ExecutorBuilder... customBuilders) { assert Node.NODE_NAME_SETTING.exists(settings); @@ -209,6 +213,8 @@ public ThreadPool(final Settings settings, final ExecutorBuilder... customBui builders.put(Names.REFRESH, new ScalingExecutorBuilder(Names.REFRESH, 1, halfProcMaxAt10, TimeValue.timeValueMinutes(5))); builders.put(Names.WARMER, new ScalingExecutorBuilder(Names.WARMER, 1, halfProcMaxAt5, TimeValue.timeValueMinutes(5))); builders.put(Names.SNAPSHOT, new ScalingExecutorBuilder(Names.SNAPSHOT, 1, halfProcMaxAt5, TimeValue.timeValueMinutes(5))); + builders.put(Names.SNAPSHOT_META, new ScalingExecutorBuilder(Names.SNAPSHOT_META, 1, Math.min(allocatedProcessors * 3, 50), + TimeValue.timeValueSeconds(30L))); builders.put(Names.FETCH_SHARD_STARTED, new ScalingExecutorBuilder(Names.FETCH_SHARD_STARTED, 1, 2 * allocatedProcessors, TimeValue.timeValueMinutes(5))); builders.put(Names.FORCE_MERGE, new FixedExecutorBuilder(settings, Names.FORCE_MERGE, 1, -1)); diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotResponseTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotResponseTests.java index cc4d15de094c6..5717def040458 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/create/CreateSnapshotResponseTests.java @@ -10,6 +10,7 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotFeatureInfo; import org.elasticsearch.snapshots.SnapshotFeatureInfoTests; import org.elasticsearch.snapshots.SnapshotId; @@ -67,7 +68,7 @@ protected CreateSnapshotResponse createTestInstance() { return new CreateSnapshotResponse( new SnapshotInfo( - snapshotId, + new Snapshot("test-repo", snapshotId), indices, dataStreams, featureStates, diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequestTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequestTests.java new file mode 100644 index 0000000000000..03fe362f2ee39 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsRequestTests.java @@ -0,0 +1,53 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ +package org.elasticsearch.action.admin.cluster.snapshots.get; + +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.search.sort.SortOrder; +import org.elasticsearch.test.ESTestCase; + +import static org.hamcrest.Matchers.containsString; + +public class GetSnapshotsRequestTests extends ESTestCase { + + public void testValidateParameters() { + { + final GetSnapshotsRequest request = new GetSnapshotsRequest("repo", "snapshot"); + assertNull(request.validate()); + request.size(0); + final ActionRequestValidationException e = request.validate(); + assertThat(e.getMessage(), containsString("size must be -1 or greater than 0")); + } + { + final GetSnapshotsRequest request = new GetSnapshotsRequest("repo", "snapshot").size(randomIntBetween(1, 500)); + assertNull(request.validate()); + } + { + final GetSnapshotsRequest request = new GetSnapshotsRequest("repo", "snapshot").verbose(false).size(randomIntBetween(1, 500)); + final ActionRequestValidationException e = request.validate(); + assertThat(e.getMessage(), containsString("can't use size limit with verbose=false")); + } + { + final GetSnapshotsRequest request = new GetSnapshotsRequest("repo", "snapshot").verbose(false) + .sort(GetSnapshotsRequest.SortBy.INDICES); + final ActionRequestValidationException e = request.validate(); + assertThat(e.getMessage(), containsString("can't use non-default sort with verbose=false")); + } + { + final GetSnapshotsRequest request = new GetSnapshotsRequest("repo", "snapshot").verbose(false).order(SortOrder.DESC); + final ActionRequestValidationException e = request.validate(); + assertThat(e.getMessage(), containsString("can't use non-default sort order with verbose=false")); + } + { + final GetSnapshotsRequest request = new GetSnapshotsRequest("repo", "snapshot").verbose(false) + .after(new GetSnapshotsRequest.After("foo", "repo", "bar")); + final ActionRequestValidationException e = request.validate(); + assertThat(e.getMessage(), containsString("can't use after with verbose=false")); + } + } +} diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponseTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponseTests.java index 305bb7cb41805..af2de94e2c374 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/snapshots/get/GetSnapshotsResponseTests.java @@ -14,6 +14,7 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.core.Map; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotFeatureInfo; import org.elasticsearch.snapshots.SnapshotFeatureInfoTests; import org.elasticsearch.snapshots.SnapshotId; @@ -56,7 +57,7 @@ protected GetSnapshotsResponse createTestInstance() { List featureInfos = randomList(0, SnapshotFeatureInfoTests::randomSnapshotFeatureInfo); snapshots.add( new SnapshotInfo( - snapshotId, + new Snapshot(randomAlphaOfLength(5), snapshotId), Arrays.asList("index1", "index2"), Collections.singletonList("ds"), featureInfos, @@ -71,7 +72,7 @@ protected GetSnapshotsResponse createTestInstance() { ) ); } - return new GetSnapshotsResponse(snapshots); + return new GetSnapshotsResponse(snapshots, Collections.emptyMap(), null); } @Override diff --git a/server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java b/server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java index a7d065bf2f4d8..911eb98f9522f 100644 --- a/server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java +++ b/server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java @@ -202,8 +202,8 @@ public RepositoryMetadata getMetadata() { } @Override - public SnapshotInfo getSnapshotInfo(SnapshotId snapshotId) { - return null; + public void getSnapshotInfo(GetSnapshotInfoContext context) { + throw new UnsupportedOperationException(); } @Override diff --git a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryRestoreTests.java b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryRestoreTests.java index 8c8b81a5b7468..4341af98b2d9b 100644 --- a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryRestoreTests.java +++ b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryRestoreTests.java @@ -177,7 +177,7 @@ public void testSnapshotWithConflictingName() throws Exception { RepositoryData.EMPTY_REPO_GEN, Metadata.builder().put(shard.indexSettings().getIndexMetadata(), false).build(), new SnapshotInfo( - snapshot.getSnapshotId(), + snapshot, shardGenerations.indices().stream().map(IndexId::getName).collect(Collectors.toList()), Collections.emptyList(), Collections.emptyList(), diff --git a/server/src/test/java/org/elasticsearch/snapshots/BlobStoreFormatTests.java b/server/src/test/java/org/elasticsearch/snapshots/BlobStoreFormatTests.java index e0d6203e20628..57aba7691e8c0 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/BlobStoreFormatTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/BlobStoreFormatTests.java @@ -78,7 +78,11 @@ public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params par public void testBlobStoreOperations() throws IOException { BlobStore blobStore = createTestBlobStore(); BlobContainer blobContainer = blobStore.blobContainer(BlobPath.EMPTY); - ChecksumBlobStoreFormat checksumSMILE = new ChecksumBlobStoreFormat<>(BLOB_CODEC, "%s", BlobObj::fromXContent); + ChecksumBlobStoreFormat checksumSMILE = new ChecksumBlobStoreFormat<>( + BLOB_CODEC, + "%s", + (repoName, parser) -> BlobObj.fromXContent(parser) + ); // Write blobs in different formats final String randomText = randomAlphaOfLengthBetween(0, 1024 * 8 * 3); @@ -88,8 +92,8 @@ public void testBlobStoreOperations() throws IOException { checksumSMILE.write(new BlobObj(compressedText), blobContainer, "check-smile-comp", true, MockBigArrays.NON_RECYCLING_INSTANCE); // Assert that all checksum blobs can be read - assertEquals(normalText, checksumSMILE.read(blobContainer, "check-smile", xContentRegistry()).getText()); - assertEquals(compressedText, checksumSMILE.read(blobContainer, "check-smile-comp", xContentRegistry()).getText()); + assertEquals(normalText, checksumSMILE.read("repo", blobContainer, "check-smile", xContentRegistry()).getText()); + assertEquals(compressedText, checksumSMILE.read("repo", blobContainer, "check-smile-comp", xContentRegistry()).getText()); } public void testCompressionIsApplied() throws IOException { @@ -99,7 +103,11 @@ public void testCompressionIsApplied() throws IOException { for (int i = 0; i < randomIntBetween(100, 300); i++) { veryRedundantText.append("Blah "); } - ChecksumBlobStoreFormat checksumFormat = new ChecksumBlobStoreFormat<>(BLOB_CODEC, "%s", BlobObj::fromXContent); + ChecksumBlobStoreFormat checksumFormat = new ChecksumBlobStoreFormat<>( + BLOB_CODEC, + "%s", + (repo, parser) -> BlobObj.fromXContent(parser) + ); BlobObj blobObj = new BlobObj(veryRedundantText.toString()); checksumFormat.write(blobObj, blobContainer, "blob-comp", true, MockBigArrays.NON_RECYCLING_INSTANCE); checksumFormat.write(blobObj, blobContainer, "blob-not-comp", false, MockBigArrays.NON_RECYCLING_INSTANCE); @@ -113,12 +121,16 @@ public void testBlobCorruption() throws IOException { BlobContainer blobContainer = blobStore.blobContainer(BlobPath.EMPTY); String testString = randomAlphaOfLength(randomInt(10000)); BlobObj blobObj = new BlobObj(testString); - ChecksumBlobStoreFormat checksumFormat = new ChecksumBlobStoreFormat<>(BLOB_CODEC, "%s", BlobObj::fromXContent); + ChecksumBlobStoreFormat checksumFormat = new ChecksumBlobStoreFormat<>( + BLOB_CODEC, + "%s", + (repo, parser) -> BlobObj.fromXContent(parser) + ); checksumFormat.write(blobObj, blobContainer, "test-path", randomBoolean(), MockBigArrays.NON_RECYCLING_INSTANCE); - assertEquals(checksumFormat.read(blobContainer, "test-path", xContentRegistry()).getText(), testString); + assertEquals(checksumFormat.read("repo", blobContainer, "test-path", xContentRegistry()).getText(), testString); randomCorruption(blobContainer, "test-path"); try { - checksumFormat.read(blobContainer, "test-path", xContentRegistry()); + checksumFormat.read("repo", blobContainer, "test-path", xContentRegistry()); fail("Should have failed due to corruption"); } catch (ElasticsearchCorruptionException | EOFException ex) { // expected exceptions from random byte corruption diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotInfoBlobSerializationTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotInfoBlobSerializationTests.java index a0e489228a053..6d1073e14dfb7 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotInfoBlobSerializationTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotInfoBlobSerializationTests.java @@ -40,7 +40,11 @@ protected SnapshotInfo copyInstance(SnapshotInfo instance, Version version) thro BigArrays.NON_RECYCLING_INSTANCE, bytes -> ActionListener.completeWith( future, - () -> BlobStoreRepository.SNAPSHOT_FORMAT.deserialize(NamedXContentRegistry.EMPTY, bytes.streamInput()) + () -> BlobStoreRepository.SNAPSHOT_FORMAT.deserialize( + instance.repository(), + NamedXContentRegistry.EMPTY, + bytes.streamInput() + ) ) ); return future.actionGet(); diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotInfoTestUtils.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotInfoTestUtils.java index ce79151c90537..1824f0f2dcfdb 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotInfoTestUtils.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotInfoTestUtils.java @@ -35,7 +35,7 @@ public class SnapshotInfoTestUtils { private SnapshotInfoTestUtils() {} static SnapshotInfo createRandomSnapshotInfo() { - final SnapshotId snapshotId = new SnapshotId(randomAlphaOfLength(5), randomAlphaOfLength(5)); + final Snapshot snapshot = new Snapshot(randomAlphaOfLength(5), new SnapshotId(randomAlphaOfLength(5), randomAlphaOfLength(5))); final List indices = Arrays.asList(randomArray(1, 10, String[]::new, () -> randomAlphaOfLengthBetween(2, 20))); final List dataStreams = Arrays.asList(randomArray(1, 10, String[]::new, () -> randomAlphaOfLengthBetween(2, 20))); @@ -57,7 +57,7 @@ static SnapshotInfo createRandomSnapshotInfo() { final Map indexSnapshotDetails = randomIndexSnapshotDetails(); return new SnapshotInfo( - snapshotId, + snapshot, indices, dataStreams, snapshotFeatureInfos, @@ -142,8 +142,9 @@ static SnapshotInfo mutateSnapshotInfo(SnapshotInfo instance) { final SnapshotId snapshotId = randomBoolean() ? new SnapshotId(instance.snapshotId().getName(), newUuid) : new SnapshotId(newName, instance.snapshotId().getUUID()); + final String repo = randomBoolean() ? instance.repository() : randomAlphaOfLength(5); return new SnapshotInfo( - snapshotId, + new Snapshot(repo, snapshotId), instance.indices(), instance.dataStreams(), instance.featureStates(), @@ -162,7 +163,7 @@ static SnapshotInfo mutateSnapshotInfo(SnapshotInfo instance) { randomArray(indicesSize, indicesSize, String[]::new, () -> randomAlphaOfLengthBetween(2, 20)) ); return new SnapshotInfo( - instance.snapshotId(), + instance.snapshot(), indices, instance.dataStreams(), instance.featureStates(), @@ -177,7 +178,7 @@ static SnapshotInfo mutateSnapshotInfo(SnapshotInfo instance) { ); case 2: return new SnapshotInfo( - instance.snapshotId(), + instance.snapshot(), instance.indices(), instance.dataStreams(), instance.featureStates(), @@ -192,7 +193,7 @@ static SnapshotInfo mutateSnapshotInfo(SnapshotInfo instance) { ); case 3: return new SnapshotInfo( - instance.snapshotId(), + instance.snapshot(), instance.indices(), instance.dataStreams(), instance.featureStates(), @@ -207,7 +208,7 @@ static SnapshotInfo mutateSnapshotInfo(SnapshotInfo instance) { ); case 4: return new SnapshotInfo( - instance.snapshotId(), + instance.snapshot(), instance.indices(), instance.dataStreams(), instance.featureStates(), @@ -224,7 +225,7 @@ static SnapshotInfo mutateSnapshotInfo(SnapshotInfo instance) { final int totalShards = randomValueOtherThan(instance.totalShards(), () -> randomIntBetween(0, 100)); final List shardFailures = randomShardFailures(randomIntBetween(0, totalShards)); return new SnapshotInfo( - instance.snapshotId(), + instance.snapshot(), instance.indices(), instance.dataStreams(), instance.featureStates(), @@ -239,7 +240,7 @@ static SnapshotInfo mutateSnapshotInfo(SnapshotInfo instance) { ); case 6: return new SnapshotInfo( - instance.snapshotId(), + instance.snapshot(), instance.indices(), instance.dataStreams(), instance.featureStates(), @@ -254,7 +255,7 @@ static SnapshotInfo mutateSnapshotInfo(SnapshotInfo instance) { ); case 7: return new SnapshotInfo( - instance.snapshotId(), + instance.snapshot(), instance.indices(), instance.dataStreams(), instance.featureStates(), @@ -273,7 +274,7 @@ static SnapshotInfo mutateSnapshotInfo(SnapshotInfo instance) { () -> Arrays.asList(randomArray(0, 10, String[]::new, () -> randomAlphaOfLengthBetween(2, 20))) ); return new SnapshotInfo( - instance.snapshotId(), + instance.snapshot(), instance.indices(), dataStreams, instance.featureStates(), @@ -288,7 +289,7 @@ static SnapshotInfo mutateSnapshotInfo(SnapshotInfo instance) { ); case 9: return new SnapshotInfo( - instance.snapshotId(), + instance.snapshot(), instance.indices(), instance.dataStreams(), randomValueOtherThan(instance.featureStates(), SnapshotInfoTestUtils::randomSnapshotFeatureInfos), @@ -303,7 +304,7 @@ static SnapshotInfo mutateSnapshotInfo(SnapshotInfo instance) { ); case 10: return new SnapshotInfo( - instance.snapshotId(), + instance.snapshot(), instance.indices(), instance.dataStreams(), instance.featureStates(), diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java index 0419a43cc8c37..10531fea17fc6 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java @@ -351,13 +351,20 @@ public void testSuccessfulSnapshotAndRestore() { Collection snapshotIds = getRepositoryData(repository).getSnapshotIds(); assertThat(snapshotIds, hasSize(1)); - final SnapshotInfo snapshotInfo = repository.getSnapshotInfo(snapshotIds.iterator().next()); + final SnapshotInfo snapshotInfo = getSnapshotInfo(repository, snapshotIds.iterator().next()); assertEquals(SnapshotState.SUCCESS, snapshotInfo.state()); assertThat(snapshotInfo.indices(), containsInAnyOrder(index)); assertEquals(shards, snapshotInfo.successfulShards()); assertEquals(0, snapshotInfo.failedShards()); } + private SnapshotInfo getSnapshotInfo(Repository repository, SnapshotId snapshotId) { + final StepListener listener = new StepListener<>(); + repository.getSnapshotInfo(snapshotId, listener); + deterministicTaskQueue.runAllRunnableTasks(); + return listener.result(); + } + public void testSnapshotWithNodeDisconnects() { final int dataNodes = randomIntBetween(2, 10); final int masterNodes = randomFrom(1, 3, 5); @@ -549,7 +556,7 @@ public void clusterChanged(ClusterChangedEvent event) { Collection snapshotIds = getRepositoryData(repository).getSnapshotIds(); assertThat(snapshotIds, hasSize(1)); - final SnapshotInfo snapshotInfo = repository.getSnapshotInfo(snapshotIds.iterator().next()); + final SnapshotInfo snapshotInfo = getSnapshotInfo(repository, snapshotIds.iterator().next()); assertEquals(SnapshotState.SUCCESS, snapshotInfo.state()); assertThat(snapshotInfo.indices(), containsInAnyOrder(index)); assertEquals(shards, snapshotInfo.successfulShards()); @@ -623,7 +630,7 @@ public void testConcurrentSnapshotCreateAndDeleteOther() { assertThat(snapshotIds, hasSize(2)); for (SnapshotId snapshotId : snapshotIds) { - final SnapshotInfo snapshotInfo = repository.getSnapshotInfo(snapshotId); + final SnapshotInfo snapshotInfo = getSnapshotInfo(repository, snapshotId); assertEquals(SnapshotState.SUCCESS, snapshotInfo.state()); assertThat(snapshotInfo.indices(), containsInAnyOrder(index)); assertEquals(shards, snapshotInfo.successfulShards()); @@ -773,7 +780,7 @@ public void testConcurrentSnapshotRestoreAndDeleteOther() { assertThat(snapshotIds, contains(createOtherSnapshotResponseStepListener.result().getSnapshotInfo().snapshotId())); for (SnapshotId snapshotId : snapshotIds) { - final SnapshotInfo snapshotInfo = repository.getSnapshotInfo(snapshotId); + final SnapshotInfo snapshotInfo = getSnapshotInfo(repository, snapshotId); assertEquals(SnapshotState.SUCCESS, snapshotInfo.state()); assertThat(snapshotInfo.indices(), containsInAnyOrder(index)); assertEquals(shards, snapshotInfo.successfulShards()); @@ -870,7 +877,7 @@ public void onFailure(Exception e) { Collection snapshotIds = repositoryData.getSnapshotIds(); assertThat(snapshotIds, hasSize(1)); - final SnapshotInfo snapshotInfo = repository.getSnapshotInfo(snapshotIds.iterator().next()); + final SnapshotInfo snapshotInfo = getSnapshotInfo(repository, snapshotIds.iterator().next()); if (partialSnapshot) { assertThat(snapshotInfo.state(), either(is(SnapshotState.SUCCESS)).or(is(SnapshotState.PARTIAL))); // Single shard for each index so we either get all indices or all except for the deleted index @@ -1148,7 +1155,7 @@ public void testSuccessfulSnapshotWithConcurrentDynamicMappingUpdates() { Collection snapshotIds = getRepositoryData(repository).getSnapshotIds(); assertThat(snapshotIds, hasSize(1)); - final SnapshotInfo snapshotInfo = repository.getSnapshotInfo(snapshotIds.iterator().next()); + final SnapshotInfo snapshotInfo = getSnapshotInfo(repository, snapshotIds.iterator().next()); assertEquals(SnapshotState.SUCCESS, snapshotInfo.state()); assertThat(snapshotInfo.indices(), containsInAnyOrder(index)); assertEquals(shards, snapshotInfo.successfulShards()); @@ -1216,7 +1223,7 @@ public void testRunConcurrentSnapshots() { assertThat(snapshotIds, hasSize(snapshotNames.size())); for (SnapshotId snapshotId : snapshotIds) { - final SnapshotInfo snapshotInfo = repository.getSnapshotInfo(snapshotId); + final SnapshotInfo snapshotInfo = getSnapshotInfo(repository, snapshotId); assertEquals(SnapshotState.SUCCESS, snapshotInfo.state()); assertThat(snapshotInfo.indices(), containsInAnyOrder(index)); assertEquals(shards, snapshotInfo.successfulShards()); diff --git a/server/src/test/java/org/elasticsearch/threadpool/ScalingThreadPoolTests.java b/server/src/test/java/org/elasticsearch/threadpool/ScalingThreadPoolTests.java index 9edb1da25db93..2e4a11bbc166e 100644 --- a/server/src/test/java/org/elasticsearch/threadpool/ScalingThreadPoolTests.java +++ b/server/src/test/java/org/elasticsearch/threadpool/ScalingThreadPoolTests.java @@ -63,7 +63,8 @@ public void testScalingThreadPoolConfiguration() throws InterruptedException { keepAlive = randomIntBetween(1, 300); builder.put("thread_pool." + threadPoolName + ".keep_alive", keepAlive + "s"); } else { - keepAlive = "generic".equals(threadPoolName) ? 30 : 300; // the defaults + keepAlive = "generic".equals(threadPoolName) || ThreadPool.Names.SNAPSHOT_META.equals(threadPoolName) + ? 30 : 300; // the defaults } runScalingThreadPoolTest(builder.build(), (clusterSettings, threadPool) -> { @@ -101,6 +102,7 @@ private int expectedSize(final String threadPoolName, final int numberOfProcesso sizes.put(ThreadPool.Names.REFRESH, ThreadPool::halfAllocatedProcessorsMaxTen); sizes.put(ThreadPool.Names.WARMER, ThreadPool::halfAllocatedProcessorsMaxFive); sizes.put(ThreadPool.Names.SNAPSHOT, ThreadPool::halfAllocatedProcessorsMaxFive); + sizes.put(ThreadPool.Names.SNAPSHOT_META, n -> Math.min(n * 3, 50)); sizes.put(ThreadPool.Names.FETCH_SHARD_STARTED, ThreadPool::twiceAllocatedProcessors); sizes.put(ThreadPool.Names.FETCH_SHARD_STORE, ThreadPool::twiceAllocatedProcessors); return sizes.get(threadPoolName).apply(numberOfProcessors); diff --git a/server/src/test/java/org/elasticsearch/threadpool/UpdateThreadPoolSettingsTests.java b/server/src/test/java/org/elasticsearch/threadpool/UpdateThreadPoolSettingsTests.java index 2d9ef32333151..9026aba4fbf98 100644 --- a/server/src/test/java/org/elasticsearch/threadpool/UpdateThreadPoolSettingsTests.java +++ b/server/src/test/java/org/elasticsearch/threadpool/UpdateThreadPoolSettingsTests.java @@ -125,7 +125,7 @@ public void testScalingExecutorType() throws InterruptedException { final int expectedMinimum = "generic".equals(threadPoolName) ? 4 : 1; assertThat(info(threadPool, threadPoolName).getMin(), equalTo(expectedMinimum)); assertThat(info(threadPool, threadPoolName).getMax(), equalTo(10)); - final long expectedKeepAlive = "generic".equals(threadPoolName) ? 30 : 300; + final long expectedKeepAlive = "generic".equals(threadPoolName) || Names.SNAPSHOT_META.equals(threadPoolName) ? 30 : 300; assertThat(info(threadPool, threadPoolName).getKeepAlive().seconds(), equalTo(expectedKeepAlive)); assertEquals(info(threadPool, threadPoolName).getThreadPoolType(), ThreadPool.ThreadPoolType.SCALING); assertThat(threadPool.executor(threadPoolName), instanceOf(EsThreadPoolExecutor.class)); diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java b/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java index 843b9eac2bf67..596c4b6889a9b 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java @@ -17,14 +17,15 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; +import org.elasticsearch.repositories.GetSnapshotInfoContext; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.IndexMetaDataGenerations; import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.RepositoryData; import org.elasticsearch.repositories.RepositoryShardId; import org.elasticsearch.repositories.ShardGenerations; -import org.elasticsearch.repositories.SnapshotShardContext; import org.elasticsearch.repositories.ShardSnapshotResult; +import org.elasticsearch.repositories.SnapshotShardContext; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.snapshots.SnapshotInfo; @@ -64,8 +65,8 @@ public RepositoryMetadata getMetadata() { } @Override - public SnapshotInfo getSnapshotInfo(SnapshotId snapshotId) { - return null; + public void getSnapshotInfo(GetSnapshotInfoContext context) { + throw new UnsupportedOperationException(); } @Override diff --git a/test/framework/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreTestUtil.java b/test/framework/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreTestUtil.java index 98fd2e5449d04..ef0377afc34d2 100644 --- a/test/framework/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreTestUtil.java +++ b/test/framework/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreTestUtil.java @@ -9,6 +9,7 @@ import org.apache.lucene.util.SameThreadExecutorService; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.cluster.ClusterChangedEvent; @@ -34,6 +35,7 @@ import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.repositories.GetSnapshotInfoContext; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.RepositoryData; import org.elasticsearch.repositories.ShardGenerations; @@ -46,6 +48,7 @@ import java.io.IOException; import java.io.InputStream; import java.nio.file.NoSuchFileException; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -114,9 +117,32 @@ public static PlainActionFuture assertConsistencyAsync(BlobStore repositoryData = RepositoryData.snapshotsFromXContent(parser, latestGen, false); } assertIndexUUIDs(repository, repositoryData); - assertSnapshotUUIDs(repository, repositoryData); - assertShardIndexGenerations(blobContainer, repositoryData.shardGenerations()); - listener.onResponse(null); + assertSnapshotUUIDs(repository, repositoryData, new ActionListener() { + @Override + public void onResponse(AssertionError assertionError) { + if (assertionError == null) { + try { + try { + assertShardIndexGenerations(blobContainer, repositoryData.shardGenerations()); + } catch (AssertionError e) { + listener.onResponse(e); + return; + } + } catch (Exception e) { + onFailure(e); + return; + } + listener.onResponse(null); + } else { + listener.onResponse(assertionError); + } + } + + @Override + public void onFailure(Exception e) { + listener.onResponse(new AssertionError(e)); + } + }); } catch (AssertionError e) { listener.onResponse(e); } @@ -183,7 +209,8 @@ private static void assertIndexUUIDs(BlobStoreRepository repository, RepositoryD } } - private static void assertSnapshotUUIDs(BlobStoreRepository repository, RepositoryData repositoryData) throws IOException { + private static void assertSnapshotUUIDs(BlobStoreRepository repository, RepositoryData repositoryData, + ActionListener listener) throws IOException { final BlobContainer repoRoot = repository.blobContainer(); final Collection snapshotIds = repositoryData.getSnapshotIds(); final List expectedSnapshotUUIDs = snapshotIds.stream().map(SnapshotId::getUUID).collect(Collectors.toList()); @@ -201,11 +228,50 @@ private static void assertSnapshotUUIDs(BlobStoreRepository repository, Reposito } else { indices = indicesContainer.children(); } + if (snapshotIds.isEmpty()) { + listener.onResponse(null); + return; + } + // Assert that for each snapshot, the relevant metadata was written to index and shard folders + final List snapshotInfos = Collections.synchronizedList(new ArrayList<>()); + repository.getSnapshotInfo( + new GetSnapshotInfoContext( + org.elasticsearch.core.List.copyOf(snapshotIds), + true, + () -> false, + (ctx, sni) -> snapshotInfos.add(sni), + new ActionListener() { + @Override + public void onResponse(Void unused) { + try { + assertSnapshotInfosConsistency(repository, repositoryData, indices, snapshotInfos); + } catch (Exception e) { + listener.onResponse(new AssertionError(e)); + return; + } catch (AssertionError e) { + listener.onResponse(e); + return; + } + listener.onResponse(null); + } + + @Override + public void onFailure(Exception e) { + listener.onResponse(new AssertionError(e)); + } + } + ) + ); + } + + private static void assertSnapshotInfosConsistency(BlobStoreRepository repository, + RepositoryData repositoryData, + Map indices, + List snapshotInfos) throws IOException { final Map maxShardCountsExpected = new HashMap<>(); final Map maxShardCountsSeen = new HashMap<>(); - // Assert that for each snapshot, the relevant metadata was written to index and shard folders - for (SnapshotId snapshotId: snapshotIds) { - final SnapshotInfo snapshotInfo = repository.getSnapshotInfo(snapshotId); + for (SnapshotInfo snapshotInfo: snapshotInfos) { + final SnapshotId snapshotId = snapshotInfo.snapshotId(); for (String index : snapshotInfo.indices()) { final IndexId indexId = repositoryData.resolveIndexId(index); assertThat(indices, hasKey(indexId.getId())); diff --git a/test/framework/src/main/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java index a75eeab682a49..4818945432596 100644 --- a/test/framework/src/main/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java @@ -13,6 +13,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse; +import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsRequest; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.support.GroupedActionListener; @@ -34,6 +35,7 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.core.Nullable; import org.elasticsearch.core.TimeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.xcontent.DeprecationHandler; @@ -51,6 +53,7 @@ import org.elasticsearch.repositories.blobstore.BlobStoreTestUtil; import org.elasticsearch.repositories.blobstore.ChecksumBlobStoreFormat; import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.snapshots.mockstore.MockRepository; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.VersionUtils; @@ -75,6 +78,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; import java.util.function.Function; import java.util.function.Predicate; import java.util.stream.StreamSupport; @@ -84,12 +88,15 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; public abstract class AbstractSnapshotIntegTestCase extends ESIntegTestCase { - private static final String OLD_VERSION_SNAPSHOT_PREFIX = "old-version-snapshot-"; + public static final String RANDOM_SNAPSHOT_NAME_PREFIX = "snap-"; + + public static final String OLD_VERSION_SNAPSHOT_PREFIX = "old-version-snapshot-"; // Large snapshot pool settings to set up nodes for tests involving multiple repositories that need to have enough // threads so that blocking some threads on one repository doesn't block other repositories from doing work @@ -364,6 +371,7 @@ protected String initWithSnapshotVersion(String repoName, Path repoPath, Version downgradedRepoData.snapshotsToXContent(XContentFactory.jsonBuilder(), version))), StandardOpenOption.TRUNCATE_EXISTING); final SnapshotInfo downgradedSnapshotInfo = SnapshotInfo.fromXContentInternal( + repoName, JsonXContent.jsonXContent.createParser( NamedXContentRegistry.EMPTY, DeprecationHandler.THROW_UNSUPPORTED_OPERATION, @@ -451,7 +459,7 @@ protected void addBwCFailedSnapshot(String repoName, String snapshotName, Map adding old version FAILED snapshot [{}] to repository [{}]", snapshotId, repoName); final SnapshotInfo snapshotInfo = new SnapshotInfo( - snapshotId, + new Snapshot(repoName, snapshotId), Collections.emptyList(), Collections.emptyList(), Collections.emptyList(), @@ -484,15 +492,24 @@ protected void awaitNoMoreRunningOperations() throws Exception { protected void awaitNoMoreRunningOperations(String viaNode) throws Exception { logger.info("--> verify no more operations in the cluster state"); - awaitClusterState(viaNode, state -> state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY).entries().isEmpty() && - state.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY).hasDeletionsInProgress() == false); + awaitClusterState( + logger, + viaNode, + state -> state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY).entries().isEmpty() + && state.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY) + .hasDeletionsInProgress() == false + ); } protected void awaitClusterState(Predicate statePredicate) throws Exception { - awaitClusterState(internalCluster().getMasterName(), statePredicate); + awaitClusterState(logger, internalCluster().getMasterName(), statePredicate); } - protected void awaitClusterState(String viaNode, Predicate statePredicate) throws Exception { + public static void awaitClusterState(Logger logger, Predicate statePredicate) throws Exception { + awaitClusterState(logger, internalCluster().getMasterName(), statePredicate); + } + + public static void awaitClusterState(Logger logger, String viaNode, Predicate statePredicate) throws Exception { final ClusterService clusterService = internalCluster().getInstance(ClusterService.class, viaNode); final ThreadPool threadPool = internalCluster().getInstance(ThreadPool.class, viaNode); final ClusterStateObserver observer = new ClusterStateObserver(clusterService, logger, threadPool.getThreadContext()); @@ -531,6 +548,13 @@ protected ActionFuture startFullSnapshot(String repoName } protected ActionFuture startFullSnapshot(String repoName, String snapshotName, boolean partial) { + return startFullSnapshot(logger, repoName, snapshotName, partial); + } + + public static ActionFuture startFullSnapshot(Logger logger, + String repoName, + String snapshotName, + boolean partial) { logger.info("--> creating full snapshot [{}] to repo [{}]", snapshotName, repoName); return clusterAdmin().prepareCreateSnapshot(repoName, snapshotName).setWaitForCompletion(true) .setPartial(partial).execute(); @@ -542,14 +566,24 @@ protected void awaitNumberOfSnapshotsInProgress(int count) throws Exception { state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY).entries().size() == count); } + public static void awaitNumberOfSnapshotsInProgress(Logger logger, int count) throws Exception { + logger.info("--> wait for [{}] snapshots to show up in the cluster state", count); + awaitClusterState(logger, state -> + state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY).entries().size() == count); + } + protected SnapshotInfo assertSuccessful(ActionFuture future) throws Exception { + return assertSuccessful(logger, future); + } + + public static SnapshotInfo assertSuccessful(Logger logger, ActionFuture future) throws Exception { logger.info("--> wait for snapshot to finish"); final SnapshotInfo snapshotInfo = future.get().getSnapshotInfo(); assertThat(snapshotInfo.state(), is(SnapshotState.SUCCESS)); return snapshotInfo; } - private static final Settings SINGLE_SHARD_NO_REPLICA = indexSettingsNoReplicas(1).build(); + public static final Settings SINGLE_SHARD_NO_REPLICA = indexSettingsNoReplicas(1).build(); protected void createIndexWithContent(String indexName) { createIndexWithContent(indexName, SINGLE_SHARD_NO_REPLICA); @@ -617,7 +651,7 @@ public static List createNSnapshots(Logger logger, String repoName, int final PlainActionFuture> allSnapshotsDone = PlainActionFuture.newFuture(); final ActionListener snapshotsListener = new GroupedActionListener<>(allSnapshotsDone, count); final List snapshotNames = new ArrayList<>(count); - final String prefix = "snap-" + UUIDs.randomBase64UUID(random()).toLowerCase(Locale.ROOT) + "-"; + final String prefix = RANDOM_SNAPSHOT_NAME_PREFIX + UUIDs.randomBase64UUID(random()).toLowerCase(Locale.ROOT) + "-"; for (int i = 0; i < count; i++) { final String snapshot = prefix + i; snapshotNames.add(snapshot); @@ -640,4 +674,39 @@ public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IO } }); } + + public static void assertSnapshotListSorted(List snapshotInfos, @Nullable GetSnapshotsRequest.SortBy sort, + SortOrder sortOrder) { + final BiConsumer assertion; + if (sort == null) { + assertion = (s1, s2) -> assertThat(s2, greaterThanOrEqualTo(s1)); + } else { + switch (sort) { + case START_TIME: + assertion = (s1, s2) -> assertThat(s2.startTime(), greaterThanOrEqualTo(s1.startTime())); + break; + case NAME: + assertion = (s1, s2) -> assertThat(s2.snapshotId().getName(), greaterThanOrEqualTo(s1.snapshotId().getName())); + break; + case DURATION: + assertion = + (s1, s2) -> assertThat(s2.endTime() - s2.startTime(), greaterThanOrEqualTo(s1.endTime() - s1.startTime())); + break; + case INDICES: + assertion = (s1, s2) -> assertThat(s2.indices().size(), greaterThanOrEqualTo(s1.indices().size())); + break; + default: + throw new AssertionError("unknown sort column [" + sort + "]"); + } + } + final BiConsumer orderAssertion; + if (sortOrder == SortOrder.ASC) { + orderAssertion = assertion; + } else { + orderAssertion = (s1, s2) -> assertion.accept(s2, s1); + } + for (int i = 0; i < snapshotInfos.size() - 1; i++) { + orderAssertion.accept(snapshotInfos.get(i), snapshotInfos.get(i + 1)); + } + } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 2ef7a29ada0a3..5d83ba0f3c888 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -1349,6 +1349,16 @@ protected final IndexResponse index(String index, String type, String id, Object return client().prepareIndex(index, type, id).setSource(source).execute().actionGet(); } + /** + * Syntactic sugar for: + *
+     *   return client().prepareIndex(index).setId(id).setSource(source).execute().actionGet();
+     * 
+ */ + protected final IndexResponse indexDoc(String index, String id, Object... source) { + return client().prepareIndex(index, "_doc").setId(id).setSource(source).execute().actionGet(); + } + /** * Syntactic sugar for: *
diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java
index d46bf3b6037a0..5fa281acb1f2b 100644
--- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java
+++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java
@@ -63,15 +63,17 @@
 import org.elasticsearch.indices.recovery.MultiChunkTransfer;
 import org.elasticsearch.indices.recovery.MultiFileWriter;
 import org.elasticsearch.indices.recovery.RecoveryState;
+import org.elasticsearch.repositories.GetSnapshotInfoContext;
 import org.elasticsearch.repositories.IndexId;
 import org.elasticsearch.repositories.IndexMetaDataGenerations;
 import org.elasticsearch.repositories.Repository;
 import org.elasticsearch.repositories.RepositoryData;
 import org.elasticsearch.repositories.RepositoryShardId;
 import org.elasticsearch.repositories.ShardGenerations;
-import org.elasticsearch.repositories.SnapshotShardContext;
 import org.elasticsearch.repositories.ShardSnapshotResult;
+import org.elasticsearch.repositories.SnapshotShardContext;
 import org.elasticsearch.repositories.blobstore.FileRestoreContext;
+import org.elasticsearch.snapshots.Snapshot;
 import org.elasticsearch.snapshots.SnapshotId;
 import org.elasticsearch.snapshots.SnapshotInfo;
 import org.elasticsearch.snapshots.SnapshotState;
@@ -173,8 +175,10 @@ private Client getRemoteClusterClient() {
     }
 
     @Override
-    public SnapshotInfo getSnapshotInfo(SnapshotId snapshotId) {
-        assert SNAPSHOT_ID.equals(snapshotId) : "RemoteClusterRepository only supports " + SNAPSHOT_ID + " as the SnapshotId";
+    public void getSnapshotInfo(GetSnapshotInfoContext context) {
+        final List snapshotIds = context.snapshotIds();
+        assert snapshotIds.size() == 1 && SNAPSHOT_ID.equals(snapshotIds.iterator().next())
+                : "RemoteClusterRepository only supports " + SNAPSHOT_ID + " as the SnapshotId but saw " + snapshotIds;
         Client remoteClient = getRemoteClusterClient();
         ClusterStateResponse response = remoteClient.admin().cluster().prepareState().clear().setMetadata(true).setNodes(true)
             .get(ccrSettings.getRecoveryActionTimeout());
@@ -183,9 +187,17 @@ public SnapshotInfo getSnapshotInfo(SnapshotId snapshotId) {
         ArrayList indices = new ArrayList<>(indicesMap.size());
         indicesMap.keysIt().forEachRemaining(indices::add);
 
-        return new SnapshotInfo(snapshotId, indices, new ArrayList<>(metadata.dataStreams().keySet()), Collections.emptyList(),
-            response.getState().getNodes().getMaxNodeVersion(), SnapshotState.SUCCESS
-        );
+        // fork to the snapshot meta pool because the context expects to run on it and asserts that it does
+        threadPool.executor(ThreadPool.Names.SNAPSHOT_META).execute(() -> context.onResponse(
+            new SnapshotInfo(
+                new Snapshot(this.metadata.name(), SNAPSHOT_ID),
+                indices,
+                new ArrayList<>(metadata.dataStreams().keySet()),
+                Collections.emptyList(),
+                response.getState().getNodes().getMaxNodeVersion(),
+                SnapshotState.SUCCESS
+            )
+        ));
     }
 
     @Override
diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/sourceonly/SourceOnlySnapshotShardTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/sourceonly/SourceOnlySnapshotShardTests.java
index 535773b303a70..94a2b4b75ac5f 100644
--- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/sourceonly/SourceOnlySnapshotShardTests.java
+++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/sourceonly/SourceOnlySnapshotShardTests.java
@@ -235,7 +235,7 @@ public void testRestoreMinmal() throws IOException {
                     ESBlobStoreRepositoryIntegTestCase.getRepositoryData(repository).getGenId(),
                     Metadata.builder().put(shard.indexSettings().getIndexMetadata(), false).build(),
                     new SnapshotInfo(
-                        snapshotId,
+                        new Snapshot(repository.getMetadata().name(), snapshotId),
                         shardGenerations.indices().stream()
                                 .map(IndexId::getName).collect(Collectors.toList()),
                         Collections.emptyList(),
diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/slm/SnapshotRetentionConfigurationTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/slm/SnapshotRetentionConfigurationTests.java
index 7fd8468d3129e..d432e23452d49 100644
--- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/slm/SnapshotRetentionConfigurationTests.java
+++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/slm/SnapshotRetentionConfigurationTests.java
@@ -9,6 +9,7 @@
 
 import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.index.shard.ShardId;
+import org.elasticsearch.snapshots.Snapshot;
 import org.elasticsearch.snapshots.SnapshotId;
 import org.elasticsearch.snapshots.SnapshotInfo;
 import org.elasticsearch.snapshots.SnapshotShardFailure;
@@ -286,7 +287,7 @@ private SnapshotInfo makeInfo(long startTime) {
         meta.put(SnapshotLifecyclePolicy.POLICY_ID_METADATA_FIELD, REPO);
         final int totalShards = between(1,20);
         SnapshotInfo snapInfo = new SnapshotInfo(
-            new SnapshotId("snap-" + randomAlphaOfLength(3), "uuid"),
+            new Snapshot(REPO, new SnapshotId("snap-" + randomAlphaOfLength(3), "uuid")),
             Collections.singletonList("foo"),
             Collections.singletonList("bar"),
             Collections.emptyList(),
@@ -322,7 +323,7 @@ private SnapshotInfo makeFailureInfo(long startTime) {
         }
         assert failureCount == failures.size();
         SnapshotInfo snapInfo = new SnapshotInfo(
-            new SnapshotId("snap-fail-" + randomAlphaOfLength(3), "uuid-fail"),
+            new Snapshot(REPO, new SnapshotId("snap-fail-" + randomAlphaOfLength(3), "uuid-fail")),
             Collections.singletonList("foo-fail"),
             Collections.singletonList("bar-fail"),
             Collections.emptyList(),
@@ -350,7 +351,7 @@ private SnapshotInfo makePartialInfo(long startTime) {
         }
         assert failureCount == failures.size();
         SnapshotInfo snapInfo = new SnapshotInfo(
-            new SnapshotId("snap-fail-" + randomAlphaOfLength(3), "uuid-fail"),
+            new Snapshot(REPO, new SnapshotId("snap-fail-" + randomAlphaOfLength(3), "uuid-fail")),
             Collections.singletonList("foo-fail"),
             Collections.singletonList("bar-fail"),
             Collections.emptyList(),
diff --git a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleTaskTests.java b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleTaskTests.java
index 66e86b628fc56..baf7ed3ccdf75 100644
--- a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleTaskTests.java
+++ b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotLifecycleTaskTests.java
@@ -25,6 +25,7 @@
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.xcontent.json.JsonXContent;
 import org.elasticsearch.index.shard.ShardId;
+import org.elasticsearch.snapshots.Snapshot;
 import org.elasticsearch.snapshots.SnapshotId;
 import org.elasticsearch.snapshots.SnapshotInfo;
 import org.elasticsearch.snapshots.SnapshotShardFailure;
@@ -242,7 +243,7 @@ public void testPartialFailureSnapshot() throws Exception {
                      long endTime = randomLongBetween(startTime, Long.MAX_VALUE);
                      return new CreateSnapshotResponse(
                          new SnapshotInfo(
-                             new SnapshotId(req.snapshot(), "uuid"),
+                             new Snapshot(req.repository(), new SnapshotId(req.snapshot(), "uuid")),
                              Arrays.asList(req.indices()),
                              Collections.emptyList(),
                              Collections.emptyList(), "snapshot started", endTime, 3, Collections.singletonList(
diff --git a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotRetentionTaskTests.java b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotRetentionTaskTests.java
index a811bed98d6e7..d95bff33de5de 100644
--- a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotRetentionTaskTests.java
+++ b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotRetentionTaskTests.java
@@ -22,6 +22,7 @@
 import org.elasticsearch.cluster.metadata.Metadata;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.core.TimeValue;
+import org.elasticsearch.snapshots.Snapshot;
 import org.elasticsearch.snapshots.SnapshotId;
 import org.elasticsearch.snapshots.SnapshotInfo;
 import org.elasticsearch.test.ClusterServiceUtils;
@@ -100,18 +101,19 @@ public void testGetAllPoliciesWithRetentionEnabled() {
     }
 
     public void testSnapshotEligibleForDeletion() {
+        final String repoName = "repo";
         SnapshotLifecyclePolicy policy = new SnapshotLifecyclePolicy("policy", "snap", "1 * * * * ?",
-            "repo", null, new SnapshotRetentionConfiguration(TimeValue.timeValueDays(30), null, null));
+            repoName, null, new SnapshotRetentionConfiguration(TimeValue.timeValueDays(30), null, null));
         SnapshotLifecyclePolicy policyWithNoRetention = new SnapshotLifecyclePolicy("policy", "snap", "1 * * * * ?",
-            "repo", null, randomBoolean() ? null : SnapshotRetentionConfiguration.EMPTY);
+            repoName, null, randomBoolean() ? null : SnapshotRetentionConfiguration.EMPTY);
         Map policyMap = Collections.singletonMap("policy", policy);
         Map policyWithNoRetentionMap = Collections.singletonMap("policy", policyWithNoRetention);
         Function>> mkInfos = i ->
-            Collections.singletonMap("repo", Collections.singletonList(i));
+            Collections.singletonMap(repoName, Collections.singletonList(i));
 
         // Test when user metadata is null
         SnapshotInfo info = new SnapshotInfo(
-            new SnapshotId("name", "uuid"),
+            new Snapshot(repoName, new SnapshotId("name", "uuid")),
             Collections.singletonList("index"),
             Collections.emptyList(),
             Collections.emptyList(),
@@ -127,7 +129,7 @@ public void testSnapshotEligibleForDeletion() {
 
         // Test when no retention is configured
         info = new SnapshotInfo(
-            new SnapshotId("name", "uuid"),
+            new Snapshot(repoName, new SnapshotId("name", "uuid")),
             Collections.singletonList("index"),
             Collections.emptyList(),
             Collections.emptyList(),
@@ -143,7 +145,7 @@ public void testSnapshotEligibleForDeletion() {
 
         // Test when user metadata is a map that doesn't contain "policy"
         info = new SnapshotInfo(
-            new SnapshotId("name", "uuid"),
+            new Snapshot(repoName, new SnapshotId("name", "uuid")),
             Collections.singletonList("index"),
             Collections.emptyList(),
             Collections.emptyList(),
@@ -159,7 +161,7 @@ public void testSnapshotEligibleForDeletion() {
 
         // Test with an ancient snapshot that should be expunged
         info = new SnapshotInfo(
-            new SnapshotId("name", "uuid"),
+            new Snapshot(repoName, new SnapshotId("name", "uuid")),
             Collections.singletonList("index"),
             Collections.emptyList(),
             Collections.emptyList(),
@@ -176,7 +178,7 @@ public void testSnapshotEligibleForDeletion() {
         // Test with a snapshot that's start date is old enough to be expunged (but the finish date is not)
         long time = System.currentTimeMillis() - TimeValue.timeValueDays(30).millis() - 1;
         info = new SnapshotInfo(
-            new SnapshotId("name", "uuid"),
+            new Snapshot(repoName, new SnapshotId("name", "uuid")),
             Collections.singletonList("index"),
             Collections.emptyList(),
             Collections.emptyList(),
@@ -192,7 +194,7 @@ public void testSnapshotEligibleForDeletion() {
 
         // Test with a fresh snapshot that should not be expunged
         info = new SnapshotInfo(
-            new SnapshotId("name", "uuid"),
+            new Snapshot(repoName, new SnapshotId("name", "uuid")),
             Collections.singletonList("index"),
             Collections.emptyList(),
             Collections.emptyList(),
@@ -228,11 +230,12 @@ private void retentionTaskTest(final boolean deletionSuccess) throws Exception {
             ClusterState state = createState(policy);
             ClusterServiceUtils.setState(clusterService, state);
 
-            final SnapshotInfo eligibleSnapshot = new SnapshotInfo(new SnapshotId("name", "uuid"), Collections.singletonList("index"),
+            final SnapshotInfo eligibleSnapshot = new SnapshotInfo(
+                    new Snapshot(repoId, new SnapshotId("name", "uuid")), Collections.singletonList("index"),
                 Collections.emptyList(), Collections.emptyList(), null, 1L, 1, Collections.emptyList(), true,
                 Collections.singletonMap("policy", policyId), 0L, Collections.emptyMap());
             final SnapshotInfo ineligibleSnapshot = new SnapshotInfo(
-                new SnapshotId("name2", "uuid2"),
+                new Snapshot(repoId, new SnapshotId("name2", "uuid2")),
                 Collections.singletonList("index"),
                 Collections.emptyList(),
                 Collections.emptyList(),
@@ -314,7 +317,7 @@ public void testErrStillRunsFailureHandlerWhenRetrieving() throws Exception {
                  void doExecute(ActionType action, Request request, ActionListener listener) {
                      if (request instanceof GetSnapshotsRequest) {
                          logger.info("--> called");
-                         listener.onResponse((Response) new GetSnapshotsResponse(Collections.emptyList()));
+                         listener.onResponse((Response) new GetSnapshotsResponse(Collections.emptyList(), Collections.emptyMap(), null));
                      } else {
                          super.doExecute(action, request, listener);
                      }
diff --git a/x-pack/plugin/repository-encrypted/src/main/java/org/elasticsearch/repositories/encrypted/EncryptedRepository.java b/x-pack/plugin/repository-encrypted/src/main/java/org/elasticsearch/repositories/encrypted/EncryptedRepository.java
index 4928aec3ca393..d4ed2830a4f5c 100644
--- a/x-pack/plugin/repository-encrypted/src/main/java/org/elasticsearch/repositories/encrypted/EncryptedRepository.java
+++ b/x-pack/plugin/repository-encrypted/src/main/java/org/elasticsearch/repositories/encrypted/EncryptedRepository.java
@@ -196,8 +196,7 @@ public Map adaptUserMetadata(Map userMetadata) {
             localRepositoryPasswordSalt,
             localRepositoryPasswordHash
         );
-        // do not wrap in Map.of; we have to be able to modify the map (remove the added entries) when finalizing the snapshot
-        return snapshotUserMetadata;
+        return org.elasticsearch.core.Map.copyOf(snapshotUserMetadata);
     }
 
     @Override