Browse Source

Split Tests out of SharedClusterSnapshotRestoreIT (#63130)

Splitting some tests out of this class that has become a catch-all
for random snapshot related tests into either existing suits that fit
better for these tests or one of two new suits to prevent timeouts
in extreme cases (e.g. `WindowsFS` + many nodes + multiple data paths per node).
No other changes to tests were made whatsoever.

Closes #61541
Armin Braun 5 years ago
parent
commit
ec3388ddbe

+ 289 - 0
server/src/internalClusterTest/java/org/elasticsearch/snapshots/CorruptedBlobStoreRepositoryIT.java

@@ -21,6 +21,9 @@ package org.elasticsearch.snapshots;
 import org.elasticsearch.Version;
 import org.elasticsearch.action.ActionRunnable;
 import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse;
+import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse;
+import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotsStatusResponse;
+import org.elasticsearch.action.index.IndexRequestBuilder;
 import org.elasticsearch.action.support.PlainActionFuture;
 import org.elasticsearch.client.Client;
 import org.elasticsearch.cluster.ClusterState;
@@ -30,6 +33,7 @@ import org.elasticsearch.common.bytes.BytesReference;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.common.xcontent.XContentFactory;
+import org.elasticsearch.core.internal.io.IOUtils;
 import org.elasticsearch.repositories.IndexId;
 import org.elasticsearch.repositories.IndexMetaDataGenerations;
 import org.elasticsearch.repositories.RepositoriesService;
@@ -41,21 +45,26 @@ import org.elasticsearch.repositories.blobstore.BlobStoreRepository;
 import org.elasticsearch.threadpool.ThreadPool;
 
 import java.io.IOException;
+import java.nio.channels.SeekableByteChannel;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.StandardOpenOption;
 import java.util.Collections;
+import java.util.List;
 import java.util.Locale;
 import java.util.Map;
 import java.util.concurrent.ExecutionException;
 import java.util.function.Function;
 import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
 import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertFileExists;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertRequestBuilderThrows;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.greaterThan;
+import static org.hamcrest.Matchers.hasSize;
 import static org.hamcrest.Matchers.is;
 
 public class CorruptedBlobStoreRepositoryIT extends AbstractSnapshotIntegTestCase {
@@ -347,6 +356,286 @@ public class CorruptedBlobStoreRepositoryIT extends AbstractSnapshotIntegTestCas
         createFullSnapshot(repoName, "snapshot-2");
     }
 
+    /**
+     * Tests that a shard snapshot with a corrupted shard index file can still be used for restore and incremental snapshots.
+     */
+    public void testSnapshotWithCorruptedShardIndexFile() throws Exception {
+        final Client client = client();
+        final Path repo = randomRepoPath();
+        final String indexName = "test-idx";
+        final int nDocs = randomIntBetween(1, 10);
+
+        logger.info("-->  creating index [{}] with [{}] documents in it", indexName, nDocs);
+        assertAcked(prepareCreate(indexName).setSettings(indexSettingsNoReplicas(1)));
+
+        final IndexRequestBuilder[] documents = new IndexRequestBuilder[nDocs];
+        for (int j = 0; j < nDocs; j++) {
+            documents[j] = client.prepareIndex(indexName).setSource("foo", "bar");
+        }
+        indexRandom(true, documents);
+        flushAndRefresh();
+
+        createRepository("test-repo", "fs", repo);
+
+        final String snapshot1 = "test-snap-1";
+        final SnapshotInfo snapshotInfo = createFullSnapshot("test-repo", snapshot1);
+        assertThat(snapshotInfo.indices(), hasSize(1));
+
+        final RepositoryData repositoryData = getRepositoryData("test-repo");
+        final Map<String, IndexId> indexIds = repositoryData.getIndices();
+        assertThat(indexIds.size(), equalTo(1));
+
+        final IndexId corruptedIndex = indexIds.get(indexName);
+        final Path shardIndexFile = repo.resolve("indices")
+                .resolve(corruptedIndex.getId()).resolve("0")
+                .resolve("index-" + repositoryData.shardGenerations().getShardGen(corruptedIndex, 0));
+
+        logger.info("-->  truncating shard index file [{}]", shardIndexFile);
+        try (SeekableByteChannel outChan = Files.newByteChannel(shardIndexFile, StandardOpenOption.WRITE)) {
+            outChan.truncate(randomInt(10));
+        }
+
+        logger.info("-->  verifying snapshot state for [{}]", snapshot1);
+        List<SnapshotInfo> snapshotInfos = clusterAdmin().prepareGetSnapshots("test-repo").get().getSnapshots("test-repo");
+        assertThat(snapshotInfos.size(), equalTo(1));
+        assertThat(snapshotInfos.get(0).state(), equalTo(SnapshotState.SUCCESS));
+        assertThat(snapshotInfos.get(0).snapshotId().getName(), equalTo(snapshot1));
+
+        logger.info("-->  deleting index [{}]", indexName);
+        assertAcked(client().admin().indices().prepareDelete(indexName));
+
+        logger.info("-->  restoring snapshot [{}]", snapshot1);
+        clusterAdmin().prepareRestoreSnapshot("test-repo", snapshot1)
+                .setRestoreGlobalState(randomBoolean())
+                .setWaitForCompletion(true)
+                .get();
+        ensureGreen();
+
+        assertDocCount(indexName, nDocs);
+
+        logger.info("-->  indexing [{}] more documents into [{}]", nDocs, indexName);
+        for (int j = 0; j < nDocs; j++) {
+            documents[j] = client.prepareIndex(indexName).setSource("foo2", "bar2");
+        }
+        indexRandom(true, documents);
+
+        final String snapshot2 = "test-snap-2";
+        logger.info("-->  creating snapshot [{}]", snapshot2);
+        final SnapshotInfo snapshotInfo2 = clusterAdmin().prepareCreateSnapshot("test-repo", snapshot2)
+                .setWaitForCompletion(true)
+                .get()
+                .getSnapshotInfo();
+        assertThat(snapshotInfo2.state(), equalTo(SnapshotState.PARTIAL));
+        assertThat(snapshotInfo2.failedShards(), equalTo(1));
+        assertThat(snapshotInfo2.successfulShards(), equalTo(snapshotInfo.totalShards() - 1));
+        assertThat(snapshotInfo2.indices(), hasSize(1));
+    }
+
+    public void testDeleteSnapshotWithMissingIndexAndShardMetadata() throws Exception {
+        Client client = client();
+
+        Path repo = randomRepoPath();
+        createRepository("test-repo", "fs", repo);
+
+        final String[] indices = {"test-idx-1", "test-idx-2"};
+        createIndex(indices);
+        logger.info("--> indexing some data");
+        indexRandom(true,
+                client().prepareIndex("test-idx-1").setSource("foo", "bar"),
+                client().prepareIndex("test-idx-2").setSource("foo", "bar"));
+
+        logger.info("--> creating snapshot");
+        CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap-1")
+                .setWaitForCompletion(true).setIndices(indices).get();
+        final SnapshotInfo snapshotInfo = createSnapshotResponse.getSnapshotInfo();
+        assertThat(snapshotInfo.successfulShards(), greaterThan(0));
+        assertThat(snapshotInfo.successfulShards(), equalTo(snapshotInfo.totalShards()));
+
+        final Map<String, IndexId> indexIds = getRepositoryData("test-repo").getIndices();
+        final Path indicesPath = repo.resolve("indices");
+
+        logger.info("--> delete index metadata and shard metadata");
+        for (String index : indices) {
+            Path shardZero = indicesPath.resolve(indexIds.get(index).getId()).resolve("0");
+            if (randomBoolean()) {
+                Files.delete(shardZero.resolve("index-" + getRepositoryData("test-repo").shardGenerations()
+                        .getShardGen(indexIds.get(index), 0)));
+            }
+            Files.delete(shardZero.resolve("snap-" + snapshotInfo.snapshotId().getUUID() + ".dat"));
+        }
+
+        startDeleteSnapshot("test-repo", "test-snap-1").get();
+
+        logger.info("--> make sure snapshot doesn't exist");
+
+        expectThrows(SnapshotMissingException.class, () -> client.admin().cluster().prepareGetSnapshots("test-repo")
+                .addSnapshots("test-snap-1").get().getSnapshots("test-repo"));
+
+        for (String index : indices) {
+            assertTrue(Files.notExists(indicesPath.resolve(indexIds.get(index).getId())));
+        }
+    }
+
+    public void testDeleteSnapshotWithMissingMetadata() throws Exception {
+        Client client = client();
+
+        Path repo = randomRepoPath();
+        createRepository("test-repo", "fs", repo);
+
+        createIndex("test-idx-1", "test-idx-2");
+        logger.info("--> indexing some data");
+        indexRandom(true,
+                client().prepareIndex("test-idx-1").setSource("foo", "bar"),
+                client().prepareIndex("test-idx-2").setSource("foo", "bar"));
+
+        logger.info("--> creating snapshot");
+        CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap-1")
+                .setWaitForCompletion(true).setIndices("test-idx-*").get();
+        assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0));
+        assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(),
+                equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()));
+
+        logger.info("--> delete global state metadata");
+        Path metadata = repo.resolve("meta-" + createSnapshotResponse.getSnapshotInfo().snapshotId().getUUID() + ".dat");
+        Files.delete(metadata);
+
+        startDeleteSnapshot("test-repo", "test-snap-1").get();
+
+        logger.info("--> make sure snapshot doesn't exist");
+        expectThrows(SnapshotMissingException.class, () -> client.admin().cluster().prepareGetSnapshots("test-repo")
+                .addSnapshots("test-snap-1").get().getSnapshots("test-repo"));
+    }
+
+    public void testDeleteSnapshotWithCorruptedSnapshotFile() throws Exception {
+        Client client = client();
+
+        Path repo = randomRepoPath();
+        createRepository("test-repo", "fs", Settings.builder()
+                .put("location", repo).put("compress", false)
+                .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES));
+
+        createIndex("test-idx-1", "test-idx-2");
+        logger.info("--> indexing some data");
+        indexRandom(true,
+                client().prepareIndex("test-idx-1").setSource("foo", "bar"),
+                client().prepareIndex("test-idx-2").setSource("foo", "bar"));
+
+        logger.info("--> creating snapshot");
+        CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap-1")
+                .setWaitForCompletion(true).setIndices("test-idx-*").get();
+        assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0));
+        assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(),
+                equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()));
+
+        logger.info("--> truncate snapshot file to make it unreadable");
+        Path snapshotPath = repo.resolve("snap-" + createSnapshotResponse.getSnapshotInfo().snapshotId().getUUID() + ".dat");
+        try(SeekableByteChannel outChan = Files.newByteChannel(snapshotPath, StandardOpenOption.WRITE)) {
+            outChan.truncate(randomInt(10));
+        }
+        startDeleteSnapshot("test-repo", "test-snap-1").get();
+
+        logger.info("--> make sure snapshot doesn't exist");
+        expectThrows(SnapshotMissingException.class,
+                () -> client.admin().cluster().prepareGetSnapshots("test-repo").addSnapshots("test-snap-1").get().
+                        getSnapshots("test-repo"));
+
+        logger.info("--> make sure that we can create the snapshot again");
+        createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap-1")
+                .setWaitForCompletion(true).setIndices("test-idx-*").get();
+        assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0));
+        assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(),
+                equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()));
+    }
+
+    /** Tests that a snapshot with a corrupted global state file can still be deleted */
+    public void testDeleteSnapshotWithCorruptedGlobalState() throws Exception {
+        final Path repo = randomRepoPath();
+
+        createRepository("test-repo", "fs", Settings.builder()
+                .put("location", repo)
+                .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES));
+
+        createIndex("test-idx-1", "test-idx-2");
+        indexRandom(true,
+                client().prepareIndex("test-idx-1").setSource("foo", "bar"),
+                client().prepareIndex("test-idx-2").setSource("foo", "bar"),
+                client().prepareIndex("test-idx-2").setSource("foo", "bar"));
+        flushAndRefresh("test-idx-1", "test-idx-2");
+
+        SnapshotInfo snapshotInfo = createFullSnapshot("test-repo", "test-snap");
+
+        final Path globalStatePath = repo.resolve("meta-" + snapshotInfo.snapshotId().getUUID() + ".dat");
+        if (randomBoolean()) {
+            // Delete the global state metadata file
+            IOUtils.deleteFilesIgnoringExceptions(globalStatePath);
+        } else {
+            // Truncate the global state metadata file
+            try (SeekableByteChannel outChan = Files.newByteChannel(globalStatePath, StandardOpenOption.WRITE)) {
+                outChan.truncate(randomInt(10));
+            }
+        }
+
+        List<SnapshotInfo> snapshotInfos = clusterAdmin().prepareGetSnapshots("test-repo").get().getSnapshots("test-repo");
+        assertThat(snapshotInfos.size(), equalTo(1));
+        assertThat(snapshotInfos.get(0).state(), equalTo(SnapshotState.SUCCESS));
+        assertThat(snapshotInfos.get(0).snapshotId().getName(), equalTo("test-snap"));
+
+        SnapshotsStatusResponse snapshotStatusResponse =
+                clusterAdmin().prepareSnapshotStatus("test-repo").setSnapshots("test-snap").get();
+        assertThat(snapshotStatusResponse.getSnapshots(), hasSize(1));
+        assertThat(snapshotStatusResponse.getSnapshots().get(0).getSnapshot().getSnapshotId().getName(), equalTo("test-snap"));
+
+        assertAcked(startDeleteSnapshot("test-repo", "test-snap").get());
+        expectThrows(SnapshotMissingException.class, () -> clusterAdmin()
+                .prepareGetSnapshots("test-repo").addSnapshots("test-snap").get().getSnapshots("test-repo"));
+        assertRequestBuilderThrows(clusterAdmin().prepareSnapshotStatus("test-repo").addSnapshots("test-snap"),
+                SnapshotMissingException.class);
+
+        createFullSnapshot("test-repo", "test-snap");
+    }
+
+    public void testSnapshotWithMissingShardLevelIndexFile() throws Exception {
+        disableRepoConsistencyCheck("This test uses a purposely broken repository so it would fail consistency checks");
+
+        Path repo = randomRepoPath();
+        createRepository("test-repo", "fs", repo);
+
+        createIndex("test-idx-1", "test-idx-2");
+        logger.info("--> indexing some data");
+        indexRandom(true,
+                client().prepareIndex("test-idx-1").setSource("foo", "bar"),
+                client().prepareIndex("test-idx-2").setSource("foo", "bar"));
+
+        logger.info("--> creating snapshot");
+        clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap-1")
+                .setWaitForCompletion(true).setIndices("test-idx-*").get();
+
+        logger.info("--> deleting shard level index file");
+        final Path indicesPath = repo.resolve("indices");
+        for (IndexId indexId : getRepositoryData("test-repo").getIndices().values()) {
+            final Path shardGen;
+            try (Stream<Path> shardFiles = Files.list(indicesPath.resolve(indexId.getId()).resolve("0"))) {
+                shardGen = shardFiles.filter(file -> file.getFileName().toString().startsWith(BlobStoreRepository.INDEX_FILE_PREFIX))
+                        .findFirst().orElseThrow(() -> new AssertionError("Failed to find shard index blob"));
+            }
+            Files.delete(shardGen);
+        }
+
+        logger.info("--> creating another snapshot");
+        CreateSnapshotResponse createSnapshotResponse =
+                clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap-2")
+                        .setWaitForCompletion(true).setIndices("test-idx-1").get();
+        assertEquals(createSnapshotResponse.getSnapshotInfo().successfulShards(),
+                createSnapshotResponse.getSnapshotInfo().totalShards() - 1);
+
+        logger.info("--> restoring the first snapshot, the repository should not have lost any shard data despite deleting index-N, " +
+                "because it uses snap-*.data files and not the index-N to determine what files to restore");
+        client().admin().indices().prepareDelete("test-idx-1", "test-idx-2").get();
+        RestoreSnapshotResponse restoreSnapshotResponse =
+                clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap-1").setWaitForCompletion(true).get();
+        assertEquals(0, restoreSnapshotResponse.getRestoreInfo().failedShards());
+    }
+
     private void assertRepositoryBlocked(Client client, String repo, String existingSnapshot) {
         logger.info("--> try to delete snapshot");
         final RepositoryException repositoryException3 = expectThrows(RepositoryException.class,

+ 750 - 0
server/src/internalClusterTest/java/org/elasticsearch/snapshots/RestoreSnapshotIT.java

@@ -0,0 +1,750 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.snapshots;
+
+import org.elasticsearch.action.ActionFuture;
+import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse;
+import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse;
+import org.elasticsearch.action.admin.indices.settings.get.GetSettingsResponse;
+import org.elasticsearch.action.admin.indices.template.get.GetIndexTemplatesResponse;
+import org.elasticsearch.action.index.IndexRequestBuilder;
+import org.elasticsearch.client.Client;
+import org.elasticsearch.cluster.block.ClusterBlocks;
+import org.elasticsearch.cluster.metadata.IndexMetadata;
+import org.elasticsearch.cluster.metadata.MappingMetadata;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.unit.ByteSizeUnit;
+import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.common.xcontent.XContentFactory;
+import org.elasticsearch.indices.InvalidIndexNameException;
+import org.elasticsearch.repositories.RepositoriesService;
+import org.elasticsearch.rest.RestStatus;
+
+import java.nio.file.Path;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import static org.elasticsearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS;
+import static org.elasticsearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS;
+import static org.elasticsearch.index.IndexSettings.INDEX_REFRESH_INTERVAL_SETTING;
+import static org.elasticsearch.index.IndexSettings.INDEX_SOFT_DELETES_SETTING;
+import static org.elasticsearch.index.query.QueryBuilders.matchQuery;
+import static org.elasticsearch.indices.recovery.RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertIndexTemplateExists;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertIndexTemplateMissing;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertRequestBuilderThrows;
+import static org.hamcrest.Matchers.allOf;
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.greaterThan;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.hamcrest.Matchers.nullValue;
+
+public class RestoreSnapshotIT extends AbstractSnapshotIntegTestCase {
+
+    public void testParallelRestoreOperations() {
+        String indexName1 = "testindex1";
+        String indexName2 = "testindex2";
+        String repoName = "test-restore-snapshot-repo";
+        String snapshotName1 = "test-restore-snapshot1";
+        String snapshotName2 = "test-restore-snapshot2";
+        Path absolutePath = randomRepoPath().toAbsolutePath();
+        logger.info("Path [{}]", absolutePath);
+        String restoredIndexName1 = indexName1 + "-restored";
+        String restoredIndexName2 = indexName2 + "-restored";
+        String expectedValue = "expected";
+
+        Client client = client();
+        // Write a document
+        String docId = Integer.toString(randomInt());
+        indexDoc(indexName1, docId, "value", expectedValue);
+
+        String docId2 = Integer.toString(randomInt());
+        indexDoc(indexName2, docId2, "value", expectedValue);
+
+        createRepository(repoName, "fs", absolutePath);
+
+        logger.info("--> snapshot");
+        CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot(repoName, snapshotName1)
+                .setWaitForCompletion(true)
+                .setIndices(indexName1)
+                .get();
+        assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0));
+        assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(),
+                equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()));
+        assertThat(createSnapshotResponse.getSnapshotInfo().state(), equalTo(SnapshotState.SUCCESS));
+
+        CreateSnapshotResponse createSnapshotResponse2 = client.admin().cluster().prepareCreateSnapshot(repoName, snapshotName2)
+                .setWaitForCompletion(true)
+                .setIndices(indexName2)
+                .get();
+        assertThat(createSnapshotResponse2.getSnapshotInfo().successfulShards(), greaterThan(0));
+        assertThat(createSnapshotResponse2.getSnapshotInfo().successfulShards(),
+                equalTo(createSnapshotResponse2.getSnapshotInfo().totalShards()));
+        assertThat(createSnapshotResponse2.getSnapshotInfo().state(), equalTo(SnapshotState.SUCCESS));
+
+        RestoreSnapshotResponse restoreSnapshotResponse1 = client.admin().cluster().prepareRestoreSnapshot(repoName, snapshotName1)
+                .setWaitForCompletion(false)
+                .setRenamePattern(indexName1)
+                .setRenameReplacement(restoredIndexName1)
+                .get();
+        RestoreSnapshotResponse restoreSnapshotResponse2 = client.admin().cluster().prepareRestoreSnapshot(repoName, snapshotName2)
+                .setWaitForCompletion(false)
+                .setRenamePattern(indexName2)
+                .setRenameReplacement(restoredIndexName2)
+                .get();
+        assertThat(restoreSnapshotResponse1.status(), equalTo(RestStatus.ACCEPTED));
+        assertThat(restoreSnapshotResponse2.status(), equalTo(RestStatus.ACCEPTED));
+        ensureGreen(restoredIndexName1, restoredIndexName2);
+        assertThat(client.prepareGet(restoredIndexName1, docId).get().isExists(), equalTo(true));
+        assertThat(client.prepareGet(restoredIndexName2, docId2).get().isExists(), equalTo(true));
+    }
+
+    public void testParallelRestoreOperationsFromSingleSnapshot() throws Exception {
+        String indexName1 = "testindex1";
+        String indexName2 = "testindex2";
+        String repoName = "test-restore-snapshot-repo";
+        String snapshotName = "test-restore-snapshot";
+        Path absolutePath = randomRepoPath().toAbsolutePath();
+        logger.info("Path [{}]", absolutePath);
+        String restoredIndexName1 = indexName1 + "-restored";
+        String restoredIndexName2 = indexName2 + "-restored";
+        String expectedValue = "expected";
+
+        Client client = client();
+        // Write a document
+        String docId = Integer.toString(randomInt());
+        indexDoc(indexName1, docId, "value", expectedValue);
+
+        String docId2 = Integer.toString(randomInt());
+        indexDoc(indexName2, docId2, "value", expectedValue);
+
+        createRepository(repoName, "fs", absolutePath);
+
+        logger.info("--> snapshot");
+        CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot(repoName, snapshotName)
+                .setWaitForCompletion(true)
+                .setIndices(indexName1, indexName2)
+                .get();
+        assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0));
+        assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(),
+                equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()));
+        assertThat(createSnapshotResponse.getSnapshotInfo().state(), equalTo(SnapshotState.SUCCESS));
+
+        ActionFuture<RestoreSnapshotResponse> restoreSnapshotResponse1 = client.admin().cluster()
+                .prepareRestoreSnapshot(repoName, snapshotName)
+                .setIndices(indexName1)
+                .setRenamePattern(indexName1)
+                .setRenameReplacement(restoredIndexName1)
+                .execute();
+
+        boolean sameSourceIndex = randomBoolean();
+
+        ActionFuture<RestoreSnapshotResponse> restoreSnapshotResponse2 = client.admin().cluster()
+                .prepareRestoreSnapshot(repoName, snapshotName)
+                .setIndices(sameSourceIndex ? indexName1 : indexName2)
+                .setRenamePattern(sameSourceIndex ? indexName1 : indexName2)
+                .setRenameReplacement(restoredIndexName2)
+                .execute();
+        assertThat(restoreSnapshotResponse1.get().status(), equalTo(RestStatus.ACCEPTED));
+        assertThat(restoreSnapshotResponse2.get().status(), equalTo(RestStatus.ACCEPTED));
+        ensureGreen(restoredIndexName1, restoredIndexName2);
+        assertThat(client.prepareGet(restoredIndexName1, docId).get().isExists(), equalTo(true));
+        assertThat(client.prepareGet(restoredIndexName2, sameSourceIndex ? docId : docId2).get().isExists(), equalTo(true));
+    }
+
+    public void testRestoreIncreasesPrimaryTerms() {
+        final String indexName = randomAlphaOfLengthBetween(5, 10).toLowerCase(Locale.ROOT);
+        createIndex(indexName, indexSettingsNoReplicas(2).build());
+        ensureGreen(indexName);
+
+        if (randomBoolean()) {
+            // open and close the index to increase the primary terms
+            for (int i = 0; i < randomInt(3); i++) {
+                assertAcked(client().admin().indices().prepareClose(indexName));
+                assertAcked(client().admin().indices().prepareOpen(indexName));
+            }
+        }
+
+        final IndexMetadata indexMetadata = clusterAdmin().prepareState().clear().setIndices(indexName)
+                .setMetadata(true).get().getState().metadata().index(indexName);
+        assertThat(indexMetadata.getSettings().get(IndexMetadata.SETTING_HISTORY_UUID), nullValue());
+        final int numPrimaries = getNumShards(indexName).numPrimaries;
+        final Map<Integer, Long> primaryTerms = IntStream.range(0, numPrimaries)
+                .boxed().collect(Collectors.toMap(shardId -> shardId, indexMetadata::primaryTerm));
+
+        createRepository("test-repo", "fs");
+        final CreateSnapshotResponse createSnapshotResponse = clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap")
+                .setWaitForCompletion(true).setIndices(indexName).get();
+        assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), equalTo(numPrimaries));
+        assertThat(createSnapshotResponse.getSnapshotInfo().failedShards(), equalTo(0));
+
+        assertAcked(client().admin().indices().prepareClose(indexName));
+
+        final RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap")
+                .setWaitForCompletion(true).get();
+        assertThat(restoreSnapshotResponse.getRestoreInfo().successfulShards(), equalTo(numPrimaries));
+        assertThat(restoreSnapshotResponse.getRestoreInfo().failedShards(), equalTo(0));
+
+        final IndexMetadata restoredIndexMetadata = clusterAdmin().prepareState().clear().setIndices(indexName)
+                .setMetadata(true).get().getState().metadata().index(indexName);
+        for (int shardId = 0; shardId < numPrimaries; shardId++) {
+            assertThat(restoredIndexMetadata.primaryTerm(shardId), greaterThan(primaryTerms.get(shardId)));
+        }
+        assertThat(restoredIndexMetadata.getSettings().get(IndexMetadata.SETTING_HISTORY_UUID), notNullValue());
+    }
+
+    public void testRestoreWithDifferentMappingsAndSettings() throws Exception {
+        createRepository("test-repo", "fs");
+
+        logger.info("--> create index with baz field");
+        assertAcked(prepareCreate("test-idx", 2, Settings.builder()
+                .put(indexSettings()).put(SETTING_NUMBER_OF_REPLICAS, between(0, 1)).put("refresh_interval", 10, TimeUnit.SECONDS)));
+
+        NumShards numShards = getNumShards("test-idx");
+
+        assertAcked(client().admin().indices().preparePutMapping("test-idx").setSource("baz", "type=text"));
+        ensureGreen();
+
+        logger.info("--> snapshot it");
+        CreateSnapshotResponse createSnapshotResponse = clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap")
+                .setWaitForCompletion(true).setIndices("test-idx").get();
+        assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0));
+        assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(),
+                equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()));
+
+        logger.info("--> delete the index and recreate it with foo field");
+        cluster().wipeIndices("test-idx");
+        assertAcked(prepareCreate("test-idx", 2, Settings.builder()
+                .put(SETTING_NUMBER_OF_SHARDS, numShards.numPrimaries).put(SETTING_NUMBER_OF_REPLICAS, between(0, 1))
+                .put("refresh_interval", 5, TimeUnit.SECONDS)));
+        assertAcked(client().admin().indices().preparePutMapping("test-idx").setSource("foo", "type=text"));
+        ensureGreen();
+
+        logger.info("--> close index");
+        client().admin().indices().prepareClose("test-idx").get();
+
+        logger.info("--> restore all indices from the snapshot");
+        RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap")
+                .setWaitForCompletion(true).execute().actionGet();
+        assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0));
+
+        logger.info("--> assert that old mapping is restored");
+        MappingMetadata mappings = clusterAdmin().prepareState().get().getState().getMetadata()
+                .getIndices().get("test-idx").mapping();
+        assertThat(mappings.sourceAsMap().toString(), containsString("baz"));
+        assertThat(mappings.sourceAsMap().toString(), not(containsString("foo")));
+
+        logger.info("--> assert that old settings are restored");
+        GetSettingsResponse getSettingsResponse = client().admin().indices().prepareGetSettings("test-idx").execute().actionGet();
+        assertThat(getSettingsResponse.getSetting("test-idx", "index.refresh_interval"), equalTo("10s"));
+    }
+
+    public void testRestoreAliases() throws Exception {
+        createRepository("test-repo", "fs");
+
+        logger.info("--> create test indices");
+        createIndex("test-idx-1", "test-idx-2", "test-idx-3");
+        ensureGreen();
+
+        logger.info("--> create aliases");
+        assertAcked(client().admin().indices().prepareAliases()
+                .addAlias("test-idx-1", "alias-123")
+                .addAlias("test-idx-2", "alias-123")
+                .addAlias("test-idx-3", "alias-123")
+                .addAlias("test-idx-1", "alias-1")
+                .get());
+
+        assertFalse(client().admin().indices().prepareGetAliases("alias-123").get().getAliases().isEmpty());
+
+        logger.info("--> snapshot");
+        assertThat(clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap")
+                        .setIndices().setWaitForCompletion(true).get().getSnapshotInfo().state(),
+                equalTo(SnapshotState.SUCCESS));
+
+        logger.info("-->  delete all indices");
+        cluster().wipeIndices("test-idx-1", "test-idx-2", "test-idx-3");
+        assertTrue(client().admin().indices().prepareGetAliases("alias-123").get().getAliases().isEmpty());
+        assertTrue(client().admin().indices().prepareGetAliases("alias-1").get().getAliases().isEmpty());
+
+        logger.info("--> restore snapshot with aliases");
+        RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap")
+                .setWaitForCompletion(true).setRestoreGlobalState(true).execute().actionGet();
+        // We don't restore any indices here
+        assertThat(restoreSnapshotResponse.getRestoreInfo().successfulShards(), allOf(greaterThan(0),
+                equalTo(restoreSnapshotResponse.getRestoreInfo().totalShards())));
+
+        logger.info("--> check that aliases are restored");
+        assertFalse(client().admin().indices().prepareGetAliases("alias-123").get().getAliases().isEmpty());
+        assertFalse(client().admin().indices().prepareGetAliases("alias-1").get().getAliases().isEmpty());
+
+        logger.info("-->  update aliases");
+        assertAcked(client().admin().indices().prepareAliases().removeAlias("test-idx-3", "alias-123"));
+        assertAcked(client().admin().indices().prepareAliases().addAlias("test-idx-3", "alias-3"));
+
+        logger.info("-->  delete and close indices");
+        cluster().wipeIndices("test-idx-1", "test-idx-2");
+        assertAcked(client().admin().indices().prepareClose("test-idx-3"));
+        assertTrue(client().admin().indices().prepareGetAliases("alias-123").get().getAliases().isEmpty());
+        assertTrue(client().admin().indices().prepareGetAliases("alias-1").get().getAliases().isEmpty());
+
+        logger.info("--> restore snapshot without aliases");
+        restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap").setWaitForCompletion(true)
+                .setRestoreGlobalState(true).setIncludeAliases(false).execute().actionGet();
+        // We don't restore any indices here
+        assertThat(restoreSnapshotResponse.getRestoreInfo().successfulShards(), allOf(greaterThan(0),
+                equalTo(restoreSnapshotResponse.getRestoreInfo().totalShards())));
+
+        logger.info("--> check that aliases are not restored and existing aliases still exist");
+        assertTrue(client().admin().indices().prepareGetAliases("alias-123").get().getAliases().isEmpty());
+        assertTrue(client().admin().indices().prepareGetAliases("alias-1").get().getAliases().isEmpty());
+        assertFalse(client().admin().indices().prepareGetAliases("alias-3").get().getAliases().isEmpty());
+    }
+
+    public void testRestoreTemplates() throws Exception {
+        createRepository("test-repo", "fs");
+
+        logger.info("-->  creating test template");
+        assertThat(client().admin().indices()
+                .preparePutTemplate("test-template")
+                .setPatterns(Collections.singletonList("te*"))
+                .setMapping(XContentFactory.jsonBuilder()
+                        .startObject()
+                        .startObject("_doc")
+                        .startObject("properties")
+                        .startObject("field1")
+                        .field("type", "text")
+                        .field("store", true)
+                        .endObject()
+                        .startObject("field2")
+                        .field("type", "keyword")
+                        .field("store", true)
+                        .endObject()
+                        .endObject()
+                        .endObject()
+                        .endObject())
+                .get().isAcknowledged(), equalTo(true));
+
+        logger.info("--> snapshot");
+        final SnapshotInfo snapshotInfo = assertSuccessful(clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap")
+                .setIndices().setWaitForCompletion(true).execute());
+        assertThat(snapshotInfo.totalShards(), equalTo(0));
+        assertThat(snapshotInfo.successfulShards(), equalTo(0));
+        assertThat(getSnapshot("test-repo", "test-snap").state(), equalTo(SnapshotState.SUCCESS));
+
+        logger.info("-->  delete test template");
+        assertThat(client().admin().indices().prepareDeleteTemplate("test-template").get().isAcknowledged(), equalTo(true));
+        GetIndexTemplatesResponse getIndexTemplatesResponse = client().admin().indices().prepareGetTemplates().get();
+        assertIndexTemplateMissing(getIndexTemplatesResponse, "test-template");
+
+        logger.info("--> restore cluster state");
+        RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap")
+                .setWaitForCompletion(true).setRestoreGlobalState(true).execute().actionGet();
+        // We don't restore any indices here
+        assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), equalTo(0));
+
+        logger.info("--> check that template is restored");
+        getIndexTemplatesResponse = client().admin().indices().prepareGetTemplates().get();
+        assertIndexTemplateExists(getIndexTemplatesResponse, "test-template");
+    }
+
+
+    public void testRenameOnRestore() throws Exception {
+        Client client = client();
+
+        createRepository("test-repo", "fs");
+
+        createIndex("test-idx-1", "test-idx-2", "test-idx-3");
+        ensureGreen();
+
+        assertAcked(client.admin().indices().prepareAliases()
+                .addAlias("test-idx-1", "alias-1", false)
+                .addAlias("test-idx-2", "alias-2", false)
+                .addAlias("test-idx-3", "alias-3", false)
+        );
+
+        indexRandomDocs("test-idx-1", 100);
+        indexRandomDocs("test-idx-2", 100);
+
+        logger.info("--> snapshot");
+        CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap")
+                .setWaitForCompletion(true).setIndices("test-idx-1", "test-idx-2").get();
+        assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0));
+        assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(),
+                equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()));
+
+        logger.info("--> restore indices with different names");
+        RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap")
+                .setRenamePattern("(.+)").setRenameReplacement("$1-copy").setWaitForCompletion(true).execute().actionGet();
+        assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0));
+
+        assertDocCount("test-idx-1-copy", 100L);
+        assertDocCount("test-idx-2-copy", 100L);
+
+        logger.info("--> close just restored indices");
+        client.admin().indices().prepareClose("test-idx-1-copy", "test-idx-2-copy").get();
+
+        logger.info("--> and try to restore these indices again");
+        restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap")
+                .setRenamePattern("(.+)").setRenameReplacement("$1-copy").setWaitForCompletion(true).execute().actionGet();
+        assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0));
+
+        assertDocCount("test-idx-1-copy", 100L);
+        assertDocCount("test-idx-2-copy", 100L);
+
+        logger.info("--> close indices");
+        assertAcked(client.admin().indices().prepareClose("test-idx-1", "test-idx-2-copy"));
+
+        logger.info("--> restore indices with different names");
+        restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap")
+                .setRenamePattern("(.+-2)").setRenameReplacement("$1-copy").setWaitForCompletion(true).execute().actionGet();
+        assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0));
+
+        logger.info("--> delete indices");
+        cluster().wipeIndices("test-idx-1", "test-idx-1-copy", "test-idx-2", "test-idx-2-copy");
+
+        logger.info("--> try renaming indices using the same name");
+        try {
+            client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setRenamePattern("(.+)")
+                    .setRenameReplacement("same-name").setWaitForCompletion(true).execute().actionGet();
+            fail("Shouldn't be here");
+        } catch (SnapshotRestoreException ex) {
+            // Expected
+        }
+
+        logger.info("--> try renaming indices using the same name");
+        try {
+            client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setRenamePattern("test-idx-2")
+                    .setRenameReplacement("test-idx-1").setWaitForCompletion(true).execute().actionGet();
+            fail("Shouldn't be here");
+        } catch (SnapshotRestoreException ex) {
+            // Expected
+        }
+
+        logger.info("--> try renaming indices using invalid index name");
+        try {
+            client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setIndices("test-idx-1").setRenamePattern(".+")
+                    .setRenameReplacement("__WRONG__").setWaitForCompletion(true).execute().actionGet();
+            fail("Shouldn't be here");
+        } catch (InvalidIndexNameException ex) {
+            // Expected
+        }
+
+        logger.info("--> try renaming indices into existing alias name");
+        try {
+            client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setIndices("test-idx-1").setRenamePattern(".+")
+                    .setRenameReplacement("alias-3").setWaitForCompletion(true).execute().actionGet();
+            fail("Shouldn't be here");
+        } catch (InvalidIndexNameException ex) {
+            // Expected
+        }
+
+        logger.info("--> try renaming indices into existing alias of itself");
+        try {
+            client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setIndices("test-idx-1").setRenamePattern("test-idx")
+                    .setRenameReplacement("alias").setWaitForCompletion(true).execute().actionGet();
+            fail("Shouldn't be here");
+        } catch (SnapshotRestoreException ex) {
+            // Expected
+        }
+
+        logger.info("--> try renaming indices into existing alias of another restored index");
+        try {
+            client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap").setIndices("test-idx-1", "test-idx-2")
+                    .setRenamePattern("test-idx-1").setRenameReplacement("alias-2").setWaitForCompletion(true).execute().actionGet();
+            fail("Shouldn't be here");
+        } catch (SnapshotRestoreException ex) {
+            // Expected
+        }
+
+        logger.info("--> try renaming indices into existing alias of itself, but don't restore aliases ");
+        restoreSnapshotResponse = client.admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap")
+                .setIndices("test-idx-1").setRenamePattern("test-idx").setRenameReplacement("alias")
+                .setWaitForCompletion(true).setIncludeAliases(false).execute().actionGet();
+        assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0));
+    }
+
+    public void testDynamicRestoreThrottling() throws Exception {
+        Client client = client();
+
+        createRepository("test-repo", "fs", Settings.builder()
+                .put("location", randomRepoPath()).put("compress", randomBoolean())
+                .put("chunk_size", 100, ByteSizeUnit.BYTES));
+
+        createIndexWithRandomDocs("test-idx", 100);
+
+        logger.info("--> snapshot");
+        client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap")
+                .setWaitForCompletion(true).setIndices("test-idx").get();
+
+        logger.info("--> delete index");
+        cluster().wipeIndices("test-idx");
+
+        logger.info("--> restore index");
+        client.admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder()
+                .put(INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.getKey(), "100b").build()).get();
+        ActionFuture<RestoreSnapshotResponse> restoreSnapshotResponse = client.admin().cluster()
+                .prepareRestoreSnapshot("test-repo", "test-snap").setWaitForCompletion(true).execute();
+
+        // check if throttling is active
+        assertBusy(() -> {
+            long restorePause = 0L;
+            for (RepositoriesService repositoriesService : internalCluster().getDataNodeInstances(RepositoriesService.class)) {
+                restorePause += repositoriesService.repository("test-repo").getRestoreThrottleTimeInNanos();
+            }
+            assertThat(restorePause, greaterThan(TimeValue.timeValueSeconds(randomIntBetween(1, 5)).nanos()));
+            assertFalse(restoreSnapshotResponse.isDone());
+        }, 30, TimeUnit.SECONDS);
+
+        // run at full speed again
+        client.admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder()
+                .putNull(INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.getKey()).build()).get();
+
+        // check that restore now completes quickly (i.e. within 10 seconds)
+        assertBusy(() -> assertTrue(restoreSnapshotResponse.isDone()));
+
+        assertThat(restoreSnapshotResponse.get().getRestoreInfo().totalShards(), greaterThan(0));
+        assertDocCount("test-idx", 100L);
+    }
+
+    public void testChangeSettingsOnRestore() throws Exception {
+        Client client = client();
+
+        createRepository("test-repo", "fs");
+
+        logger.info("--> create test index with case-preserving search analyzer");
+
+        Settings.Builder indexSettings = Settings.builder()
+                .put(indexSettings())
+                .put(SETTING_NUMBER_OF_REPLICAS, between(0, 1))
+                .put(INDEX_REFRESH_INTERVAL_SETTING.getKey(), "10s")
+                .put("index.analysis.analyzer.my_analyzer.type", "custom")
+                .put("index.analysis.analyzer.my_analyzer.tokenizer", "standard");
+
+        assertAcked(prepareCreate("test-idx", 2, indexSettings));
+
+        int numberOfShards = getNumShards("test-idx").numPrimaries;
+        assertAcked(client().admin().indices().preparePutMapping("test-idx")
+                .setSource("field1", "type=text,analyzer=standard,search_analyzer=my_analyzer"));
+        final int numdocs = randomIntBetween(10, 100);
+        IndexRequestBuilder[] builders = new IndexRequestBuilder[numdocs];
+        for (int i = 0; i < builders.length; i++) {
+            builders[i] = client().prepareIndex("test-idx").setId(Integer.toString(i)).setSource("field1", "Foo bar " + i);
+        }
+        indexRandom(true, builders);
+        flushAndRefresh();
+
+        assertHitCount(client.prepareSearch("test-idx").setSize(0).setQuery(matchQuery("field1", "foo")).get(), numdocs);
+        assertHitCount(client.prepareSearch("test-idx").setSize(0).setQuery(matchQuery("field1", "Foo")).get(), 0);
+        assertHitCount(client.prepareSearch("test-idx").setSize(0).setQuery(matchQuery("field1", "bar")).get(), numdocs);
+
+        logger.info("--> snapshot it");
+        CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap")
+                .setWaitForCompletion(true).setIndices("test-idx").get();
+        assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0));
+        assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(),
+                equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()));
+
+        logger.info("--> delete the index and recreate it while changing refresh interval and analyzer");
+        cluster().wipeIndices("test-idx");
+
+        Settings newIndexSettings = Settings.builder()
+                .put("refresh_interval", "5s")
+                .put("index.analysis.analyzer.my_analyzer.type", "standard")
+                .build();
+
+        Settings newIncorrectIndexSettings = Settings.builder()
+                .put(newIndexSettings)
+                .put(SETTING_NUMBER_OF_SHARDS, numberOfShards + 100)
+                .build();
+
+        logger.info("--> try restoring while changing the number of shards - should fail");
+        assertRequestBuilderThrows(client.admin().cluster()
+                .prepareRestoreSnapshot("test-repo", "test-snap")
+                .setIgnoreIndexSettings("index.analysis.*")
+                .setIndexSettings(newIncorrectIndexSettings)
+                .setWaitForCompletion(true), SnapshotRestoreException.class);
+
+        logger.info("--> try restoring while changing the number of replicas to a negative number - should fail");
+        Settings newIncorrectReplicasIndexSettings = Settings.builder()
+                .put(newIndexSettings)
+                .put(SETTING_NUMBER_OF_REPLICAS.substring(IndexMetadata.INDEX_SETTING_PREFIX.length()), randomIntBetween(-10, -1))
+                .build();
+        assertRequestBuilderThrows(client.admin().cluster()
+                .prepareRestoreSnapshot("test-repo", "test-snap")
+                .setIgnoreIndexSettings("index.analysis.*")
+                .setIndexSettings(newIncorrectReplicasIndexSettings)
+                .setWaitForCompletion(true), IllegalArgumentException.class);
+
+        logger.info("--> restore index with correct settings from the snapshot");
+        RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster()
+                .prepareRestoreSnapshot("test-repo", "test-snap")
+                .setIgnoreIndexSettings("index.analysis.*")
+                .setIndexSettings(newIndexSettings)
+                .setWaitForCompletion(true).execute().actionGet();
+        assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0));
+
+        logger.info("--> assert that correct settings are restored");
+        GetSettingsResponse getSettingsResponse = client.admin().indices().prepareGetSettings("test-idx").execute().actionGet();
+        assertThat(getSettingsResponse.getSetting("test-idx", INDEX_REFRESH_INTERVAL_SETTING.getKey()), equalTo("5s"));
+        // Make sure that number of shards didn't change
+        assertThat(getSettingsResponse.getSetting("test-idx", SETTING_NUMBER_OF_SHARDS), equalTo("" + numberOfShards));
+        assertThat(getSettingsResponse.getSetting("test-idx", "index.analysis.analyzer.my_analyzer.type"), equalTo("standard"));
+
+        assertHitCount(client.prepareSearch("test-idx").setSize(0).setQuery(matchQuery("field1", "Foo")).get(), numdocs);
+        assertHitCount(client.prepareSearch("test-idx").setSize(0).setQuery(matchQuery("field1", "bar")).get(), numdocs);
+
+        logger.info("--> delete the index and recreate it while deleting all index settings");
+        cluster().wipeIndices("test-idx");
+
+        logger.info("--> restore index with correct settings from the snapshot");
+        restoreSnapshotResponse = client.admin().cluster()
+                .prepareRestoreSnapshot("test-repo", "test-snap")
+                .setIgnoreIndexSettings("*") // delete everything we can delete
+                .setIndexSettings(newIndexSettings)
+                .setWaitForCompletion(true).execute().actionGet();
+        assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0));
+
+        logger.info("--> assert that correct settings are restored and index is still functional");
+        getSettingsResponse = client.admin().indices().prepareGetSettings("test-idx").execute().actionGet();
+        assertThat(getSettingsResponse.getSetting("test-idx", INDEX_REFRESH_INTERVAL_SETTING.getKey()), equalTo("5s"));
+        // Make sure that number of shards didn't change
+        assertThat(getSettingsResponse.getSetting("test-idx", SETTING_NUMBER_OF_SHARDS), equalTo("" + numberOfShards));
+
+        assertHitCount(client.prepareSearch("test-idx").setSize(0).setQuery(matchQuery("field1", "Foo")).get(), numdocs);
+        assertHitCount(client.prepareSearch("test-idx").setSize(0).setQuery(matchQuery("field1", "bar")).get(), numdocs);
+    }
+
+    public void testRecreateBlocksOnRestore() throws Exception {
+        Client client = client();
+
+        createRepository("test-repo", "fs");
+
+        Settings.Builder indexSettings = Settings.builder()
+                .put(indexSettings())
+                .put(SETTING_NUMBER_OF_REPLICAS, between(0, 1))
+                .put(INDEX_REFRESH_INTERVAL_SETTING.getKey(), "10s");
+
+        logger.info("--> create index");
+        assertAcked(prepareCreate("test-idx", 2, indexSettings));
+
+        try {
+            List<String> initialBlockSettings = randomSubsetOf(randomInt(3),
+                    IndexMetadata.SETTING_BLOCKS_WRITE, IndexMetadata.SETTING_BLOCKS_METADATA, IndexMetadata.SETTING_READ_ONLY);
+            Settings.Builder initialSettingsBuilder = Settings.builder();
+            for (String blockSetting : initialBlockSettings) {
+                initialSettingsBuilder.put(blockSetting, true);
+            }
+            Settings initialSettings = initialSettingsBuilder.build();
+            logger.info("--> using initial block settings {}", initialSettings);
+
+            if (!initialSettings.isEmpty()) {
+                logger.info("--> apply initial blocks to index");
+                client().admin().indices().prepareUpdateSettings("test-idx").setSettings(initialSettingsBuilder).get();
+            }
+
+            logger.info("--> snapshot index");
+            CreateSnapshotResponse createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap")
+                    .setWaitForCompletion(true).setIndices("test-idx").get();
+            assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0));
+            assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(),
+                    equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()));
+
+            logger.info("--> remove blocks and delete index");
+            disableIndexBlock("test-idx", IndexMetadata.SETTING_BLOCKS_METADATA);
+            disableIndexBlock("test-idx", IndexMetadata.SETTING_READ_ONLY);
+            disableIndexBlock("test-idx", IndexMetadata.SETTING_BLOCKS_WRITE);
+            disableIndexBlock("test-idx", IndexMetadata.SETTING_BLOCKS_READ);
+            cluster().wipeIndices("test-idx");
+
+            logger.info("--> restore index with additional block changes");
+            List<String> changeBlockSettings = randomSubsetOf(randomInt(4),
+                    IndexMetadata.SETTING_BLOCKS_METADATA, IndexMetadata.SETTING_BLOCKS_WRITE,
+                    IndexMetadata.SETTING_READ_ONLY, IndexMetadata.SETTING_BLOCKS_READ);
+            Settings.Builder changedSettingsBuilder = Settings.builder();
+            for (String blockSetting : changeBlockSettings) {
+                changedSettingsBuilder.put(blockSetting, randomBoolean());
+            }
+            Settings changedSettings = changedSettingsBuilder.build();
+            logger.info("--> applying changed block settings {}", changedSettings);
+
+            RestoreSnapshotResponse restoreSnapshotResponse = client.admin().cluster()
+                    .prepareRestoreSnapshot("test-repo", "test-snap")
+                    .setIndexSettings(changedSettings)
+                    .setWaitForCompletion(true).execute().actionGet();
+            assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0));
+
+            ClusterBlocks blocks = client.admin().cluster().prepareState().clear().setBlocks(true).get().getState().blocks();
+            // compute current index settings (as we cannot query them if they contain SETTING_BLOCKS_METADATA)
+            Settings mergedSettings = Settings.builder()
+                    .put(initialSettings)
+                    .put(changedSettings)
+                    .build();
+            logger.info("--> merged block settings {}", mergedSettings);
+
+            logger.info("--> checking consistency between settings and blocks");
+            assertThat(mergedSettings.getAsBoolean(IndexMetadata.SETTING_BLOCKS_METADATA, false),
+                    is(blocks.hasIndexBlock("test-idx", IndexMetadata.INDEX_METADATA_BLOCK)));
+            assertThat(mergedSettings.getAsBoolean(IndexMetadata.SETTING_BLOCKS_READ, false),
+                    is(blocks.hasIndexBlock("test-idx", IndexMetadata.INDEX_READ_BLOCK)));
+            assertThat(mergedSettings.getAsBoolean(IndexMetadata.SETTING_BLOCKS_WRITE, false),
+                    is(blocks.hasIndexBlock("test-idx", IndexMetadata.INDEX_WRITE_BLOCK)));
+            assertThat(mergedSettings.getAsBoolean(IndexMetadata.SETTING_READ_ONLY, false),
+                    is(blocks.hasIndexBlock("test-idx", IndexMetadata.INDEX_READ_ONLY_BLOCK)));
+        } finally {
+            logger.info("--> cleaning up blocks");
+            disableIndexBlock("test-idx", IndexMetadata.SETTING_BLOCKS_METADATA);
+            disableIndexBlock("test-idx", IndexMetadata.SETTING_READ_ONLY);
+            disableIndexBlock("test-idx", IndexMetadata.SETTING_BLOCKS_WRITE);
+            disableIndexBlock("test-idx", IndexMetadata.SETTING_BLOCKS_READ);
+        }
+    }
+
+    public void testForbidDisableSoftDeletesDuringRestore() throws Exception {
+        createRepository("test-repo", "fs");
+        final Settings.Builder settings = Settings.builder();
+        if (randomBoolean()) {
+            settings.put(INDEX_SOFT_DELETES_SETTING.getKey(), true);
+        }
+        createIndex("test-index", settings.build());
+        ensureGreen();
+        if (randomBoolean()) {
+            indexRandomDocs("test-index", between(0, 100));
+            flush("test-index");
+        }
+        clusterAdmin().prepareCreateSnapshot("test-repo", "snapshot-0")
+                .setIndices("test-index").setWaitForCompletion(true).get();
+        final SnapshotRestoreException restoreError = expectThrows(SnapshotRestoreException.class,
+                () -> clusterAdmin().prepareRestoreSnapshot("test-repo", "snapshot-0")
+                        .setIndexSettings(Settings.builder().put(INDEX_SOFT_DELETES_SETTING.getKey(), false))
+                        .setRenamePattern("test-index").setRenameReplacement("new-index")
+                        .get());
+        assertThat(restoreError.getMessage(), containsString("cannot disable setting [index.soft_deletes.enabled] on restore"));
+    }
+}

