瀏覽代碼

Add Searchable Snapshots Cache Stats API to HLRC (#71858)

In #71701 we added a new REST API that provides statistics 
about the searchable snapshots cache on Frozen Tier.

This commit adds the necessary plumbing to expose this API 
in the High Level REST Client. It also exposes the documentation 
of the Mount Snapshot API that was created in #68949 but not 
made accessible.
Tanguy Leroux 4 年之前
父節點
當前提交
571acfe3e3

+ 3 - 0
client/rest-high-level/build.gradle

@@ -116,4 +116,7 @@ testClusters.all {
   extraConfigFile nodeCert.name, nodeCert
   extraConfigFile nodeTrustStore.name, nodeTrustStore
   extraConfigFile pkiTrustCert.name, pkiTrustCert
+
+  setting 'xpack.searchable.snapshot.shared_cache.size', '1mb'
+  setting 'xpack.searchable.snapshot.shared_cache.region_size', '16kb'
 }

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

@@ -10,6 +10,8 @@ package org.elasticsearch.client;
 
 import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse;
+import org.elasticsearch.client.searchable_snapshots.CachesStatsRequest;
+import org.elasticsearch.client.searchable_snapshots.CachesStatsResponse;
 import org.elasticsearch.client.searchable_snapshots.MountSnapshotRequest;
 
 import java.io.IOException;
@@ -74,4 +76,47 @@ public class SearchableSnapshotsClient {
         );
     }
 
