From 169f7d177465b45fdf187998e48a74380daf1ab6 Mon Sep 17 00:00:00 2001 From: Michael Peterson Date: Mon, 7 Aug 2023 12:32:06 -0400 Subject: [PATCH 01/11] Add specific cluster error info, shard info and additional metadata for CCS when minimizing roundtrips (#97731) For CCS searches with ccs_minimize_roundtrips=true, when an error is returned, it is unclear which cluster caused the problem. This commit adds additional accounting and error information to the search response for each cluster involved in a cross-cluster search. The _clusters section of the SearchResponse has a new details section added with an entry for each cluster (remote and local). It includes status info, shard accounting counters and error information that are added incrementally as the search happens. The search on each cluster can be in one of 5 states: RUNNING SUCCESSFUL - all shards were successfully searched (successful or skipped) PARTIAL - some shard searches failed, but at least one succeeded and partial data has been returned SKIPPED - no shards were successfully searched (all failed or cluster unavailable) when skip_unavailable=true FAILED - no shards were successfully searched (all failed or cluster unavailable) when skip_unavailable=false A new SearchResponse.Cluster object has been added. Each TransportSearchAction.CCSActionListener (one for each cluster) has a reference to a separate Cluster instance and updates once it gets back information from its cluster. The SearchResponse.Clusters object only uses the new Cluster object for CCS minimize_roundtrips=true. For local-only searches and CCS minimize_roundtrips=false, it uses the current Clusters object as before. Follow on work will change CCS minimize_roundtrips=false to also use the new Cluster model and update state in the _cluster/details section. The Cluster objects are immutable, so a CAS operation is required to swap in new state to the map of Cluster objects held by the `SearchResponse.Clusters` class. This concurrency model is a little bit of overkill for the minimize_roundtrips=true use case, but it will be necessary for supporting minimize_roundtrips=false, since updates there will be done per shard, not per cluster. --- .../search-across-clusters.asciidoc | 190 +++-- .../org/elasticsearch/TransportVersion.java | 3 +- .../action/search/SearchResponse.java | 426 ++++++++-- .../action/search/TransportSearchAction.java | 145 +++- .../action/search/SearchResponseTests.java | 13 +- .../search/TransportSearchActionTests.java | 25 + .../test/AbstractMultiClustersTestCase.java | 37 +- .../search/CrossClusterAsyncSearchIT.java | 785 +++++++++++++++++- .../xpack/search/AsyncSearchTask.java | 4 +- .../xpack/search/MutableSearchResponse.java | 34 +- .../search/AsyncSearchResponseTests.java | 508 +++++++++++- .../search/AsyncStatusResponseTests.java | 110 ++- .../xpack/search/ThrowingQueryBuilder.java | 31 +- .../search/action/AsyncStatusResponse.java | 2 +- 14 files changed, 2080 insertions(+), 233 deletions(-) diff --git a/docs/reference/search/search-your-data/search-across-clusters.asciidoc b/docs/reference/search/search-your-data/search-across-clusters.asciidoc index 6c478ec3ca9e0..d4a37ae03dd9f 100644 --- a/docs/reference/search/search-your-data/search-across-clusters.asciidoc +++ b/docs/reference/search/search-your-data/search-across-clusters.asciidoc @@ -130,15 +130,29 @@ The API returns the following response: "took": 150, "timed_out": false, "_shards": { - "total": 1, - "successful": 1, + "total": 3, + "successful": 3, "failed": 0, "skipped": 0 }, "_clusters": { "total": 1, "successful": 1, - "skipped": 0 + "skipped": 0, + "details": { + "cluster_one": { <1> + "status": "successful", + "indices": "my-index-000001", + "took": 148, + "timed_out": false, + "_shards": { + "total": 3, + "successful": 3, + "skipped": 0, + "failed": 0 + } + } + } }, "hits": { "total" : { @@ -148,7 +162,7 @@ The API returns the following response: "max_score": 1, "hits": [ { - "_index": "cluster_one:my-index-000001", <1> + "_index": "cluster_one:my-index-000001", <2> "_id": "0", "_score": 1, "_source": { @@ -171,10 +185,18 @@ The API returns the following response: // TESTRESPONSE[s/"took": 150/"took": "$body.took"/] // TESTRESPONSE[s/"max_score": 1/"max_score": "$body.hits.max_score"/] // TESTRESPONSE[s/"_score": 1/"_score": "$body.hits.hits.0._score"/] - -<1> The search response body includes the name of the remote cluster in the +// TESTRESPONSE[s/"total": 3/"total": "$body._shards.total"/] +// TESTRESPONSE[s/"successful": 3/"successful": "$body._shards.successful"/] +// TESTRESPONSE[s/"skipped": 0/"skipped": "$body._shards.skipped"/] +// TESTRESPONSE[s/"failed": 3/"failed": "$body._shards.failed"/] +// TESTRESPONSE[s/"took": 148/"took": "$body._clusters.details.cluster_one.took"/] + +<1> The details section shows information about the search on each cluster. +<2> The search response body includes the name of the remote cluster in the `_index` parameter. + + [discrete] [[ccs-search-multi-remote-cluster]] ==== Search multiple remote clusters @@ -208,15 +230,53 @@ The API returns the following response: "timed_out": false, "num_reduce_phases": 4, "_shards": { - "total": 3, - "successful": 3, + "total": 12, + "successful": 12, "failed": 0, "skipped": 0 }, "_clusters": { "total": 3, "successful": 3, - "skipped": 0 + "skipped": 0, + "details": { + "(local)": { <1> + "status": "successful", + "indices": "my-index-000001", + "took": 21, + "timed_out": false, + "_shards": { + "total": 5, + "successful": 5, + "skipped": 0, + "failed": 0 + } + }, + "cluster_one": { + "status": "successful", + "indices": "my-index-000001", + "took": 48, + "timed_out": false, + "_shards": { + "total": 4, + "successful": 4, + "skipped": 0, + "failed": 0 + } + }, + "cluster_two": { + "status": "successful", + "indices": "my-index-000001", + "took": 141, + "timed_out": false, + "_shards": { + "total" : 3, + "successful" : 3, + "skipped": 0, + "failed": 0 + } + } + } }, "hits": { "total" : { @@ -226,7 +286,7 @@ The API returns the following response: "max_score": 1, "hits": [ { - "_index": "my-index-000001", <1> + "_index": "my-index-000001", <2> "_id": "0", "_score": 2, "_source": { @@ -243,7 +303,7 @@ The API returns the following response: } }, { - "_index": "cluster_one:my-index-000001", <2> + "_index": "cluster_one:my-index-000001", <3> "_id": "0", "_score": 1, "_source": { @@ -260,7 +320,7 @@ The API returns the following response: } }, { - "_index": "cluster_two:my-index-000001", <3> + "_index": "cluster_two:my-index-000001", <4> "_id": "0", "_score": 1, "_source": { @@ -284,14 +344,23 @@ The API returns the following response: // TESTRESPONSE[s/"max_score": 1/"max_score": "$body.hits.max_score"/] // TESTRESPONSE[s/"_score": 1/"_score": "$body.hits.hits.0._score"/] // TESTRESPONSE[s/"_score": 2/"_score": "$body.hits.hits.1._score"/] - -<1> This document's `_index` parameter doesn't include a cluster name. This +// TESTRESPONSE[s/"total": 12/"total": "$body._shards.total"/] +// TESTRESPONSE[s/"successful": 12/"successful": "$body._shards.successful"/] +// TESTRESPONSE[s/"total": 5/"total": "$body._clusters.details.(local)._shards.total"/] +// TESTRESPONSE[s/"successful": 5/"successful": "$body._clusters.details.(local)._shards.successful"/] +// TESTRESPONSE[s/"took": 21/"took": "$body._clusters.details.(local).took"/] +// TESTRESPONSE[s/"total": 4/"total": "$body._clusters.details.cluster_one._shards.total"/] +// TESTRESPONSE[s/"successful": 4/"successful": "$body._clusters.details.cluster_one._shards.successful"/] +// TESTRESPONSE[s/"took": 48/"took": "$body._clusters.details.cluster_one.took"/] +// TESTRESPONSE[s/"total" : 3/"total": "$body._clusters.details.cluster_two._shards.total"/] +// TESTRESPONSE[s/"successful" : 3/"successful": "$body._clusters.details.cluster_two._shards.successful"/] +// TESTRESPONSE[s/"took": 141/"took": "$body._clusters.details.cluster_two.took"/] + +<1> The local (querying) cluster is identified as "(local)". +<2> This document's `_index` parameter doesn't include a cluster name. This means the document came from the local cluster. -<2> This document came from `cluster_one`. -<3> This document came from `cluster_two`. - - - +<3> This document came from `cluster_one`. +<4> This document came from `cluster_two`. [discrete] @@ -321,7 +390,7 @@ POST /my-index-000001,cluster_one:my-index-000001,cluster_two:my-index-000001/_a } -------------------------------------------------- // TEST[continued] -// TEST[s/ccs_minimize_roundtrips=true/ccs_minimize_roundtrips=true&wait_for_completion_timeout=1s&keep_on_completion=true/] +// TEST[s/ccs_minimize_roundtrips=true/ccs_minimize_roundtrips=true&wait_for_completion_timeout=100ms&keep_on_completion=true/] The API returns the following response: @@ -360,21 +429,7 @@ The API returns the following response: } } -------------------------------------------------- -// TESTRESPONSE[s/FklQYndoTDJ2VEFlMEVBTzFJMGhJVFEaLVlKYndBWWZSMUdicUc4WVlEaFl4ZzoxNTU=/$body.id/] -// TESTRESPONSE[s/"is_partial": true/"is_partial": $body.is_partial/] -// TESTRESPONSE[s/"is_running": true/"is_running": $body.is_running/] -// TESTRESPONSE[s/1685563581380/$body.start_time_in_millis/] -// TESTRESPONSE[s/1685995581380/$body.expiration_time_in_millis/] -// TESTRESPONSE[s/"response"/"completion_time_in_millis": $body.completion_time_in_millis,\n "response"/] -// TESTRESPONSE[s/"num_reduce_phases": 0/"num_reduce_phases": "$body.response.num_reduce_phases"/] -// TESTRESPONSE[s/"took": 1020/"took": "$body.response.took"/] -// TESTRESPONSE[s/"total": 8/"total": $body.response._shards.total/] -// TESTRESPONSE[s/"successful": 0/"successful": $body.response._shards.successful/] -// TESTRESPONSE[s/"successful" : 0/"successful": $body.response._clusters.successful/] -// TESTRESPONSE[s/"value": 0/"value": "$body.response.hits.total.value"/] -// TESTRESPONSE[s/"max_score": null/"max_score": "$body.response.hits.max_score"/] -// TESTRESPONSE[s/"hits": \[\]/"hits": $body.response.hits.hits/] - +// TEST[skip: terminated_early is absent from final results so is hard to reproduce here] <1> The async search id. <2> When `ccs_minimize_roundtrips` = `true` and searches on the remote clusters @@ -385,7 +440,6 @@ across all clusters only when the search is completed. and all are currently running (since `successful` and `skipped` both equal 0). - If you query the <> endpoint while the query is still running, you will see an update in the `_clusters` and `_shards` section of the response when the local search has finished. @@ -394,7 +448,7 @@ the response when the local search has finished. -------------------------------------------------- GET /_async_search/FklQYndoTDJ2VEFlMEVBTzFJMGhJVFEaLVlKYndBWWZSMUdicUc4WVlEaFl4ZzoxNTU= -------------------------------------------------- -// TEST[skip: terminated_early is absent from final results so is hard to reproduce here] +// TEST[continued s/FklQYndoTDJ2VEFlMEVBTzFJMGhJVFEaLVlKYndBWWZSMUdicUc4WVlEaFl4ZzoxNTU=/\${body.id}/] Response: @@ -443,8 +497,6 @@ until all remote searches have finished (either successfully or been skipped). shown until searches on all clusters have been completed and merged. - - After searches on all the clusters have completed, when you query the <> endpoint, you will see the final status of the `_clusters` and `_shards` section as well as the hits. @@ -480,7 +532,45 @@ Response: "_clusters": { "total": 3, "successful": 3, <3> - "skipped": 0 + "skipped": 0, + "details": { + "(local)": { + "status": "successful", + "indices": "my-index-000001", + "took": 14382, + "timed_out": false, + "_shards": { + "total": 10, + "successful": 10, + "skipped": 0, + "failed": 0 + } + }, + "cluster_one": { + "status": "successful", + "indices": "my-index-000001", + "took": 22193, + "timed_out": false, + "_shards": { + "total": 12, + "successful": 12, + "skipped": 0, + "failed": 0 + } + }, + "cluster_two": { + "status": "successful", + "indices": "my-index-000001", + "took": 27550, + "timed_out": false, + "_shards": { + "total": 6, + "successful": 6, + "skipped": 0, + "failed": 0 + } + } + } }, "hits": { "total": { @@ -500,6 +590,9 @@ Response: // TESTRESPONSE[s/1685996911108/$body.expiration_time_in_millis/] // TESTRESPONSE[s/1685564938727/$body.completion_time_in_millis/] // TESTRESPONSE[s/"took": 27619/"took": "$body.response.took"/] +// TESTRESPONSE[s/"took": 14382/"took": "$body.$_path"/] +// TESTRESPONSE[s/"took": 22193/"took": "$body.$_path"/] +// TESTRESPONSE[s/"took": 27550/"took": "$body.$_path"/] // TESTRESPONSE[s/"total": 28/"total": $body.response._shards.total/] // TESTRESPONSE[s/"successful": 28/"successful": $body.response._shards.successful/] // TESTRESPONSE[s/"successful": 3/"successful": $body.response._clusters.successful/] @@ -507,6 +600,8 @@ Response: // TESTRESPONSE[s/"relation": "eq"/"relation": "$body.response.hits.total.relation"/] // TESTRESPONSE[s/"max_score": 1.8293576/"max_score": "$body.response.hits.max_score"/] // TESTRESPONSE[s/"hits": \[...list of hits here...\]/"hits": $body.response.hits.hits/] +// TESTRESPONSE[s/"total": \d+/"total": $body.$_path/] +// TESTRESPONSE[s/"successful": \d+/"successful": $body.$_path/] <1> Once the search has finished, the completion_time is present. @@ -515,7 +610,6 @@ were searched across all clusters and that all were successful. <3> The `_clusters` section shows that searches on all 3 clusters were successful. - [discrete] [[ccs-async-search-minimize-roundtrips-false]] === Using async search for {ccs} with ccs_minimize_roundtrips=false @@ -540,7 +634,7 @@ Example using the same set up as in the previous section (`ccs_minimize_roundtri [source,console] -------------------------------------------------- -GET /my-index-000001,cluster_one:my-index-000001,cluster_two:my-index-000001/_async_search?ccs_minimize_roundtrips=false +POST /my-index-000001,cluster_one:my-index-000001,cluster_two:my-index-000001/_async_search?ccs_minimize_roundtrips=false { "query": { "match": { @@ -551,7 +645,7 @@ GET /my-index-000001,cluster_one:my-index-000001,cluster_two:my-index-000001/_as } -------------------------------------------------- // TEST[continued] -// TEST[s/ccs_minimize_roundtrips=false/ccs_minimize_roundtrips=false&wait_for_completion_timeout=1s&keep_on_completion=true/] +// TEST[s/ccs_minimize_roundtrips=false/ccs_minimize_roundtrips=false&wait_for_completion_timeout=2s&keep_on_completion=true/] The API returns the following response if the query takes longer than @@ -568,7 +662,6 @@ the `wait_for_completion_timeout` duration (see <>). "response": { "took": 1020, "timed_out": false, - "num_reduce_phases": 0, "_shards": { "total": 28, <1> "successful": 0, @@ -597,13 +690,8 @@ the `wait_for_completion_timeout` duration (see <>). // TESTRESPONSE[s/1685563581380/$body.start_time_in_millis/] // TESTRESPONSE[s/1685995581380/$body.expiration_time_in_millis/] // TESTRESPONSE[s/"response"/"completion_time_in_millis": $body.completion_time_in_millis,\n "response"/] -// TESTRESPONSE[s/"num_reduce_phases": 0/"num_reduce_phases": "$body.response.num_reduce_phases"/] -// TESTRESPONSE[s/"took": 1020/"took": "$body.response.took"/] -// TESTRESPONSE[s/"total": 28/"total": $body.response._shards.total/] -// TESTRESPONSE[s/"successful": 0/"successful": $body.response._shards.successful/] -// TESTRESPONSE[s/"successful": 3/"successful": $body.response._clusters.successful/] -// TESTRESPONSE[s/"value": 0/"value": "$body.response.hits.total.value"/] // TESTRESPONSE[s/"max_score": null/"max_score": "$body.response.hits.max_score"/] +// TESTRESPONSE[s/\d+/$body.$_path/] // TESTRESPONSE[s/"hits": \[\]/"hits": $body.response.hits.hits/] <1> All shards from all clusters in scope for the search are listed here. Watch this @@ -612,8 +700,6 @@ section for updates to monitor search progress. gathered from all 3 clusters and that all will be searched (none are being skipped). - - [discrete] [[skip-unavailable-clusters]] === Optional remote clusters diff --git a/server/src/main/java/org/elasticsearch/TransportVersion.java b/server/src/main/java/org/elasticsearch/TransportVersion.java index cf956f0a89e75..5f8c2d1f96e60 100644 --- a/server/src/main/java/org/elasticsearch/TransportVersion.java +++ b/server/src/main/java/org/elasticsearch/TransportVersion.java @@ -175,9 +175,10 @@ private static TransportVersion registerTransportVersion(int id, String uniqueId public static final TransportVersion V_8_500_050 = registerTransportVersion(8_500_050, "69722fa2-7c0a-4227-86fb-6d6a9a0a0321"); public static final TransportVersion V_8_500_051 = registerTransportVersion(8_500_051, "a28b43bc-bb5f-4406-afcf-26900aa98a71"); public static final TransportVersion V_8_500_052 = registerTransportVersion(8_500_052, "2d382b3d-9838-4cce-84c8-4142113e5c2b"); + public static final TransportVersion V_8_500_053 = registerTransportVersion(8_500_053, "aa603bae-01e2-380a-8950-6604468e8c6d"); private static class CurrentHolder { - private static final TransportVersion CURRENT = findCurrent(V_8_500_052); + private static final TransportVersion CURRENT = findCurrent(V_8_500_053); // finds the pluggable current version, or uses the given fallback private static TransportVersion findCurrent(TransportVersion fallback) { diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchResponse.java b/server/src/main/java/org/elasticsearch/action/search/SearchResponse.java index 4900f0a9ae26f..691dc577b0e61 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchResponse.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchResponse.java @@ -11,6 +11,7 @@ import org.apache.lucene.search.TotalHits; import org.elasticsearch.TransportVersion; import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.Iterators; import org.elasticsearch.common.io.stream.StreamInput; @@ -30,6 +31,7 @@ import org.elasticsearch.search.profile.SearchProfileResults; import org.elasticsearch.search.profile.SearchProfileShardResult; import org.elasticsearch.search.suggest.Suggest; +import org.elasticsearch.transport.RemoteClusterAware; import org.elasticsearch.xcontent.ParseField; import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.ToXContentFragment; @@ -39,10 +41,15 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; import java.util.Iterator; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Objects; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Predicate; import java.util.function.Supplier; import static org.elasticsearch.action.search.ShardSearchFailure.readShardSearchFailure; @@ -465,7 +472,8 @@ public String toString() { /** * Holds info about the clusters that the search was executed on: how many in total, how many of them were successful - * and how many of them were skipped. + * and how many of them were skipped and further details in a Map of Cluster objects + * (when doing a cross-cluster search). */ public static class Clusters implements ToXContentFragment, Writeable { @@ -477,67 +485,88 @@ public static class Clusters implements ToXContentFragment, Writeable { static final ParseField TOTAL_FIELD = new ParseField("total"); private final int total; - private final int successful; - private final int skipped; - // NOTE: these two new fields (remoteClusters and ccsMinimizeRoundtrips) have not been added to the wire protocol - // or equals/hashCode methods. They are needed for CCS only (async-search CCS in particular). If we need to write - // these to the .async-search system index in the future, we may want to refactor Clusters to allow async-search - // to subclass it. - private final transient int remoteClusters; + private final int successful; // not used for minimize_roundtrips=true; dynamically determined from clusterInfo map + private final int skipped; // not used for minimize_roundtrips=true; dynamically determined from clusterInfo map + + // key to map is clusterAlias on the primary querying cluster of a CCS minimize_roundtrips=true query + // the Map itself is immutable after construction - all Clusters will be accounted for at the start of the search + // updates to the Cluster occur by CAS swapping in new Cluster objects into the AtomicReference in the map. + private final Map> clusterInfo; + + // this field is not Writeable, as it is only needed on the initial "querying cluster" coordinator of a CCS search private final transient boolean ccsMinimizeRoundtrips; /** - * A Clusters object meant for use with CCS holding additional information about - * the number of remote clusters and whether ccsMinimizeRoundtrips is being used. - * @param total total number of clusters in the search - * @param successful number of clusters that have successfully completed the search - * @param skipped number of clusters that were skipped (e.g., unavailable or other error) - * @param remoteClusters number of remote clusters in the search - * @param ccsMinimizeRoundtrips specifies whether a CCS search is using minimizeRoundtrips feature + * For use with cross-cluster searches. + * When minimizing roundtrips, the number of successful and skipped clusters is not known until + * the end of the search and it the information in SearchResponse.Cluster object will be updated + * as each cluster returns. + * @param localIndices The localIndices to be searched - null if no local indices are to be searched + * @param remoteClusterIndices mapping of clusterAlias -> OriginalIndices for each remote cluster + * @param ccsMinimizeRoundtrips whether minimizing roundtrips for the CCS */ - public Clusters(int total, int successful, int skipped, int remoteClusters, boolean ccsMinimizeRoundtrips) { - assert total >= 0 && successful >= 0 && skipped >= 0 && remoteClusters >= 0 - : "total: " + total + " successful: " + successful + " skipped: " + skipped + " remote: " + remoteClusters; - assert successful <= total : "total: " + total + " successful: " + successful + " skipped: " + skipped; - assert remoteClusters <= total : "total: " + total + " remote: " + remoteClusters; - assert ccsMinimizeRoundtrips == false || remoteClusters > 0 - : "ccsMinimizeRoundtrips is true but remoteClusters count is not a positive number: " + remoteClusters; - int localCount = total - remoteClusters; - assert localCount == 0 || localCount == 1 : "total - remoteClusters should only be 0 or 1"; - this.total = total; - this.successful = successful; - this.skipped = skipped; - this.remoteClusters = remoteClusters; + public Clusters( + @Nullable OriginalIndices localIndices, + Map remoteClusterIndices, + boolean ccsMinimizeRoundtrips + ) { + this.total = remoteClusterIndices.size() + (localIndices == null ? 0 : 1); + assert total >= 1 : "No local indices or remote clusters passed in"; + this.successful = 0; // calculated from clusterInfo map for minimize_roundtrips + this.skipped = 0; // calculated from clusterInfo map for minimize_roundtrips this.ccsMinimizeRoundtrips = ccsMinimizeRoundtrips; + Map> m = new HashMap<>(); + if (localIndices != null) { + String localKey = RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY; + Cluster c = new Cluster(localKey, String.join(",", localIndices.indices())); + m.put(localKey, new AtomicReference<>(c)); + } + for (Map.Entry remote : remoteClusterIndices.entrySet()) { + String clusterAlias = remote.getKey(); + Cluster c = new Cluster(clusterAlias, String.join(",", remote.getValue().indices())); + m.put(clusterAlias, new AtomicReference<>(c)); + } + this.clusterInfo = Collections.unmodifiableMap(m); } /** - * Assumes ccsMinimizeRoundtrips=false. - * We are not tracking number of remote clusters in this search. + * Used for searches that are either not cross-cluster or CCS with minimize_roundtrips=false. + * For CCS minimize_roundtrips=true use {@code Clusters(OriginalIndices, Map, boolean)} + * @param total total number of clusters in the search + * @param successful number of successful clusters in the search + * @param skipped number of skipped clusters (skipped can only happen for remote clusters with skip_unavailable=true) */ public Clusters(int total, int successful, int skipped) { - this(total, successful, skipped, true); - } - - /** - * @param finalState if true, then do an assert that total = successful + skipped. This is true - * only when the cluster is in its final state, not an initial or intermediate state. - */ - Clusters(int total, int successful, int skipped, boolean finalState) { assert total >= 0 && successful >= 0 && skipped >= 0 && successful <= total : "total: " + total + " successful: " + successful + " skipped: " + skipped; - assert finalState == false || skipped == total - successful - : "total: " + total + " successful: " + successful + " skipped: " + skipped; + assert skipped == total - successful : "total: " + total + " successful: " + successful + " skipped: " + skipped; this.total = total; this.successful = successful; this.skipped = skipped; - this.remoteClusters = -1; // means "unknown" and not needed for this usage this.ccsMinimizeRoundtrips = false; + this.clusterInfo = Collections.emptyMap(); // will never be used if created from this constructor } public Clusters(StreamInput in) throws IOException { - // when coming across the wire, we don't have context to know if this Cluster is in a final state, so set finalState=false - this(in.readVInt(), in.readVInt(), in.readVInt(), false); + this.total = in.readVInt(); + this.successful = in.readVInt(); + this.skipped = in.readVInt(); + if (in.getTransportVersion().onOrAfter(TransportVersion.V_8_500_053)) { + List clusterList = in.readList(Cluster::new); + if (clusterList.isEmpty()) { + this.clusterInfo = Collections.emptyMap(); + } else { + Map> m = new HashMap<>(); + clusterList.forEach(c -> m.put(c.getClusterAlias(), new AtomicReference<>(c))); + this.clusterInfo = Collections.unmodifiableMap(m); + } + } else { + this.clusterInfo = Collections.emptyMap(); + } + this.ccsMinimizeRoundtrips = false; + assert total >= 0 : "total is negative: " + total; + assert total >= successful + skipped + : "successful + skipped is larger than total. total: " + total + " successful: " + successful + " skipped: " + skipped; } @Override @@ -545,6 +574,14 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVInt(total); out.writeVInt(successful); out.writeVInt(skipped); + if (out.getTransportVersion().onOrAfter(TransportVersion.V_8_500_053)) { + if (clusterInfo != null) { + List clusterList = clusterInfo.values().stream().map(AtomicReference::get).toList(); + out.writeList(clusterList); + } else { + out.writeList(Collections.emptyList()); + } + } } @Override @@ -552,8 +589,16 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (total > 0) { builder.startObject(_CLUSTERS_FIELD.getPreferredName()); builder.field(TOTAL_FIELD.getPreferredName(), total); - builder.field(SUCCESSFUL_FIELD.getPreferredName(), successful); - builder.field(SKIPPED_FIELD.getPreferredName(), skipped); + builder.field(SUCCESSFUL_FIELD.getPreferredName(), getSuccessful()); + builder.field(SKIPPED_FIELD.getPreferredName(), getSkipped()); + // TODO: add FAILED_FIELD + if (clusterInfo.size() > 0) { + builder.startObject("details"); + for (AtomicReference cluster : clusterInfo.values()) { + cluster.get().toXContent(builder, params); + } + builder.endObject(); + } builder.endObject(); } return builder; @@ -570,22 +615,37 @@ public int getTotal() { * @return how many total clusters the search was executed successfully on */ public int getSuccessful() { - return successful; + if (clusterInfo.isEmpty()) { + return successful; + } else { + return determineCountFromClusterInfo( + cluster -> cluster.getStatus() == Cluster.Status.SUCCESSFUL || cluster.getStatus() == Cluster.Status.PARTIAL + ); + } } /** - * @return how many total clusters were used during the execution of the search request + * When Clusters is using the clusterInfo map (and Cluster objects are being updated in various + * ActionListener threads), this method will count how many clusters match the passed in predicate. + * + * @param predicate to evaluate + * @return count of clusters matching the predicate */ - public int getSkipped() { - return skipped; + private int determineCountFromClusterInfo(Predicate predicate) { + return (int) clusterInfo.values().stream().filter(c -> predicate.test(c.get())).count(); } /** - * @return how many remote clusters were using during the execution of the search request - * If not set, returns -1, meaning 'unknown'. + * @return how many total clusters were used during the execution of the search request */ - public int getRemoteClusters() { - return remoteClusters; + public int getSkipped() { + if (clusterInfo.isEmpty()) { + return skipped; + } else { + return determineCountFromClusterInfo(cluster -> + // TODO: change this after adding an XContent field for FAILED clusters + cluster.getStatus() == Cluster.Status.SKIPPED || cluster.getStatus() == Cluster.Status.FAILED); + } } /** @@ -595,6 +655,14 @@ public boolean isCcsMinimizeRoundtrips() { return ccsMinimizeRoundtrips; } + /** + * @param clusterAlias The cluster alias as specified in the cluster collection + * @return Cluster object associated with teh clusterAlias or null if not present + */ + public AtomicReference getCluster(String clusterAlias) { + return clusterInfo.get(clusterAlias); + } + @Override public boolean equals(Object o) { if (this == o) { @@ -614,18 +682,246 @@ public int hashCode() { @Override public String toString() { - return "Clusters{total=" + total + ", successful=" + successful + ", skipped=" + skipped + '}'; - } - - public String toStringExtended() { - return Strings.format( - "Clusters{total=%d, successful=%d, skipped=%d, remote=%d, ccsMinimizeRoundtrips=%s}", - total, - successful, - skipped, - remoteClusters, - ccsMinimizeRoundtrips - ); + return "Clusters{total=" + total + ", successful=" + getSuccessful() + ", skipped=" + getSkipped() + '}'; + } + + /** + * @return true if any underlying Cluster objects have PARTIAL, SKIPPED, FAILED or RUNNING status. + */ + public boolean hasPartialResults() { + for (AtomicReference cluster : clusterInfo.values()) { + switch (cluster.get().getStatus()) { + case PARTIAL, SKIPPED, FAILED, RUNNING -> { + return true; + } + } + } + return false; + } + } + + /** + * Represents the search metadata about a particular cluster involved in a cross-cluster search. + * The Cluster object can represent either the local cluster or a remote cluster. + * For the local cluster, clusterAlias should be specified as RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY. + * Its XContent is put into the "details" section the "_clusters" entry in the SearchResponse. + * This is an immutable class, so updates made during the search progress (especially important for async + * CCS searches) must be done by replacing the Cluster object with a new one. + * See the Clusters clusterInfo Map for details. + */ + public static class Cluster implements ToXContentFragment, Writeable { + private final String clusterAlias; + private final String indexExpression; // original index expression from the user for this cluster + private final Status status; + private final Integer totalShards; + private final Integer successfulShards; + private final Integer skippedShards; + private final Integer failedShards; + private final List failures; + private final TimeValue took; // search latency in millis for this cluster sub-search + private final boolean timedOut; + + /** + * Marks the status of a Cluster search involved in a Cross-Cluster search. + */ + public enum Status { + RUNNING, // still running + SUCCESSFUL, // all shards completed search + PARTIAL, // only some shards completed the search, partial results from cluster + SKIPPED, // entire cluster was skipped + FAILED; // search was failed due to errors on this cluster + + @Override + public String toString() { + return this.name().toLowerCase(Locale.ROOT); + } + } + + /** + * Create a Cluster object representing the initial RUNNING state of a Cluster. + * + * @param clusterAlias clusterAlias as defined in the remote cluster settings or RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY + * for the local cluster + * @param indexExpression the original (not resolved/concrete) indices expression provided for this cluster. + */ + public Cluster(String clusterAlias, String indexExpression) { + this(clusterAlias, indexExpression, Status.RUNNING, null, null, null, null, null, null, false); + } + + /** + * Create a Cluster with a new Status and one or more ShardSearchFailures. This constructor + * should only be used for fatal failures where shard counters (total, successful, skipped, failed) + * are not known (unset). + * @param clusterAlias clusterAlias as defined in the remote cluster settings or RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY + * for the local cluster + * @param indexExpression the original (not resolved/concrete) indices expression provided for this cluster. + * @param status current status of the search on this Cluster + * @param failures list of failures that occurred during the search on this Cluster + */ + public Cluster(String clusterAlias, String indexExpression, Status status, List failures) { + this(clusterAlias, indexExpression, status, null, null, null, null, failures, null, false); + } + + public Cluster( + String clusterAlias, + String indexExpression, + Status status, + Integer totalShards, + Integer successfulShards, + Integer skippedShards, + Integer failedShards, + List failures, + TimeValue took, + boolean timedOut + ) { + assert clusterAlias != null : "clusterAlias cannot be null"; + assert indexExpression != null : "indexExpression of Cluster cannot be null"; + assert status != null : "status of Cluster cannot be null"; + this.clusterAlias = clusterAlias; + this.indexExpression = indexExpression; + this.status = status; + this.totalShards = totalShards; + this.successfulShards = successfulShards; + this.skippedShards = skippedShards; + this.failedShards = failedShards; + this.failures = failures == null ? Collections.emptyList() : Collections.unmodifiableList(failures); + this.took = took; + this.timedOut = timedOut; + } + + public Cluster(StreamInput in) throws IOException { + this.clusterAlias = in.readString(); + this.indexExpression = in.readString(); + this.status = Status.valueOf(in.readString().toUpperCase(Locale.ROOT)); + this.totalShards = in.readOptionalInt(); + this.successfulShards = in.readOptionalInt(); + this.skippedShards = in.readOptionalInt(); + this.failedShards = in.readOptionalInt(); + Long took = in.readOptionalLong(); + if (took == null) { + this.took = null; + } else { + this.took = new TimeValue(took); + } + this.timedOut = in.readBoolean(); + this.failures = Collections.unmodifiableList(in.readList(ShardSearchFailure::readShardSearchFailure)); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(clusterAlias); + out.writeString(indexExpression); + out.writeString(status.toString()); + out.writeOptionalInt(totalShards); + out.writeOptionalInt(successfulShards); + out.writeOptionalInt(skippedShards); + out.writeOptionalInt(failedShards); + out.writeOptionalLong(took == null ? null : took.millis()); + out.writeBoolean(timedOut); + out.writeList(failures); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + String name = clusterAlias; + if (clusterAlias.equals("")) { + name = "(local)"; + } + builder.startObject(name); + { + builder.field("status", getStatus().toString()); + builder.field("indices", indexExpression); + if (took != null) { + builder.field("took", took.millis()); + } + builder.field("timed_out", timedOut); + if (totalShards != null) { + builder.startObject("_shards"); + builder.field("total", totalShards); + if (successfulShards != null) { + builder.field("successful", successfulShards); + } + if (skippedShards != null) { + builder.field("skipped", skippedShards); + } + if (failedShards != null) { + builder.field("failed", failedShards); + } + builder.endObject(); + } + if (failures != null && failures.size() > 0) { + builder.startArray("failures"); + for (ShardSearchFailure failure : failures) { + failure.toXContent(builder, params); + } + builder.endArray(); + } + } + builder.endObject(); + return builder; + } + + public String getClusterAlias() { + return clusterAlias; + } + + public String getIndexExpression() { + return indexExpression; + } + + public Status getStatus() { + return status; + } + + public boolean isTimedOut() { + return timedOut; + } + + public List getFailures() { + return failures; + } + + public TimeValue getTook() { + return took; + } + + public Integer getTotalShards() { + return totalShards; + } + + public Integer getSuccessfulShards() { + return successfulShards; + } + + public Integer getSkippedShards() { + return skippedShards; + } + + public Integer getFailedShards() { + return failedShards; + } + + @Override + public String toString() { + return "Cluster{" + + "clusterAlias='" + + clusterAlias + + '\'' + + ", status=" + + status + + ", failures=" + + failures + + ", totalShards=" + + totalShards + + ", successfulShards=" + + successfulShards + + ", skippedShards=" + + skippedShards + + ", failedShards=" + + failedShards + + ", searchLatencyMillis=" + + took + + '}'; } } diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index f80bacc6495db..7249f6f843979 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -8,6 +8,8 @@ package org.elasticsearch.action.search; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.TransportVersion; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListenerResponseHandler; @@ -106,6 +108,7 @@ public class TransportSearchAction extends HandledTransportAction { + private static final Logger logger = LogManager.getLogger(TransportSearchAction.class); private static final DeprecationLogger DEPRECATION_LOGGER = DeprecationLogger.getLogger(TransportSearchAction.class); public static final String FROZEN_INDICES_DEPRECATION_MESSAGE = "Searching frozen indices [{}] is deprecated." + " Consider cold or frozen tiers in place of frozen indices. The frozen feature will be removed in a feature release."; @@ -323,8 +326,7 @@ void executeRequest( && rewritten.source().aggregations() != null ? searchService.aggReduceContextBuilder(task::isCancelled, rewritten.source().aggregations()) : null; - final int totalClusters = (localIndices == null ? 0 : 1) + remoteClusterIndices.size(); - var initClusters = new SearchResponse.Clusters(totalClusters, 0, 0, remoteClusterIndices.size(), true); + SearchResponse.Clusters initClusters = new SearchResponse.Clusters(localIndices, remoteClusterIndices, true); if (localIndices == null) { // Notify the progress listener that a CCS with minimize_roundtrips is happening remote-only (no local shards) task.getProgressListener().notifyListShards(Collections.emptyList(), Collections.emptyList(), initClusters, false); @@ -334,6 +336,7 @@ void executeRequest( rewritten, localIndices, remoteClusterIndices, + initClusters, timeProvider, aggregationReduceContextBuilder, remoteClusterService, @@ -456,11 +459,15 @@ static boolean shouldMinimizeRoundtrips(SearchRequest searchRequest) { || source.collapse().getInnerHits().isEmpty(); } + /** + * Handles ccs_minimize_roundtrips=true + */ static void ccsRemoteReduce( TaskId parentTaskId, SearchRequest searchRequest, OriginalIndices localIndices, Map remoteIndices, + SearchResponse.Clusters clusters, SearchTimeProvider timeProvider, AggregationReduceContext.Builder aggReduceContextBuilder, RemoteClusterService remoteClusterService, @@ -493,10 +500,14 @@ static void ccsRemoteReduce( remoteClusterClient.search(ccsSearchRequest, new ActionListener() { @Override public void onResponse(SearchResponse searchResponse) { + // TODO: in CCS fail fast ticket we may need to fail the query if the cluster is marked as FAILED + // overwrite the existing cluster entry with the updated one + ccsClusterInfoUpdate(searchResponse, clusters.getCluster(clusterAlias), skipUnavailable); Map profileResults = searchResponse.getProfileResults(); SearchProfileResults profile = profileResults == null || profileResults.isEmpty() ? null : new SearchProfileResults(profileResults); + InternalSearchResponse internalSearchResponse = new InternalSearchResponse( searchResponse.getHits(), (InternalAggregations) searchResponse.getAggregations(), @@ -506,6 +517,7 @@ public void onResponse(SearchResponse searchResponse) { searchResponse.isTerminatedEarly(), searchResponse.getNumReducePhases() ); + listener.onResponse( new SearchResponse( internalSearchResponse, @@ -515,7 +527,7 @@ public void onResponse(SearchResponse searchResponse) { searchResponse.getSkippedShards(), timeProvider.buildTookInMillis(), searchResponse.getShardFailures(), - new SearchResponse.Clusters(1, 1, 0), + clusters, searchResponse.pointInTimeId() ) ); @@ -523,8 +535,10 @@ public void onResponse(SearchResponse searchResponse) { @Override public void onFailure(Exception e) { + ShardSearchFailure failure = new ShardSearchFailure(e); + ccsClusterInfoUpdate(failure, clusters.getCluster(clusterAlias), skipUnavailable); if (skipUnavailable) { - listener.onResponse(SearchResponse.empty(timeProvider::buildTookInMillis, new SearchResponse.Clusters(1, 0, 1))); + listener.onResponse(SearchResponse.empty(timeProvider::buildTookInMillis, clusters)); } else { listener.onFailure(wrapRemoteClusterFailure(clusterAlias, e)); } @@ -559,7 +573,7 @@ public void onFailure(Exception e) { skippedClusters, exceptions, searchResponseMerger, - totalClusters, + clusters, listener ); Client remoteClusterClient = remoteClusterService.getRemoteClusterClient( @@ -577,7 +591,7 @@ public void onFailure(Exception e) { skippedClusters, exceptions, searchResponseMerger, - totalClusters, + clusters, listener ); SearchRequest ccsLocalSearchRequest = SearchRequest.subSearchRequest( @@ -644,6 +658,7 @@ static void collectSearchShards( responsesCountDown, skippedClusters, exceptions, + null, listener ) { @Override @@ -704,6 +719,9 @@ Map createFinalResponse() { } } + /** + * Only used for ccs_minimize_roundtrips=true pathway + */ private static ActionListener createCCSListener( String clusterAlias, boolean skipUnavailable, @@ -711,7 +729,7 @@ private static ActionListener createCCSListener( AtomicInteger skippedClusters, AtomicReference exceptions, SearchResponseMerger searchResponseMerger, - int totalClusters, + SearchResponse.Clusters clusters, ActionListener originalListener ) { return new CCSActionListener( @@ -720,25 +738,109 @@ private static ActionListener createCCSListener( countDown, skippedClusters, exceptions, + clusters.getCluster(clusterAlias), originalListener ) { @Override void innerOnResponse(SearchResponse searchResponse) { + // TODO: in CCS fail fast ticket we may need to fail the query if the cluster gets marked as FAILED + ccsClusterInfoUpdate(searchResponse, cluster, skipUnavailable); searchResponseMerger.add(searchResponse); } @Override SearchResponse createFinalResponse() { - SearchResponse.Clusters clusters = new SearchResponse.Clusters( - totalClusters, - searchResponseMerger.numResponses(), - skippedClusters.get() - ); return searchResponseMerger.getMergedResponse(clusters); } }; } + /** + * Creates a new Cluster object using the {@link ShardSearchFailure} info and skip_unavailable + * flag to set Status. The new Cluster object is swapped into the clusterRef {@link AtomicReference}. + */ + static void ccsClusterInfoUpdate( + ShardSearchFailure failure, + AtomicReference clusterRef, + boolean skipUnavailable + ) { + SearchResponse.Cluster.Status status; + if (skipUnavailable) { + status = SearchResponse.Cluster.Status.SKIPPED; + } else { + status = SearchResponse.Cluster.Status.FAILED; + } + boolean swapped; + do { + SearchResponse.Cluster orig = clusterRef.get(); + String clusterAlias = orig.getClusterAlias(); + List failures; + if (orig.getFailures() != null) { + failures = new ArrayList<>(orig.getFailures()); + } else { + failures = new ArrayList<>(1); + } + failures.add(failure); + String indexExpression = orig.getIndexExpression(); + SearchResponse.Cluster updated = new SearchResponse.Cluster(clusterAlias, indexExpression, status, failures); + swapped = clusterRef.compareAndSet(orig, updated); + } while (swapped == false); + } + + /** + * Helper method common to multiple ccs_minimize_roundtrips=true code paths. + * Used to update a specific SearchResponse.Cluster object state based upon + * the SearchResponse coming from the cluster coordinator the search was performed on. + * @param searchResponse SearchResponse from cluster sub-search + * @param clusterRef AtomicReference of the Cluster object to be updated + */ + private static void ccsClusterInfoUpdate( + SearchResponse searchResponse, + AtomicReference clusterRef, + boolean skipUnavailable + ) { + /* + * Cluster Status logic: + * 1) FAILED if all shards failed and skip_unavailable=false + * 2) SKIPPED if all shards failed and skip_unavailable=true + * 3) PARTIAL if it timed out + * 4) PARTIAL if it at least one of the shards succeeded but not all + * 5) SUCCESSFUL if no shards failed (and did not time out) + */ + SearchResponse.Cluster.Status status; + if (searchResponse.getFailedShards() >= searchResponse.getTotalShards()) { + if (skipUnavailable) { + status = SearchResponse.Cluster.Status.SKIPPED; + } else { + status = SearchResponse.Cluster.Status.FAILED; + } + } else if (searchResponse.isTimedOut()) { + status = SearchResponse.Cluster.Status.PARTIAL; + } else if (searchResponse.getFailedShards() > 0) { + status = SearchResponse.Cluster.Status.PARTIAL; + } else { + status = SearchResponse.Cluster.Status.SUCCESSFUL; + } + + boolean swapped; + do { + SearchResponse.Cluster orig = clusterRef.get(); + SearchResponse.Cluster updated = new SearchResponse.Cluster( + orig.getClusterAlias(), + orig.getIndexExpression(), + status, + searchResponse.getTotalShards(), + searchResponse.getSuccessfulShards(), + searchResponse.getSkippedShards(), + searchResponse.getFailedShards(), + Arrays.asList(searchResponse.getShardFailures()), + searchResponse.getTook(), + searchResponse.isTimedOut() + ); + swapped = clusterRef.compareAndSet(orig, updated); + } while (swapped == false); + } + void executeLocalSearch( Task task, SearchTimeProvider timeProvider, @@ -1263,19 +1365,25 @@ private static void failIfOverShardCountLimit(ClusterService clusterService, int } abstract static class CCSActionListener implements ActionListener { - private final String clusterAlias; - private final boolean skipUnavailable; + protected final String clusterAlias; + protected final boolean skipUnavailable; private final CountDown countDown; private final AtomicInteger skippedClusters; private final AtomicReference exceptions; + protected final AtomicReference cluster; private final ActionListener originalListener; + protected final long startTime; + /** + * Used by both minimize_roundtrips true and false + */ CCSActionListener( String clusterAlias, boolean skipUnavailable, CountDown countDown, AtomicInteger skippedClusters, AtomicReference exceptions, + @Nullable AtomicReference cluster, // null for ccs_minimize_roundtrips=false ActionListener originalListener ) { this.clusterAlias = clusterAlias; @@ -1283,7 +1391,9 @@ abstract static class CCSActionListener implements Acti this.countDown = countDown; this.skippedClusters = skippedClusters; this.exceptions = exceptions; + this.cluster = cluster; this.originalListener = originalListener; + this.startTime = System.currentTimeMillis(); } @Override @@ -1296,9 +1406,16 @@ public final void onResponse(Response response) { @Override public final void onFailure(Exception e) { + ShardSearchFailure f = new ShardSearchFailure(e); if (skipUnavailable) { + if (cluster != null) { + ccsClusterInfoUpdate(f, cluster, skipUnavailable); + } skippedClusters.incrementAndGet(); } else { + if (cluster != null) { + ccsClusterInfoUpdate(f, cluster, skipUnavailable); + } Exception exception = e; if (RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY.equals(clusterAlias) == false) { exception = wrapRemoteClusterFailure(clusterAlias, e); diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchResponseTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchResponseTests.java index 564c6f1b08dbf..c83111ddfa98a 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchResponseTests.java @@ -135,18 +135,7 @@ static SearchResponse.Clusters randomClusters() { int totalClusters = randomIntBetween(0, 10); int successfulClusters = randomIntBetween(0, totalClusters); int skippedClusters = totalClusters - successfulClusters; - if (randomBoolean()) { - return new SearchResponse.Clusters(totalClusters, successfulClusters, skippedClusters); - } else { - int remoteClusters = totalClusters; - if (totalClusters > 0 && randomBoolean()) { - // remoteClusters can be same as total cluster count or one less (when doing local search) - remoteClusters--; - } - // Clusters has an assert that if ccsMinimizeRoundtrips = true, then remoteClusters must be > 0 - boolean ccsMinimizeRoundtrips = (remoteClusters > 0 ? randomBoolean() : false); - return new SearchResponse.Clusters(totalClusters, successfulClusters, skippedClusters, remoteClusters, ccsMinimizeRoundtrips); - } + return new SearchResponse.Clusters(totalClusters, successfulClusters, skippedClusters); } /** diff --git a/server/src/test/java/org/elasticsearch/action/search/TransportSearchActionTests.java b/server/src/test/java/org/elasticsearch/action/search/TransportSearchActionTests.java index 238e6e8fe9d67..1c53e90d28c34 100644 --- a/server/src/test/java/org/elasticsearch/action/search/TransportSearchActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/TransportSearchActionTests.java @@ -523,11 +523,14 @@ public void testCCSRemoteReduceMergeFails() throws Exception { ActionListener.wrap(r -> fail("no response expected"), failure::set), latch ); + SearchResponse.Clusters initClusters = new SearchResponse.Clusters(localIndices, remoteIndicesByCluster, true); + TransportSearchAction.ccsRemoteReduce( new TaskId("n", 1), searchRequest, localIndices, remoteIndicesByCluster, + initClusters, timeProvider, emptyReduceContextBuilder(), remoteClusterService, @@ -588,11 +591,14 @@ public void testCCSRemoteReduce() throws Exception { ActionTestUtils.assertNoFailureListener(response::set), latch ); + SearchResponse.Clusters initClusters = new SearchResponse.Clusters(localIndices, remoteIndicesByCluster, true); + TransportSearchAction.ccsRemoteReduce( new TaskId("n", 1), searchRequest, localIndices, remoteIndicesByCluster, + initClusters, timeProvider, emptyReduceContextBuilder(), remoteClusterService, @@ -626,11 +632,13 @@ public void testCCSRemoteReduce() throws Exception { ActionListener.wrap(r -> fail("no response expected"), failure::set), latch ); + SearchResponse.Clusters initClusters = new SearchResponse.Clusters(localIndices, remoteIndicesByCluster, true); TransportSearchAction.ccsRemoteReduce( new TaskId("n", 1), searchRequest, localIndices, remoteIndicesByCluster, + initClusters, timeProvider, emptyReduceContextBuilder(), remoteClusterService, @@ -685,11 +693,14 @@ public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connecti ActionListener.wrap(r -> fail("no response expected"), failure::set), latch ); + SearchResponse.Clusters initClusters = new SearchResponse.Clusters(localIndices, remoteIndicesByCluster, true); + TransportSearchAction.ccsRemoteReduce( new TaskId("n", 1), searchRequest, localIndices, remoteIndicesByCluster, + initClusters, timeProvider, emptyReduceContextBuilder(), remoteClusterService, @@ -726,11 +737,18 @@ public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connecti ActionTestUtils.assertNoFailureListener(response::set), latch ); + Set clusterAliases = new HashSet<>(remoteClusterService.getRegisteredRemoteClusterNames()); + if (localIndices != null) { + clusterAliases.add(""); + } + SearchResponse.Clusters initClusters = new SearchResponse.Clusters(localIndices, remoteIndicesByCluster, true); + TransportSearchAction.ccsRemoteReduce( new TaskId("n", 1), searchRequest, localIndices, remoteIndicesByCluster, + initClusters, timeProvider, emptyReduceContextBuilder(), remoteClusterService, @@ -779,11 +797,18 @@ public void onNodeDisconnected(DiscoveryNode node, Transport.Connection connecti ActionTestUtils.assertNoFailureListener(response::set), latch ); + Set clusterAliases = new HashSet<>(remoteClusterService.getRegisteredRemoteClusterNames()); + if (localIndices != null) { + clusterAliases.add(""); + } + SearchResponse.Clusters initClusters = new SearchResponse.Clusters(localIndices, remoteIndicesByCluster, true); + TransportSearchAction.ccsRemoteReduce( new TaskId("n", 1), searchRequest, localIndices, remoteIndicesByCluster, + initClusters, timeProvider, emptyReduceContextBuilder(), remoteClusterService, diff --git a/test/framework/src/main/java/org/elasticsearch/test/AbstractMultiClustersTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/AbstractMultiClustersTestCase.java index bf134528cbdc1..61a1bef41101d 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/AbstractMultiClustersTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/AbstractMultiClustersTestCase.java @@ -12,13 +12,16 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.action.admin.cluster.remote.RemoteInfoAction; import org.elasticsearch.action.admin.cluster.remote.RemoteInfoRequest; +import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; import org.elasticsearch.client.internal.Client; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.IOUtils; +import org.elasticsearch.core.Strings; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.transport.RemoteClusterAware; +import org.elasticsearch.transport.RemoteClusterService; import org.elasticsearch.transport.RemoteConnectionInfo; import org.elasticsearch.transport.TransportService; import org.junit.After; @@ -46,6 +49,7 @@ public abstract class AbstractMultiClustersTestCase extends ESTestCase { public static final String LOCAL_CLUSTER = RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY; + public static final boolean DEFAULT_SKIP_UNAVAILABLE = RemoteClusterService.REMOTE_CLUSTER_SKIP_UNAVAILABLE.getDefault(Settings.EMPTY); private static final Logger LOGGER = LogManager.getLogger(AbstractMultiClustersTestCase.class); @@ -55,6 +59,10 @@ protected Collection remoteClusterAlias() { return randomSubsetOf(List.of("cluster-a", "cluster-b")); } + protected Map skipUnavailableForRemoteClusters() { + return Map.of("cluster-a", DEFAULT_SKIP_UNAVAILABLE, "cluster-b", DEFAULT_SKIP_UNAVAILABLE); + } + protected Collection> nodePlugins(String clusterAlias) { return Collections.emptyList(); } @@ -171,26 +179,37 @@ protected void configureAndConnectsToRemoteClusters() throws Exception { protected void configureRemoteCluster(String clusterAlias, Collection seedNodes) throws Exception { final String remoteClusterSettingPrefix = "cluster.remote." + clusterAlias + "."; Settings.Builder settings = Settings.builder(); - final List seedAdresses = seedNodes.stream().map(node -> { + final List seedAddresses = seedNodes.stream().map(node -> { final TransportService transportService = cluster(clusterAlias).getInstance(TransportService.class, node); return transportService.boundAddress().publishAddress().toString(); }).toList(); + boolean skipUnavailable = skipUnavailableForRemoteClusters().containsKey(clusterAlias) + ? skipUnavailableForRemoteClusters().get(clusterAlias) + : DEFAULT_SKIP_UNAVAILABLE; + Settings.Builder builder; if (randomBoolean()) { LOGGER.info("--> use sniff mode with seed [{}], remote nodes [{}]", Collectors.joining(","), seedNodes); - settings.putNull(remoteClusterSettingPrefix + "proxy_address") + builder = settings.putNull(remoteClusterSettingPrefix + "proxy_address") .put(remoteClusterSettingPrefix + "mode", "sniff") - .put(remoteClusterSettingPrefix + "seeds", String.join(",", seedAdresses)) - .build(); + .put(remoteClusterSettingPrefix + "seeds", String.join(",", seedAddresses)); } else { - final String proxyNode = randomFrom(seedAdresses); + final String proxyNode = randomFrom(seedAddresses); LOGGER.info("--> use proxy node [{}], remote nodes [{}]", proxyNode, seedNodes); - settings.putNull(remoteClusterSettingPrefix + "seeds") + builder = settings.putNull(remoteClusterSettingPrefix + "seeds") .put(remoteClusterSettingPrefix + "mode", "proxy") - .put(remoteClusterSettingPrefix + "proxy_address", proxyNode) - .build(); + .put(remoteClusterSettingPrefix + "proxy_address", proxyNode); + } + if (skipUnavailable != DEFAULT_SKIP_UNAVAILABLE) { + builder.put(remoteClusterSettingPrefix + "skip_unavailable", String.valueOf(skipUnavailable)); + } + builder.build(); + + ClusterUpdateSettingsResponse resp = client().admin().cluster().prepareUpdateSettings().setPersistentSettings(settings).get(); + if (skipUnavailable != DEFAULT_SKIP_UNAVAILABLE) { + String key = Strings.format("cluster.remote.%s.skip_unavailable", clusterAlias); + assertEquals(String.valueOf(skipUnavailable), resp.getPersistentSettings().get(key)); } - client().admin().cluster().prepareUpdateSettings().setPersistentSettings(settings).get(); assertBusy(() -> { List remoteConnectionInfos = client().execute(RemoteInfoAction.INSTANCE, new RemoteInfoRequest()) .actionGet() diff --git a/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/CrossClusterAsyncSearchIT.java b/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/CrossClusterAsyncSearchIT.java index 01d2acad55bb9..9d496dcaa70a8 100644 --- a/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/CrossClusterAsyncSearchIT.java +++ b/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/CrossClusterAsyncSearchIT.java @@ -14,12 +14,13 @@ import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksResponse; import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse; import org.elasticsearch.action.search.SearchAction; +import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.metadata.IndexMetadata; -import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.IndexModule; @@ -37,6 +38,7 @@ import org.elasticsearch.tasks.TaskInfo; import org.elasticsearch.test.AbstractMultiClustersTestCase; import org.elasticsearch.test.InternalTestCluster; +import org.elasticsearch.transport.RemoteClusterAware; import org.elasticsearch.transport.TransportService; import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.XContentFactory; @@ -58,7 +60,9 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; @@ -87,6 +91,11 @@ protected Collection remoteClusterAlias() { return List.of(REMOTE_CLUSTER); } + @Override + protected Map skipUnavailableForRemoteClusters() { + return Map.of(REMOTE_CLUSTER, randomBoolean()); + } + @Override protected boolean reuseClusters() { return false; @@ -116,17 +125,697 @@ public List> getQueries() { throw new IllegalStateException("not implemented"); } ); + QuerySpec throwingSpec = new QuerySpec<>(ThrowingQueryBuilder.NAME, ThrowingQueryBuilder::new, p -> { + throw new IllegalStateException("not implemented"); + }); + + return List.of(slowRunningSpec, throwingSpec); + } + } + + public void testClusterDetailsAfterSuccessfulCCS() throws Exception { + Map testClusterInfo = setupTwoClusters(); + String localIndex = (String) testClusterInfo.get("local.index"); + String remoteIndex = (String) testClusterInfo.get("remote.index"); + int localNumShards = (Integer) testClusterInfo.get("local.num_shards"); + int remoteNumShards = (Integer) testClusterInfo.get("remote.num_shards"); + + SearchListenerPlugin.blockQueryPhase(); + + SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(localIndex, REMOTE_CLUSTER + ":" + remoteIndex); + request.setCcsMinimizeRoundtrips(true); + request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1)); + request.setKeepOnCompletion(true); + request.getSearchRequest().source(new SearchSourceBuilder().query(new MatchAllQueryBuilder()).size(1000)); + + AsyncSearchResponse response = submitAsyncSearch(request); + assertNotNull(response.getSearchResponse()); + assertTrue(response.isRunning()); + + { + SearchResponse.Clusters clusters = response.getSearchResponse().getClusters(); + assertThat(clusters.getTotal(), equalTo(2)); + assertTrue("search cluster results should be marked as partial", clusters.hasPartialResults()); + + SearchResponse.Cluster localClusterSearchInfo = clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY).get(); + assertNotNull(localClusterSearchInfo); + assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.RUNNING)); + + SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get(); + assertNotNull(remoteClusterSearchInfo); + assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.RUNNING)); + } + + SearchListenerPlugin.waitSearchStarted(); + SearchListenerPlugin.allowQueryPhase(); + + assertBusy(() -> { + AsyncStatusResponse statusResponse = getAsyncStatus(response.getId()); + assertFalse(statusResponse.isRunning()); + assertNotNull(statusResponse.getCompletionStatus()); + }); + + { + AsyncSearchResponse finishedResponse = getAsyncSearch(response.getId()); + + SearchResponse.Clusters clusters = finishedResponse.getSearchResponse().getClusters(); + assertFalse("search cluster results should NOT be marked as partial", clusters.hasPartialResults()); + assertThat(clusters.getTotal(), equalTo(2)); + assertThat(clusters.getSuccessful(), equalTo(2)); + assertThat(clusters.getSkipped(), equalTo(0)); + + SearchResponse.Cluster localClusterSearchInfo = clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY).get(); + assertNotNull(localClusterSearchInfo); + assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL)); + assertThat(localClusterSearchInfo.getTotalShards(), equalTo(localNumShards)); + assertThat(localClusterSearchInfo.getSuccessfulShards(), equalTo(localNumShards)); + assertThat(localClusterSearchInfo.getSkippedShards(), equalTo(0)); + assertThat(localClusterSearchInfo.getFailedShards(), equalTo(0)); + assertThat(localClusterSearchInfo.getFailures().size(), equalTo(0)); + assertThat(localClusterSearchInfo.getTook().millis(), greaterThan(0L)); + + SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get(); + assertNotNull(remoteClusterSearchInfo); + assertThat(remoteClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL)); + assertThat(remoteClusterSearchInfo.getTotalShards(), equalTo(remoteNumShards)); + assertThat(remoteClusterSearchInfo.getSuccessfulShards(), equalTo(remoteNumShards)); + assertThat(remoteClusterSearchInfo.getSkippedShards(), equalTo(0)); + assertThat(remoteClusterSearchInfo.getFailedShards(), equalTo(0)); + assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(0)); + assertThat(remoteClusterSearchInfo.getTook().millis(), greaterThan(0L)); + } + + // check that the async_search/status response includes the same cluster details + { + AsyncStatusResponse statusResponse = getAsyncStatus(response.getId()); + + SearchResponse.Clusters clusters = statusResponse.getClusters(); + assertFalse("search cluster results should NOT be marked as partial", clusters.hasPartialResults()); + assertThat(clusters.getTotal(), equalTo(2)); + assertThat(clusters.getSuccessful(), equalTo(2)); + assertThat(clusters.getSkipped(), equalTo(0)); + + SearchResponse.Cluster localClusterSearchInfo = clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY).get(); + assertNotNull(localClusterSearchInfo); + assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL)); + assertThat(localClusterSearchInfo.getTotalShards(), equalTo(localNumShards)); + assertThat(localClusterSearchInfo.getSuccessfulShards(), equalTo(localNumShards)); + assertThat(localClusterSearchInfo.getSkippedShards(), equalTo(0)); + assertThat(localClusterSearchInfo.getFailedShards(), equalTo(0)); + assertThat(localClusterSearchInfo.getFailures().size(), equalTo(0)); + assertThat(localClusterSearchInfo.getTook().millis(), greaterThan(0L)); + + SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get(); + assertNotNull(remoteClusterSearchInfo); + assertThat(remoteClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL)); + assertThat(remoteClusterSearchInfo.getTotalShards(), equalTo(remoteNumShards)); + assertThat(remoteClusterSearchInfo.getSuccessfulShards(), equalTo(remoteNumShards)); + assertThat(remoteClusterSearchInfo.getSkippedShards(), equalTo(0)); + assertThat(remoteClusterSearchInfo.getFailedShards(), equalTo(0)); + assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(0)); + assertThat(remoteClusterSearchInfo.getTook().millis(), greaterThan(0L)); + } + } + + public void testClusterDetailsAfterCCSWithFailuresOnAllShards() throws Exception { + Map testClusterInfo = setupTwoClusters(); + String localIndex = (String) testClusterInfo.get("local.index"); + String remoteIndex = (String) testClusterInfo.get("remote.index"); + boolean skipUnavailable = (Boolean) testClusterInfo.get("remote.skip_unavailable"); + + SearchListenerPlugin.blockQueryPhase(); + + SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(localIndex, REMOTE_CLUSTER + ":" + remoteIndex); + request.setCcsMinimizeRoundtrips(true); + request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1)); + request.setKeepOnCompletion(true); + // shardId -1 means to throw the Exception on all shards, so should result in complete search failure + ThrowingQueryBuilder queryBuilder = new ThrowingQueryBuilder(randomLong(), new IllegalStateException("index corrupted"), -1); + request.getSearchRequest().source(new SearchSourceBuilder().query(queryBuilder).size(10)); + + AsyncSearchResponse response = submitAsyncSearch(request); + assertNotNull(response.getSearchResponse()); + assertTrue(response.isRunning()); + + { + SearchResponse.Clusters clusters = response.getSearchResponse().getClusters(); + assertThat(clusters.getTotal(), equalTo(2)); + assertTrue("search cluster results should be marked as partial", clusters.hasPartialResults()); + + SearchResponse.Cluster localClusterSearchInfo = clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY).get(); + assertNotNull(localClusterSearchInfo); + assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.RUNNING)); + + SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get(); + assertNotNull(remoteClusterSearchInfo); + assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.RUNNING)); + } + + SearchListenerPlugin.waitSearchStarted(); + SearchListenerPlugin.allowQueryPhase(); + + assertBusy(() -> { + AsyncStatusResponse statusResponse = getAsyncStatus(response.getId()); + assertFalse(statusResponse.isRunning()); + assertNotNull(statusResponse.getCompletionStatus()); + }); + + { + AsyncSearchResponse finishedResponse = getAsyncSearch(response.getId()); + + SearchResponse.Clusters clusters = finishedResponse.getSearchResponse().getClusters(); + assertThat(clusters.getTotal(), equalTo(2)); + assertThat(clusters.getSuccessful(), equalTo(0)); + assertThat(clusters.getSkipped(), equalTo(2)); + + SearchResponse.Cluster localClusterSearchInfo = clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY).get(); + assertNotNull(localClusterSearchInfo); + assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.FAILED)); + assertNull(localClusterSearchInfo.getTotalShards()); + assertNull(localClusterSearchInfo.getSuccessfulShards()); + assertNull(localClusterSearchInfo.getSkippedShards()); + assertNull(localClusterSearchInfo.getFailedShards()); + assertThat(localClusterSearchInfo.getFailures().size(), equalTo(1)); + assertNull(localClusterSearchInfo.getTook()); + assertFalse(localClusterSearchInfo.isTimedOut()); + ShardSearchFailure localShardSearchFailure = localClusterSearchInfo.getFailures().get(0); + assertTrue("should have 'index corrupted' in reason", localShardSearchFailure.reason().contains("index corrupted")); + + SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get(); + assertNotNull(remoteClusterSearchInfo); + SearchResponse.Cluster.Status expectedStatus = skipUnavailable + ? SearchResponse.Cluster.Status.SKIPPED + : SearchResponse.Cluster.Status.FAILED; + assertThat(remoteClusterSearchInfo.getStatus(), equalTo(expectedStatus)); + assertNull(remoteClusterSearchInfo.getTotalShards()); + assertNull(remoteClusterSearchInfo.getSuccessfulShards()); + assertNull(remoteClusterSearchInfo.getSkippedShards()); + assertNull(remoteClusterSearchInfo.getFailedShards()); + assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(1)); + assertNull(remoteClusterSearchInfo.getTook()); + assertFalse(remoteClusterSearchInfo.isTimedOut()); + ShardSearchFailure remoteShardSearchFailure = remoteClusterSearchInfo.getFailures().get(0); + assertTrue("should have 'index corrupted' in reason", remoteShardSearchFailure.reason().contains("index corrupted")); + } + // check that the async_search/status response includes the same cluster details + { + AsyncStatusResponse statusResponse = getAsyncStatus(response.getId()); + SearchResponse.Clusters clusters = statusResponse.getClusters(); + assertThat(clusters.getTotal(), equalTo(2)); + assertThat(clusters.getSuccessful(), equalTo(0)); + assertThat(clusters.getSkipped(), equalTo(2)); + + SearchResponse.Cluster localClusterSearchInfo = clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY).get(); + assertNotNull(localClusterSearchInfo); + assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.FAILED)); + assertNull(localClusterSearchInfo.getTotalShards()); + assertNull(localClusterSearchInfo.getSuccessfulShards()); + assertNull(localClusterSearchInfo.getSkippedShards()); + assertNull(localClusterSearchInfo.getFailedShards()); + assertThat(localClusterSearchInfo.getFailures().size(), equalTo(1)); + assertNull(localClusterSearchInfo.getTook()); + assertFalse(localClusterSearchInfo.isTimedOut()); + ShardSearchFailure localShardSearchFailure = localClusterSearchInfo.getFailures().get(0); + assertTrue("should have 'index corrupted' in reason", localShardSearchFailure.reason().contains("index corrupted")); + + SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get(); + assertNotNull(remoteClusterSearchInfo); + SearchResponse.Cluster.Status expectedStatus = skipUnavailable + ? SearchResponse.Cluster.Status.SKIPPED + : SearchResponse.Cluster.Status.FAILED; + assertThat(remoteClusterSearchInfo.getStatus(), equalTo(expectedStatus)); + assertNull(remoteClusterSearchInfo.getTotalShards()); + assertNull(remoteClusterSearchInfo.getSuccessfulShards()); + assertNull(remoteClusterSearchInfo.getSkippedShards()); + assertNull(remoteClusterSearchInfo.getFailedShards()); + assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(1)); + assertNull(remoteClusterSearchInfo.getTook()); + assertFalse(remoteClusterSearchInfo.isTimedOut()); + ShardSearchFailure remoteShardSearchFailure = remoteClusterSearchInfo.getFailures().get(0); + assertTrue("should have 'index corrupted' in reason", remoteShardSearchFailure.reason().contains("index corrupted")); + } + } + + public void testClusterDetailsAfterCCSWithFailuresOnOneShardOnly() throws Exception { + Map testClusterInfo = setupTwoClusters(); + String localIndex = (String) testClusterInfo.get("local.index"); + String remoteIndex = (String) testClusterInfo.get("remote.index"); + int localNumShards = (Integer) testClusterInfo.get("local.num_shards"); + int remoteNumShards = (Integer) testClusterInfo.get("remote.num_shards"); + + SearchListenerPlugin.blockQueryPhase(); + + SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(localIndex, REMOTE_CLUSTER + ":" + remoteIndex); + request.setCcsMinimizeRoundtrips(true); + request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1)); + request.setKeepOnCompletion(true); + // shardId 0 means to throw the Exception only on shard 0; all others should work + ThrowingQueryBuilder queryBuilder = new ThrowingQueryBuilder(randomLong(), new IllegalStateException("index corrupted"), 0); + request.getSearchRequest().source(new SearchSourceBuilder().query(queryBuilder).size(10)); + + AsyncSearchResponse response = submitAsyncSearch(request); + assertNotNull(response.getSearchResponse()); + assertTrue(response.isRunning()); + + { + SearchResponse.Clusters clusters = response.getSearchResponse().getClusters(); + assertThat(clusters.getTotal(), equalTo(2)); + assertTrue("search cluster results should be marked as partial", clusters.hasPartialResults()); + + SearchResponse.Cluster localClusterSearchInfo = clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY).get(); + assertNotNull(localClusterSearchInfo); + assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.RUNNING)); + + SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get(); + assertNotNull(remoteClusterSearchInfo); + assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.RUNNING)); + } + + SearchListenerPlugin.waitSearchStarted(); + SearchListenerPlugin.allowQueryPhase(); + + assertBusy(() -> { + AsyncStatusResponse statusResponse = getAsyncStatus(response.getId()); + assertFalse(statusResponse.isRunning()); + assertNotNull(statusResponse.getCompletionStatus()); + }); + + { + AsyncSearchResponse finishedResponse = getAsyncSearch(response.getId()); + + SearchResponse.Clusters clusters = finishedResponse.getSearchResponse().getClusters(); + assertThat(clusters.getTotal(), equalTo(2)); + assertThat(clusters.getSuccessful(), equalTo(2)); + assertThat(clusters.getSkipped(), equalTo(0)); + + SearchResponse.Cluster localClusterSearchInfo = clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY).get(); + assertNotNull(localClusterSearchInfo); + assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.PARTIAL)); + assertThat(localClusterSearchInfo.getTotalShards(), equalTo(localNumShards)); + assertThat(localClusterSearchInfo.getSuccessfulShards(), equalTo(localNumShards - 1)); + assertThat(localClusterSearchInfo.getSkippedShards(), equalTo(0)); + assertThat(localClusterSearchInfo.getFailedShards(), equalTo(1)); + assertThat(localClusterSearchInfo.getFailures().size(), equalTo(1)); + assertThat(localClusterSearchInfo.getTook().millis(), greaterThan(0L)); + ShardSearchFailure localShardSearchFailure = localClusterSearchInfo.getFailures().get(0); + assertTrue("should have 'index corrupted' in reason", localShardSearchFailure.reason().contains("index corrupted")); + + SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get(); + assertNotNull(remoteClusterSearchInfo); + assertThat(remoteClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.PARTIAL)); + assertThat(remoteClusterSearchInfo.getTotalShards(), equalTo(remoteNumShards)); + assertThat(remoteClusterSearchInfo.getSuccessfulShards(), equalTo(remoteNumShards - 1)); + assertThat(remoteClusterSearchInfo.getSkippedShards(), equalTo(0)); + assertThat(remoteClusterSearchInfo.getFailedShards(), equalTo(1)); + assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(1)); + assertThat(remoteClusterSearchInfo.getTook().millis(), greaterThan(0L)); + ShardSearchFailure remoteShardSearchFailure = remoteClusterSearchInfo.getFailures().get(0); + assertTrue("should have 'index corrupted' in reason", remoteShardSearchFailure.reason().contains("index corrupted")); + } + // check that the async_search/status response includes the same cluster details + { + AsyncStatusResponse statusResponse = getAsyncStatus(response.getId()); + SearchResponse.Clusters clusters = statusResponse.getClusters(); + assertThat(clusters.getTotal(), equalTo(2)); + assertThat(clusters.getSuccessful(), equalTo(2)); + assertThat(clusters.getSkipped(), equalTo(0)); + + SearchResponse.Cluster localClusterSearchInfo = clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY).get(); + assertNotNull(localClusterSearchInfo); + assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.PARTIAL)); + assertThat(localClusterSearchInfo.getTotalShards(), equalTo(localNumShards)); + assertThat(localClusterSearchInfo.getSuccessfulShards(), equalTo(localNumShards - 1)); + assertThat(localClusterSearchInfo.getSkippedShards(), equalTo(0)); + assertThat(localClusterSearchInfo.getFailedShards(), equalTo(1)); + assertThat(localClusterSearchInfo.getFailures().size(), equalTo(1)); + assertThat(localClusterSearchInfo.getTook().millis(), greaterThan(0L)); + ShardSearchFailure localShardSearchFailure = localClusterSearchInfo.getFailures().get(0); + assertTrue("should have 'index corrupted' in reason", localShardSearchFailure.reason().contains("index corrupted")); + + SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get(); + assertNotNull(remoteClusterSearchInfo); + assertThat(remoteClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.PARTIAL)); + assertThat(remoteClusterSearchInfo.getTotalShards(), equalTo(remoteNumShards)); + assertThat(remoteClusterSearchInfo.getSuccessfulShards(), equalTo(remoteNumShards - 1)); + assertThat(remoteClusterSearchInfo.getSkippedShards(), equalTo(0)); + assertThat(remoteClusterSearchInfo.getFailedShards(), equalTo(1)); + assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(1)); + assertThat(remoteClusterSearchInfo.getTook().millis(), greaterThan(0L)); + ShardSearchFailure remoteShardSearchFailure = remoteClusterSearchInfo.getFailures().get(0); + assertTrue("should have 'index corrupted' in reason", remoteShardSearchFailure.reason().contains("index corrupted")); + } + } + + public void testClusterDetailsAfterCCSWithFailuresOnOneClusterOnly() throws Exception { + Map testClusterInfo = setupTwoClusters(); + String localIndex = (String) testClusterInfo.get("local.index"); + String remoteIndex = (String) testClusterInfo.get("remote.index"); + int localNumShards = (Integer) testClusterInfo.get("local.num_shards"); + boolean skipUnavailable = (Boolean) testClusterInfo.get("remote.skip_unavailable"); - return List.of(slowRunningSpec); + SearchListenerPlugin.blockQueryPhase(); + + SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(localIndex, REMOTE_CLUSTER + ":" + remoteIndex); + request.setCcsMinimizeRoundtrips(true); + request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1)); + request.setKeepOnCompletion(true); + // throw Exception of all shards of remoteIndex, but against localIndex + ThrowingQueryBuilder queryBuilder = new ThrowingQueryBuilder( + randomLong(), + new IllegalStateException("index corrupted"), + remoteIndex + ); + request.getSearchRequest().source(new SearchSourceBuilder().query(queryBuilder).size(10)); + + AsyncSearchResponse response = submitAsyncSearch(request); + assertNotNull(response.getSearchResponse()); + assertTrue(response.isRunning()); + { + SearchResponse.Clusters clusters = response.getSearchResponse().getClusters(); + assertThat(clusters.getTotal(), equalTo(2)); + assertTrue("search cluster results should be marked as partial", clusters.hasPartialResults()); + + SearchResponse.Cluster localClusterSearchInfo = clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY).get(); + assertNotNull(localClusterSearchInfo); + assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.RUNNING)); + + SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get(); + assertNotNull(remoteClusterSearchInfo); + assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.RUNNING)); + } + + SearchListenerPlugin.waitSearchStarted(); + SearchListenerPlugin.allowQueryPhase(); + + assertBusy(() -> { + AsyncStatusResponse statusResponse = getAsyncStatus(response.getId()); + assertFalse(statusResponse.isRunning()); + assertNotNull(statusResponse.getCompletionStatus()); + }); + + { + AsyncSearchResponse finishedResponse = getAsyncSearch(response.getId()); + + SearchResponse.Clusters clusters = finishedResponse.getSearchResponse().getClusters(); + assertThat(clusters.getTotal(), equalTo(2)); + assertThat(clusters.getSuccessful(), equalTo(1)); + assertThat(clusters.getSkipped(), equalTo(1)); + + SearchResponse.Cluster localClusterSearchInfo = clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY).get(); + assertNotNull(localClusterSearchInfo); + assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL)); + assertThat(localClusterSearchInfo.getTotalShards(), equalTo(localNumShards)); + assertThat(localClusterSearchInfo.getSuccessfulShards(), equalTo(localNumShards)); + assertThat(localClusterSearchInfo.getSkippedShards(), equalTo(0)); + assertThat(localClusterSearchInfo.getFailedShards(), equalTo(0)); + assertThat(localClusterSearchInfo.getFailures().size(), equalTo(0)); + assertThat(localClusterSearchInfo.getTook().millis(), greaterThan(0L)); + + SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get(); + assertNotNull(remoteClusterSearchInfo); + SearchResponse.Cluster.Status expectedStatus = skipUnavailable + ? SearchResponse.Cluster.Status.SKIPPED + : SearchResponse.Cluster.Status.FAILED; + assertThat(remoteClusterSearchInfo.getStatus(), equalTo(expectedStatus)); + assertNull(remoteClusterSearchInfo.getTotalShards()); + assertNull(remoteClusterSearchInfo.getSuccessfulShards()); + assertNull(remoteClusterSearchInfo.getSkippedShards()); + assertNull(remoteClusterSearchInfo.getFailedShards()); + assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(1)); + assertNull(remoteClusterSearchInfo.getTook()); + assertFalse(remoteClusterSearchInfo.isTimedOut()); + ShardSearchFailure remoteShardSearchFailure = remoteClusterSearchInfo.getFailures().get(0); + assertTrue("should have 'index corrupted' in reason", remoteShardSearchFailure.reason().contains("index corrupted")); + } + // check that the async_search/status response includes the same cluster details + { + AsyncStatusResponse statusResponse = getAsyncStatus(response.getId()); + SearchResponse.Clusters clusters = statusResponse.getClusters(); + assertThat(clusters.getTotal(), equalTo(2)); + assertThat(clusters.getSuccessful(), equalTo(1)); + assertThat(clusters.getSkipped(), equalTo(1)); + + SearchResponse.Cluster localClusterSearchInfo = clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY).get(); + assertNotNull(localClusterSearchInfo); + assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL)); + assertThat(localClusterSearchInfo.getTotalShards(), equalTo(localNumShards)); + assertThat(localClusterSearchInfo.getSuccessfulShards(), equalTo(localNumShards)); + assertThat(localClusterSearchInfo.getSkippedShards(), equalTo(0)); + assertThat(localClusterSearchInfo.getFailedShards(), equalTo(0)); + assertThat(localClusterSearchInfo.getFailures().size(), equalTo(0)); + assertThat(localClusterSearchInfo.getTook().millis(), greaterThan(0L)); + + SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get(); + assertNotNull(remoteClusterSearchInfo); + SearchResponse.Cluster.Status expectedStatus = skipUnavailable + ? SearchResponse.Cluster.Status.SKIPPED + : SearchResponse.Cluster.Status.FAILED; + assertThat(remoteClusterSearchInfo.getStatus(), equalTo(expectedStatus)); + assertNull(remoteClusterSearchInfo.getTotalShards()); + assertNull(remoteClusterSearchInfo.getSuccessfulShards()); + assertNull(remoteClusterSearchInfo.getSkippedShards()); + assertNull(remoteClusterSearchInfo.getFailedShards()); + assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(1)); + assertNull(remoteClusterSearchInfo.getTook()); + assertFalse(remoteClusterSearchInfo.isTimedOut()); + ShardSearchFailure remoteShardSearchFailure = remoteClusterSearchInfo.getFailures().get(0); + assertTrue("should have 'index corrupted' in reason", remoteShardSearchFailure.reason().contains("index corrupted")); + } + } + + public void testRemoteClusterOnlyCCSSuccessfulResult() throws Exception { + // for remote-only queries, we can't use the SearchListenerPlugin since that listens for search + // stage on the local cluster, so we only test final state of the search response + SearchListenerPlugin.negate(); + + Map testClusterInfo = setupTwoClusters(); + String remoteIndex = (String) testClusterInfo.get("remote.index"); + int remoteNumShards = (Integer) testClusterInfo.get("remote.num_shards"); + + // search only the remote cluster + SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(REMOTE_CLUSTER + ":" + remoteIndex); + request.setCcsMinimizeRoundtrips(true); + request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1)); + request.setKeepOnCompletion(true); + request.getSearchRequest().source(new SearchSourceBuilder().query(new MatchAllQueryBuilder()).size(1000)); + + AsyncSearchResponse response = submitAsyncSearch(request); + assertNotNull(response.getSearchResponse()); + + assertBusy(() -> { + AsyncStatusResponse statusResponse = getAsyncStatus(response.getId()); + assertFalse(statusResponse.isRunning()); + assertNotNull(statusResponse.getCompletionStatus()); + }); + + { + AsyncSearchResponse finishedResponse = getAsyncSearch(response.getId()); + + SearchResponse.Clusters clusters = finishedResponse.getSearchResponse().getClusters(); + assertFalse("search cluster results should NOT be marked as partial", clusters.hasPartialResults()); + assertThat(clusters.getTotal(), equalTo(1)); + assertThat(clusters.getSuccessful(), equalTo(1)); + assertThat(clusters.getSkipped(), equalTo(0)); + + assertNull(clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY)); + + SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get(); + assertNotNull(remoteClusterSearchInfo); + assertThat(remoteClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL)); + assertThat(remoteClusterSearchInfo.getTotalShards(), equalTo(remoteNumShards)); + assertThat(remoteClusterSearchInfo.getSuccessfulShards(), equalTo(remoteNumShards)); + assertThat(remoteClusterSearchInfo.getSkippedShards(), equalTo(0)); + assertThat(remoteClusterSearchInfo.getFailedShards(), equalTo(0)); + assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(0)); + assertThat(remoteClusterSearchInfo.getTook().millis(), greaterThan(0L)); + } + + // check that the async_search/status response includes the same cluster details + { + AsyncStatusResponse statusResponse = getAsyncStatus(response.getId()); + + SearchResponse.Clusters clusters = statusResponse.getClusters(); + assertFalse("search cluster results should NOT be marked as partial", clusters.hasPartialResults()); + assertThat(clusters.getTotal(), equalTo(1)); + assertThat(clusters.getSuccessful(), equalTo(1)); + assertThat(clusters.getSkipped(), equalTo(0)); + + assertNull(clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY)); + + SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get(); + assertNotNull(remoteClusterSearchInfo); + assertThat(remoteClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL)); + assertThat(remoteClusterSearchInfo.getTotalShards(), equalTo(remoteNumShards)); + assertThat(remoteClusterSearchInfo.getSuccessfulShards(), equalTo(remoteNumShards)); + assertThat(remoteClusterSearchInfo.getSkippedShards(), equalTo(0)); + assertThat(remoteClusterSearchInfo.getFailedShards(), equalTo(0)); + assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(0)); + assertThat(remoteClusterSearchInfo.getTook().millis(), greaterThan(0L)); + } + } + + public void testRemoteClusterOnlyCCSWithFailuresOnOneShardOnly() throws Exception { + // for remote-only queries, we can't use the SearchListenerPlugin since that listens for search + // stage on the local cluster, so we only test final state of the search response + SearchListenerPlugin.negate(); + + Map testClusterInfo = setupTwoClusters(); + String remoteIndex = (String) testClusterInfo.get("remote.index"); + int remoteNumShards = (Integer) testClusterInfo.get("remote.num_shards"); + + SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(REMOTE_CLUSTER + ":" + remoteIndex); + request.setCcsMinimizeRoundtrips(true); + request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1)); + request.setKeepOnCompletion(true); + // shardId 0 means to throw the Exception only on shard 0; all others should work + ThrowingQueryBuilder queryBuilder = new ThrowingQueryBuilder(randomLong(), new IllegalStateException("index corrupted"), 0); + request.getSearchRequest().source(new SearchSourceBuilder().query(queryBuilder).size(10)); + + AsyncSearchResponse response = submitAsyncSearch(request); + assertNotNull(response.getSearchResponse()); + assertTrue(response.isRunning()); + + assertBusy(() -> { + AsyncStatusResponse statusResponse = getAsyncStatus(response.getId()); + assertFalse(statusResponse.isRunning()); + assertNotNull(statusResponse.getCompletionStatus()); + }); + + { + AsyncSearchResponse finishedResponse = getAsyncSearch(response.getId()); + + SearchResponse.Clusters clusters = finishedResponse.getSearchResponse().getClusters(); + assertThat(clusters.getTotal(), equalTo(1)); + assertThat(clusters.getSuccessful(), equalTo(1)); + assertThat(clusters.getSkipped(), equalTo(0)); + + assertNull(clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY)); + + SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get(); + assertNotNull(remoteClusterSearchInfo); + assertThat(remoteClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.PARTIAL)); + assertThat(remoteClusterSearchInfo.getTotalShards(), equalTo(remoteNumShards)); + assertThat(remoteClusterSearchInfo.getSuccessfulShards(), equalTo(remoteNumShards - 1)); + assertThat(remoteClusterSearchInfo.getSkippedShards(), equalTo(0)); + assertThat(remoteClusterSearchInfo.getFailedShards(), equalTo(1)); + assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(1)); + assertThat(remoteClusterSearchInfo.getTook().millis(), greaterThan(0L)); + ShardSearchFailure remoteShardSearchFailure = remoteClusterSearchInfo.getFailures().get(0); + assertTrue("should have 'index corrupted' in reason", remoteShardSearchFailure.reason().contains("index corrupted")); + } + // check that the async_search/status response includes the same cluster details + { + AsyncStatusResponse statusResponse = getAsyncStatus(response.getId()); + SearchResponse.Clusters clusters = statusResponse.getClusters(); + assertThat(clusters.getTotal(), equalTo(1)); + assertThat(clusters.getSuccessful(), equalTo(1)); + assertThat(clusters.getSkipped(), equalTo(0)); + + assertNull(clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY)); + + SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get(); + assertNotNull(remoteClusterSearchInfo); + assertThat(remoteClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.PARTIAL)); + assertThat(remoteClusterSearchInfo.getTotalShards(), equalTo(remoteNumShards)); + assertThat(remoteClusterSearchInfo.getSuccessfulShards(), equalTo(remoteNumShards - 1)); + assertThat(remoteClusterSearchInfo.getSkippedShards(), equalTo(0)); + assertThat(remoteClusterSearchInfo.getFailedShards(), equalTo(1)); + assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(1)); + assertThat(remoteClusterSearchInfo.getTook().millis(), greaterThan(0L)); + ShardSearchFailure remoteShardSearchFailure = remoteClusterSearchInfo.getFailures().get(0); + assertTrue("should have 'index corrupted' in reason", remoteShardSearchFailure.reason().contains("index corrupted")); + } + } + + public void testRemoteClusterOnlyCCSWithFailuresOnAllShards() throws Exception { + // for remote-only queries, we can't use the SearchListenerPlugin since that listens for search + // stage on the local cluster, so we only test final state of the search response + SearchListenerPlugin.negate(); + + Map testClusterInfo = setupTwoClusters(); + String remoteIndex = (String) testClusterInfo.get("remote.index"); + boolean skipUnavailable = (Boolean) testClusterInfo.get("remote.skip_unavailable"); + + SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(REMOTE_CLUSTER + ":" + remoteIndex); + request.setCcsMinimizeRoundtrips(true); + request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1)); + request.setKeepOnCompletion(true); + // shardId -1 means to throw the Exception on all shards, so should result in complete search failure + ThrowingQueryBuilder queryBuilder = new ThrowingQueryBuilder(randomLong(), new IllegalStateException("index corrupted"), -1); + request.getSearchRequest().source(new SearchSourceBuilder().query(queryBuilder).size(10)); + + AsyncSearchResponse response = submitAsyncSearch(request); + assertNotNull(response.getSearchResponse()); + assertTrue(response.isRunning()); + + assertBusy(() -> { + AsyncStatusResponse statusResponse = getAsyncStatus(response.getId()); + assertFalse(statusResponse.isRunning()); + assertNotNull(statusResponse.getCompletionStatus()); + }); + + { + AsyncSearchResponse finishedResponse = getAsyncSearch(response.getId()); + + SearchResponse.Clusters clusters = finishedResponse.getSearchResponse().getClusters(); + assertThat(clusters.getTotal(), equalTo(1)); + assertThat(clusters.getSuccessful(), equalTo(0)); + assertThat(clusters.getSkipped(), equalTo(1)); + + assertNull(clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY)); + + SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get(); + assertNotNull(remoteClusterSearchInfo); + SearchResponse.Cluster.Status expectedStatus = skipUnavailable + ? SearchResponse.Cluster.Status.SKIPPED + : SearchResponse.Cluster.Status.FAILED; + assertThat(remoteClusterSearchInfo.getStatus(), equalTo(expectedStatus)); + assertNull(remoteClusterSearchInfo.getTotalShards()); + assertNull(remoteClusterSearchInfo.getSuccessfulShards()); + assertNull(remoteClusterSearchInfo.getSkippedShards()); + assertNull(remoteClusterSearchInfo.getFailedShards()); + assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(1)); + assertNull(remoteClusterSearchInfo.getTook()); + assertFalse(remoteClusterSearchInfo.isTimedOut()); + ShardSearchFailure remoteShardSearchFailure = remoteClusterSearchInfo.getFailures().get(0); + assertTrue("should have 'index corrupted' in reason", remoteShardSearchFailure.reason().contains("index corrupted")); + } + // check that the async_search/status response includes the same cluster details + { + AsyncStatusResponse statusResponse = getAsyncStatus(response.getId()); + SearchResponse.Clusters clusters = statusResponse.getClusters(); + assertThat(clusters.getTotal(), equalTo(1)); + assertThat(clusters.getSuccessful(), equalTo(0)); + assertThat(clusters.getSkipped(), equalTo(1)); + + assertNull(clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY)); + + SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get(); + assertNotNull(remoteClusterSearchInfo); + SearchResponse.Cluster.Status expectedStatus = skipUnavailable + ? SearchResponse.Cluster.Status.SKIPPED + : SearchResponse.Cluster.Status.FAILED; + assertThat(remoteClusterSearchInfo.getStatus(), equalTo(expectedStatus)); + assertNull(remoteClusterSearchInfo.getTotalShards()); + assertNull(remoteClusterSearchInfo.getSuccessfulShards()); + assertNull(remoteClusterSearchInfo.getSkippedShards()); + assertNull(remoteClusterSearchInfo.getFailedShards()); + assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(1)); + assertNull(remoteClusterSearchInfo.getTook()); + assertFalse(remoteClusterSearchInfo.isTimedOut()); + ShardSearchFailure remoteShardSearchFailure = remoteClusterSearchInfo.getFailures().get(0); + assertTrue("should have 'index corrupted' in reason", remoteShardSearchFailure.reason().contains("index corrupted")); } } public void testCancelViaTasksAPI() throws Exception { - setupTwoClusters(); + Map testClusterInfo = setupTwoClusters(); + String localIndex = (String) testClusterInfo.get("local.index"); + String remoteIndex = (String) testClusterInfo.get("remote.index"); SearchListenerPlugin.blockQueryPhase(); - SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest("demo", REMOTE_CLUSTER + ":prod"); + SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(localIndex, REMOTE_CLUSTER + ":" + remoteIndex); request.setCcsMinimizeRoundtrips(randomBoolean()); request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1)); request.setKeepOnCompletion(true); @@ -250,11 +939,13 @@ public void testCancelViaTasksAPI() throws Exception { } public void testCancelViaAsyncSearchDelete() throws Exception { - setupTwoClusters(); + Map testClusterInfo = setupTwoClusters(); + String localIndex = (String) testClusterInfo.get("local.index"); + String remoteIndex = (String) testClusterInfo.get("remote.index"); SearchListenerPlugin.blockQueryPhase(); - SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest("demo", REMOTE_CLUSTER + ":prod"); + SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(localIndex, REMOTE_CLUSTER + ":" + remoteIndex); request.setCcsMinimizeRoundtrips(randomBoolean()); request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1)); request.setKeepOnCompletion(true); @@ -348,7 +1039,9 @@ public void testCancelViaAsyncSearchDelete() throws Exception { @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/97286") public void testCancellationViaTimeoutWithAllowPartialResultsSetToFalse() throws Exception { - setupTwoClusters(); + Map testClusterInfo = setupTwoClusters(); + String localIndex = (String) testClusterInfo.get("local.index"); + String remoteIndex = (String) testClusterInfo.get("remote.index"); SearchListenerPlugin.blockQueryPhase(); @@ -357,7 +1050,7 @@ public void testCancellationViaTimeoutWithAllowPartialResultsSetToFalse() throws SlowRunningQueryBuilder slowRunningQueryBuilder = new SlowRunningQueryBuilder(searchTimeout.millis() * 5); SearchSourceBuilder sourceBuilder = new SearchSourceBuilder().query(slowRunningQueryBuilder).timeout(searchTimeout); - SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest("demo", REMOTE_CLUSTER + ":prod"); + SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(localIndex, REMOTE_CLUSTER + ":" + remoteIndex); request.setCcsMinimizeRoundtrips(randomBoolean()); request.getSearchRequest().source(sourceBuilder); request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1)); @@ -457,48 +1150,50 @@ protected AcknowledgedResponse deleteAsyncSearch(String id) throws ExecutionExce return client().execute(DeleteAsyncResultAction.INSTANCE, new DeleteAsyncResultRequest(id)).get(); } - private void setupTwoClusters() throws Exception { - assertAcked(client(LOCAL_CLUSTER).admin().indices().prepareCreate("demo")); - indexDocs(client(LOCAL_CLUSTER), "demo"); + private Map setupTwoClusters() { + String localIndex = "demo"; + int numShardsLocal = randomIntBetween(3, 6); + Settings localSettings = indexSettings(numShardsLocal, 0).build(); + assertAcked(client(LOCAL_CLUSTER).admin().indices().prepareCreate(localIndex).setSettings(localSettings)); + indexDocs(client(LOCAL_CLUSTER), localIndex); + + String remoteIndex = "prod"; + int numShardsRemote = randomIntBetween(3, 6); final InternalTestCluster remoteCluster = cluster(REMOTE_CLUSTER); - remoteCluster.ensureAtLeastNumDataNodes(1); - final Settings.Builder allocationFilter = Settings.builder(); - if (randomBoolean()) { - remoteCluster.ensureAtLeastNumDataNodes(3); - List remoteDataNodes = remoteCluster.clusterService() - .state() - .nodes() - .stream() - .filter(DiscoveryNode::canContainData) - .map(DiscoveryNode::getName) - .toList(); - assertThat(remoteDataNodes.size(), Matchers.greaterThanOrEqualTo(3)); - List seedNodes = randomSubsetOf(between(1, remoteDataNodes.size() - 1), remoteDataNodes); - disconnectFromRemoteClusters(); - configureRemoteCluster(REMOTE_CLUSTER, seedNodes); - if (randomBoolean()) { - // Using proxy connections - allocationFilter.put("index.routing.allocation.exclude._name", String.join(",", seedNodes)); - } else { - allocationFilter.put("index.routing.allocation.include._name", String.join(",", seedNodes)); - } - } + remoteCluster.ensureAtLeastNumDataNodes(randomIntBetween(1, 3)); + final Settings.Builder remoteSettings = Settings.builder(); + remoteSettings.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, numShardsRemote); + assertAcked( client(REMOTE_CLUSTER).admin() .indices() - .prepareCreate("prod") - .setSettings(Settings.builder().put(allocationFilter.build()).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0)) + .prepareCreate(remoteIndex) + .setSettings(Settings.builder().put(remoteSettings.build()).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0)) ); assertFalse( client(REMOTE_CLUSTER).admin() .cluster() - .prepareHealth("prod") + .prepareHealth(remoteIndex) .setWaitForYellowStatus() .setTimeout(TimeValue.timeValueSeconds(10)) .get() .isTimedOut() ); - indexDocs(client(REMOTE_CLUSTER), "prod"); + indexDocs(client(REMOTE_CLUSTER), remoteIndex); + + String skipUnavailableKey = Strings.format("cluster.remote.%s.skip_unavailable", REMOTE_CLUSTER); + Setting skipUnavailableSetting = cluster(REMOTE_CLUSTER).clusterService().getClusterSettings().get(skipUnavailableKey); + boolean skipUnavailable = (boolean) cluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY).clusterService() + .getClusterSettings() + .get(skipUnavailableSetting); + + Map clusterInfo = new HashMap<>(); + clusterInfo.put("local.num_shards", numShardsLocal); + clusterInfo.put("local.index", localIndex); + clusterInfo.put("remote.num_shards", numShardsRemote); + clusterInfo.put("remote.index", remoteIndex); + clusterInfo.put("remote.skip_unavailable", skipUnavailable); + return clusterInfo; } private int indexDocs(Client client, String index) { @@ -520,6 +1215,22 @@ public static class SearchListenerPlugin extends Plugin { private static final AtomicReference queryLatch = new AtomicReference<>(); private static final AtomicReference failedQueryLatch = new AtomicReference<>(); + /** + * For tests that cannot use SearchListenerPlugin, ensure all latches are unset to + * avoid test problems around searches of the .async-search index + */ + static void negate() { + if (startedLatch.get() != null) { + startedLatch.get().countDown(); + } + if (queryLatch.get() != null) { + queryLatch.get().countDown(); + } + if (failedQueryLatch.get() != null) { + failedQueryLatch.get().countDown(); + } + } + static void reset() { startedLatch.set(new CountDownLatch(1)); failedQueryLatch.set(new CountDownLatch(1)); diff --git a/x-pack/plugin/async-search/src/main/java/org/elasticsearch/xpack/search/AsyncSearchTask.java b/x-pack/plugin/async-search/src/main/java/org/elasticsearch/xpack/search/AsyncSearchTask.java index 3388bb88de670..2f3e5409642f7 100644 --- a/x-pack/plugin/async-search/src/main/java/org/elasticsearch/xpack/search/AsyncSearchTask.java +++ b/x-pack/plugin/async-search/src/main/java/org/elasticsearch/xpack/search/AsyncSearchTask.java @@ -433,7 +433,7 @@ public void onPartialReduce(List shards, TotalHits totalHits, Inter */ reducedAggs = () -> InternalAggregations.topLevelReduce(singletonList(aggregations), aggReduceContextSupplier.get()); } - searchResponse.get().updatePartialResponse(shards.size(), totalHits, reducedAggs, reducePhase, false); + searchResponse.get().updatePartialResponse(shards.size(), totalHits, reducedAggs, reducePhase); } /** @@ -444,7 +444,7 @@ public void onPartialReduce(List shards, TotalHits totalHits, Inter public void onFinalReduce(List shards, TotalHits totalHits, InternalAggregations aggregations, int reducePhase) { // best effort to cancel expired tasks checkCancellation(); - searchResponse.get().updatePartialResponse(shards.size(), totalHits, () -> aggregations, reducePhase, true); + searchResponse.get().updatePartialResponse(shards.size(), totalHits, () -> aggregations, reducePhase); } @Override diff --git a/x-pack/plugin/async-search/src/main/java/org/elasticsearch/xpack/search/MutableSearchResponse.java b/x-pack/plugin/async-search/src/main/java/org/elasticsearch/xpack/search/MutableSearchResponse.java index 6ec62a5923c49..dc6c780c64644 100644 --- a/x-pack/plugin/async-search/src/main/java/org/elasticsearch/xpack/search/MutableSearchResponse.java +++ b/x-pack/plugin/async-search/src/main/java/org/elasticsearch/xpack/search/MutableSearchResponse.java @@ -6,8 +6,6 @@ */ package org.elasticsearch.xpack.search; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; import org.apache.lucene.search.TotalHits; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; @@ -38,12 +36,10 @@ * run concurrently to 1 and ensures that we pause the search progress when an {@link AsyncSearchResponse} is built. */ class MutableSearchResponse { - - private static final Logger logger = LogManager.getLogger(MutableSearchResponse.class); private static final TotalHits EMPTY_TOTAL_HITS = new TotalHits(0L, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO); private final int totalShards; private final int skippedShards; - private Clusters clusters; + private final Clusters clusters; private final AtomicArray queryFailures; private final ThreadContext threadContext; @@ -90,16 +86,13 @@ class MutableSearchResponse { /** * Updates the response with the result of a partial reduction. * @param reducedAggs is a strategy for producing the reduced aggs - * @param isFinalLocalReduce true if the local cluster search has finished (during CCS with minimize_roundtrips, this can be true - * even while the overall search is still running on remote clusters) */ @SuppressWarnings("HiddenField") synchronized void updatePartialResponse( int successfulShards, TotalHits totalHits, Supplier reducedAggs, - int reducePhase, - boolean isFinalLocalReduce + int reducePhase ) { failIfFrozen(); if (reducePhase < this.reducePhase) { @@ -112,20 +105,6 @@ synchronized void updatePartialResponse( this.totalHits = totalHits; this.reducedAggsSource = reducedAggs; this.reducePhase = reducePhase; - if (isFinalLocalReduce && clusters.isCcsMinimizeRoundtrips()) { - // currently only ccsMinimizeRoundTrip=true creates Clusters in their initial state (where successful=0) - // ccsMinimizeRoundtrips=false creates Clusters in its final state even at the beginning (successful+skipped=total) - // so update the clusters object 'successful' count if local cluster search is done AND ccsMinimizeRoundtrips=true - Clusters newClusters = new Clusters( - clusters.getTotal(), - clusters.getSuccessful() + 1, - clusters.getSkipped(), - clusters.getRemoteClusters(), - clusters.isCcsMinimizeRoundtrips() - ); - this.clusters = newClusters; - logger.debug("Updating Clusters info to indicate that the local cluster search has completed: {}", newClusters); - } } /** @@ -140,10 +119,17 @@ assert shardsInResponseMatchExpected(response, ccsMinimizeRoundtrips) this.responseHeaders = threadContext.getResponseHeaders(); this.finalResponse = response; - this.isPartial = false; + this.isPartial = isPartialResponse(response); this.frozen = true; } + private boolean isPartialResponse(SearchResponse response) { + if (response.getClusters() == null) { + return true; + } + return response.getClusters().hasPartialResults(); + } + /** * Updates the response with a fatal failure. This method preserves the partial response * received from previous updates diff --git a/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/AsyncSearchResponseTests.java b/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/AsyncSearchResponseTests.java index 3ed5a51ee0417..c4485e237319a 100644 --- a/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/AsyncSearchResponseTests.java +++ b/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/AsyncSearchResponseTests.java @@ -7,20 +7,27 @@ package org.elasticsearch.xpack.search; +import org.apache.lucene.index.CorruptIndexException; import org.elasticsearch.TransportVersion; +import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponseSections; import org.elasticsearch.action.search.ShardSearchFailure; +import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentElasticsearchExtension; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.script.ScriptException; import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchModule; +import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.internal.InternalSearchResponse; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.transport.RemoteClusterAware; import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.XContentType; @@ -32,13 +39,17 @@ import java.util.Arrays; import java.util.Collections; import java.util.Date; +import java.util.HashMap; import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicReference; import static java.util.Collections.emptyList; import static org.elasticsearch.xpack.core.async.GetAsyncResultRequestTests.randomSearchId; public class AsyncSearchResponseTests extends ESTestCase { - private SearchResponse searchResponse = randomSearchResponse(); + private SearchResponse searchResponse = randomSearchResponse(randomBoolean()); private NamedWriteableRegistry namedWriteableRegistry; @Before @@ -113,12 +124,18 @@ static AsyncSearchResponse randomAsyncSearchResponse(String searchId, SearchResp }; } - static SearchResponse randomSearchResponse() { + static SearchResponse randomSearchResponse(boolean ccs) { long tookInMillis = randomNonNegativeLong(); int totalShards = randomIntBetween(1, Integer.MAX_VALUE); int successfulShards = randomIntBetween(0, totalShards); int skippedShards = randomIntBetween(0, successfulShards); InternalSearchResponse internalSearchResponse = InternalSearchResponse.EMPTY_WITH_TOTAL_HITS; + SearchResponse.Clusters clusters; + if (ccs) { + clusters = createCCSClusterObjects(20, 19, true, 10, 1, 2); + } else { + clusters = SearchResponse.Clusters.EMPTY; + } return new SearchResponse( internalSearchResponse, null, @@ -127,7 +144,7 @@ static SearchResponse randomSearchResponse() { skippedShards, tookInMillis, ShardSearchFailure.EMPTY_ARRAY, - SearchResponse.Clusters.EMPTY + clusters ); } @@ -201,7 +218,7 @@ public void testToXContentWithSearchResponseAfterCompletion() throws IOException 9, 1, took, - new ShardSearchFailure[0], + ShardSearchFailure.EMPTY_ARRAY, SearchResponse.Clusters.EMPTY ); @@ -290,6 +307,347 @@ public void testToXContentWithSearchResponseAfterCompletion() throws IOException } } + public void testToXContentWithCCSSearchResponseWhileRunning() throws IOException { + boolean isRunning = true; + long startTimeMillis = 1689352924517L; + long expirationTimeMillis = 1689784924517L; + long took = 22968L; + + SearchHits hits = SearchHits.EMPTY_WITHOUT_TOTAL_HITS; + SearchResponseSections sections = new SearchResponseSections(hits, null, null, false, null, null, 2); + + SearchResponse.Clusters clusters = createCCSClusterObjects(3, 3, true); + + SearchResponse searchResponse = new SearchResponse(sections, null, 10, 9, 1, took, ShardSearchFailure.EMPTY_ARRAY, clusters); + + AsyncSearchResponse asyncSearchResponse = new AsyncSearchResponse( + "id", + searchResponse, + null, + true, + isRunning, + startTimeMillis, + expirationTimeMillis + ); + + try (XContentBuilder builder = XContentBuilder.builder(XContentType.JSON.xContent())) { + builder.prettyPrint(); + asyncSearchResponse.toXContent(builder, ToXContent.EMPTY_PARAMS); + assertEquals(Strings.format(""" + { + "id" : "id", + "is_partial" : true, + "is_running" : true, + "start_time_in_millis" : %s, + "expiration_time_in_millis" : %s, + "response" : { + "took" : %s, + "timed_out" : false, + "num_reduce_phases" : 2, + "_shards" : { + "total" : 10, + "successful" : 9, + "skipped" : 1, + "failed" : 0 + }, + "_clusters" : { + "total" : 3, + "successful" : 0, + "skipped" : 0, + "details" : { + "cluster_1" : { + "status" : "running", + "indices" : "foo,bar*", + "timed_out" : false + }, + "cluster_2" : { + "status" : "running", + "indices" : "foo,bar*", + "timed_out" : false + }, + "cluster_0" : { + "status" : "running", + "indices" : "foo,bar*", + "timed_out" : false + } + } + }, + "hits" : { + "max_score" : 0.0, + "hits" : [ ] + } + } + }""", startTimeMillis, expirationTimeMillis, took), Strings.toString(builder)); + } + + try (XContentBuilder builder = XContentBuilder.builder(XContentType.JSON.xContent())) { + builder.prettyPrint(); + builder.humanReadable(true); + asyncSearchResponse.toXContent(builder, new ToXContent.MapParams(Collections.singletonMap("human", "true"))); + assertEquals( + Strings.format( + """ + { + "id" : "id", + "is_partial" : true, + "is_running" : true, + "start_time" : "%s", + "start_time_in_millis" : %s, + "expiration_time" : "%s", + "expiration_time_in_millis" : %s, + "response" : { + "took" : %s, + "timed_out" : false, + "num_reduce_phases" : 2, + "_shards" : { + "total" : 10, + "successful" : 9, + "skipped" : 1, + "failed" : 0 + }, + "_clusters" : { + "total" : 3, + "successful" : 0, + "skipped" : 0, + "details" : { + "cluster_1" : { + "status" : "running", + "indices" : "foo,bar*", + "timed_out" : false + }, + "cluster_2" : { + "status" : "running", + "indices" : "foo,bar*", + "timed_out" : false + }, + "cluster_0" : { + "status" : "running", + "indices" : "foo,bar*", + "timed_out" : false + } + } + }, + "hits" : { + "max_score" : 0.0, + "hits" : [ ] + } + } + }""", + XContentElasticsearchExtension.DEFAULT_FORMATTER.format(Instant.ofEpochMilli(startTimeMillis)), + startTimeMillis, + XContentElasticsearchExtension.DEFAULT_FORMATTER.format(Instant.ofEpochMilli(expirationTimeMillis)), + expirationTimeMillis, + took + ), + Strings.toString(builder) + ); + } + } + + // completion_time should be present since search has completed + public void testToXContentWithCCSSearchResponseAfterCompletion() throws IOException { + boolean isRunning = false; + long startTimeMillis = 1689352924517L; + long expirationTimeMillis = 1689784924517L; + long took = 22968L; + long expectedCompletionTime = startTimeMillis + took; + + SearchHits hits = SearchHits.EMPTY_WITHOUT_TOTAL_HITS; + SearchResponseSections sections = new SearchResponseSections(hits, null, null, true, null, null, 2); + SearchResponse.Clusters clusters = createCCSClusterObjects(4, 3, true); + + AtomicReference clusterRef = clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY); + SearchResponse.Cluster localCluster = clusterRef.get(); + SearchResponse.Cluster updated = new SearchResponse.Cluster( + localCluster.getClusterAlias(), + localCluster.getIndexExpression(), + SearchResponse.Cluster.Status.SUCCESSFUL, + 10, + 10, + 3, + 0, + Collections.emptyList(), + new TimeValue(11111), + false + ); + boolean swapped = clusterRef.compareAndSet(localCluster, updated); + assertTrue("CAS swap failed for cluster " + updated, swapped); + + clusterRef = clusters.getCluster("cluster_0"); + SearchResponse.Cluster cluster0 = clusterRef.get(); + updated = new SearchResponse.Cluster( + cluster0.getClusterAlias(), + cluster0.getIndexExpression(), + SearchResponse.Cluster.Status.SUCCESSFUL, + 8, + 8, + 1, + 0, + Collections.emptyList(), + new TimeValue(7777), + false + ); + swapped = clusterRef.compareAndSet(cluster0, updated); + assertTrue("CAS swap failed for cluster " + updated, swapped); + + clusterRef = clusters.getCluster("cluster_1"); + SearchResponse.Cluster cluster1 = clusterRef.get(); + ShardSearchFailure failure1 = new ShardSearchFailure( + new NullPointerException("NPE details"), + new SearchShardTarget("nodeId0", new ShardId("foo", UUID.randomUUID().toString(), 0), "cluster_1") + ); + ShardSearchFailure failure2 = new ShardSearchFailure( + new CorruptIndexException("abc", "123"), + new SearchShardTarget("nodeId0", new ShardId("bar1", UUID.randomUUID().toString(), 0), "cluster_1") + ); + updated = new SearchResponse.Cluster( + cluster1.getClusterAlias(), + cluster1.getIndexExpression(), + SearchResponse.Cluster.Status.SKIPPED, + 2, + 0, + 0, + 2, + List.of(failure1, failure2), + null, + false + ); + swapped = clusterRef.compareAndSet(cluster1, updated); + assertTrue("CAS swap failed for cluster " + updated, swapped); + + clusterRef = clusters.getCluster("cluster_2"); + SearchResponse.Cluster cluster2 = clusterRef.get(); + updated = new SearchResponse.Cluster( + cluster2.getClusterAlias(), + cluster2.getIndexExpression(), + SearchResponse.Cluster.Status.PARTIAL, + 8, + 8, + 0, + 0, + Collections.emptyList(), + new TimeValue(17322), + true + ); + swapped = clusterRef.compareAndSet(cluster2, updated); + assertTrue("CAS swap failed for cluster " + updated, swapped); + + SearchResponse searchResponse = new SearchResponse(sections, null, 10, 9, 1, took, new ShardSearchFailure[0], clusters); + + AsyncSearchResponse asyncSearchResponse = new AsyncSearchResponse( + "id", + searchResponse, + null, + false, + isRunning, + startTimeMillis, + expirationTimeMillis + ); + + try (XContentBuilder builder = XContentBuilder.builder(XContentType.JSON.xContent())) { + builder.prettyPrint(); + asyncSearchResponse.toXContent(builder, ToXContent.EMPTY_PARAMS); + assertEquals(Strings.format(""" + { + "id" : "id", + "is_partial" : false, + "is_running" : false, + "start_time_in_millis" : %s, + "expiration_time_in_millis" : %s, + "completion_time_in_millis" : %s, + "response" : { + "took" : %s, + "timed_out" : true, + "num_reduce_phases" : 2, + "_shards" : { + "total" : 10, + "successful" : 9, + "skipped" : 1, + "failed" : 0 + }, + "_clusters" : { + "total" : 4, + "successful" : 3, + "skipped" : 1, + "details" : { + "(local)" : { + "status" : "successful", + "indices" : "foo,bar*", + "took" : 11111, + "timed_out" : false, + "_shards" : { + "total" : 10, + "successful" : 10, + "skipped" : 3, + "failed" : 0 + } + }, + "cluster_1" : { + "status" : "skipped", + "indices" : "foo,bar*", + "timed_out" : false, + "_shards" : { + "total" : 2, + "successful" : 0, + "skipped" : 0, + "failed" : 2 + }, + "failures" : [ + { + "shard" : 0, + "index" : "cluster_1:foo", + "node" : "nodeId0", + "reason" : { + "type" : "null_pointer_exception", + "reason" : "NPE details" + } + }, + { + "shard" : 0, + "index" : "cluster_1:bar1", + "node" : "nodeId0", + "reason" : { + "type" : "corrupt_index_exception", + "reason" : "abc (resource=123)" + } + } + ] + }, + "cluster_2" : { + "status" : "partial", + "indices" : "foo,bar*", + "took" : 17322, + "timed_out" : true, + "_shards" : { + "total" : 8, + "successful" : 8, + "skipped" : 0, + "failed" : 0 + } + }, + "cluster_0" : { + "status" : "successful", + "indices" : "foo,bar*", + "took" : 7777, + "timed_out" : false, + "_shards" : { + "total" : 8, + "successful" : 8, + "skipped" : 1, + "failed" : 0 + } + } + } + }, + "hits" : { + "max_score" : 0.0, + "hits" : [ ] + } + } + }""", startTimeMillis, expirationTimeMillis, expectedCompletionTime, took), Strings.toString(builder)); + } + } + // completion_time should NOT be present since search is still running public void testToXContentWithSearchResponseWhileRunning() throws IOException { boolean isRunning = true; @@ -306,7 +664,7 @@ public void testToXContentWithSearchResponseWhileRunning() throws IOException { 9, 1, took, - new ShardSearchFailure[0], + ShardSearchFailure.EMPTY_ARRAY, SearchResponse.Clusters.EMPTY ); @@ -389,4 +747,144 @@ public void testToXContentWithSearchResponseWhileRunning() throws IOException { ); } } + + static SearchResponse.Clusters createCCSClusterObjects(int totalClusters, int remoteClusters, boolean ccsMinimizeRoundtrips) { + OriginalIndices localIndices = null; + if (totalClusters > remoteClusters) { + localIndices = new OriginalIndices(new String[] { "foo", "bar*" }, IndicesOptions.lenientExpand()); + } + assert remoteClusters > 0 : "CCS Cluster must have at least one remote cluster"; + Map remoteClusterIndices = new HashMap<>(); + for (int i = 0; i < remoteClusters; i++) { + remoteClusterIndices.put("cluster_" + i, new OriginalIndices(new String[] { "foo", "bar*" }, IndicesOptions.lenientExpand())); + } + + return new SearchResponse.Clusters(localIndices, remoteClusterIndices, ccsMinimizeRoundtrips); + } + + static SearchResponse.Clusters createCCSClusterObjects( + int totalClusters, + int remoteClusters, + boolean ccsMinimizeRoundtrips, + int successfulClusters, + int skippedClusters, + int partialClusters + ) { + assert successfulClusters + skippedClusters <= totalClusters : "successful + skipped > totalClusters"; + assert totalClusters == remoteClusters || totalClusters - remoteClusters == 1 + : "totalClusters and remoteClusters must be same or total = remote + 1"; + assert successfulClusters + skippedClusters + partialClusters > 0 : "successful + skipped + partial must be > 0"; + + SearchResponse.Clusters clusters = createCCSClusterObjects(totalClusters, remoteClusters, ccsMinimizeRoundtrips); + + int successful = successfulClusters; + int skipped = skippedClusters; + int partial = partialClusters; + if (totalClusters > remoteClusters) { + String localAlias = RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY; + AtomicReference localRef = clusters.getCluster(localAlias); + SearchResponse.Cluster orig = localRef.get(); + SearchResponse.Cluster updated; + if (successful > 0) { + updated = new SearchResponse.Cluster( + localAlias, + localRef.get().getIndexExpression(), + SearchResponse.Cluster.Status.SUCCESSFUL, + 5, + 5, + 0, + 0, + Collections.emptyList(), + new TimeValue(1000), + false + ); + successful--; + } else if (skipped > 0) { + updated = new SearchResponse.Cluster( + localAlias, + localRef.get().getIndexExpression(), + SearchResponse.Cluster.Status.SKIPPED, + 5, + 0, + 0, + 5, + Collections.emptyList(), + new TimeValue(1000), + false + ); + skipped--; + } else { + updated = new SearchResponse.Cluster( + localAlias, + localRef.get().getIndexExpression(), + SearchResponse.Cluster.Status.PARTIAL, + 5, + 2, + 1, + 3, + Collections.emptyList(), + new TimeValue(1000), + false + ); + partial--; + } + boolean swapped = localRef.compareAndSet(orig, updated); + assertTrue("CAS swap failed for cluster " + updated, swapped); + } + + int numClusters = successful + skipped + partial; + + for (int i = 0; i < numClusters; i++) { + String clusterAlias = "cluster_" + i; + AtomicReference clusterRef = clusters.getCluster(clusterAlias); + SearchResponse.Cluster orig = clusterRef.get(); + SearchResponse.Cluster updated; + if (successful > 0) { + updated = new SearchResponse.Cluster( + clusterAlias, + clusterRef.get().getIndexExpression(), + SearchResponse.Cluster.Status.SUCCESSFUL, + 5, + 5, + 0, + 0, + Collections.emptyList(), + new TimeValue(1000), + false + ); + successful--; + } else if (skipped > 0) { + updated = new SearchResponse.Cluster( + clusterAlias, + clusterRef.get().getIndexExpression(), + SearchResponse.Cluster.Status.SKIPPED, + 5, + 0, + 0, + 5, + Collections.emptyList(), + new TimeValue(1000), + false + ); + skipped--; + } else { + updated = new SearchResponse.Cluster( + clusterAlias, + clusterRef.get().getIndexExpression(), + SearchResponse.Cluster.Status.PARTIAL, + 5, + 2, + 1, + 3, + Collections.emptyList(), + new TimeValue(1000), + false + ); + partial--; + } + boolean swapped = clusterRef.compareAndSet(orig, updated); + assertTrue("CAS swap failed for cluster " + updated, swapped); + } + return clusters; + } } diff --git a/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/AsyncStatusResponseTests.java b/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/AsyncStatusResponseTests.java index 3e4aa163660b1..eabd2e08f5745 100644 --- a/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/AsyncStatusResponseTests.java +++ b/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/AsyncStatusResponseTests.java @@ -47,7 +47,7 @@ protected AsyncStatusResponse createTestInstance() { SearchResponse.Clusters clusters = switch (randomIntBetween(0, 3)) { case 1 -> SearchResponse.Clusters.EMPTY; case 2 -> new SearchResponse.Clusters(1, 1, 0); - case 3 -> new SearchResponse.Clusters(4, 1, 0, 3, true); + case 3 -> AsyncSearchResponseTests.createCCSClusterObjects(4, 3, true); default -> null; // case 0 }; return new AsyncStatusResponse( @@ -80,7 +80,7 @@ protected AsyncStatusResponse mutateInstance(AsyncStatusResponse instance) { SearchResponse.Clusters clusters = switch (randomIntBetween(0, 3)) { case 1 -> SearchResponse.Clusters.EMPTY; case 2 -> new SearchResponse.Clusters(1, 1, 0); - case 3 -> new SearchResponse.Clusters(4, 1, 0, 3, true); + case 3 -> AsyncSearchResponseTests.createCCSClusterObjects(4, 3, true); // new SearchResponse.Clusters(4, 1, 0, 3, true); default -> null; // case 0 }; return new AsyncStatusResponse( @@ -140,7 +140,7 @@ public void testToXContent() throws IOException { %s } """, args); - } else { + } else if (clusters.getTotal() == 1) { Object[] args = new Object[] { response.getId(), response.isRunning(), @@ -180,6 +180,69 @@ public void testToXContent() throws IOException { %s } """, args); + } else { + Object[] args = new Object[] { + response.getId(), + response.isRunning(), + response.isPartial(), + response.getStartTime(), + response.getExpirationTime(), + completionTimeEntry, + response.getTotalShards(), + response.getSuccessfulShards(), + response.getSkippedShards(), + response.getFailedShards(), + clusters.getTotal(), + clusters.getSuccessful(), + clusters.getSkipped(), + response.getCompletionStatus() == null ? "" : Strings.format(""" + ,"completion_status" : %s""", response.getCompletionStatus().getStatus()) }; + + expectedJson = Strings.format(""" + { + "id" : "%s", + "is_running" : %s, + "is_partial" : %s, + "start_time_in_millis" : %s, + "expiration_time_in_millis" : %s, + %s + "_shards" : { + "total" : %s, + "successful" : %s, + "skipped" : %s, + "failed" : %s + }, + "_clusters": { + "total": %s, + "successful": %s, + "skipped": %s, + "details": { + "(local)": { + "status": "running", + "indices": "foo,bar*", + "timed_out": false + }, + "cluster_1": { + "status": "running", + "indices": "foo,bar*", + "timed_out": false + }, + "cluster_2": { + "status": "running", + "indices": "foo,bar*", + "timed_out": false + }, + "cluster_0": { + "status": "running", + "indices": "foo,bar*", + "timed_out": false + } + } + } + %s + } + """, args); + } response.toXContent(builder, ToXContent.EMPTY_PARAMS); assertEquals(XContentHelper.stripWhitespace(expectedJson), Strings.toString(builder)); @@ -187,10 +250,11 @@ public void testToXContent() throws IOException { } public void testGetStatusFromStoredSearchRandomizedInputs() { + boolean ccs = randomBoolean(); String searchId = randomSearchId(); AsyncSearchResponse asyncSearchResponse = AsyncSearchResponseTests.randomAsyncSearchResponse( searchId, - AsyncSearchResponseTests.randomSearchResponse() + AsyncSearchResponseTests.randomSearchResponse(ccs) ); if (asyncSearchResponse.getSearchResponse() == null @@ -241,7 +305,7 @@ public void testGetStatusFromStoredSearchFailedShardsScenario() { int successfulShards = randomIntBetween(0, totalShards); int skippedShards = randomIntBetween(0, successfulShards); InternalSearchResponse internalSearchResponse = InternalSearchResponse.EMPTY_WITH_TOTAL_HITS; - SearchResponse.Clusters clusters = new SearchResponse.Clusters(100, 99, 1, 99, false); + SearchResponse.Clusters clusters = new SearchResponse.Clusters(100, 99, 1); SearchResponse searchResponse = new SearchResponse( internalSearchResponse, null, @@ -268,6 +332,7 @@ public void testGetStatusFromStoredSearchWithEmptyClustersSuccessfullyCompleted( int successfulShards = randomIntBetween(0, totalShards); int skippedShards = randomIntBetween(0, successfulShards); InternalSearchResponse internalSearchResponse = InternalSearchResponse.EMPTY_WITH_TOTAL_HITS; + SearchResponse searchResponse = new SearchResponse( internalSearchResponse, null, @@ -294,7 +359,26 @@ public void testGetStatusFromStoredSearchWithNonEmptyClustersSuccessfullyComplet int successfulShards = randomIntBetween(0, totalShards); int skippedShards = randomIntBetween(0, successfulShards); InternalSearchResponse internalSearchResponse = InternalSearchResponse.EMPTY_WITH_TOTAL_HITS; - SearchResponse.Clusters clusters = new SearchResponse.Clusters(100, 99, 1, 99, false); + + int totalClusters; + int successfulClusters; + int skippedClusters; + SearchResponse.Clusters clusters; + if (randomBoolean()) { + // local search only + totalClusters = 1; + successfulClusters = 1; + skippedClusters = 0; + clusters = new SearchResponse.Clusters(totalClusters, successfulClusters, skippedClusters); + } else { + // CCS search + totalClusters = 80; + int successful = randomInt(60); + int partial = randomInt(20); + successfulClusters = successful + partial; + skippedClusters = totalClusters - successfulClusters; + clusters = AsyncSearchResponseTests.createCCSClusterObjects(80, 80, true, successful, skippedClusters, partial); + } SearchResponse searchResponse = new SearchResponse( internalSearchResponse, null, @@ -311,7 +395,9 @@ public void testGetStatusFromStoredSearchWithNonEmptyClustersSuccessfullyComplet assertNotNull(statusFromStoredSearch); assertEquals(0, statusFromStoredSearch.getFailedShards()); assertEquals(statusFromStoredSearch.getCompletionStatus(), RestStatus.OK); - assertEquals(100, statusFromStoredSearch.getClusters().getTotal()); + assertEquals(totalClusters, statusFromStoredSearch.getClusters().getTotal()); + assertEquals(skippedClusters, statusFromStoredSearch.getClusters().getSkipped()); + assertEquals(successfulClusters, statusFromStoredSearch.getClusters().getSuccessful()); } public void testGetStatusFromStoredSearchWithNonEmptyClustersStillRunning() { @@ -322,7 +408,11 @@ public void testGetStatusFromStoredSearchWithNonEmptyClustersStillRunning() { int successfulShards = randomIntBetween(0, totalShards); int skippedShards = randomIntBetween(0, successfulShards); InternalSearchResponse internalSearchResponse = InternalSearchResponse.EMPTY_WITH_TOTAL_HITS; - SearchResponse.Clusters clusters = new SearchResponse.Clusters(100, 2, 3, 99, true); + int successful = randomInt(10); + int partial = randomInt(10); + int skipped = randomInt(10); + SearchResponse.Clusters clusters = AsyncSearchResponseTests.createCCSClusterObjects(100, 99, true, successful, skipped, partial); + SearchResponse searchResponse = new SearchResponse( internalSearchResponse, null, @@ -341,7 +431,7 @@ public void testGetStatusFromStoredSearchWithNonEmptyClustersStillRunning() { assertEquals(0, statusFromStoredSearch.getFailedShards()); assertNull("completion_status should not be present if still running", statusFromStoredSearch.getCompletionStatus()); assertEquals(100, statusFromStoredSearch.getClusters().getTotal()); - assertEquals(2, statusFromStoredSearch.getClusters().getSuccessful()); - assertEquals(3, statusFromStoredSearch.getClusters().getSkipped()); + assertEquals(successful + partial, statusFromStoredSearch.getClusters().getSuccessful()); + assertEquals(skipped, statusFromStoredSearch.getClusters().getSkipped()); } } diff --git a/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/ThrowingQueryBuilder.java b/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/ThrowingQueryBuilder.java index 57aa9245c17b9..f65e7e5376b87 100644 --- a/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/ThrowingQueryBuilder.java +++ b/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/ThrowingQueryBuilder.java @@ -27,15 +27,36 @@ class ThrowingQueryBuilder extends AbstractQueryBuilder { private final long randomUID; private final RuntimeException failure; private final int shardId; + private final String index; /** * Creates a {@link ThrowingQueryBuilder} with the provided randomUID. + * + * @param randomUID used solely for identification + * @param failure what exception to throw + * @param shardId what shardId to throw the exception. If shardId is less than 0, it will throw for all shards. */ ThrowingQueryBuilder(long randomUID, RuntimeException failure, int shardId) { super(); this.randomUID = randomUID; this.failure = failure; this.shardId = shardId; + this.index = null; + } + + /** + * Creates a {@link ThrowingQueryBuilder} with the provided randomUID. + * + * @param randomUID used solely for identification + * @param failure what exception to throw + * @param index what index to throw the exception against (all shards of that index) + */ + ThrowingQueryBuilder(long randomUID, RuntimeException failure, String index) { + super(); + this.randomUID = randomUID; + this.failure = failure; + this.shardId = Integer.MAX_VALUE; + this.index = index; } ThrowingQueryBuilder(StreamInput in) throws IOException { @@ -43,6 +64,11 @@ class ThrowingQueryBuilder extends AbstractQueryBuilder { this.randomUID = in.readLong(); this.failure = in.readException(); this.shardId = in.readVInt(); + if (in.getTransportVersion().onOrAfter(TransportVersion.V_8_500_040)) { + this.index = in.readOptionalString(); + } else { + this.index = null; + } } @Override @@ -50,6 +76,9 @@ protected void doWriteTo(StreamOutput out) throws IOException { out.writeLong(randomUID); out.writeException(failure); out.writeVInt(shardId); + if (out.getTransportVersion().onOrAfter(TransportVersion.V_8_500_040)) { + out.writeOptionalString(index); + } } @Override @@ -64,7 +93,7 @@ protected Query doToQuery(SearchExecutionContext context) { return new Query() { @Override public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException { - if (context.getShardId() == shardId) { + if (context.getShardId() == shardId || shardId < 0 || context.index().getName().equals(index)) { throw failure; } return delegate.createWeight(searcher, scoreMode, boost); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/search/action/AsyncStatusResponse.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/search/action/AsyncStatusResponse.java index 43118eff57fd8..4d544b6bceb38 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/search/action/AsyncStatusResponse.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/search/action/AsyncStatusResponse.java @@ -189,7 +189,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } RestActions.buildBroadcastShardsHeader(builder, params, totalShards, successfulShards, skippedShards, failedShards, null); if (clusters != null) { - builder = clusters.toXContent(builder, null); + builder = clusters.toXContent(builder, params); } if (isRunning == false) { // completion status information is only available for a completed search builder.field("completion_status", completionStatus.getStatus()); From da389b465abbf50fe56dee5140a1f3af8a09348d Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 7 Aug 2023 18:14:03 +0100 Subject: [PATCH 02/11] Improve testclusters shutdown logging (#98253) Today the logging when shutting down a node does not indicate to which node it pertains. This commit adds a little more detail to aid with debugging. --- .../gradle/testclusters/ElasticsearchNode.java | 6 +++--- .../elasticsearch/test/cluster/util/ProcessUtils.java | 10 +++++++--- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/build-tools/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchNode.java b/build-tools/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchNode.java index 1a34eb750692c..f0ab67fe51a34 100644 --- a/build-tools/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchNode.java +++ b/build-tools/src/main/java/org/elasticsearch/gradle/testclusters/ElasticsearchNode.java @@ -1213,12 +1213,12 @@ private void waitForProcessToExit(ProcessHandle processHandle) { try { processHandle.onExit().get(ES_DESTROY_TIMEOUT, ES_DESTROY_TIMEOUT_UNIT); } catch (InterruptedException e) { - LOGGER.info("Interrupted while waiting for ES process", e); + LOGGER.info("[{}] Interrupted while waiting for ES process", name, e); Thread.currentThread().interrupt(); } catch (ExecutionException e) { - LOGGER.info("Failure while waiting for process to exist", e); + LOGGER.info("[{}] Failure while waiting for process to exist", name, e); } catch (TimeoutException e) { - LOGGER.info("Timed out waiting for process to exit", e); + LOGGER.info("[{}] Timed out waiting for process to exit", name, e); } } diff --git a/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/util/ProcessUtils.java b/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/util/ProcessUtils.java index 9685eac566a18..bcbf6cd9bbc37 100644 --- a/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/util/ProcessUtils.java +++ b/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/util/ProcessUtils.java @@ -117,7 +117,11 @@ public static void stopHandle(ProcessHandle processHandle, boolean forcibly) { if (processHandle.isAlive() == false) { return; } - LOGGER.info("Process did not terminate after {}, stopping it forcefully", PROCESS_DESTROY_TIMEOUT); + LOGGER.info( + "Process did not terminate after {}, stopping it forcefully: {}", + PROCESS_DESTROY_TIMEOUT, + processHandle.info() + ); processHandle.destroyForcibly(); } @@ -147,9 +151,9 @@ private static void waitForProcessToExit(ProcessHandle processHandle) { return processHandle.isAlive() == false; }); } catch (ExecutionException e) { - LOGGER.info("Failure while waiting for process to exist", e); + LOGGER.info("Failure while waiting for process to exit: {}", processHandle.info(), e); } catch (TimeoutException e) { - LOGGER.info("Timed out waiting for process to exit", e); + LOGGER.info("Timed out waiting for process to exit: {}", processHandle.info(), e); } } From 6e9b649dc12e27dd3b26fc0e1bb2396dd7f2a603 Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 7 Aug 2023 18:14:49 +0100 Subject: [PATCH 03/11] Implement ObjectPath#toString (#98251) If a REST test fails because of something unexpected in a response, it's useful to be able to see the full response. This commit adds a `toString()` implementation to `ObjectPath` to help with this kind of debugging. --- .../main/java/org/elasticsearch/test/rest/ObjectPath.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/ObjectPath.java b/test/framework/src/main/java/org/elasticsearch/test/rest/ObjectPath.java index 5cbb8138c7b50..412cc310458e1 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/ObjectPath.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/ObjectPath.java @@ -166,4 +166,9 @@ public XContentBuilder toXContentBuilder(XContent xContent) throws IOException { } return builder; } + + @Override + public String toString() { + return "ObjectPath[" + object + "]"; + } } From 303b51a4899001b6c41e94226d6c9c59dd1fea1c Mon Sep 17 00:00:00 2001 From: Tim Vernum Date: Tue, 8 Aug 2023 11:14:10 +1000 Subject: [PATCH 04/11] Enable reset-feature-state action on serverless (#98249) This action exists primarily for testing purposes and is useful when testing serverless projects --- .../rest/action/admin/cluster/RestResetFeatureStateAction.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestResetFeatureStateAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestResetFeatureStateAction.java index c307f56a4e8c2..a11cc65d1e92a 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestResetFeatureStateAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestResetFeatureStateAction.java @@ -15,12 +15,15 @@ import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.rest.Scope; +import org.elasticsearch.rest.ServerlessScope; import org.elasticsearch.rest.action.RestToXContentListener; import java.io.IOException; import java.util.List; /** Rest handler for feature state reset requests */ +@ServerlessScope(Scope.INTERNAL) public class RestResetFeatureStateAction extends BaseRestHandler { @Override From 52e445c7ba1ccf36da71606dec7625e65e8b24fa Mon Sep 17 00:00:00 2001 From: Tim Vernum Date: Tue, 8 Aug 2023 12:31:05 +1000 Subject: [PATCH 05/11] Mute CrossClusterAsyncSearchIT (#98273) Multiple test methods are failing Relates: #98272 --- .../elasticsearch/xpack/search/CrossClusterAsyncSearchIT.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/CrossClusterAsyncSearchIT.java b/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/CrossClusterAsyncSearchIT.java index 9d496dcaa70a8..dc0393f5f0b2b 100644 --- a/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/CrossClusterAsyncSearchIT.java +++ b/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/CrossClusterAsyncSearchIT.java @@ -7,6 +7,7 @@ package org.elasticsearch.xpack.search; +import org.apache.lucene.tests.util.LuceneTestCase; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.ActionFuture; @@ -82,6 +83,7 @@ * This IT test copies the setup and general approach that the {@code CrossClusterSearchIT} test * used for testing synchronous CCS. */ +@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/98272") public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase { private static final String REMOTE_CLUSTER = "cluster_a"; From 9c07f1df6fb91c30e658bde73cddbfc9e1460474 Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Tue, 8 Aug 2023 15:02:40 +1000 Subject: [PATCH 06/11] RCS2 - Fix remote nodes collection for scroll (#98186) For RCS 2.0, we added a RemoteClusterNodesAction (#93893) for collecting remote cluster server nodes. This action is used in two places: 1. Collecting nodes that can be directly connected in sniff mode 2. In the method RemoteClusterService#collectNodes However, in the 2nd use case, we should collect all ndoes from the remote cluster instead of just those nodes that have remote cluster server enabled. This is because the method is used in the context of search instead of building connection. A remote node can be *not* directly accessible but still hosts searchable data. Search requests should be sent to all the data nodes and *not* limited to just the server nodes. This PR fixes this issue by augmenting RemoteClusterNodesAction to support retrieving either server nodes or all nodes. The caller can decide which set of nodes are of interest depending on the context. --- .../remote/RemoteClusterNodesAction.java | 66 ++++--- .../transport/RemoteClusterConnection.java | 2 +- .../transport/SniffConnectionStrategy.java | 2 +- .../remote/RemoteClusterNodesActionTests.java | 74 +++++++- .../test/cluster/ClusterHandle.java | 8 +- .../cluster/DefaultElasticsearchCluster.java | 4 +- .../cluster/local/LocalClusterFactory.java | 12 +- .../cluster/local/LocalClusterHandle.java | 4 +- .../test/cluster/local/LocalClusterSpec.java | 4 + ...AbstractRemoteClusterSecurityTestCase.java | 29 ++- ...lusterSecurityFcActionAuthorizationIT.java | 5 +- .../RemoteClusterSecurityRestIT.java | 70 ++++--- .../RemoteClusterSecurityTopologyRestIT.java | 171 ++++++++++++++++++ 13 files changed, 377 insertions(+), 74 deletions(-) create mode 100644 x-pack/plugin/security/qa/multi-cluster/src/javaRestTest/java/org/elasticsearch/xpack/remotecluster/RemoteClusterSecurityTopologyRestIT.java diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/remote/RemoteClusterNodesAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/remote/RemoteClusterNodesAction.java index 56ca7980347f5..e324b2a8433e3 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/remote/RemoteClusterNodesAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/remote/RemoteClusterNodesAction.java @@ -19,6 +19,7 @@ import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; +import org.elasticsearch.action.support.nodes.BaseNodeResponse; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; @@ -43,13 +44,23 @@ public RemoteClusterNodesAction() { } public static class Request extends ActionRequest { + public static final Request ALL_NODES = new Request(false); + public static final Request REMOTE_CLUSTER_SERVER_NODES = new Request(true); + private final boolean remoteClusterServer; - public static final Request INSTANCE = new Request(); - - public Request() {} + private Request(boolean remoteClusterServer) { + this.remoteClusterServer = remoteClusterServer; + } public Request(StreamInput in) throws IOException { super(in); + this.remoteClusterServer = in.readBoolean(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeBoolean(remoteClusterServer); } @Override @@ -82,7 +93,6 @@ public List getNodes() { } public static class TransportAction extends HandledTransportAction { - private final TransportService transportService; @Inject @@ -93,27 +103,39 @@ public TransportAction(TransportService transportService, ActionFilters actionFi @Override protected void doExecute(Task task, Request request, ActionListener listener) { - final NodesInfoRequest nodesInfoRequest = new NodesInfoRequest(); - nodesInfoRequest.clear(); - nodesInfoRequest.addMetrics(NodesInfoRequest.Metric.REMOTE_CLUSTER_SERVER.metricName()); final ThreadContext threadContext = transportService.getThreadPool().getThreadContext(); try (var ignore = threadContext.stashContext()) { threadContext.markAsSystemContext(); - transportService.sendRequest( - transportService.getLocalNode(), - NodesInfoAction.NAME, - nodesInfoRequest, - new ActionListenerResponseHandler<>(listener.delegateFailureAndWrap((l, response) -> { - final List remoteClusterNodes = response.getNodes().stream().map(nodeInfo -> { - final RemoteClusterServerInfo remoteClusterServerInfo = nodeInfo.getInfo(RemoteClusterServerInfo.class); - if (remoteClusterServerInfo == null) { - return null; - } - return nodeInfo.getNode().withTransportAddress(remoteClusterServerInfo.getAddress().publishAddress()); - }).filter(Objects::nonNull).toList(); - l.onResponse(new Response(remoteClusterNodes)); - }), NodesInfoResponse::new, TransportResponseHandler.TRANSPORT_WORKER) - ); + if (request.remoteClusterServer) { + final NodesInfoRequest nodesInfoRequest = new NodesInfoRequest().clear() + .addMetrics(NodesInfoRequest.Metric.REMOTE_CLUSTER_SERVER.metricName()); + transportService.sendRequest( + transportService.getLocalNode(), + NodesInfoAction.NAME, + nodesInfoRequest, + new ActionListenerResponseHandler<>(listener.delegateFailureAndWrap((l, response) -> { + final List remoteClusterNodes = response.getNodes().stream().map(nodeInfo -> { + final RemoteClusterServerInfo remoteClusterServerInfo = nodeInfo.getInfo(RemoteClusterServerInfo.class); + if (remoteClusterServerInfo == null) { + return null; + } + return nodeInfo.getNode().withTransportAddress(remoteClusterServerInfo.getAddress().publishAddress()); + }).filter(Objects::nonNull).toList(); + l.onResponse(new Response(remoteClusterNodes)); + }), NodesInfoResponse::new, TransportResponseHandler.TRANSPORT_WORKER) + ); + } else { + final NodesInfoRequest nodesInfoRequest = new NodesInfoRequest().clear(); + transportService.sendRequest( + transportService.getLocalNode(), + NodesInfoAction.NAME, + nodesInfoRequest, + new ActionListenerResponseHandler<>(listener.delegateFailureAndWrap((l, response) -> { + final List nodes = response.getNodes().stream().map(BaseNodeResponse::getNode).toList(); + l.onResponse(new Response(nodes)); + }), NodesInfoResponse::new, TransportResponseHandler.TRANSPORT_WORKER) + ); + } } } } diff --git a/server/src/main/java/org/elasticsearch/transport/RemoteClusterConnection.java b/server/src/main/java/org/elasticsearch/transport/RemoteClusterConnection.java index 1e1a1525a4733..8499aa03d7808 100644 --- a/server/src/main/java/org/elasticsearch/transport/RemoteClusterConnection.java +++ b/server/src/main/java/org/elasticsearch/transport/RemoteClusterConnection.java @@ -125,7 +125,7 @@ void collectNodes(ActionListener> listener) { transportService.sendRequest( connection, RemoteClusterNodesAction.NAME, - RemoteClusterNodesAction.Request.INSTANCE, + RemoteClusterNodesAction.Request.ALL_NODES, TransportRequestOptions.EMPTY, new ActionListenerResponseHandler<>(contextPreservingActionListener.map(response -> { final Map nodeLookup = response.getNodes() diff --git a/server/src/main/java/org/elasticsearch/transport/SniffConnectionStrategy.java b/server/src/main/java/org/elasticsearch/transport/SniffConnectionStrategy.java index 0893e29e04721..b190c4e87f927 100644 --- a/server/src/main/java/org/elasticsearch/transport/SniffConnectionStrategy.java +++ b/server/src/main/java/org/elasticsearch/transport/SniffConnectionStrategy.java @@ -312,7 +312,7 @@ private void collectRemoteNodes(Iterator> seedNodesSuppl // Use different action to collect nodes information depending on the connection model if (REMOTE_CLUSTER_PROFILE.equals(connectionManager.getConnectionProfile().getTransportProfile())) { action = RemoteClusterNodesAction.NAME; - request = RemoteClusterNodesAction.Request.INSTANCE; + request = RemoteClusterNodesAction.Request.REMOTE_CLUSTER_SERVER_NODES; sniffResponseHandler = new RemoteClusterNodesSniffResponseHandler(connection, listener, seedNodesSuppliers); } else { action = ClusterStateAction.NAME; diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/remote/RemoteClusterNodesActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/remote/RemoteClusterNodesActionTests.java index 0709d9314f19c..f2c1cfbbe44e7 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/remote/RemoteClusterNodesActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/remote/RemoteClusterNodesActionTests.java @@ -39,6 +39,7 @@ import java.util.stream.Collectors; import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; @@ -53,7 +54,7 @@ public static void ensureFeatureFlag() { assumeTrue("untrusted remote cluster feature flag must be enabled", TcpTransport.isUntrustedRemoteClusterEnabled()); } - public void testDoExecute() { + public void testDoExecuteForRemoteServerNodes() { final ThreadPool threadPool = mock(ThreadPool.class); final ThreadContext threadContext = new ThreadContext(Settings.EMPTY); when(threadPool.getThreadContext()).thenReturn(threadContext); @@ -122,7 +123,7 @@ public void testDoExecute() { ); final PlainActionFuture future = new PlainActionFuture<>(); - action.doExecute(mock(Task.class), RemoteClusterNodesAction.Request.INSTANCE, future); + action.doExecute(mock(Task.class), RemoteClusterNodesAction.Request.REMOTE_CLUSTER_SERVER_NODES, future); final List actualNodes = future.actionGet().getNodes(); assertThat(Set.copyOf(actualNodes), equalTo(expectedRemoteServerNodes)); @@ -132,6 +133,75 @@ public void testDoExecute() { ); } + public void testDoExecuteForRemoteNodes() { + final ThreadPool threadPool = mock(ThreadPool.class); + final ThreadContext threadContext = new ThreadContext(Settings.EMPTY); + when(threadPool.getThreadContext()).thenReturn(threadContext); + + final TransportService transportService = mock(TransportService.class); + final DiscoveryNode localNode = mock(DiscoveryNode.class); + when(transportService.getLocalNode()).thenReturn(localNode); + when(transportService.getThreadPool()).thenReturn(threadPool); + + // Prepare nodesInfo response + final int numberOfNodes = randomIntBetween(1, 6); + final List nodeInfos = new ArrayList<>(); + final Set expectedRemoteNodes = new HashSet<>(); + for (int i = 0; i < numberOfNodes; i++) { + final DiscoveryNode node = randomNode(i); + expectedRemoteNodes.add(node); + nodeInfos.add( + new NodeInfo( + Version.CURRENT, + TransportVersion.current(), + null, + node, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ) + ); + } + + final NodesInfoResponse nodesInfoResponse = new NodesInfoResponse( + new ClusterName(randomAlphaOfLengthBetween(3, 8)), + nodeInfos, + List.of() + ); + + doAnswer(invocation -> { + final NodesInfoRequest nodesInfoRequest = invocation.getArgument(2); + assertThat(nodesInfoRequest.requestedMetrics(), empty()); + final ActionListenerResponseHandler handler = invocation.getArgument(3); + handler.handleResponse(nodesInfoResponse); + return null; + }).when(transportService).sendRequest(eq(localNode), eq(NodesInfoAction.NAME), any(NodesInfoRequest.class), any()); + + final RemoteClusterNodesAction.TransportAction action = new RemoteClusterNodesAction.TransportAction( + transportService, + mock(ActionFilters.class) + ); + + final PlainActionFuture future = new PlainActionFuture<>(); + action.doExecute(mock(Task.class), RemoteClusterNodesAction.Request.ALL_NODES, future); + + final List actualNodes = future.actionGet().getNodes(); + assertThat(Set.copyOf(actualNodes), equalTo(expectedRemoteNodes)); + assertThat( + actualNodes.stream().map(DiscoveryNode::getAddress).collect(Collectors.toUnmodifiableSet()), + equalTo(expectedRemoteNodes.stream().map(DiscoveryNode::getAddress).collect(Collectors.toUnmodifiableSet())) + ); + } + private DiscoveryNode randomNode(final int id) { return DiscoveryNodeUtils.builder(Integer.toString(id)).name("node-" + id).roles(Set.of()).build(); } diff --git a/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/ClusterHandle.java b/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/ClusterHandle.java index 9c10b90180869..676a3c6fa81ed 100644 --- a/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/ClusterHandle.java +++ b/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/ClusterHandle.java @@ -79,7 +79,7 @@ public interface ClusterHandle extends Closeable { /** * Returns a comma-separated list of TCP transport endpoints for cluster. If this method is called on an unstarted cluster, the cluster - * will be started. This method is thread-safe and subsequent calls will wait for cluster start and availability. + * will be started. This method is thread-safe and subsequent calls will wait for cluster start and availability.\ * * @return cluster node TCP transport endpoints */ @@ -96,14 +96,18 @@ public interface ClusterHandle extends Closeable { /** * Returns a comma-separated list of remote cluster server endpoints for cluster. If this method is called on an unstarted cluster, * the cluster will be started. This method is thread-safe and subsequent calls will wait for cluster start and availability. + * Note individual node can enable or disable remote cluster server independently. When a node has remote cluster server disabled, + * an empty string is returned for that node. Hence, it is possible for this method to return something like "[::1]:63300,,". * * @return cluster node remote cluster server endpoints */ - String getRemoteClusterServerEndpoint(); + String getRemoteClusterServerEndpoints(); /** * Returns the remote cluster server endpoint for the node at the given index. If this method is called on an unstarted cluster, * the cluster will be started. This method is thread-safe and subsequent calls will wait for cluster start and availability. + * Note individual node can enable or disable remote cluster server independently. When a node has remote cluster server disabled, + * an empty string is returned. * * @return cluster node remote cluster server endpoints */ diff --git a/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/DefaultElasticsearchCluster.java b/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/DefaultElasticsearchCluster.java index 0470b55b63a2b..6b6330b122776 100644 --- a/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/DefaultElasticsearchCluster.java +++ b/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/DefaultElasticsearchCluster.java @@ -115,9 +115,9 @@ public String getTransportEndpoint(int index) { } @Override - public String getRemoteClusterServerEndpoint() { + public String getRemoteClusterServerEndpoints() { checkHandle(); - return handle.getRemoteClusterServerEndpoint(); + return handle.getRemoteClusterServerEndpoints(); } @Override diff --git a/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/local/LocalClusterFactory.java b/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/local/LocalClusterFactory.java index afb0230c9e34d..2e4f16399c1af 100644 --- a/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/local/LocalClusterFactory.java +++ b/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/local/LocalClusterFactory.java @@ -196,11 +196,15 @@ public String getTransportEndpoint() { } public String getRemoteClusterServerEndpoint() { - Path portsFile = workingDir.resolve("logs").resolve("remote_cluster.ports"); - if (Files.notExists(portsFile)) { - waitUntilReady(); + if (spec.isRemoteClusterServerEnabled()) { + Path portsFile = workingDir.resolve("logs").resolve("remote_cluster.ports"); + if (Files.notExists(portsFile)) { + waitUntilReady(); + } + return readPortsFile(portsFile).get(0); + } else { + return ""; } - return readPortsFile(portsFile).get(0); } public void deletePortsFiles() { diff --git a/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/local/LocalClusterHandle.java b/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/local/LocalClusterHandle.java index 1e79f9d499f5b..4d735c777eba5 100644 --- a/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/local/LocalClusterHandle.java +++ b/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/local/LocalClusterHandle.java @@ -129,14 +129,14 @@ public String getTransportEndpoint(int index) { } @Override - public String getRemoteClusterServerEndpoint() { + public String getRemoteClusterServerEndpoints() { start(); return execute(() -> nodes.parallelStream().map(Node::getRemoteClusterServerEndpoint).collect(Collectors.joining(","))); } @Override public String getRemoteClusterServerEndpoint(int index) { - return getRemoteClusterServerEndpoint().split(",")[index]; + return getRemoteClusterServerEndpoints().split(",")[index]; } @Override diff --git a/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/local/LocalClusterSpec.java b/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/local/LocalClusterSpec.java index 6271a05f54e63..2cca08c86b570 100644 --- a/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/local/LocalClusterSpec.java +++ b/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/local/LocalClusterSpec.java @@ -197,6 +197,10 @@ public boolean isSecurityEnabled() { return Boolean.parseBoolean(getSetting("xpack.security.enabled", getVersion().onOrAfter("8.0.0") ? "true" : "false")); } + public boolean isRemoteClusterServerEnabled() { + return Boolean.parseBoolean(getSetting("remote_cluster_server.enabled", "false")); + } + public boolean isMasterEligible() { return getSetting("node.roles", "master").contains("master"); } diff --git a/x-pack/plugin/security/qa/multi-cluster/src/javaRestTest/java/org/elasticsearch/xpack/remotecluster/AbstractRemoteClusterSecurityTestCase.java b/x-pack/plugin/security/qa/multi-cluster/src/javaRestTest/java/org/elasticsearch/xpack/remotecluster/AbstractRemoteClusterSecurityTestCase.java index 85251c22644b7..d2f0d18ee4477 100644 --- a/x-pack/plugin/security/qa/multi-cluster/src/javaRestTest/java/org/elasticsearch/xpack/remotecluster/AbstractRemoteClusterSecurityTestCase.java +++ b/x-pack/plugin/security/qa/multi-cluster/src/javaRestTest/java/org/elasticsearch/xpack/remotecluster/AbstractRemoteClusterSecurityTestCase.java @@ -8,6 +8,7 @@ package org.elasticsearch.xpack.remotecluster; import org.apache.http.HttpHost; +import org.apache.http.client.methods.HttpPost; import org.elasticsearch.client.Request; import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.Response; @@ -31,7 +32,9 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.nio.charset.StandardCharsets; +import java.util.Arrays; import java.util.Base64; +import java.util.Locale; import java.util.Map; import static org.hamcrest.Matchers.equalTo; @@ -111,6 +114,24 @@ public static void closeFulfillingClusterClient() throws IOException { } } + protected static String headerFromRandomAuthMethod(final String username, final SecureString password) throws IOException { + final boolean useBearerTokenAuth = randomBoolean(); + if (useBearerTokenAuth) { + final Request request = new Request(HttpPost.METHOD_NAME, "/_security/oauth2/token"); + request.setJsonEntity(String.format(Locale.ROOT, """ + { + "grant_type":"password", + "username":"%s", + "password":"%s" + } + """, username, password)); + final Map responseBody = entityAsMap(adminClient().performRequest(request)); + return "Bearer " + responseBody.get("access_token"); + } else { + return basicAuthHeaderValue(username, password); + } + } + @Override protected String getTestRestCluster() { return queryCluster.getHttpAddress(0); @@ -187,7 +208,6 @@ protected void configureRemoteCluster( updateClusterSettings(builder.build()); // Ensure remote cluster is connected - final int numberOfFcNodes = targetFulfillingCluster.getHttpAddresses().split(",").length; final Request remoteInfoRequest = new Request("GET", "/_remote/info"); assertBusy(() -> { final Response remoteInfoResponse = adminClient().performRequest(remoteInfoRequest); @@ -195,6 +215,13 @@ protected void configureRemoteCluster( final ObjectPath remoteInfoObjectPath = assertOKAndCreateObjectPath(remoteInfoResponse); assertThat(remoteInfoObjectPath.evaluate(clusterAlias + ".connected"), is(true)); if (false == isProxyMode) { + int numberOfFcNodes = (int) Arrays.stream(targetFulfillingCluster.getRemoteClusterServerEndpoints().split(",")) + .filter(endpoint -> endpoint.length() > 0) + .count(); + if (numberOfFcNodes == 0) { + // The cluster is an RCS 1.0 remote cluster + numberOfFcNodes = targetFulfillingCluster.getTransportEndpoints().split(",").length; + } assertThat(remoteInfoObjectPath.evaluate(clusterAlias + ".num_nodes_connected"), equalTo(numberOfFcNodes)); } final String credentialsValue = remoteInfoObjectPath.evaluate(clusterAlias + ".cluster_credentials"); diff --git a/x-pack/plugin/security/qa/multi-cluster/src/javaRestTest/java/org/elasticsearch/xpack/remotecluster/RemoteClusterSecurityFcActionAuthorizationIT.java b/x-pack/plugin/security/qa/multi-cluster/src/javaRestTest/java/org/elasticsearch/xpack/remotecluster/RemoteClusterSecurityFcActionAuthorizationIT.java index cb37dd9d638d8..614e36c36637b 100644 --- a/x-pack/plugin/security/qa/multi-cluster/src/javaRestTest/java/org/elasticsearch/xpack/remotecluster/RemoteClusterSecurityFcActionAuthorizationIT.java +++ b/x-pack/plugin/security/qa/multi-cluster/src/javaRestTest/java/org/elasticsearch/xpack/remotecluster/RemoteClusterSecurityFcActionAuthorizationIT.java @@ -288,7 +288,10 @@ public void testRestApiKeyIsNotAllowedOnRemoteClusterPort() throws IOException { final ElasticsearchSecurityException e = expectThrows( ElasticsearchSecurityException.class, - () -> remoteClusterClient.execute(RemoteClusterNodesAction.INSTANCE, RemoteClusterNodesAction.Request.INSTANCE).actionGet() + () -> remoteClusterClient.execute( + RemoteClusterNodesAction.INSTANCE, + RemoteClusterNodesAction.Request.REMOTE_CLUSTER_SERVER_NODES + ).actionGet() ); assertThat( e.getMessage(), diff --git a/x-pack/plugin/security/qa/multi-cluster/src/javaRestTest/java/org/elasticsearch/xpack/remotecluster/RemoteClusterSecurityRestIT.java b/x-pack/plugin/security/qa/multi-cluster/src/javaRestTest/java/org/elasticsearch/xpack/remotecluster/RemoteClusterSecurityRestIT.java index b1dc87ac5a57e..1ae30e019e390 100644 --- a/x-pack/plugin/security/qa/multi-cluster/src/javaRestTest/java/org/elasticsearch/xpack/remotecluster/RemoteClusterSecurityRestIT.java +++ b/x-pack/plugin/security/qa/multi-cluster/src/javaRestTest/java/org/elasticsearch/xpack/remotecluster/RemoteClusterSecurityRestIT.java @@ -7,13 +7,11 @@ package org.elasticsearch.xpack.remotecluster; -import org.apache.http.client.methods.HttpPost; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.client.Request; import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.Response; import org.elasticsearch.client.ResponseException; -import org.elasticsearch.common.settings.SecureString; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.Strings; import org.elasticsearch.search.SearchHit; @@ -43,31 +41,36 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; public class RemoteClusterSecurityRestIT extends AbstractRemoteClusterSecurityTestCase { private static final AtomicReference> API_KEY_MAP_REF = new AtomicReference<>(); private static final AtomicReference> REST_API_KEY_MAP_REF = new AtomicReference<>(); private static final AtomicBoolean SSL_ENABLED_REF = new AtomicBoolean(); + private static final AtomicBoolean NODE1_RCS_SERVER_ENABLED = new AtomicBoolean(); + private static final AtomicBoolean NODE2_RCS_SERVER_ENABLED = new AtomicBoolean(); static { fulfillingCluster = ElasticsearchCluster.local() .name("fulfilling-cluster") .nodes(3) .apply(commonClusterConfig) - .setting("remote_cluster_server.enabled", "true") .setting("remote_cluster.port", "0") - .setting("xpack.security.remote_cluster_server.ssl.enabled", String.valueOf(SSL_ENABLED_REF.get())) + .setting("xpack.security.remote_cluster_server.ssl.enabled", () -> String.valueOf(SSL_ENABLED_REF.get())) .setting("xpack.security.remote_cluster_server.ssl.key", "remote-cluster.key") .setting("xpack.security.remote_cluster_server.ssl.certificate", "remote-cluster.crt") .setting("xpack.security.authc.token.enabled", "true") .keystore("xpack.security.remote_cluster_server.ssl.secure_key_passphrase", "remote-cluster-password") + .node(0, spec -> spec.setting("remote_cluster_server.enabled", "true")) + .node(1, spec -> spec.setting("remote_cluster_server.enabled", () -> String.valueOf(NODE1_RCS_SERVER_ENABLED.get()))) + .node(2, spec -> spec.setting("remote_cluster_server.enabled", () -> String.valueOf(NODE2_RCS_SERVER_ENABLED.get()))) .build(); queryCluster = ElasticsearchCluster.local() .name("query-cluster") .apply(commonClusterConfig) - .setting("xpack.security.remote_cluster_client.ssl.enabled", String.valueOf(SSL_ENABLED_REF.get())) + .setting("xpack.security.remote_cluster_client.ssl.enabled", () -> String.valueOf(SSL_ENABLED_REF.get())) .setting("xpack.security.remote_cluster_client.ssl.certificate_authorities", "remote-cluster-ca.crt") .setting("xpack.security.authc.token.enabled", "true") .keystore("cluster.remote.my_remote_cluster.credentials", () -> { @@ -114,9 +117,11 @@ public class RemoteClusterSecurityRestIT extends AbstractRemoteClusterSecurityTe // Use a RuleChain to ensure that fulfilling cluster is started before query cluster // `SSL_ENABLED_REF` is used to control the SSL-enabled setting on the test clusters // We set it here, since randomization methods are not available in the static initialize context above - public static TestRule clusterRule = RuleChain.outerRule(new RunnableTestRuleAdapter(() -> SSL_ENABLED_REF.set(usually()))) - .around(fulfillingCluster) - .around(queryCluster); + public static TestRule clusterRule = RuleChain.outerRule(new RunnableTestRuleAdapter(() -> { + SSL_ENABLED_REF.set(usually()); + NODE1_RCS_SERVER_ENABLED.set(randomBoolean()); + NODE2_RCS_SERVER_ENABLED.set(randomBoolean()); + })).around(fulfillingCluster).around(queryCluster); public void testCrossClusterSearch() throws Exception { configureRemoteCluster(); @@ -151,7 +156,8 @@ public void testCrossClusterSearch() throws Exception { { "index": { "_index": "shared-metrics" } } { "name": "metric3" } { "index": { "_index": "shared-metrics" } } - { "name": "metric4" }\n""")); + { "name": "metric4" } + """)); assertOK(performRequestAgainstFulfillingCluster(bulkRequest)); } @@ -353,24 +359,34 @@ public void testCrossClusterSearch() throws Exception { @SuppressWarnings("unchecked") public void testNodesInfo() throws IOException { - final Request request = new Request("GET", "/_nodes/transport,remote_cluster_server"); + final Request request = new Request("GET", "/_nodes/settings,transport,remote_cluster_server"); final Response response = performRequestAgainstFulfillingCluster(request); assertOK(response); final Map responseMap = responseAsMap(response); assertThat(ObjectPath.eval("_nodes.total", responseMap), equalTo(3)); final Map nodes = ObjectPath.eval("nodes", responseMap); - nodes.forEach((k, v) -> { - final Map node = (Map) v; + int numberOfRemoteClusterServerNodes = 0; + for (Map.Entry entry : nodes.entrySet()) { + final Map node = (Map) entry.getValue(); // remote cluster is not reported in transport profiles assertThat(ObjectPath.eval("transport.profiles", node), anEmptyMap()); - final List boundAddresses = ObjectPath.eval("remote_cluster_server.bound_address", node); - assertThat(boundAddresses, notNullValue()); - assertThat(boundAddresses, not(empty())); - final String publishAddress = ObjectPath.eval("remote_cluster_server.publish_address", node); - assertThat(publishAddress, notNullValue()); - }); + if (Boolean.parseBoolean(ObjectPath.eval("settings.remote_cluster_server.enabled", node))) { + numberOfRemoteClusterServerNodes += 1; + final List boundAddresses = ObjectPath.eval("remote_cluster_server.bound_address", node); + assertThat(boundAddresses, notNullValue()); + assertThat(boundAddresses, not(empty())); + final String publishAddress = ObjectPath.eval("remote_cluster_server.publish_address", node); + assertThat(publishAddress, notNullValue()); + } else { + assertThat(ObjectPath.eval("remote_cluster_server", node), nullValue()); + } + } + assertThat( + numberOfRemoteClusterServerNodes, + equalTo(1 + (NODE1_RCS_SERVER_ENABLED.get() ? 1 : 0) + (NODE2_RCS_SERVER_ENABLED.get() ? 1 : 0)) + ); } private Response performRequestWithRemoteSearchUser(final Request request) throws IOException { @@ -393,22 +409,4 @@ private Response performRequestWithLocalSearchUser(final Request request) throws ); return client().performRequest(request); } - - private String headerFromRandomAuthMethod(final String username, final SecureString password) throws IOException { - final boolean useBearerTokenAuth = randomBoolean(); - if (useBearerTokenAuth) { - final Request request = new Request(HttpPost.METHOD_NAME, "/_security/oauth2/token"); - request.setJsonEntity(String.format(Locale.ROOT, """ - { - "grant_type":"password", - "username":"%s", - "password":"%s" - } - """, username, password)); - final Map responseBody = entityAsMap(adminClient().performRequest(request)); - return "Bearer " + responseBody.get("access_token"); - } else { - return basicAuthHeaderValue(username, password); - } - } } diff --git a/x-pack/plugin/security/qa/multi-cluster/src/javaRestTest/java/org/elasticsearch/xpack/remotecluster/RemoteClusterSecurityTopologyRestIT.java b/x-pack/plugin/security/qa/multi-cluster/src/javaRestTest/java/org/elasticsearch/xpack/remotecluster/RemoteClusterSecurityTopologyRestIT.java new file mode 100644 index 0000000000000..63b43925a9931 --- /dev/null +++ b/x-pack/plugin/security/qa/multi-cluster/src/javaRestTest/java/org/elasticsearch/xpack/remotecluster/RemoteClusterSecurityTopologyRestIT.java @@ -0,0 +1,171 @@ +/* + * 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; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.remotecluster; + +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.Response; +import org.elasticsearch.core.Strings; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.test.cluster.ElasticsearchCluster; +import org.elasticsearch.test.cluster.util.resource.Resource; +import org.elasticsearch.test.junit.RunnableTestRuleAdapter; +import org.elasticsearch.test.rest.ObjectPath; +import org.junit.ClassRule; +import org.junit.rules.RuleChain; +import org.junit.rules.TestRule; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; + +public class RemoteClusterSecurityTopologyRestIT extends AbstractRemoteClusterSecurityTestCase { + + private static final AtomicReference> API_KEY_MAP_REF = new AtomicReference<>(); + private static final AtomicBoolean NODE1_RCS_SERVER_ENABLED = new AtomicBoolean(); + + static { + fulfillingCluster = ElasticsearchCluster.local() + .name("fulfilling-cluster") + .nodes(3) + .apply(commonClusterConfig) + .setting("remote_cluster.port", "0") + .setting("xpack.security.remote_cluster_server.ssl.enabled", "true") + .setting("xpack.security.remote_cluster_server.ssl.key", "remote-cluster.key") + .setting("xpack.security.remote_cluster_server.ssl.certificate", "remote-cluster.crt") + .setting("xpack.security.authc.token.enabled", "true") + .keystore("xpack.security.remote_cluster_server.ssl.secure_key_passphrase", "remote-cluster-password") + .node(0, spec -> spec.setting("remote_cluster_server.enabled", "true")) + .node(1, spec -> spec.setting("remote_cluster_server.enabled", () -> String.valueOf(NODE1_RCS_SERVER_ENABLED.get()))) + // at least one remote node has server disabled + .node(2, spec -> spec.setting("remote_cluster_server.enabled", "false")) + .build(); + + queryCluster = ElasticsearchCluster.local() + .name("query-cluster") + .apply(commonClusterConfig) + .setting("xpack.security.remote_cluster_client.ssl.enabled", "true") + .setting("xpack.security.remote_cluster_client.ssl.certificate_authorities", "remote-cluster-ca.crt") + .setting("xpack.security.authc.token.enabled", "true") + .keystore("cluster.remote.my_remote_cluster.credentials", () -> { + if (API_KEY_MAP_REF.get() == null) { + final Map apiKeyMap = createCrossClusterAccessApiKey(""" + { + "search": [ + { + "names": ["index*", "not_found_index", "shared-metrics"] + } + ] + }"""); + API_KEY_MAP_REF.set(apiKeyMap); + } + return (String) API_KEY_MAP_REF.get().get("encoded"); + }) + // Define a bogus API key for another remote cluster + .keystore("cluster.remote.invalid_remote.credentials", randomEncodedApiKey()) + // Define remote with a REST API key to observe expected failure + .rolesFile(Resource.fromClasspath("roles.yml")) + .user(REMOTE_METRIC_USER, PASS.toString(), "read_remote_shared_metrics") + .build(); + } + + @ClassRule + public static TestRule clusterRule = RuleChain.outerRule(new RunnableTestRuleAdapter(() -> { + NODE1_RCS_SERVER_ENABLED.set(randomBoolean()); + })).around(fulfillingCluster).around(queryCluster); + + public void testCrossClusterScrollWithSniffModeWhenSomeRemoteNodesAreNotDirectlyAccessible() throws Exception { + configureRemoteCluster(false); + + // Fulfilling cluster + { + // Spread the shards to all nodes + final Request createIndexRequest = new Request("PUT", "shared-metrics"); + createIndexRequest.setJsonEntity(""" + { + "settings": { + "number_of_shards": 3, + "number_of_replicas": 0 + } + }"""); + assertOK(performRequestAgainstFulfillingCluster(createIndexRequest)); + + // Index some documents, so we can attempt to search them from the querying cluster + final Request bulkRequest = new Request("POST", "/_bulk?refresh=true"); + bulkRequest.setJsonEntity(Strings.format(""" + { "index": { "_index": "shared-metrics" } } + { "name": "metric1" } + { "index": { "_index": "shared-metrics" } } + { "name": "metric2" } + { "index": { "_index": "shared-metrics" } } + { "name": "metric3" } + { "index": { "_index": "shared-metrics" } } + { "name": "metric4" } + { "index": { "_index": "shared-metrics" } } + { "name": "metric5" } + { "index": { "_index": "shared-metrics" } } + { "name": "metric6" } + """)); + assertOK(performRequestAgainstFulfillingCluster(bulkRequest)); + } + + // Query cluster + { + final var documentFieldValues = new HashSet<>(); + final var searchRequest = new Request("GET", "/my_remote_cluster:*/_search?scroll=1h&size=1"); + final SearchResponse searchResponse = SearchResponse.fromXContent( + responseAsParser(performRequestWithRemoteMetricUser(searchRequest)) + ); + assertThat(searchResponse.getHits().getTotalHits().value, equalTo(6L)); + assertThat(Arrays.stream(searchResponse.getHits().getHits()).map(SearchHit::getIndex).toList(), contains("shared-metrics")); + documentFieldValues.add(searchResponse.getHits().getHits()[0].getSourceAsMap().get("name")); + + // Scroll should be able to fetch all documents from all nodes even when some nodes are not directly accessible in sniff mode + final String scrollId = searchResponse.getScrollId(); + final Request scrollRequest = new Request("GET", "/_search/scroll"); + scrollRequest.setJsonEntity(Strings.format(""" + { "scroll_id": "%s" } + """, scrollId)); + // Fetch all documents + for (int i = 0; i < 5; i++) { + final SearchResponse scrollResponse = SearchResponse.fromXContent( + responseAsParser(performRequestWithRemoteMetricUser(scrollRequest)) + ); + assertThat(scrollResponse.getHits().getTotalHits().value, equalTo(6L)); + assertThat(Arrays.stream(scrollResponse.getHits().getHits()).map(SearchHit::getIndex).toList(), contains("shared-metrics")); + documentFieldValues.add(scrollResponse.getHits().getHits()[0].getSourceAsMap().get("name")); + } + assertThat(documentFieldValues, containsInAnyOrder("metric1", "metric2", "metric3", "metric4", "metric5", "metric6")); + + // Scroll from all nodes should be freed + final Request deleteScrollRequest = new Request("DELETE", "/_search/scroll"); + deleteScrollRequest.setJsonEntity(Strings.format(""" + { "scroll_id": "%s" } + """, scrollId)); + final ObjectPath deleteScrollObjectPath = assertOKAndCreateObjectPath(performRequestWithRemoteMetricUser(deleteScrollRequest)); + assertThat(deleteScrollObjectPath.evaluate("succeeded"), is(true)); + assertThat(deleteScrollObjectPath.evaluate("num_freed"), equalTo(3)); + } + } + + private Response performRequestWithRemoteMetricUser(final Request request) throws IOException { + request.setOptions( + RequestOptions.DEFAULT.toBuilder().addHeader("Authorization", headerFromRandomAuthMethod(REMOTE_METRIC_USER, PASS)) + ); + return client().performRequest(request); + } +} From 2aca985e2916e5f97d14fcae63fa3a446f1520a0 Mon Sep 17 00:00:00 2001 From: Tim Vernum Date: Tue, 8 Aug 2023 15:10:58 +1000 Subject: [PATCH 07/11] Read operator privs enabled from Env settings (#98246) Security.settings incorrectly caches the node startup settings which does not take into account setting changes that may take place after the node has started (e.g. "addtionalSettings()" from plugins). This commit fixes the behaviour for the OperatorPrivileges enabled setting, until the bug can be fixed more generally --- docs/changelog/98246.yaml | 5 +++++ .../main/java/org/elasticsearch/xpack/security/Security.java | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) create mode 100644 docs/changelog/98246.yaml diff --git a/docs/changelog/98246.yaml b/docs/changelog/98246.yaml new file mode 100644 index 0000000000000..595e97838cbc7 --- /dev/null +++ b/docs/changelog/98246.yaml @@ -0,0 +1,5 @@ +pr: 98246 +summary: Read operator privs enabled from Env settings +area: Authorization +type: enhancement +issues: [] diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java index 3572e802c9fd8..257e1a0181743 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java @@ -905,7 +905,7 @@ Collection createComponents( final AuthenticationFailureHandler failureHandler = createAuthenticationFailureHandler(realms, extensionComponents); // operator privileges are enabled either explicitly via the setting or if running serverless - final boolean operatorPrivilegesEnabled = OPERATOR_PRIVILEGES_ENABLED.get(settings) || DiscoveryNode.isServerless(); + final boolean operatorPrivilegesEnabled = OPERATOR_PRIVILEGES_ENABLED.get(environment.settings()) || DiscoveryNode.isServerless(); if (operatorPrivilegesEnabled) { logger.info("operator privileges are enabled"); From 3093c40b8b33d7d95a2728335d9ae960a1cf2f41 Mon Sep 17 00:00:00 2001 From: Tim Vernum Date: Tue, 8 Aug 2023 15:29:24 +1000 Subject: [PATCH 08/11] Make RestController pluggable (#98187) This commit changes the ActionModules to allow the RestController to be provided by an internal plugin. It renames `RestInterceptorActionPlugin` to `RestServerActionPlugin` and adds a new `getRestController` method to it. There may be multiple RestServerActionPlugins installed on a node, but only 1 may provide a Rest Wrapper (getRestHandlerInterceptor) and only 1 may provide a RestController (getRestController). --- docs/changelog/98187.yaml | 5 + qa/custom-rest-controller/build.gradle | 10 ++ .../elasticsearch/test/CustomRestPlugin.java | 92 +++++++++++++++++++ .../interceptor/CustomRestPluginIT.java | 64 +++++++++++++ server/src/main/java/module-info.java | 2 +- .../elasticsearch/action/ActionModule.java | 76 ++++++++++----- ...lugin.java => RestServerActionPlugin.java} | 23 ++++- .../action/ActionModuleTests.java | 79 ++++++++++++++-- .../module/BasicServerModuleTests.java | 14 ++- .../core/LocalStateCompositeXPackPlugin.java | 8 +- .../xpack/security/Security.java | 4 +- .../xpack/security/SecurityTests.java | 2 +- 12 files changed, 339 insertions(+), 40 deletions(-) create mode 100644 docs/changelog/98187.yaml create mode 100644 qa/custom-rest-controller/build.gradle create mode 100644 qa/custom-rest-controller/src/javaRestTest/java/co/elastic/elasticsearch/test/CustomRestPlugin.java create mode 100644 qa/custom-rest-controller/src/javaRestTest/java/org/elasticsearch/plugins/interceptor/CustomRestPluginIT.java rename server/src/main/java/org/elasticsearch/plugins/interceptor/{RestInterceptorActionPlugin.java => RestServerActionPlugin.java} (68%) diff --git a/docs/changelog/98187.yaml b/docs/changelog/98187.yaml new file mode 100644 index 0000000000000..8163d3a215ad4 --- /dev/null +++ b/docs/changelog/98187.yaml @@ -0,0 +1,5 @@ +pr: 98187 +summary: Make `RestController` pluggable +area: Infra/REST API +type: enhancement +issues: [] diff --git a/qa/custom-rest-controller/build.gradle b/qa/custom-rest-controller/build.gradle new file mode 100644 index 0000000000000..5fd5bc9baa335 --- /dev/null +++ b/qa/custom-rest-controller/build.gradle @@ -0,0 +1,10 @@ +/* + * 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. + */ + +apply plugin: 'elasticsearch.internal-java-rest-test' + diff --git a/qa/custom-rest-controller/src/javaRestTest/java/co/elastic/elasticsearch/test/CustomRestPlugin.java b/qa/custom-rest-controller/src/javaRestTest/java/co/elastic/elasticsearch/test/CustomRestPlugin.java new file mode 100644 index 0000000000000..504802d5cfd7c --- /dev/null +++ b/qa/custom-rest-controller/src/javaRestTest/java/co/elastic/elasticsearch/test/CustomRestPlugin.java @@ -0,0 +1,92 @@ +/* + * 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 co.elastic.elasticsearch.test; + +import org.elasticsearch.client.internal.node.NodeClient; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.indices.breaker.CircuitBreakerService; +import org.elasticsearch.logging.LogManager; +import org.elasticsearch.logging.Logger; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.plugins.interceptor.RestServerActionPlugin; +import org.elasticsearch.rest.RestChannel; +import org.elasticsearch.rest.RestController; +import org.elasticsearch.rest.RestHandler; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.tracing.Tracer; +import org.elasticsearch.usage.UsageService; + +import java.util.function.UnaryOperator; + +public class CustomRestPlugin extends Plugin implements RestServerActionPlugin { + + private static final Logger logger = LogManager.getLogger(CustomRestPlugin.class); + + private static void echoHeader(String name, RestRequest request, ThreadContext threadContext) { + var value = request.header(name); + if (value != null) { + threadContext.addResponseHeader(name, value); + } + } + + public static class CustomInterceptor implements RestHandler { + + private final ThreadContext threadContext; + private final RestHandler delegate; + + public CustomInterceptor(ThreadContext threadContext, RestHandler delegate) { + this.threadContext = threadContext; + this.delegate = delegate; + } + + @Override + public void handleRequest(RestRequest request, RestChannel channel, NodeClient client) throws Exception { + logger.info("intercept request {} {}", request.method(), request.uri()); + echoHeader("x-test-interceptor", request, threadContext); + delegate.handleRequest(request, channel, client); + } + + } + + public static class CustomController extends RestController { + public CustomController( + UnaryOperator handlerWrapper, + NodeClient client, + CircuitBreakerService circuitBreakerService, + UsageService usageService, + Tracer tracer + ) { + super(handlerWrapper, client, circuitBreakerService, usageService, tracer); + } + + @Override + public void dispatchRequest(RestRequest request, RestChannel channel, ThreadContext threadContext) { + logger.info("dispatch request {} {}", request.method(), request.uri()); + echoHeader("x-test-controller", request, threadContext); + super.dispatchRequest(request, channel, threadContext); + } + } + + @Override + public UnaryOperator getRestHandlerInterceptor(ThreadContext threadContext) { + return handler -> new CustomInterceptor(threadContext, handler); + } + + @Override + public RestController getRestController( + UnaryOperator handlerWrapper, + NodeClient client, + CircuitBreakerService circuitBreakerService, + UsageService usageService, + Tracer tracer + ) { + return new CustomController(handlerWrapper, client, circuitBreakerService, usageService, tracer); + } + +} diff --git a/qa/custom-rest-controller/src/javaRestTest/java/org/elasticsearch/plugins/interceptor/CustomRestPluginIT.java b/qa/custom-rest-controller/src/javaRestTest/java/org/elasticsearch/plugins/interceptor/CustomRestPluginIT.java new file mode 100644 index 0000000000000..45a88fcfa4d5d --- /dev/null +++ b/qa/custom-rest-controller/src/javaRestTest/java/org/elasticsearch/plugins/interceptor/CustomRestPluginIT.java @@ -0,0 +1,64 @@ +/* + * 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.plugins.interceptor; + +import co.elastic.elasticsearch.test.CustomRestPlugin; + +import org.elasticsearch.client.Request; +import org.elasticsearch.client.RequestOptions; +import org.elasticsearch.client.Response; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.test.ESIntegTestCase; + +import java.io.IOException; +import java.util.Collection; + +import static org.elasticsearch.common.util.CollectionUtils.appendToCopyNoNullElements; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.notNullValue; + +public class CustomRestPluginIT extends ESIntegTestCase { + + @Override + protected Collection> nodePlugins() { + return appendToCopyNoNullElements(super.nodePlugins(), CustomRestPlugin.class); + } + + @Override + protected boolean addMockHttpTransport() { + return false; // enable http + } + + public void testInterceptor() throws Exception { + var headerValue = randomAlphaOfLengthBetween(4, 12); + assertThat(doRequest("x-test-interceptor", headerValue), equalTo(headerValue)); + assertThat(doRequest("x-test-interceptor", null), equalTo(null)); + } + + public void testController() throws Exception { + var headerValue = randomAlphaOfLengthBetween(4, 12); + assertThat(doRequest("x-test-controller", headerValue), equalTo(headerValue)); + assertThat(doRequest("x-test-controller", null), equalTo(null)); + } + + private String doRequest(String headerName, String headerValue) throws IOException { + assertThat(headerName, notNullValue()); + + var client = getRestClient(); + RequestOptions.Builder options = RequestOptions.DEFAULT.toBuilder(); + if (headerValue != null) { + options.addHeader(headerName, headerValue); + } + var request = new Request("GET", "/_nodes/_local/plugins"); + request.setOptions(options); + + final Response response = client.performRequest(request); + return response.getHeader(headerName); + } +} diff --git a/server/src/main/java/module-info.java b/server/src/main/java/module-info.java index 2b7fb56b890a2..a6c92378b4bda 100644 --- a/server/src/main/java/module-info.java +++ b/server/src/main/java/module-info.java @@ -295,7 +295,7 @@ exports org.elasticsearch.persistent; exports org.elasticsearch.persistent.decider; exports org.elasticsearch.plugins; - exports org.elasticsearch.plugins.interceptor to org.elasticsearch.security; + exports org.elasticsearch.plugins.interceptor to org.elasticsearch.security, org.elasticsearch.serverless.rest; exports org.elasticsearch.plugins.spi; exports org.elasticsearch.repositories; exports org.elasticsearch.repositories.blobstore; diff --git a/server/src/main/java/org/elasticsearch/action/ActionModule.java b/server/src/main/java/org/elasticsearch/action/ActionModule.java index 3e127325191c9..2304115614af8 100644 --- a/server/src/main/java/org/elasticsearch/action/ActionModule.java +++ b/server/src/main/java/org/elasticsearch/action/ActionModule.java @@ -305,7 +305,7 @@ import org.elasticsearch.persistent.UpdatePersistentTaskStatusAction; import org.elasticsearch.plugins.ActionPlugin; import org.elasticsearch.plugins.ActionPlugin.ActionHandler; -import org.elasticsearch.plugins.interceptor.RestInterceptorActionPlugin; +import org.elasticsearch.plugins.interceptor.RestServerActionPlugin; import org.elasticsearch.reservedstate.ReservedClusterStateHandler; import org.elasticsearch.reservedstate.service.ReservedClusterStateService; import org.elasticsearch.rest.RestController; @@ -469,6 +469,7 @@ import java.util.Optional; import java.util.Set; import java.util.function.Consumer; +import java.util.function.Function; import java.util.function.Supplier; import java.util.function.UnaryOperator; import java.util.stream.Collectors; @@ -545,37 +546,68 @@ public ActionModule( new RestHeaderDefinition(Task.X_ELASTIC_PRODUCT_ORIGIN_HTTP_HEADER, false) ) ).collect(Collectors.toSet()); - UnaryOperator restInterceptor = null; + UnaryOperator restInterceptor = getRestServerComponent( + "REST interceptor", + actionPlugins, + restPlugin -> restPlugin.getRestHandlerInterceptor(threadPool.getThreadContext()) + ); + mappingRequestValidators = new RequestValidators<>( + actionPlugins.stream().flatMap(p -> p.mappingRequestValidators().stream()).toList() + ); + indicesAliasesRequestRequestValidators = new RequestValidators<>( + actionPlugins.stream().flatMap(p -> p.indicesAliasesRequestValidators().stream()).toList() + ); + headersToCopy = headers; + + var customController = getRestServerComponent( + "REST controller", + actionPlugins, + restPlugin -> restPlugin.getRestController(restInterceptor, nodeClient, circuitBreakerService, usageService, tracer) + ); + if (customController != null) { + restController = customController; + } else { + restController = new RestController(restInterceptor, nodeClient, circuitBreakerService, usageService, tracer); + } + reservedClusterStateService = new ReservedClusterStateService(clusterService, reservedStateHandlers); + } + + private static T getRestServerComponent( + String type, + List actionPlugins, + Function function + ) { + T result = null; for (ActionPlugin plugin : actionPlugins) { - if (plugin instanceof RestInterceptorActionPlugin riplugin) { - UnaryOperator newRestInterceptor = riplugin.getRestHandlerInterceptor(threadPool.getThreadContext()); - if (newRestInterceptor != null) { - logger.debug("Using REST interceptor from plugin " + plugin.getClass().getName()); - if (plugin.getClass().getCanonicalName() == null - || plugin.getClass().getCanonicalName().startsWith("org.elasticsearch.xpack") == false) { + if (plugin instanceof RestServerActionPlugin restPlugin) { + var newInstance = function.apply(restPlugin); + if (newInstance != null) { + logger.debug("Using custom {} from plugin {}", type, plugin.getClass().getName()); + if (isInternalPlugin(plugin) == false) { throw new IllegalArgumentException( "The " + plugin.getClass().getName() - + " plugin tried to install a custom REST " - + "interceptor. This functionality is not available anymore." + + " plugin tried to install a custom " + + type + + ". This functionality is not available to external plugins." ); } - if (restInterceptor != null) { - throw new IllegalArgumentException("Cannot have more than one plugin implementing a REST interceptor"); + if (result != null) { + throw new IllegalArgumentException("Cannot have more than one plugin implementing a " + type); } - restInterceptor = newRestInterceptor; + result = newInstance; } } } - mappingRequestValidators = new RequestValidators<>( - actionPlugins.stream().flatMap(p -> p.mappingRequestValidators().stream()).toList() - ); - indicesAliasesRequestRequestValidators = new RequestValidators<>( - actionPlugins.stream().flatMap(p -> p.indicesAliasesRequestValidators().stream()).toList() - ); - headersToCopy = headers; - restController = new RestController(restInterceptor, nodeClient, circuitBreakerService, usageService, tracer); - reservedClusterStateService = new ReservedClusterStateService(clusterService, reservedStateHandlers); + return result; + } + + private static boolean isInternalPlugin(ActionPlugin plugin) { + final String canonicalName = plugin.getClass().getCanonicalName(); + if (canonicalName == null) { + return false; + } + return canonicalName.startsWith("org.elasticsearch.xpack.") || canonicalName.startsWith("co.elastic.elasticsearch."); } /** diff --git a/server/src/main/java/org/elasticsearch/plugins/interceptor/RestInterceptorActionPlugin.java b/server/src/main/java/org/elasticsearch/plugins/interceptor/RestServerActionPlugin.java similarity index 68% rename from server/src/main/java/org/elasticsearch/plugins/interceptor/RestInterceptorActionPlugin.java rename to server/src/main/java/org/elasticsearch/plugins/interceptor/RestServerActionPlugin.java index 5ef571f6219e8..efabc85268acc 100644 --- a/server/src/main/java/org/elasticsearch/plugins/interceptor/RestInterceptorActionPlugin.java +++ b/server/src/main/java/org/elasticsearch/plugins/interceptor/RestServerActionPlugin.java @@ -8,16 +8,22 @@ package org.elasticsearch.plugins.interceptor; +import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.plugins.ActionPlugin; +import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestHandler; +import org.elasticsearch.tracing.Tracer; +import org.elasticsearch.usage.UsageService; import java.util.function.UnaryOperator; /** * An action plugin that intercepts incoming the REST requests. */ -public interface RestInterceptorActionPlugin extends ActionPlugin { +public interface RestServerActionPlugin extends ActionPlugin { /** * Returns a function used to intercept each rest request before handling the request. @@ -41,4 +47,19 @@ public interface RestInterceptorActionPlugin extends ActionPlugin { * Note: Only one installed plugin may implement a rest interceptor. */ UnaryOperator getRestHandlerInterceptor(ThreadContext threadContext); + + /** + * Returns a replacement {@link RestController} to be used in the server. + * Note: Only one installed plugin may override the rest controller. + */ + @Nullable + default RestController getRestController( + @Nullable UnaryOperator handlerWrapper, + NodeClient client, + CircuitBreakerService circuitBreakerService, + UsageService usageService, + Tracer tracer + ) { + return null; + } } diff --git a/server/src/test/java/org/elasticsearch/action/ActionModuleTests.java b/server/src/test/java/org/elasticsearch/action/ActionModuleTests.java index 294ef5cb4c6a2..6975cc22adaa9 100644 --- a/server/src/test/java/org/elasticsearch/action/ActionModuleTests.java +++ b/server/src/test/java/org/elasticsearch/action/ActionModuleTests.java @@ -23,9 +23,10 @@ import org.elasticsearch.common.settings.SettingsModule; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.indices.TestIndexNameExpressionResolver; +import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.plugins.ActionPlugin; import org.elasticsearch.plugins.ActionPlugin.ActionHandler; -import org.elasticsearch.plugins.interceptor.RestInterceptorActionPlugin; +import org.elasticsearch.plugins.interceptor.RestServerActionPlugin; import org.elasticsearch.rest.RestChannel; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestHandler; @@ -36,6 +37,7 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.tracing.Tracer; import org.elasticsearch.usage.UsageService; import org.hamcrest.Matchers; @@ -259,7 +261,7 @@ public void test3rdPartyHandlerIsNotInstalled() { SettingsModule settingsModule = new SettingsModule(Settings.EMPTY); ThreadPool threadPool = new TestThreadPool(getTestName()); - ActionPlugin secPlugin = new SecPlugin(); + ActionPlugin secPlugin = new SecPlugin(true, false); try { UsageService usageService = new UsageService(); @@ -286,7 +288,45 @@ public void test3rdPartyHandlerIsNotInstalled() { e.getMessage(), Matchers.equalTo( "The org.elasticsearch.action.ActionModuleTests$SecPlugin plugin tried to " - + "install a custom REST interceptor. This functionality is not available anymore." + + "install a custom REST interceptor. This functionality is not available to external plugins." + ) + ); + } finally { + threadPool.shutdown(); + } + } + + public void test3rdPartyRestControllerIsNotInstalled() { + SettingsModule settingsModule = new SettingsModule(Settings.EMPTY); + ThreadPool threadPool = new TestThreadPool(getTestName()); + ActionPlugin secPlugin = new SecPlugin(false, true); + try { + UsageService usageService = new UsageService(); + + Exception e = expectThrows( + IllegalArgumentException.class, + () -> new ActionModule( + settingsModule.getSettings(), + TestIndexNameExpressionResolver.newInstance(threadPool.getThreadContext()), + settingsModule.getIndexScopedSettings(), + settingsModule.getClusterSettings(), + settingsModule.getSettingsFilter(), + threadPool, + Arrays.asList(secPlugin), + null, + null, + usageService, + null, + null, + mock(ClusterService.class), + List.of() + ) + ); + assertThat( + e.getMessage(), + Matchers.equalTo( + "The org.elasticsearch.action.ActionModuleTests$SecPlugin plugin tried to install a custom REST controller." + + " This functionality is not available to external plugins." ) ); } finally { @@ -304,10 +344,37 @@ public List routes() { public void handleRequest(RestRequest request, RestChannel channel, NodeClient client) throws Exception {} } - class SecPlugin implements ActionPlugin, RestInterceptorActionPlugin { + class SecPlugin implements ActionPlugin, RestServerActionPlugin { + private final boolean installInterceptor; + private final boolean installController; + + SecPlugin(boolean installInterceptor, boolean installController) { + this.installInterceptor = installInterceptor; + this.installController = installController; + } + @Override public UnaryOperator getRestHandlerInterceptor(ThreadContext threadContext) { - return UnaryOperator.identity(); + if (installInterceptor) { + return UnaryOperator.identity(); + } else { + return null; + } } - }; + + @Override + public RestController getRestController( + UnaryOperator handlerWrapper, + NodeClient client, + CircuitBreakerService circuitBreakerService, + UsageService usageService, + Tracer tracer + ) { + if (installController) { + return new RestController(handlerWrapper, client, circuitBreakerService, usageService, tracer); + } else { + return null; + } + } + } } diff --git a/server/src/test/java/org/elasticsearch/module/BasicServerModuleTests.java b/server/src/test/java/org/elasticsearch/module/BasicServerModuleTests.java index 73a087d244f6e..c283a2f1a06d2 100644 --- a/server/src/test/java/org/elasticsearch/module/BasicServerModuleTests.java +++ b/server/src/test/java/org/elasticsearch/module/BasicServerModuleTests.java @@ -38,9 +38,17 @@ public class BasicServerModuleTests extends ESTestCase { public void testQualifiedExports() { var md = getServerDescriptor(); - // The package containing the RestInterceptor type, org.elasticsearch.plugins.interceptor, - // should only be exported to security. - assertThat(md.exports(), hasItem(exportsOf("org.elasticsearch.plugins.interceptor", Set.of("org.elasticsearch.security")))); + // The package containing the RestServerActionPlugin (RestInterceptor) type, org.elasticsearch.plugins.interceptor, + // should only be exported to security or serverless (rest controller) + assertThat( + md.exports(), + hasItem( + exportsOf( + "org.elasticsearch.plugins.interceptor", + Set.of("org.elasticsearch.security", "org.elasticsearch.serverless.rest") + ) + ) + ); // additional qualified export constraint go here } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/LocalStateCompositeXPackPlugin.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/LocalStateCompositeXPackPlugin.java index af0f0187dc003..37358c483a749 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/LocalStateCompositeXPackPlugin.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/LocalStateCompositeXPackPlugin.java @@ -80,7 +80,7 @@ import org.elasticsearch.plugins.SearchPlugin; import org.elasticsearch.plugins.ShutdownAwarePlugin; import org.elasticsearch.plugins.SystemIndexPlugin; -import org.elasticsearch.plugins.interceptor.RestInterceptorActionPlugin; +import org.elasticsearch.plugins.interceptor.RestServerActionPlugin; import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.repositories.Repository; import org.elasticsearch.rest.RestController; @@ -138,7 +138,7 @@ public class LocalStateCompositeXPackPlugin extends XPackPlugin SystemIndexPlugin, SearchPlugin, ShutdownAwarePlugin, - RestInterceptorActionPlugin { + RestServerActionPlugin { private XPackLicenseState licenseState; private SSLService sslService; @@ -441,8 +441,8 @@ public UnaryOperator getRestHandlerInterceptor(ThreadContext thread // There can be only one. List> items = filterPlugins(ActionPlugin.class).stream() - .filter(RestInterceptorActionPlugin.class::isInstance) - .map(RestInterceptorActionPlugin.class::cast) + .filter(RestServerActionPlugin.class::isInstance) + .map(RestServerActionPlugin.class::cast) .map(p -> p.getRestHandlerInterceptor(threadContext)) .filter(Objects::nonNull) .toList(); diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java index 257e1a0181743..f10d3dc832e09 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java @@ -79,7 +79,7 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.SearchPlugin; import org.elasticsearch.plugins.SystemIndexPlugin; -import org.elasticsearch.plugins.interceptor.RestInterceptorActionPlugin; +import org.elasticsearch.plugins.interceptor.RestServerActionPlugin; import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.reservedstate.ReservedClusterStateHandler; import org.elasticsearch.rest.RestController; @@ -419,7 +419,7 @@ public class Security extends Plugin MapperPlugin, ExtensiblePlugin, SearchPlugin, - RestInterceptorActionPlugin { + RestServerActionPlugin { public static final String SECURITY_CRYPTO_THREAD_POOL_NAME = XPackField.SECURITY + "-crypto"; diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java index 0b9a009bc7af2..1ffb4fde5b443 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/SecurityTests.java @@ -756,7 +756,7 @@ public void testSecurityRestHandlerInterceptorCanBeInstalled() throws IllegalAcc "Security rest interceptor", ActionModule.class.getName(), Level.DEBUG, - "Using REST interceptor from plugin org.elasticsearch.xpack.security.Security" + "Using custom REST interceptor from plugin org.elasticsearch.xpack.security.Security" ) ); From 847ec45baaa841c9906834197c113e36a41e6a59 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 8 Aug 2023 07:09:25 +0100 Subject: [PATCH 09/11] Remove bound on SEARCH_COORDINATION default size (#98264) Today by default the `SEARCH_COORDINATION` pool is sized at half the allocated processors, or five if there are more than ten CPUs. Yet, if we scale up a node to have more than ten CPUs, we probably want to scale up the number of search coordination threads to match. This commit removes the limit of five threads. --- docs/reference/modules/threadpool.asciidoc | 4 ++-- .../elasticsearch/threadpool/ThreadPool.java | 13 +++++++++--- .../threadpool/ThreadPoolTests.java | 20 +++++++++++++++++++ 3 files changed, 32 insertions(+), 5 deletions(-) diff --git a/docs/reference/modules/threadpool.asciidoc b/docs/reference/modules/threadpool.asciidoc index 4047bb98c721a..70da29576765d 100644 --- a/docs/reference/modules/threadpool.asciidoc +++ b/docs/reference/modules/threadpool.asciidoc @@ -23,8 +23,8 @@ There are several thread pools, but the important ones include: `search_coordination`:: For lightweight search-related coordination operations. Thread pool type is - `fixed` with a size of a max of `min(5, (`<>`) / 2)`, and queue_size of `1000`. + `fixed` with a size of `(`<>`) / 2`, + and queue_size of `1000`. `get`:: For get operations. Thread pool type is diff --git a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java index db757f642005a..c345d7ca00f4c 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java +++ b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java @@ -182,9 +182,11 @@ public ThreadPool(final Settings settings, final ExecutorBuilder... customBui final Map builders = new HashMap<>(); final int allocatedProcessors = EsExecutors.allocatedProcessors(settings); + final int halfProc = halfAllocatedProcessors(allocatedProcessors); final int halfProcMaxAt5 = halfAllocatedProcessorsMaxFive(allocatedProcessors); final int halfProcMaxAt10 = halfAllocatedProcessorsMaxTen(allocatedProcessors); final int genericThreadPoolMax = boundedBy(4 * allocatedProcessors, 128, 512); + builders.put( Names.GENERIC, new ScalingExecutorBuilder(Names.GENERIC, 4, genericThreadPoolMax, TimeValue.timeValueSeconds(30), false) @@ -216,7 +218,7 @@ public ThreadPool(final Settings settings, final ExecutorBuilder... customBui ); builders.put( Names.SEARCH_COORDINATION, - new FixedExecutorBuilder(settings, Names.SEARCH_COORDINATION, halfProcMaxAt5, 1000, TaskTrackingConfig.DEFAULT) + new FixedExecutorBuilder(settings, Names.SEARCH_COORDINATION, halfProc, 1000, TaskTrackingConfig.DEFAULT) ); builders.put( Names.AUTO_COMPLETE, @@ -588,15 +590,20 @@ public ScheduledExecutorService scheduler() { * than value, otherwise value */ static int boundedBy(int value, int min, int max) { + assert min < max : min + " vs " + max; return Math.min(max, Math.max(min, value)); } + static int halfAllocatedProcessors(final int allocatedProcessors) { + return (allocatedProcessors + 1) / 2; + } + static int halfAllocatedProcessorsMaxFive(final int allocatedProcessors) { - return boundedBy((allocatedProcessors + 1) / 2, 1, 5); + return boundedBy(halfAllocatedProcessors(allocatedProcessors), 1, 5); } static int halfAllocatedProcessorsMaxTen(final int allocatedProcessors) { - return boundedBy((allocatedProcessors + 1) / 2, 1, 10); + return boundedBy(halfAllocatedProcessors(allocatedProcessors), 1, 10); } static int twiceAllocatedProcessors(final int allocatedProcessors) { diff --git a/server/src/test/java/org/elasticsearch/threadpool/ThreadPoolTests.java b/server/src/test/java/org/elasticsearch/threadpool/ThreadPoolTests.java index 5faf926c3b9a2..97f0037f20367 100644 --- a/server/src/test/java/org/elasticsearch/threadpool/ThreadPoolTests.java +++ b/server/src/test/java/org/elasticsearch/threadpool/ThreadPoolTests.java @@ -334,6 +334,26 @@ public void testForceMergeThreadPoolSize() { } } + public void testSearchCoordinationThreadPoolSize() { + final int expectedSize = randomIntBetween(1, EsExecutors.allocatedProcessors(Settings.EMPTY) / 2); + final int allocatedProcessors = Math.min( + EsExecutors.allocatedProcessors(Settings.EMPTY), + expectedSize * 2 - (randomIntBetween(0, 1)) + ); + final ThreadPool threadPool = new TestThreadPool( + "test", + Settings.builder().put(EsExecutors.NODE_PROCESSORS_SETTING.getKey(), allocatedProcessors).build() + ); + try { + ThreadPool.Info info = threadPool.info(ThreadPool.Names.SEARCH_COORDINATION); + assertThat(info.getThreadPoolType(), equalTo(ThreadPool.ThreadPoolType.FIXED)); + assertThat(info.getMin(), equalTo(expectedSize)); + assertThat(info.getMax(), equalTo(expectedSize)); + } finally { + assertTrue(terminate(threadPool)); + } + } + public void testGetMaxSnapshotCores() { int allocatedProcessors = randomIntBetween(1, 16); assertThat( From cd48337754c85810091750fda716b059a643eabb Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 8 Aug 2023 07:09:40 +0100 Subject: [PATCH 10/11] Fork CCS search-shards handling (#98209) Similar to #98124, this action involves doing potentially O(#shards) work on both sender and receiver so it'd be best to avoid the transport worker. --- docs/changelog/98209.yaml | 5 +++++ .../shards/TransportClusterSearchShardsAction.java | 2 +- .../action/search/TransportSearchAction.java | 11 ++++------- .../action/search/TransportSearchShardsAction.java | 3 ++- 4 files changed, 12 insertions(+), 9 deletions(-) create mode 100644 docs/changelog/98209.yaml diff --git a/docs/changelog/98209.yaml b/docs/changelog/98209.yaml new file mode 100644 index 0000000000000..7939137429d14 --- /dev/null +++ b/docs/changelog/98209.yaml @@ -0,0 +1,5 @@ +pr: 98209 +summary: Fork CCS search-shards handling +area: Search +type: bug +issues: [] diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/shards/TransportClusterSearchShardsAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/shards/TransportClusterSearchShardsAction.java index 5b514dd32c957..9cbcbf821c75a 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/shards/TransportClusterSearchShardsAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/shards/TransportClusterSearchShardsAction.java @@ -57,7 +57,7 @@ public TransportClusterSearchShardsAction( ClusterSearchShardsRequest::new, indexNameExpressionResolver, ClusterSearchShardsResponse::new, - ThreadPool.Names.SAME + ThreadPool.Names.SEARCH_COORDINATION ); this.indicesService = indicesService; } diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 7249f6f843979..fe33aad227b40 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -74,7 +74,6 @@ import org.elasticsearch.transport.RemoteTransportException; import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportRequestOptions; -import org.elasticsearch.transport.TransportResponseHandler; import org.elasticsearch.transport.TransportService; import java.util.ArrayList; @@ -663,6 +662,7 @@ static void collectSearchShards( ) { @Override void innerOnResponse(SearchShardsResponse searchShardsResponse) { + assert ThreadPool.assertCurrentThreadPool(ThreadPool.Names.SEARCH_COORDINATION); searchShardsResponses.put(clusterAlias, searchShardsResponse); } @@ -676,6 +676,7 @@ Map createFinalResponse() { skipUnavailable == false, ActionListener.wrap(connection -> { final String[] indices = entry.getValue().indices(); + final Executor responseExecutor = transportService.getThreadPool().executor(ThreadPool.Names.SEARCH_COORDINATION); // TODO: support point-in-time if (searchContext == null && connection.getTransportVersion().onOrAfter(TransportVersion.V_8_500_010)) { SearchShardsRequest searchShardsRequest = new SearchShardsRequest( @@ -692,11 +693,7 @@ Map createFinalResponse() { SearchShardsAction.NAME, searchShardsRequest, TransportRequestOptions.EMPTY, - new ActionListenerResponseHandler<>( - singleListener, - SearchShardsResponse::new, - TransportResponseHandler.TRANSPORT_WORKER - ) + new ActionListenerResponseHandler<>(singleListener, SearchShardsResponse::new, responseExecutor) ); } else { ClusterSearchShardsRequest searchShardsRequest = new ClusterSearchShardsRequest(indices).indicesOptions( @@ -710,7 +707,7 @@ Map createFinalResponse() { new ActionListenerResponseHandler<>( singleListener.map(SearchShardsResponse::fromLegacyResponse), ClusterSearchShardsResponse::new, - TransportResponseHandler.TRANSPORT_WORKER + responseExecutor ) ); } diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchShardsAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchShardsAction.java index acc600a218152..19af8ad8a1beb 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchShardsAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchShardsAction.java @@ -60,7 +60,7 @@ public TransportSearchShardsAction( SearchTransportService searchTransportService, IndexNameExpressionResolver indexNameExpressionResolver ) { - super(SearchShardsAction.NAME, transportService, actionFilters, SearchShardsRequest::new); + super(SearchShardsAction.NAME, transportService, actionFilters, SearchShardsRequest::new, ThreadPool.Names.SEARCH_COORDINATION); this.transportService = transportService; this.transportSearchAction = transportSearchAction; this.searchService = searchService; @@ -73,6 +73,7 @@ public TransportSearchShardsAction( @Override protected void doExecute(Task task, SearchShardsRequest searchShardsRequest, ActionListener listener) { + assert ThreadPool.assertCurrentThreadPool(ThreadPool.Names.SEARCH_COORDINATION); final long relativeStartNanos = System.nanoTime(); SearchRequest original = new SearchRequest(searchShardsRequest.indices()).indicesOptions(searchShardsRequest.indicesOptions()) .routing(searchShardsRequest.routing()) From 837dcc204c15f27e8770bb8374cb6025c34d5b61 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 8 Aug 2023 07:10:00 +0100 Subject: [PATCH 11/11] Fork CCS remote-cluster responses (#98124) Today during CCS we process responses from remote clusters on the transport worker thread, but that's not the right place for this work. This commit moves it to the `SEARCH_COORDINATION` pool. Closes #97997 --- docs/changelog/98124.yaml | 6 ++++++ .../elasticsearch/action/search/TransportSearchAction.java | 4 +--- 2 files changed, 7 insertions(+), 3 deletions(-) create mode 100644 docs/changelog/98124.yaml diff --git a/docs/changelog/98124.yaml b/docs/changelog/98124.yaml new file mode 100644 index 0000000000000..01d7e313fff90 --- /dev/null +++ b/docs/changelog/98124.yaml @@ -0,0 +1,6 @@ +pr: 98124 +summary: Fork CCS remote-cluster responses +area: Search +type: bug +issues: + - 97997 diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index fe33aad227b40..9dfe6aa8184c0 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -46,7 +46,6 @@ import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.util.Maps; import org.elasticsearch.common.util.concurrent.CountDown; -import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.Index; @@ -474,8 +473,7 @@ static void ccsRemoteReduce( ActionListener listener, BiConsumer> localSearchConsumer ) { - // TODO pick a more appropriate executor for this work - see https://github.com/elastic/elasticsearch/issues/97997 - final var remoteClientResponseExecutor = EsExecutors.DIRECT_EXECUTOR_SERVICE; + final var remoteClientResponseExecutor = threadPool.executor(ThreadPool.Names.SEARCH_COORDINATION); if (localIndices == null && remoteIndices.size() == 1) { // if we are searching against a single remote cluster, we simply forward the original search request to such cluster // and we directly perform final reduction in the remote cluster