File diff suppressed because it is too large
+ 18 - 1027
server/src/internalClusterTest/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java


+ 223 - 0
server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotCustomPluginStateIT.java

@@ -0,0 +1,223 @@
+/*
+ * Licensed to Elasticsearch under one or more contributor
+ * license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright
+ * ownership. Elasticsearch licenses this file to you under
+ * the Apache License, Version 2.0 (the "License"); you may
+ * not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.elasticsearch.snapshots;
+
+import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse;
+import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse;
+import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotStatus;
+import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotsStatusResponse;
+import org.elasticsearch.action.admin.cluster.storedscripts.GetStoredScriptResponse;
+import org.elasticsearch.action.admin.indices.template.get.GetIndexTemplatesResponse;
+import org.elasticsearch.action.ingest.DeletePipelineRequest;
+import org.elasticsearch.action.ingest.GetPipelineResponse;
+import org.elasticsearch.common.bytes.BytesArray;
+import org.elasticsearch.common.bytes.BytesReference;
+import org.elasticsearch.common.xcontent.XContentFactory;
+import org.elasticsearch.common.xcontent.XContentType;
+import org.elasticsearch.ingest.IngestTestPlugin;
+import org.elasticsearch.plugins.Plugin;
+import org.elasticsearch.script.MockScriptEngine;
+import org.elasticsearch.script.StoredScriptsIT;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+
+import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertIndexTemplateExists;
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertIndexTemplateMissing;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.greaterThan;
+
+public class SnapshotCustomPluginStateIT extends AbstractSnapshotIntegTestCase {
+
+    @Override
+    protected Collection<Class<? extends Plugin>> nodePlugins() {
+        return Arrays.asList(IngestTestPlugin.class, StoredScriptsIT.CustomScriptPlugin.class);
+    }
+
+    public void testIncludeGlobalState() throws Exception {
+        createRepository("test-repo", "fs");
+
+        boolean testTemplate = randomBoolean();
+        boolean testPipeline = randomBoolean();
+        boolean testScript = (testTemplate == false && testPipeline == false) || randomBoolean(); // At least something should be stored
+
+        if(testTemplate) {
+            logger.info("-->  creating test template");
+            assertThat(client().admin().indices()
+                    .preparePutTemplate("test-template")
+                    .setPatterns(Collections.singletonList("te*"))
+                    .setMapping(XContentFactory.jsonBuilder()
+                            .startObject()
+                            .startObject("_doc")
+                            .startObject("properties")
+                            .startObject("field1")
+                            .field("type", "text")
+                            .field("store", true)
+                            .endObject()
+                            .startObject("field2")
+                            .field("type", "keyword")
+                            .field("store", true)
+                            .endObject()
+                            .endObject()
+                            .endObject()
+                            .endObject())
+                    .get().isAcknowledged(), equalTo(true));
+        }
+
+        if(testPipeline) {
+            logger.info("-->  creating test pipeline");
+            BytesReference pipelineSource = BytesReference.bytes(jsonBuilder().startObject()
+                    .field("description", "my_pipeline")
+                    .startArray("processors")
+                    .startObject()
+                    .startObject("test")
+                    .endObject()
+                    .endObject()
+                    .endArray()
+                    .endObject());
+            assertAcked(clusterAdmin().preparePutPipeline("barbaz", pipelineSource, XContentType.JSON).get());
+        }
+
+        if(testScript) {
+            logger.info("-->  creating test script");
+            assertAcked(clusterAdmin().preparePutStoredScript()
+                    .setId("foobar")
+                    .setContent(new BytesArray(
+                            "{\"script\": { \"lang\": \"" + MockScriptEngine.NAME + "\", \"source\": \"1\"} }"), XContentType.JSON));
+        }
+
+        logger.info("--> snapshot without global state");
+        CreateSnapshotResponse createSnapshotResponse = clusterAdmin()
+                .prepareCreateSnapshot("test-repo", "test-snap-no-global-state").setIndices().setIncludeGlobalState(false)
+                .setWaitForCompletion(true).get();
+        assertThat(createSnapshotResponse.getSnapshotInfo().totalShards(), equalTo(0));
+        assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), equalTo(0));
+        assertThat(getSnapshot("test-repo", "test-snap-no-global-state").state(), equalTo(SnapshotState.SUCCESS));
+        SnapshotsStatusResponse snapshotsStatusResponse = clusterAdmin().prepareSnapshotStatus("test-repo")
+                .addSnapshots("test-snap-no-global-state").get();
+        assertThat(snapshotsStatusResponse.getSnapshots().size(), equalTo(1));
+        SnapshotStatus snapshotStatus = snapshotsStatusResponse.getSnapshots().get(0);
+        assertThat(snapshotStatus.includeGlobalState(), equalTo(false));
+
+        logger.info("--> snapshot with global state");
+        createSnapshotResponse = clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap-with-global-state")
+                .setIndices().setIncludeGlobalState(true).setWaitForCompletion(true).get();
+        assertThat(createSnapshotResponse.getSnapshotInfo().totalShards(), equalTo(0));
+        assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), equalTo(0));
+        assertThat(getSnapshot("test-repo", "test-snap-with-global-state").state(), equalTo(SnapshotState.SUCCESS));
+        snapshotsStatusResponse = clusterAdmin().prepareSnapshotStatus("test-repo")
+                .addSnapshots("test-snap-with-global-state").get();
+        assertThat(snapshotsStatusResponse.getSnapshots().size(), equalTo(1));
+        snapshotStatus = snapshotsStatusResponse.getSnapshots().get(0);
+        assertThat(snapshotStatus.includeGlobalState(), equalTo(true));
+
+        if (testTemplate) {
+            logger.info("-->  delete test template");
+            cluster().wipeTemplates("test-template");
+            GetIndexTemplatesResponse getIndexTemplatesResponse = client().admin().indices().prepareGetTemplates().get();
+            assertIndexTemplateMissing(getIndexTemplatesResponse, "test-template");
+        }
+
+        if (testPipeline) {
+            logger.info("-->  delete test pipeline");
+            assertAcked(clusterAdmin().deletePipeline(new DeletePipelineRequest("barbaz")).get());
+        }
+
+        if (testScript) {
+            logger.info("-->  delete test script");
+            assertAcked(clusterAdmin().prepareDeleteStoredScript("foobar").get());
+        }
+
+        logger.info("--> try restoring cluster state from snapshot without global state");
+        RestoreSnapshotResponse restoreSnapshotResponse = clusterAdmin()
+                .prepareRestoreSnapshot("test-repo", "test-snap-no-global-state")
+                .setWaitForCompletion(true).setRestoreGlobalState(true).execute().actionGet();
+        assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), equalTo(0));
+
+        logger.info("--> check that template wasn't restored");
+        GetIndexTemplatesResponse getIndexTemplatesResponse = client().admin().indices().prepareGetTemplates().get();
+        assertIndexTemplateMissing(getIndexTemplatesResponse, "test-template");
+
+        logger.info("--> restore cluster state");
+        restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap-with-global-state")
+                .setWaitForCompletion(true).setRestoreGlobalState(true).execute().actionGet();
+        assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), equalTo(0));
+
+        if (testTemplate) {
+            logger.info("--> check that template is restored");
+            getIndexTemplatesResponse = client().admin().indices().prepareGetTemplates().get();
+            assertIndexTemplateExists(getIndexTemplatesResponse, "test-template");
+        }
+
+        if (testPipeline) {
+            logger.info("--> check that pipeline is restored");
+            GetPipelineResponse getPipelineResponse = clusterAdmin().prepareGetPipeline("barbaz").get();
+            assertTrue(getPipelineResponse.isFound());
+        }
+
+        if (testScript) {
+            logger.info("--> check that script is restored");
+            GetStoredScriptResponse getStoredScriptResponse = clusterAdmin().prepareGetStoredScript("foobar").get();
+            assertNotNull(getStoredScriptResponse.getSource());
+        }
+
+        createIndexWithRandomDocs("test-idx", 100);
+
+        logger.info("--> snapshot without global state but with indices");
+        createSnapshotResponse = clusterAdmin().prepareCreateSnapshot("test-repo", "test-snap-no-global-state-with-index")
+                .setIndices("test-idx").setIncludeGlobalState(false).setWaitForCompletion(true).get();
+        assertThat(createSnapshotResponse.getSnapshotInfo().totalShards(), greaterThan(0));
+        assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(),
+                equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()));
+        assertThat(getSnapshot("test-repo", "test-snap-no-global-state-with-index").state(), equalTo(SnapshotState.SUCCESS));
+
+        logger.info("-->  delete global state and index ");
+        cluster().wipeIndices("test-idx");
+        if (testTemplate) {
+            cluster().wipeTemplates("test-template");
+        }
+        if (testPipeline) {
+            assertAcked(clusterAdmin().deletePipeline(new DeletePipelineRequest("barbaz")).get());
+        }
+
+        if (testScript) {
+            assertAcked(clusterAdmin().prepareDeleteStoredScript("foobar").get());
+        }
+
+        getIndexTemplatesResponse = client().admin().indices().prepareGetTemplates().get();
+        assertIndexTemplateMissing(getIndexTemplatesResponse, "test-template");
+
+        logger.info("--> try restoring index and cluster state from snapshot without global state");
+        restoreSnapshotResponse = clusterAdmin().prepareRestoreSnapshot("test-repo", "test-snap-no-global-state-with-index")
+                .setWaitForCompletion(true).setRestoreGlobalState(true).execute().actionGet();
+        assertThat(restoreSnapshotResponse.getRestoreInfo().totalShards(), greaterThan(0));
+        assertThat(restoreSnapshotResponse.getRestoreInfo().failedShards(), equalTo(0));
+
+        logger.info("--> check that global state wasn't restored but index was");
+        getIndexTemplatesResponse = client().admin().indices().prepareGetTemplates().get();
+        assertIndexTemplateMissing(getIndexTemplatesResponse, "test-template");
+        assertFalse(clusterAdmin().prepareGetPipeline("barbaz").get().isFound());
+        assertNull(clusterAdmin().prepareGetStoredScript("foobar").get().getSource());
+        assertDocCount("test-idx", 100L);
+    }
+}

+ 266 - 0
server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStatusApisIT.java

@@ -28,17 +28,30 @@ import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotIndexShar
 import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotStats;
 import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotStatus;
 import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotsStatusRequest;
+import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotsStatusResponse;
+import org.elasticsearch.client.Client;
 import org.elasticsearch.cluster.SnapshotsInProgress;
+import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.core.internal.io.IOUtils;
 import org.elasticsearch.repositories.blobstore.BlobStoreRepository;
+import org.elasticsearch.threadpool.ThreadPool;
 
 import java.io.IOException;
 import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Locale;
+import java.util.Map;
 import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.greaterThan;
 import static org.hamcrest.Matchers.hasSize;
@@ -47,6 +60,13 @@ import static org.hamcrest.Matchers.is;
 
 public class SnapshotStatusApisIT extends AbstractSnapshotIntegTestCase {
 
+    @Override
+    protected Settings nodeSettings(int nodeOrdinal) {
+        return Settings.builder().put(super.nodeSettings(nodeOrdinal))
+                .put(ThreadPool.ESTIMATED_TIME_INTERVAL_SETTING.getKey(), 0) // We have tests that check by-timestamp order
+                .build();
+    }
+
     public void testStatusApiConsistency() {
         createRepository("test-repo", "fs");
 
@@ -250,6 +270,252 @@ public class SnapshotStatusApisIT extends AbstractSnapshotIntegTestCase {
         assertThat(responseSnapshotTwo.get().getSnapshotInfo().state(), is(SnapshotState.SUCCESS));
     }
 
+    public void testGetSnapshotsNoRepos() {
+        ensureGreen();
+        GetSnapshotsResponse getSnapshotsResponse = clusterAdmin()
+                .prepareGetSnapshots(new String[]{"_all"})
+                .setSnapshots(randomFrom("_all", "*"))
+                .get();
+
+        assertTrue(getSnapshotsResponse.getRepositories().isEmpty());
+        assertTrue(getSnapshotsResponse.getFailedResponses().isEmpty());
+        assertTrue(getSnapshotsResponse.getSuccessfulResponses().isEmpty());
+    }
+
+    public void testGetSnapshotsMultipleRepos() throws Exception {
+        final Client client = client();
+
+        List<String> snapshotList = new ArrayList<>();
+        List<String> repoList = new ArrayList<>();
+        Map<String, List<String>> repo2SnapshotNames = new HashMap<>();
+
+        logger.info("--> create an index and index some documents");
+        final String indexName = "test-idx";
+        createIndexWithRandomDocs(indexName, 10);
+
+        for (int repoIndex = 0; repoIndex < randomIntBetween(2, 5); repoIndex++) {
+            final String repoName = "repo" + repoIndex;
+            repoList.add(repoName);
+            final Path repoPath = randomRepoPath();
+            logger.info("--> create repository with name " + repoName);
+            assertAcked(client.admin().cluster().preparePutRepository(repoName)
+                    .setType("fs").setSettings(Settings.builder().put("location", repoPath).build()));
+            List<String> snapshotNames = new ArrayList<>();
+            repo2SnapshotNames.put(repoName, snapshotNames);
+
+            for (int snapshotIndex = 0; snapshotIndex < randomIntBetween(2, 5); snapshotIndex++) {
+                final String snapshotName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT);
+                snapshotList.add(snapshotName);
+                // Wait for at least 1ms to ensure that snapshots can be ordered by timestamp deterministically
+                for (final ThreadPool threadPool : internalCluster().getInstances(ThreadPool.class)) {
+                    final long startMillis = threadPool.absoluteTimeInMillis();
+                    assertBusy(() -> assertThat(threadPool.absoluteTimeInMillis(), greaterThan(startMillis)));
+                }
+                logger.info("--> create snapshot with index {} and name {} in repository {}", snapshotIndex, snapshotName, repoName);
+                CreateSnapshotResponse createSnapshotResponse = client.admin()
+                        .cluster()
+                        .prepareCreateSnapshot(repoName, snapshotName)
+                        .setWaitForCompletion(true)
+                        .setIndices(indexName)
+                        .get();
+                assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0));
+                snapshotNames.add(snapshotName);
+            }
+        }
+
+        logger.info("--> get and verify snapshots");
+        GetSnapshotsResponse getSnapshotsResponse = client.admin().cluster()
+                .prepareGetSnapshots(randomFrom(new String[]{"_all"}, new String[]{"repo*"}, repoList.toArray(new String[0])))
+                .setSnapshots(randomFrom("_all", "*"))
+                .get();
+
+        for (Map.Entry<String, List<String>> repo2Names : repo2SnapshotNames.entrySet()) {
+            String repo = repo2Names.getKey();
+            List<String> snapshotNames = repo2Names.getValue();
+            List<SnapshotInfo> snapshots = getSnapshotsResponse.getSnapshots(repo);
+            assertEquals(snapshotNames, snapshots.stream().map(s -> s.snapshotId().getName()).collect(Collectors.toList()));
+        }
+
+        logger.info("--> specify all snapshot names with ignoreUnavailable=false");
+        GetSnapshotsResponse getSnapshotsResponse2 = client.admin().cluster()
+                .prepareGetSnapshots(randomFrom("_all", "repo*"))
+                .setIgnoreUnavailable(false)
+                .setSnapshots(snapshotList.toArray(new String[0]))
+                .get();
+
+        for (String repo : repoList) {
+            expectThrows(SnapshotMissingException.class, () -> getSnapshotsResponse2.getSnapshots(repo));
+        }
+
+
+        logger.info("--> specify all snapshot names with ignoreUnavailable=true");
+        GetSnapshotsResponse getSnapshotsResponse3 = client.admin().cluster()
+                .prepareGetSnapshots(randomFrom("_all", "repo*"))
+                .setIgnoreUnavailable(true)
+                .setSnapshots(snapshotList.toArray(new String[0]))
+                .get();
+
+        for (Map.Entry<String, List<String>> repo2Names : repo2SnapshotNames.entrySet()) {
+            String repo = repo2Names.getKey();
+            List<String> snapshotNames = repo2Names.getValue();
+            List<SnapshotInfo> snapshots = getSnapshotsResponse3.getSnapshots(repo);
+            assertEquals(snapshotNames, snapshots.stream().map(s -> s.snapshotId().getName()).collect(Collectors.toList()));
+        }
+    }
+
+    public void testSnapshotStatusOnFailedSnapshot() throws Exception {
+        String repoName = "test-repo";
+        createRepository(repoName, "fs");
+        final String snapshot = "test-snap-1";
+        addBwCFailedSnapshot(repoName, snapshot, Collections.emptyMap());
+
+        logger.info("--> creating good index");
+        assertAcked(prepareCreate("test-idx-good").setSettings(indexSettingsNoReplicas(1)));
+        ensureGreen();
+        indexRandomDocs("test-idx-good", randomIntBetween(1, 5));
+
+        final SnapshotsStatusResponse snapshotsStatusResponse =
+                clusterAdmin().prepareSnapshotStatus(repoName).setSnapshots(snapshot).get();
+        assertEquals(1, snapshotsStatusResponse.getSnapshots().size());
+        assertEquals(SnapshotsInProgress.State.FAILED, snapshotsStatusResponse.getSnapshots().get(0).getState());
+    }
+
+    public void testGetSnapshotsRequest() throws Exception {
+        final String repositoryName = "test-repo";
+        final String indexName = "test-idx";
+        final Client client = client();
+
+        createRepository(repositoryName, "mock", Settings.builder()
+                .put("location", randomRepoPath()).put("compress", false)
+                .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES).put("wait_after_unblock", 200));
+
+        logger.info("--> get snapshots on an empty repository");
+        expectThrows(SnapshotMissingException.class, () -> client.admin()
+                .cluster()
+                .prepareGetSnapshots(repositoryName)
+                .addSnapshots("non-existent-snapshot")
+                .get()
+                .getSnapshots(repositoryName));
+        // with ignore unavailable set to true, should not throw an exception
+        GetSnapshotsResponse getSnapshotsResponse = client.admin()
+                .cluster()
+                .prepareGetSnapshots(repositoryName)
+                .setIgnoreUnavailable(true)
+                .addSnapshots("non-existent-snapshot")
+                .get();
+        assertThat(getSnapshotsResponse.getSnapshots(repositoryName).size(), equalTo(0));
+
+        logger.info("--> creating an index and indexing documents");
+        // Create index on 2 nodes and make sure each node has a primary by setting no replicas
+        assertAcked(prepareCreate(indexName, 1, Settings.builder().put("number_of_replicas", 0)));
+        ensureGreen();
+        indexRandomDocs(indexName, 10);
+
+        // make sure we return only the in-progress snapshot when taking the first snapshot on a clean repository
+        // take initial snapshot with a block, making sure we only get 1 in-progress snapshot returned
+        // block a node so the create snapshot operation can remain in progress
+        final String initialBlockedNode = blockNodeWithIndex(repositoryName, indexName);
+        client.admin().cluster().prepareCreateSnapshot(repositoryName, "snap-on-empty-repo")
+                .setWaitForCompletion(false)
+                .setIndices(indexName)
+                .get();
+        waitForBlock(initialBlockedNode, repositoryName, TimeValue.timeValueSeconds(60)); // wait for block to kick in
+        getSnapshotsResponse = client.admin().cluster()
+                .prepareGetSnapshots("test-repo")
+                .setSnapshots(randomFrom("_all", "_current", "snap-on-*", "*-on-empty-repo", "snap-on-empty-repo"))
+                .get();
+        assertEquals(1, getSnapshotsResponse.getSnapshots("test-repo").size());
+        assertEquals("snap-on-empty-repo", getSnapshotsResponse.getSnapshots("test-repo").get(0).snapshotId().getName());
+        unblockNode(repositoryName, initialBlockedNode); // unblock node
+        startDeleteSnapshot(repositoryName, "snap-on-empty-repo").get();
+
+        final int numSnapshots = randomIntBetween(1, 3) + 1;
+        logger.info("--> take {} snapshot(s)", numSnapshots - 1);
+        final String[] snapshotNames = new String[numSnapshots];
+        for (int i = 0; i < numSnapshots - 1; i++) {
+            final String snapshotName = randomAlphaOfLength(8).toLowerCase(Locale.ROOT);
+            CreateSnapshotResponse createSnapshotResponse = client.admin()
+                    .cluster()
+                    .prepareCreateSnapshot(repositoryName, snapshotName)
+                    .setWaitForCompletion(true)
+                    .setIndices(indexName)
+                    .get();
+            assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0));
+            snapshotNames[i] = snapshotName;
+        }
+        logger.info("--> take another snapshot to be in-progress");
+        // add documents so there are data files to block on
+        for (int i = 10; i < 20; i++) {
+            indexDoc(indexName, Integer.toString(i), "foo", "bar" + i);
+        }
+        refresh();
+
+        final String inProgressSnapshot = randomAlphaOfLength(8).toLowerCase(Locale.ROOT);
+        snapshotNames[numSnapshots - 1] = inProgressSnapshot;
+        // block a node so the create snapshot operation can remain in progress
+        final String blockedNode = blockNodeWithIndex(repositoryName, indexName);
+        client.admin().cluster().prepareCreateSnapshot(repositoryName, inProgressSnapshot)
+                .setWaitForCompletion(false)
+                .setIndices(indexName)
+                .get();
+        waitForBlock(blockedNode, repositoryName, TimeValue.timeValueSeconds(60)); // wait for block to kick in
+
+        logger.info("--> get all snapshots with a current in-progress");
+        // with ignore unavailable set to true, should not throw an exception
+        final List<String> snapshotsToGet = new ArrayList<>();
+        if (randomBoolean()) {
+            // use _current plus the individual names of the finished snapshots
+            snapshotsToGet.add("_current");
+            for (int i = 0; i < numSnapshots - 1; i++) {
+                snapshotsToGet.add(snapshotNames[i]);
+            }
+        } else {
+            snapshotsToGet.add("_all");
+        }
+        getSnapshotsResponse = client.admin().cluster()
+                .prepareGetSnapshots(repositoryName)
+                .setSnapshots(snapshotsToGet.toArray(Strings.EMPTY_ARRAY))
+                .get();
+        List<String> sortedNames = Arrays.asList(snapshotNames);
+        Collections.sort(sortedNames);
+        assertThat(getSnapshotsResponse.getSnapshots(repositoryName).size(), equalTo(numSnapshots));
+        assertThat(getSnapshotsResponse.getSnapshots(repositoryName).stream()
+                .map(s -> s.snapshotId().getName())
+                .sorted()
+                .collect(Collectors.toList()), equalTo(sortedNames));
+
+        getSnapshotsResponse = client.admin().cluster()
+                .prepareGetSnapshots(repositoryName)
+                .addSnapshots(snapshotNames)
+                .get();
+        sortedNames = Arrays.asList(snapshotNames);
+        Collections.sort(sortedNames);
+        assertThat(getSnapshotsResponse.getSnapshots(repositoryName).size(), equalTo(numSnapshots));
+        assertThat(getSnapshotsResponse.getSnapshots(repositoryName).stream()
+                .map(s -> s.snapshotId().getName())
+                .sorted()
+                .collect(Collectors.toList()), equalTo(sortedNames));
+
+        logger.info("--> make sure duplicates are not returned in the response");
+        String regexName = snapshotNames[randomIntBetween(0, numSnapshots - 1)];
+        final int splitPos = regexName.length() / 2;
+        final String firstRegex = regexName.substring(0, splitPos) + "*";
+        final String secondRegex = "*" + regexName.substring(splitPos);
+        getSnapshotsResponse = client.admin().cluster()
+                .prepareGetSnapshots(repositoryName)
+                .addSnapshots(snapshotNames)
+                .addSnapshots(firstRegex, secondRegex)
+                .get();
+        assertThat(getSnapshotsResponse.getSnapshots(repositoryName).size(), equalTo(numSnapshots));
+        assertThat(getSnapshotsResponse.getSnapshots(repositoryName).stream()
+                .map(s -> s.snapshotId().getName())
+                .sorted()
+                .collect(Collectors.toList()), equalTo(sortedNames));
+
+        unblockNode(repositoryName, blockedNode); // unblock node
+        waitForCompletion(repositoryName, inProgressSnapshot, TimeValue.timeValueSeconds(60));
+    }
+
     private static SnapshotIndexShardStatus stateFirstShard(SnapshotStatus snapshotStatus, String indexName) {
         return snapshotStatus.getIndices().get(indexName).getShards().get(0);
     }

Some files were not shown because too many files changed in this diff