Browse Source

Add Searchable Snapshots APIs to HLRC (#68949)

This commit adds the Searchable Snapshots APIs (the mount snapshot API)
to the high-level REST client.
Jason Tedor 4 years ago
parent
commit
985ae9a1ae

+ 11 - 0
client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java

@@ -264,6 +264,7 @@ public class RestHighLevelClient implements Closeable {
     private final EqlClient eqlClient = new EqlClient(this);
     private final AsyncSearchClient asyncSearchClient = new AsyncSearchClient(this);
     private final TextStructureClient textStructureClient = new TextStructureClient(this);
+    private final SearchableSnapshotsClient searchableSnapshotsClient = new SearchableSnapshotsClient(this);
 
     /**
      * Creates a {@link RestHighLevelClient} given the low level {@link RestClientBuilder} that allows to build the
@@ -454,6 +455,16 @@ public class RestHighLevelClient implements Closeable {
         return textStructureClient;
     }
 
+    /**
+     * A wrapper for the {@link RestHighLevelClient} that provides methods for accessing the Searchable Snapshots APIs.
+     * <p>
+     * See the <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/searchable-snapshots-apis.html">Searchable Snapshots
+     * APIs on elastic.co</a> for more information.
+     */
+    public SearchableSnapshotsClient searchableSnapshots() {
+        return searchableSnapshotsClient;
+    }
+
     /**
      * Provides methods for accessing the Elastic Licensed Migration APIs that
      * are shipped with the default distribution of Elasticsearch. All of

+ 77 - 0
client/rest-high-level/src/main/java/org/elasticsearch/client/SearchableSnapshotsClient.java

@@ -0,0 +1,77 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0 and the Server Side Public License, v 1; you may not use this file except
+ * in compliance with, at your election, the Elastic License 2.0 or the Server
+ * Side Public License, v 1.
+ */
+
+package org.elasticsearch.client;
+
+import org.elasticsearch.action.ActionListener;
+import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse;
+import org.elasticsearch.client.searchable_snapshots.MountSnapshotRequest;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Objects;
+
+/**
+ * A wrapper for the {@link RestHighLevelClient} that provides methods for accessing searchable snapshots APIs.
+ *
+ * See the <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/searchable-snapshots-apis.html">Searchable Snapshots
+ * APIs on elastic.co</a> for more information.
+ */
+public class SearchableSnapshotsClient {
+
+    private RestHighLevelClient restHighLevelClient;
+
+    public SearchableSnapshotsClient(final RestHighLevelClient restHighLevelClient) {
+        this.restHighLevelClient = Objects.requireNonNull(restHighLevelClient);
+    }
+
+    /**
+     * Executes the mount snapshot API, which mounts a snapshot as a searchable snapshot.
+     *
+     *  See <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/searchable-snapshots-api-mount-snapshot.html"> the
+     *  docs</a> for more information.
+     *
+     * @param request the request
+     * @param options the request options
+     * @return the response
+     * @throws IOException if an I/O exception occurred sending the request, or receiving or parsing the response
+     */
+    public RestoreSnapshotResponse mountSnapshot(final MountSnapshotRequest request, final RequestOptions options) throws IOException {
+        return restHighLevelClient.performRequestAndParseEntity(
+            request,
+            SearchableSnapshotsRequestConverters::mountSnapshot,
+            options,
+            RestoreSnapshotResponse::fromXContent,
+            Collections.emptySet()
+        );
+    }
+
+    /**
+     * Asynchronously executes the mount snapshot API, which mounts a snapshot as a searchable snapshot.
+     *
+     * @param request the request
+     * @param options the request options
+     * @param listener the listener to be notified upon request completion
+     * @return cancellable that may be used to cancel the request
+     */
+    public Cancellable mountSnapshotAsync(
+        final MountSnapshotRequest request,
+        final RequestOptions options,
+        final ActionListener<RestoreSnapshotResponse> listener)
+    {
+        return restHighLevelClient.performRequestAsyncAndParseEntity(
+            request,
+            SearchableSnapshotsRequestConverters::mountSnapshot,
+            options,
+            RestoreSnapshotResponse::fromXContent,
+            listener,
+            Collections.emptySet()
+        );
+    }
+
+}

+ 44 - 0
client/rest-high-level/src/main/java/org/elasticsearch/client/SearchableSnapshotsRequestConverters.java

@@ -0,0 +1,44 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0 and the Server Side Public License, v 1; you may not use this file except
+ * in compliance with, at your election, the Elastic License 2.0 or the Server
+ * Side Public License, v 1.
+ */
+
+package org.elasticsearch.client;
+
+import org.apache.http.client.methods.HttpPost;
+import org.elasticsearch.client.searchable_snapshots.MountSnapshotRequest;
+
+import java.io.IOException;
+
+import static org.elasticsearch.client.RequestConverters.REQUEST_BODY_CONTENT_TYPE;
+import static org.elasticsearch.client.RequestConverters.createEntity;
+
+final class SearchableSnapshotsRequestConverters {
+
+    static Request mountSnapshot(final MountSnapshotRequest mountSnapshotRequest) throws IOException {
+        final String endpoint = new RequestConverters.EndpointBuilder()
+            .addPathPartAsIs("_snapshot")
+            .addPathPart(mountSnapshotRequest.getRepository())
+            .addPathPart(mountSnapshotRequest.getSnapshot())
+            .addPathPartAsIs("_mount")
+            .build();
+        final Request request = new Request(HttpPost.METHOD_NAME, endpoint);
+        final RequestConverters.Params parameters = new RequestConverters.Params();
+        if (mountSnapshotRequest.getMasterTimeout() != null) {
+            parameters.withMasterTimeout(mountSnapshotRequest.getMasterTimeout());
+        }
+        if (mountSnapshotRequest.getWaitForCompletion() != null) {
+            parameters.withWaitForCompletion(mountSnapshotRequest.getWaitForCompletion());
+        }
+        if (mountSnapshotRequest.getStorage() != null) {
+            parameters.putParam("storage", mountSnapshotRequest.getStorage().storageName());
+        }
+        request.addParameters(parameters.asMap());
+        request.setEntity(createEntity(mountSnapshotRequest, REQUEST_BODY_CONTENT_TYPE));
+        return request;
+    }
+
+}

+ 156 - 0
client/rest-high-level/src/main/java/org/elasticsearch/client/searchable_snapshots/MountSnapshotRequest.java

@@ -0,0 +1,156 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0 and the Server Side Public License, v 1; you may not use this file except
+ * in compliance with, at your election, the Elastic License 2.0 or the Server
+ * Side Public License, v 1.
+ */
+
+package org.elasticsearch.client.searchable_snapshots;
+
+import org.elasticsearch.client.Validatable;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.common.xcontent.ToXContentObject;
+import org.elasticsearch.common.xcontent.XContentBuilder;
+
+import java.io.IOException;
+import java.util.Objects;
+
+public class MountSnapshotRequest implements ToXContentObject, Validatable {
+
+    private final String repository;
+
+    public String getRepository() {
+        return repository;
+    }
+
+    private final String snapshot;
+
+    public String getSnapshot() {
+        return snapshot;
+    }
+
+    private final String index;
+
+    public String getIndex() {
+        return index;
+    }
+
+    public MountSnapshotRequest(final String repository, final String snapshot, final String index) {
+        this.repository = Objects.requireNonNull(repository);
+        this.snapshot = Objects.requireNonNull(snapshot);
+        this.index = Objects.requireNonNull(index);
+    }
+
+    private TimeValue masterTimeout;
+
+    public TimeValue getMasterTimeout() {
+        return masterTimeout;
+    }
+
+    public MountSnapshotRequest masterTimeout(final TimeValue masterTimeout) {
+        this.masterTimeout = masterTimeout;
+        return this;
+    }
+
+    private Boolean waitForCompletion;
+
+    public Boolean getWaitForCompletion() {
+        return waitForCompletion;
+    }
+
+    public MountSnapshotRequest waitForCompletion(final boolean waitForCompletion) {
+        this.waitForCompletion = waitForCompletion;
+        return this;
+    }
+
+    public enum Storage {
+
+        FULL_COPY("full_copy"),
+        SHARED_CACHE("shared_cache");
+
+        private final String storageName;
+
+        public String storageName() {
+            return storageName;
+        }
+
+        Storage(final String storageName) {
+            this.storageName = storageName;
+        }
+
+    }
+
+    private Storage storage;
+
+    public Storage getStorage() {
+        return storage;
+    }
+
+    public MountSnapshotRequest storage(final Storage storage) {
+        this.storage = storage;
+        return this;
+    }
+
+    private String renamedIndex;
+
+    public String getRenamedIndex() {
+        return renamedIndex;
+    }
+
+    public MountSnapshotRequest renamedIndex(final String renamedIndex) {
+        this.renamedIndex = renamedIndex;
+        return this;
+    }
+
+    private Settings indexSettings;
+
+    public Settings getIndexSettings() {
+        return indexSettings;
+    }
+
+    public MountSnapshotRequest indexSettings(final Settings indexSettings) {
+        this.indexSettings = indexSettings;
+        return this;
+    }
+
+    private String[] ignoredIndexSettings;
+
+    public String[] getIgnoredIndexSettings() {
+        return ignoredIndexSettings;
+    }
+
+    public MountSnapshotRequest ignoredIndexSettings(final String[] ignoredIndexSettings) {
+        if (ignoredIndexSettings != null) {
+            for (final String ignoredIndexSetting : ignoredIndexSettings) {
+                Objects.requireNonNull(ignoredIndexSetting);
+            }
+        }
+        this.ignoredIndexSettings = ignoredIndexSettings;
+        return this;
+    }
+
+    @Override
+    public XContentBuilder toXContent(final XContentBuilder builder, final Params params) throws IOException {
+        builder.startObject();
+        {
+            builder.field("index", index);
+            if (renamedIndex != null) {
+                builder.field("renamed_index", renamedIndex);
+            }
+            if (indexSettings != null) {
+                builder.startObject("index_settings"); {
+                    indexSettings.toXContent(builder, params);
+                }
+                builder.endObject();
+            }
+            if (ignoredIndexSettings != null) {
+                builder.array("ignored_index_settings", ignoredIndexSettings);
+            }
+        }
+        builder.endObject();
+        return builder;
+    }
+
+}

+ 1 - 0
client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientTests.java

@@ -918,6 +918,7 @@ public class RestHighLevelClientTests extends ESTestCase {
                                 apiName.startsWith("enrich.") == false &&
                                 apiName.startsWith("transform.") == false &&
                                 apiName.startsWith("text_structure.") == false &&
+                                apiName.startsWith("searchable_snapshots.") == false &&
                                 apiName.startsWith("eql.") == false &&
                                 apiName.endsWith("freeze") == false &&
                                 apiName.endsWith("reload_analyzers") == false &&

+ 82 - 0
client/rest-high-level/src/test/java/org/elasticsearch/client/SearchableSnapshotsIT.java

@@ -0,0 +1,82 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0 and the Server Side Public License, v 1; you may not use this file except
+ * in compliance with, at your election, the Elastic License 2.0 or the Server
+ * Side Public License, v 1.
+ */
+
+package org.elasticsearch.client;
+
+import org.elasticsearch.action.admin.cluster.repositories.put.PutRepositoryRequest;
+import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotRequest;
+import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse;
+import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse;
+import org.elasticsearch.action.index.IndexRequest;
+import org.elasticsearch.action.index.IndexResponse;
+import org.elasticsearch.action.search.SearchRequest;
+import org.elasticsearch.action.search.SearchResponse;
+import org.elasticsearch.action.support.WriteRequest;
+import org.elasticsearch.action.support.master.AcknowledgedResponse;
+import org.elasticsearch.client.indices.CreateIndexRequest;
+import org.elasticsearch.client.indices.CreateIndexResponse;
+import org.elasticsearch.client.searchable_snapshots.MountSnapshotRequest;
+import org.elasticsearch.common.xcontent.XContentType;
+import org.elasticsearch.repositories.fs.FsRepository;
+import org.elasticsearch.rest.RestStatus;
+
+import java.io.IOException;
+
+import static org.hamcrest.Matchers.anEmptyMap;
+import static org.hamcrest.Matchers.is;
+
+public class SearchableSnapshotsIT extends ESRestHighLevelClientTestCase {
+
+    public void testMountSnapshot() throws IOException {
+        {
+            final CreateIndexRequest request = new CreateIndexRequest("index");
+            final CreateIndexResponse response = highLevelClient().indices().create(request, RequestOptions.DEFAULT);
+            assertThat(response.isAcknowledged(), is(true));
+        }
+
+        {
+            final IndexRequest request = new IndexRequest("index")
+                .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)
+                .source("{}", XContentType.JSON);
+            final IndexResponse response = highLevelClient().index(request, RequestOptions.DEFAULT);
+            assertThat(response.status(), is(RestStatus.CREATED));
+        }
+
+        {
+            final PutRepositoryRequest request = new PutRepositoryRequest("repository");
+            request.settings("{\"location\": \".\"}", XContentType.JSON);
+            request.type(FsRepository.TYPE);
+            final AcknowledgedResponse response = highLevelClient().snapshot().createRepository(request, RequestOptions.DEFAULT);
+            assertThat(response.isAcknowledged(), is(true));
+        }
+
+        {
+            final CreateSnapshotRequest request =
+                new CreateSnapshotRequest("repository", "snapshot").waitForCompletion(true);
+            final CreateSnapshotResponse response = highLevelClient().snapshot().create(request, RequestOptions.DEFAULT);
+            assertThat(response.getSnapshotInfo().status(), is(RestStatus.OK));
+        }
+
+        {
+            final MountSnapshotRequest request = new MountSnapshotRequest("repository", "snapshot", "index")
+                .waitForCompletion(true)
+                .renamedIndex("renamed_index");
+            final SearchableSnapshotsClient client = new SearchableSnapshotsClient(highLevelClient());
+            final RestoreSnapshotResponse response = execute(request, client::mountSnapshot, client::mountSnapshotAsync);
+            assertThat(response.getRestoreInfo().successfulShards(), is(1));
+        }
+
+        {
+            final SearchRequest request = new SearchRequest("renamed_index");
+            final SearchResponse response = highLevelClient().search(request, RequestOptions.DEFAULT);
+            assertThat(response.getHits().getTotalHits().value, is(1L));
+            assertThat(response.getHits().getHits()[0].getSourceAsMap(), anEmptyMap());
+        }
+    }
+
+}

+ 58 - 0
client/rest-high-level/src/test/java/org/elasticsearch/client/SearchableSnapshotsRequestConvertersTests.java

@@ -0,0 +1,58 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0 and the Server Side Public License, v 1; you may not use this file except
+ * in compliance with, at your election, the Elastic License 2.0 or the Server
+ * Side Public License, v 1.
+ */
+
+package org.elasticsearch.client;
+
+import org.apache.http.client.methods.HttpPost;
+import org.elasticsearch.client.searchable_snapshots.MountSnapshotRequest;
+import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.test.ESTestCase;
+
+import java.io.IOException;
+
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.nullValue;
+
+public class SearchableSnapshotsRequestConvertersTests extends ESTestCase {
+
+    public void testMountSnapshot() throws IOException  {
+        final MountSnapshotRequest request =
+            new MountSnapshotRequest(randomAlphaOfLength(8), randomAlphaOfLength(8), randomAlphaOfLength(8));
+        if (randomBoolean()) {
+            request.masterTimeout(TimeValue.parseTimeValue(randomTimeValue(), "master_timeout"));
+        }
+        if (randomBoolean()) {
+            request.waitForCompletion(randomBoolean());
+        }
+        if (randomBoolean()) {
+            request.storage(randomFrom(MountSnapshotRequest.Storage.FULL_COPY, MountSnapshotRequest.Storage.SHARED_CACHE));
+        }
+        final Request result = SearchableSnapshotsRequestConverters.mountSnapshot(request);
+        assertThat(result.getMethod(), equalTo(HttpPost.METHOD_NAME));
+        assertThat(result.getEndpoint(), equalTo("/_snapshot/" + request.getRepository() + "/" + request.getSnapshot() + "/_mount"));
+        if (request.getMasterTimeout() != null) {
+            final TimeValue expectedValue = request.getMasterTimeout();
+            assertThat(result.getParameters().get("master_timeout"), is(expectedValue.getStringRep()));
+        } else {
+            assertThat(result.getParameters().get("master_timeout"), nullValue());
+        }
+        if (request.getWaitForCompletion() != null) {
+            assertThat(result.getParameters().get("wait_for_completion"), is(Boolean.toString(request.getWaitForCompletion())));
+        } else {
+            assertThat(result.getParameters().get("wait_for_completion"), nullValue());
+        }
+        if (request.getStorage() != null) {
+            assertThat(result.getParameters().get("storage"), is(request.getStorage().storageName()));
+        } else {
+            assertThat(result.getParameters().get("storage"), nullValue());
+        }
+        RequestConvertersTests.assertToXContentBody(request, result.getEntity());
+    }
+
+}

+ 133 - 0
client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/SearchableSnapshotsDocumentationIT.java

@@ -0,0 +1,133 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0 and the Server Side Public License, v 1; you may not use this file except
+ * in compliance with, at your election, the Elastic License 2.0 or the Server
+ * Side Public License, v 1.
+ */
+
+package org.elasticsearch.client.documentation;
+
+import org.elasticsearch.action.ActionListener;
+import org.elasticsearch.action.LatchedActionListener;
+import org.elasticsearch.action.admin.cluster.repositories.put.PutRepositoryRequest;
+import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotRequest;
+import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse;
+import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse;
+import org.elasticsearch.action.index.IndexRequest;
+import org.elasticsearch.action.index.IndexResponse;
+import org.elasticsearch.action.support.WriteRequest;
+import org.elasticsearch.action.support.master.AcknowledgedResponse;
+import org.elasticsearch.client.ESRestHighLevelClientTestCase;
+import org.elasticsearch.client.RequestOptions;
+import org.elasticsearch.client.RestHighLevelClient;
+import org.elasticsearch.client.indices.CreateIndexRequest;
+import org.elasticsearch.client.indices.CreateIndexResponse;
+import org.elasticsearch.client.searchable_snapshots.MountSnapshotRequest;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.common.xcontent.XContentType;
+import org.elasticsearch.repositories.fs.FsRepository;
+import org.elasticsearch.rest.RestStatus;
+import org.elasticsearch.snapshots.RestoreInfo;
+
+import java.io.IOException;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import static org.hamcrest.Matchers.is;
+
+public class SearchableSnapshotsDocumentationIT extends ESRestHighLevelClientTestCase {
+
+    public void testMountSnapshot() throws IOException, InterruptedException {
+        final RestHighLevelClient client = highLevelClient();
+        {
+            final CreateIndexRequest request = new CreateIndexRequest("index");
+            final CreateIndexResponse response = client.indices().create(request, RequestOptions.DEFAULT);
+            assertTrue(response.isAcknowledged());
+        }
+
+        {
+            final IndexRequest request = new IndexRequest("index")
+                .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)
+                .source("{}", XContentType.JSON);
+            final IndexResponse response = client.index(request, RequestOptions.DEFAULT);
+            assertThat(response.status(), is(RestStatus.CREATED));
+        }
+
+        {
+            final PutRepositoryRequest request = new PutRepositoryRequest("repository");
+            request.settings("{\"location\": \".\"}", XContentType.JSON);
+            request.type(FsRepository.TYPE);
+            final AcknowledgedResponse response = client.snapshot().createRepository(request, RequestOptions.DEFAULT);
+            assertTrue(response.isAcknowledged());
+        }
+
+        {
+            final CreateSnapshotRequest request =
+                new CreateSnapshotRequest("repository", "snapshot").waitForCompletion(true);
+            final CreateSnapshotResponse response = client.snapshot().create(request, RequestOptions.DEFAULT);
+            assertThat(response.getSnapshotInfo().status(), is(RestStatus.OK));
+        }
+
+        // tag::searchable-snapshots-mount-snapshot-request
+        final MountSnapshotRequest request = new MountSnapshotRequest(
+            "repository", // <1>
+            "snapshot", // <2>
+            "index" // <3>
+        );
+        request.masterTimeout(TimeValue.timeValueSeconds(30)); // <4>
+        request.waitForCompletion(true); // <5>
+        request.storage(MountSnapshotRequest.Storage.FULL_COPY); // <6>
+        request.renamedIndex("renamed_index"); // <7>
+        final Settings indexSettings = Settings.builder()
+            .put("index.number_of_replicas", 0)
+            .build();
+        request.indexSettings(indexSettings); // <8>
+        request.ignoredIndexSettings(
+            new String[]{"index.refresh_interval"}); // <9>
+        // end::searchable-snapshots-mount-snapshot-request
+
+        // tag::searchable-snapshots-mount-snapshot-execute
+        final RestoreSnapshotResponse response = client
+            .searchableSnapshots()
+            .mountSnapshot(request, RequestOptions.DEFAULT);
+        // end::searchable-snapshots-mount-snapshot-execute
+
+        // tag::searchable-snapshots-mount-snapshot-response
+        final RestoreInfo restoreInfo = response.getRestoreInfo(); // <1>
+        // end::searchable-snapshots-mount-snapshot-response
+
+        // tag::searchable-snapshots-mount-snapshot-execute-listener
+        ActionListener<RestoreSnapshotResponse> listener =
+            new ActionListener<RestoreSnapshotResponse>() {
+
+                @Override
+                public void onResponse(
+                    final RestoreSnapshotResponse response) { // <1>
+                    final RestoreInfo restoreInfo = response.getRestoreInfo();
+                }
+
+                @Override
+                public void onFailure(final Exception e) {
+                    // <2>
+                }
+
+            };
+        // end::searchable-snapshots-mount-snapshot-execute-listener
+
+        final CountDownLatch latch = new CountDownLatch(1);
+        listener = new LatchedActionListener<>(listener, latch);
+
+        // tag::searchable-snapshots-mount-snapshot-execute-async
+        client.searchableSnapshots().mountSnapshotAsync(
+            request,
+            RequestOptions.DEFAULT,
+            listener // <1>
+        );
+        // end::searchable-snapshots-mount-snapshot-execute-async
+
+        assertTrue(latch.await(30L, TimeUnit.SECONDS));
+    }
+
+}

+ 40 - 0
docs/java-rest/high-level/searchable_snapshots/mount_snapshot.asciidoc

@@ -0,0 +1,40 @@
+--
+:api: searchable-snapshots-mount-snapshot
+:request: MountSnapshotRequest
+:response: MountSnapshotResponse
+--
+[role="xpack"]
+[id="{upid}-{api}"]
+=== Mount Snapshot API
+
+[id="{upid}-{api}-request"]
+==== Request
+
+The Mount Snapshot API mounts a snapshot as a searchable snapshot.
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests-file}[{api}-request]
+--------------------------------------------------
+<1> The name of the repository.
+<2> The name of the snapshot.
+<3> The name of the index in the snapshot to mount.
+<4> (optional) The period to wait for a connection to the master node.
+<5> (optional) If true, the request blocks until the operation is complete.
+<6> (optional) The type of local storage to use for the mounted snapshot.
+<7> (optional) The name of the index that will be created.
+<8> (optional) Settings that should be added to the mounted index.
+<9> (optional) Names of settings that should be unset when the index is mounted.
+
+[id="{upid}-{api}-response"]
+==== Response
+
+The returned +{response}+ indicates if the mounted snapshot was restored.
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests-file}[{api}-response]
+--------------------------------------------------
+<1> The restore info of the mounted snapshot.
+
+include::../execution.asciidoc[]