+    /**
+     * Executes the cache stats API, which provides statistics about searchable snapshot cache.
+     *
+     *  See <a href="https://www.elastic.co/guide/en/elasticsearch/reference/current/searchable-snapshots-api-cache-stats.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 CachesStatsResponse cacheStats(final CachesStatsRequest request, final RequestOptions options) throws IOException {
+        return restHighLevelClient.performRequestAndParseEntity(
+            request,
+            SearchableSnapshotsRequestConverters::cacheStats,
+            options,
+            CachesStatsResponse::fromXContent,
+            Collections.emptySet()
+        );
+    }
+
+    /**
+     * Asynchronously executes the cache stats API, which provides statistics about searchable snapshot cache.
+     *
+     * @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 cacheStatsAsync(
+        final CachesStatsRequest request,
+        final RequestOptions options,
+        final ActionListener<CachesStatsResponse> listener)
+    {
+        return restHighLevelClient.performRequestAsyncAndParseEntity(
+            request,
+            SearchableSnapshotsRequestConverters::cacheStats,
+            options,
+            CachesStatsResponse::fromXContent,
+            listener,
+            Collections.emptySet()
+        );
+    }
 }

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

@@ -8,7 +8,9 @@
 
 package org.elasticsearch.client;
 
+import org.apache.http.client.methods.HttpGet;
 import org.apache.http.client.methods.HttpPost;
+import org.elasticsearch.client.searchable_snapshots.CachesStatsRequest;
 import org.elasticsearch.client.searchable_snapshots.MountSnapshotRequest;
 
 import java.io.IOException;
@@ -41,4 +43,13 @@ final class SearchableSnapshotsRequestConverters {
         return request;
     }
 
+    static Request cacheStats(final CachesStatsRequest cacheStatsRequest) {
+        final RequestConverters.EndpointBuilder endpoint = new RequestConverters.EndpointBuilder()
+            .addPathPartAsIs("_searchable_snapshots");
+        if (cacheStatsRequest.getNodesIds() != null) {
+            endpoint.addCommaSeparatedPathParts(cacheStatsRequest.getNodesIds());
+        }
+        endpoint.addPathPartAsIs("cache", "stats");
+        return new Request(HttpGet.METHOD_NAME, endpoint.build());
+    }
 }

+ 41 - 0
client/rest-high-level/src/main/java/org/elasticsearch/client/searchable_snapshots/CachesStatsRequest.java

@@ -0,0 +1,41 @@
+/*
+ * 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.client.ValidationException;
+
+import java.util.Optional;
+
+public class CachesStatsRequest implements Validatable {
+
+    private final String[] nodesIds;
+
+    public CachesStatsRequest(String... nodesIds) {
+        this.nodesIds = nodesIds;
+    }
+
+    public String[] getNodesIds() {
+        return nodesIds;
+    }
+
+    @Override
+    public Optional<ValidationException> validate() {
+        if (nodesIds != null) {
+            for (String nodeId : nodesIds) {
+                if (nodeId == null || nodeId.isEmpty()) {
+                    final ValidationException validation = new ValidationException();
+                    validation.addValidationError("Node ids cannot be null or empty");
+                    return Optional.of(validation);
+                }
+            }
+        }
+        return Optional.empty();
+    }
+}

+ 177 - 0
client/rest-high-level/src/main/java/org/elasticsearch/client/searchable_snapshots/CachesStatsResponse.java

@@ -0,0 +1,177 @@
+/*
+ * 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.common.ParseField;
+import org.elasticsearch.common.xcontent.ConstructingObjectParser;
+import org.elasticsearch.common.xcontent.XContentParser;
+
+import java.util.List;
+
+import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg;
+
+public class CachesStatsResponse {
+
+    private final List<NodeCachesStats> nodeCachesStats;
+
+    private CachesStatsResponse(List<NodeCachesStats> nodeCachesStats) {
+        this.nodeCachesStats = nodeCachesStats != null ? nodeCachesStats : List.of();
+    }
+
+    public List<NodeCachesStats> getNodeCachesStats() {
+        return nodeCachesStats;
+    }
+
+    @SuppressWarnings("unchecked")
+    private static final ConstructingObjectParser<CachesStatsResponse, Void> PARSER = new ConstructingObjectParser<>(
+        "caches_stats_response", true, args -> new CachesStatsResponse((List<NodeCachesStats>) args[0]));
+    static {
+        PARSER.declareNamedObjects(constructorArg(), (p, c, nodeId) -> NodeCachesStats.PARSER.apply(p, nodeId), new ParseField("nodes"));
+    }
+
+    public static CachesStatsResponse fromXContent(XContentParser parser) {
+        return PARSER.apply(parser, null);
+    }
+
+    public static class NodeCachesStats {
+
+        private final String nodeId;
+        private final SharedCacheStats sharedCacheStats;
+
+        public NodeCachesStats(String nodeId, SharedCacheStats sharedCacheStats) {
+            this.nodeId = nodeId;
+            this.sharedCacheStats = sharedCacheStats;
+        }
+
+        public String getNodeId() {
+            return nodeId;
+        }
+
+        public SharedCacheStats getSharedCacheStats() {
+            return sharedCacheStats;
+        }
+
+        private static final ConstructingObjectParser<NodeCachesStats, String> PARSER = new ConstructingObjectParser<>(
+            "node_caches_stats", true, (args, nodeId) -> new NodeCachesStats(nodeId, (SharedCacheStats) args[0]));
+        static {
+            PARSER.declareObject(constructorArg(), (p, c) -> SharedCacheStats.fromXContent(p), new ParseField("shared_cache"));
+        }
+
+        public static NodeCachesStats fromXContent(XContentParser parser, String nodeId) {
+            return PARSER.apply(parser, nodeId);
+        }
+    }
+
+    public static class SharedCacheStats {
+
+        private final int numRegions;
+        private final long size;
+        private final long regionSize;
+        private final long writes;
+        private final long bytesWritten;
+        private final long reads;
+        private final long bytesRead;
+        private final long evictions;
+
+        SharedCacheStats(
+            int numRegions,
+            long size,
+            long regionSize,
+            long writes,
+            long bytesWritten,
+            long reads,
+            long bytesRead,
+            long evictions
+        ) {
+            this.numRegions = numRegions;
+            this.size = size;
+            this.regionSize = regionSize;
+            this.writes = writes;
+            this.bytesWritten = bytesWritten;
+            this.reads = reads;
+            this.bytesRead = bytesRead;
+            this.evictions = evictions;
+        }
+
+        public int getNumRegions() {
+            return numRegions;
+        }
+
+        public long getSize() {
+            return size;
+        }
+
+        public long getRegionSize() {
+            return regionSize;
+        }
+
+        public long getWrites() {
+            return writes;
+        }
+
+        public long getBytesWritten() {
+            return bytesWritten;
+        }
+
+        public long getReads() {
+            return reads;
+        }
+
+        public long getBytesRead() {
+            return bytesRead;
+        }
+
+        public long getEvictions() {
+            return evictions;
+        }
+
+        private static final ConstructingObjectParser<SharedCacheStats, Void> PARSER = new ConstructingObjectParser<>(
+            "shared_cache_stats",
+            true,
+            args -> new SharedCacheStats(
+                (int) args[0],
+                (long) args[1],
+                (long) args[2],
+                (long) args[3],
+                (long) args[4],
+                (long) args[5],
+                (long) args[6],
+                (long) args[7]
+            )
+        );
+        static {
+            PARSER.declareInt(constructorArg(), new ParseField("num_regions"));
+            PARSER.declareLong(constructorArg(), new ParseField("size_in_bytes"));
+            PARSER.declareLong(constructorArg(), new ParseField("region_size_in_bytes"));
+            PARSER.declareLong(constructorArg(), new ParseField("writes"));
+            PARSER.declareLong(constructorArg(), new ParseField("bytes_written_in_bytes"));
+            PARSER.declareLong(constructorArg(), new ParseField("reads"));
+            PARSER.declareLong(constructorArg(), new ParseField("bytes_read_in_bytes"));
+            PARSER.declareLong(constructorArg(), new ParseField("evictions"));
+        }
+
+        public static SharedCacheStats fromXContent(XContentParser parser) {
+            return PARSER.apply(parser, null);
+        }
+
+        @Override
+        public String toString() {
+            return "SharedCacheStats{" +
+                "numRegions=" + numRegions +
+                ", size=" + size +
+                ", regionSize=" + regionSize +
+                ", writes=" + writes +
+                ", bytesWritten=" + bytesWritten +
+                ", reads=" + reads +
+                ", bytesRead=" + bytesRead +
+                ", evictions=" + evictions +
+                '}';
+        }
+    }
+}

+ 77 - 11
client/rest-high-level/src/test/java/org/elasticsearch/client/SearchableSnapshotsIT.java

@@ -12,27 +12,43 @@ import org.elasticsearch.action.admin.cluster.repositories.put.PutRepositoryRequ
 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.admin.indices.delete.DeleteIndexRequest;
+import org.elasticsearch.action.bulk.BulkRequest;
+import org.elasticsearch.action.bulk.BulkResponse;
 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.CachesStatsRequest;
+import org.elasticsearch.client.searchable_snapshots.CachesStatsResponse;
 import org.elasticsearch.client.searchable_snapshots.MountSnapshotRequest;
+import org.elasticsearch.common.unit.ByteSizeUnit;
 import org.elasticsearch.common.xcontent.XContentType;
+import org.elasticsearch.index.query.QueryBuilders;
 import org.elasticsearch.repositories.fs.FsRepository;
 import org.elasticsearch.rest.RestStatus;
+import org.elasticsearch.search.builder.SearchSourceBuilder;
+import org.junit.Before;
 
 import java.io.IOException;
+import java.util.List;
 
-import static org.hamcrest.Matchers.anEmptyMap;
+import static org.hamcrest.Matchers.aMapWithSize;
+import static org.hamcrest.Matchers.emptyOrNullString;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.greaterThan;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
 import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.notNullValue;
 
 public class SearchableSnapshotsIT extends ESRestHighLevelClientTestCase {
 
-    public void testMountSnapshot() throws IOException {
+    @Before
+    public void init() throws Exception {
         {
             final CreateIndexRequest request = new CreateIndexRequest("index");
             final CreateIndexResponse response = highLevelClient().indices().create(request, RequestOptions.DEFAULT);
@@ -40,11 +56,14 @@ public class SearchableSnapshotsIT extends ESRestHighLevelClientTestCase {
         }
 
         {
-            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 BulkRequest request = new BulkRequest().setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE);
+            for (int i = 0; i < 100; i++) {
+                request.add(new IndexRequest("index")
+                    .source(XContentType.JSON, "num", i, "text", randomAlphaOfLengthBetween(3, 10)));
+            }
+            final BulkResponse response = highLevelClient().bulk(request, RequestOptions.DEFAULT);
+            assertThat(response.status(), is(RestStatus.OK));
+            assertThat(response.hasFailures(), is(false));
         }
 
         {
@@ -57,11 +76,19 @@ public class SearchableSnapshotsIT extends ESRestHighLevelClientTestCase {
 
         {
             final CreateSnapshotRequest request =
-                new CreateSnapshotRequest("repository", "snapshot").waitForCompletion(true);
+                new CreateSnapshotRequest("repository", "snapshot").waitForCompletion(true).includeGlobalState(false);
             final CreateSnapshotResponse response = highLevelClient().snapshot().create(request, RequestOptions.DEFAULT);
             assertThat(response.getSnapshotInfo().status(), is(RestStatus.OK));
         }
 
+        {
+            final DeleteIndexRequest request = new DeleteIndexRequest("index");
+            final AcknowledgedResponse response = highLevelClient().indices().delete(request, RequestOptions.DEFAULT);
+            assertThat(response.isAcknowledged(), is(true));
+        }
+    }
+
+    public void testMountSnapshot() throws IOException {
         {
             final MountSnapshotRequest request = new MountSnapshotRequest("repository", "snapshot", "index")
                 .waitForCompletion(true)
@@ -74,9 +101,48 @@ public class SearchableSnapshotsIT extends ESRestHighLevelClientTestCase {
         {
             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());
+            assertThat(response.getHits().getTotalHits().value, is(100L));
+            assertThat(response.getHits().getHits()[0].getSourceAsMap(), aMapWithSize(2));
         }
     }
 
+    public void testCacheStats() throws Exception {
+        final SearchableSnapshotsClient client = new SearchableSnapshotsClient(highLevelClient());
+        {
+            final MountSnapshotRequest request = new MountSnapshotRequest("repository", "snapshot", "index")
+                .waitForCompletion(true)
+                .renamedIndex("mounted_index")
+                .storage(MountSnapshotRequest.Storage.SHARED_CACHE);
+            final RestoreSnapshotResponse response = execute(request, client::mountSnapshot, client::mountSnapshotAsync);
+            assertThat(response.getRestoreInfo().successfulShards(), is(1));
+        }
+
+        {
+            final SearchRequest request = new SearchRequest("mounted_index")
+                .source(new SearchSourceBuilder().query(QueryBuilders.rangeQuery("num").from(50)));
+            final SearchResponse response = highLevelClient().search(request, RequestOptions.DEFAULT);
+            assertThat(response.getHits().getTotalHits().value, is(50L));
+            assertThat(response.getHits().getHits()[0].getSourceAsMap(), aMapWithSize(2));
+        }
+
+        {
+            final CachesStatsRequest request = new CachesStatsRequest();
+            final CachesStatsResponse response = execute(request, client::cacheStats, client::cacheStatsAsync);
+
+            final List<CachesStatsResponse.NodeCachesStats> nodesCachesStats = response.getNodeCachesStats();
+            assertThat(nodesCachesStats, notNullValue());
+            assertThat(nodesCachesStats.size(), equalTo(1));
+            assertThat(nodesCachesStats.get(0).getNodeId(), not(emptyOrNullString()));
+
+            final CachesStatsResponse.SharedCacheStats stats = nodesCachesStats.get(0).getSharedCacheStats();
+            assertThat(stats.getNumRegions(), equalTo(64));
+            assertThat(stats.getSize(), equalTo(ByteSizeUnit.MB.toBytes(1L)));
+            assertThat(stats.getRegionSize(), equalTo(ByteSizeUnit.KB.toBytes(16L)));
+            assertThat(stats.getWrites(), greaterThanOrEqualTo(1L));
+            assertThat(stats.getBytesWritten(), greaterThan(0L));
+            assertThat(stats.getReads(), greaterThanOrEqualTo(1L));
+            assertThat(stats.getBytesRead(), greaterThan(0L));
+            assertThat(stats.getEvictions(), equalTo(0L));
+        }
+    }
 }

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

@@ -8,7 +8,9 @@
 
 package org.elasticsearch.client;
 
+import org.apache.http.client.methods.HttpGet;
 import org.apache.http.client.methods.HttpPost;
+import org.elasticsearch.client.searchable_snapshots.CachesStatsRequest;
 import org.elasticsearch.client.searchable_snapshots.MountSnapshotRequest;
 import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.test.ESTestCase;
@@ -55,4 +57,17 @@ public class SearchableSnapshotsRequestConvertersTests extends ESTestCase {
         RequestConvertersTests.assertToXContentBody(request, result.getEntity());
     }
 
+    public void testCachesStats() throws IOException  {
+        {
+            final Request request = SearchableSnapshotsRequestConverters.cacheStats(new CachesStatsRequest());
+            assertThat(request.getMethod(), equalTo(HttpGet.METHOD_NAME));
+            assertThat(request.getEndpoint(), equalTo("/_searchable_snapshots/cache/stats"));
+        }
+        {
+            final String[] nodesIds = generateRandomStringArray(10, 5, false, false);
+            final Request request = SearchableSnapshotsRequestConverters.cacheStats(new CachesStatsRequest(nodesIds));
+            assertThat(request.getMethod(), equalTo(HttpGet.METHOD_NAME));
+            assertThat(request.getEndpoint(), equalTo("/_searchable_snapshots/" + String.join(",", nodesIds) + "/cache/stats"));
+        }
+    }
 }

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

@@ -23,6 +23,9 @@ 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.CachesStatsRequest;
+import org.elasticsearch.client.searchable_snapshots.CachesStatsResponse;
+import org.elasticsearch.client.searchable_snapshots.CachesStatsResponse.NodeCachesStats;
 import org.elasticsearch.client.searchable_snapshots.MountSnapshotRequest;
 import org.elasticsearch.common.settings.Settings;
 import org.elasticsearch.common.unit.TimeValue;
@@ -32,6 +35,7 @@ import org.elasticsearch.rest.RestStatus;
 import org.elasticsearch.snapshots.RestoreInfo;
 
 import java.io.IOException;
+import java.util.List;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
@@ -130,4 +134,56 @@ public class SearchableSnapshotsDocumentationIT extends ESRestHighLevelClientTes
         assertTrue(latch.await(30L, TimeUnit.SECONDS));
     }
 
+    public void testCachesStatsSnapshot() throws Exception {
+        final RestHighLevelClient client = highLevelClient();
+
+        // tag::searchable-snapshots-caches-stats-request
+        CachesStatsRequest request = new CachesStatsRequest(); // <1>
+        request = new CachesStatsRequest(  // <2>
+            "eerrtBMtQEisohZzxBLUSw",
+            "klksqQSSzASDqDMLQ"
+        );
+        // end::searchable-snapshots-caches-stats-request
+
+        // tag::searchable-snapshots-caches-stats-execute
+        final CachesStatsResponse response = client
+            .searchableSnapshots()
+            .cacheStats(request, RequestOptions.DEFAULT);
+        // end::searchable-snapshots-caches-stats-execute
+
+        // tag::searchable-snapshots-caches-stats-response
+        final List<NodeCachesStats> nodeCachesStats =
+            response.getNodeCachesStats(); // <1>
+        // end::searchable-snapshots-caches-stats-response
+
+        // tag::searchable-snapshots-caches-stats-execute-listener
+        ActionListener<CachesStatsResponse> listener =
+            new ActionListener<CachesStatsResponse>() {
+
+                @Override
+                public void onResponse(final CachesStatsResponse response) {
+                    // <1>
+                }
+
+                @Override
+                public void onFailure(final Exception e) {
+                    // <2>
+                }
+            };
+        // end::searchable-snapshots-caches-stats-execute-listener
+
+        final CountDownLatch latch = new CountDownLatch(1);
+        listener = new LatchedActionListener<>(listener, latch);
+
+        // tag::searchable-snapshots-caches-stats-execute-async
+        client.searchableSnapshots().cacheStatsAsync(
+            request,
+            RequestOptions.DEFAULT,
+            listener // <1>
+        );
+        // end::searchable-snapshots-caches-stats-execute-async
+
+        assertTrue(latch.await(30L, TimeUnit.SECONDS));
+    }
+
 }

+ 33 - 0
docs/java-rest/high-level/searchable_snapshots/caches_stats.asciidoc

@@ -0,0 +1,33 @@
+--
+:api: searchable-snapshots-caches-stats
+:request: CachesStatsRequest
+:response: CachesStatsResponse
+--
+[role="xpack"]
+[id="{upid}-{api}"]
+=== Cache Stats API
+
+[id="{upid}-{api}-request"]
+==== Request
+
+The Cache Stats API provides statistics about searchable snapshot shared cache.
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests-file}[{api}-request]
+--------------------------------------------------
+<1> Example of a request targeting all data nodes.
+<2> Example of a request targeting two specific nodes.
+
+[id="{upid}-{api}-response"]
+==== Response
+
+The returned +{response}+ provides the following statistics:
+
+["source","java",subs="attributes,callouts,macros"]
+--------------------------------------------------
+include-tagged::{doc-tests-file}[{api}-response]
+--------------------------------------------------
+<1> The list of cache statistics for all nodes involved in the request.
+
+include::../execution.asciidoc[]

+ 14 - 0
docs/java-rest/high-level/supported-apis.asciidoc

@@ -689,6 +689,20 @@ include::ilm/snapshot_lifecycle_management_status.asciidoc[]
 include::ilm/execute_snapshot_lifecycle_policy.asciidoc[]
 include::ilm/execute_snapshot_lifecycle_retention.asciidoc[]
 
+[role="xpack"]
+== Searchable Snapshots APIs
+
+:upid: {mainid}-searchable-snapshots
+:doc-tests-file: {doc-tests}/SearchableSnapshotsDocumentationIT.java
+
+The Java High Level REST Client supports the following Searchable Snapshots APIs:
+
+* <<{upid}-searchable-snapshots-mount-snapshot>>
+* <<{upid}-searchable-snapshots-caches-stats>>
+
+
+include::searchable_snapshots/mount_snapshot.asciidoc[]
+include::searchable_snapshots/caches_stats.asciidoc[]
 
 [role="xpack"]
 [[transform_apis]]