mirror of
https://github.com/elastic/elasticsearch.git
synced 2025-04-24 23:27:25 -04:00
Support cluster/details for CCS minimize_roundtrips=false (#98457)
This commit tracks progress for each shard search by cluster alias using a new SearchProgressListener (CCSSingleCoordinatorSearchProgressListener). Both sync and async CCS searches use this new progress listener when minimize_roundtrips=false. Two of the SearchProgressListener method had to be extended to allow tracking per-cluster took values (TransportSearchAction.SearchTimeProvider) and whether searches timed out (by passing in QuerySearchResult to the onQueryResult listener method). This commit brings parity between minimize_roundtrips=true and false to have the same _cluster/details sections in CCS search responses. Note that there are still a few differences between minimize_roundtrips=true and false. 1. The per-cluster took value for minimize_roundtrips=true is accurate, but the for 'false' it is only measured at the granualarity of each partial reduce, so the per cluster took time is overestimated in basically all cases. 2. For minimize_roundtrips=true, a skip_unavailable=false cluster that disconnects during the search or has all searches on all shards fail, will cause the entire search to fail. This is (still) not true for minimize_roundtrips=false. The search is only failed if the skip_unavailable=false cluster cannot be connected to at the start of the search. (This will likely be changed in a follow up ticket that implements fail-fast logic for in-progress searches that should fail due to a skip_unavailable=true cluster failing.) 3. The shard accounting for minimize_roundtrips=false is always accurate (total shard counts are known at the start of the search). For minimize_roundtrips=true, the shard accounting is only accurate per cluster unless all clusters have successful (or partially successful) searches. For clusters that have failures we do not have shard count info.
This commit is contained in:
parent
e31eed37c5
commit
649821e992
25 changed files with 2040 additions and 504 deletions
5
docs/changelog/98457.yaml
Normal file
5
docs/changelog/98457.yaml
Normal file
|
@ -0,0 +1,5 @@
|
|||
pr: 98457
|
||||
summary: Support cluster/details for CCS minimize_roundtrips=false
|
||||
area: Search
|
||||
type: enhancement
|
||||
issues: []
|
|
@ -76,17 +76,19 @@ PUT _cluster/settings
|
|||
"remote": {
|
||||
"cluster_one": {
|
||||
"seeds": [
|
||||
"127.0.0.1:9300"
|
||||
]
|
||||
"35.238.149.1:9300"
|
||||
],
|
||||
"skip_unavailable": true
|
||||
},
|
||||
"cluster_two": {
|
||||
"seeds": [
|
||||
"127.0.0.1:9301"
|
||||
]
|
||||
"35.238.149.2:9300"
|
||||
],
|
||||
"skip_unavailable": false
|
||||
},
|
||||
"cluster_three": {
|
||||
"cluster_three": { <1>
|
||||
"seeds": [
|
||||
"127.0.0.1:9302"
|
||||
"35.238.149.3:9300"
|
||||
]
|
||||
}
|
||||
}
|
||||
|
@ -95,7 +97,12 @@ PUT _cluster/settings
|
|||
}
|
||||
--------------------------------
|
||||
// TEST[setup:host]
|
||||
// TEST[s/127.0.0.1:930\d+/\${transport_host}/]
|
||||
// TEST[s/35.238.149.\d+:930\d+/\${transport_host}/]
|
||||
|
||||
<1> Since `skip_unavailable` was not set on `cluster_three`, it uses
|
||||
the default of `false`. See the <<skip-unavailable-clusters>>
|
||||
section for details.
|
||||
|
||||
|
||||
[discrete]
|
||||
[[ccs-search-remote-cluster]]
|
||||
|
@ -111,6 +118,7 @@ The following <<search-search,search>> API request searches the
|
|||
--------------------------------------------------
|
||||
GET /cluster_one:my-index-000001/_search
|
||||
{
|
||||
"size": 1,
|
||||
"query": {
|
||||
"match": {
|
||||
"user.id": "kimchy"
|
||||
|
@ -122,7 +130,9 @@ GET /cluster_one:my-index-000001/_search
|
|||
// TEST[continued]
|
||||
// TEST[setup:my_index]
|
||||
|
||||
The API returns the following response:
|
||||
The API returns the following response. Note that when you
|
||||
search one or more remote clusters, a `_clusters` section is
|
||||
included to provide information about the search on each cluster.
|
||||
|
||||
[source,console-result]
|
||||
--------------------------------------------------
|
||||
|
@ -130,8 +140,8 @@ The API returns the following response:
|
|||
"took": 150,
|
||||
"timed_out": false,
|
||||
"_shards": {
|
||||
"total": 3,
|
||||
"successful": 3,
|
||||
"total": 12,
|
||||
"successful": 12,
|
||||
"failed": 0,
|
||||
"skipped": 0
|
||||
},
|
||||
|
@ -141,13 +151,13 @@ The API returns the following response:
|
|||
"skipped": 0,
|
||||
"details": {
|
||||
"cluster_one": { <1>
|
||||
"status": "successful",
|
||||
"indices": "my-index-000001",
|
||||
"took": 148,
|
||||
"status": "successful", <2>
|
||||
"indices": "my-index-000001", <3>
|
||||
"took": 148, <4>
|
||||
"timed_out": false,
|
||||
"_shards": {
|
||||
"total": 3,
|
||||
"successful": 3,
|
||||
"_shards": { <5>
|
||||
"total": 12,
|
||||
"successful": 12,
|
||||
"skipped": 0,
|
||||
"failed": 0
|
||||
}
|
||||
|
@ -162,7 +172,7 @@ The API returns the following response:
|
|||
"max_score": 1,
|
||||
"hits": [
|
||||
{
|
||||
"_index": "cluster_one:my-index-000001", <2>
|
||||
"_index": "cluster_one:my-index-000001", <6>
|
||||
"_id": "0",
|
||||
"_score": 1,
|
||||
"_source": {
|
||||
|
@ -185,14 +195,22 @@ 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"/]
|
||||
// TESTRESPONSE[s/"total": 3/"total": "$body._shards.total"/]
|
||||
// TESTRESPONSE[s/"successful": 3/"successful": "$body._shards.successful"/]
|
||||
// TESTRESPONSE[s/"total": 12/"total": "$body._shards.total"/]
|
||||
// TESTRESPONSE[s/"successful": 12/"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
|
||||
<1> The `_clusters/details` section shows metadata about the search on each cluster.
|
||||
<2> The cluster status can be one of: *running*, *successful* (searches on all shards
|
||||
were successful), *partial* (searches on at least one shard of the cluster was successful
|
||||
and at least one failed), *skipped* (the search failed on a cluster marked with
|
||||
`skip_unavailable`=`true`) or *failed* (the search failed on a cluster marked with
|
||||
`skip_unavailable`=`false`).
|
||||
<3> The index expression supplied by the user. If you provide a wildcard such as `logs-*`,
|
||||
this section will show the value with the wildcard, not the concrete indices being searched.
|
||||
<4> How long (in milliseconds) the sub-search took on that cluster.
|
||||
<5> The shard details for the sub-search on that cluster.
|
||||
<6> The search response body includes the name of the remote cluster in the
|
||||
`_index` parameter.
|
||||
|
||||
|
||||
|
@ -204,8 +222,9 @@ The API returns the following response:
|
|||
The following <<search,search>> API request searches the `my-index-000001` index on
|
||||
three clusters:
|
||||
|
||||
* Your local cluster
|
||||
* Two remote clusters, `cluster_one` and `cluster_two`
|
||||
* The local ("querying") cluster, with 10 shards
|
||||
* Two remote clusters, `cluster_one`, with 12 shards and `cluster_two`
|
||||
with 6 shards.
|
||||
|
||||
[source,console]
|
||||
--------------------------------------------------
|
||||
|
@ -230,8 +249,8 @@ The API returns the following response:
|
|||
"timed_out": false,
|
||||
"num_reduce_phases": 4,
|
||||
"_shards": {
|
||||
"total": 12,
|
||||
"successful": 12,
|
||||
"total": 28,
|
||||
"successful": 28,
|
||||
"failed": 0,
|
||||
"skipped": 0
|
||||
},
|
||||
|
@ -246,8 +265,8 @@ The API returns the following response:
|
|||
"took": 21,
|
||||
"timed_out": false,
|
||||
"_shards": {
|
||||
"total": 5,
|
||||
"successful": 5,
|
||||
"total": 10,
|
||||
"successful": 10,
|
||||
"skipped": 0,
|
||||
"failed": 0
|
||||
}
|
||||
|
@ -258,8 +277,8 @@ The API returns the following response:
|
|||
"took": 48,
|
||||
"timed_out": false,
|
||||
"_shards": {
|
||||
"total": 4,
|
||||
"successful": 4,
|
||||
"total": 12,
|
||||
"successful": 12,
|
||||
"skipped": 0,
|
||||
"failed": 0
|
||||
}
|
||||
|
@ -270,8 +289,8 @@ The API returns the following response:
|
|||
"took": 141,
|
||||
"timed_out": false,
|
||||
"_shards": {
|
||||
"total" : 3,
|
||||
"successful" : 3,
|
||||
"total" : 6,
|
||||
"successful" : 6,
|
||||
"skipped": 0,
|
||||
"failed": 0
|
||||
}
|
||||
|
@ -344,16 +363,16 @@ 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"/]
|
||||
// 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/"total": 28/"total": "$body._shards.total"/]
|
||||
// TESTRESPONSE[s/"successful": 28/"successful": "$body._shards.successful"/]
|
||||
// TESTRESPONSE[s/"total": 10/"total": "$body._clusters.details.(local)._shards.total"/]
|
||||
// TESTRESPONSE[s/"successful": 10/"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/"total": 12/"total": "$body._clusters.details.cluster_one._shards.total"/]
|
||||
// TESTRESPONSE[s/"successful": 12/"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/"total" : 6/"total": "$body._clusters.details.cluster_two._shards.total"/]
|
||||
// TESTRESPONSE[s/"successful" : 6/"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)".
|
||||
|
@ -368,14 +387,12 @@ means the document came from the local cluster.
|
|||
=== Using async search for {ccs} with ccs_minimize_roundtrips=true
|
||||
|
||||
Remote clusters can be queried asynchronously using the <<async-search,async search>> API.
|
||||
Async searches accept a <<ccs-minimize-roundtrips,`ccs_minimize_roundtrips`>> parameter
|
||||
that defaults to `false`. See <<ccs-min-roundtrips>> to learn more about this option.
|
||||
A {ccs} accepts a <<ccs-minimize-roundtrips,`ccs_minimize_roundtrips`>> parameter. For
|
||||
asynchronous searches it defaults to `false`. (Note: for synchronous searches it defaults to `true`.)
|
||||
See <<ccs-min-roundtrips>> to learn more about this option.
|
||||
|
||||
The following request does an asynchronous search of the `my-index-000001` index using
|
||||
`ccs_minimize_roundtrips=true` against three clusters:
|
||||
|
||||
* The local cluster, with 8 shards
|
||||
* Two remote clusters, `cluster_one` and `cluster_two`, with 10 shards each
|
||||
`ccs_minimize_roundtrips=true` against three clusters (same ones as the previous example).
|
||||
|
||||
[source,console]
|
||||
--------------------------------------------------
|
||||
|
@ -408,7 +425,7 @@ The API returns the following response:
|
|||
"timed_out": false,
|
||||
"num_reduce_phases": 0,
|
||||
"_shards": {
|
||||
"total": 8, <2>
|
||||
"total": 10, <2>
|
||||
"successful": 0,
|
||||
"failed": 0,
|
||||
"skipped": 0
|
||||
|
@ -416,7 +433,24 @@ The API returns the following response:
|
|||
"_clusters": { <3>
|
||||
"total" : 3,
|
||||
"successful" : 0,
|
||||
"skipped": 0
|
||||
"skipped": 0,
|
||||
"details": {
|
||||
"(local)": {
|
||||
"status": "running",
|
||||
"indices": "my-index-000001",
|
||||
"timed_out": false
|
||||
},
|
||||
"cluster_one": {
|
||||
"status": "running",
|
||||
"indices": "my-index-000001",
|
||||
"timed_out": false
|
||||
},
|
||||
"cluster_one": {
|
||||
"status": "running",
|
||||
"indices": "my-index-000001",
|
||||
"timed_out": false
|
||||
}
|
||||
}
|
||||
},
|
||||
"hits": {
|
||||
"total" : {
|
||||
|
@ -429,16 +463,17 @@ The API returns the following response:
|
|||
}
|
||||
}
|
||||
--------------------------------------------------
|
||||
// TEST[skip: terminated_early is absent from final results so is hard to reproduce here]
|
||||
// TEST[skip: hard to reproduce initial state]
|
||||
|
||||
<1> The async search id.
|
||||
<2> When `ccs_minimize_roundtrips` = `true` and searches on the remote clusters
|
||||
are still running, this section indicates the number of shards in scope for the
|
||||
local cluster only. This will be updated to include the total number of shards
|
||||
across all clusters only when the search is completed.
|
||||
across all clusters only when the search is completed. When
|
||||
`ccs_minimize_roundtrips`= `false`, the total shard count is known up front and
|
||||
will be correct.
|
||||
<3> The `_clusters` section indicates that 3 clusters are in scope for the search
|
||||
and all are currently running (since `successful` and `skipped` both equal 0).
|
||||
|
||||
and all are currently running.
|
||||
|
||||
If you query the <<get-async-search,get async search>> endpoint while the query is
|
||||
still running, you will see an update in the `_clusters` and `_shards` section of
|
||||
|
@ -465,15 +500,39 @@ Response:
|
|||
"timed_out": false,
|
||||
"terminated_early": false,
|
||||
"_shards": {
|
||||
"total": 8,
|
||||
"successful": 8, <1>
|
||||
"total": 10,
|
||||
"successful": 10, <1>
|
||||
"skipped": 0,
|
||||
"failed": 0
|
||||
},
|
||||
"_clusters": {
|
||||
"total": 3,
|
||||
"successful": 1, <2>
|
||||
"skipped": 0
|
||||
"skipped": 0,
|
||||
"details": {
|
||||
"(local)": {
|
||||
"status": "successful",
|
||||
"indices": "my-index-000001",
|
||||
"took": 2034,
|
||||
"timed_out": false,
|
||||
"_shards": {
|
||||
"total": 10,
|
||||
"successful": 10,
|
||||
"skipped": 0,
|
||||
"failed": 0
|
||||
}
|
||||
},
|
||||
"cluster_one": {
|
||||
"status": "running",
|
||||
"indices": "my-index-000001",
|
||||
"timed_out": false
|
||||
},
|
||||
"cluster_two": {
|
||||
"status": "running",
|
||||
"indices": "my-index-000001",
|
||||
"timed_out": false
|
||||
}
|
||||
}
|
||||
},
|
||||
"hits": {
|
||||
"total": {
|
||||
|
@ -486,20 +545,21 @@ Response:
|
|||
}
|
||||
}
|
||||
--------------------------------------------------
|
||||
// TEST[skip: terminated_early is absent from final results so is hard to reproduce here]
|
||||
// TEST[skip: hard to reproduce intermediate results]
|
||||
|
||||
|
||||
<1> All the local cluster shards have completed.
|
||||
<2> The local cluster search has completed, so the "successful" clusters entry
|
||||
is set to 1. The `_clusters` response section will not be updated for the remote clusters
|
||||
until all remote searches have finished (either successfully or been skipped).
|
||||
is set to 1. The `_clusters` response metadata will be updated as each cluster
|
||||
finishes.
|
||||
<3> Number of hits from the local cluster search. Final hits are not
|
||||
shown until searches on all clusters have been completed and merged.
|
||||
|
||||
|
||||
After searches on all the clusters have completed, when you query the
|
||||
<<get-async-search,get async search>> endpoint, you will see the final
|
||||
status of the `_clusters` and `_shards` section as well as the hits.
|
||||
After searches on all the clusters have completed, querying the
|
||||
<<get-async-search,get async search>> endpoint will show the final
|
||||
status of the `_clusters` and `_shards` section as well as the hits
|
||||
and any aggregation results.
|
||||
|
||||
[source,console]
|
||||
--------------------------------------------------
|
||||
|
@ -610,12 +670,341 @@ were searched across all clusters and that all were successful.
|
|||
<3> The `_clusters` section shows that searches on all 3 clusters were successful.
|
||||
|
||||
|
||||
[discrete]
|
||||
[[cross-cluster-search-failures]]
|
||||
=== {ccs-cap} failures
|
||||
|
||||
Failures during a {ccs} can result in one of two conditions:
|
||||
|
||||
. partial results (2xx HTTP status code)
|
||||
. a failed search (4xx or 5xx HTTP status code)
|
||||
|
||||
Failure details will be present in the search response in both cases.
|
||||
|
||||
A search will be failed if a cluster marked with `skip_unavailable`=`false`
|
||||
is unavailable, disconnects during the search, or has search failures on
|
||||
all shards. In all other cases, failures will result in partial results.
|
||||
|
||||
Search failures on individual shards will be present in both the `_shards`
|
||||
section and the `_clusters` section of the response.
|
||||
|
||||
A failed search will have an additional top-level `errors` entry in the response.
|
||||
|
||||
Here is an example of a search with partial results due to a failure on one shard
|
||||
of one cluster. The search would be similar to ones shown previously. The
|
||||
`_async_search/status` endpoint is used here to show the completion status and
|
||||
not show the hits.
|
||||
|
||||
[source,console]
|
||||
--------------------------------------------------
|
||||
GET /_async_search/status/FmpwbThueVB4UkRDeUxqb1l4akIza3cbWEJyeVBPQldTV3FGZGdIeUVabXBldzoyMDIw
|
||||
--------------------------------------------------
|
||||
// TEST[continued s/FmpwbThueVB4UkRDeUxqb1l4akIza3cbWEJyeVBPQldTV3FGZGdIeUVabXBldzoyMDIw/\${body.id}/]
|
||||
|
||||
|
||||
Response:
|
||||
|
||||
[source,console-result]
|
||||
--------------------------------------------------
|
||||
{
|
||||
"id": "FmpwbThueVB4UkRDeUxqb1l4akIza3cbWEJyeVBPQldTV3FGZGdIeUVabXBldzoyMDIw",
|
||||
"is_partial": true, <1>
|
||||
"is_running": false,
|
||||
"start_time_in_millis": 1692106901478,
|
||||
"expiration_time_in_millis": 1692538901478,
|
||||
"completion_time_in_millis": 1692106903547,
|
||||
"response": {
|
||||
"took": 2069,
|
||||
"timed_out": false,
|
||||
"num_reduce_phases": 4,
|
||||
"_shards": {
|
||||
"total": 28,
|
||||
"successful": 27,
|
||||
"skipped": 0,
|
||||
"failed": 1,
|
||||
"failures": [ <2>
|
||||
{
|
||||
"shard": 1,
|
||||
"index": "cluster_two:my-index-000001",
|
||||
"node": "LMpUnAu0QEeCUMfg_56sAg",
|
||||
"reason": {
|
||||
"type": "query_shard_exception",
|
||||
"reason": "failed to create query: [my-index-000001][1] exception message here",
|
||||
"index_uuid": "4F2VWx8RQSeIhUE-nksvCQ",
|
||||
"index": "cluster_two:my-index-000001",
|
||||
"caused_by": {
|
||||
"type": "runtime_exception",
|
||||
"reason": "runtime_exception: [my-index-000001][1] exception message here"
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
||||
},
|
||||
"_clusters": {
|
||||
"total": 3,
|
||||
"successful": 3, <3>
|
||||
"skipped": 0,
|
||||
"details": {
|
||||
"(local)": {
|
||||
"status": "successful",
|
||||
"indices": "my-index-000001",
|
||||
"took": 1753,
|
||||
"timed_out": false,
|
||||
"_shards": {
|
||||
"total": 10,
|
||||
"successful": 10,
|
||||
"skipped": 0,
|
||||
"failed": 0
|
||||
}
|
||||
},
|
||||
"cluster_one": {
|
||||
"status": "successful",
|
||||
"indices": "my-index-000001",
|
||||
"took": 2054,
|
||||
"timed_out": false,
|
||||
"_shards": {
|
||||
"total": 12,
|
||||
"successful": 12,
|
||||
"skipped": 0,
|
||||
"failed": 0
|
||||
}
|
||||
},
|
||||
"cluster_two": {
|
||||
"status": "partial", <4>
|
||||
"indices": "my-index-000001",
|
||||
"took": 2039,
|
||||
"timed_out": false,
|
||||
"_shards": {
|
||||
"total": 6,
|
||||
"successful": 5,
|
||||
"skipped": 0,
|
||||
"failed": 1 <5>
|
||||
},
|
||||
"failures": [ <6>
|
||||
{
|
||||
"shard": 1,
|
||||
"index": "cluster_two:my-index-000001",
|
||||
"node": "LMpUnAu0QEeCUMfg_56sAg",
|
||||
"reason": {
|
||||
"type": "query_shard_exception",
|
||||
"reason": "failed to create query: [my-index-000001][1] exception message here",
|
||||
"index_uuid": "4F2VWx8RQSeIhUE-nksvCQ",
|
||||
"index": "cluster_two:my-index-000001",
|
||||
"caused_by": {
|
||||
"type": "runtime_exception",
|
||||
"reason": "runtime_exception: [my-index-000001][1] exception message here"
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
},
|
||||
"hits": {
|
||||
}
|
||||
}
|
||||
}
|
||||
--------------------------------------------------
|
||||
// TEST[skip: hard to reproduce failure results]
|
||||
|
||||
|
||||
<1> The search results are marked as partial, since at least one shard search failed.
|
||||
<2> The `_shards` section includes shard failure info.
|
||||
<3> Clusters that have partial results are still marked as successful. They are
|
||||
marked with status "skipped" (or "failed") only if no data was returned from the search.
|
||||
<4> The `partial` status has been applied to the cluster with partial results.
|
||||
<5> The failed shard count is shown.
|
||||
<6> The shard failures are listed under the cluster/details entry also.
|
||||
|
||||
|
||||
|
||||
Here is an example where both `cluster_one` and `cluster_two` lost connectivity
|
||||
during a {ccs}. Since `cluster_one` is marked as `skip_unavailable`=`true`,
|
||||
its status is `skipped` and since `cluster_two` is marked as `skip_unavailable`=`false`,
|
||||
its status is `failed`. Since there was a `failed` cluster, a top level `error`
|
||||
is also present and this returns an HTTP status of 500 (not shown).
|
||||
|
||||
If you want the search to still return results even when a cluster is
|
||||
unavailable, set `skip_unavailable`=`true` for all the remote clusters.
|
||||
|
||||
[source,console]
|
||||
--------------------------------------------------
|
||||
GET /_async_search/FjktRGJ1Y2w1U0phLTRhZnVyeUZ2MVEbWEJyeVBPQldTV3FGZGdIeUVabXBldzo5NzA4
|
||||
--------------------------------------------------
|
||||
// TEST[continued s/FjktRGJ1Y2w1U0phLTRhZnVyeUZ2MVEbWEJyeVBPQldTV3FGZGdIeUVabXBldzo5NzA4/\${body.id}/]
|
||||
|
||||
|
||||
Response:
|
||||
|
||||
[source,console-result]
|
||||
--------------------------------------------------
|
||||
{
|
||||
"id": "FjktRGJ1Y2w1U0phLTRhZnVyeUZ2MVEbWEJyeVBPQldTV3FGZGdIeUVabXBldzo5NzA4",
|
||||
"is_partial": true,
|
||||
"is_running": false,
|
||||
"start_time_in_millis": 1692112102650,
|
||||
"expiration_time_in_millis": 1692544102650,
|
||||
"completion_time_in_millis": 1692112106177,
|
||||
"response": {
|
||||
"took": 3527,
|
||||
"timed_out": false,
|
||||
"terminated_early": false,
|
||||
"_shards": {
|
||||
"total": 10, <1>
|
||||
"successful": 10,
|
||||
"skipped": 0,
|
||||
"failed": 0
|
||||
},
|
||||
"_clusters": {
|
||||
"total": 3,
|
||||
"successful": 1,
|
||||
"skipped": 2, <2>
|
||||
"details": {
|
||||
"(local)": {
|
||||
"status": "successful",
|
||||
"indices": "my-index-000001",
|
||||
"took": 1473,
|
||||
"timed_out": false,
|
||||
"_shards": {
|
||||
"total": 10,
|
||||
"successful": 10,
|
||||
"skipped": 0,
|
||||
"failed": 0
|
||||
}
|
||||
},
|
||||
"cluster_one": {
|
||||
"status": "skipped", <3>
|
||||
"indices": "my-index-000001",
|
||||
"timed_out": false,
|
||||
"failures": [
|
||||
{
|
||||
"shard": -1,
|
||||
"index": null,
|
||||
"reason": {
|
||||
"type": "node_disconnected_exception", <4>
|
||||
"reason": "[myhostname1][35.238.149.1:9300][indices:data/read/search] disconnected"
|
||||
}
|
||||
}
|
||||
]
|
||||
},
|
||||
"cluster_two": {
|
||||
"status": "failed", <5>
|
||||
"indices": "my-index-000001",
|
||||
"timed_out": false,
|
||||
"failures": [
|
||||
{
|
||||
"shard": -1,
|
||||
"index": null,
|
||||
"reason": {
|
||||
"type": "node_disconnected_exception",
|
||||
"reason": "[myhostname2][35.238.149.2:9300][indices:data/read/search] disconnected"
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
},
|
||||
"hits": {
|
||||
},
|
||||
}
|
||||
"error": { <6>
|
||||
"type": "status_exception",
|
||||
"reason": "error while executing search",
|
||||
"caused_by": {
|
||||
"type": "node_disconnected_exception",
|
||||
"reason": "[myhostname2][35.238.149.2:9300][indices:data/read/search] disconnected"
|
||||
}
|
||||
}
|
||||
}
|
||||
--------------------------------------------------
|
||||
// TEST[skip: hard to reproduce failure results]
|
||||
|
||||
<1> The shard accounting will often be only partial when errors like this occur,
|
||||
since we need to be able to get shard info from remote clusters on each search.
|
||||
<2> The skipped counter is used for both "skipped" and "failed" clusters.
|
||||
<3> `cluster_one` disconnected during the search and it returned no results.
|
||||
Since it is marked in the remote cluster configuration as `skip_unavailable`=`true`,
|
||||
its status is "skipped", which will not fail the entire search.
|
||||
<4> The failures list shows that the remote cluster node disconnected from the
|
||||
querying cluster.
|
||||
<5> `cluster_two` status is "failed", since it is marked in the remote cluster
|
||||
configuration as `skip_unavailable`=`false`.
|
||||
<6> A top level `error` entry is included when there is a "failed" cluster.
|
||||
|
||||
|
||||
[discrete]
|
||||
[[exclude-problematic-clusters]]
|
||||
=== Excluding clusters or indices from a {ccs}
|
||||
|
||||
If you use a wildcard to include a large list of clusters and/or indices,
|
||||
you can explicitly exclude one or more clusters or indices with a `-` minus
|
||||
sign in front of the cluster or index.
|
||||
|
||||
To exclude an entire cluster, you would put the minus sign in front of the
|
||||
cluster alias, such as: `-mycluster:*`. When excluding a cluster, you must
|
||||
use `*` in the index position or an error will be returned.
|
||||
|
||||
To exclude a specific remote index, you would put the minus sign in front
|
||||
of the index, such as `mycluster:-myindex`.
|
||||
|
||||
*Exclude a remote cluster*
|
||||
|
||||
Here's how you would exclude `cluster_three` from a
|
||||
{ccs} that uses a wildcard to specify a list of clusters:
|
||||
|
||||
[source,console]
|
||||
--------------------------------------------------
|
||||
POST /my-index-000001,cluster*:my-index-000001,-cluster_three:*/_async_search <1>
|
||||
{
|
||||
"query": {
|
||||
"match": {
|
||||
"user.id": "kimchy"
|
||||
}
|
||||
},
|
||||
"_source": ["user.id", "message", "http.response.status_code"]
|
||||
}
|
||||
--------------------------------------------------
|
||||
// TEST[continued]
|
||||
// TEST[s/ccs_minimize_roundtrips=true/ccs_minimize_roundtrips=true&wait_for_completion_timeout=100ms&keep_on_completion=true/]
|
||||
|
||||
<1> The `cluster*` notation would naturally include `cluster_one`, `cluster_two` and `cluster_three`.
|
||||
To exclude `cluster_three` use a `-` before the cluster name along with a simple wildcard `*` in
|
||||
the index position. This indicates that you do not want the search to make any contact with
|
||||
`cluster_three`.
|
||||
|
||||
|
||||
*Exclude a remote index*
|
||||
|
||||
Suppose you want to search all indices matching `my-index-*` but you want to exclude
|
||||
`my-index-000001` on `cluster_three`. Here's how you could do that:
|
||||
|
||||
[source,console]
|
||||
--------------------------------------------------
|
||||
POST /my-index-000001,cluster*:my-index-*,cluster_three:-my-index-000001/_async_search <1>
|
||||
{
|
||||
"query": {
|
||||
"match": {
|
||||
"user.id": "kimchy"
|
||||
}
|
||||
},
|
||||
"_source": ["user.id", "message", "http.response.status_code"]
|
||||
}
|
||||
--------------------------------------------------
|
||||
// TEST[continued]
|
||||
// TEST[s/ccs_minimize_roundtrips=true/ccs_minimize_roundtrips=true&wait_for_completion_timeout=100ms&keep_on_completion=true/]
|
||||
|
||||
<1> This will *not* exclude `cluster_three` from the search. It will still be
|
||||
contacted and told to search any indexes matching `my-index-*` except for
|
||||
`my-index-000001`.
|
||||
|
||||
|
||||
|
||||
[discrete]
|
||||
[[ccs-async-search-minimize-roundtrips-false]]
|
||||
=== Using async search for {ccs} with ccs_minimize_roundtrips=false
|
||||
|
||||
The `_shards` and `_clusters` section of the response behave differently
|
||||
when `ccs_minimize_roundtrips` is `false` in asynchronous searches.
|
||||
The `_shards` and `_clusters` section of the response behave
|
||||
differently when `ccs_minimize_roundtrips` is `false`.
|
||||
|
||||
Key differences are:
|
||||
|
||||
|
@ -626,11 +1015,10 @@ of shards is gathered from all clusters before the search starts.
|
|||
shards complete, so you will get a more accurate accounting of progress during a
|
||||
long-running search compared to when minimize roundtrips is used.
|
||||
|
||||
. The `_cluster` section starts off in its final state, showing which clusters
|
||||
were successful or skipped based on gathering shard information before the actual
|
||||
search phase against each shard begins.
|
||||
. The `_cluster` section starts off listing all of its shard counts, since
|
||||
they are also obtained before the query phase begins.
|
||||
|
||||
Example using the same set up as in the previous section (`ccs_minimize_roundtrips=true`):
|
||||
Example using the same setup as in the previous section (`ccs_minimize_roundtrips=true`):
|
||||
|
||||
[source,console]
|
||||
--------------------------------------------------
|
||||
|
@ -670,8 +1058,43 @@ the `wait_for_completion_timeout` duration (see <<async-search>>).
|
|||
},
|
||||
"_clusters": {
|
||||
"total" : 3,
|
||||
"successful": 3, <2>
|
||||
"skipped": 0
|
||||
"successful": 0,
|
||||
"skipped": 0,
|
||||
"details": { <2>
|
||||
"(local)": {
|
||||
"status": "running",
|
||||
"indices": "my-index-000001",
|
||||
"timed_out": false,
|
||||
"_shards": {
|
||||
"total": 10,
|
||||
"successful": 0,
|
||||
"skipped": 0,
|
||||
"failed": 0
|
||||
}
|
||||
},
|
||||
"cluster_one": {
|
||||
"status": "running",
|
||||
"indices": "my-index-000001",
|
||||
"timed_out": false,
|
||||
"_shards": {
|
||||
"total": 12,
|
||||
"successful": 0,
|
||||
"skipped": 0,
|
||||
"failed": 0
|
||||
}
|
||||
},
|
||||
"cluster_two": {
|
||||
"status": "running",
|
||||
"indices": "my-index-000001",
|
||||
"timed_out": false,
|
||||
"_shards": {
|
||||
"total": 6,
|
||||
"successful": 0,
|
||||
"skipped": 0,
|
||||
"failed": 0
|
||||
}
|
||||
}
|
||||
}
|
||||
},
|
||||
"hits": {
|
||||
"total" : {
|
||||
|
@ -684,34 +1107,31 @@ the `wait_for_completion_timeout` duration (see <<async-search>>).
|
|||
}
|
||||
}
|
||||
--------------------------------------------------
|
||||
// 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/"max_score": null/"max_score": "$body.response.hits.max_score"/]
|
||||
// TESTRESPONSE[s/\d+/$body.$_path/]
|
||||
// TESTRESPONSE[s/"hits": \[\]/"hits": $body.response.hits.hits/]
|
||||
// TEST[skip: hard to reproduce intermediate results]
|
||||
|
||||
|
||||
<1> All shards from all clusters in scope for the search are listed here. Watch this
|
||||
section for updates to monitor search progress.
|
||||
section and/or the _clusters section for updates to monitor search progress.
|
||||
<2> The `_clusters` section shows that shard information was successfully
|
||||
gathered from all 3 clusters and that all will be searched (none are being skipped).
|
||||
gathered from all 3 clusters and the total shard count on each cluster is listed.
|
||||
|
||||
|
||||
|
||||
|
||||
[discrete]
|
||||
[[skip-unavailable-clusters]]
|
||||
=== Optional remote clusters
|
||||
|
||||
By default, a {ccs} fails if a remote cluster in the request returns an
|
||||
error or is unavailable. Use the `skip_unavailable` cluster
|
||||
setting to mark a specific remote cluster as optional for {ccs}.
|
||||
By default, a {ccs} fails if a remote cluster in the request is unavailable
|
||||
or returns an error where the search on all shards failed. Use the
|
||||
`skip_unavailable` cluster setting to mark a specific remote cluster as
|
||||
optional for {ccs}.
|
||||
|
||||
If `skip_unavailable` is `true`, a {ccs}:
|
||||
|
||||
* Skips the remote cluster if its nodes are unavailable during the search. The
|
||||
response's `_cluster.skipped` value contains a count of any skipped clusters.
|
||||
response's `_clusters.skipped` value contains a count of any skipped clusters
|
||||
and the `_clusters.details` section of the response will show a `skipped` status.
|
||||
|
||||
* Ignores errors returned by the remote cluster, such as errors related to
|
||||
unavailable shards or indices. This can include errors related to search
|
||||
|
@ -724,7 +1144,7 @@ when searching the remote cluster. This means searches on the remote cluster may
|
|||
return partial results.
|
||||
|
||||
The following <<cluster-update-settings,cluster update settings>>
|
||||
API request changes `cluster_two`'s `skip_unavailable` setting to `true`.
|
||||
API request changes `skip_unavailable` setting to `true` for `cluster_two`.
|
||||
|
||||
[source,console]
|
||||
--------------------------------
|
||||
|
@ -738,7 +1158,10 @@ PUT _cluster/settings
|
|||
// TEST[continued]
|
||||
|
||||
If `cluster_two` is disconnected or unavailable during a {ccs}, {es} won't
|
||||
include matching documents from that cluster in the final results.
|
||||
include matching documents from that cluster in the final results. If at
|
||||
least one shard provides results, those results will be used and the
|
||||
search will return partial data. (If doing {ccs} using async search,
|
||||
the `is_partial` field will be set to `true` to indicate partial results.)
|
||||
|
||||
[discrete]
|
||||
[[ccs-network-delays]]
|
||||
|
@ -774,6 +1197,25 @@ network roundtrips, and sets the parameter `ccs_minimize_roundtrips` to `false`.
|
|||
|
||||
[discrete]
|
||||
[[ccs-min-roundtrips]]
|
||||
==== Considerations for choosing whether to minimize roundtrips in a {ccs}
|
||||
|
||||
For cross-cluster searches that query a large number of shards, the minimize roundtrips
|
||||
option typically provides much better performance. This is especially true if the clusters
|
||||
being searched have high network latency (e.g., distant geographic regions).
|
||||
|
||||
However, not minimizing roundtrips allows you to get back incremental results of
|
||||
any aggregations in your query when using async-search while the search is still
|
||||
running.
|
||||
|
||||
By default, synchronous searches minimize roundtrips, while asynchronous searches
|
||||
do not. You can override the default by using the `ccs_minimize_roundtrips` parameter,
|
||||
setting it to either `true` or `false`, as shown in several examples earlier in this
|
||||
document.
|
||||
|
||||
|
||||
[discrete]
|
||||
[[ccs-min-roundtrips-true]]
|
||||
|
||||
==== Minimize network roundtrips
|
||||
|
||||
Here's how {ccs} works when you minimize network roundtrips.
|
||||
|
@ -886,8 +1328,8 @@ on the same version of {es}. If you need to maintain clusters with different
|
|||
versions, you can:
|
||||
|
||||
* Maintain a dedicated cluster for {ccs}. Keep this cluster on the earliest
|
||||
version needed to search the other clusters. For example, if you have 7.17 and 8.x clusters, you can maintain a dedicated 7.17 cluster to use
|
||||
as the local cluster for {ccs}.
|
||||
version needed to search the other clusters. For example, if you have 7.17 and 8.x clusters,
|
||||
you can maintain a dedicated 7.17 cluster to use as the local cluster for {ccs}.
|
||||
|
||||
* Keep each cluster no more than one minor version apart. This lets you use any
|
||||
cluster as the local cluster when running a {ccs}.
|
||||
|
|
|
@ -1075,7 +1075,6 @@ public class CCSDuelIT extends ESRestTestCase {
|
|||
assertEquals(clustersMRT.getSuccessful(), clustersMRTFalse.getSuccessful());
|
||||
assertEquals(clustersMRT.getSkipped(), clustersMRTFalse.getSkipped());
|
||||
|
||||
boolean removeSkipped = searchRequest.source().collapse() != null;
|
||||
Map<String, Object> minimizeRoundtripsResponseMap = responseToMap(minimizeRoundtripsSearchResponse);
|
||||
if (clustersMRT.hasClusterObjects() && clustersMRTFalse.hasClusterObjects()) {
|
||||
Map<String, Object> fanOutResponseMap = responseToMap(fanOutSearchResponse);
|
||||
|
@ -1148,7 +1147,6 @@ public class CCSDuelIT extends ESRestTestCase {
|
|||
assertEquals(clustersMRT.getSuccessful(), clustersMRTFalse.getSuccessful());
|
||||
assertEquals(clustersMRT.getSkipped(), clustersMRTFalse.getSkipped());
|
||||
|
||||
boolean removeSkipped = searchRequest.source().collapse() != null;
|
||||
Map<String, Object> minimizeRoundtripsResponseMap = responseToMap(minimizeRoundtripsSearchResponse);
|
||||
if (clustersMRT.hasClusterObjects() && clustersMRTFalse.hasClusterObjects()) {
|
||||
Map<String, Object> fanOutResponseMap = responseToMap(fanOutSearchResponse);
|
||||
|
|
|
@ -9,24 +9,57 @@
|
|||
package org.elasticsearch.action.search;
|
||||
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.CollectionUtils;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.index.query.MatchAllQueryBuilder;
|
||||
import org.elasticsearch.plugins.Plugin;
|
||||
import org.elasticsearch.plugins.SearchPlugin;
|
||||
import org.elasticsearch.search.builder.PointInTimeBuilder;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.search.ccs.CrossClusterSearchIT;
|
||||
import org.elasticsearch.search.query.ThrowingQueryBuilder;
|
||||
import org.elasticsearch.test.AbstractMultiClustersTestCase;
|
||||
import org.elasticsearch.transport.RemoteClusterAware;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
|
||||
import static org.hamcrest.Matchers.containsString;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
||||
|
||||
public class CCSPointInTimeIT extends AbstractMultiClustersTestCase {
|
||||
|
||||
public static final String REMOTE_CLUSTER = "remote_cluster";
|
||||
|
||||
@Override
|
||||
protected Collection<String> remoteClusterAlias() {
|
||||
return List.of("remote_cluster");
|
||||
return List.of(REMOTE_CLUSTER);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Collection<Class<? extends Plugin>> nodePlugins(String clusterAlias) {
|
||||
return CollectionUtils.appendToCopy(super.nodePlugins(clusterAlias), CrossClusterSearchIT.TestQueryBuilderPlugin.class);
|
||||
}
|
||||
|
||||
public static class TestQueryBuilderPlugin extends Plugin implements SearchPlugin {
|
||||
public TestQueryBuilderPlugin() {}
|
||||
|
||||
@Override
|
||||
public List<QuerySpec<?>> getQueries() {
|
||||
QuerySpec<ThrowingQueryBuilder> throwingSpec = new QuerySpec<>(ThrowingQueryBuilder.NAME, ThrowingQueryBuilder::new, p -> {
|
||||
throw new IllegalStateException("not implemented");
|
||||
});
|
||||
|
||||
return List.of(throwingSpec);
|
||||
}
|
||||
}
|
||||
|
||||
void indexDocs(Client client, String index, int numDocs) {
|
||||
|
@ -39,7 +72,7 @@ public class CCSPointInTimeIT extends AbstractMultiClustersTestCase {
|
|||
|
||||
public void testBasic() {
|
||||
final Client localClient = client(LOCAL_CLUSTER);
|
||||
final Client remoteClient = client("remote_cluster");
|
||||
final Client remoteClient = client(REMOTE_CLUSTER);
|
||||
int localNumDocs = randomIntBetween(10, 50);
|
||||
assertAcked(localClient.admin().indices().prepareCreate("local_test"));
|
||||
indexDocs(localClient, "local_test", localNumDocs);
|
||||
|
@ -71,11 +104,102 @@ public class CCSPointInTimeIT extends AbstractMultiClustersTestCase {
|
|||
.get();
|
||||
assertNoFailures(resp);
|
||||
assertHitCount(resp, (includeLocalIndex ? localNumDocs : 0) + remoteNumDocs);
|
||||
|
||||
SearchResponse.Clusters clusters = resp.getClusters();
|
||||
int expectedNumClusters = 1 + (includeLocalIndex ? 1 : 0);
|
||||
assertThat(clusters.getTotal(), equalTo(expectedNumClusters));
|
||||
assertThat(clusters.getSuccessful(), equalTo(expectedNumClusters));
|
||||
assertThat(clusters.getSkipped(), equalTo(0));
|
||||
|
||||
if (includeLocalIndex) {
|
||||
AtomicReference<SearchResponse.Cluster> localClusterRef = clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY);
|
||||
assertNotNull(localClusterRef);
|
||||
assertOneSuccessfulShard(localClusterRef.get());
|
||||
}
|
||||
|
||||
AtomicReference<SearchResponse.Cluster> remoteClusterRef = clusters.getCluster(REMOTE_CLUSTER);
|
||||
assertNotNull(remoteClusterRef);
|
||||
assertOneSuccessfulShard(remoteClusterRef.get());
|
||||
|
||||
} finally {
|
||||
closePointInTime(pitId);
|
||||
}
|
||||
}
|
||||
|
||||
public void testFailuresOnOneShardsWithPointInTime() throws ExecutionException, InterruptedException {
|
||||
final Client localClient = client(LOCAL_CLUSTER);
|
||||
final Client remoteClient = client(REMOTE_CLUSTER);
|
||||
int localNumDocs = randomIntBetween(10, 50);
|
||||
int numShards = randomIntBetween(2, 4);
|
||||
Settings clusterSettings = indexSettings(numShards, randomIntBetween(0, 1)).build();
|
||||
assertAcked(localClient.admin().indices().prepareCreate("local_test").setSettings(clusterSettings));
|
||||
indexDocs(localClient, "local_test", localNumDocs);
|
||||
|
||||
int remoteNumDocs = randomIntBetween(10, 50);
|
||||
assertAcked(remoteClient.admin().indices().prepareCreate("remote_test").setSettings(clusterSettings));
|
||||
indexDocs(remoteClient, "remote_test", remoteNumDocs);
|
||||
boolean includeLocalIndex = randomBoolean();
|
||||
List<String> indices = new ArrayList<>();
|
||||
if (includeLocalIndex) {
|
||||
indices.add(randomFrom("*", "local_*", "local_test"));
|
||||
}
|
||||
indices.add(randomFrom("*:*", "remote_cluster:*", "remote_cluster:remote_test"));
|
||||
String pitId = openPointInTime(indices.toArray(new String[0]), TimeValue.timeValueMinutes(2));
|
||||
try {
|
||||
if (randomBoolean()) {
|
||||
localClient.prepareIndex("local_test").setId("local_new").setSource().get();
|
||||
localClient.admin().indices().prepareRefresh().get();
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
remoteClient.prepareIndex("remote_test").setId("remote_new").setSource().get();
|
||||
remoteClient.admin().indices().prepareRefresh().get();
|
||||
}
|
||||
// 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);
|
||||
SearchRequest searchRequest = new SearchRequest();
|
||||
searchRequest.source(new SearchSourceBuilder().query(queryBuilder).size(10).pointInTimeBuilder(new PointInTimeBuilder(pitId)));
|
||||
SearchResponse searchResponse = client(LOCAL_CLUSTER).search(searchRequest).get();
|
||||
|
||||
SearchResponse.Clusters clusters = searchResponse.getClusters();
|
||||
int expectedNumClusters = 1 + (includeLocalIndex ? 1 : 0);
|
||||
assertThat(clusters.getTotal(), equalTo(expectedNumClusters));
|
||||
assertThat(clusters.getSuccessful(), equalTo(expectedNumClusters));
|
||||
assertThat(clusters.getSkipped(), equalTo(0));
|
||||
|
||||
if (includeLocalIndex) {
|
||||
AtomicReference<SearchResponse.Cluster> localClusterRef = clusters.getCluster(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY);
|
||||
assertNotNull(localClusterRef);
|
||||
assertOneFailedShard(localClusterRef.get(), numShards);
|
||||
}
|
||||
AtomicReference<SearchResponse.Cluster> remoteClusterRef = clusters.getCluster(REMOTE_CLUSTER);
|
||||
assertNotNull(remoteClusterRef);
|
||||
assertOneFailedShard(remoteClusterRef.get(), numShards);
|
||||
|
||||
} finally {
|
||||
closePointInTime(pitId);
|
||||
}
|
||||
}
|
||||
|
||||
private static void assertOneSuccessfulShard(SearchResponse.Cluster cluster) {
|
||||
assertThat(cluster.getTotalShards(), equalTo(1));
|
||||
assertThat(cluster.getSuccessfulShards(), equalTo(1));
|
||||
assertThat(cluster.getFailedShards(), equalTo(0));
|
||||
assertThat(cluster.getFailures().size(), equalTo(0));
|
||||
assertThat(cluster.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL));
|
||||
assertThat(cluster.getTook().millis(), greaterThanOrEqualTo(0L));
|
||||
assertFalse(cluster.isTimedOut());
|
||||
}
|
||||
|
||||
private static void assertOneFailedShard(SearchResponse.Cluster cluster, int totalShards) {
|
||||
assertThat(cluster.getSuccessfulShards(), equalTo(totalShards - 1));
|
||||
assertThat(cluster.getFailedShards(), equalTo(1));
|
||||
assertThat(cluster.getFailures().size(), equalTo(1));
|
||||
assertThat(cluster.getFailures().get(0).reason(), containsString("index corrupted"));
|
||||
assertThat(cluster.getStatus(), equalTo(SearchResponse.Cluster.Status.PARTIAL));
|
||||
assertThat(cluster.getTook().millis(), greaterThanOrEqualTo(0L));
|
||||
assertFalse(cluster.isTimedOut());
|
||||
}
|
||||
|
||||
private String openPointInTime(String[] indices, TimeValue keepAlive) {
|
||||
OpenPointInTimeRequest request = new OpenPointInTimeRequest(indices).keepAlive(keepAlive);
|
||||
final OpenPointInTimeResponse response = client().execute(OpenPointInTimeAction.INSTANCE, request).actionGet();
|
||||
|
|
|
@ -18,6 +18,7 @@ import org.elasticsearch.search.SearchShardTarget;
|
|||
import org.elasticsearch.search.aggregations.AggregationBuilders;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.search.query.QuerySearchResult;
|
||||
import org.elasticsearch.search.sort.FieldSortBuilder;
|
||||
import org.elasticsearch.search.sort.SortOrder;
|
||||
import org.elasticsearch.tasks.TaskId;
|
||||
|
@ -109,14 +110,15 @@ public class SearchProgressActionListenerIT extends ESSingleNodeTestCase {
|
|||
List<SearchShard> searchShards,
|
||||
List<SearchShard> skippedShards,
|
||||
SearchResponse.Clusters clusters,
|
||||
boolean fetchPhase
|
||||
boolean fetchPhase,
|
||||
TransportSearchAction.SearchTimeProvider timeProvider
|
||||
) {
|
||||
shardsListener.set(searchShards);
|
||||
assertEquals(fetchPhase, hasFetchPhase);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onQueryResult(int shardIndex) {
|
||||
public void onQueryResult(int shardIndex, QuerySearchResult result) {
|
||||
assertThat(shardIndex, lessThan(shardsListener.get().size()));
|
||||
numQueryResults.incrementAndGet();
|
||||
}
|
||||
|
|
|
@ -11,33 +11,34 @@ package org.elasticsearch.search.ccs;
|
|||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.SearchType;
|
||||
import org.elasticsearch.action.search.ShardSearchFailure;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.PlainActionFuture;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetadata;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.settings.Setting;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.CollectionUtils;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.index.query.MatchAllQueryBuilder;
|
||||
import org.elasticsearch.index.query.RangeQueryBuilder;
|
||||
import org.elasticsearch.plugins.Plugin;
|
||||
import org.elasticsearch.plugins.SearchPlugin;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.search.query.SlowRunningQueryBuilder;
|
||||
import org.elasticsearch.search.query.ThrowingQueryBuilder;
|
||||
import org.elasticsearch.test.AbstractMultiClustersTestCase;
|
||||
import org.elasticsearch.test.InternalTestCluster;
|
||||
import org.elasticsearch.transport.RemoteClusterAware;
|
||||
import org.elasticsearch.transport.RemoteTransportException;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||
import static org.hamcrest.Matchers.containsString;
|
||||
|
@ -69,8 +70,7 @@ public class CrossClusterSearchIT extends AbstractMultiClustersTestCase {
|
|||
|
||||
@Override
|
||||
protected Collection<Class<? extends Plugin>> nodePlugins(String clusterAlias) {
|
||||
List<Class<? extends Plugin>> plugs = Arrays.asList(TestQueryBuilderPlugin.class);
|
||||
return Stream.concat(super.nodePlugins(clusterAlias).stream(), plugs.stream()).collect(Collectors.toList());
|
||||
return CollectionUtils.appendToCopy(super.nodePlugins(clusterAlias), CrossClusterSearchIT.TestQueryBuilderPlugin.class);
|
||||
}
|
||||
|
||||
public static class TestQueryBuilderPlugin extends Plugin implements SearchPlugin {
|
||||
|
@ -78,11 +78,18 @@ public class CrossClusterSearchIT extends AbstractMultiClustersTestCase {
|
|||
|
||||
@Override
|
||||
public List<QuerySpec<?>> getQueries() {
|
||||
QuerySpec<SlowRunningQueryBuilder> slowRunningSpec = new QuerySpec<>(
|
||||
SlowRunningQueryBuilder.NAME,
|
||||
SlowRunningQueryBuilder::new,
|
||||
p -> {
|
||||
throw new IllegalStateException("not implemented");
|
||||
}
|
||||
);
|
||||
QuerySpec<ThrowingQueryBuilder> throwingSpec = new QuerySpec<>(ThrowingQueryBuilder.NAME, ThrowingQueryBuilder::new, p -> {
|
||||
throw new IllegalStateException("not implemented");
|
||||
});
|
||||
|
||||
return List.of(throwingSpec);
|
||||
return List.of(slowRunningSpec, throwingSpec);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -93,18 +100,26 @@ public class CrossClusterSearchIT extends AbstractMultiClustersTestCase {
|
|||
int localNumShards = (Integer) testClusterInfo.get("local.num_shards");
|
||||
int remoteNumShards = (Integer) testClusterInfo.get("remote.num_shards");
|
||||
|
||||
PlainActionFuture<SearchResponse> queryFuture = new PlainActionFuture<>();
|
||||
SearchRequest searchRequest = new SearchRequest(localIndex, REMOTE_CLUSTER + ":" + remoteIndex);
|
||||
if (randomBoolean()) {
|
||||
searchRequest = searchRequest.scroll("1m");
|
||||
}
|
||||
searchRequest.allowPartialSearchResults(false);
|
||||
boolean minimizeRoundtrips = true; // TODO: support MRT=false
|
||||
if (randomBoolean()) {
|
||||
searchRequest.setBatchedReduceSize(randomIntBetween(3, 20));
|
||||
}
|
||||
boolean minimizeRoundtrips = randomBoolean();
|
||||
searchRequest.setCcsMinimizeRoundtrips(minimizeRoundtrips);
|
||||
boolean dfs = randomBoolean();
|
||||
if (dfs) {
|
||||
searchRequest.searchType(SearchType.DFS_QUERY_THEN_FETCH);
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
searchRequest.setPreFilterShardSize(1);
|
||||
}
|
||||
searchRequest.source(new SearchSourceBuilder().query(new MatchAllQueryBuilder()).size(10));
|
||||
|
||||
searchRequest.source(new SearchSourceBuilder().query(new MatchAllQueryBuilder()).size(1000));
|
||||
client(LOCAL_CLUSTER).search(searchRequest, queryFuture);
|
||||
|
||||
assertBusy(() -> assertTrue(queryFuture.isDone()));
|
||||
|
||||
SearchResponse searchResponse = queryFuture.get();
|
||||
SearchResponse searchResponse = client(LOCAL_CLUSTER).search(searchRequest).get();
|
||||
assertNotNull(searchResponse);
|
||||
|
||||
SearchResponse.Clusters clusters = searchResponse.getClusters();
|
||||
|
@ -145,20 +160,27 @@ public class CrossClusterSearchIT extends AbstractMultiClustersTestCase {
|
|||
int localNumShards = (Integer) testClusterInfo.get("local.num_shards");
|
||||
int remoteNumShards = (Integer) testClusterInfo.get("remote.num_shards");
|
||||
|
||||
PlainActionFuture<SearchResponse> queryFuture = new PlainActionFuture<>();
|
||||
SearchRequest searchRequest = new SearchRequest(localIndex, REMOTE_CLUSTER + ":" + remoteIndex);
|
||||
searchRequest.allowPartialSearchResults(false);
|
||||
boolean minimizeRoundtrips = true; // TODO support MRT=false
|
||||
boolean minimizeRoundtrips = randomBoolean();
|
||||
searchRequest.setCcsMinimizeRoundtrips(minimizeRoundtrips);
|
||||
if (randomBoolean()) {
|
||||
searchRequest.setBatchedReduceSize(randomIntBetween(3, 20));
|
||||
}
|
||||
boolean dfs = randomBoolean();
|
||||
if (dfs) {
|
||||
searchRequest.searchType(SearchType.DFS_QUERY_THEN_FETCH);
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
searchRequest.setPreFilterShardSize(1);
|
||||
}
|
||||
searchRequest.setPreFilterShardSize(1);
|
||||
RangeQueryBuilder rangeQueryBuilder = new RangeQueryBuilder("@timestamp").from(EARLIEST_TIMESTAMP - 2000)
|
||||
.to(EARLIEST_TIMESTAMP - 1000);
|
||||
|
||||
searchRequest.source(new SearchSourceBuilder().query(rangeQueryBuilder).size(1000));
|
||||
client(LOCAL_CLUSTER).search(searchRequest, queryFuture);
|
||||
searchRequest.source(new SearchSourceBuilder().query(rangeQueryBuilder).size(10));
|
||||
|
||||
assertBusy(() -> assertTrue(queryFuture.isDone()));
|
||||
|
||||
SearchResponse searchResponse = queryFuture.get();
|
||||
SearchResponse searchResponse = client(LOCAL_CLUSTER).search(searchRequest).get();
|
||||
assertNotNull(searchResponse);
|
||||
|
||||
SearchResponse.Clusters clusters = searchResponse.getClusters();
|
||||
|
@ -175,7 +197,12 @@ public class CrossClusterSearchIT extends AbstractMultiClustersTestCase {
|
|||
assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL));
|
||||
assertThat(localClusterSearchInfo.getTotalShards(), equalTo(localNumShards));
|
||||
assertThat(localClusterSearchInfo.getSuccessfulShards(), equalTo(localNumShards));
|
||||
if (dfs) {
|
||||
// with DFS_QUERY_THEN_FETCH, the local shards are never skipped
|
||||
assertThat(localClusterSearchInfo.getSkippedShards(), equalTo(0));
|
||||
} else {
|
||||
assertThat(localClusterSearchInfo.getSkippedShards(), equalTo(localNumShards - 1));
|
||||
}
|
||||
assertThat(localClusterSearchInfo.getFailedShards(), equalTo(0));
|
||||
assertThat(localClusterSearchInfo.getFailures().size(), equalTo(0));
|
||||
assertThat(localClusterSearchInfo.getTook().millis(), greaterThanOrEqualTo(0L));
|
||||
|
@ -183,7 +210,11 @@ public class CrossClusterSearchIT extends AbstractMultiClustersTestCase {
|
|||
assertThat(remoteClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL));
|
||||
assertThat(remoteClusterSearchInfo.getTotalShards(), equalTo(remoteNumShards));
|
||||
assertThat(remoteClusterSearchInfo.getSuccessfulShards(), equalTo(remoteNumShards));
|
||||
assertThat(remoteClusterSearchInfo.getSkippedShards(), equalTo(0));
|
||||
if (clusters.isCcsMinimizeRoundtrips()) {
|
||||
assertThat(remoteClusterSearchInfo.getSkippedShards(), equalTo(remoteNumShards - 1));
|
||||
} else {
|
||||
assertThat(remoteClusterSearchInfo.getSkippedShards(), equalTo(remoteNumShards));
|
||||
}
|
||||
assertThat(remoteClusterSearchInfo.getFailedShards(), equalTo(0));
|
||||
assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(0));
|
||||
assertThat(remoteClusterSearchInfo.getTook().millis(), greaterThanOrEqualTo(0L));
|
||||
|
@ -196,20 +227,25 @@ public class CrossClusterSearchIT extends AbstractMultiClustersTestCase {
|
|||
int localNumShards = (Integer) testClusterInfo.get("local.num_shards");
|
||||
int remoteNumShards = (Integer) testClusterInfo.get("remote.num_shards");
|
||||
|
||||
PlainActionFuture<SearchResponse> queryFuture = new PlainActionFuture<>();
|
||||
SearchRequest searchRequest = new SearchRequest(localIndex, REMOTE_CLUSTER + ":" + remoteIndex);
|
||||
searchRequest.allowPartialSearchResults(true);
|
||||
boolean minimizeRoundtrips = true; // TODO support MRT=false
|
||||
boolean minimizeRoundtrips = randomBoolean();
|
||||
searchRequest.setCcsMinimizeRoundtrips(minimizeRoundtrips);
|
||||
|
||||
if (randomBoolean()) {
|
||||
searchRequest.setBatchedReduceSize(randomIntBetween(3, 20));
|
||||
}
|
||||
boolean dfs = randomBoolean();
|
||||
if (dfs) {
|
||||
searchRequest.searchType(SearchType.DFS_QUERY_THEN_FETCH);
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
searchRequest.setPreFilterShardSize(1);
|
||||
}
|
||||
// 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);
|
||||
searchRequest.source(new SearchSourceBuilder().query(queryBuilder).size(10));
|
||||
client(LOCAL_CLUSTER).search(searchRequest, queryFuture);
|
||||
|
||||
assertBusy(() -> assertTrue(queryFuture.isDone()));
|
||||
|
||||
SearchResponse searchResponse = queryFuture.get();
|
||||
SearchResponse searchResponse = client(LOCAL_CLUSTER).search(searchRequest).get();
|
||||
assertNotNull(searchResponse);
|
||||
|
||||
SearchResponse.Clusters clusters = searchResponse.getClusters();
|
||||
|
@ -219,27 +255,11 @@ public class CrossClusterSearchIT extends AbstractMultiClustersTestCase {
|
|||
|
||||
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"));
|
||||
assertOneFailedShard(localClusterSearchInfo, localNumShards);
|
||||
|
||||
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"));
|
||||
assertOneFailedShard(remoteClusterSearchInfo, remoteNumShards);
|
||||
}
|
||||
|
||||
public void testClusterDetailsAfterCCSWithFailuresOnRemoteClusterOnly() throws Exception {
|
||||
|
@ -247,13 +267,23 @@ public class CrossClusterSearchIT extends AbstractMultiClustersTestCase {
|
|||
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");
|
||||
boolean skipUnavailable = (Boolean) testClusterInfo.get("remote.skip_unavailable");
|
||||
|
||||
PlainActionFuture<SearchResponse> queryFuture = new PlainActionFuture<>();
|
||||
SearchRequest searchRequest = new SearchRequest(localIndex, REMOTE_CLUSTER + ":" + remoteIndex);
|
||||
searchRequest.allowPartialSearchResults(true);
|
||||
boolean minimizeRoundtrips = true; // TODO support MRT=false
|
||||
searchRequest.setCcsMinimizeRoundtrips(minimizeRoundtrips);
|
||||
searchRequest.setCcsMinimizeRoundtrips(randomBoolean());
|
||||
if (randomBoolean()) {
|
||||
searchRequest.setBatchedReduceSize(randomIntBetween(3, 20));
|
||||
}
|
||||
boolean dfs = randomBoolean();
|
||||
if (dfs) {
|
||||
searchRequest.searchType(SearchType.DFS_QUERY_THEN_FETCH);
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
searchRequest.setPreFilterShardSize(1);
|
||||
}
|
||||
|
||||
// throw Exception on all shards of remoteIndex, but not against localIndex
|
||||
ThrowingQueryBuilder queryBuilder = new ThrowingQueryBuilder(
|
||||
|
@ -262,11 +292,14 @@ public class CrossClusterSearchIT extends AbstractMultiClustersTestCase {
|
|||
remoteIndex
|
||||
);
|
||||
searchRequest.source(new SearchSourceBuilder().query(queryBuilder).size(10));
|
||||
client(LOCAL_CLUSTER).search(searchRequest, queryFuture);
|
||||
|
||||
boolean minimizeRoundtrips = TransportSearchAction.shouldMinimizeRoundtrips(searchRequest);
|
||||
|
||||
client(LOCAL_CLUSTER).search(searchRequest, queryFuture);
|
||||
assertBusy(() -> assertTrue(queryFuture.isDone()));
|
||||
|
||||
if (skipUnavailable == false) {
|
||||
// dfs=true overrides the minimize_roundtrips=true setting and does not minimize roundtrips
|
||||
if (skipUnavailable == false && minimizeRoundtrips && dfs == false) {
|
||||
ExecutionException ee = expectThrows(ExecutionException.class, () -> queryFuture.get());
|
||||
assertNotNull(ee.getCause());
|
||||
assertThat(ee.getCause(), instanceOf(RemoteTransportException.class));
|
||||
|
@ -277,7 +310,9 @@ public class CrossClusterSearchIT extends AbstractMultiClustersTestCase {
|
|||
assertNotNull(searchResponse);
|
||||
|
||||
SearchResponse.Clusters clusters = searchResponse.getClusters();
|
||||
if (dfs == false) {
|
||||
assertThat(clusters.isCcsMinimizeRoundtrips(), equalTo(minimizeRoundtrips));
|
||||
}
|
||||
assertThat(clusters.getTotal(), equalTo(2));
|
||||
assertThat(clusters.getSuccessful(), equalTo(1));
|
||||
assertThat(clusters.getSkipped(), equalTo(1));
|
||||
|
@ -299,11 +334,19 @@ public class CrossClusterSearchIT extends AbstractMultiClustersTestCase {
|
|||
? SearchResponse.Cluster.Status.SKIPPED
|
||||
: SearchResponse.Cluster.Status.FAILED;
|
||||
assertThat(remoteClusterSearchInfo.getStatus(), equalTo(expectedStatus));
|
||||
if (clusters.isCcsMinimizeRoundtrips()) {
|
||||
assertNull(remoteClusterSearchInfo.getTotalShards());
|
||||
assertNull(remoteClusterSearchInfo.getSuccessfulShards());
|
||||
assertNull(remoteClusterSearchInfo.getSkippedShards());
|
||||
assertNull(remoteClusterSearchInfo.getFailedShards());
|
||||
assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(1));
|
||||
} else {
|
||||
assertThat(remoteClusterSearchInfo.getTotalShards(), equalTo(remoteNumShards));
|
||||
assertThat(remoteClusterSearchInfo.getSuccessfulShards(), equalTo(0));
|
||||
assertThat(remoteClusterSearchInfo.getSkippedShards(), equalTo(0));
|
||||
assertThat(remoteClusterSearchInfo.getFailedShards(), equalTo(remoteNumShards));
|
||||
assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(remoteNumShards));
|
||||
}
|
||||
assertNull(remoteClusterSearchInfo.getTook());
|
||||
assertFalse(remoteClusterSearchInfo.isTimedOut());
|
||||
ShardSearchFailure remoteShardSearchFailure = remoteClusterSearchInfo.getFailures().get(0);
|
||||
|
@ -311,22 +354,86 @@ public class CrossClusterSearchIT extends AbstractMultiClustersTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public void testCCSWithSearchTimeoutOnRemoteCluster() throws Exception {
|
||||
Map<String, Object> 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");
|
||||
|
||||
SearchRequest searchRequest = new SearchRequest(localIndex, REMOTE_CLUSTER + ":" + remoteIndex);
|
||||
searchRequest.allowPartialSearchResults(true);
|
||||
searchRequest.setCcsMinimizeRoundtrips(randomBoolean());
|
||||
|
||||
if (randomBoolean()) {
|
||||
searchRequest.setBatchedReduceSize(randomIntBetween(3, 20));
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
searchRequest.searchType(SearchType.DFS_QUERY_THEN_FETCH);
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
searchRequest.setPreFilterShardSize(1);
|
||||
}
|
||||
|
||||
TimeValue searchTimeout = new TimeValue(100, TimeUnit.MILLISECONDS);
|
||||
// query builder that will sleep for the specified amount of time in the query phase
|
||||
SlowRunningQueryBuilder slowRunningQueryBuilder = new SlowRunningQueryBuilder(searchTimeout.millis() * 5);
|
||||
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder().query(slowRunningQueryBuilder).timeout(searchTimeout);
|
||||
searchRequest.source(sourceBuilder);
|
||||
|
||||
SearchResponse searchResponse = client(LOCAL_CLUSTER).search(searchRequest).get();
|
||||
assertNotNull(searchResponse);
|
||||
|
||||
SearchResponse.Clusters clusters = searchResponse.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));
|
||||
assertTrue(localClusterSearchInfo.isTimedOut());
|
||||
assertThat(localClusterSearchInfo.getIndexExpression(), equalTo(localIndex));
|
||||
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(), greaterThanOrEqualTo(0L));
|
||||
|
||||
SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get();
|
||||
assertNotNull(remoteClusterSearchInfo);
|
||||
assertThat(remoteClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.PARTIAL));
|
||||
assertTrue(remoteClusterSearchInfo.isTimedOut());
|
||||
assertThat(remoteClusterSearchInfo.getIndexExpression(), equalTo(remoteIndex));
|
||||
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(), greaterThanOrEqualTo(0L));
|
||||
}
|
||||
|
||||
public void testRemoteClusterOnlyCCSSuccessfulResult() throws Exception {
|
||||
Map<String, Object> testClusterInfo = setupTwoClusters();
|
||||
String remoteIndex = (String) testClusterInfo.get("remote.index");
|
||||
int remoteNumShards = (Integer) testClusterInfo.get("remote.num_shards");
|
||||
|
||||
PlainActionFuture<SearchResponse> queryFuture = new PlainActionFuture<>();
|
||||
SearchRequest searchRequest = new SearchRequest(REMOTE_CLUSTER + ":" + remoteIndex);
|
||||
searchRequest.allowPartialSearchResults(false);
|
||||
boolean minimizeRoundtrips = true; // TODO support MRT=false
|
||||
searchRequest.setCcsMinimizeRoundtrips(minimizeRoundtrips);
|
||||
searchRequest.source(new SearchSourceBuilder().query(new MatchAllQueryBuilder()).size(1000));
|
||||
client(LOCAL_CLUSTER).search(searchRequest, queryFuture);
|
||||
searchRequest.setCcsMinimizeRoundtrips(randomBoolean());
|
||||
if (randomBoolean()) {
|
||||
searchRequest.setBatchedReduceSize(randomIntBetween(3, 20));
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
searchRequest.searchType(SearchType.DFS_QUERY_THEN_FETCH);
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
searchRequest.setPreFilterShardSize(1);
|
||||
}
|
||||
searchRequest.source(new SearchSourceBuilder().query(new MatchAllQueryBuilder()).size(10));
|
||||
|
||||
assertBusy(() -> assertTrue(queryFuture.isDone()));
|
||||
|
||||
SearchResponse searchResponse = queryFuture.get();
|
||||
SearchResponse searchResponse = client(LOCAL_CLUSTER).search(searchRequest).get();
|
||||
assertNotNull(searchResponse);
|
||||
|
||||
SearchResponse.Clusters clusters = searchResponse.getClusters();
|
||||
|
@ -353,20 +460,24 @@ public class CrossClusterSearchIT extends AbstractMultiClustersTestCase {
|
|||
String remoteIndex = (String) testClusterInfo.get("remote.index");
|
||||
int remoteNumShards = (Integer) testClusterInfo.get("remote.num_shards");
|
||||
|
||||
PlainActionFuture<SearchResponse> queryFuture = new PlainActionFuture<>();
|
||||
SearchRequest searchRequest = new SearchRequest(REMOTE_CLUSTER + ":" + remoteIndex);
|
||||
searchRequest.allowPartialSearchResults(true);
|
||||
boolean minimizeRoundtrips = true; // TODO support MRT=false
|
||||
searchRequest.setCcsMinimizeRoundtrips(minimizeRoundtrips);
|
||||
searchRequest.setCcsMinimizeRoundtrips(randomBoolean());
|
||||
if (randomBoolean()) {
|
||||
searchRequest.setBatchedReduceSize(randomIntBetween(3, 20));
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
searchRequest.searchType(SearchType.DFS_QUERY_THEN_FETCH);
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
searchRequest.setPreFilterShardSize(1);
|
||||
}
|
||||
|
||||
// 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);
|
||||
searchRequest.source(new SearchSourceBuilder().query(queryBuilder).size(10));
|
||||
client(LOCAL_CLUSTER).search(searchRequest, queryFuture);
|
||||
|
||||
assertBusy(() -> assertTrue(queryFuture.isDone()));
|
||||
|
||||
SearchResponse searchResponse = queryFuture.get();
|
||||
SearchResponse searchResponse = client(LOCAL_CLUSTER).search(searchRequest).get();
|
||||
assertNotNull(searchResponse);
|
||||
|
||||
SearchResponse.Clusters clusters = searchResponse.getClusters();
|
||||
|
@ -378,37 +489,38 @@ public class CrossClusterSearchIT extends AbstractMultiClustersTestCase {
|
|||
|
||||
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"));
|
||||
assertOneFailedShard(remoteClusterSearchInfo, remoteNumShards);
|
||||
}
|
||||
|
||||
public void testRemoteClusterOnlyCCSWithFailuresOnAllShards() throws Exception {
|
||||
Map<String, Object> testClusterInfo = setupTwoClusters();
|
||||
String remoteIndex = (String) testClusterInfo.get("remote.index");
|
||||
int remoteNumShards = (Integer) testClusterInfo.get("remote.num_shards");
|
||||
boolean skipUnavailable = (Boolean) testClusterInfo.get("remote.skip_unavailable");
|
||||
|
||||
PlainActionFuture<SearchResponse> queryFuture = new PlainActionFuture<>();
|
||||
SearchRequest searchRequest = new SearchRequest(REMOTE_CLUSTER + ":" + remoteIndex);
|
||||
searchRequest.allowPartialSearchResults(true);
|
||||
boolean minimizeRoundtrips = true; // TODO support MRT=false
|
||||
searchRequest.setCcsMinimizeRoundtrips(minimizeRoundtrips);
|
||||
searchRequest.setCcsMinimizeRoundtrips(randomBoolean());
|
||||
if (randomBoolean()) {
|
||||
searchRequest.setBatchedReduceSize(randomIntBetween(3, 20));
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
searchRequest.searchType(SearchType.DFS_QUERY_THEN_FETCH);
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
searchRequest.setPreFilterShardSize(1);
|
||||
}
|
||||
|
||||
// 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);
|
||||
searchRequest.source(new SearchSourceBuilder().query(queryBuilder).size(10));
|
||||
client(LOCAL_CLUSTER).search(searchRequest, queryFuture);
|
||||
|
||||
boolean minimizeRoundtrips = TransportSearchAction.shouldMinimizeRoundtrips(searchRequest);
|
||||
|
||||
client(LOCAL_CLUSTER).search(searchRequest, queryFuture);
|
||||
assertBusy(() -> assertTrue(queryFuture.isDone()));
|
||||
|
||||
if (skipUnavailable == false) {
|
||||
if (skipUnavailable == false || minimizeRoundtrips == false) {
|
||||
ExecutionException ee = expectThrows(ExecutionException.class, () -> queryFuture.get());
|
||||
assertNotNull(ee.getCause());
|
||||
Throwable rootCause = ExceptionsHelper.unwrap(ee, IllegalStateException.class);
|
||||
|
@ -429,19 +541,11 @@ public class CrossClusterSearchIT extends AbstractMultiClustersTestCase {
|
|||
? SearchResponse.Cluster.Status.SKIPPED
|
||||
: SearchResponse.Cluster.Status.FAILED;
|
||||
assertThat(remoteClusterSearchInfo.getStatus(), equalTo(expectedStatus));
|
||||
if (clusters.isCcsMinimizeRoundtrips()) {
|
||||
assertNull(remoteClusterSearchInfo.getTotalShards());
|
||||
assertNull(remoteClusterSearchInfo.getSuccessfulShards());
|
||||
assertNull(remoteClusterSearchInfo.getSkippedShards());
|
||||
assertNull(remoteClusterSearchInfo.getFailedShards());
|
||||
assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(1));
|
||||
} else {
|
||||
assertThat(remoteClusterSearchInfo.getTotalShards(), equalTo(remoteNumShards));
|
||||
assertThat(remoteClusterSearchInfo.getSuccessfulShards(), equalTo(0));
|
||||
assertThat(remoteClusterSearchInfo.getSkippedShards(), equalTo(0));
|
||||
assertThat(remoteClusterSearchInfo.getFailedShards(), equalTo(remoteNumShards));
|
||||
assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(remoteNumShards));
|
||||
}
|
||||
assertNull(remoteClusterSearchInfo.getTook());
|
||||
assertFalse(remoteClusterSearchInfo.isTimedOut());
|
||||
ShardSearchFailure remoteShardSearchFailure = remoteClusterSearchInfo.getFailures().get(0);
|
||||
|
@ -449,10 +553,23 @@ public class CrossClusterSearchIT extends AbstractMultiClustersTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
private static void assertOneFailedShard(SearchResponse.Cluster cluster, int totalShards) {
|
||||
assertNotNull(cluster);
|
||||
assertThat(cluster.getStatus(), equalTo(SearchResponse.Cluster.Status.PARTIAL));
|
||||
assertThat(cluster.getTotalShards(), equalTo(totalShards));
|
||||
assertThat(cluster.getSuccessfulShards(), equalTo(totalShards - 1));
|
||||
assertThat(cluster.getSkippedShards(), equalTo(0));
|
||||
assertThat(cluster.getFailedShards(), equalTo(1));
|
||||
assertThat(cluster.getFailures().size(), equalTo(1));
|
||||
assertThat(cluster.getTook().millis(), greaterThan(0L));
|
||||
ShardSearchFailure remoteShardSearchFailure = cluster.getFailures().get(0);
|
||||
assertTrue("should have 'index corrupted' in reason", remoteShardSearchFailure.reason().contains("index corrupted"));
|
||||
}
|
||||
|
||||
private Map<String, Object> setupTwoClusters() {
|
||||
String localIndex = "demo";
|
||||
int numShardsLocal = randomIntBetween(3, 6);
|
||||
Settings localSettings = indexSettings(numShardsLocal, 0).build();
|
||||
int numShardsLocal = randomIntBetween(2, 10);
|
||||
Settings localSettings = indexSettings(numShardsLocal, randomIntBetween(0, 1)).build();
|
||||
assertAcked(
|
||||
client(LOCAL_CLUSTER).admin()
|
||||
.indices()
|
||||
|
@ -463,17 +580,14 @@ public class CrossClusterSearchIT extends AbstractMultiClustersTestCase {
|
|||
indexDocs(client(LOCAL_CLUSTER), localIndex);
|
||||
|
||||
String remoteIndex = "prod";
|
||||
int numShardsRemote = randomIntBetween(3, 6);
|
||||
int numShardsRemote = randomIntBetween(2, 10);
|
||||
final InternalTestCluster remoteCluster = cluster(REMOTE_CLUSTER);
|
||||
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(remoteIndex)
|
||||
.setSettings(Settings.builder().put(remoteSettings.build()).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0))
|
||||
.setSettings(indexSettings(numShardsRemote, randomIntBetween(0, 1)))
|
||||
.setMapping("@timestamp", "type=date", "f", "type=text")
|
||||
);
|
||||
assertFalse(
|
||||
|
@ -503,7 +617,7 @@ public class CrossClusterSearchIT extends AbstractMultiClustersTestCase {
|
|||
}
|
||||
|
||||
private int indexDocs(Client client, String index) {
|
||||
int numDocs = between(50, 100);
|
||||
int numDocs = between(500, 1200);
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
long ts = EARLIEST_TIMESTAMP + i;
|
||||
if (i == numDocs - 1) {
|
||||
|
|
|
@ -181,6 +181,7 @@ public record TransportVersion(int id) implements VersionId<TransportVersion> {
|
|||
public static final TransportVersion V_8_500_065 = registerTransportVersion(8_500_065, "4e253c58-1b3d-11ee-be56-0242ac120002");
|
||||
public static final TransportVersion V_8_500_066 = registerTransportVersion(8_500_066, "F398ECC6-5D2A-4BD8-A9E8-1101F030DF85");
|
||||
|
||||
public static final TransportVersion V_8_500_067 = registerTransportVersion(8_500_067, "a7c86604-a917-4aff-9a1b-a4d44c3dbe02");
|
||||
/*
|
||||
* STOP! READ THIS FIRST! No, really,
|
||||
* ____ _____ ___ ____ _ ____ _____ _ ____ _____ _ _ ___ ____ _____ ___ ____ ____ _____ _
|
||||
|
@ -203,7 +204,7 @@ public record TransportVersion(int id) implements VersionId<TransportVersion> {
|
|||
*/
|
||||
|
||||
private static class CurrentHolder {
|
||||
private static final TransportVersion CURRENT = findCurrent(V_8_500_066);
|
||||
private static final TransportVersion CURRENT = findCurrent(V_8_500_067);
|
||||
|
||||
// finds the pluggable current version, or uses the given fallback
|
||||
private static TransportVersion findCurrent(TransportVersion fallback) {
|
||||
|
|
|
@ -177,7 +177,8 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
|
|||
SearchProgressListener.buildSearchShards(this.shardsIts),
|
||||
SearchProgressListener.buildSearchShards(toSkipShardsIts),
|
||||
clusters,
|
||||
sourceBuilder == null || sourceBuilder.size() > 0
|
||||
sourceBuilder == null || sourceBuilder.size() > 0,
|
||||
timeProvider
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,317 @@
|
|||
/*
|
||||
* 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.action.search;
|
||||
|
||||
import org.apache.lucene.search.TotalHits;
|
||||
import org.elasticsearch.common.util.CollectionUtils;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||
import org.elasticsearch.search.query.QuerySearchResult;
|
||||
import org.elasticsearch.transport.RemoteClusterAware;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
/**
|
||||
* Use this progress listener for cross-cluster searches where a single
|
||||
* coordinator is used for all clusters (minimize_roundtrips=false).
|
||||
* It updates state in the SearchResponse.Clusters object as the search
|
||||
* progresses so that the metadata required for the _clusters/details
|
||||
* section in the SearchResponse is accurate.
|
||||
*/
|
||||
public class CCSSingleCoordinatorSearchProgressListener extends SearchProgressListener {
|
||||
|
||||
private SearchResponse.Clusters clusters;
|
||||
private TransportSearchAction.SearchTimeProvider timeProvider;
|
||||
|
||||
/**
|
||||
* Executed when shards are ready to be queried (after can-match)
|
||||
*
|
||||
* @param shards The list of shards to query.
|
||||
* @param skipped The list of skipped shards.
|
||||
* @param clusters The statistics for remote clusters included in the search.
|
||||
* @param fetchPhase <code>true</code> if the search needs a fetch phase, <code>false</code> otherwise.
|
||||
**/
|
||||
@Override
|
||||
public void onListShards(
|
||||
List<SearchShard> shards,
|
||||
List<SearchShard> skipped,
|
||||
SearchResponse.Clusters clusters,
|
||||
boolean fetchPhase,
|
||||
TransportSearchAction.SearchTimeProvider timeProvider
|
||||
) {
|
||||
assert clusters.isCcsMinimizeRoundtrips() == false : "minimize_roundtrips must be false to use this SearchListener";
|
||||
|
||||
this.clusters = clusters;
|
||||
this.timeProvider = timeProvider;
|
||||
|
||||
// Partition by clusterAlias and get counts
|
||||
Map<String, Integer> skippedByClusterAlias = partitionCountsByClusterAlias(skipped);
|
||||
// the 'shards' list does not include the shards in the 'skipped' list, so combine counts from both to get total
|
||||
Map<String, Integer> totalByClusterAlias = partitionCountsByClusterAlias(shards);
|
||||
skippedByClusterAlias.forEach((cluster, count) -> totalByClusterAlias.merge(cluster, count, Integer::sum));
|
||||
|
||||
for (Map.Entry<String, Integer> entry : totalByClusterAlias.entrySet()) {
|
||||
String clusterAlias = entry.getKey();
|
||||
AtomicReference<SearchResponse.Cluster> clusterRef = clusters.getCluster(clusterAlias);
|
||||
assert clusterRef.get().getTotalShards() == null : "total shards should not be set on a Cluster before onListShards";
|
||||
|
||||
int totalCount = entry.getValue();
|
||||
int skippedCount = skippedByClusterAlias.getOrDefault(clusterAlias, 0);
|
||||
TimeValue took = null;
|
||||
|
||||
boolean swapped;
|
||||
do {
|
||||
SearchResponse.Cluster curr = clusterRef.get();
|
||||
SearchResponse.Cluster.Status status = curr.getStatus();
|
||||
assert status == SearchResponse.Cluster.Status.RUNNING : "should have RUNNING status during onListShards but has " + status;
|
||||
|
||||
// if all shards are marked as skipped, the search is done - mark as SUCCESSFUL
|
||||
if (skippedCount == totalCount) {
|
||||
took = new TimeValue(timeProvider.buildTookInMillis());
|
||||
status = SearchResponse.Cluster.Status.SUCCESSFUL;
|
||||
}
|
||||
|
||||
SearchResponse.Cluster updated = new SearchResponse.Cluster.Builder(curr).setStatus(status)
|
||||
.setTotalShards(totalCount)
|
||||
.setSuccessfulShards(skippedCount)
|
||||
.setSkippedShards(skippedCount)
|
||||
.setFailedShards(0)
|
||||
.setTook(took)
|
||||
.setTimedOut(false)
|
||||
.build();
|
||||
|
||||
swapped = clusterRef.compareAndSet(curr, updated);
|
||||
assert swapped : "compareAndSet in onListShards should never fail due to race condition";
|
||||
} while (swapped == false);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Executed when a shard returns a query result.
|
||||
*
|
||||
* @param shardIndex The index of the shard in the list provided by {@link SearchProgressListener#onListShards}.
|
||||
* @param queryResult QuerySearchResult holding the result for a SearchShardTarget
|
||||
*/
|
||||
@Override
|
||||
public void onQueryResult(int shardIndex, QuerySearchResult queryResult) {
|
||||
// we only need to update Cluster state here if the search has timed out, since:
|
||||
// 1) this is the only callback that gets search timedOut info and
|
||||
// 2) the onFinalReduce will get all these shards again so the final accounting can be done there
|
||||
// for queries that did not time out
|
||||
if (queryResult.searchTimedOut() && clusters.hasClusterObjects()) {
|
||||
SearchShardTarget shardTarget = queryResult.getSearchShardTarget();
|
||||
String clusterAlias = shardTarget.getClusterAlias();
|
||||
if (clusterAlias == null) {
|
||||
clusterAlias = RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY;
|
||||
}
|
||||
AtomicReference<SearchResponse.Cluster> clusterRef = clusters.getCluster(clusterAlias);
|
||||
boolean swapped;
|
||||
do {
|
||||
SearchResponse.Cluster curr = clusterRef.get();
|
||||
if (curr.isTimedOut()) {
|
||||
break; // cluster has already been marked as timed out on some other shard
|
||||
}
|
||||
if (curr.getStatus() == SearchResponse.Cluster.Status.FAILED || curr.getStatus() == SearchResponse.Cluster.Status.SKIPPED) {
|
||||
break; // safety check to make sure it hasn't hit a terminal FAILED/SKIPPED state where timeouts don't matter
|
||||
}
|
||||
SearchResponse.Cluster updated = new SearchResponse.Cluster.Builder(curr).setTimedOut(true).build();
|
||||
swapped = clusterRef.compareAndSet(curr, updated);
|
||||
} while (swapped == false);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Executed when a shard reports a query failure.
|
||||
*
|
||||
* @param shardIndex The index of the shard in the list provided by {@link SearchProgressListener#onListShards})}.
|
||||
* @param shardTarget The last shard target that thrown an exception.
|
||||
* @param e The cause of the failure.
|
||||
*/
|
||||
@Override
|
||||
public void onQueryFailure(int shardIndex, SearchShardTarget shardTarget, Exception e) {
|
||||
if (clusters.hasClusterObjects() == false) {
|
||||
return;
|
||||
}
|
||||
String clusterAlias = shardTarget.getClusterAlias();
|
||||
if (clusterAlias == null) {
|
||||
clusterAlias = RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY;
|
||||
}
|
||||
AtomicReference<SearchResponse.Cluster> clusterRef = clusters.getCluster(clusterAlias);
|
||||
boolean swapped;
|
||||
do {
|
||||
TimeValue took = null;
|
||||
SearchResponse.Cluster curr = clusterRef.get();
|
||||
SearchResponse.Cluster.Status status = SearchResponse.Cluster.Status.RUNNING;
|
||||
int numFailedShards = curr.getFailedShards() == null ? 1 : curr.getFailedShards() + 1;
|
||||
|
||||
assert curr.getTotalShards() != null : "total shards should be set on the Cluster but not for " + clusterAlias;
|
||||
if (curr.getTotalShards() == numFailedShards) {
|
||||
if (curr.isSkipUnavailable()) {
|
||||
status = SearchResponse.Cluster.Status.SKIPPED;
|
||||
} else {
|
||||
status = SearchResponse.Cluster.Status.FAILED;
|
||||
// TODO in the fail-fast ticket, should we throw an exception here to stop the search?
|
||||
}
|
||||
} else if (curr.getTotalShards() == numFailedShards + curr.getSuccessfulShards()) {
|
||||
status = SearchResponse.Cluster.Status.PARTIAL;
|
||||
took = new TimeValue(timeProvider.buildTookInMillis());
|
||||
}
|
||||
|
||||
// creates a new unmodifiable list
|
||||
List<ShardSearchFailure> failures = CollectionUtils.appendToCopy(curr.getFailures(), new ShardSearchFailure(e, shardTarget));
|
||||
SearchResponse.Cluster updated = new SearchResponse.Cluster.Builder(curr).setStatus(status)
|
||||
.setFailedShards(numFailedShards)
|
||||
.setFailures(failures)
|
||||
.setTook(took)
|
||||
.build();
|
||||
|
||||
swapped = clusterRef.compareAndSet(curr, updated);
|
||||
} while (swapped == false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Executed when a partial reduce is created. The number of partial reduce can be controlled via
|
||||
* {@link SearchRequest#setBatchedReduceSize(int)}.
|
||||
*
|
||||
* Note that onPartialReduce and onFinalReduce are called with cumulative data so far.
|
||||
* For example if the first call to onPartialReduce has 5 shards, the second call will
|
||||
* have those same 5 shards plus the new batch. onFinalReduce will see all those
|
||||
* shards one final time.
|
||||
*
|
||||
* @param shards The list of shards that are part of this reduce.
|
||||
* @param totalHits The total number of hits in this reduce.
|
||||
* @param aggs The partial result for aggregations.
|
||||
* @param reducePhase The version number for this reduce.
|
||||
*/
|
||||
@Override
|
||||
public void onPartialReduce(List<SearchShard> shards, TotalHits totalHits, InternalAggregations aggs, int reducePhase) {
|
||||
Map<String, Integer> totalByClusterAlias = partitionCountsByClusterAlias(shards);
|
||||
|
||||
for (Map.Entry<String, Integer> entry : totalByClusterAlias.entrySet()) {
|
||||
String clusterAlias = entry.getKey();
|
||||
int successfulCount = entry.getValue().intValue();
|
||||
|
||||
AtomicReference<SearchResponse.Cluster> clusterRef = clusters.getCluster(clusterAlias);
|
||||
boolean swapped;
|
||||
do {
|
||||
SearchResponse.Cluster curr = clusterRef.get();
|
||||
SearchResponse.Cluster.Status status = curr.getStatus();
|
||||
if (status != SearchResponse.Cluster.Status.RUNNING) {
|
||||
// don't swap in a new Cluster if the final state has already been set
|
||||
break;
|
||||
}
|
||||
TimeValue took = null;
|
||||
int successfulShards = successfulCount + curr.getSkippedShards();
|
||||
if (successfulShards == curr.getTotalShards()) {
|
||||
status = curr.isTimedOut() ? SearchResponse.Cluster.Status.PARTIAL : SearchResponse.Cluster.Status.SUCCESSFUL;
|
||||
took = new TimeValue(timeProvider.buildTookInMillis());
|
||||
} else if (successfulShards + curr.getFailedShards() == curr.getTotalShards()) {
|
||||
status = SearchResponse.Cluster.Status.PARTIAL;
|
||||
took = new TimeValue(timeProvider.buildTookInMillis());
|
||||
}
|
||||
|
||||
SearchResponse.Cluster updated = new SearchResponse.Cluster.Builder(curr).setStatus(status)
|
||||
.setSuccessfulShards(successfulShards)
|
||||
.setTook(took)
|
||||
.build();
|
||||
|
||||
swapped = clusterRef.compareAndSet(curr, updated);
|
||||
} while (swapped == false);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Executed once when the final reduce is created.
|
||||
*
|
||||
* Note that his will see all the shards, even if they have been passed to the onPartialReduce
|
||||
* method already.
|
||||
*
|
||||
* @param shards The list of shards that are part of this reduce.
|
||||
* @param totalHits The total number of hits in this reduce.
|
||||
* @param aggs The final result for aggregations.
|
||||
* @param reducePhase The version number for this reduce.
|
||||
*/
|
||||
@Override
|
||||
public void onFinalReduce(List<SearchShard> shards, TotalHits totalHits, InternalAggregations aggs, int reducePhase) {
|
||||
if (clusters.hasClusterObjects() == false) {
|
||||
return;
|
||||
}
|
||||
|
||||
Map<String, Integer> totalByClusterAlias = partitionCountsByClusterAlias(shards);
|
||||
|
||||
for (Map.Entry<String, Integer> entry : totalByClusterAlias.entrySet()) {
|
||||
String clusterAlias = entry.getKey();
|
||||
int successfulCount = entry.getValue().intValue();
|
||||
|
||||
AtomicReference<SearchResponse.Cluster> clusterRef = clusters.getCluster(clusterAlias);
|
||||
boolean swapped;
|
||||
do {
|
||||
SearchResponse.Cluster curr = clusterRef.get();
|
||||
SearchResponse.Cluster.Status status = curr.getStatus();
|
||||
if (status != SearchResponse.Cluster.Status.RUNNING) {
|
||||
// don't swap in a new Cluster if the final state has already been set
|
||||
break;
|
||||
}
|
||||
TimeValue took = new TimeValue(timeProvider.buildTookInMillis());
|
||||
int successfulShards = successfulCount + curr.getSkippedShards();
|
||||
assert successfulShards + curr.getFailedShards() == curr.getTotalShards()
|
||||
: "successfulShards("
|
||||
+ successfulShards
|
||||
+ ") + failedShards("
|
||||
+ curr.getFailedShards()
|
||||
+ ") != totalShards ("
|
||||
+ curr.getTotalShards()
|
||||
+ ')';
|
||||
if (curr.isTimedOut() || successfulShards < curr.getTotalShards()) {
|
||||
status = SearchResponse.Cluster.Status.PARTIAL;
|
||||
} else {
|
||||
assert successfulShards == curr.getTotalShards()
|
||||
: "successful (" + successfulShards + ") should equal total(" + curr.getTotalShards() + ") if get here";
|
||||
status = SearchResponse.Cluster.Status.SUCCESSFUL;
|
||||
}
|
||||
SearchResponse.Cluster updated = new SearchResponse.Cluster.Builder(curr).setStatus(status)
|
||||
.setSuccessfulShards(successfulShards)
|
||||
.setTook(took)
|
||||
.build();
|
||||
swapped = clusterRef.compareAndSet(curr, updated);
|
||||
} while (swapped == false);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Executed when a shard returns a fetch result.
|
||||
*
|
||||
* @param shardIndex The index of the shard in the list provided by {@link SearchProgressListener#onListShards})}.
|
||||
*/
|
||||
@Override
|
||||
public void onFetchResult(int shardIndex) {}
|
||||
|
||||
/**
|
||||
* Executed when a shard reports a fetch failure.
|
||||
*
|
||||
* @param shardIndex The index of the shard in the list provided by {@link SearchProgressListener#onListShards})}.
|
||||
* @param shardTarget The last shard target that thrown an exception.
|
||||
* @param exc The cause of the failure.
|
||||
*/
|
||||
@Override
|
||||
public void onFetchFailure(int shardIndex, SearchShardTarget shardTarget, Exception exc) {}
|
||||
|
||||
private Map<String, Integer> partitionCountsByClusterAlias(List<SearchShard> shards) {
|
||||
final Map<String, Integer> res = new HashMap<>();
|
||||
for (SearchShard shard : shards) {
|
||||
res.merge(Objects.requireNonNullElse(shard.clusterAlias(), RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY), 1, Integer::sum);
|
||||
}
|
||||
return res;
|
||||
}
|
||||
}
|
|
@ -113,7 +113,7 @@ public class QueryPhaseResultConsumer extends ArraySearchPhaseResults<SearchPhas
|
|||
public void consumeResult(SearchPhaseResult result, Runnable next) {
|
||||
super.consumeResult(result, () -> {});
|
||||
QuerySearchResult querySearchResult = result.queryResult();
|
||||
progressListener.notifyQueryResult(querySearchResult.getShardIndex());
|
||||
progressListener.notifyQueryResult(querySearchResult.getShardIndex(), querySearchResult);
|
||||
pendingMerges.consume(querySearchResult, next);
|
||||
}
|
||||
|
||||
|
|
|
@ -27,6 +27,7 @@ import java.util.function.BiFunction;
|
|||
final class SearchDfsQueryThenFetchAsyncAction extends AbstractSearchAsyncAction<DfsSearchResult> {
|
||||
|
||||
private final QueryPhaseResultConsumer queryPhaseResultConsumer;
|
||||
private final SearchProgressListener progressListener;
|
||||
|
||||
SearchDfsQueryThenFetchAsyncAction(
|
||||
final Logger logger,
|
||||
|
@ -63,7 +64,8 @@ final class SearchDfsQueryThenFetchAsyncAction extends AbstractSearchAsyncAction
|
|||
clusters
|
||||
);
|
||||
this.queryPhaseResultConsumer = queryPhaseResultConsumer;
|
||||
SearchProgressListener progressListener = task.getProgressListener();
|
||||
this.progressListener = task.getProgressListener();
|
||||
// don't build the SearchShard list (can be expensive) if the SearchProgressListener won't use it
|
||||
if (progressListener != SearchProgressListener.NOOP) {
|
||||
notifyListShards(progressListener, clusters, request.source());
|
||||
}
|
||||
|
@ -98,4 +100,9 @@ final class SearchDfsQueryThenFetchAsyncAction extends AbstractSearchAsyncAction
|
|||
context
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void onShardGroupFailure(int shardIndex, SearchShardTarget shardTarget, Exception exc) {
|
||||
progressListener.notifyQueryFailure(shardIndex, shardTarget, exc);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -16,6 +16,7 @@ import org.elasticsearch.cluster.routing.GroupShardsIterator;
|
|||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||
import org.elasticsearch.search.query.QuerySearchResult;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
@ -39,15 +40,23 @@ public abstract class SearchProgressListener {
|
|||
* @param skippedShards The list of skipped shards.
|
||||
* @param clusters The statistics for remote clusters included in the search.
|
||||
* @param fetchPhase <code>true</code> if the search needs a fetch phase, <code>false</code> otherwise.
|
||||
* @param timeProvider absolute and relative time provider for this search
|
||||
**/
|
||||
protected void onListShards(List<SearchShard> shards, List<SearchShard> skippedShards, Clusters clusters, boolean fetchPhase) {}
|
||||
protected void onListShards(
|
||||
List<SearchShard> shards,
|
||||
List<SearchShard> skippedShards,
|
||||
Clusters clusters,
|
||||
boolean fetchPhase,
|
||||
TransportSearchAction.SearchTimeProvider timeProvider
|
||||
) {}
|
||||
|
||||
/**
|
||||
* Executed when a shard returns a query result.
|
||||
*
|
||||
* @param shardIndex The index of the shard in the list provided by {@link SearchProgressListener#onListShards} )}.
|
||||
* @param queryResult
|
||||
*/
|
||||
protected void onQueryResult(int shardIndex) {}
|
||||
protected void onQueryResult(int shardIndex, QuerySearchResult queryResult) {}
|
||||
|
||||
/**
|
||||
* Executed when a shard reports a query failure.
|
||||
|
@ -95,18 +104,24 @@ public abstract class SearchProgressListener {
|
|||
*/
|
||||
protected void onFetchFailure(int shardIndex, SearchShardTarget shardTarget, Exception exc) {}
|
||||
|
||||
final void notifyListShards(List<SearchShard> shards, List<SearchShard> skippedShards, Clusters clusters, boolean fetchPhase) {
|
||||
final void notifyListShards(
|
||||
List<SearchShard> shards,
|
||||
List<SearchShard> skippedShards,
|
||||
Clusters clusters,
|
||||
boolean fetchPhase,
|
||||
TransportSearchAction.SearchTimeProvider timeProvider
|
||||
) {
|
||||
this.shards = shards;
|
||||
try {
|
||||
onListShards(shards, skippedShards, clusters, fetchPhase);
|
||||
onListShards(shards, skippedShards, clusters, fetchPhase, timeProvider);
|
||||
} catch (Exception e) {
|
||||
logger.warn("Failed to execute progress listener on list shards", e);
|
||||
}
|
||||
}
|
||||
|
||||
final void notifyQueryResult(int shardIndex) {
|
||||
final void notifyQueryResult(int shardIndex, QuerySearchResult queryResult) {
|
||||
try {
|
||||
onQueryResult(shardIndex);
|
||||
onQueryResult(shardIndex, queryResult);
|
||||
} catch (Exception e) {
|
||||
logger.warn(() -> "[" + shards.get(shardIndex) + "] Failed to execute progress listener on query result", e);
|
||||
}
|
||||
|
|
|
@ -77,7 +77,7 @@ class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction<SearchPh
|
|||
// register the release of the query consumer to free up the circuit breaker memory
|
||||
// at the end of the search
|
||||
addReleasable(resultConsumer);
|
||||
|
||||
// don't build the SearchShard list (can be expensive) if the SearchProgressListener won't use it
|
||||
if (progressListener != SearchProgressListener.NOOP) {
|
||||
notifyListShards(progressListener, clusters, request.source());
|
||||
}
|
||||
|
|
|
@ -474,8 +474,8 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
|
|||
// updates to the Cluster occur by CAS swapping in new Cluster objects into the AtomicReference in the map.
|
||||
private final Map<String, AtomicReference<Cluster>> 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;
|
||||
// not Writeable since it is only needed on the (primary) CCS coordinator
|
||||
private transient Boolean ccsMinimizeRoundtrips;
|
||||
|
||||
/**
|
||||
* For use with cross-cluster searches.
|
||||
|
@ -485,39 +485,44 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
|
|||
* @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
|
||||
* @param skipUnavailablePredicate given a cluster alias, returns true if that cluster is skip_unavailable=true
|
||||
* and false otherwise
|
||||
*/
|
||||
public Clusters(
|
||||
@Nullable OriginalIndices localIndices,
|
||||
Map<String, OriginalIndices> remoteClusterIndices,
|
||||
boolean ccsMinimizeRoundtrips
|
||||
boolean ccsMinimizeRoundtrips,
|
||||
Predicate<String> skipUnavailablePredicate
|
||||
) {
|
||||
assert remoteClusterIndices.size() > 0 : "At least one remote cluster must be passed into this Cluster constructor";
|
||||
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<String, AtomicReference<Cluster>> m = new HashMap<>();
|
||||
if (localIndices != null) {
|
||||
String localKey = RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY;
|
||||
Cluster c = new Cluster(localKey, String.join(",", localIndices.indices()));
|
||||
Cluster c = new Cluster(localKey, String.join(",", localIndices.indices()), false);
|
||||
m.put(localKey, new AtomicReference<>(c));
|
||||
}
|
||||
for (Map.Entry<String, OriginalIndices> remote : remoteClusterIndices.entrySet()) {
|
||||
String clusterAlias = remote.getKey();
|
||||
Cluster c = new Cluster(clusterAlias, String.join(",", remote.getValue().indices()));
|
||||
boolean skipUnavailable = skipUnavailablePredicate.test(clusterAlias);
|
||||
Cluster c = new Cluster(clusterAlias, String.join(",", remote.getValue().indices()), skipUnavailable);
|
||||
m.put(clusterAlias, new AtomicReference<>(c));
|
||||
}
|
||||
this.clusterInfo = Collections.unmodifiableMap(m);
|
||||
}
|
||||
|
||||
/**
|
||||
* Used for searches that are either not cross-cluster or CCS with minimize_roundtrips=false.
|
||||
* Used for searches that are either not cross-cluster.
|
||||
* For CCS minimize_roundtrips=true use {@code Clusters(OriginalIndices, Map<String, OriginalIndices>, 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) {
|
||||
// TODO: change assert to total == 1 or total = 0 - this should probably only be used for local searches now
|
||||
assert total >= 0 && successful >= 0 && skipped >= 0 && successful <= total
|
||||
: "total: " + total + " successful: " + successful + " skipped: " + skipped;
|
||||
assert skipped == total - successful : "total: " + total + " successful: " + successful + " skipped: " + skipped;
|
||||
|
@ -544,7 +549,6 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
|
|||
} 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;
|
||||
|
@ -690,7 +694,7 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
|
|||
/**
|
||||
* @return whether this search was a cross cluster search done with ccsMinimizeRoundtrips=true
|
||||
*/
|
||||
public boolean isCcsMinimizeRoundtrips() {
|
||||
public Boolean isCcsMinimizeRoundtrips() {
|
||||
return ccsMinimizeRoundtrips;
|
||||
}
|
||||
|
||||
|
@ -726,14 +730,19 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
|
|||
|
||||
/**
|
||||
* @return true if any underlying Cluster objects have PARTIAL, SKIPPED, FAILED or RUNNING status.
|
||||
* or any Cluster is marked as timedOut.
|
||||
*/
|
||||
public boolean hasPartialResults() {
|
||||
for (AtomicReference<Cluster> cluster : clusterInfo.values()) {
|
||||
switch (cluster.get().getStatus()) {
|
||||
for (AtomicReference<Cluster> clusterRef : clusterInfo.values()) {
|
||||
Cluster cluster = clusterRef.get();
|
||||
switch (cluster.getStatus()) {
|
||||
case PARTIAL, SKIPPED, FAILED, RUNNING -> {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
if (cluster.isTimedOut()) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
@ -746,6 +755,13 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
|
|||
return clusterInfo.keySet().size() > 0;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return true if this Clusters object has been initialized with remote Cluster objects
|
||||
* This will be false for local-cluster (non-CCS) only searches.
|
||||
*/
|
||||
public boolean hasRemoteClusters() {
|
||||
return total > 1 || clusterInfo.keySet().stream().anyMatch(alias -> alias != RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -761,8 +777,11 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
|
|||
static final ParseField INDICES_FIELD = new ParseField("indices");
|
||||
static final ParseField STATUS_FIELD = new ParseField("status");
|
||||
|
||||
private static final boolean SKIP_UNAVAILABLE_DEFAULT = false;
|
||||
|
||||
private final String clusterAlias;
|
||||
private final String indexExpression; // original index expression from the user for this cluster
|
||||
private final boolean skipUnavailable;
|
||||
private final Status status;
|
||||
private final Integer totalShards;
|
||||
private final Integer successfulShards;
|
||||
|
@ -794,9 +813,10 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
|
|||
* @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 skipUnavailable whether this Cluster is marked as skip_unavailable in remote cluster settings
|
||||
*/
|
||||
public Cluster(String clusterAlias, String indexExpression) {
|
||||
this(clusterAlias, indexExpression, Status.RUNNING, null, null, null, null, null, null, false);
|
||||
public Cluster(String clusterAlias, String indexExpression, boolean skipUnavailable) {
|
||||
this(clusterAlias, indexExpression, skipUnavailable, Status.RUNNING, null, null, null, null, null, null, false);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -806,16 +826,24 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
|
|||
* @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 skipUnavailable whether cluster is marked as skip_unavailable in remote cluster settings
|
||||
* @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<ShardSearchFailure> failures) {
|
||||
this(clusterAlias, indexExpression, status, null, null, null, null, failures, null, false);
|
||||
public Cluster(
|
||||
String clusterAlias,
|
||||
String indexExpression,
|
||||
boolean skipUnavailable,
|
||||
Status status,
|
||||
List<ShardSearchFailure> failures
|
||||
) {
|
||||
this(clusterAlias, indexExpression, skipUnavailable, status, null, null, null, null, failures, null, false);
|
||||
}
|
||||
|
||||
public Cluster(
|
||||
String clusterAlias,
|
||||
String indexExpression,
|
||||
boolean skipUnavailable,
|
||||
Status status,
|
||||
Integer totalShards,
|
||||
Integer successfulShards,
|
||||
|
@ -830,6 +858,7 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
|
|||
assert status != null : "status of Cluster cannot be null";
|
||||
this.clusterAlias = clusterAlias;
|
||||
this.indexExpression = indexExpression;
|
||||
this.skipUnavailable = skipUnavailable;
|
||||
this.status = status;
|
||||
this.totalShards = totalShards;
|
||||
this.successfulShards = successfulShards;
|
||||
|
@ -856,6 +885,97 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
|
|||
}
|
||||
this.timedOut = in.readBoolean();
|
||||
this.failures = Collections.unmodifiableList(in.readList(ShardSearchFailure::readShardSearchFailure));
|
||||
if (in.getTransportVersion().onOrAfter(TransportVersion.V_8_500_066)) {
|
||||
this.skipUnavailable = in.readBoolean();
|
||||
} else {
|
||||
this.skipUnavailable = SKIP_UNAVAILABLE_DEFAULT;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Since the Cluster object is immutable, use this Builder class to create
|
||||
* a new Cluster object using the "copyFrom" Cluster passed in and set only
|
||||
* changed values.
|
||||
*
|
||||
* Since the clusterAlias, indexExpression and skipUnavailable fields are
|
||||
* never changed once set, this Builder provides no setter method for them.
|
||||
* All other fields can be set and override the value in the "copyFrom" Cluster.
|
||||
*/
|
||||
public static class Builder {
|
||||
private Status status;
|
||||
private Integer totalShards;
|
||||
private Integer successfulShards;
|
||||
private Integer skippedShards;
|
||||
private Integer failedShards;
|
||||
private List<ShardSearchFailure> failures;
|
||||
private TimeValue took;
|
||||
private Boolean timedOut;
|
||||
private Cluster original;
|
||||
|
||||
public Builder(Cluster copyFrom) {
|
||||
this.original = copyFrom;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return new Cluster object using the new values passed in via setters
|
||||
* or the values in the "copyFrom" Cluster object set in the
|
||||
* Builder constructor.
|
||||
*/
|
||||
public Cluster build() {
|
||||
return new Cluster(
|
||||
original.getClusterAlias(),
|
||||
original.getIndexExpression(),
|
||||
original.isSkipUnavailable(),
|
||||
status != null ? status : original.getStatus(),
|
||||
totalShards != null ? totalShards : original.getTotalShards(),
|
||||
successfulShards != null ? successfulShards : original.getSuccessfulShards(),
|
||||
skippedShards != null ? skippedShards : original.getSkippedShards(),
|
||||
failedShards != null ? failedShards : original.getFailedShards(),
|
||||
failures != null ? failures : original.getFailures(),
|
||||
took != null ? took : original.getTook(),
|
||||
timedOut != null ? timedOut : original.isTimedOut()
|
||||
);
|
||||
}
|
||||
|
||||
public Builder setStatus(Status status) {
|
||||
this.status = status;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setTotalShards(int totalShards) {
|
||||
this.totalShards = totalShards;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setSuccessfulShards(int successfulShards) {
|
||||
this.successfulShards = successfulShards;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setSkippedShards(int skippedShards) {
|
||||
this.skippedShards = skippedShards;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setFailedShards(int failedShards) {
|
||||
this.failedShards = failedShards;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setFailures(List<ShardSearchFailure> failures) {
|
||||
this.failures = failures;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setTook(TimeValue took) {
|
||||
this.took = took;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setTimedOut(boolean timedOut) {
|
||||
this.timedOut = timedOut;
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -870,6 +990,9 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
|
|||
out.writeOptionalLong(took == null ? null : took.millis());
|
||||
out.writeBoolean(timedOut);
|
||||
out.writeCollection(failures);
|
||||
if (out.getTransportVersion().onOrAfter(TransportVersion.V_8_500_067)) {
|
||||
out.writeBoolean(skipUnavailable);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -985,10 +1108,12 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
|
|||
Integer skippedShardsFinal = skippedShards == -1 ? null : skippedShards;
|
||||
Integer failedShardsFinal = failedShards == -1 ? null : failedShards;
|
||||
TimeValue tookTimeValue = took == -1L ? null : new TimeValue(took);
|
||||
boolean skipUnavailable = SKIP_UNAVAILABLE_DEFAULT; // skipUnavailable is not exposed to XContent, so just use default
|
||||
|
||||
return new Cluster(
|
||||
clusterName,
|
||||
indexExpression,
|
||||
skipUnavailable,
|
||||
SearchResponse.Cluster.Status.valueOf(status.toUpperCase(Locale.ROOT)),
|
||||
totalShardsFinal,
|
||||
successfulShardsFinal,
|
||||
|
@ -1008,6 +1133,10 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
|
|||
return indexExpression;
|
||||
}
|
||||
|
||||
public boolean isSkipUnavailable() {
|
||||
return skipUnavailable;
|
||||
}
|
||||
|
||||
public Status getStatus() {
|
||||
return status;
|
||||
}
|
||||
|
@ -1043,13 +1172,11 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
|
|||
@Override
|
||||
public String toString() {
|
||||
return "Cluster{"
|
||||
+ "clusterAlias='"
|
||||
+ "alias='"
|
||||
+ clusterAlias
|
||||
+ '\''
|
||||
+ ", status="
|
||||
+ status
|
||||
+ ", failures="
|
||||
+ failures
|
||||
+ ", totalShards="
|
||||
+ totalShards
|
||||
+ ", successfulShards="
|
||||
|
@ -1058,8 +1185,17 @@ public class SearchResponse extends ActionResponse implements ChunkedToXContentO
|
|||
+ skippedShards
|
||||
+ ", failedShards="
|
||||
+ failedShards
|
||||
+ ", searchLatencyMillis="
|
||||
+ ", failures(sz)="
|
||||
+ failures.size()
|
||||
+ ", took="
|
||||
+ took
|
||||
+ ", timedOut="
|
||||
+ timedOut
|
||||
+ ", indexExpression='"
|
||||
+ indexExpression
|
||||
+ '\''
|
||||
+ ", skipUnavailable="
|
||||
+ skipUnavailable
|
||||
+ '}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -109,4 +109,9 @@ public final class SearchShardsResponse extends ActionResponse {
|
|||
assert groups.stream().noneMatch(SearchShardsGroup::preFiltered) : "legacy responses must not have preFiltered set";
|
||||
return new SearchShardsResponse(groups, Arrays.asList(oldResp.getNodes()), aliasFilters);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "SearchShardsResponse{" + "groups=" + groups + ", nodes=" + nodes + ", aliasFilters=" + aliasFilters + '}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -44,6 +44,7 @@ import org.elasticsearch.common.logging.DeprecationCategory;
|
|||
import org.elasticsearch.common.logging.DeprecationLogger;
|
||||
import org.elasticsearch.common.settings.Setting;
|
||||
import org.elasticsearch.common.settings.Setting.Property;
|
||||
import org.elasticsearch.common.util.CollectionUtils;
|
||||
import org.elasticsearch.common.util.Maps;
|
||||
import org.elasticsearch.common.util.concurrent.CountDown;
|
||||
import org.elasticsearch.core.Nullable;
|
||||
|
@ -90,7 +91,6 @@ import java.util.Set;
|
|||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.function.BiConsumer;
|
||||
import java.util.function.BiFunction;
|
||||
|
@ -256,7 +256,7 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
* to moving backwards due to NTP and other such complexities, etc.). There are also issues with
|
||||
* using a relative clock for reporting real time. Thus, we simply separate these two uses.
|
||||
*/
|
||||
record SearchTimeProvider(long absoluteStartMillis, long relativeStartNanos, LongSupplier relativeCurrentNanosProvider) {
|
||||
public record SearchTimeProvider(long absoluteStartMillis, long relativeStartNanos, LongSupplier relativeCurrentNanosProvider) {
|
||||
|
||||
/**
|
||||
* Instantiates a new search time provider. The absolute start time is the real clock time
|
||||
|
@ -269,9 +269,9 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
* @param relativeStartNanos the relative start time in nanoseconds
|
||||
* @param relativeCurrentNanosProvider provides the current relative time
|
||||
*/
|
||||
SearchTimeProvider {}
|
||||
public SearchTimeProvider {}
|
||||
|
||||
long buildTookInMillis() {
|
||||
public long buildTookInMillis() {
|
||||
return TimeUnit.NANOSECONDS.toMillis(relativeCurrentNanosProvider.getAsLong() - relativeStartNanos);
|
||||
}
|
||||
}
|
||||
|
@ -295,6 +295,7 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
);
|
||||
ActionListener<SearchRequest> rewriteListener = listener.delegateFailureAndWrap((delegate, rewritten) -> {
|
||||
final SearchContextId searchContext;
|
||||
// key to map is clusterAlias
|
||||
final Map<String, OriginalIndices> remoteClusterIndices;
|
||||
if (ccsCheckCompatibility) {
|
||||
checkCCSVersionCompatibility(rewritten);
|
||||
|
@ -327,17 +328,23 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
&& rewritten.source().aggregations() != null
|
||||
? searchService.aggReduceContextBuilder(task::isCancelled, rewritten.source().aggregations())
|
||||
: null;
|
||||
SearchResponse.Clusters initClusters = new SearchResponse.Clusters(localIndices, remoteClusterIndices, true);
|
||||
SearchResponse.Clusters clusters = new SearchResponse.Clusters(
|
||||
localIndices,
|
||||
remoteClusterIndices,
|
||||
true,
|
||||
alias -> remoteClusterService.isSkipUnavailable(alias)
|
||||
);
|
||||
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);
|
||||
task.getProgressListener()
|
||||
.notifyListShards(Collections.emptyList(), Collections.emptyList(), clusters, false, timeProvider);
|
||||
}
|
||||
ccsRemoteReduce(
|
||||
parentTaskId,
|
||||
rewritten,
|
||||
localIndices,
|
||||
remoteClusterIndices,
|
||||
initClusters,
|
||||
clusters,
|
||||
timeProvider,
|
||||
aggregationReduceContextBuilder,
|
||||
remoteClusterService,
|
||||
|
@ -349,13 +356,18 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
r,
|
||||
localIndices,
|
||||
clusterState,
|
||||
initClusters,
|
||||
clusters,
|
||||
searchContext,
|
||||
searchPhaseProvider.apply(l)
|
||||
)
|
||||
);
|
||||
} else {
|
||||
AtomicInteger skippedClusters = new AtomicInteger(0);
|
||||
SearchResponse.Clusters clusters = new SearchResponse.Clusters(
|
||||
localIndices,
|
||||
remoteClusterIndices,
|
||||
false,
|
||||
alias -> remoteClusterService.isSkipUnavailable(alias)
|
||||
);
|
||||
// TODO: pass parentTaskId
|
||||
collectSearchShards(
|
||||
rewritten.indicesOptions(),
|
||||
|
@ -364,8 +376,9 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
rewritten.source() != null ? rewritten.source().query() : null,
|
||||
Objects.requireNonNullElse(rewritten.allowPartialSearchResults(), searchService.defaultAllowPartialSearchResults()),
|
||||
searchContext,
|
||||
skippedClusters,
|
||||
remoteClusterIndices,
|
||||
clusters,
|
||||
timeProvider,
|
||||
transportService,
|
||||
delegate.delegateFailureAndWrap((finalDelegate, searchShardsResponses) -> {
|
||||
final BiFunction<String, String, DiscoveryNode> clusterNodeLookup = getRemoteClusterNodeLookup(
|
||||
|
@ -392,9 +405,6 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
remoteAliasFilters
|
||||
);
|
||||
}
|
||||
int localClusters = localIndices == null ? 0 : 1;
|
||||
int totalClusters = remoteClusterIndices.size() + localClusters;
|
||||
int successfulClusters = searchShardsResponses.size() + localClusters;
|
||||
executeSearch(
|
||||
task,
|
||||
timeProvider,
|
||||
|
@ -404,7 +414,7 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
clusterNodeLookup,
|
||||
clusterState,
|
||||
remoteAliasFilters,
|
||||
new SearchResponse.Clusters(totalClusters, successfulClusters, skippedClusters.get()),
|
||||
clusters,
|
||||
searchContext,
|
||||
searchPhaseProvider.apply(finalDelegate)
|
||||
);
|
||||
|
@ -437,7 +447,7 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
}
|
||||
}
|
||||
|
||||
static boolean shouldMinimizeRoundtrips(SearchRequest searchRequest) {
|
||||
public static boolean shouldMinimizeRoundtrips(SearchRequest searchRequest) {
|
||||
if (searchRequest.isCcsMinimizeRoundtrips() == false) {
|
||||
return false;
|
||||
}
|
||||
|
@ -551,7 +561,6 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
timeProvider,
|
||||
aggReduceContextBuilder
|
||||
);
|
||||
AtomicInteger skippedClusters = new AtomicInteger(0);
|
||||
final AtomicReference<Exception> exceptions = new AtomicReference<>();
|
||||
int totalClusters = remoteIndices.size() + (localIndices == null ? 0 : 1);
|
||||
final CountDown countDown = new CountDown(totalClusters);
|
||||
|
@ -571,7 +580,6 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
clusterAlias,
|
||||
skipUnavailable,
|
||||
countDown,
|
||||
skippedClusters,
|
||||
exceptions,
|
||||
searchResponseMerger,
|
||||
clusters,
|
||||
|
@ -589,7 +597,6 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY,
|
||||
false,
|
||||
countDown,
|
||||
skippedClusters,
|
||||
exceptions,
|
||||
searchResponseMerger,
|
||||
clusters,
|
||||
|
@ -633,6 +640,9 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
return new SearchResponseMerger(from, size, trackTotalHitsUpTo, timeProvider, aggReduceContextBuilder);
|
||||
}
|
||||
|
||||
/**
|
||||
* Used for ccs_minimize_roundtrips=false
|
||||
*/
|
||||
static void collectSearchShards(
|
||||
IndicesOptions indicesOptions,
|
||||
String preference,
|
||||
|
@ -640,8 +650,9 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
QueryBuilder query,
|
||||
boolean allowPartialResults,
|
||||
SearchContextId searchContext,
|
||||
AtomicInteger skippedClusters,
|
||||
Map<String, OriginalIndices> remoteIndicesByCluster,
|
||||
SearchResponse.Clusters clusters,
|
||||
SearchTimeProvider timeProvider,
|
||||
TransportService transportService,
|
||||
ActionListener<Map<String, SearchShardsResponse>> listener
|
||||
) {
|
||||
|
@ -657,14 +668,14 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
clusterAlias,
|
||||
skipUnavailable,
|
||||
responsesCountDown,
|
||||
skippedClusters,
|
||||
exceptions,
|
||||
null,
|
||||
clusters.getCluster(clusterAlias),
|
||||
listener
|
||||
) {
|
||||
@Override
|
||||
void innerOnResponse(SearchShardsResponse searchShardsResponse) {
|
||||
assert ThreadPool.assertCurrentThreadPool(ThreadPool.Names.SEARCH_COORDINATION);
|
||||
ccsClusterInfoUpdate(searchShardsResponse, cluster, timeProvider);
|
||||
searchShardsResponses.put(clusterAlias, searchShardsResponse);
|
||||
}
|
||||
|
||||
|
@ -698,6 +709,7 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
new ActionListenerResponseHandler<>(singleListener, SearchShardsResponse::new, responseExecutor)
|
||||
);
|
||||
} else {
|
||||
// does not do a can-match
|
||||
ClusterSearchShardsRequest searchShardsRequest = new ClusterSearchShardsRequest(indices).indicesOptions(
|
||||
indicesOptions
|
||||
).local(true).preference(preference).routing(routing);
|
||||
|
@ -725,7 +737,6 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
String clusterAlias,
|
||||
boolean skipUnavailable,
|
||||
CountDown countDown,
|
||||
AtomicInteger skippedClusters,
|
||||
AtomicReference<Exception> exceptions,
|
||||
SearchResponseMerger searchResponseMerger,
|
||||
SearchResponse.Clusters clusters,
|
||||
|
@ -735,7 +746,6 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
clusterAlias,
|
||||
skipUnavailable,
|
||||
countDown,
|
||||
skippedClusters,
|
||||
exceptions,
|
||||
clusters.getCluster(clusterAlias),
|
||||
originalListener
|
||||
|
@ -772,16 +782,9 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
boolean swapped;
|
||||
do {
|
||||
SearchResponse.Cluster orig = clusterRef.get();
|
||||
String clusterAlias = orig.getClusterAlias();
|
||||
List<ShardSearchFailure> 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);
|
||||
// returns unmodifiable list based on the original one passed plus the appended failure
|
||||
List<ShardSearchFailure> failures = CollectionUtils.appendToCopy(orig.getFailures(), failure);
|
||||
SearchResponse.Cluster updated = new SearchResponse.Cluster.Builder(orig).setStatus(status).setFailures(failures).build();
|
||||
swapped = clusterRef.compareAndSet(orig, updated);
|
||||
} while (swapped == false);
|
||||
}
|
||||
|
@ -824,22 +827,51 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
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()
|
||||
);
|
||||
SearchResponse.Cluster updated = new SearchResponse.Cluster.Builder(orig).setStatus(status)
|
||||
.setTotalShards(searchResponse.getTotalShards())
|
||||
.setSuccessfulShards(searchResponse.getSuccessfulShards())
|
||||
.setSkippedShards(searchResponse.getSkippedShards())
|
||||
.setFailedShards(searchResponse.getFailedShards())
|
||||
.setFailures(Arrays.asList(searchResponse.getShardFailures()))
|
||||
.setTook(searchResponse.getTook())
|
||||
.setTimedOut(searchResponse.isTimedOut())
|
||||
.build();
|
||||
swapped = clusterRef.compareAndSet(orig, updated);
|
||||
} while (swapped == false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Edge case ---
|
||||
* Typically we don't need to update a Cluster object after the SearchShards API call, since the
|
||||
* skipped shards will be passed into SearchProgressListener.onListShards.
|
||||
* However, there is an edge case where the remote SearchShards API call returns no shards at all.
|
||||
* So in that case, nothing for this cluster will be passed to onListShards, so we need to update
|
||||
* the Cluster object to SUCCESSFUL status with shard counts of 0 and a filled in 'took' value.
|
||||
*
|
||||
* @param response from SearchShards API call to remote cluster
|
||||
* @param clusterRef Reference Cluster to be updated
|
||||
* @param timeProvider search time provider (for setting took value)
|
||||
*/
|
||||
private static void ccsClusterInfoUpdate(
|
||||
SearchShardsResponse response,
|
||||
AtomicReference<SearchResponse.Cluster> clusterRef,
|
||||
SearchTimeProvider timeProvider
|
||||
) {
|
||||
if (response.getGroups().isEmpty()) {
|
||||
clusterRef.updateAndGet(
|
||||
orig -> new SearchResponse.Cluster.Builder(orig).setStatus(SearchResponse.Cluster.Status.SUCCESSFUL)
|
||||
.setTotalShards(0)
|
||||
.setSuccessfulShards(0)
|
||||
.setSkippedShards(0)
|
||||
.setFailedShards(0)
|
||||
.setFailures(Collections.emptyList())
|
||||
.setTook(new TimeValue(timeProvider.buildTookInMillis()))
|
||||
.setTimedOut(false)
|
||||
.build()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
void executeLocalSearch(
|
||||
Task task,
|
||||
SearchTimeProvider timeProvider,
|
||||
|
@ -1231,6 +1263,14 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
})
|
||||
);
|
||||
} else {
|
||||
// for synchronous CCS minimize_roundtrips=false, use the CCSSingleCoordinatorSearchProgressListener
|
||||
// (AsyncSearchTask will not return SearchProgressListener.NOOP, since it uses its own progress listener
|
||||
// which delegates to CCSSingleCoordinatorSearchProgressListener when minimizing roundtrips)
|
||||
if (clusters.isCcsMinimizeRoundtrips() == false
|
||||
&& clusters.hasRemoteClusters()
|
||||
&& task.getProgressListener() == SearchProgressListener.NOOP) {
|
||||
task.setProgressListener(new CCSSingleCoordinatorSearchProgressListener());
|
||||
}
|
||||
final QueryPhaseResultConsumer queryResultConsumer = searchPhaseController.newSearchPhaseResults(
|
||||
executor,
|
||||
circuitBreaker,
|
||||
|
@ -1367,7 +1407,6 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
protected final String clusterAlias;
|
||||
protected final boolean skipUnavailable;
|
||||
private final CountDown countDown;
|
||||
private final AtomicInteger skippedClusters;
|
||||
private final AtomicReference<Exception> exceptions;
|
||||
protected final AtomicReference<SearchResponse.Cluster> cluster;
|
||||
private final ActionListener<FinalResponse> originalListener;
|
||||
|
@ -1380,7 +1419,6 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
String clusterAlias,
|
||||
boolean skipUnavailable,
|
||||
CountDown countDown,
|
||||
AtomicInteger skippedClusters,
|
||||
AtomicReference<Exception> exceptions,
|
||||
@Nullable AtomicReference<SearchResponse.Cluster> cluster, // null for ccs_minimize_roundtrips=false
|
||||
ActionListener<FinalResponse> originalListener
|
||||
|
@ -1388,7 +1426,6 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
this.clusterAlias = clusterAlias;
|
||||
this.skipUnavailable = skipUnavailable;
|
||||
this.countDown = countDown;
|
||||
this.skippedClusters = skippedClusters;
|
||||
this.exceptions = exceptions;
|
||||
this.cluster = cluster;
|
||||
this.originalListener = originalListener;
|
||||
|
@ -1411,7 +1448,7 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
if (cluster != null) {
|
||||
ccsClusterInfoUpdate(f, cluster, skipUnavailable);
|
||||
}
|
||||
skippedClusters.incrementAndGet();
|
||||
// skippedClusters.incrementAndGet();
|
||||
} else {
|
||||
if (cluster != null) {
|
||||
ccsClusterInfoUpdate(f, cluster, skipUnavailable);
|
||||
|
|
|
@ -92,7 +92,13 @@ public class QueryPhaseResultConsumerTests extends ESTestCase {
|
|||
for (int i = 0; i < 10; i++) {
|
||||
searchShards.add(new SearchShard(null, new ShardId("index", "uuid", i)));
|
||||
}
|
||||
searchProgressListener.notifyListShards(searchShards, Collections.emptyList(), SearchResponse.Clusters.EMPTY, false);
|
||||
long timestamp = randomLongBetween(1000, Long.MAX_VALUE - 1000);
|
||||
TransportSearchAction.SearchTimeProvider timeProvider = new TransportSearchAction.SearchTimeProvider(
|
||||
timestamp,
|
||||
timestamp,
|
||||
() -> timestamp + 1000
|
||||
);
|
||||
searchProgressListener.notifyListShards(searchShards, Collections.emptyList(), SearchResponse.Clusters.EMPTY, false, timeProvider);
|
||||
|
||||
SearchRequest searchRequest = new SearchRequest("index");
|
||||
searchRequest.setBatchedReduceSize(2);
|
||||
|
@ -142,13 +148,14 @@ public class QueryPhaseResultConsumerTests extends ESTestCase {
|
|||
List<SearchShard> shards,
|
||||
List<SearchShard> skippedShards,
|
||||
SearchResponse.Clusters clusters,
|
||||
boolean fetchPhase
|
||||
boolean fetchPhase,
|
||||
TransportSearchAction.SearchTimeProvider timeProvider
|
||||
) {
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void onQueryResult(int shardIndex) {
|
||||
protected void onQueryResult(int shardIndex, QuerySearchResult queryResult) {
|
||||
onQueryResult.incrementAndGet();
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
|
|
@ -1128,7 +1128,7 @@ public class SearchPhaseControllerTests extends ESTestCase {
|
|||
AtomicReference<TotalHits> totalHitsListener = new AtomicReference<>();
|
||||
SearchProgressListener progressListener = new SearchProgressListener() {
|
||||
@Override
|
||||
public void onQueryResult(int shardIndex) {
|
||||
public void onQueryResult(int shardIndex, QuerySearchResult queryResult) {
|
||||
assertThat(shardIndex, lessThan(expectedNumResults));
|
||||
numQueryResultListener.incrementAndGet();
|
||||
}
|
||||
|
|
|
@ -200,7 +200,7 @@ public class SearchResponseTests extends ESTestCase {
|
|||
remoteClusterIndices.put("cluster_" + i, new OriginalIndices(new String[] { "foo", "bar*" }, IndicesOptions.lenientExpand()));
|
||||
}
|
||||
|
||||
SearchResponse.Clusters clusters = new SearchResponse.Clusters(localIndices, remoteClusterIndices, ccsMinimizeRoundtrips);
|
||||
var clusters = new SearchResponse.Clusters(localIndices, remoteClusterIndices, ccsMinimizeRoundtrips, alias -> false);
|
||||
|
||||
int successful = successfulClusters;
|
||||
int skipped = skippedClusters;
|
||||
|
@ -252,6 +252,7 @@ public class SearchResponseTests extends ESTestCase {
|
|||
SearchResponse.Cluster update = new SearchResponse.Cluster(
|
||||
cluster.getClusterAlias(),
|
||||
cluster.getIndexExpression(),
|
||||
false,
|
||||
status,
|
||||
totalShards,
|
||||
successfulShards,
|
||||
|
@ -626,4 +627,44 @@ public class SearchResponseTests extends ESTestCase {
|
|||
deserialized.getClusters().toXContent(builder, ToXContent.EMPTY_PARAMS);
|
||||
assertEquals(0, Strings.toString(builder).length());
|
||||
}
|
||||
|
||||
public void testClustersHasRemoteCluster() {
|
||||
// local cluster search Clusters objects
|
||||
assertFalse(SearchResponse.Clusters.EMPTY.hasRemoteClusters());
|
||||
assertFalse(new SearchResponse.Clusters(1, 1, 0).hasRemoteClusters());
|
||||
|
||||
// CCS search Cluster objects
|
||||
|
||||
// TODO: this variant of Clusters should not be allowed in a future ticket, but adding to test for now
|
||||
assertTrue(new SearchResponse.Clusters(3, 2, 1).hasRemoteClusters());
|
||||
{
|
||||
Map<String, OriginalIndices> remoteClusterIndices = new HashMap<>();
|
||||
remoteClusterIndices.put("remote1", new OriginalIndices(new String[] { "*" }, IndicesOptions.LENIENT_EXPAND_OPEN));
|
||||
|
||||
var c = new SearchResponse.Clusters(null, remoteClusterIndices, randomBoolean(), alias -> randomBoolean());
|
||||
assertTrue(c.hasRemoteClusters());
|
||||
}
|
||||
|
||||
{
|
||||
OriginalIndices localIndices = new OriginalIndices(new String[] { "foo*" }, IndicesOptions.LENIENT_EXPAND_OPEN);
|
||||
|
||||
Map<String, OriginalIndices> remoteClusterIndices = new HashMap<>();
|
||||
remoteClusterIndices.put("remote1", new OriginalIndices(new String[] { "*" }, IndicesOptions.LENIENT_EXPAND_OPEN));
|
||||
|
||||
var c = new SearchResponse.Clusters(localIndices, remoteClusterIndices, randomBoolean(), alias -> randomBoolean());
|
||||
assertTrue(c.hasRemoteClusters());
|
||||
}
|
||||
|
||||
{
|
||||
OriginalIndices localIndices = new OriginalIndices(new String[] { "foo*" }, IndicesOptions.LENIENT_EXPAND_OPEN);
|
||||
|
||||
Map<String, OriginalIndices> remoteClusterIndices = new HashMap<>();
|
||||
remoteClusterIndices.put("remote1", new OriginalIndices(new String[] { "*" }, IndicesOptions.LENIENT_EXPAND_OPEN));
|
||||
remoteClusterIndices.put("remote2", new OriginalIndices(new String[] { "a*" }, IndicesOptions.LENIENT_EXPAND_OPEN));
|
||||
remoteClusterIndices.put("remote3", new OriginalIndices(new String[] { "b*" }, IndicesOptions.LENIENT_EXPAND_OPEN));
|
||||
|
||||
var c = new SearchResponse.Clusters(localIndices, remoteClusterIndices, randomBoolean(), alias -> randomBoolean());
|
||||
assertTrue(c.hasRemoteClusters());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -108,7 +108,6 @@ import java.util.Set;
|
|||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.function.BiFunction;
|
||||
import java.util.function.Function;
|
||||
|
@ -523,14 +522,12 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
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,
|
||||
new SearchResponse.Clusters(localIndices, remoteIndicesByCluster, true, alias -> randomBoolean()),
|
||||
timeProvider,
|
||||
emptyReduceContextBuilder(),
|
||||
remoteClusterService,
|
||||
|
@ -591,14 +588,12 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
ActionTestUtils.assertNoFailureListener(response::set),
|
||||
latch
|
||||
);
|
||||
SearchResponse.Clusters initClusters = new SearchResponse.Clusters(localIndices, remoteIndicesByCluster, true);
|
||||
|
||||
TransportSearchAction.ccsRemoteReduce(
|
||||
new TaskId("n", 1),
|
||||
searchRequest,
|
||||
localIndices,
|
||||
remoteIndicesByCluster,
|
||||
initClusters,
|
||||
new SearchResponse.Clusters(localIndices, remoteIndicesByCluster, true, alias -> randomBoolean()),
|
||||
timeProvider,
|
||||
emptyReduceContextBuilder(),
|
||||
remoteClusterService,
|
||||
|
@ -632,13 +627,12 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
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,
|
||||
new SearchResponse.Clusters(localIndices, remoteIndicesByCluster, true, alias -> randomBoolean()),
|
||||
timeProvider,
|
||||
emptyReduceContextBuilder(),
|
||||
remoteClusterService,
|
||||
|
@ -693,14 +687,12 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
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,
|
||||
new SearchResponse.Clusters(localIndices, remoteIndicesByCluster, true, alias -> randomBoolean()),
|
||||
timeProvider,
|
||||
emptyReduceContextBuilder(),
|
||||
remoteClusterService,
|
||||
|
@ -741,14 +733,12 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
if (localIndices != null) {
|
||||
clusterAliases.add("");
|
||||
}
|
||||
SearchResponse.Clusters initClusters = new SearchResponse.Clusters(localIndices, remoteIndicesByCluster, true);
|
||||
|
||||
TransportSearchAction.ccsRemoteReduce(
|
||||
new TaskId("n", 1),
|
||||
searchRequest,
|
||||
localIndices,
|
||||
remoteIndicesByCluster,
|
||||
initClusters,
|
||||
new SearchResponse.Clusters(localIndices, remoteIndicesByCluster, true, alias -> randomBoolean()),
|
||||
timeProvider,
|
||||
emptyReduceContextBuilder(),
|
||||
remoteClusterService,
|
||||
|
@ -801,14 +791,12 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
if (localIndices != null) {
|
||||
clusterAliases.add("");
|
||||
}
|
||||
SearchResponse.Clusters initClusters = new SearchResponse.Clusters(localIndices, remoteIndicesByCluster, true);
|
||||
|
||||
TransportSearchAction.ccsRemoteReduce(
|
||||
new TaskId("n", 1),
|
||||
searchRequest,
|
||||
localIndices,
|
||||
remoteIndicesByCluster,
|
||||
initClusters,
|
||||
new SearchResponse.Clusters(localIndices, remoteIndicesByCluster, true, alias -> randomBoolean()),
|
||||
timeProvider,
|
||||
emptyReduceContextBuilder(),
|
||||
remoteClusterService,
|
||||
|
@ -859,11 +847,12 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
service.start();
|
||||
service.acceptIncomingRequests();
|
||||
|
||||
TransportSearchAction.SearchTimeProvider timeProvider = new TransportSearchAction.SearchTimeProvider(0, 0, () -> 0);
|
||||
RemoteClusterService remoteClusterService = service.getRemoteClusterService();
|
||||
{
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
AtomicReference<Map<String, SearchShardsResponse>> response = new AtomicReference<>();
|
||||
AtomicInteger skippedClusters = new AtomicInteger();
|
||||
var clusters = new SearchResponse.Clusters(null, remoteIndicesByCluster, false, clusterAlias -> true);
|
||||
TransportSearchAction.collectSearchShards(
|
||||
IndicesOptions.lenientExpandOpen(),
|
||||
null,
|
||||
|
@ -871,13 +860,13 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
new MatchAllQueryBuilder(),
|
||||
randomBoolean(),
|
||||
null,
|
||||
skippedClusters,
|
||||
remoteIndicesByCluster,
|
||||
clusters,
|
||||
timeProvider,
|
||||
service,
|
||||
new LatchedActionListener<>(ActionTestUtils.assertNoFailureListener(response::set), latch)
|
||||
);
|
||||
awaitLatch(latch, 5, TimeUnit.SECONDS);
|
||||
assertEquals(0, skippedClusters.get());
|
||||
assertNotNull(response.get());
|
||||
Map<String, SearchShardsResponse> map = response.get();
|
||||
assertEquals(numClusters, map.size());
|
||||
|
@ -887,11 +876,12 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
SearchShardsResponse shardsResponse = map.get(clusterAlias);
|
||||
assertThat(shardsResponse.getNodes(), hasSize(1));
|
||||
}
|
||||
assertThat(clusters.getSkipped(), equalTo(0));
|
||||
}
|
||||
{
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
AtomicReference<Exception> failure = new AtomicReference<>();
|
||||
AtomicInteger skippedClusters = new AtomicInteger(0);
|
||||
var clusters = new SearchResponse.Clusters(null, remoteIndicesByCluster, false, clusterAlias -> true);
|
||||
TransportSearchAction.collectSearchShards(
|
||||
IndicesOptions.lenientExpandOpen(),
|
||||
"index_not_found",
|
||||
|
@ -899,13 +889,14 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
new MatchAllQueryBuilder(),
|
||||
randomBoolean(),
|
||||
null,
|
||||
skippedClusters,
|
||||
remoteIndicesByCluster,
|
||||
clusters,
|
||||
timeProvider,
|
||||
service,
|
||||
new LatchedActionListener<>(ActionListener.wrap(r -> fail("no response expected"), failure::set), latch)
|
||||
);
|
||||
awaitLatch(latch, 5, TimeUnit.SECONDS);
|
||||
assertEquals(0, skippedClusters.get());
|
||||
assertEquals(numClusters, clusters.getSkipped());
|
||||
assertNotNull(failure.get());
|
||||
assertThat(failure.get(), instanceOf(RemoteTransportException.class));
|
||||
RemoteTransportException remoteTransportException = (RemoteTransportException) failure.get();
|
||||
|
@ -937,8 +928,8 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
|
||||
{
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
AtomicInteger skippedClusters = new AtomicInteger(0);
|
||||
AtomicReference<Exception> failure = new AtomicReference<>();
|
||||
var clusters = new SearchResponse.Clusters(null, remoteIndicesByCluster, false, clusterAlias -> false);
|
||||
TransportSearchAction.collectSearchShards(
|
||||
IndicesOptions.lenientExpandOpen(),
|
||||
null,
|
||||
|
@ -946,13 +937,14 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
new MatchAllQueryBuilder(),
|
||||
randomBoolean(),
|
||||
null,
|
||||
skippedClusters,
|
||||
remoteIndicesByCluster,
|
||||
clusters,
|
||||
timeProvider,
|
||||
service,
|
||||
new LatchedActionListener<>(ActionListener.wrap(r -> fail("no response expected"), failure::set), latch)
|
||||
);
|
||||
awaitLatch(latch, 5, TimeUnit.SECONDS);
|
||||
assertEquals(0, skippedClusters.get());
|
||||
assertEquals(numDisconnectedClusters, clusters.getSkipped());
|
||||
assertNotNull(failure.get());
|
||||
assertThat(failure.get(), instanceOf(RemoteTransportException.class));
|
||||
assertThat(failure.get().getMessage(), containsString("error while communicating with remote cluster ["));
|
||||
|
@ -966,8 +958,8 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
|
||||
{
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
AtomicInteger skippedClusters = new AtomicInteger(0);
|
||||
AtomicReference<Map<String, SearchShardsResponse>> response = new AtomicReference<>();
|
||||
var clusters = new SearchResponse.Clusters(null, remoteIndicesByCluster, false, clusterAlias -> true);
|
||||
TransportSearchAction.collectSearchShards(
|
||||
IndicesOptions.lenientExpandOpen(),
|
||||
null,
|
||||
|
@ -975,8 +967,9 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
new MatchAllQueryBuilder(),
|
||||
randomBoolean(),
|
||||
null,
|
||||
skippedClusters,
|
||||
remoteIndicesByCluster,
|
||||
clusters,
|
||||
timeProvider,
|
||||
service,
|
||||
new LatchedActionListener<>(ActionTestUtils.assertNoFailureListener(response::set), latch)
|
||||
);
|
||||
|
@ -984,7 +977,7 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
assertNotNull(response.get());
|
||||
Map<String, SearchShardsResponse> map = response.get();
|
||||
assertEquals(numClusters - disconnectedNodesIndices.size(), map.size());
|
||||
assertEquals(skippedClusters.get(), disconnectedNodesIndices.size());
|
||||
assertEquals(disconnectedNodesIndices.size(), clusters.getSkipped());
|
||||
for (int i = 0; i < numClusters; i++) {
|
||||
String clusterAlias = "remote" + i;
|
||||
if (disconnectedNodesIndices.contains(i)) {
|
||||
|
@ -1011,8 +1004,8 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
// run the following under assertBusy as connections are lazily reestablished
|
||||
assertBusy(() -> {
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
AtomicInteger skippedClusters = new AtomicInteger(0);
|
||||
AtomicReference<Map<String, SearchShardsResponse>> response = new AtomicReference<>();
|
||||
var clusters = new SearchResponse.Clusters(null, remoteIndicesByCluster, false, clusterAlias -> true);
|
||||
TransportSearchAction.collectSearchShards(
|
||||
IndicesOptions.lenientExpandOpen(),
|
||||
null,
|
||||
|
@ -1020,13 +1013,14 @@ public class TransportSearchActionTests extends ESTestCase {
|
|||
new MatchAllQueryBuilder(),
|
||||
randomBoolean(),
|
||||
null,
|
||||
skippedClusters,
|
||||
remoteIndicesByCluster,
|
||||
clusters,
|
||||
timeProvider,
|
||||
service,
|
||||
new LatchedActionListener<>(ActionTestUtils.assertNoFailureListener(response::set), latch)
|
||||
);
|
||||
awaitLatch(latch, 5, TimeUnit.SECONDS);
|
||||
assertEquals(0, skippedClusters.get());
|
||||
assertEquals(0, clusters.getSkipped());
|
||||
assertNotNull(response.get());
|
||||
Map<String, SearchShardsResponse> map = response.get();
|
||||
assertEquals(numClusters, map.size());
|
||||
|
|
|
@ -1,11 +1,12 @@
|
|||
/*
|
||||
* 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.
|
||||
* 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.xpack.search;
|
||||
package org.elasticsearch.search.query;
|
||||
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Query;
|
||||
|
@ -28,20 +29,38 @@ import java.io.IOException;
|
|||
*
|
||||
* This QueryBuilder is useful in tests that need a slow running query, such as when
|
||||
* you are trying to have a query timeout.
|
||||
*
|
||||
* The sleep can be specified to happen on all indices or only on a specified index.
|
||||
* After sleeping (if at all), it performs a MatchAll query.
|
||||
*/
|
||||
public class SlowRunningQueryBuilder extends AbstractQueryBuilder<SlowRunningQueryBuilder> {
|
||||
|
||||
public static final String NAME = "slow";
|
||||
|
||||
private long sleepTime;
|
||||
private String index;
|
||||
|
||||
/**
|
||||
* Sleep for sleepTime millis on all indexes
|
||||
* @param sleepTime
|
||||
*/
|
||||
public SlowRunningQueryBuilder(long sleepTime) {
|
||||
this.sleepTime = sleepTime;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sleep for sleepTime millis but only on the specified index
|
||||
* @param sleepTime
|
||||
*/
|
||||
public SlowRunningQueryBuilder(long sleepTime, String index) {
|
||||
this.sleepTime = sleepTime;
|
||||
this.index = index;
|
||||
}
|
||||
|
||||
public SlowRunningQueryBuilder(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
this.sleepTime = in.readLong();
|
||||
this.index = in.readOptionalString();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -57,6 +76,7 @@ public class SlowRunningQueryBuilder extends AbstractQueryBuilder<SlowRunningQue
|
|||
@Override
|
||||
protected void doWriteTo(StreamOutput out) throws IOException {
|
||||
out.writeLong(sleepTime);
|
||||
out.writeOptionalString(index);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -78,7 +98,9 @@ public class SlowRunningQueryBuilder extends AbstractQueryBuilder<SlowRunningQue
|
|||
return new Query() {
|
||||
@Override
|
||||
public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
|
||||
if (index == null || context.index().getName().equals(index)) {
|
||||
sleep();
|
||||
}
|
||||
return delegate.createWeight(searcher, scoreMode, boost);
|
||||
}
|
||||
|
|
@ -7,7 +7,6 @@
|
|||
|
||||
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;
|
||||
|
@ -16,7 +15,9 @@ import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRespo
|
|||
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.SearchType;
|
||||
import org.elasticsearch.action.search.ShardSearchFailure;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.master.AcknowledgedResponse;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetadata;
|
||||
|
@ -27,6 +28,7 @@ import org.elasticsearch.common.xcontent.ChunkedToXContent;
|
|||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.index.IndexModule;
|
||||
import org.elasticsearch.index.query.MatchAllQueryBuilder;
|
||||
import org.elasticsearch.index.query.RangeQueryBuilder;
|
||||
import org.elasticsearch.index.shard.SearchOperationListener;
|
||||
import org.elasticsearch.plugins.Plugin;
|
||||
import org.elasticsearch.plugins.SearchPlugin;
|
||||
|
@ -35,6 +37,7 @@ import org.elasticsearch.search.builder.SearchSourceBuilder;
|
|||
import org.elasticsearch.search.internal.LegacyReaderContext;
|
||||
import org.elasticsearch.search.internal.ReaderContext;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
import org.elasticsearch.search.query.SlowRunningQueryBuilder;
|
||||
import org.elasticsearch.search.query.ThrowingQueryBuilder;
|
||||
import org.elasticsearch.tasks.CancellableTask;
|
||||
import org.elasticsearch.tasks.TaskCancelledException;
|
||||
|
@ -60,7 +63,6 @@ import org.elasticsearch.xpack.core.search.action.SubmitAsyncSearchRequest;
|
|||
import org.hamcrest.Matchers;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
|
@ -74,17 +76,18 @@ import java.util.stream.Collectors;
|
|||
import java.util.stream.Stream;
|
||||
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||
import static org.hamcrest.Matchers.containsString;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.greaterThan;
|
||||
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
||||
import static org.hamcrest.Matchers.instanceOf;
|
||||
import static org.hamcrest.Matchers.matchesRegex;
|
||||
import static org.hamcrest.Matchers.not;
|
||||
|
||||
@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/98272")
|
||||
public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
||||
|
||||
private static final String REMOTE_CLUSTER = "cluster_a";
|
||||
private static final long EARLIEST_TIMESTAMP = 1691348810000L;
|
||||
private static final long LATEST_TIMESTAMP = 1691348820000L;
|
||||
|
||||
@Override
|
||||
protected Collection<String> remoteClusterAlias() {
|
||||
|
@ -143,10 +146,17 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
SearchListenerPlugin.blockQueryPhase();
|
||||
|
||||
SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(localIndex, REMOTE_CLUSTER + ":" + remoteIndex);
|
||||
request.setCcsMinimizeRoundtrips(true);
|
||||
request.setCcsMinimizeRoundtrips(randomBoolean());
|
||||
request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1));
|
||||
request.setKeepOnCompletion(true);
|
||||
request.getSearchRequest().source(new SearchSourceBuilder().query(new MatchAllQueryBuilder()).size(1000));
|
||||
request.getSearchRequest().source(new SearchSourceBuilder().query(new MatchAllQueryBuilder()).size(10));
|
||||
if (randomBoolean()) {
|
||||
request.setBatchedReduceSize(randomIntBetween(2, 256));
|
||||
}
|
||||
boolean dfs = randomBoolean();
|
||||
if (dfs) {
|
||||
request.getSearchRequest().searchType(SearchType.DFS_QUERY_THEN_FETCH);
|
||||
}
|
||||
|
||||
AsyncSearchResponse response = submitAsyncSearch(request);
|
||||
assertNotNull(response.getSearchResponse());
|
||||
|
@ -169,12 +179,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
SearchListenerPlugin.waitSearchStarted();
|
||||
SearchListenerPlugin.allowQueryPhase();
|
||||
|
||||
assertBusy(() -> {
|
||||
AsyncStatusResponse statusResponse = getAsyncStatus(response.getId());
|
||||
assertFalse(statusResponse.isRunning());
|
||||
assertNotNull(statusResponse.getCompletionStatus());
|
||||
});
|
||||
|
||||
waitForSearchTasksToFinish();
|
||||
{
|
||||
AsyncSearchResponse finishedResponse = getAsyncSearch(response.getId());
|
||||
|
||||
|
@ -237,48 +242,126 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
// CCS with a search where the timestamp of the query cannot match so should be SUCCESSFUL with all shards skipped
|
||||
// during can-match
|
||||
public void testCCSClusterDetailsWhereAllShardsSkippedInCanMatch() throws Exception {
|
||||
Map<String, Object> 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(randomBoolean());
|
||||
request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1));
|
||||
request.setKeepOnCompletion(true);
|
||||
if (randomBoolean()) {
|
||||
request.setBatchedReduceSize(randomIntBetween(2, 256));
|
||||
}
|
||||
boolean dfs = randomBoolean();
|
||||
if (dfs) {
|
||||
request.getSearchRequest().searchType(SearchType.DFS_QUERY_THEN_FETCH);
|
||||
}
|
||||
RangeQueryBuilder rangeQueryBuilder = new RangeQueryBuilder("@timestamp").from(100).to(2000);
|
||||
request.getSearchRequest().source(new SearchSourceBuilder().query(rangeQueryBuilder).size(10));
|
||||
|
||||
boolean minimizeRoundtrips = TransportSearchAction.shouldMinimizeRoundtrips(request.getSearchRequest());
|
||||
|
||||
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();
|
||||
|
||||
waitForSearchTasksToFinish();
|
||||
{
|
||||
AsyncSearchResponse finishedResponse = getAsyncSearch(response.getId());
|
||||
assertNotNull(finishedResponse);
|
||||
|
||||
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);
|
||||
SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get();
|
||||
assertNotNull(remoteClusterSearchInfo);
|
||||
|
||||
assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL));
|
||||
assertThat(localClusterSearchInfo.getTotalShards(), equalTo(localNumShards));
|
||||
assertThat(localClusterSearchInfo.getSuccessfulShards(), equalTo(localNumShards));
|
||||
if (dfs) {
|
||||
// no skipped shards locally when DFS_QUERY_THEN_FETCH is used
|
||||
assertThat(localClusterSearchInfo.getSkippedShards(), equalTo(0));
|
||||
} else {
|
||||
assertThat(localClusterSearchInfo.getSkippedShards(), equalTo(localNumShards - 1));
|
||||
}
|
||||
assertThat(localClusterSearchInfo.getFailedShards(), equalTo(0));
|
||||
assertThat(localClusterSearchInfo.getFailures().size(), equalTo(0));
|
||||
assertThat(localClusterSearchInfo.getTook().millis(), greaterThanOrEqualTo(0L));
|
||||
|
||||
assertThat(remoteClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.SUCCESSFUL));
|
||||
assertThat(remoteClusterSearchInfo.getTotalShards(), equalTo(remoteNumShards));
|
||||
assertThat(remoteClusterSearchInfo.getSuccessfulShards(), equalTo(remoteNumShards));
|
||||
if (minimizeRoundtrips) {
|
||||
assertThat(remoteClusterSearchInfo.getSkippedShards(), equalTo(remoteNumShards - 1));
|
||||
} else {
|
||||
assertThat(remoteClusterSearchInfo.getSkippedShards(), equalTo(remoteNumShards));
|
||||
}
|
||||
assertThat(remoteClusterSearchInfo.getFailedShards(), equalTo(0));
|
||||
assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(0));
|
||||
assertThat(remoteClusterSearchInfo.getTook().millis(), greaterThanOrEqualTo(0L));
|
||||
}
|
||||
}
|
||||
|
||||
public void testClusterDetailsAfterCCSWithFailuresOnAllShards() throws Exception {
|
||||
Map<String, Object> 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");
|
||||
boolean skipUnavailable = (Boolean) testClusterInfo.get("remote.skip_unavailable");
|
||||
|
||||
SearchListenerPlugin.blockQueryPhase();
|
||||
|
||||
SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(localIndex, REMOTE_CLUSTER + ":" + remoteIndex);
|
||||
request.setCcsMinimizeRoundtrips(true);
|
||||
request.setCcsMinimizeRoundtrips(randomBoolean());
|
||||
request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1));
|
||||
if (randomBoolean()) {
|
||||
request.setBatchedReduceSize(randomIntBetween(2, 256));
|
||||
}
|
||||
request.setKeepOnCompletion(true);
|
||||
boolean dfs = randomBoolean();
|
||||
if (dfs) {
|
||||
request.getSearchRequest().searchType(SearchType.DFS_QUERY_THEN_FETCH);
|
||||
}
|
||||
// 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));
|
||||
|
||||
boolean minimizeRoundtrips = TransportSearchAction.shouldMinimizeRoundtrips(request.getSearchRequest());
|
||||
|
||||
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());
|
||||
});
|
||||
waitForSearchTasksToFinish();
|
||||
|
||||
{
|
||||
AsyncSearchResponse finishedResponse = getAsyncSearch(response.getId());
|
||||
|
@ -291,15 +374,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
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"));
|
||||
assertAllShardsFailed(minimizeRoundtrips, localClusterSearchInfo, localNumShards);
|
||||
|
||||
SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get();
|
||||
assertNotNull(remoteClusterSearchInfo);
|
||||
|
@ -307,15 +382,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
? 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"));
|
||||
assertAllShardsFailed(minimizeRoundtrips, remoteClusterSearchInfo, remoteNumShards);
|
||||
}
|
||||
// check that the async_search/status response includes the same cluster details
|
||||
{
|
||||
|
@ -328,15 +395,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
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"));
|
||||
assertAllShardsFailed(minimizeRoundtrips, localClusterSearchInfo, localNumShards);
|
||||
|
||||
SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get();
|
||||
assertNotNull(remoteClusterSearchInfo);
|
||||
|
@ -344,15 +403,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
? 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"));
|
||||
assertAllShardsFailed(minimizeRoundtrips, remoteClusterSearchInfo, remoteNumShards);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -366,9 +417,16 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
SearchListenerPlugin.blockQueryPhase();
|
||||
|
||||
SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(localIndex, REMOTE_CLUSTER + ":" + remoteIndex);
|
||||
request.setCcsMinimizeRoundtrips(true);
|
||||
request.setCcsMinimizeRoundtrips(randomBoolean());
|
||||
request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1));
|
||||
request.setKeepOnCompletion(true);
|
||||
if (randomBoolean()) {
|
||||
request.setBatchedReduceSize(randomIntBetween(2, 256));
|
||||
}
|
||||
boolean dfs = randomBoolean();
|
||||
if (dfs) {
|
||||
request.getSearchRequest().searchType(SearchType.DFS_QUERY_THEN_FETCH);
|
||||
}
|
||||
// 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));
|
||||
|
@ -394,15 +452,10 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
SearchListenerPlugin.waitSearchStarted();
|
||||
SearchListenerPlugin.allowQueryPhase();
|
||||
|
||||
assertBusy(() -> {
|
||||
AsyncStatusResponse statusResponse = getAsyncStatus(response.getId());
|
||||
assertFalse(statusResponse.isRunning());
|
||||
assertNotNull(statusResponse.getCompletionStatus());
|
||||
});
|
||||
waitForSearchTasksToFinish();
|
||||
|
||||
{
|
||||
AsyncSearchResponse finishedResponse = getAsyncSearch(response.getId());
|
||||
|
||||
SearchResponse.Clusters clusters = finishedResponse.getSearchResponse().getClusters();
|
||||
assertThat(clusters.getTotal(), equalTo(2));
|
||||
assertThat(clusters.getSuccessful(), equalTo(2));
|
||||
|
@ -471,14 +524,23 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
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");
|
||||
boolean skipUnavailable = (Boolean) testClusterInfo.get("remote.skip_unavailable");
|
||||
|
||||
SearchListenerPlugin.blockQueryPhase();
|
||||
|
||||
SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(localIndex, REMOTE_CLUSTER + ":" + remoteIndex);
|
||||
request.setCcsMinimizeRoundtrips(true);
|
||||
request.setCcsMinimizeRoundtrips(randomBoolean());
|
||||
request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1));
|
||||
request.setKeepOnCompletion(true);
|
||||
if (randomBoolean()) {
|
||||
request.setBatchedReduceSize(randomIntBetween(2, 256));
|
||||
}
|
||||
boolean dfs = randomBoolean();
|
||||
if (dfs) {
|
||||
request.getSearchRequest().searchType(SearchType.DFS_QUERY_THEN_FETCH);
|
||||
}
|
||||
|
||||
// throw Exception of all shards of remoteIndex, but against localIndex
|
||||
ThrowingQueryBuilder queryBuilder = new ThrowingQueryBuilder(
|
||||
randomLong(),
|
||||
|
@ -487,6 +549,8 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
);
|
||||
request.getSearchRequest().source(new SearchSourceBuilder().query(queryBuilder).size(10));
|
||||
|
||||
boolean minimizeRoundtrips = TransportSearchAction.shouldMinimizeRoundtrips(request.getSearchRequest());
|
||||
|
||||
AsyncSearchResponse response = submitAsyncSearch(request);
|
||||
assertNotNull(response.getSearchResponse());
|
||||
assertTrue(response.isRunning());
|
||||
|
@ -507,11 +571,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
SearchListenerPlugin.waitSearchStarted();
|
||||
SearchListenerPlugin.allowQueryPhase();
|
||||
|
||||
assertBusy(() -> {
|
||||
AsyncStatusResponse statusResponse = getAsyncStatus(response.getId());
|
||||
assertFalse(statusResponse.isRunning());
|
||||
assertNotNull(statusResponse.getCompletionStatus());
|
||||
});
|
||||
waitForSearchTasksToFinish();
|
||||
|
||||
{
|
||||
AsyncSearchResponse finishedResponse = getAsyncSearch(response.getId());
|
||||
|
@ -537,11 +597,19 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
? SearchResponse.Cluster.Status.SKIPPED
|
||||
: SearchResponse.Cluster.Status.FAILED;
|
||||
assertThat(remoteClusterSearchInfo.getStatus(), equalTo(expectedStatus));
|
||||
if (minimizeRoundtrips) {
|
||||
assertNull(remoteClusterSearchInfo.getTotalShards());
|
||||
assertNull(remoteClusterSearchInfo.getSuccessfulShards());
|
||||
assertNull(remoteClusterSearchInfo.getSkippedShards());
|
||||
assertNull(remoteClusterSearchInfo.getFailedShards());
|
||||
assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(1));
|
||||
} else {
|
||||
assertThat(remoteClusterSearchInfo.getTotalShards(), equalTo(remoteNumShards));
|
||||
assertThat(remoteClusterSearchInfo.getSuccessfulShards(), equalTo(0));
|
||||
assertThat(remoteClusterSearchInfo.getSkippedShards(), equalTo(0));
|
||||
assertThat(remoteClusterSearchInfo.getFailedShards(), equalTo(remoteNumShards));
|
||||
assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(remoteNumShards));
|
||||
}
|
||||
assertNull(remoteClusterSearchInfo.getTook());
|
||||
assertFalse(remoteClusterSearchInfo.isTimedOut());
|
||||
ShardSearchFailure remoteShardSearchFailure = remoteClusterSearchInfo.getFailures().get(0);
|
||||
|
@ -571,11 +639,15 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
? SearchResponse.Cluster.Status.SKIPPED
|
||||
: SearchResponse.Cluster.Status.FAILED;
|
||||
assertThat(remoteClusterSearchInfo.getStatus(), equalTo(expectedStatus));
|
||||
if (minimizeRoundtrips) {
|
||||
assertNull(remoteClusterSearchInfo.getTotalShards());
|
||||
assertNull(remoteClusterSearchInfo.getSuccessfulShards());
|
||||
assertNull(remoteClusterSearchInfo.getSkippedShards());
|
||||
assertNull(remoteClusterSearchInfo.getFailedShards());
|
||||
assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(1));
|
||||
} else {
|
||||
assertThat(remoteClusterSearchInfo.getTotalShards(), equalTo(remoteNumShards));
|
||||
assertThat(remoteClusterSearchInfo.getFailedShards(), equalTo(remoteNumShards));
|
||||
assertThat(remoteClusterSearchInfo.getFailures().size(), equalTo(remoteNumShards));
|
||||
}
|
||||
assertNull(remoteClusterSearchInfo.getTook());
|
||||
assertFalse(remoteClusterSearchInfo.isTimedOut());
|
||||
ShardSearchFailure remoteShardSearchFailure = remoteClusterSearchInfo.getFailures().get(0);
|
||||
|
@ -583,6 +655,105 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public void testCCSWithSearchTimeout() throws Exception {
|
||||
Map<String, Object> 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");
|
||||
|
||||
TimeValue searchTimeout = new TimeValue(100, TimeUnit.MILLISECONDS);
|
||||
// query builder that will sleep for the specified amount of time in the query phase
|
||||
SlowRunningQueryBuilder slowRunningQueryBuilder = new SlowRunningQueryBuilder(searchTimeout.millis() * 5);
|
||||
SearchSourceBuilder sourceBuilder = new SearchSourceBuilder().query(slowRunningQueryBuilder).timeout(searchTimeout);
|
||||
|
||||
SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(localIndex, REMOTE_CLUSTER + ":" + remoteIndex);
|
||||
request.setCcsMinimizeRoundtrips(randomBoolean());
|
||||
request.getSearchRequest().source(sourceBuilder);
|
||||
if (randomBoolean()) {
|
||||
request.setBatchedReduceSize(randomIntBetween(2, 256));
|
||||
}
|
||||
request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1));
|
||||
request.getSearchRequest().allowPartialSearchResults(true);
|
||||
request.setKeepOnCompletion(true);
|
||||
boolean dfs = randomBoolean();
|
||||
if (dfs) {
|
||||
request.getSearchRequest().searchType(SearchType.DFS_QUERY_THEN_FETCH);
|
||||
}
|
||||
|
||||
AsyncSearchResponse response = submitAsyncSearch(request);
|
||||
assertNotNull(response.getSearchResponse());
|
||||
|
||||
waitForSearchTasksToFinish();
|
||||
|
||||
{
|
||||
AsyncSearchResponse finishedResponse = getAsyncSearch(response.getId());
|
||||
assertTrue(finishedResponse.getSearchResponse().isTimedOut());
|
||||
|
||||
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);
|
||||
// PARTIAL expected since timedOut=true
|
||||
assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.PARTIAL));
|
||||
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(), greaterThanOrEqualTo(0L));
|
||||
assertTrue(localClusterSearchInfo.isTimedOut());
|
||||
|
||||
SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get();
|
||||
assertNotNull(remoteClusterSearchInfo);
|
||||
// PARTIAL expected since timedOut=true
|
||||
assertThat(remoteClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.PARTIAL));
|
||||
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(), greaterThanOrEqualTo(0L));
|
||||
assertTrue(remoteClusterSearchInfo.isTimedOut());
|
||||
}
|
||||
// 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);
|
||||
// PARTIAL expected since timedOut=true
|
||||
assertThat(localClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.PARTIAL));
|
||||
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(), greaterThanOrEqualTo(0L));
|
||||
assertTrue(localClusterSearchInfo.isTimedOut());
|
||||
|
||||
SearchResponse.Cluster remoteClusterSearchInfo = clusters.getCluster(REMOTE_CLUSTER).get();
|
||||
assertNotNull(remoteClusterSearchInfo);
|
||||
// PARTIAL expected since timedOut=true
|
||||
assertThat(remoteClusterSearchInfo.getStatus(), equalTo(SearchResponse.Cluster.Status.PARTIAL));
|
||||
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(localClusterSearchInfo.getTook().millis(), greaterThanOrEqualTo(0L));
|
||||
assertTrue(remoteClusterSearchInfo.isTimedOut());
|
||||
}
|
||||
}
|
||||
|
||||
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
|
||||
|
@ -594,19 +765,22 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
|
||||
// search only the remote cluster
|
||||
SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(REMOTE_CLUSTER + ":" + remoteIndex);
|
||||
request.setCcsMinimizeRoundtrips(true);
|
||||
request.setCcsMinimizeRoundtrips(randomBoolean());
|
||||
request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1));
|
||||
request.setKeepOnCompletion(true);
|
||||
request.getSearchRequest().source(new SearchSourceBuilder().query(new MatchAllQueryBuilder()).size(1000));
|
||||
if (randomBoolean()) {
|
||||
request.setBatchedReduceSize(randomIntBetween(2, 256));
|
||||
}
|
||||
boolean dfs = randomBoolean();
|
||||
if (dfs) {
|
||||
request.getSearchRequest().searchType(SearchType.DFS_QUERY_THEN_FETCH);
|
||||
}
|
||||
request.getSearchRequest().source(new SearchSourceBuilder().query(new MatchAllQueryBuilder()).size(10));
|
||||
|
||||
AsyncSearchResponse response = submitAsyncSearch(request);
|
||||
assertNotNull(response.getSearchResponse());
|
||||
|
||||
assertBusy(() -> {
|
||||
AsyncStatusResponse statusResponse = getAsyncStatus(response.getId());
|
||||
assertFalse(statusResponse.isRunning());
|
||||
assertNotNull(statusResponse.getCompletionStatus());
|
||||
});
|
||||
waitForSearchTasksToFinish();
|
||||
|
||||
{
|
||||
AsyncSearchResponse finishedResponse = getAsyncSearch(response.getId());
|
||||
|
@ -664,22 +838,24 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
int remoteNumShards = (Integer) testClusterInfo.get("remote.num_shards");
|
||||
|
||||
SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(REMOTE_CLUSTER + ":" + remoteIndex);
|
||||
request.setCcsMinimizeRoundtrips(true);
|
||||
request.setCcsMinimizeRoundtrips(randomBoolean());
|
||||
request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1));
|
||||
request.setKeepOnCompletion(true);
|
||||
if (randomBoolean()) {
|
||||
request.setBatchedReduceSize(randomIntBetween(2, 256));
|
||||
}
|
||||
boolean dfs = randomBoolean();
|
||||
if (dfs) {
|
||||
request.getSearchRequest().searchType(SearchType.DFS_QUERY_THEN_FETCH);
|
||||
}
|
||||
// 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());
|
||||
});
|
||||
waitForSearchTasksToFinish();
|
||||
|
||||
{
|
||||
AsyncSearchResponse finishedResponse = getAsyncSearch(response.getId());
|
||||
|
@ -734,26 +910,31 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
|
||||
Map<String, Object> testClusterInfo = setupTwoClusters();
|
||||
String remoteIndex = (String) testClusterInfo.get("remote.index");
|
||||
int remoteNumShards = (Integer) testClusterInfo.get("remote.num_shards");
|
||||
boolean skipUnavailable = (Boolean) testClusterInfo.get("remote.skip_unavailable");
|
||||
|
||||
SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(REMOTE_CLUSTER + ":" + remoteIndex);
|
||||
request.setCcsMinimizeRoundtrips(true);
|
||||
request.setCcsMinimizeRoundtrips(randomBoolean());
|
||||
request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1));
|
||||
request.setKeepOnCompletion(true);
|
||||
if (randomBoolean()) {
|
||||
request.setBatchedReduceSize(randomIntBetween(2, 256));
|
||||
}
|
||||
boolean dfs = randomBoolean();
|
||||
if (dfs) {
|
||||
request.getSearchRequest().searchType(SearchType.DFS_QUERY_THEN_FETCH);
|
||||
}
|
||||
|
||||
// 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));
|
||||
|
||||
boolean minimizeRoundtrips = TransportSearchAction.shouldMinimizeRoundtrips(request.getSearchRequest());
|
||||
|
||||
AsyncSearchResponse response = submitAsyncSearch(request);
|
||||
assertNotNull(response.getSearchResponse());
|
||||
assertTrue(response.isRunning());
|
||||
|
||||
assertBusy(() -> {
|
||||
AsyncStatusResponse statusResponse = getAsyncStatus(response.getId());
|
||||
assertFalse(statusResponse.isRunning());
|
||||
assertNotNull(statusResponse.getCompletionStatus());
|
||||
});
|
||||
|
||||
waitForSearchTasksToFinish();
|
||||
{
|
||||
AsyncSearchResponse finishedResponse = getAsyncSearch(response.getId());
|
||||
|
||||
|
@ -770,15 +951,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
? 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"));
|
||||
assertAllShardsFailed(minimizeRoundtrips, remoteClusterSearchInfo, remoteNumShards);
|
||||
}
|
||||
// check that the async_search/status response includes the same cluster details
|
||||
{
|
||||
|
@ -796,15 +969,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
? 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"));
|
||||
assertAllShardsFailed(minimizeRoundtrips, remoteClusterSearchInfo, remoteNumShards);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -819,8 +984,11 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
request.setCcsMinimizeRoundtrips(randomBoolean());
|
||||
request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1));
|
||||
request.setKeepOnCompletion(true);
|
||||
if (randomBoolean()) {
|
||||
request.setBatchedReduceSize(randomIntBetween(2, 256));
|
||||
}
|
||||
request.getSearchRequest().allowPartialSearchResults(false);
|
||||
request.getSearchRequest().source(new SearchSourceBuilder().query(new MatchAllQueryBuilder()).size(1000));
|
||||
request.getSearchRequest().source(new SearchSourceBuilder().query(new MatchAllQueryBuilder()).size(10));
|
||||
|
||||
AsyncSearchResponse response = submitAsyncSearch(request);
|
||||
assertNotNull(response.getSearchResponse());
|
||||
|
@ -904,18 +1072,8 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
}
|
||||
|
||||
assertBusy(() -> assertTrue(cancelFuture.isDone()));
|
||||
assertBusy(() -> {
|
||||
final Iterable<TransportService> transportServices = cluster(REMOTE_CLUSTER).getInstances(TransportService.class);
|
||||
for (TransportService transportService : transportServices) {
|
||||
assertThat(transportService.getTaskManager().getBannedTaskIds(), Matchers.empty());
|
||||
}
|
||||
});
|
||||
|
||||
// wait until search status endpoint reports it as completed
|
||||
assertBusy(() -> {
|
||||
AsyncStatusResponse statusResponseAfterCompletion = getAsyncStatus(response.getId());
|
||||
assertNotNull(statusResponseAfterCompletion.getCompletionStatus());
|
||||
});
|
||||
waitForSearchTasksToFinish();
|
||||
|
||||
AsyncStatusResponse statusResponseAfterCompletion = getAsyncStatus(response.getId());
|
||||
assertTrue(statusResponseAfterCompletion.isPartial());
|
||||
|
@ -938,7 +1096,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
ChunkedToXContent.wrapAsToXContent(searchResponseAfterCompletion)
|
||||
.toXContent(XContentFactory.jsonBuilder(), ToXContent.EMPTY_PARAMS)
|
||||
);
|
||||
assertThat(json, containsString("task cancelled [by user request]"));
|
||||
assertThat(json, matchesRegex(".*task (was)?\s*cancelled.*"));
|
||||
}
|
||||
|
||||
public void testCancelViaAsyncSearchDelete() throws Exception {
|
||||
|
@ -952,8 +1110,11 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
request.setCcsMinimizeRoundtrips(randomBoolean());
|
||||
request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1));
|
||||
request.setKeepOnCompletion(true);
|
||||
if (randomBoolean()) {
|
||||
request.setBatchedReduceSize(randomIntBetween(2, 256));
|
||||
}
|
||||
request.getSearchRequest().allowPartialSearchResults(false);
|
||||
request.getSearchRequest().source(new SearchSourceBuilder().query(new MatchAllQueryBuilder()).size(1000));
|
||||
request.getSearchRequest().source(new SearchSourceBuilder().query(new MatchAllQueryBuilder()).size(10));
|
||||
|
||||
AsyncSearchResponse response = submitAsyncSearch(request);
|
||||
assertNotNull(response.getSearchResponse());
|
||||
|
@ -1031,16 +1192,11 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
SearchListenerPlugin.allowQueryPhase();
|
||||
}
|
||||
|
||||
waitForSearchTasksToFinish();
|
||||
|
||||
assertBusy(() -> expectThrows(ExecutionException.class, () -> getAsyncStatus(response.getId())));
|
||||
assertBusy(() -> {
|
||||
final Iterable<TransportService> transportServices = cluster(REMOTE_CLUSTER).getInstances(TransportService.class);
|
||||
for (TransportService transportService : transportServices) {
|
||||
assertThat(transportService.getTaskManager().getBannedTaskIds(), Matchers.empty());
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/97286")
|
||||
public void testCancellationViaTimeoutWithAllowPartialResultsSetToFalse() throws Exception {
|
||||
Map<String, Object> testClusterInfo = setupTwoClusters();
|
||||
String localIndex = (String) testClusterInfo.get("local.index");
|
||||
|
@ -1056,6 +1212,9 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(localIndex, REMOTE_CLUSTER + ":" + remoteIndex);
|
||||
request.setCcsMinimizeRoundtrips(randomBoolean());
|
||||
request.getSearchRequest().source(sourceBuilder);
|
||||
if (randomBoolean()) {
|
||||
request.setBatchedReduceSize(randomIntBetween(2, 256));
|
||||
}
|
||||
request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1));
|
||||
request.getSearchRequest().allowPartialSearchResults(false);
|
||||
request.setWaitForCompletionTimeout(TimeValue.timeValueMillis(1));
|
||||
|
@ -1102,26 +1261,24 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
// query phase has begun, so wait for query failure (due to timeout)
|
||||
SearchListenerPlugin.waitQueryFailure();
|
||||
|
||||
// wait for the async_search task to be cancelled or unregistered
|
||||
// wait for search tasks to complete and be unregistered
|
||||
assertBusy(() -> {
|
||||
ListTasksResponse taskResponses = client().admin().cluster().prepareListTasks().setDetailed(true).get();
|
||||
List<TaskInfo> asyncSearchTaskInfos = new ArrayList<>();
|
||||
for (TaskInfo task : taskResponses.getTasks()) {
|
||||
if (task.action().contains("search")) {
|
||||
if (task.description().contains("async_search{indices[")) {
|
||||
asyncSearchTaskInfos.add(task);
|
||||
}
|
||||
}
|
||||
}
|
||||
ListTasksResponse listTasksResponse = client(LOCAL_CLUSTER).admin()
|
||||
.cluster()
|
||||
.prepareListTasks()
|
||||
.setActions(SearchAction.INSTANCE.name())
|
||||
.get();
|
||||
List<TaskInfo> tasks = listTasksResponse.getTasks();
|
||||
assertThat(tasks.size(), equalTo(0));
|
||||
|
||||
if (asyncSearchTaskInfos.size() > 0) {
|
||||
// if still present, and it is cancelled, then we can proceed with the test
|
||||
assertTrue(asyncSearchTaskInfos.get(0).cancelled());
|
||||
}
|
||||
// if not present, then it has been unregistered and the async search should no longer be running, so can proceed
|
||||
}, 30, TimeUnit.SECONDS);
|
||||
|
||||
assertBusy(() -> { assertFalse(getAsyncStatus(response.getId()).isRunning()); });
|
||||
ListTasksResponse remoteTasksResponse = client(REMOTE_CLUSTER).admin()
|
||||
.cluster()
|
||||
.prepareListTasks()
|
||||
.setActions(SearchAction.INSTANCE.name())
|
||||
.get();
|
||||
List<TaskInfo> remoteTasks = remoteTasksResponse.getTasks();
|
||||
assertThat(remoteTasks.size(), equalTo(0));
|
||||
});
|
||||
|
||||
AsyncStatusResponse statusResponse = getAsyncStatus(response.getId());
|
||||
assertFalse(statusResponse.isRunning());
|
||||
|
@ -1129,6 +1286,28 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
assertEquals(0, statusResponse.getSkippedShards());
|
||||
assertThat(statusResponse.getFailedShards(), greaterThanOrEqualTo(1));
|
||||
|
||||
waitForSearchTasksToFinish();
|
||||
}
|
||||
|
||||
private void waitForSearchTasksToFinish() throws Exception {
|
||||
assertBusy(() -> {
|
||||
ListTasksResponse listTasksResponse = client(LOCAL_CLUSTER).admin()
|
||||
.cluster()
|
||||
.prepareListTasks()
|
||||
.setActions(SearchAction.INSTANCE.name())
|
||||
.get();
|
||||
List<TaskInfo> tasks = listTasksResponse.getTasks();
|
||||
assertThat(tasks.size(), equalTo(0));
|
||||
|
||||
ListTasksResponse remoteTasksResponse = client(REMOTE_CLUSTER).admin()
|
||||
.cluster()
|
||||
.prepareListTasks()
|
||||
.setActions(SearchAction.INSTANCE.name())
|
||||
.get();
|
||||
List<TaskInfo> remoteTasks = remoteTasksResponse.getTasks();
|
||||
assertThat(remoteTasks.size(), equalTo(0));
|
||||
});
|
||||
|
||||
assertBusy(() -> {
|
||||
final Iterable<TransportService> transportServices = cluster(REMOTE_CLUSTER).getInstances(TransportService.class);
|
||||
for (TransportService transportService : transportServices) {
|
||||
|
@ -1137,6 +1316,26 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
});
|
||||
}
|
||||
|
||||
private static void assertAllShardsFailed(boolean minimizeRoundtrips, SearchResponse.Cluster cluster, int numShards) {
|
||||
if (minimizeRoundtrips) {
|
||||
assertNull(cluster.getTotalShards());
|
||||
assertNull(cluster.getSuccessfulShards());
|
||||
assertNull(cluster.getSkippedShards());
|
||||
assertNull(cluster.getFailedShards());
|
||||
assertThat(cluster.getFailures().size(), equalTo(1));
|
||||
} else {
|
||||
assertThat(cluster.getTotalShards(), equalTo(numShards));
|
||||
assertThat(cluster.getSuccessfulShards(), equalTo(0));
|
||||
assertThat(cluster.getSkippedShards(), equalTo(0));
|
||||
assertThat(cluster.getFailedShards(), equalTo(numShards));
|
||||
assertThat(cluster.getFailures().size(), equalTo(numShards));
|
||||
}
|
||||
assertNull(cluster.getTook());
|
||||
assertFalse(cluster.isTimedOut());
|
||||
ShardSearchFailure shardSearchFailure = cluster.getFailures().get(0);
|
||||
assertTrue("should have 'index corrupted' in reason", shardSearchFailure.reason().contains("index corrupted"));
|
||||
}
|
||||
|
||||
protected AsyncSearchResponse submitAsyncSearch(SubmitAsyncSearchRequest request) throws ExecutionException, InterruptedException {
|
||||
return client(LOCAL_CLUSTER).execute(SubmitAsyncSearchAction.INSTANCE, request).get();
|
||||
}
|
||||
|
@ -1155,23 +1354,31 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
|
||||
private Map<String, Object> 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));
|
||||
int numShardsLocal = randomIntBetween(2, 12);
|
||||
Settings localSettings = indexSettings(numShardsLocal, randomIntBetween(0, 1)).build();
|
||||
assertAcked(
|
||||
client(LOCAL_CLUSTER).admin()
|
||||
.indices()
|
||||
.prepareCreate(localIndex)
|
||||
.setSettings(localSettings)
|
||||
.setMapping("@timestamp", "type=date", "f", "type=text")
|
||||
);
|
||||
indexDocs(client(LOCAL_CLUSTER), localIndex);
|
||||
|
||||
String remoteIndex = "prod";
|
||||
int numShardsRemote = randomIntBetween(3, 6);
|
||||
int numShardsRemote = randomIntBetween(2, 12);
|
||||
final InternalTestCluster remoteCluster = cluster(REMOTE_CLUSTER);
|
||||
remoteCluster.ensureAtLeastNumDataNodes(randomIntBetween(1, 3));
|
||||
final Settings.Builder remoteSettings = Settings.builder();
|
||||
remoteSettings.put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, numShardsRemote);
|
||||
remoteSettings.put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, randomIntBetween(0, 1));
|
||||
|
||||
assertAcked(
|
||||
client(REMOTE_CLUSTER).admin()
|
||||
.indices()
|
||||
.prepareCreate(remoteIndex)
|
||||
.setSettings(Settings.builder().put(remoteSettings.build()).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0))
|
||||
.setSettings(Settings.builder().put(remoteSettings.build()))
|
||||
.setMapping("@timestamp", "type=date", "f", "type=text")
|
||||
);
|
||||
assertFalse(
|
||||
client(REMOTE_CLUSTER).admin()
|
||||
|
@ -1200,9 +1407,13 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
}
|
||||
|
||||
private int indexDocs(Client client, String index) {
|
||||
int numDocs = between(1, 10);
|
||||
int numDocs = between(500, 1200);
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
client.prepareIndex(index).setSource("f", "v").get();
|
||||
long ts = EARLIEST_TIMESTAMP + i;
|
||||
if (i == numDocs - 1) {
|
||||
ts = LATEST_TIMESTAMP;
|
||||
}
|
||||
client.prepareIndex(index).setSource("f", "v", "@timestamp", ts).get();
|
||||
}
|
||||
client.admin().indices().prepareRefresh(index).get();
|
||||
return numDocs;
|
||||
|
|
|
@ -13,6 +13,7 @@ import org.elasticsearch.ExceptionsHelper;
|
|||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest;
|
||||
import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksResponse;
|
||||
import org.elasticsearch.action.search.CCSSingleCoordinatorSearchProgressListener;
|
||||
import org.elasticsearch.action.search.SearchProgressActionListener;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
|
@ -20,11 +21,13 @@ import org.elasticsearch.action.search.SearchResponse.Clusters;
|
|||
import org.elasticsearch.action.search.SearchShard;
|
||||
import org.elasticsearch.action.search.SearchTask;
|
||||
import org.elasticsearch.action.search.ShardSearchFailure;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.aggregations.AggregationReduceContext;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||
import org.elasticsearch.search.query.QuerySearchResult;
|
||||
import org.elasticsearch.tasks.TaskId;
|
||||
import org.elasticsearch.tasks.TaskManager;
|
||||
import org.elasticsearch.threadpool.Scheduler.Cancellable;
|
||||
|
@ -367,20 +370,32 @@ final class AsyncSearchTask extends SearchTask implements AsyncTask {
|
|||
|
||||
class Listener extends SearchProgressActionListener {
|
||||
|
||||
// needed when there's a single coordinator for all CCS search phases (minimize_roundtrips=false)
|
||||
private CCSSingleCoordinatorSearchProgressListener delegate;
|
||||
|
||||
@Override
|
||||
protected void onQueryResult(int shardIndex) {
|
||||
protected void onQueryResult(int shardIndex, QuerySearchResult queryResult) {
|
||||
checkCancellation();
|
||||
if (delegate != null) {
|
||||
delegate.onQueryResult(shardIndex, queryResult);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void onFetchResult(int shardIndex) {
|
||||
checkCancellation();
|
||||
if (delegate != null) {
|
||||
delegate.onFetchResult(shardIndex);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void onQueryFailure(int shardIndex, SearchShardTarget shardTarget, Exception exc) {
|
||||
// best effort to cancel expired tasks
|
||||
checkCancellation();
|
||||
if (delegate != null) {
|
||||
delegate.onQueryFailure(shardIndex, shardTarget, exc);
|
||||
}
|
||||
searchResponse.get()
|
||||
.addQueryFailure(
|
||||
shardIndex,
|
||||
|
@ -401,11 +416,27 @@ final class AsyncSearchTask extends SearchTask implements AsyncTask {
|
|||
// in which case the final response already includes results as well as shard fetch failures)
|
||||
}
|
||||
|
||||
/**
|
||||
* onListShards is guaranteed to be the first SearchProgressListener method called and
|
||||
* the search will not progress until this returns, so this is a safe place to initialize state
|
||||
* that is needed for handling subsequent callbacks.
|
||||
*/
|
||||
@Override
|
||||
protected void onListShards(List<SearchShard> shards, List<SearchShard> skipped, Clusters clusters, boolean fetchPhase) {
|
||||
protected void onListShards(
|
||||
List<SearchShard> shards,
|
||||
List<SearchShard> skipped,
|
||||
Clusters clusters,
|
||||
boolean fetchPhase,
|
||||
TransportSearchAction.SearchTimeProvider timeProvider
|
||||
) {
|
||||
// best effort to cancel expired tasks
|
||||
checkCancellation();
|
||||
assert clusters.isCcsMinimizeRoundtrips() != null : "CCS minimize_roundtrips value must be set in this context";
|
||||
ccsMinimizeRoundtrips = clusters.isCcsMinimizeRoundtrips();
|
||||
if (ccsMinimizeRoundtrips == false && clusters.hasClusterObjects()) {
|
||||
delegate = new CCSSingleCoordinatorSearchProgressListener();
|
||||
delegate.onListShards(shards, skipped, clusters, fetchPhase, timeProvider);
|
||||
}
|
||||
searchResponse.compareAndSet(
|
||||
null,
|
||||
new MutableSearchResponse(shards.size() + skipped.size(), skipped.size(), clusters, threadPool.getThreadContext())
|
||||
|
@ -417,6 +448,9 @@ final class AsyncSearchTask extends SearchTask implements AsyncTask {
|
|||
public void onPartialReduce(List<SearchShard> shards, TotalHits totalHits, InternalAggregations aggregations, int reducePhase) {
|
||||
// best effort to cancel expired tasks
|
||||
checkCancellation();
|
||||
if (delegate != null) {
|
||||
delegate.onPartialReduce(shards, totalHits, aggregations, reducePhase);
|
||||
}
|
||||
// The way that the MutableSearchResponse will build the aggs.
|
||||
Supplier<InternalAggregations> reducedAggs;
|
||||
if (aggregations == null) {
|
||||
|
@ -444,6 +478,9 @@ final class AsyncSearchTask extends SearchTask implements AsyncTask {
|
|||
public void onFinalReduce(List<SearchShard> shards, TotalHits totalHits, InternalAggregations aggregations, int reducePhase) {
|
||||
// best effort to cancel expired tasks
|
||||
checkCancellation();
|
||||
if (delegate != null) {
|
||||
delegate.onFinalReduce(shards, totalHits, aggregations, reducePhase);
|
||||
}
|
||||
searchResponse.get().updatePartialResponse(shards.size(), totalHits, () -> aggregations, reducePhase);
|
||||
}
|
||||
|
||||
|
|
|
@ -465,6 +465,7 @@ public class AsyncSearchResponseTests extends ESTestCase {
|
|||
SearchResponse.Cluster updated = new SearchResponse.Cluster(
|
||||
localCluster.getClusterAlias(),
|
||||
localCluster.getIndexExpression(),
|
||||
false,
|
||||
SearchResponse.Cluster.Status.SUCCESSFUL,
|
||||
10,
|
||||
10,
|
||||
|
@ -482,6 +483,7 @@ public class AsyncSearchResponseTests extends ESTestCase {
|
|||
updated = new SearchResponse.Cluster(
|
||||
cluster0.getClusterAlias(),
|
||||
cluster0.getIndexExpression(),
|
||||
false,
|
||||
SearchResponse.Cluster.Status.SUCCESSFUL,
|
||||
8,
|
||||
8,
|
||||
|
@ -507,6 +509,7 @@ public class AsyncSearchResponseTests extends ESTestCase {
|
|||
updated = new SearchResponse.Cluster(
|
||||
cluster1.getClusterAlias(),
|
||||
cluster1.getIndexExpression(),
|
||||
false,
|
||||
SearchResponse.Cluster.Status.SKIPPED,
|
||||
2,
|
||||
0,
|
||||
|
@ -524,6 +527,7 @@ public class AsyncSearchResponseTests extends ESTestCase {
|
|||
updated = new SearchResponse.Cluster(
|
||||
cluster2.getClusterAlias(),
|
||||
cluster2.getIndexExpression(),
|
||||
false,
|
||||
SearchResponse.Cluster.Status.PARTIAL,
|
||||
8,
|
||||
8,
|
||||
|
@ -764,7 +768,7 @@ public class AsyncSearchResponseTests extends ESTestCase {
|
|||
remoteClusterIndices.put("cluster_" + i, new OriginalIndices(new String[] { "foo", "bar*" }, IndicesOptions.lenientExpand()));
|
||||
}
|
||||
|
||||
return new SearchResponse.Clusters(localIndices, remoteClusterIndices, ccsMinimizeRoundtrips);
|
||||
return new SearchResponse.Clusters(localIndices, remoteClusterIndices, ccsMinimizeRoundtrips, alias -> false);
|
||||
}
|
||||
|
||||
static SearchResponse.Clusters createCCSClusterObjects(
|
||||
|
@ -794,6 +798,7 @@ public class AsyncSearchResponseTests extends ESTestCase {
|
|||
updated = new SearchResponse.Cluster(
|
||||
localAlias,
|
||||
localRef.get().getIndexExpression(),
|
||||
false,
|
||||
SearchResponse.Cluster.Status.SUCCESSFUL,
|
||||
5,
|
||||
5,
|
||||
|
@ -808,6 +813,7 @@ public class AsyncSearchResponseTests extends ESTestCase {
|
|||
updated = new SearchResponse.Cluster(
|
||||
localAlias,
|
||||
localRef.get().getIndexExpression(),
|
||||
false,
|
||||
SearchResponse.Cluster.Status.SKIPPED,
|
||||
5,
|
||||
0,
|
||||
|
@ -822,6 +828,7 @@ public class AsyncSearchResponseTests extends ESTestCase {
|
|||
updated = new SearchResponse.Cluster(
|
||||
localAlias,
|
||||
localRef.get().getIndexExpression(),
|
||||
false,
|
||||
SearchResponse.Cluster.Status.PARTIAL,
|
||||
5,
|
||||
2,
|
||||
|
@ -848,6 +855,7 @@ public class AsyncSearchResponseTests extends ESTestCase {
|
|||
updated = new SearchResponse.Cluster(
|
||||
clusterAlias,
|
||||
clusterRef.get().getIndexExpression(),
|
||||
false,
|
||||
SearchResponse.Cluster.Status.SUCCESSFUL,
|
||||
5,
|
||||
5,
|
||||
|
@ -862,6 +870,7 @@ public class AsyncSearchResponseTests extends ESTestCase {
|
|||
updated = new SearchResponse.Cluster(
|
||||
clusterAlias,
|
||||
clusterRef.get().getIndexExpression(),
|
||||
false,
|
||||
SearchResponse.Cluster.Status.SKIPPED,
|
||||
5,
|
||||
0,
|
||||
|
@ -876,6 +885,7 @@ public class AsyncSearchResponseTests extends ESTestCase {
|
|||
updated = new SearchResponse.Cluster(
|
||||
clusterAlias,
|
||||
clusterRef.get().getIndexExpression(),
|
||||
false,
|
||||
SearchResponse.Cluster.Status.PARTIAL,
|
||||
5,
|
||||
2,
|
||||
|
|
|
@ -14,6 +14,7 @@ import org.elasticsearch.action.search.SearchRequest;
|
|||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.SearchShard;
|
||||
import org.elasticsearch.action.search.ShardSearchFailure;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.common.breaker.CircuitBreaker;
|
||||
import org.elasticsearch.common.breaker.CircuitBreakingException;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
|
@ -71,6 +72,10 @@ public class AsyncSearchTaskTests extends ESTestCase {
|
|||
threadPool.shutdownNow();
|
||||
}
|
||||
|
||||
static TransportSearchAction.SearchTimeProvider createTimeProvider() {
|
||||
return new TransportSearchAction.SearchTimeProvider(System.currentTimeMillis(), System.nanoTime(), System::nanoTime);
|
||||
}
|
||||
|
||||
private AsyncSearchTask createAsyncSearchTask() {
|
||||
return new AsyncSearchTask(
|
||||
0L,
|
||||
|
@ -158,7 +163,8 @@ public class AsyncSearchTaskTests extends ESTestCase {
|
|||
thread.start();
|
||||
}
|
||||
assertFalse(latch.await(numThreads * 2, TimeUnit.MILLISECONDS));
|
||||
task.getSearchProgressActionListener().onListShards(shards, skippedShards, SearchResponse.Clusters.EMPTY, false);
|
||||
task.getSearchProgressActionListener()
|
||||
.onListShards(shards, skippedShards, SearchResponse.Clusters.EMPTY, false, createTimeProvider());
|
||||
latch.await();
|
||||
}
|
||||
|
||||
|
@ -191,7 +197,7 @@ public class AsyncSearchTaskTests extends ESTestCase {
|
|||
public void testWithFailureAndGetResponseFailureDuringReduction() throws InterruptedException {
|
||||
AsyncSearchTask task = createAsyncSearchTask();
|
||||
task.getSearchProgressActionListener()
|
||||
.onListShards(Collections.emptyList(), Collections.emptyList(), SearchResponse.Clusters.EMPTY, false);
|
||||
.onListShards(Collections.emptyList(), Collections.emptyList(), SearchResponse.Clusters.EMPTY, false, createTimeProvider());
|
||||
InternalAggregations aggs = InternalAggregations.from(
|
||||
Collections.singletonList(
|
||||
new StringTerms(
|
||||
|
@ -256,7 +262,8 @@ public class AsyncSearchTaskTests extends ESTestCase {
|
|||
skippedShards.add(new SearchShard(null, new ShardId("0", "0", 1)));
|
||||
}
|
||||
int totalShards = numShards + numSkippedShards;
|
||||
task.getSearchProgressActionListener().onListShards(shards, skippedShards, SearchResponse.Clusters.EMPTY, false);
|
||||
task.getSearchProgressActionListener()
|
||||
.onListShards(shards, skippedShards, SearchResponse.Clusters.EMPTY, false, createTimeProvider());
|
||||
for (int i = 0; i < numShards; i++) {
|
||||
task.getSearchProgressActionListener()
|
||||
.onPartialReduce(shards.subList(i, i + 1), new TotalHits(0, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO), null, 0);
|
||||
|
@ -282,7 +289,8 @@ public class AsyncSearchTaskTests extends ESTestCase {
|
|||
skippedShards.add(new SearchShard(null, new ShardId("0", "0", 1)));
|
||||
}
|
||||
int totalShards = numShards + numSkippedShards;
|
||||
task.getSearchProgressActionListener().onListShards(shards, skippedShards, SearchResponse.Clusters.EMPTY, false);
|
||||
task.getSearchProgressActionListener()
|
||||
.onListShards(shards, skippedShards, SearchResponse.Clusters.EMPTY, false, createTimeProvider());
|
||||
for (int i = 0; i < numShards; i++) {
|
||||
task.getSearchProgressActionListener()
|
||||
.onPartialReduce(shards.subList(i, i + 1), new TotalHits(0, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO), null, 0);
|
||||
|
@ -318,7 +326,8 @@ public class AsyncSearchTaskTests extends ESTestCase {
|
|||
skippedShards.add(new SearchShard(null, new ShardId("0", "0", 1)));
|
||||
}
|
||||
int totalShards = numShards + numSkippedShards;
|
||||
task.getSearchProgressActionListener().onListShards(shards, skippedShards, SearchResponse.Clusters.EMPTY, false);
|
||||
task.getSearchProgressActionListener()
|
||||
.onListShards(shards, skippedShards, SearchResponse.Clusters.EMPTY, false, createTimeProvider());
|
||||
for (int i = 0; i < numShards; i++) {
|
||||
task.getSearchProgressActionListener()
|
||||
.onPartialReduce(shards.subList(0, i + 1), new TotalHits(0, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO), null, 0);
|
||||
|
@ -350,7 +359,8 @@ public class AsyncSearchTaskTests extends ESTestCase {
|
|||
skippedShards.add(new SearchShard(null, new ShardId("0", "0", 1)));
|
||||
}
|
||||
int totalShards = numShards + numSkippedShards;
|
||||
task.getSearchProgressActionListener().onListShards(shards, skippedShards, SearchResponse.Clusters.EMPTY, false);
|
||||
task.getSearchProgressActionListener()
|
||||
.onListShards(shards, skippedShards, SearchResponse.Clusters.EMPTY, false, createTimeProvider());
|
||||
|
||||
listener.onFailure(new SearchPhaseExecutionException("fetch", "boum", ShardSearchFailure.EMPTY_ARRAY));
|
||||
assertCompletionListeners(task, totalShards, 0, numSkippedShards, 0, true, true);
|
||||
|
@ -376,7 +386,7 @@ public class AsyncSearchTaskTests extends ESTestCase {
|
|||
}
|
||||
}, TimeValue.timeValueMillis(500L));
|
||||
asyncSearchTask.getSearchProgressActionListener()
|
||||
.onListShards(Collections.emptyList(), Collections.emptyList(), SearchResponse.Clusters.EMPTY, false);
|
||||
.onListShards(Collections.emptyList(), Collections.emptyList(), SearchResponse.Clusters.EMPTY, false, createTimeProvider());
|
||||
assertTrue(latch.await(1000, TimeUnit.SECONDS));
|
||||
assertThat(failure.get(), instanceOf(RuntimeException.class));
|
||||
}
|
||||
|
@ -385,7 +395,7 @@ public class AsyncSearchTaskTests extends ESTestCase {
|
|||
throwOnSchedule = true;
|
||||
AsyncSearchTask asyncSearchTask = createAsyncSearchTask();
|
||||
asyncSearchTask.getSearchProgressActionListener()
|
||||
.onListShards(Collections.emptyList(), Collections.emptyList(), SearchResponse.Clusters.EMPTY, false);
|
||||
.onListShards(Collections.emptyList(), Collections.emptyList(), SearchResponse.Clusters.EMPTY, false, createTimeProvider());
|
||||
CountDownLatch latch = new CountDownLatch(1);
|
||||
AtomicReference<Exception> failure = new AtomicReference<>();
|
||||
// onListShards has already been executed, then addCompletionListener is executed immediately
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue