|
@@ -26,6 +26,7 @@ import org.elasticsearch.action.ListenableActionFuture;
|
|
|
import org.elasticsearch.action.admin.cluster.repositories.put.PutRepositoryResponse;
|
|
|
import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse;
|
|
|
import org.elasticsearch.action.admin.cluster.snapshots.delete.DeleteSnapshotResponse;
|
|
|
+import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsRequest;
|
|
|
import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsResponse;
|
|
|
import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse;
|
|
|
import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotIndexShardStage;
|
|
@@ -54,6 +55,7 @@ import org.elasticsearch.cluster.metadata.MetaDataIndexStateService;
|
|
|
import org.elasticsearch.cluster.metadata.SnapshotId;
|
|
|
import org.elasticsearch.cluster.routing.allocation.decider.FilterAllocationDecider;
|
|
|
import org.elasticsearch.common.Priority;
|
|
|
+import org.elasticsearch.common.Strings;
|
|
|
import org.elasticsearch.common.collect.ImmutableOpenMap;
|
|
|
import org.elasticsearch.common.settings.Settings;
|
|
|
import org.elasticsearch.common.unit.ByteSizeUnit;
|
|
@@ -77,6 +79,7 @@ import java.util.concurrent.CountDownLatch;
|
|
|
import java.util.concurrent.ExecutionException;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
+import static org.elasticsearch.client.Requests.getSnapshotsRequest;
|
|
|
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.common.settings.Settings.settingsBuilder;
|
|
@@ -2047,4 +2050,53 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
|
|
|
assertThat(ex.getMessage(), containsString("Invalid snapshot name"));
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ public void testListCorruptedSnapshot() throws Exception {
|
|
|
+ Client client = client();
|
|
|
+ Path repo = randomRepoPath();
|
|
|
+ logger.info("--> creating repository at " + repo.toAbsolutePath());
|
|
|
+ assertAcked(client.admin().cluster().preparePutRepository("test-repo")
|
|
|
+ .setType("fs").setSettings(Settings.settingsBuilder()
|
|
|
+ .put("location", repo)
|
|
|
+ .put("chunk_size", randomIntBetween(100, 1000), ByteSizeUnit.BYTES)));
|
|
|
+
|
|
|
+ createIndex("test-idx-1", "test-idx-2", "test-idx-3");
|
|
|
+ ensureYellow();
|
|
|
+ logger.info("--> indexing some data");
|
|
|
+ indexRandom(true,
|
|
|
+ client().prepareIndex("test-idx-1", "doc").setSource("foo", "bar"),
|
|
|
+ client().prepareIndex("test-idx-2", "doc").setSource("foo", "bar"),
|
|
|
+ client().prepareIndex("test-idx-3", "doc").setSource("foo", "bar"));
|
|
|
+
|
|
|
+ logger.info("--> creating 2 snapshots");
|
|
|
+ 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()));
|
|
|
+
|
|
|
+ createSnapshotResponse = client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap-2").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-test-snap-2.dat");
|
|
|
+ try(SeekableByteChannel outChan = Files.newByteChannel(snapshotPath, StandardOpenOption.WRITE)) {
|
|
|
+ outChan.truncate(randomInt(10));
|
|
|
+ }
|
|
|
+
|
|
|
+ logger.info("--> get snapshots request should return both snapshots");
|
|
|
+ List<SnapshotInfo> snapshotInfos = client.admin().cluster()
|
|
|
+ .prepareGetSnapshots("test-repo")
|
|
|
+ .setIgnoreUnavailable(true).get().getSnapshots();
|
|
|
+
|
|
|
+ assertThat(snapshotInfos.size(), equalTo(1));
|
|
|
+ assertThat(snapshotInfos.get(0).state(), equalTo(SnapshotState.SUCCESS));
|
|
|
+ assertThat(snapshotInfos.get(0).name(), equalTo("test-snap-1"));
|
|
|
+
|
|
|
+ try {
|
|
|
+ client.admin().cluster().prepareGetSnapshots("test-repo").setIgnoreUnavailable(false).get().getSnapshots();
|
|
|
+ } catch (SnapshotException ex) {
|
|
|
+ assertThat(ex.snapshot().getRepository(), equalTo("test-repo"));
|
|
|
+ assertThat(ex.snapshot().getSnapshot(), equalTo("test-snap-2"));
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|